ipn/ipnlocal: move handling of expired nodes to LocalBackend

In order to be able to synthesize a new NetMap when a node expires, have
LocalBackend start a timer when receiving a new NetMap that fires
slightly after the next node expires. Additionally, move the logic that
updates expired nodes into LocalBackend so it runs on every netmap
(whether received from controlclient or self-triggered).

Updates #6932

Signed-off-by: Andrew Dunham <andrew@du.nham.ca>
Change-Id: I833390e16ad188983eac29eb34cc7574f555f2f3
pull/6969/head
Andrew Dunham 1 year ago
parent 6d85a94767
commit 3a018e51bb

@ -77,6 +77,7 @@ type Direct struct {
popBrowser func(url string) // or nil
c2nHandler http.Handler // or nil
onClientVersion func(*tailcfg.ClientVersion) // or nil
onControlTime func(time.Time) // or nil
dialPlan ControlDialPlanner // can be nil
@ -116,6 +117,7 @@ type Options struct {
LinkMonitor *monitor.Mon // optional link monitor
PopBrowserURL func(url string) // optional func to open browser
OnClientVersion func(*tailcfg.ClientVersion) // optional func to inform GUI of client version status
OnControlTime func(time.Time) // optional func to notify callers of new time from control
Dialer *tsdial.Dialer // non-nil
C2NHandler http.Handler // or nil
@ -244,6 +246,7 @@ func NewDirect(opts Options) (*Direct, error) {
pinger: opts.Pinger,
popBrowser: opts.PopBrowserURL,
onClientVersion: opts.OnClientVersion,
onControlTime: opts.OnControlTime,
c2nHandler: opts.C2NHandler,
dialer: opts.Dialer,
dialPlan: opts.DialPlan,
@ -1016,6 +1019,9 @@ func (c *Direct) sendMapRequest(ctx context.Context, maxPolls int, readOnly bool
}
if resp.ControlTime != nil && !resp.ControlTime.IsZero() {
c.logf.JSON(1, "controltime", resp.ControlTime.UTC())
if c.onControlTime != nil {
c.onControlTime(*resp.ControlTime)
}
}
if resp.KeepAlive {
vlogf("netmap: got keep-alive")

@ -9,7 +9,6 @@ import (
"log"
"net/netip"
"sort"
"time"
"tailscale.com/envknob"
"tailscale.com/tailcfg"
@ -53,12 +52,6 @@ type mapSession struct {
lastPopBrowserURL string
stickyDebug tailcfg.Debug // accumulated opt.Bool values
lastTKAInfo *tailcfg.TKAInfo
previouslyExpired map[tailcfg.StableNodeID]bool // to avoid log spam
// clockDelta stores the delta between the current time and the time
// received from control such that:
// time.Now().Add(clockDelta) == MapResponse.ControlTime
clockDelta time.Duration
// netMapBuilding is non-nil during a netmapForResponse call,
// containing the value to be returned, once fully populated.
@ -67,12 +60,11 @@ type mapSession struct {
func newMapSession(privateNodeKey key.NodePrivate) *mapSession {
ms := &mapSession{
privateNodeKey: privateNodeKey,
logf: logger.Discard,
vlogf: logger.Discard,
lastDNSConfig: new(tailcfg.DNSConfig),
lastUserProfile: map[tailcfg.UserID]tailcfg.UserProfile{},
previouslyExpired: map[tailcfg.StableNodeID]bool{},
privateNodeKey: privateNodeKey,
logf: logger.Discard,
vlogf: logger.Discard,
lastDNSConfig: new(tailcfg.DNSConfig),
lastUserProfile: map[tailcfg.UserID]tailcfg.UserProfile{},
}
return ms
}
@ -93,7 +85,6 @@ func (ms *mapSession) addUserProfile(userID tailcfg.UserID) {
// information from prior MapResponse values.
func (ms *mapSession) netmapForResponse(resp *tailcfg.MapResponse) *netmap.NetworkMap {
undeltaPeers(resp, ms.previousPeers)
ms.flagExpiredPeers(resp)
ms.previousPeers = cloneNodes(resp.Peers) // defensive/lazy clone, since this escapes to who knows where
for _, up := range resp.UserProfiles {
@ -352,83 +343,6 @@ func undeltaPeers(mapRes *tailcfg.MapResponse, prev []*tailcfg.Node) {
mapRes.PeersRemoved = nil
}
// For extra defense-in-depth, when we're testing expired nodes we check
// ControlTime against this 'epoch' (set to the approximate time that this code
// was written) such that if control (or Headscale, etc.) sends a ControlTime
// that's sufficiently far in the past, we can safely ignore it.
var flagExpiredPeersEpoch = time.Unix(1673373066, 0)
// If the offset between the current time and the time received from control is
// larger than this, we store an offset in our mapSession to adjust future
// clock timings.
const minClockDelta = 1 * time.Minute
// flagExpiredPeers updates mapRes.Peers, mutating all peers that have expired,
// taking into account any clock skew detected by using the ControlTime field
// in the MapResponse. We don't actually remove expired peers from the Peers
// array; instead, we clear some fields of the Node object, and set
// Node.Expired so other parts of the codebase can provide more clear error
// messages when attempting to e.g. ping an expired node.
//
// This is additionally a defense-in-depth against something going wrong with
// control such that we start seeing expired peers with a valid Endpoints or
// DERP field.
func (ms *mapSession) flagExpiredPeers(mapRes *tailcfg.MapResponse) {
localNow := clockNow()
// If we have a ControlTime field, update our delta.
if mapRes.ControlTime != nil && !mapRes.ControlTime.IsZero() {
delta := mapRes.ControlTime.Sub(localNow)
if delta.Abs() > minClockDelta {
ms.logf("[v1] netmap: flagExpiredPeers: setting clock delta to %v", delta)
ms.clockDelta = delta
} else {
ms.clockDelta = 0
}
}
// Adjust our current time by any saved delta to adjust for clock skew.
controlNow := localNow.Add(ms.clockDelta)
if controlNow.Before(flagExpiredPeersEpoch) {
ms.logf("netmap: flagExpiredPeers: [unexpected] delta-adjusted current time is before hardcoded epoch; skipping")
return
}
for _, peer := range mapRes.Peers {
// Nodes that don't expire have KeyExpiry set to the zero time;
// skip those and peers that are already marked as expired
// (e.g. from control).
if peer.KeyExpiry.IsZero() || peer.KeyExpiry.After(controlNow) {
delete(ms.previouslyExpired, peer.StableID)
continue
} else if peer.Expired {
continue
}
if !ms.previouslyExpired[peer.StableID] {
ms.logf("[v1] netmap: flagExpiredPeers: clearing expired peer %v", peer.StableID)
ms.previouslyExpired[peer.StableID] = true
}
// Actually mark the node as expired
peer.Expired = true
// Control clears the Endpoints and DERP fields of expired
// nodes; do so here as well. The Expired bool is the correct
// thing to set, but this replicates the previous behaviour.
//
// NOTE: this is insufficient to actually break connectivity,
// since we discover endpoints via DERP, and due to DERP return
// path optimization.
peer.Endpoints = nil
peer.DERP = ""
// Defense-in-depth: break the node's public key as well, in
// case something tries to communicate.
peer.Key = key.NodePublicWithBadOldPrefix(peer.Key)
}
}
// ptrCopy returns a pointer to a newly allocated shallow copy of *v.
func ptrCopy[T any](v *T) *T {
if v == nil {

@ -308,112 +308,6 @@ func TestUndeltaPeers(t *testing.T) {
}
}
func TestFlagExpiredPeers(t *testing.T) {
n := func(id tailcfg.NodeID, name string, expiry time.Time, mod ...func(*tailcfg.Node)) *tailcfg.Node {
n := &tailcfg.Node{ID: id, Name: name, KeyExpiry: expiry}
for _, f := range mod {
f(n)
}
return n
}
now := time.Unix(1673373129, 0)
oldClockNow := clockNow
clockNow = func() time.Time { return now }
t.Cleanup(func() { clockNow = oldClockNow })
timeInPast := now.Add(-1 * time.Hour)
timeInFuture := now.Add(1 * time.Hour)
timeBeforeEpoch := flagExpiredPeersEpoch.Add(-1 * time.Second)
if now.Before(timeBeforeEpoch) {
panic("current time in test cannot be before epoch")
}
var expiredKey key.NodePublic
if err := expiredKey.UnmarshalText([]byte("nodekey:6da774d5d7740000000000000000000000000000000000000000000000000000")); err != nil {
panic(err)
}
tests := []struct {
name string
mapRes *tailcfg.MapResponse
want []*tailcfg.Node
}{
{
name: "no_expiry",
mapRes: &tailcfg.MapResponse{
ControlTime: &now,
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInFuture),
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInFuture),
},
},
{
name: "expiry",
mapRes: &tailcfg.MapResponse{
ControlTime: &now,
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInPast),
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInPast, func(n *tailcfg.Node) {
n.Expired = true
n.Key = expiredKey
}),
},
},
{
name: "bad_ControlTime",
mapRes: &tailcfg.MapResponse{
// ControlTime here is intentionally before our hardcoded epoch
ControlTime: &timeBeforeEpoch,
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeBeforeEpoch.Add(-1*time.Hour)), // before ControlTime
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeBeforeEpoch.Add(-1*time.Hour)), // should have expired, but ControlTime is before epoch
},
},
{
name: "tagged_node",
mapRes: &tailcfg.MapResponse{
ControlTime: &now,
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", time.Time{}), // tagged node; zero expiry
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", time.Time{}), // not expired
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
ms := newTestMapSession(t)
ms.flagExpiredPeers(tt.mapRes)
if !reflect.DeepEqual(tt.mapRes.Peers, tt.want) {
t.Errorf("wrong results\n got: %s\nwant: %s", formatNodes(tt.mapRes.Peers), formatNodes(tt.want))
}
})
}
}
func formatNodes(nodes []*tailcfg.Node) string {
var sb strings.Builder
for i, n := range nodes {

@ -0,0 +1,122 @@
// Copyright (c) 2023 Tailscale Inc & AUTHORS All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package ipnlocal
import (
"time"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/types/key"
"tailscale.com/types/logger"
"tailscale.com/types/netmap"
)
// For extra defense-in-depth, when we're testing expired nodes we check
// ControlTime against this 'epoch' (set to the approximate time that this code
// was written) such that if control (or Headscale, etc.) sends a ControlTime
// that's sufficiently far in the past, we can safely ignore it.
var flagExpiredPeersEpoch = time.Unix(1673373066, 0)
// If the offset between the current time and the time received from control is
// larger than this, we store an offset in our expiryManager to adjust future
// clock timings.
const minClockDelta = 1 * time.Minute
// expiryManager tracks the state of expired nodes and the delta from the
// current clock time to the time returned from control, and allows mutating a
// netmap to mark peers as expired based on the current delta-adjusted time.
type expiryManager struct {
// previouslyExpired stores nodes that have already expired so we can
// only log on state transitions.
previouslyExpired map[tailcfg.StableNodeID]bool
// clockDelta stores the delta between the current time and the time
// received from control such that:
// time.Now().Add(clockDelta) == MapResponse.ControlTime
clockDelta syncs.AtomicValue[time.Duration]
logf logger.Logf
timeNow func() time.Time
}
func newExpiryManager(logf logger.Logf) *expiryManager {
return &expiryManager{
previouslyExpired: map[tailcfg.StableNodeID]bool{},
logf: logf,
timeNow: time.Now,
}
}
// onControlTime is called whenever we receive a new timestamp from the control
// server to store the delta.
func (em *expiryManager) onControlTime(t time.Time) {
localNow := em.timeNow()
delta := t.Sub(localNow)
if delta.Abs() > minClockDelta {
em.logf("[v1] netmap: flagExpiredPeers: setting clock delta to %v", delta)
em.clockDelta.Store(delta)
} else {
em.clockDelta.Store(0)
}
}
// flagExpiredPeers updates mapRes.Peers, mutating all peers that have expired,
// taking into account any clock skew detected by using the ControlTime field
// in the MapResponse. We don't actually remove expired peers from the Peers
// array; instead, we clear some fields of the Node object, and set
// Node.Expired so other parts of the codebase can provide more clear error
// messages when attempting to e.g. ping an expired node.
//
// This is additionally a defense-in-depth against something going wrong with
// control such that we start seeing expired peers with a valid Endpoints or
// DERP field.
//
// This function is safe to call concurrently with onControlTime but not
// concurrently with any other call to flagExpiredPeers.
func (em *expiryManager) flagExpiredPeers(netmap *netmap.NetworkMap) {
localNow := em.timeNow()
// Adjust our current time by any saved delta to adjust for clock skew.
controlNow := localNow.Add(em.clockDelta.Load())
if controlNow.Before(flagExpiredPeersEpoch) {
em.logf("netmap: flagExpiredPeers: [unexpected] delta-adjusted current time is before hardcoded epoch; skipping")
return
}
for _, peer := range netmap.Peers {
// Nodes that don't expire have KeyExpiry set to the zero time;
// skip those and peers that are already marked as expired
// (e.g. from control).
if peer.KeyExpiry.IsZero() || peer.KeyExpiry.After(controlNow) {
delete(em.previouslyExpired, peer.StableID)
continue
} else if peer.Expired {
continue
}
if !em.previouslyExpired[peer.StableID] {
em.logf("[v1] netmap: flagExpiredPeers: clearing expired peer %v", peer.StableID)
em.previouslyExpired[peer.StableID] = true
}
// Actually mark the node as expired
peer.Expired = true
// Control clears the Endpoints and DERP fields of expired
// nodes; do so here as well. The Expired bool is the correct
// thing to set, but this replicates the previous behaviour.
//
// NOTE: this is insufficient to actually break connectivity,
// since we discover endpoints via DERP, and due to DERP return
// path optimization.
peer.Endpoints = nil
peer.DERP = ""
// Defense-in-depth: break the node's public key as well, in
// case something tries to communicate.
peer.Key = key.NodePublicWithBadOldPrefix(peer.Key)
}
}

@ -0,0 +1,150 @@
// Copyright (c) 2023 Tailscale Inc & AUTHORS All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package ipnlocal
import (
"fmt"
"reflect"
"strings"
"testing"
"time"
"tailscale.com/tailcfg"
"tailscale.com/types/key"
"tailscale.com/types/netmap"
)
func TestFlagExpiredPeers(t *testing.T) {
n := func(id tailcfg.NodeID, name string, expiry time.Time, mod ...func(*tailcfg.Node)) *tailcfg.Node {
n := &tailcfg.Node{ID: id, Name: name, KeyExpiry: expiry}
for _, f := range mod {
f(n)
}
return n
}
now := time.Unix(1673373129, 0)
timeInPast := now.Add(-1 * time.Hour)
timeInFuture := now.Add(1 * time.Hour)
timeBeforeEpoch := flagExpiredPeersEpoch.Add(-1 * time.Second)
if now.Before(timeBeforeEpoch) {
panic("current time in test cannot be before epoch")
}
var expiredKey key.NodePublic
if err := expiredKey.UnmarshalText([]byte("nodekey:6da774d5d7740000000000000000000000000000000000000000000000000000")); err != nil {
panic(err)
}
tests := []struct {
name string
controlTime *time.Time
netmap *netmap.NetworkMap
want []*tailcfg.Node
}{
{
name: "no_expiry",
controlTime: &now,
netmap: &netmap.NetworkMap{
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInFuture),
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInFuture),
},
},
{
name: "expiry",
controlTime: &now,
netmap: &netmap.NetworkMap{
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInPast),
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeInPast, func(n *tailcfg.Node) {
n.Expired = true
n.Key = expiredKey
}),
},
},
{
name: "bad_ControlTime",
// controlTime here is intentionally before our hardcoded epoch
controlTime: &timeBeforeEpoch,
netmap: &netmap.NetworkMap{
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeBeforeEpoch.Add(-1*time.Hour)), // before ControlTime
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", timeBeforeEpoch.Add(-1*time.Hour)), // should have expired, but ControlTime is before epoch
},
},
{
name: "tagged_node",
controlTime: &now,
netmap: &netmap.NetworkMap{
Peers: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", time.Time{}), // tagged node; zero expiry
},
},
want: []*tailcfg.Node{
n(1, "foo", timeInFuture),
n(2, "bar", time.Time{}), // not expired
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
em := newExpiryManager(t.Logf)
em.timeNow = func() time.Time { return now }
if tt.controlTime != nil {
em.onControlTime(*tt.controlTime)
}
em.flagExpiredPeers(tt.netmap)
if !reflect.DeepEqual(tt.netmap.Peers, tt.want) {
t.Errorf("wrong results\n got: %s\nwant: %s", formatNodes(tt.netmap.Peers), formatNodes(tt.want))
}
})
}
}
func formatNodes(nodes []*tailcfg.Node) string {
var sb strings.Builder
for i, n := range nodes {
if i > 0 {
sb.WriteString(", ")
}
fmt.Fprintf(&sb, "(%d, %q", n.ID, n.Name)
if n.Online != nil {
fmt.Fprintf(&sb, ", online=%v", *n.Online)
}
if n.LastSeen != nil {
fmt.Fprintf(&sb, ", lastSeen=%v", n.LastSeen.Unix())
}
if n.Key != (key.NodePublic{}) {
fmt.Fprintf(&sb, ", key=%v", n.Key.String())
}
if n.Expired {
fmt.Fprintf(&sb, ", expired=true")
}
sb.WriteString(")")
}
return sb.String()
}

@ -139,8 +139,9 @@ type LocalBackend struct {
portpollOnce sync.Once // guards starting readPoller
gotPortPollRes chan struct{} // closed upon first readPoller result
newDecompressor func() (controlclient.Decompressor, error)
varRoot string // or empty if SetVarRoot never called
logFlushFunc func() // or nil if SetLogFlusher wasn't called
varRoot string // or empty if SetVarRoot never called
logFlushFunc func() // or nil if SetLogFlusher wasn't called
em *expiryManager // non-nil
sshAtomicBool atomic.Bool
shutdownCalled bool // if Shutdown has been called
@ -151,6 +152,7 @@ type LocalBackend struct {
filterAtomic atomic.Pointer[filter.Filter]
containsViaIPFuncAtomic syncs.AtomicValue[func(netip.Addr) bool]
shouldInterceptTCPPortAtomic syncs.AtomicValue[func(uint16) bool]
numClientStatusCalls atomic.Uint32
// The mutex protects the following elements.
mu sync.Mutex
@ -170,6 +172,7 @@ type LocalBackend struct {
hostinfo *tailcfg.Hostinfo
// netMap is not mutated in-place once set.
netMap *netmap.NetworkMap
nmExpiryTimer *time.Timer // for updating netMap on node expiry; can be nil
nodeByAddr map[netip.Addr]*tailcfg.Node
activeLogin string // last logged LoginName from netMap
engineStatus ipn.EngineStatus
@ -275,6 +278,7 @@ func NewLocalBackend(logf logger.Logf, logid string, store ipn.StateStore, state
backendLogID: logid,
state: ipn.NoState,
portpoll: portpoll,
em: newExpiryManager(logf),
gotPortPollRes: make(chan struct{}),
loginFlags: loginFlags,
}
@ -805,7 +809,65 @@ func (b *LocalBackend) setClientStatus(st controlclient.Status) {
return
}
// Track the number of calls
currCall := b.numClientStatusCalls.Add(1)
b.mu.Lock()
// Handle node expiry in the netmap
if st.NetMap != nil {
b.em.flagExpiredPeers(st.NetMap)
// Always stop the existing netmap timer if we have a netmap;
// it's possible that we have no nodes expiring, so we should
// always cancel the timer and then possibly restart it below.
if b.nmExpiryTimer != nil {
// Ignore if we can't stop; the atomic check in the
// AfterFunc (below) will skip running.
b.nmExpiryTimer.Stop()
// Nil so we don't attempt to stop on the next netmap
b.nmExpiryTimer = nil
}
now := time.Now()
// Figure out when the next node in the netmap is expiring so we can
// start a timer to reconfigure at that point.
var nextExpiry time.Time // zero if none
for _, peer := range st.NetMap.Peers {
if peer.KeyExpiry.IsZero() {
continue // tagged node
} else if peer.Expired {
// Peer already expired; Expired is set by the
// flagExpiredPeers function, above.
continue
}
if nextExpiry.IsZero() || peer.KeyExpiry.Before(nextExpiry) {
nextExpiry = peer.KeyExpiry
}
}
if !nextExpiry.IsZero() {
tmrDuration := nextExpiry.Sub(now) + 10*time.Second
b.nmExpiryTimer = time.AfterFunc(tmrDuration, func() {
// Skip if the world has moved on past the
// saved call (e.g. if we race stopping this
// timer).
if b.numClientStatusCalls.Load() != currCall {
return
}
b.logf("setClientStatus: netmap expiry timer triggered after %v", tmrDuration)
// Call ourselves with the current status again; the logic in
// setClientStatus will take care of updating the expired field
// of peers in the netmap.
b.setClientStatus(st)
})
}
}
wasBlocked := b.blocked
keyExpiryExtended := false
if st.NetMap != nil {
@ -1314,6 +1376,7 @@ func (b *LocalBackend) Start(opts ipn.Options) error {
Pinger: b,
PopBrowserURL: b.tellClientToBrowseToURL,
OnClientVersion: b.onClientVersion,
OnControlTime: b.em.onControlTime,
Dialer: b.Dialer(),
Status: b.setClientStatus,
C2NHandler: http.HandlerFunc(b.handleC2N),

Loading…
Cancel
Save