cmd/k8s-operator,k8s-operator: allow the operator to deploy exit nodes via Connector custom resource (#10724)

cmd/k8s-operator/deploy/crds,k8s-operator/apis/v1alpha1: allow to define an exit node via Connector CR.

Make it possible to define an exit node to be deployed to a Kubernetes cluster
via Connector Custom resource.

Also changes to Connector API so that one Connector corresponds
to one Tailnet node that can be either a subnet router or an exit
node or both.

The Kubernetes operator parses Connector custom resource and,
if .spec.isExitNode is set, configures that Tailscale node deployed
for that connector as an exit node.

Signed-off-by: Irbe Krumina <irbe@tailscale.com>
Co-authored-by: Anton Tolchanov <anton@tailscale.com>
This commit is contained in:
Irbe Krumina 2024-01-09 14:13:22 +00:00 committed by GitHub
parent 953fa80c6f
commit 05093ea7d9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 1064 additions and 804 deletions

View File

@ -10,7 +10,6 @@ import (
"fmt"
"net/netip"
"slices"
"strings"
"sync"
"time"
@ -33,17 +32,15 @@ import (
)
const (
reasonSubnetRouterCreationFailed = "SubnetRouterCreationFailed"
reasonSubnetRouterCreated = "SubnetRouterCreated"
reasonSubnetRouterCleanupFailed = "SubnetRouterCleanupFailed"
reasonSubnetRouterCleanupInProgress = "SubnetRouterCleanupInProgress"
reasonSubnetRouterInvalid = "SubnetRouterInvalid"
reasonConnectorCreationFailed = "ConnectorCreationFailed"
messageSubnetRouterCreationFailed = "Failed creating subnet router for routes %s: %v"
messageSubnetRouterInvalid = "Subnet router is invalid: %v"
messageSubnetRouterCreated = "Created subnet router for routes %s"
messageSubnetRouterCleanupFailed = "Failed cleaning up subnet router resources: %v"
msgSubnetRouterCleanupInProgress = "SubnetRouterCleanupInProgress"
reasonConnectorCreated = "ConnectorCreated"
reasonConnectorCleanupFailed = "ConnectorCleanupFailed"
reasonConnectorCleanupInProgress = "ConnectorCleanupInProgress"
reasonConnectorInvalid = "ConnectorInvalid"
messageConnectorCreationFailed = "Failed creating Connector: %v"
messageConnectorInvalid = "Connector is invalid: %v"
shortRequeue = time.Second * 5
)
@ -61,42 +58,44 @@ type ConnectorReconciler struct {
mu sync.Mutex // protects following
// subnetRouters tracks the subnet routers managed by this Tailscale
// Operator instance.
subnetRouters set.Slice[types.UID]
subnetRouters set.Slice[types.UID] // for subnet routers gauge
exitNodes set.Slice[types.UID] // for exit nodes gauge
}
var (
// gaugeIngressResources tracks the number of subnet routers that we're
// currently managing.
gaugeSubnetRouterResources = clientmetric.NewGauge("k8s_subnet_router_resources")
// gaugeConnectorResources tracks the overall number of Connectors currently managed by this operator instance.
gaugeConnectorResources = clientmetric.NewGauge("k8s_connector_resources")
// gaugeConnectorSubnetRouterResources tracks the number of Connectors managed by this operator instance that are subnet routers.
gaugeConnectorSubnetRouterResources = clientmetric.NewGauge("k8s_connector_subnetrouter_resources")
// gaugeConnectorExitNodeResources tracks the number of Connectors currently managed by this operator instance that are exit nodes.
gaugeConnectorExitNodeResources = clientmetric.NewGauge("k8s_connector_exitnode_resources")
)
func (a *ConnectorReconciler) Reconcile(ctx context.Context, req reconcile.Request) (_ reconcile.Result, err error) {
logger := a.logger.With("connector", req.Name)
func (a *ConnectorReconciler) Reconcile(ctx context.Context, req reconcile.Request) (res reconcile.Result, err error) {
logger := a.logger.With("Connector", req.Name)
logger.Debugf("starting reconcile")
defer logger.Debugf("reconcile finished")
cn := new(tsapi.Connector)
err = a.Get(ctx, req.NamespacedName, cn)
if apierrors.IsNotFound(err) {
logger.Debugf("connector not found, assuming it was deleted")
logger.Debugf("Connector not found, assuming it was deleted")
return reconcile.Result{}, nil
} else if err != nil {
return reconcile.Result{}, fmt.Errorf("failed to get tailscale.com Connector: %w", err)
}
if !cn.DeletionTimestamp.IsZero() {
logger.Debugf("connector is being deleted or should not be exposed, cleaning up components")
logger.Debugf("Connector is being deleted or should not be exposed, cleaning up resources")
ix := xslices.Index(cn.Finalizers, FinalizerName)
if ix < 0 {
logger.Debugf("no finalizer, nothing to do")
return reconcile.Result{}, nil
}
if done, err := a.maybeCleanupSubnetRouter(ctx, logger, cn); err != nil {
if done, err := a.maybeCleanupConnector(ctx, logger, cn); err != nil {
return reconcile.Result{}, err
} else if !done {
logger.Debugf("cleanup not finished, will retry...")
logger.Debugf("Connector resource cleanup not yet finished, will retry...")
return reconcile.Result{RequeueAfter: shortRequeue}, nil
}
@ -104,93 +103,110 @@ func (a *ConnectorReconciler) Reconcile(ctx context.Context, req reconcile.Reque
if err := a.Update(ctx, cn); err != nil {
return reconcile.Result{}, err
}
logger.Infof("connector resources cleaned up")
logger.Infof("Connector resources cleaned up")
return reconcile.Result{}, nil
}
oldCnStatus := cn.Status.DeepCopy()
defer func() {
if cn.Status.SubnetRouter == nil {
tsoperator.SetConnectorCondition(cn, tsapi.ConnectorReady, metav1.ConditionUnknown, "", "", cn.Generation, a.clock, logger)
} else if cn.Status.SubnetRouter.Ready == metav1.ConditionTrue {
tsoperator.SetConnectorCondition(cn, tsapi.ConnectorReady, metav1.ConditionTrue, reasonSubnetRouterCreated, reasonSubnetRouterCreated, cn.Generation, a.clock, logger)
} else {
tsoperator.SetConnectorCondition(cn, tsapi.ConnectorReady, metav1.ConditionFalse, cn.Status.SubnetRouter.Reason, cn.Status.SubnetRouter.Reason, cn.Generation, a.clock, logger)
}
setStatus := func(cn *tsapi.Connector, conditionType tsapi.ConnectorConditionType, status metav1.ConditionStatus, reason, message string) (reconcile.Result, error) {
tsoperator.SetConnectorCondition(cn, tsapi.ConnectorReady, status, reason, message, cn.Generation, a.clock, logger)
if !apiequality.Semantic.DeepEqual(oldCnStatus, cn.Status) {
// an error encountered here should get returned by the Reconcile function
// An error encountered here should get returned by the Reconcile function.
if updateErr := a.Client.Status().Update(ctx, cn); updateErr != nil {
err = updateErr
err = errors.Wrap(err, updateErr.Error())
}
}
}()
return res, err
}
if !slices.Contains(cn.Finalizers, FinalizerName) {
// This log line is printed exactly once during initial provisioning,
// because once the finalizer is in place this block gets skipped. So,
// this is a nice place to tell the operator that the high level,
// multi-reconcile operation is underway.
logger.Infof("ensuring connector is set up")
logger.Infof("ensuring Connector is set up")
cn.Finalizers = append(cn.Finalizers, FinalizerName)
if err := a.Update(ctx, cn); err != nil {
err = fmt.Errorf("failed to add finalizer: %w", err)
logger.Errorf("error adding finalizer: %v", err)
return reconcile.Result{}, err
logger.Errorf("error adding finalizer: %w", err)
return setStatus(cn, tsapi.ConnectorReady, metav1.ConditionFalse, reasonConnectorCreationFailed, reasonConnectorCreationFailed)
}
}
// A Connector with unset .spec.subnetRouter and unset
// cn.spec.subnetRouter.Routes will be rejected at apply time (because
// these fields are set as required by our CRD validation). This check
// is here for if our CRD validation breaks unnoticed we don't crash the
// operator with nil pointer exception.
if cn.Spec.SubnetRouter == nil || len(cn.Spec.SubnetRouter.Routes) < 1 {
return reconcile.Result{}, nil
if err := a.validate(cn); err != nil {
logger.Errorf("error validating Connector spec: %w", err)
message := fmt.Sprintf(messageConnectorInvalid, err)
a.recorder.Eventf(cn, corev1.EventTypeWarning, reasonConnectorInvalid, message)
return setStatus(cn, tsapi.ConnectorReady, metav1.ConditionFalse, reasonConnectorInvalid, message)
}
if err := validateSubnetRouter(*cn.Spec.SubnetRouter); err != nil {
msg := fmt.Sprintf(messageSubnetRouterInvalid, err)
cn.Status.SubnetRouter = &tsapi.SubnetRouterStatus{
Ready: metav1.ConditionFalse,
Reason: reasonSubnetRouterInvalid,
Message: msg,
}
a.recorder.Eventf(cn, corev1.EventTypeWarning, reasonSubnetRouterInvalid, msg)
return reconcile.Result{}, nil
if err = a.maybeProvisionConnector(ctx, logger, cn); err != nil {
logger.Errorf("error creating Connector resources: %w", err)
message := fmt.Sprintf(messageConnectorCreationFailed, err)
a.recorder.Eventf(cn, corev1.EventTypeWarning, reasonConnectorCreationFailed, message)
return setStatus(cn, tsapi.ConnectorReady, metav1.ConditionFalse, reasonConnectorCreationFailed, message)
}
var sb strings.Builder
sb.WriteString(string(cn.Spec.SubnetRouter.Routes[0]))
for _, r := range cn.Spec.SubnetRouter.Routes[1:] {
sb.WriteString(fmt.Sprintf(",%s", r))
logger.Info("Connector resources synced")
cn.Status.IsExitNode = cn.Spec.ExitNode
if cn.Spec.SubnetRouter != nil {
cn.Status.SubnetRoutes = cn.Spec.SubnetRouter.AdvertiseRoutes.Stringify()
return setStatus(cn, tsapi.ConnectorReady, metav1.ConditionTrue, reasonConnectorCreated, reasonConnectorCreated)
}
cidrsS := sb.String()
logger.Debugf("ensuring a subnet router is deployed")
err = a.maybeProvisionSubnetRouter(ctx, logger, cn, cidrsS)
if err != nil {
msg := fmt.Sprintf(messageSubnetRouterCreationFailed, cidrsS, err)
cn.Status.SubnetRouter = &tsapi.SubnetRouterStatus{
Ready: metav1.ConditionFalse,
Reason: reasonSubnetRouterCreationFailed,
Message: msg,
}
a.recorder.Eventf(cn, corev1.EventTypeWarning, reasonSubnetRouterCreationFailed, msg)
return reconcile.Result{}, err
}
cn.Status.SubnetRouter = &tsapi.SubnetRouterStatus{
Routes: cidrsS,
Ready: metav1.ConditionTrue,
Reason: reasonSubnetRouterCreated,
Message: fmt.Sprintf(messageSubnetRouterCreated, cidrsS),
}
return reconcile.Result{}, nil
cn.Status.SubnetRoutes = ""
return setStatus(cn, tsapi.ConnectorReady, metav1.ConditionTrue, reasonConnectorCreated, reasonConnectorCreated)
}
func (a *ConnectorReconciler) maybeCleanupSubnetRouter(ctx context.Context, logger *zap.SugaredLogger, cn *tsapi.Connector) (bool, error) {
if done, err := a.ssr.Cleanup(ctx, logger, childResourceLabels(cn.Name, a.tsnamespace, "subnetrouter")); err != nil {
return false, fmt.Errorf("failed to cleanup: %w", err)
// maybeProvisionConnector ensures that any new resources required for this
// Connector instance are deployed to the cluster.
func (a *ConnectorReconciler) maybeProvisionConnector(ctx context.Context, logger *zap.SugaredLogger, cn *tsapi.Connector) error {
hostname := cn.Name + "-connector"
if cn.Spec.Hostname != "" {
hostname = string(cn.Spec.Hostname)
}
crl := childResourceLabels(cn.Name, a.tsnamespace, "connector")
sts := &tailscaleSTSConfig{
ParentResourceName: cn.Name,
ParentResourceUID: string(cn.UID),
Hostname: hostname,
ChildResourceLabels: crl,
Tags: cn.Spec.Tags.Stringify(),
Connector: &connector{
isExitNode: cn.Spec.ExitNode,
},
}
if cn.Spec.SubnetRouter != nil && len(cn.Spec.SubnetRouter.AdvertiseRoutes) > 0 {
sts.Connector.routes = cn.Spec.SubnetRouter.AdvertiseRoutes.Stringify()
}
a.mu.Lock()
if sts.Connector.isExitNode {
a.exitNodes.Add(cn.UID)
} else {
a.exitNodes.Remove(cn.UID)
}
if sts.Connector.routes != "" {
a.subnetRouters.Add(cn.GetUID())
} else {
a.subnetRouters.Remove(cn.GetUID())
}
a.mu.Unlock()
gaugeConnectorSubnetRouterResources.Set(int64(a.subnetRouters.Len()))
gaugeConnectorExitNodeResources.Set(int64(a.exitNodes.Len()))
var connectors set.Slice[types.UID]
connectors.AddSlice(a.exitNodes.Slice())
connectors.AddSlice(a.subnetRouters.Slice())
gaugeConnectorResources.Set(int64(connectors.Len()))
_, err := a.ssr.Provision(ctx, logger, sts)
return err
}
func (a *ConnectorReconciler) maybeCleanupConnector(ctx context.Context, logger *zap.SugaredLogger, cn *tsapi.Connector) (bool, error) {
if done, err := a.ssr.Cleanup(ctx, logger, childResourceLabels(cn.Name, a.tsnamespace, "connector")); err != nil {
return false, fmt.Errorf("failed to cleanup Connector resources: %w", err)
} else if !done {
logger.Debugf("cleanup not done yet, waiting for next reconcile")
logger.Debugf("Connector cleanup not done yet, waiting for next reconcile")
return false, nil
}
@ -198,44 +214,39 @@ func (a *ConnectorReconciler) maybeCleanupSubnetRouter(ctx context.Context, logg
// exactly once at the very end of cleanup, because the final step of
// cleanup removes the tailscale finalizer, which will make all future
// reconciles exit early.
logger.Infof("cleaned up subnet router")
logger.Infof("cleaned up Connector resources")
a.mu.Lock()
defer a.mu.Unlock()
a.subnetRouters.Remove(cn.UID)
gaugeSubnetRouterResources.Set(int64(a.subnetRouters.Len()))
a.exitNodes.Remove(cn.UID)
a.mu.Unlock()
gaugeConnectorExitNodeResources.Set(int64(a.exitNodes.Len()))
gaugeConnectorSubnetRouterResources.Set(int64(a.subnetRouters.Len()))
var connectors set.Slice[types.UID]
connectors.AddSlice(a.exitNodes.Slice())
connectors.AddSlice(a.subnetRouters.Slice())
gaugeConnectorResources.Set(int64(connectors.Len()))
return true, nil
}
// maybeProvisionSubnetRouter maybe deploys subnet router that exposes a subset of cluster cidrs to the tailnet
func (a *ConnectorReconciler) maybeProvisionSubnetRouter(ctx context.Context, logger *zap.SugaredLogger, cn *tsapi.Connector, cidrs string) error {
if cn.Spec.SubnetRouter == nil || len(cn.Spec.SubnetRouter.Routes) < 1 {
func (a *ConnectorReconciler) validate(cn *tsapi.Connector) error {
// Connector fields are already validated at apply time with CEL validation
// on custom resource fields. The checks here are a backup in case the
// CEL validation breaks without us noticing.
if !(cn.Spec.SubnetRouter != nil || cn.Spec.ExitNode) {
return errors.New("invalid spec: a Connector must expose subnet routes or act as an exit node (or both)")
}
if cn.Spec.SubnetRouter == nil {
return nil
}
a.mu.Lock()
a.subnetRouters.Add(cn.UID)
gaugeSubnetRouterResources.Set(int64(a.subnetRouters.Len()))
a.mu.Unlock()
crl := childResourceLabels(cn.Name, a.tsnamespace, "subnetrouter")
hostname := hostnameForSubnetRouter(cn)
sts := &tailscaleSTSConfig{
ParentResourceName: cn.Name,
ParentResourceUID: string(cn.UID),
Hostname: hostname,
ChildResourceLabels: crl,
Routes: cidrs,
}
for _, tag := range cn.Spec.SubnetRouter.Tags {
sts.Tags = append(sts.Tags, string(tag))
}
_, err := a.ssr.Provision(ctx, logger, sts)
return err
return validateSubnetRouter(cn.Spec.SubnetRouter)
}
func validateSubnetRouter(sb tsapi.SubnetRouter) error {
func validateSubnetRouter(sb *tsapi.SubnetRouter) error {
if len(sb.AdvertiseRoutes) < 1 {
return errors.New("invalid subnet router spec: no routes defined")
}
var err error
for _, route := range sb.Routes {
for _, route := range sb.AdvertiseRoutes {
pfx, e := netip.ParsePrefix(string(route))
if e != nil {
err = errors.Wrap(err, fmt.Sprintf("route %s is invalid: %v", route, err))
@ -247,13 +258,3 @@ func validateSubnetRouter(sb tsapi.SubnetRouter) error {
}
return err
}
func hostnameForSubnetRouter(cn *tsapi.Connector) string {
if cn.Spec.SubnetRouter == nil {
return ""
}
if cn.Spec.SubnetRouter.Hostname != "" {
return string(cn.Spec.SubnetRouter.Hostname)
}
return cn.Name + "-" + "subnetrouter"
}

View File

@ -17,10 +17,11 @@ import (
"sigs.k8s.io/controller-runtime/pkg/client/fake"
tsapi "tailscale.com/k8s-operator/apis/v1alpha1"
"tailscale.com/tstest"
"tailscale.com/types/ptr"
)
func TestConnector(t *testing.T) {
// Create a Connector that defines a Tailscale node that advertises
// 10.40.0.0/14 route and acts as an exit node.
cn := &tsapi.Connector{
ObjectMeta: metav1.ObjectMeta{
Name: "test",
@ -32,8 +33,9 @@ func TestConnector(t *testing.T) {
},
Spec: tsapi.ConnectorSpec{
SubnetRouter: &tsapi.SubnetRouter{
Routes: []tsapi.Route{"10.40.0.0/14"},
AdvertiseRoutes: []tsapi.Route{"10.40.0.0/14"},
},
ExitNode: true,
},
}
fc := fake.NewClientBuilder().
@ -48,7 +50,6 @@ func TestConnector(t *testing.T) {
}
cl := tstest.NewClock(tstest.ClockOpts{})
// Create a Connector with a subnet router definition
cr := &ConnectorReconciler{
Client: fc,
ssr: &tailscaleSTSReconciler{
@ -63,26 +64,61 @@ func TestConnector(t *testing.T) {
}
expectReconciled(t, cr, "", "test")
fullName, shortName := findGenName(t, fc, "", "test", "subnetrouter")
fullName, shortName := findGenName(t, fc, "", "test", "connector")
expectEqual(t, fc, expectedSecret(fullName, "", "subnetrouter"))
expectEqual(t, fc, expectedConnectorSTS(shortName, fullName, "10.40.0.0/14"))
opts := configOpts{
stsName: shortName,
secretName: fullName,
parentType: "connector",
hostname: "test-connector",
shouldUseDeclarativeConfig: true,
isExitNode: true,
subnetRoutes: "10.40.0.0/14",
confFileHash: "9321660203effb80983eaecc7b5ac5a8c53934926f46e895b9fe295dcfc5a904",
}
expectEqual(t, fc, expectedSecret(t, opts))
expectEqual(t, fc, expectedSTS(opts))
// Add another CIDR
// Add another route to be advertised.
mustUpdate[tsapi.Connector](t, fc, "", "test", func(conn *tsapi.Connector) {
conn.Spec.SubnetRouter.Routes = []tsapi.Route{"10.40.0.0/14", "10.44.0.0/20"}
conn.Spec.SubnetRouter.AdvertiseRoutes = []tsapi.Route{"10.40.0.0/14", "10.44.0.0/20"}
})
opts.subnetRoutes = "10.40.0.0/14,10.44.0.0/20"
opts.confFileHash = "fb6c4daf67425f983985750cd8d6f2beae77e614fcb34176604571f5623d6862"
expectReconciled(t, cr, "", "test")
expectEqual(t, fc, expectedConnectorSTS(shortName, fullName, "10.40.0.0/14,10.44.0.0/20"))
// Remove a CIDR
expectEqual(t, fc, expectedSTS(opts))
// Remove a route.
mustUpdate[tsapi.Connector](t, fc, "", "test", func(conn *tsapi.Connector) {
conn.Spec.SubnetRouter.Routes = []tsapi.Route{"10.44.0.0/20"}
conn.Spec.SubnetRouter.AdvertiseRoutes = []tsapi.Route{"10.44.0.0/20"}
})
opts.subnetRoutes = "10.44.0.0/20"
opts.confFileHash = "bacba177bcfe3849065cf6fee53d658a9bb4144197ac5b861727d69ea99742bb"
expectReconciled(t, cr, "", "test")
expectEqual(t, fc, expectedConnectorSTS(shortName, fullName, "10.44.0.0/20"))
expectEqual(t, fc, expectedSTS(opts))
// Delete the Connector
// Remove the subnet router.
mustUpdate[tsapi.Connector](t, fc, "", "test", func(conn *tsapi.Connector) {
conn.Spec.SubnetRouter = nil
})
opts.subnetRoutes = ""
opts.confFileHash = "7c421a99128eb80e79a285a82702f19f8f720615542a15bd794858a6275d8079"
expectReconciled(t, cr, "", "test")
expectEqual(t, fc, expectedSTS(opts))
// Re-add the subnet router.
mustUpdate[tsapi.Connector](t, fc, "", "test", func(conn *tsapi.Connector) {
conn.Spec.SubnetRouter = &tsapi.SubnetRouter{
AdvertiseRoutes: []tsapi.Route{"10.44.0.0/20"},
}
})
opts.subnetRoutes = "10.44.0.0/20"
opts.confFileHash = "bacba177bcfe3849065cf6fee53d658a9bb4144197ac5b861727d69ea99742bb"
expectReconciled(t, cr, "", "test")
expectEqual(t, fc, expectedSTS(opts))
// Delete the Connector.
if err = fc.Delete(context.Background(), cn); err != nil {
t.Fatalf("error deleting Connector: %v", err)
}
@ -93,72 +129,57 @@ func TestConnector(t *testing.T) {
expectMissing[appsv1.StatefulSet](t, fc, "operator-ns", shortName)
expectMissing[corev1.Secret](t, fc, "operator-ns", fullName)
}
func expectedConnectorSTS(stsName, secretName, routes string) *appsv1.StatefulSet {
return &appsv1.StatefulSet{
TypeMeta: metav1.TypeMeta{
Kind: "StatefulSet",
APIVersion: "apps/v1",
},
// Create a Connector that advertises a route and is not an exit node.
cn = &tsapi.Connector{
ObjectMeta: metav1.ObjectMeta{
Name: stsName,
Namespace: "operator-ns",
Labels: map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-ns": "",
"tailscale.com/parent-resource-type": "subnetrouter",
},
Name: "test",
UID: types.UID("1234-UID"),
},
Spec: appsv1.StatefulSetSpec{
Replicas: ptr.To[int32](1),
Selector: &metav1.LabelSelector{
MatchLabels: map[string]string{"app": "1234-UID"},
},
ServiceName: stsName,
Template: corev1.PodTemplateSpec{
ObjectMeta: metav1.ObjectMeta{
DeletionGracePeriodSeconds: ptr.To[int64](10),
Labels: map[string]string{"app": "1234-UID"},
Annotations: map[string]string{
"tailscale.com/operator-last-set-hostname": "test-subnetrouter",
},
},
Spec: corev1.PodSpec{
ServiceAccountName: "proxies",
InitContainers: []corev1.Container{
{
Name: "sysctler",
Image: "tailscale/tailscale",
Command: []string{"/bin/sh"},
Args: []string{"-c", "sysctl -w net.ipv4.ip_forward=1 net.ipv6.conf.all.forwarding=1"},
SecurityContext: &corev1.SecurityContext{
Privileged: ptr.To(true),
},
},
},
Containers: []corev1.Container{
{
Name: "tailscale",
Image: "tailscale/tailscale",
Env: []corev1.EnvVar{
{Name: "TS_USERSPACE", Value: "false"},
{Name: "TS_AUTH_ONCE", Value: "true"},
{Name: "TS_KUBE_SECRET", Value: secretName},
{Name: "TS_HOSTNAME", Value: "test-subnetrouter"},
{Name: "TS_ROUTES", Value: routes},
},
SecurityContext: &corev1.SecurityContext{
Capabilities: &corev1.Capabilities{
Add: []corev1.Capability{"NET_ADMIN"},
},
},
ImagePullPolicy: "Always",
},
},
},
TypeMeta: metav1.TypeMeta{
Kind: tsapi.ConnectorKind,
APIVersion: "tailscale.io/v1alpha1",
},
Spec: tsapi.ConnectorSpec{
SubnetRouter: &tsapi.SubnetRouter{
AdvertiseRoutes: []tsapi.Route{"10.40.0.0/14"},
},
},
}
opts.subnetRoutes = "10.44.0.0/14"
opts.isExitNode = false
mustCreate(t, fc, cn)
expectReconciled(t, cr, "", "test")
fullName, shortName = findGenName(t, fc, "", "test", "connector")
opts = configOpts{
stsName: shortName,
secretName: fullName,
parentType: "connector",
shouldUseDeclarativeConfig: true,
subnetRoutes: "10.40.0.0/14",
hostname: "test-connector",
confFileHash: "57d922331890c9b1c8c6ae664394cb254334c551d9cd9db14537b5d9da9fb17e",
}
expectEqual(t, fc, expectedSecret(t, opts))
expectEqual(t, fc, expectedSTS(opts))
// Add an exit node.
mustUpdate[tsapi.Connector](t, fc, "", "test", func(conn *tsapi.Connector) {
conn.Spec.ExitNode = true
})
opts.isExitNode = true
opts.confFileHash = "1499b591fd97a50f0330db6ec09979792c49890cf31f5da5bb6a3f50dba1e77a"
expectReconciled(t, cr, "", "test")
expectEqual(t, fc, expectedSTS(opts))
// Delete the Connector.
if err = fc.Delete(context.Background(), cn); err != nil {
t.Fatalf("error deleting Connector: %v", err)
}
expectRequeue(t, cr, "", "test")
expectReconciled(t, cr, "", "test")
expectMissing[appsv1.StatefulSet](t, fc, "operator-ns", shortName)
expectMissing[corev1.Secret](t, fc, "operator-ns", fullName)
}

View File

@ -16,11 +16,15 @@ spec:
scope: Cluster
versions:
- additionalPrinterColumns:
- description: Cluster CIDR ranges exposed to tailnet via subnet router
jsonPath: .status.subnetRouter.routes
- description: CIDR ranges exposed to tailnet by a subnet router defined via this Connector instance.
jsonPath: .status.subnetRoutes
name: SubnetRoutes
type: string
- description: Status of the components deployed by the connector
- description: Whether this Connector instance defines an exit node.
jsonPath: .status.isExitNode
name: IsExitNode
type: string
- description: Status of the deployed Connector resources.
jsonPath: .status.conditions[?(@.type == "ConnectorReady")].reason
name: Status
type: string
@ -40,38 +44,40 @@ spec:
metadata:
type: object
spec:
description: Desired state of the Connector resource.
description: ConnectorSpec describes the desired Tailscale component.
type: object
required:
- subnetRouter
properties:
exitNode:
description: ExitNode defines whether the Connector node should act as a Tailscale exit node. Defaults to false. https://tailscale.com/kb/1103/exit-nodes
type: boolean
hostname:
description: Hostname is the tailnet hostname that should be assigned to the Connector node. If unset, hostname defaults to <connector name>-connector. Hostname can contain lower case letters, numbers and dashes, it must not start or end with a dash and must be between 2 and 63 characters long.
type: string
pattern: ^[a-z0-9][a-z0-9-]{0,61}[a-z0-9]$
subnetRouter:
description: SubnetRouter configures a Tailscale subnet router to be deployed in the cluster. If unset no subnet router will be deployed. https://tailscale.com/kb/1019/subnets/
description: SubnetRouter defines subnet routes that the Connector node should expose to tailnet. If unset, none are exposed. https://tailscale.com/kb/1019/subnets/
type: object
required:
- routes
- advertiseRoutes
properties:
hostname:
description: Hostname is the tailnet hostname that should be assigned to the subnet router node. If unset hostname is defaulted to <connector name>-subnetrouter. Hostname can contain lower case letters, numbers and dashes, it must not start or end with a dash and must be between 2 and 63 characters long.
type: string
pattern: ^[a-z0-9][a-z0-9-]{0,61}[a-z0-9]$
routes:
description: Routes refer to in-cluster CIDRs that the subnet router should make available. Route values must be strings that represent a valid IPv4 or IPv6 CIDR range. Values can be Tailscale 4via6 subnet routes. https://tailscale.com/kb/1201/4via6-subnets/
advertiseRoutes:
description: AdvertiseRoutes refer to CIDRs that the subnet router should make available. Route values must be strings that represent a valid IPv4 or IPv6 CIDR range. Values can be Tailscale 4via6 subnet routes. https://tailscale.com/kb/1201/4via6-subnets/
type: array
minItems: 1
items:
type: string
format: cidr
tags:
description: Tags that the Tailscale node will be tagged with. If you want the subnet router to be autoapproved, you can configure Tailscale ACLs to autoapprove the subnetrouter's CIDRs for these tags. See https://tailscale.com/kb/1018/acls/#auto-approvers-for-routes-and-exit-nodes Defaults to tag:k8s. If you specify custom tags here, you must also make tag:k8s-operator owner of the custom tag. See https://tailscale.com/kb/1236/kubernetes-operator/#setting-up-the-kubernetes-operator. Tags cannot be changed once a Connector has been created. Tag values must be in form ^tag:[a-zA-Z][a-zA-Z0-9-]*$.
type: array
items:
type: string
pattern: ^tag:[a-zA-Z][a-zA-Z0-9-]*$
x-kubernetes-validations:
- rule: has(self.tags) == has(oldSelf.tags)
message: Subnetrouter tags cannot be changed. Delete and redeploy the Connector if you need to change it.
tags:
description: Tags that the Tailscale node will be tagged with. Defaults to [tag:k8s]. To autoapprove the subnet routes or exit node defined by a Connector, you can configure Tailscale ACLs to give these tags the necessary permissions. See https://tailscale.com/kb/1018/acls/#auto-approvers-for-routes-and-exit-nodes. If you specify custom tags here, you must also make the operator an owner of these tags. See https://tailscale.com/kb/1236/kubernetes-operator/#setting-up-the-kubernetes-operator. Tags cannot be changed once a Connector node has been created. Tag values must be in form ^tag:[a-zA-Z][a-zA-Z0-9-]*$.
type: array
items:
type: string
pattern: ^tag:[a-zA-Z][a-zA-Z0-9-]*$
x-kubernetes-validations:
- rule: has(self.subnetRouter) || self.exitNode == true
message: A Connector needs to be either an exit node or a subnet router, or both.
status:
description: Status of the Connector. This is set and managed by the Tailscale operator.
description: ConnectorStatus describes the status of the Connector. This is set and managed by the Tailscale operator.
type: object
properties:
conditions:
@ -107,27 +113,12 @@ spec:
x-kubernetes-list-map-keys:
- type
x-kubernetes-list-type: map
subnetRouter:
description: SubnetRouter status is the current status of a subnet router
type: object
required:
- message
- ready
- reason
- routes
properties:
message:
description: Message is a more verbose reason for the current subnet router status
type: string
ready:
description: Ready is the ready status of the subnet router
type: string
reason:
description: Reason is the reason for the subnet router status
type: string
routes:
description: Routes are the CIDRs currently exposed via subnet router
type: string
isExitNode:
description: IsExitNode is set to true if the Connector acts as an exit node.
type: boolean
subnetRoutes:
description: SubnetRoutes are the routes currently exposed to tailnet via this Connector instance.
type: string
served: true
storage: true
subresources:

View File

@ -0,0 +1,19 @@
# Before applying ensure that the operator owns tag:prod.
# https://tailscale.com/kb/1236/kubernetes-operator/#setting-up-the-kubernetes-operator.
# To set up autoapproval set tag:prod as approver for 10.40.0.0/14 route and exit node.
# Otherwise approve it manually in Machines panel once the
# ts-prod Tailscale node has been created.
# See https://tailscale.com/kb/1018/acls/#auto-approvers-for-routes-and-exit-nodes
apiVersion: tailscale.com/v1alpha1
kind: Connector
metadata:
name: prod
spec:
tags:
- "tag:prod"
hostname: ts-prod
subnetRouter:
advertiseRoutes:
- "10.40.0.0/14"
- "192.168.0.0/14"
exitNode: true

View File

@ -1,17 +0,0 @@
# Before applyong this ensure that the operator is owner of tag:subnet.
# https://tailscale.com/kb/1236/kubernetes-operator/#setting-up-the-kubernetes-operator.
# To set up autoapproval set tag:subnet as approver for 10.40.0.0/14 route
# otherwise you will need to approve it manually in control panel once the
# subnet router has been created.
# https://tailscale.com/kb/1019/subnets/#advertise-subnet-routes
apiVersion: tailscale.com/v1alpha1
kind: Connector
metadata:
name: exposepods
spec:
subnetRouter:
routes:
- "10.40.0.0/14"
tags:
- "tag:subnet"
hostname: pods-subnetrouter

View File

@ -6,24 +6,15 @@
package main
import (
"context"
"fmt"
"strings"
"sync"
"testing"
"time"
"github.com/google/go-cmp/cmp"
"go.uber.org/zap"
appsv1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
apierrors "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/client/fake"
"sigs.k8s.io/controller-runtime/pkg/reconcile"
"tailscale.com/client/tailscale"
"tailscale.com/types/ptr"
)
@ -67,15 +58,18 @@ func TestLoadBalancerClass(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
opts := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSTS(o))
expectEqual(t, fc, expectedSecret(t, opts))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(opts))
// Normally the Tailscale proxy pod would come up here and write its info
// into the secret. Simulate that, then verify reconcile again and verify
@ -159,6 +153,7 @@ func TestLoadBalancerClass(t *testing.T) {
}
expectEqual(t, fc, want)
}
func TestTailnetTargetFQDNAnnotation(t *testing.T) {
fc := fake.NewFakeClient()
ft := &fakeTSClient{}
@ -204,15 +199,17 @@ func TestTailnetTargetFQDNAnnotation(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
tailnetTargetFQDN: tailnetTargetFQDN,
hostname: "default-test",
}
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(o))
want := &corev1.Service{
TypeMeta: metav1.TypeMeta{
@ -235,14 +232,8 @@ func TestTailnetTargetFQDNAnnotation(t *testing.T) {
},
}
expectEqual(t, fc, want)
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
o = stsOpts{
name: shortName,
secretName: fullName,
tailnetTargetFQDN: tailnetTargetFQDN,
hostname: "default-test",
}
expectEqual(t, fc, expectedSTS(o))
// Change the tailscale-target-fqdn annotation which should update the
@ -272,6 +263,7 @@ func TestTailnetTargetFQDNAnnotation(t *testing.T) {
expectMissing[corev1.Service](t, fc, "operator-ns", shortName)
expectMissing[corev1.Secret](t, fc, "operator-ns", fullName)
}
func TestTailnetTargetIPAnnotation(t *testing.T) {
fc := fake.NewFakeClient()
ft := &fakeTSClient{}
@ -317,15 +309,17 @@ func TestTailnetTargetIPAnnotation(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
tailnetTargetIP: tailnetTargetIP,
hostname: "default-test",
}
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(o))
want := &corev1.Service{
TypeMeta: metav1.TypeMeta{
@ -348,14 +342,8 @@ func TestTailnetTargetIPAnnotation(t *testing.T) {
},
}
expectEqual(t, fc, want)
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
o = stsOpts{
name: shortName,
secretName: fullName,
tailnetTargetIP: tailnetTargetIP,
hostname: "default-test",
}
expectEqual(t, fc, expectedSTS(o))
// Change the tailscale-target-ip annotation which should update the
@ -428,14 +416,17 @@ func TestAnnotations(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(o))
want := &corev1.Service{
TypeMeta: metav1.TypeMeta{
@ -533,14 +524,17 @@ func TestAnnotationIntoLB(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(o))
// Normally the Tailscale proxy pod would come up here and write its info
@ -586,11 +580,6 @@ func TestAnnotationIntoLB(t *testing.T) {
expectReconciled(t, sr, "default", "test")
// None of the proxy machinery should have changed...
expectEqual(t, fc, expectedHeadlessService(shortName))
o = stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
}
expectEqual(t, fc, expectedSTS(o))
// ... but the service should have a LoadBalancer status.
@ -666,14 +655,17 @@ func TestLBIntoAnnotation(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(o))
// Normally the Tailscale proxy pod would come up here and write its info
@ -737,11 +729,6 @@ func TestLBIntoAnnotation(t *testing.T) {
expectReconciled(t, sr, "default", "test")
expectEqual(t, fc, expectedHeadlessService(shortName))
o = stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
}
expectEqual(t, fc, expectedSTS(o))
want = &corev1.Service{
@ -809,14 +796,17 @@ func TestCustomHostname(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "reindeer-flotilla",
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "reindeer-flotilla",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSecret(t, o))
expectEqual(t, fc, expectedHeadlessService(shortName))
expectEqual(t, fc, expectedSTS(o))
want := &corev1.Service{
TypeMeta: metav1.TypeMeta{
@ -920,11 +910,14 @@ func TestCustomPriorityClassName(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
o := stsOpts{
name: shortName,
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "tailscale-critical",
priorityClassName: "custom-priority-class-name",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSTS(o))
@ -971,12 +964,14 @@ func TestDefaultLoadBalancer(t *testing.T) {
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
// expectEqual(t, fc, expectedSecret(fullName, "default", "svc"))
expectEqual(t, fc, expectedHeadlessService(shortName))
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSTS(o))
}
@ -1022,331 +1017,19 @@ func TestProxyFirewallMode(t *testing.T) {
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
o := stsOpts{
name: shortName,
secretName: fullName,
hostname: "default-test",
firewallMode: "nftables",
o := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
firewallMode: "nftables",
clusterTargetIP: "10.20.30.40",
}
expectEqual(t, fc, expectedSTS(o))
}
func expectedSecret(name, parentNamespace, typ string) *corev1.Secret {
return &corev1.Secret{
TypeMeta: metav1.TypeMeta{
Kind: "Secret",
APIVersion: "v1",
},
ObjectMeta: metav1.ObjectMeta{
Name: name,
Namespace: "operator-ns",
Labels: map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-ns": parentNamespace,
"tailscale.com/parent-resource-type": typ,
},
},
StringData: map[string]string{
"authkey": "secret-authkey",
},
}
}
func expectedHeadlessService(name string) *corev1.Service {
return &corev1.Service{
TypeMeta: metav1.TypeMeta{
Kind: "Service",
APIVersion: "v1",
},
ObjectMeta: metav1.ObjectMeta{
Name: name,
GenerateName: "ts-test-",
Namespace: "operator-ns",
Labels: map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-ns": "default",
"tailscale.com/parent-resource-type": "svc",
},
},
Spec: corev1.ServiceSpec{
Selector: map[string]string{
"app": "1234-UID",
},
ClusterIP: "None",
},
}
}
func expectedSTS(opts stsOpts) *appsv1.StatefulSet {
containerEnv := []corev1.EnvVar{
{Name: "TS_USERSPACE", Value: "false"},
{Name: "TS_AUTH_ONCE", Value: "true"},
{Name: "TS_KUBE_SECRET", Value: opts.secretName},
{Name: "TS_HOSTNAME", Value: opts.hostname},
}
annots := map[string]string{
"tailscale.com/operator-last-set-hostname": opts.hostname,
}
if opts.tailnetTargetIP != "" {
annots["tailscale.com/operator-last-set-ts-tailnet-target-ip"] = opts.tailnetTargetIP
containerEnv = append(containerEnv, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_IP",
Value: opts.tailnetTargetIP,
})
} else if opts.tailnetTargetFQDN != "" {
annots["tailscale.com/operator-last-set-ts-tailnet-target-fqdn"] = opts.tailnetTargetFQDN
containerEnv = append(containerEnv, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_FQDN",
Value: opts.tailnetTargetFQDN,
})
} else {
containerEnv = append(containerEnv, corev1.EnvVar{
Name: "TS_DEST_IP",
Value: "10.20.30.40",
})
annots["tailscale.com/operator-last-set-cluster-ip"] = "10.20.30.40"
}
if opts.firewallMode != "" {
containerEnv = append(containerEnv, corev1.EnvVar{
Name: "TS_DEBUG_FIREWALL_MODE",
Value: opts.firewallMode,
})
}
return &appsv1.StatefulSet{
TypeMeta: metav1.TypeMeta{
Kind: "StatefulSet",
APIVersion: "apps/v1",
},
ObjectMeta: metav1.ObjectMeta{
Name: opts.name,
Namespace: "operator-ns",
Labels: map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-ns": "default",
"tailscale.com/parent-resource-type": "svc",
},
},
Spec: appsv1.StatefulSetSpec{
Replicas: ptr.To[int32](1),
Selector: &metav1.LabelSelector{
MatchLabels: map[string]string{"app": "1234-UID"},
},
ServiceName: opts.name,
Template: corev1.PodTemplateSpec{
ObjectMeta: metav1.ObjectMeta{
Annotations: annots,
DeletionGracePeriodSeconds: ptr.To[int64](10),
Labels: map[string]string{"app": "1234-UID"},
},
Spec: corev1.PodSpec{
ServiceAccountName: "proxies",
PriorityClassName: opts.priorityClassName,
InitContainers: []corev1.Container{
{
Name: "sysctler",
Image: "tailscale/tailscale",
Command: []string{"/bin/sh"},
Args: []string{"-c", "sysctl -w net.ipv4.ip_forward=1 net.ipv6.conf.all.forwarding=1"},
SecurityContext: &corev1.SecurityContext{
Privileged: ptr.To(true),
},
},
},
Containers: []corev1.Container{
{
Name: "tailscale",
Image: "tailscale/tailscale",
Env: containerEnv,
SecurityContext: &corev1.SecurityContext{
Capabilities: &corev1.Capabilities{
Add: []corev1.Capability{"NET_ADMIN"},
},
},
ImagePullPolicy: "Always",
},
},
},
},
},
}
}
func findGenName(t *testing.T, client client.Client, ns, name, typ string) (full, noSuffix string) {
t.Helper()
labels := map[string]string{
LabelManaged: "true",
LabelParentName: name,
LabelParentNamespace: ns,
LabelParentType: typ,
}
s, err := getSingleObject[corev1.Secret](context.Background(), client, "operator-ns", labels)
if err != nil {
t.Fatalf("finding secret for %q: %v", name, err)
}
if s == nil {
t.Fatalf("no secret found for %q %s %+#v", name, ns, labels)
}
return s.GetName(), strings.TrimSuffix(s.GetName(), "-0")
}
func mustCreate(t *testing.T, client client.Client, obj client.Object) {
t.Helper()
if err := client.Create(context.Background(), obj); err != nil {
t.Fatalf("creating %q: %v", obj.GetName(), err)
}
}
func mustUpdate[T any, O ptrObject[T]](t *testing.T, client client.Client, ns, name string, update func(O)) {
t.Helper()
obj := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: name,
Namespace: ns,
}, obj); err != nil {
t.Fatalf("getting %q: %v", name, err)
}
update(obj)
if err := client.Update(context.Background(), obj); err != nil {
t.Fatalf("updating %q: %v", name, err)
}
}
func mustUpdateStatus[T any, O ptrObject[T]](t *testing.T, client client.Client, ns, name string, update func(O)) {
t.Helper()
obj := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: name,
Namespace: ns,
}, obj); err != nil {
t.Fatalf("getting %q: %v", name, err)
}
update(obj)
if err := client.Status().Update(context.Background(), obj); err != nil {
t.Fatalf("updating %q: %v", name, err)
}
}
func expectEqual[T any, O ptrObject[T]](t *testing.T, client client.Client, want O) {
t.Helper()
got := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: want.GetName(),
Namespace: want.GetNamespace(),
}, got); err != nil {
t.Fatalf("getting %q: %v", want.GetName(), err)
}
// The resource version changes eagerly whenever the operator does even a
// no-op update. Asserting a specific value leads to overly brittle tests,
// so just remove it from both got and want.
got.SetResourceVersion("")
want.SetResourceVersion("")
if diff := cmp.Diff(got, want); diff != "" {
t.Fatalf("unexpected object (-got +want):\n%s", diff)
}
}
func expectMissing[T any, O ptrObject[T]](t *testing.T, client client.Client, ns, name string) {
t.Helper()
obj := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: name,
Namespace: ns,
}, obj); !apierrors.IsNotFound(err) {
t.Fatalf("object %s/%s unexpectedly present, wanted missing", ns, name)
}
}
func expectReconciled(t *testing.T, sr reconcile.Reconciler, ns, name string) {
t.Helper()
req := reconcile.Request{
NamespacedName: types.NamespacedName{
Namespace: ns,
Name: name,
},
}
res, err := sr.Reconcile(context.Background(), req)
if err != nil {
t.Fatalf("Reconcile: unexpected error: %v", err)
}
if res.Requeue {
t.Fatalf("unexpected immediate requeue")
}
if res.RequeueAfter != 0 {
t.Fatalf("unexpected timed requeue (%v)", res.RequeueAfter)
}
}
func expectRequeue(t *testing.T, sr reconcile.Reconciler, ns, name string) {
t.Helper()
req := reconcile.Request{
NamespacedName: types.NamespacedName{
Name: name,
Namespace: ns,
},
}
res, err := sr.Reconcile(context.Background(), req)
if err != nil {
t.Fatalf("Reconcile: unexpected error: %v", err)
}
if res.RequeueAfter == 0 {
t.Fatalf("expected timed requeue, got success")
}
}
type stsOpts struct {
name string
secretName string
hostname string
priorityClassName string
firewallMode string
tailnetTargetIP string
tailnetTargetFQDN string
}
type fakeTSClient struct {
sync.Mutex
keyRequests []tailscale.KeyCapabilities
deleted []string
}
func (c *fakeTSClient) CreateKey(ctx context.Context, caps tailscale.KeyCapabilities) (string, *tailscale.Key, error) {
c.Lock()
defer c.Unlock()
c.keyRequests = append(c.keyRequests, caps)
k := &tailscale.Key{
ID: "key",
Created: time.Now(),
Capabilities: caps,
}
return "secret-authkey", k, nil
}
func (c *fakeTSClient) DeleteDevice(ctx context.Context, deviceID string) error {
c.Lock()
defer c.Unlock()
c.deleted = append(c.deleted, deviceID)
return nil
}
func (c *fakeTSClient) KeyRequests() []tailscale.KeyCapabilities {
c.Lock()
defer c.Unlock()
return c.keyRequests
}
func (c *fakeTSClient) Deleted() []string {
c.Lock()
defer c.Unlock()
return c.deleted
}
func Test_isMagicDNSName(t *testing.T) {
tests := []struct {
in string

View File

@ -7,6 +7,7 @@ package main
import (
"context"
"crypto/sha256"
_ "embed"
"encoding/json"
"errors"
@ -26,6 +27,7 @@ import (
"sigs.k8s.io/yaml"
"tailscale.com/client/tailscale"
"tailscale.com/ipn"
"tailscale.com/net/netutil"
"tailscale.com/tailcfg"
"tailscale.com/tsnet"
"tailscale.com/types/opt"
@ -54,11 +56,17 @@ const (
AnnotationFunnel = "tailscale.com/funnel"
// Annotations set by the operator on pods to trigger restarts when the
// hostname, IP or FQDN changes.
// hostname, IP, FQDN or tailscaled config changes.
podAnnotationLastSetClusterIP = "tailscale.com/operator-last-set-cluster-ip"
podAnnotationLastSetHostname = "tailscale.com/operator-last-set-hostname"
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.
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"
)
type tailscaleSTSConfig struct {
@ -66,22 +74,26 @@ type tailscaleSTSConfig struct {
ParentResourceUID string
ChildResourceLabels map[string]string
ServeConfig *ipn.ServeConfig
// Tailscale target in cluster we are setting up ingress for
ClusterTargetIP string
ServeConfig *ipn.ServeConfig
ClusterTargetIP string // ingress target
// Tailscale IP of a Tailscale service we are setting up egress for
TailnetTargetIP string
TailnetTargetIP string // egress target IP
// Tailscale FQDN of a Tailscale service we are setting up egress for
TailnetTargetFQDN string
TailnetTargetFQDN string // egress target FQDN
Hostname string
Tags []string // if empty, use defaultTags
// Routes is a list of CIDRs to pass via --advertise-routes flag
// Should only be set if this is config for subnetRouter
Routes string
// Connector specifies a configuration of a Connector instance if that's
// what this StatefulSet should be created for.
Connector *connector
}
type connector struct {
// routes is a list of subnet routes that this Connector should expose.
routes string
// isExitNode defines whether this Connector should act as an exit node.
isExitNode bool
}
type tailscaleSTSReconciler struct {
@ -111,16 +123,17 @@ func (a *tailscaleSTSReconciler) IsHTTPSEnabledOnTailnet() bool {
// up to date.
func (a *tailscaleSTSReconciler) Provision(ctx context.Context, logger *zap.SugaredLogger, sts *tailscaleSTSConfig) (*corev1.Service, error) {
// Do full reconcile.
// TODO (don't create Service for the Connector)
hsvc, err := a.reconcileHeadlessService(ctx, logger, sts)
if err != nil {
return nil, fmt.Errorf("failed to reconcile headless service: %w", err)
}
secretName, err := a.createOrGetSecret(ctx, logger, sts, hsvc)
secretName, tsConfigHash, 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)
_, err = a.reconcileSTS(ctx, logger, sts, hsvc, secretName, tsConfigHash)
if err != nil {
return nil, fmt.Errorf("failed to reconcile statefulset: %w", err)
}
@ -234,7 +247,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, error) {
func (a *tailscaleSTSReconciler) createOrGetSecret(ctx context.Context, logger *zap.SugaredLogger, stsC *tailscaleSTSConfig, hsvc *corev1.Service) (string, string, error) {
secret := &corev1.Secret{
ObjectMeta: metav1.ObjectMeta{
// Hardcode a -0 suffix so that in future, if we support
@ -250,22 +263,25 @@ 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 "", "", err
}
var (
authKey, hash string
)
if orig == nil {
// Secret doesn't exist yet, create one. Initially it contains
// only the Tailscale authkey, but once Tailscale starts it'll
// also store the daemon state.
sts, err := getSingleObject[appsv1.StatefulSet](ctx, a.Client, a.operatorNamespace, stsC.ChildResourceLabels)
if err != nil {
return "", err
return "", "", 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
}
// Create API Key secret which is going to be used by the statefulset
// to authenticate with Tailscale.
@ -274,30 +290,42 @@ func (a *tailscaleSTSReconciler) createOrGetSecret(ctx context.Context, logger *
if len(tags) == 0 {
tags = a.defaultTags
}
authKey, err := a.newAuthKey(ctx, tags)
authKey, err = a.newAuthKey(ctx, tags)
if err != nil {
return "", err
return "", "", err
}
}
if !shouldDoTailscaledDeclarativeConfig(stsC) && authKey != "" {
mak.Set(&secret.StringData, "authkey", authKey)
}
if shouldDoTailscaledDeclarativeConfig(stsC) {
confFileBytes, h, err := tailscaledConfig(stsC, authKey, orig)
if err != nil {
return "", "", fmt.Errorf("error creating tailscaled config: %w", err)
}
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 "", "", err
}
mak.Set(&secret.StringData, "serve-config", string(j))
}
if orig != nil {
logger.Debugf("patching existing state Secret with values %s", secret.Data[tailscaledConfigKey])
if err := a.Patch(ctx, secret, client.MergeFrom(orig)); err != nil {
return "", err
return "", "", err
}
} else {
logger.Debugf("creating new state Secret with authkey %s", secret.Data[tailscaledConfigKey])
if err := a.Create(ctx, secret); err != nil {
return "", err
return "", "", err
}
}
return secret.Name, nil
return secret.Name, hash, nil
}
// DeviceInfo returns the device ID and hostname for the Tailscale device
@ -325,7 +353,6 @@ func (a *tailscaleSTSReconciler) DeviceInfo(ctx context.Context, childLabels map
return "", "", nil, err
}
}
return id, hostname, ips, nil
}
@ -353,7 +380,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, authKeySecret string) (*appsv1.StatefulSet, error) {
func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.SugaredLogger, sts *tailscaleSTSConfig, headlessSvc *corev1.Service, proxySecret, tsConfigHash string) (*appsv1.StatefulSet, error) {
var ss appsv1.StatefulSet
if sts.ServeConfig != nil {
if err := yaml.Unmarshal(userspaceProxyYaml, &ss); err != nil {
@ -373,30 +400,90 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
}
container := &ss.Spec.Template.Spec.Containers[0]
container.Image = a.proxyImage
ss.ObjectMeta = metav1.ObjectMeta{
Name: headlessSvc.Name,
Namespace: a.operatorNamespace,
Labels: sts.ChildResourceLabels,
}
ss.Spec.ServiceName = headlessSvc.Name
ss.Spec.Selector = &metav1.LabelSelector{
MatchLabels: map[string]string{
"app": sts.ParentResourceUID,
},
}
mak.Set(&ss.Spec.Template.Labels, "app", sts.ParentResourceUID)
// Generic containerboot configuration options.
container.Env = append(container.Env,
corev1.EnvVar{
Name: "TS_KUBE_SECRET",
Value: authKeySecret,
Value: proxySecret,
},
corev1.EnvVar{
)
if !shouldDoTailscaledDeclarativeConfig(sts) {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_HOSTNAME",
Value: sts.Hostname,
})
// containerboot currently doesn't have a way to re-read the hostname/ip as
// it is passed via an environment variable. So we need to restart the
// container when the value changes. We do this by adding an annotation to
// the pod template that contains the last value we set.
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetHostname, sts.Hostname)
}
// Configure containeboot to run tailscaled with a configfile read from the state Secret.
if shouldDoTailscaledDeclarativeConfig(sts) {
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetConfigFileHash, tsConfigHash)
ss.Spec.Template.Spec.Volumes = append(ss.Spec.Template.Spec.Volumes, corev1.Volume{
Name: "tailscaledconfig",
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: proxySecret,
Items: []corev1.KeyToPath{{
Key: tailscaledConfigKey,
Path: tailscaledConfigKey,
}},
},
},
})
container.VolumeMounts = append(container.VolumeMounts, corev1.VolumeMount{
Name: "tailscaledconfig",
ReadOnly: true,
MountPath: "/etc/tsconfig",
})
container.Env = append(container.Env, corev1.EnvVar{
Name: "EXPERIMENTAL_TS_CONFIGFILE_PATH",
Value: "/etc/tsconfig/tailscaled",
})
}
if a.tsFirewallMode != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_DEBUG_FIREWALL_MODE",
Value: a.tsFirewallMode,
})
}
ss.Spec.Template.Spec.PriorityClassName = a.proxyPriorityClassName
// Ingress/egress proxy configuration options.
if sts.ClusterTargetIP != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_DEST_IP",
Value: sts.ClusterTargetIP,
})
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetClusterIP, sts.ClusterTargetIP)
} else if sts.TailnetTargetIP != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_IP",
Value: sts.TailnetTargetIP,
})
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetTailnetTargetIP, sts.TailnetTargetIP)
} else if sts.TailnetTargetFQDN != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_FQDN",
Value: sts.TailnetTargetFQDN,
})
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetTailnetTargetFQDN, sts.TailnetTargetFQDN)
} else if sts.ServeConfig != nil {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_SERVE_CONFIG",
@ -411,7 +498,7 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
Name: "serve-config",
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: authKeySecret,
SecretName: proxySecret,
Items: []corev1.KeyToPath{{
Key: "serve-config",
Path: "serve-config",
@ -419,56 +506,48 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
},
},
})
} else if len(sts.Routes) > 0 {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_ROUTES",
Value: sts.Routes,
})
}
if a.tsFirewallMode != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_DEBUG_FIREWALL_MODE",
Value: a.tsFirewallMode,
},
)
}
ss.ObjectMeta = metav1.ObjectMeta{
Name: headlessSvc.Name,
Namespace: a.operatorNamespace,
Labels: sts.ChildResourceLabels,
}
ss.Spec.ServiceName = headlessSvc.Name
ss.Spec.Selector = &metav1.LabelSelector{
MatchLabels: map[string]string{
"app": sts.ParentResourceUID,
},
}
// containerboot currently doesn't have a way to re-read the hostname/ip as
// it is passed via an environment variable. So we need to restart the
// container when the value changes. We do this by adding an annotation to
// the pod template that contains the last value we set.
ss.Spec.Template.Annotations = map[string]string{
podAnnotationLastSetHostname: sts.Hostname,
}
if sts.ClusterTargetIP != "" {
ss.Spec.Template.Annotations[podAnnotationLastSetClusterIP] = sts.ClusterTargetIP
}
if sts.TailnetTargetIP != "" {
ss.Spec.Template.Annotations[podAnnotationLastSetTailnetTargetIP] = sts.TailnetTargetIP
}
if sts.TailnetTargetFQDN != "" {
ss.Spec.Template.Annotations[podAnnotationLastSetTailnetTargetFQDN] = sts.TailnetTargetFQDN
}
ss.Spec.Template.Labels = map[string]string{
"app": sts.ParentResourceUID,
}
ss.Spec.Template.Spec.PriorityClassName = a.proxyPriorityClassName
logger.Debugf("reconciling statefulset %s/%s", ss.GetNamespace(), ss.GetName())
return createOrUpdate(ctx, a.Client, a.operatorNamespace, &ss, func(s *appsv1.StatefulSet) { s.Spec = ss.Spec })
}
// 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",
Locked: "false",
Hostname: &stsC.Hostname,
}
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)
}
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)
}
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
}
// ptrObject is a type constraint for pointer types that implement
// client.Object.
type ptrObject[T any] interface {
@ -476,6 +555,24 @@ type ptrObject[T any] interface {
*T
}
// hashBytes produces a hash for the provided bytes 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
// is used to determine if the container running the Connector Tailscale node
// needs to be restarted. The container does not need restarting when the only
// 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)
if err != nil {
return "", fmt.Errorf("error calculating hash: %w", err)
}
return fmt.Sprintf("%x", h.Sum(nil)), nil
}
// createOrUpdate adds obj to the k8s cluster, unless the object already exists,
// in which case update is called to make changes to it. If update is nil, the
// existing object is returned unmodified.
@ -579,3 +676,10 @@ func nameForService(svc *corev1.Service) (string, error) {
func isValidFirewallMode(m string) bool {
return m == "auto" || m == "nftables" || m == "iptables"
}
// shouldDoTailscaledDeclarativeConfig determines whether the proxy instance
// should be configured to run tailscaled only with a all config opts passed to
// tailscaled.
func shouldDoTailscaledDeclarativeConfig(stsC *tailscaleSTSConfig) bool {
return stsC.Connector != nil
}

View File

@ -0,0 +1,411 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
//go:build !plan9
package main
import (
"context"
"encoding/json"
"net/netip"
"strings"
"sync"
"testing"
"time"
"github.com/google/go-cmp/cmp"
appsv1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
apierrors "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/reconcile"
"tailscale.com/client/tailscale"
"tailscale.com/ipn"
"tailscale.com/types/ptr"
"tailscale.com/util/mak"
)
// confgOpts contains configuration options for creating cluster resources for
// Tailscale proxies.
type configOpts struct {
stsName string
secretName string
hostname string
namespace string
parentType string
priorityClassName string
firewallMode string
tailnetTargetIP string
tailnetTargetFQDN string
clusterTargetIP string
subnetRoutes string
isExitNode bool
shouldUseDeclarativeConfig bool // tailscaled in proxy should be configured using config file
confFileHash string
}
func expectedSTS(opts configOpts) *appsv1.StatefulSet {
tsContainer := corev1.Container{
Name: "tailscale",
Image: "tailscale/tailscale",
Env: []corev1.EnvVar{
{Name: "TS_USERSPACE", Value: "false"},
{Name: "TS_AUTH_ONCE", Value: "true"},
{Name: "TS_KUBE_SECRET", Value: opts.secretName},
},
SecurityContext: &corev1.SecurityContext{
Capabilities: &corev1.Capabilities{
Add: []corev1.Capability{"NET_ADMIN"},
},
},
ImagePullPolicy: "Always",
}
annots := make(map[string]string)
var volumes []corev1.Volume
if opts.shouldUseDeclarativeConfig {
volumes = []corev1.Volume{
{
Name: "tailscaledconfig",
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: opts.secretName,
Items: []corev1.KeyToPath{
{
Key: "tailscaled",
Path: "tailscaled",
},
},
},
},
},
}
tsContainer.VolumeMounts = []corev1.VolumeMount{{
Name: "tailscaledconfig",
ReadOnly: true,
MountPath: "/etc/tsconfig",
}}
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{
Name: "EXPERIMENTAL_TS_CONFIGFILE_PATH",
Value: "/etc/tsconfig/tailscaled",
})
annots["tailscale.com/operator-last-set-config-file-hash"] = opts.confFileHash
} else {
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{Name: "TS_HOSTNAME", Value: opts.hostname})
annots["tailscale.com/operator-last-set-hostname"] = opts.hostname
}
if opts.firewallMode != "" {
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{
Name: "TS_DEBUG_FIREWALL_MODE",
Value: opts.firewallMode,
})
}
if opts.tailnetTargetIP != "" {
annots["tailscale.com/operator-last-set-ts-tailnet-target-ip"] = opts.tailnetTargetIP
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_IP",
Value: opts.tailnetTargetIP,
})
} else if opts.tailnetTargetFQDN != "" {
annots["tailscale.com/operator-last-set-ts-tailnet-target-fqdn"] = opts.tailnetTargetFQDN
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_FQDN",
Value: opts.tailnetTargetFQDN,
})
} else if opts.clusterTargetIP != "" {
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{
Name: "TS_DEST_IP",
Value: opts.clusterTargetIP,
})
annots["tailscale.com/operator-last-set-cluster-ip"] = opts.clusterTargetIP
}
return &appsv1.StatefulSet{
TypeMeta: metav1.TypeMeta{
Kind: "StatefulSet",
APIVersion: "apps/v1",
},
ObjectMeta: metav1.ObjectMeta{
Name: opts.stsName,
Namespace: "operator-ns",
Labels: map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-ns": opts.namespace,
"tailscale.com/parent-resource-type": opts.parentType,
},
},
Spec: appsv1.StatefulSetSpec{
Replicas: ptr.To[int32](1),
Selector: &metav1.LabelSelector{
MatchLabels: map[string]string{"app": "1234-UID"},
},
ServiceName: opts.stsName,
Template: corev1.PodTemplateSpec{
ObjectMeta: metav1.ObjectMeta{
Annotations: annots,
DeletionGracePeriodSeconds: ptr.To[int64](10),
Labels: map[string]string{"app": "1234-UID"},
},
Spec: corev1.PodSpec{
ServiceAccountName: "proxies",
PriorityClassName: opts.priorityClassName,
InitContainers: []corev1.Container{
{
Name: "sysctler",
Image: "tailscale/tailscale",
Command: []string{"/bin/sh"},
Args: []string{"-c", "sysctl -w net.ipv4.ip_forward=1 net.ipv6.conf.all.forwarding=1"},
SecurityContext: &corev1.SecurityContext{
Privileged: ptr.To(true),
},
},
},
Containers: []corev1.Container{tsContainer},
Volumes: volumes,
},
},
},
}
}
func expectedHeadlessService(name string) *corev1.Service {
return &corev1.Service{
TypeMeta: metav1.TypeMeta{
Kind: "Service",
APIVersion: "v1",
},
ObjectMeta: metav1.ObjectMeta{
Name: name,
GenerateName: "ts-test-",
Namespace: "operator-ns",
Labels: map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-ns": "default",
"tailscale.com/parent-resource-type": "svc",
},
},
Spec: corev1.ServiceSpec{
Selector: map[string]string{
"app": "1234-UID",
},
ClusterIP: "None",
},
}
}
func expectedSecret(t *testing.T, opts configOpts) *corev1.Secret {
t.Helper()
labels := map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
"tailscale.com/parent-resource-type": opts.parentType,
}
s := &corev1.Secret{
TypeMeta: metav1.TypeMeta{
Kind: "Secret",
APIVersion: "v1",
},
ObjectMeta: metav1.ObjectMeta{
Name: opts.secretName,
Namespace: "operator-ns",
},
}
if !opts.shouldUseDeclarativeConfig {
mak.Set(&s.StringData, "authkey", "secret-authkey")
labels["tailscale.com/parent-resource-ns"] = opts.namespace
} else {
conf := &ipn.ConfigVAlpha{
Version: "alpha0",
AcceptDNS: "false",
Hostname: &opts.hostname,
Locked: "false",
AuthKey: ptr.To("secret-authkey"),
}
var routes []netip.Prefix
if opts.subnetRoutes != "" || opts.isExitNode {
r := opts.subnetRoutes
if opts.isExitNode {
r = "0.0.0.0/0,::/0," + r
}
for _, rr := range strings.Split(r, ",") {
prefix, err := netip.ParsePrefix(rr)
if err != nil {
t.Fatal(err)
}
routes = append(routes, prefix)
}
}
conf.AdvertiseRoutes = routes
b, err := json.Marshal(conf)
if err != nil {
t.Fatalf("error marshalling tailscaled config")
}
mak.Set(&s.StringData, "tailscaled", string(b))
labels["tailscale.com/parent-resource-ns"] = "" // Connector is cluster scoped
}
s.Labels = labels
return s
}
func findGenName(t *testing.T, client client.Client, ns, name, typ string) (full, noSuffix string) {
t.Helper()
labels := map[string]string{
LabelManaged: "true",
LabelParentName: name,
LabelParentNamespace: ns,
LabelParentType: typ,
}
s, err := getSingleObject[corev1.Secret](context.Background(), client, "operator-ns", labels)
if err != nil {
t.Fatalf("finding secret for %q: %v", name, err)
}
if s == nil {
t.Fatalf("no secret found for %q %s %+#v", name, ns, labels)
}
return s.GetName(), strings.TrimSuffix(s.GetName(), "-0")
}
func mustCreate(t *testing.T, client client.Client, obj client.Object) {
t.Helper()
if err := client.Create(context.Background(), obj); err != nil {
t.Fatalf("creating %q: %v", obj.GetName(), err)
}
}
func mustUpdate[T any, O ptrObject[T]](t *testing.T, client client.Client, ns, name string, update func(O)) {
t.Helper()
obj := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: name,
Namespace: ns,
}, obj); err != nil {
t.Fatalf("getting %q: %v", name, err)
}
update(obj)
if err := client.Update(context.Background(), obj); err != nil {
t.Fatalf("updating %q: %v", name, err)
}
}
func mustUpdateStatus[T any, O ptrObject[T]](t *testing.T, client client.Client, ns, name string, update func(O)) {
t.Helper()
obj := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: name,
Namespace: ns,
}, obj); err != nil {
t.Fatalf("getting %q: %v", name, err)
}
update(obj)
if err := client.Status().Update(context.Background(), obj); err != nil {
t.Fatalf("updating %q: %v", name, err)
}
}
func expectEqual[T any, O ptrObject[T]](t *testing.T, client client.Client, want O) {
t.Helper()
got := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: want.GetName(),
Namespace: want.GetNamespace(),
}, got); err != nil {
t.Fatalf("getting %q: %v", want.GetName(), err)
}
// The resource version changes eagerly whenever the operator does even a
// no-op update. Asserting a specific value leads to overly brittle tests,
// so just remove it from both got and want.
got.SetResourceVersion("")
want.SetResourceVersion("")
if diff := cmp.Diff(got, want); diff != "" {
t.Fatalf("unexpected object (-got +want):\n%s", diff)
}
}
func expectMissing[T any, O ptrObject[T]](t *testing.T, client client.Client, ns, name string) {
t.Helper()
obj := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
Name: name,
Namespace: ns,
}, obj); !apierrors.IsNotFound(err) {
t.Fatalf("object %s/%s unexpectedly present, wanted missing", ns, name)
}
}
func expectReconciled(t *testing.T, sr reconcile.Reconciler, ns, name string) {
t.Helper()
req := reconcile.Request{
NamespacedName: types.NamespacedName{
Namespace: ns,
Name: name,
},
}
res, err := sr.Reconcile(context.Background(), req)
if err != nil {
t.Fatalf("Reconcile: unexpected error: %v", err)
}
if res.Requeue {
t.Fatalf("unexpected immediate requeue")
}
if res.RequeueAfter != 0 {
t.Fatalf("unexpected timed requeue (%v)", res.RequeueAfter)
}
}
func expectRequeue(t *testing.T, sr reconcile.Reconciler, ns, name string) {
t.Helper()
req := reconcile.Request{
NamespacedName: types.NamespacedName{
Name: name,
Namespace: ns,
},
}
res, err := sr.Reconcile(context.Background(), req)
if err != nil {
t.Fatalf("Reconcile: unexpected error: %v", err)
}
if res.RequeueAfter == 0 {
t.Fatalf("expected timed requeue, got success")
}
}
type fakeTSClient struct {
sync.Mutex
keyRequests []tailscale.KeyCapabilities
deleted []string
}
func (c *fakeTSClient) CreateKey(ctx context.Context, caps tailscale.KeyCapabilities) (string, *tailscale.Key, error) {
c.Lock()
defer c.Unlock()
c.keyRequests = append(c.keyRequests, caps)
k := &tailscale.Key{
ID: "key",
Created: time.Now(),
Capabilities: caps,
}
return "secret-authkey", k, nil
}
func (c *fakeTSClient) DeleteDevice(ctx context.Context, deviceID string) error {
c.Lock()
defer c.Unlock()
c.deleted = append(c.deleted, deviceID)
return nil
}
func (c *fakeTSClient) KeyRequests() []tailscale.KeyCapabilities {
c.Lock()
defer c.Unlock()
return c.keyRequests
}
func (c *fakeTSClient) Deleted() []string {
c.Lock()
defer c.Unlock()
return c.deleted
}

View File

@ -6,6 +6,9 @@
package v1alpha1
import (
"fmt"
"strings"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
)
@ -17,17 +20,19 @@ var ConnectorKind = "Connector"
// +kubebuilder:object:root=true
// +kubebuilder:subresource:status
// +kubebuilder:resource:scope=Cluster,shortName=cn
// +kubebuilder:printcolumn:name="SubnetRoutes",type="string",JSONPath=`.status.subnetRouter.routes`,description="Cluster CIDR ranges exposed to tailnet via subnet router"
// +kubebuilder:printcolumn:name="Status",type="string",JSONPath=`.status.conditions[?(@.type == "ConnectorReady")].reason`,description="Status of the components deployed by the connector"
// +kubebuilder:printcolumn:name="SubnetRoutes",type="string",JSONPath=`.status.subnetRoutes`,description="CIDR ranges exposed to tailnet by a subnet router defined via this Connector instance."
// +kubebuilder:printcolumn:name="IsExitNode",type="string",JSONPath=`.status.isExitNode`,description="Whether this Connector instance defines an exit node."
// +kubebuilder:printcolumn:name="Status",type="string",JSONPath=`.status.conditions[?(@.type == "ConnectorReady")].reason`,description="Status of the deployed Connector resources."
type Connector struct {
metav1.TypeMeta `json:",inline"`
metav1.ObjectMeta `json:"metadata,omitempty"`
// Desired state of the Connector resource.
// ConnectorSpec describes the desired Tailscale component.
Spec ConnectorSpec `json:"spec"`
// Status of the Connector. This is set and managed by the Tailscale operator.
// ConnectorStatus describes the status of the Connector. This is set
// and managed by the Tailscale operator.
// +optional
Status ConnectorStatus `json:"status"`
}
@ -41,40 +46,73 @@ type ConnectorList struct {
Items []Connector `json:"items"`
}
// ConnectorSpec defines the desired state of a ConnectorSpec.
// ConnectorSpec describes a Tailscale node to be deployed in the cluster.
// +kubebuilder:validation:XValidation:rule="has(self.subnetRouter) || self.exitNode == true",message="A Connector needs to be either an exit node or a subnet router, or both."
type ConnectorSpec struct {
// SubnetRouter configures a Tailscale subnet router to be deployed in
// the cluster. If unset no subnet router will be deployed.
// Tags that the Tailscale node will be tagged with.
// Defaults to [tag:k8s].
// To autoapprove the subnet routes or exit node defined by a Connector,
// you can configure Tailscale ACLs to give these tags the necessary
// permissions.
// See https://tailscale.com/kb/1018/acls/#auto-approvers-for-routes-and-exit-nodes.
// If you specify custom tags here, you must also make the operator an owner of these tags.
// See https://tailscale.com/kb/1236/kubernetes-operator/#setting-up-the-kubernetes-operator.
// Tags cannot be changed once a Connector node has been created.
// Tag values must be in form ^tag:[a-zA-Z][a-zA-Z0-9-]*$.
// +optional
Tags Tags `json:"tags,omitempty"`
// Hostname is the tailnet hostname that should be assigned to the
// Connector node. If unset, hostname defaults to <connector
// name>-connector. Hostname can contain lower case letters, numbers and
// dashes, it must not start or end with a dash and must be between 2
// and 63 characters long.
// +optional
Hostname Hostname `json:"hostname,omitempty"`
// SubnetRouter defines subnet routes that the Connector node should
// expose to tailnet. If unset, none are exposed.
// https://tailscale.com/kb/1019/subnets/
// +optional
SubnetRouter *SubnetRouter `json:"subnetRouter"`
// ExitNode defines whether the Connector node should act as a
// Tailscale exit node. Defaults to false.
// https://tailscale.com/kb/1103/exit-nodes
// +optional
ExitNode bool `json:"exitNode"`
}
// SubnetRouter describes a subnet router.
// +kubebuilder:validation:XValidation:rule="has(self.tags) == has(oldSelf.tags)",message="Subnetrouter tags cannot be changed. Delete and redeploy the Connector if you need to change it."
// SubnetRouter defines subnet routes that should be exposed to tailnet via a
// Connector node.
type SubnetRouter struct {
// Routes refer to in-cluster CIDRs that the subnet router should make
// AdvertiseRoutes refer to CIDRs that the subnet router should make
// available. Route values must be strings that represent a valid IPv4
// or IPv6 CIDR range. Values can be Tailscale 4via6 subnet routes.
// https://tailscale.com/kb/1201/4via6-subnets/
Routes []Route `json:"routes"`
// Tags that the Tailscale node will be tagged with. If you want the
// subnet router to be autoapproved, you can configure Tailscale ACLs to
// autoapprove the subnetrouter's CIDRs for these tags.
// See https://tailscale.com/kb/1018/acls/#auto-approvers-for-routes-and-exit-nodes
// Defaults to tag:k8s.
// If you specify custom tags here, you must also make tag:k8s-operator owner of the custom tag.
// See https://tailscale.com/kb/1236/kubernetes-operator/#setting-up-the-kubernetes-operator.
// Tags cannot be changed once a Connector has been created.
// Tag values must be in form ^tag:[a-zA-Z][a-zA-Z0-9-]*$.
// +optional
Tags []Tag `json:"tags,omitempty"`
// Hostname is the tailnet hostname that should be assigned to the
// subnet router node. If unset hostname is defaulted to <connector
// name>-subnetrouter. Hostname can contain lower case letters, numbers
// and dashes, it must not start or end with a dash and must be between
// 2 and 63 characters long.
// +optional
Hostname Hostname `json:"hostname,omitempty"`
AdvertiseRoutes Routes `json:"advertiseRoutes"`
}
type Tags []Tag
func (tags Tags) Stringify() []string {
stringTags := make([]string, len(tags))
for i, t := range tags {
stringTags[i] = string(t)
}
return stringTags
}
// +kubebuilder:validation:MinItems=1
type Routes []Route
func (routes Routes) Stringify() string {
if len(routes) < 1 {
return ""
}
var sb strings.Builder
sb.WriteString(string(routes[0]))
for _, r := range routes[1:] {
sb.WriteString(fmt.Sprintf(",%s", r))
}
return sb.String()
}
// +kubebuilder:validation:Type=string
@ -91,28 +129,19 @@ type Hostname string
// ConnectorStatus defines the observed state of the Connector.
type ConnectorStatus struct {
// List of status conditions to indicate the status of the Connector.
// Known condition types are `ConnectorReady`.
// +listType=map
// +listMapKey=type
// +optional
Conditions []ConnectorCondition `json:"conditions"`
// SubnetRouter status is the current status of a subnet router
// SubnetRoutes are the routes currently exposed to tailnet via this
// Connector instance.
// +optional
SubnetRouter *SubnetRouterStatus `json:"subnetRouter"`
}
// SubnetRouter status is the current status of a subnet router if deployed
type SubnetRouterStatus struct {
// Routes are the CIDRs currently exposed via subnet router
Routes string `json:"routes"`
// Ready is the ready status of the subnet router
Ready metav1.ConditionStatus `json:"ready"`
// Reason is the reason for the subnet router status
Reason string `json:"reason"`
// Message is a more verbose reason for the current subnet router status
Message string `json:"message"`
SubnetRoutes string `json:"subnetRoutes"`
// IsExitNode is set to true if the Connector acts as an exit node.
// +optional
IsExitNode bool `json:"isExitNode"`
}
// ConnectorCondition contains condition information for a Connector.
@ -147,7 +176,7 @@ type ConnectorCondition struct {
ObservedGeneration int64 `json:"observedGeneration,omitempty"`
}
// ConnectorConditionType represents a Connector condition type
// ConnectorConditionType represents a Connector condition type.
type ConnectorConditionType string
const (

View File

@ -92,6 +92,11 @@ func (in *ConnectorList) DeepCopyObject() runtime.Object {
// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (in *ConnectorSpec) DeepCopyInto(out *ConnectorSpec) {
*out = *in
if in.Tags != nil {
in, out := &in.Tags, &out.Tags
*out = make(Tags, len(*in))
copy(*out, *in)
}
if in.SubnetRouter != nil {
in, out := &in.SubnetRouter, &out.SubnetRouter
*out = new(SubnetRouter)
@ -119,11 +124,6 @@ func (in *ConnectorStatus) DeepCopyInto(out *ConnectorStatus) {
(*in)[i].DeepCopyInto(&(*out)[i])
}
}
if in.SubnetRouter != nil {
in, out := &in.SubnetRouter, &out.SubnetRouter
*out = new(SubnetRouterStatus)
**out = **in
}
}
// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ConnectorStatus.
@ -137,16 +137,30 @@ func (in *ConnectorStatus) DeepCopy() *ConnectorStatus {
}
// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (in *SubnetRouter) DeepCopyInto(out *SubnetRouter) {
*out = *in
if in.Routes != nil {
in, out := &in.Routes, &out.Routes
*out = make([]Route, len(*in))
func (in Routes) DeepCopyInto(out *Routes) {
{
in := &in
*out = make(Routes, len(*in))
copy(*out, *in)
}
if in.Tags != nil {
in, out := &in.Tags, &out.Tags
*out = make([]Tag, len(*in))
}
// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new Routes.
func (in Routes) DeepCopy() Routes {
if in == nil {
return nil
}
out := new(Routes)
in.DeepCopyInto(out)
return *out
}
// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (in *SubnetRouter) DeepCopyInto(out *SubnetRouter) {
*out = *in
if in.AdvertiseRoutes != nil {
in, out := &in.AdvertiseRoutes, &out.AdvertiseRoutes
*out = make(Routes, len(*in))
copy(*out, *in)
}
}
@ -162,16 +176,20 @@ func (in *SubnetRouter) DeepCopy() *SubnetRouter {
}
// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (in *SubnetRouterStatus) DeepCopyInto(out *SubnetRouterStatus) {
*out = *in
func (in Tags) DeepCopyInto(out *Tags) {
{
in := &in
*out = make(Tags, len(*in))
copy(*out, *in)
}
}
// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new SubnetRouterStatus.
func (in *SubnetRouterStatus) DeepCopy() *SubnetRouterStatus {
// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new Tags.
func (in Tags) DeepCopy() Tags {
if in == nil {
return nil
}
out := new(SubnetRouterStatus)
out := new(Tags)
in.DeepCopyInto(out)
return out
return *out
}