mirror of
https://github.com/tailscale/tailscale.git
synced 2024-11-29 04:55:31 +00:00
control/controlclient, types/netmap: start plumbing delta netmap updates
Currently only the top four most popular changes: endpoints, DERP home, online, and LastSeen. Updates #1909 Change-Id: I03152da176b2b95232b56acabfb55dcdfaa16b79 Signed-off-by: Brad Fitzpatrick <bradfitz@tailscale.com>
This commit is contained in:
parent
c0ade132e6
commit
3af051ea27
@ -432,6 +432,8 @@ type mapRoutineState struct {
|
||||
bo *backoff.Backoff
|
||||
}
|
||||
|
||||
var _ NetmapDeltaUpdater = mapRoutineState{}
|
||||
|
||||
func (mrs mapRoutineState) UpdateFullNetmap(nm *netmap.NetworkMap) {
|
||||
c := mrs.c
|
||||
|
||||
@ -453,6 +455,28 @@ func (mrs mapRoutineState) UpdateFullNetmap(nm *netmap.NetworkMap) {
|
||||
mrs.bo.BackOff(ctx, nil)
|
||||
}
|
||||
|
||||
func (mrs mapRoutineState) UpdateNetmapDelta(muts []netmap.NodeMutation) bool {
|
||||
c := mrs.c
|
||||
|
||||
c.mu.Lock()
|
||||
goodState := c.loggedIn && c.inMapPoll
|
||||
ndu, canDelta := c.observer.(NetmapDeltaUpdater)
|
||||
c.mu.Unlock()
|
||||
|
||||
if !goodState || !canDelta {
|
||||
return false
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(c.mapCtx, 2*time.Second)
|
||||
defer cancel()
|
||||
|
||||
var ok bool
|
||||
err := c.observerQueue.RunSync(ctx, func() {
|
||||
ok = ndu.UpdateNetmapDelta(muts)
|
||||
})
|
||||
return err == nil && ok
|
||||
}
|
||||
|
||||
// mapRoutine is responsible for keeping a read-only streaming connection to the
|
||||
// control server, and keeping the netmap up to date.
|
||||
func (c *Auto) mapRoutine() {
|
||||
@ -752,6 +776,27 @@ func (q *execQueue) Add(f func()) {
|
||||
}
|
||||
}
|
||||
|
||||
// RunSync waits for the queue to be drained and then synchronously runs f.
|
||||
// It returns an error if the queue is closed before f is run or ctx expires.
|
||||
func (q *execQueue) RunSync(ctx context.Context, f func()) error {
|
||||
for {
|
||||
if err := q.wait(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
q.mu.Lock()
|
||||
if q.inFlight {
|
||||
q.mu.Unlock()
|
||||
continue
|
||||
}
|
||||
defer q.mu.Unlock()
|
||||
if q.closed {
|
||||
return errors.New("closed")
|
||||
}
|
||||
f()
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (q *execQueue) run(f func()) {
|
||||
f()
|
||||
|
||||
|
@ -194,6 +194,19 @@ type NetmapUpdater interface {
|
||||
// the diff themselves between the previous full & next full network maps.
|
||||
}
|
||||
|
||||
// NetmapDeltaUpdater is an optional interface that can be implemented by
|
||||
// NetmapUpdater implementations to receive delta updates from the controlclient
|
||||
// rather than just full updates.
|
||||
type NetmapDeltaUpdater interface {
|
||||
// UpdateNetmapDelta is called with discrete changes to the network map.
|
||||
//
|
||||
// The ok result is whether the implementation was able to apply the
|
||||
// mutations. It might return false if its internal state doesn't
|
||||
// support applying them or a NetmapUpdater it's wrapping doesn't
|
||||
// implement the NetmapDeltaUpdater optional method.
|
||||
UpdateNetmapDelta([]netmap.NodeMutation) (ok bool)
|
||||
}
|
||||
|
||||
// NewDirect returns a new Direct client.
|
||||
func NewDirect(opts Options) (*Direct, error) {
|
||||
if opts.ServerURL == "" {
|
||||
@ -1301,6 +1314,7 @@ func (ms *mapSession) setControlKnobsFromNodeAttrs(selfNodeAttrs []string) {
|
||||
disableDRPO bool
|
||||
disableUPnP bool
|
||||
randomizeClientPort bool
|
||||
disableDeltaUpdates bool
|
||||
oneCGNAT opt.Bool
|
||||
forceBackgroundSTUN bool
|
||||
)
|
||||
@ -1320,6 +1334,8 @@ func (ms *mapSession) setControlKnobsFromNodeAttrs(selfNodeAttrs []string) {
|
||||
oneCGNAT.Set(false)
|
||||
case tailcfg.NodeAttrDebugForceBackgroundSTUN:
|
||||
forceBackgroundSTUN = true
|
||||
case tailcfg.NodeAttrDisableDeltaUpdates:
|
||||
disableDeltaUpdates = true
|
||||
}
|
||||
}
|
||||
k.KeepFullWGConfig.Store(keepFullWG)
|
||||
@ -1328,6 +1344,7 @@ func (ms *mapSession) setControlKnobsFromNodeAttrs(selfNodeAttrs []string) {
|
||||
k.RandomizeClientPort.Store(randomizeClientPort)
|
||||
k.OneCGNAT.Store(oneCGNAT)
|
||||
k.ForceBackgroundSTUN.Store(forceBackgroundSTUN)
|
||||
k.DisableDeltaUpdates.Store(disableDeltaUpdates)
|
||||
}
|
||||
|
||||
// ipForwardingBroken reports whether the system's IP forwarding is disabled
|
||||
|
@ -14,6 +14,7 @@
|
||||
"sort"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"tailscale.com/control/controlknobs"
|
||||
"tailscale.com/envknob"
|
||||
@ -39,7 +40,7 @@
|
||||
// one MapRequest).
|
||||
type mapSession struct {
|
||||
// Immutable fields.
|
||||
nu NetmapUpdater // called on changes (in addition to the optional hooks below)
|
||||
netmapUpdater NetmapUpdater // called on changes (in addition to the optional hooks below)
|
||||
controlKnobs *controlknobs.Knobs // or nil
|
||||
privateNodeKey key.NodePrivate
|
||||
publicNodeKey key.NodePublic
|
||||
@ -98,7 +99,7 @@ type mapSession struct {
|
||||
// It must have its Close method called to release resources.
|
||||
func newMapSession(privateNodeKey key.NodePrivate, nu NetmapUpdater, controlKnobs *controlknobs.Knobs) *mapSession {
|
||||
ms := &mapSession{
|
||||
nu: nu,
|
||||
netmapUpdater: nu,
|
||||
controlKnobs: controlKnobs,
|
||||
privateNodeKey: privateNodeKey,
|
||||
publicNodeKey: privateNodeKey.Public(),
|
||||
@ -197,8 +198,16 @@ func (ms *mapSession) HandleNonKeepAliveMapResponse(ctx context.Context, resp *t
|
||||
|
||||
ms.updateStateFromResponse(resp)
|
||||
|
||||
nm := ms.netmap()
|
||||
if ms.tryHandleIncrementally(resp) {
|
||||
ms.onConciseNetMapSummary(ms.lastNetmapSummary) // every 5s log
|
||||
return nil
|
||||
}
|
||||
|
||||
// We have to rebuild the whole netmap (lots of garbage & work downstream of
|
||||
// our UpdateFullNetmap call). This is the part we tried to avoid but
|
||||
// some field mutations (especially rare ones) aren't yet handled.
|
||||
|
||||
nm := ms.netmap()
|
||||
ms.lastNetmapSummary = nm.VeryConcise()
|
||||
ms.onConciseNetMapSummary(ms.lastNetmapSummary)
|
||||
|
||||
@ -207,10 +216,25 @@ func (ms *mapSession) HandleNonKeepAliveMapResponse(ctx context.Context, resp *t
|
||||
ms.onSelfNodeChanged(nm)
|
||||
}
|
||||
|
||||
ms.nu.UpdateFullNetmap(nm)
|
||||
ms.netmapUpdater.UpdateFullNetmap(nm)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *mapSession) tryHandleIncrementally(res *tailcfg.MapResponse) bool {
|
||||
if ms.controlKnobs != nil && ms.controlKnobs.DisableDeltaUpdates.Load() {
|
||||
return false
|
||||
}
|
||||
nud, ok := ms.netmapUpdater.(NetmapDeltaUpdater)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
mutations, ok := netmap.MutationsFromMapResponse(res, time.Now())
|
||||
if ok && len(mutations) > 0 {
|
||||
return nud.UpdateNetmapDelta(mutations)
|
||||
}
|
||||
return ok
|
||||
}
|
||||
|
||||
// updateStats are some stats from updateStateFromResponse, primarily for
|
||||
// testing. It's meant to be cheap enough to always compute, though. It doesn't
|
||||
// allocate.
|
||||
|
@ -38,6 +38,11 @@ type Knobs struct {
|
||||
// ForceBackgroundSTUN forces netcheck STUN queries to keep
|
||||
// running in magicsock, even when idle.
|
||||
ForceBackgroundSTUN atomic.Bool
|
||||
|
||||
// DisableDeltaUpdates is whether the node should not process
|
||||
// incremental (delta) netmap updates and should treat all netmap
|
||||
// changes as "full" ones as tailscaled did in 1.48.x and earlier.
|
||||
DisableDeltaUpdates atomic.Bool
|
||||
}
|
||||
|
||||
// AsDebugJSON returns k as something that can be marshalled with json.Marshal
|
||||
@ -53,5 +58,6 @@ func (k *Knobs) AsDebugJSON() map[string]any {
|
||||
"RandomizeClientPort": k.RandomizeClientPort.Load(),
|
||||
"OneCGNAT": k.OneCGNAT.Load(),
|
||||
"ForceBackgroundSTUN": k.ForceBackgroundSTUN.Load(),
|
||||
"DisableDeltaUpdates": k.DisableDeltaUpdates.Load(),
|
||||
}
|
||||
}
|
||||
|
21
control/controlknobs/controlknobs_test.go
Normal file
21
control/controlknobs/controlknobs_test.go
Normal file
@ -0,0 +1,21 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package controlknobs
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestAsDebugJSON(t *testing.T) {
|
||||
var nilPtr *Knobs
|
||||
if got := nilPtr.AsDebugJSON(); got != nil {
|
||||
t.Errorf("AsDebugJSON(nil) = %v; want nil", got)
|
||||
}
|
||||
k := new(Knobs)
|
||||
got := k.AsDebugJSON()
|
||||
if want := reflect.TypeOf(Knobs{}).NumField(); len(got) != want {
|
||||
t.Errorf("AsDebugJSON map has %d fields; want %v", len(got), want)
|
||||
}
|
||||
}
|
@ -203,9 +203,8 @@ type LocalBackend struct {
|
||||
capFileSharing bool // whether netMap contains the file sharing capability
|
||||
capTailnetLock bool // whether netMap contains the tailnet lock capability
|
||||
// hostinfo is mutated in-place while mu is held.
|
||||
hostinfo *tailcfg.Hostinfo
|
||||
// netMap is not mutated in-place once set.
|
||||
netMap *netmap.NetworkMap
|
||||
hostinfo *tailcfg.Hostinfo
|
||||
netMap *netmap.NetworkMap // not mutated in place once set (except for Peers slice)
|
||||
nmExpiryTimer tstime.TimerController // for updating netMap on node expiry; can be nil
|
||||
nodeByAddr map[netip.Addr]tailcfg.NodeView
|
||||
activeLogin string // last logged LoginName from netMap
|
||||
@ -1121,6 +1120,56 @@ func (b *LocalBackend) SetControlClientStatus(c controlclient.Client, st control
|
||||
b.authReconfig()
|
||||
}
|
||||
|
||||
var _ controlclient.NetmapDeltaUpdater = (*LocalBackend)(nil)
|
||||
|
||||
// UpdateNetmapDelta implements controlclient.NetmapDeltaUpdater.
|
||||
func (b *LocalBackend) UpdateNetmapDelta(muts []netmap.NodeMutation) (handled bool) {
|
||||
mc, err := b.magicConn()
|
||||
if err != nil {
|
||||
panic(err) // shouldn't happen
|
||||
}
|
||||
if !mc.UpdateNetmapDelta(muts) {
|
||||
return false
|
||||
}
|
||||
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return b.updateNetmapDeltaLocked(muts)
|
||||
}
|
||||
|
||||
func (b *LocalBackend) updateNetmapDeltaLocked(muts []netmap.NodeMutation) (handled bool) {
|
||||
if b.netMap == nil {
|
||||
return false
|
||||
}
|
||||
peers := b.netMap.Peers
|
||||
|
||||
for _, m := range muts {
|
||||
// LocalBackend only cares about some types of mutations.
|
||||
// (magicsock cares about different ones.)
|
||||
switch m.(type) {
|
||||
case netmap.NodeMutationOnline, netmap.NodeMutationLastSeen:
|
||||
default:
|
||||
continue
|
||||
}
|
||||
|
||||
nodeID := m.NodeIDBeingMutated()
|
||||
idx := b.netMap.PeerIndexByNodeID(nodeID)
|
||||
if idx == -1 {
|
||||
continue
|
||||
}
|
||||
mut := peers[idx].AsStruct()
|
||||
|
||||
switch m := m.(type) {
|
||||
case netmap.NodeMutationOnline:
|
||||
mut.Online = ptr.To(m.Online)
|
||||
case netmap.NodeMutationLastSeen:
|
||||
mut.LastSeen = ptr.To(m.LastSeen)
|
||||
}
|
||||
peers[idx] = mut.View()
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// setExitNodeID updates prefs to reference an exit node by ID, rather
|
||||
// than by IP. It returns whether prefs was mutated.
|
||||
func setExitNodeID(prefs *ipn.Prefs, nm *netmap.NetworkMap) (prefsChanged bool) {
|
||||
|
@ -26,6 +26,7 @@
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/logid"
|
||||
"tailscale.com/types/netmap"
|
||||
"tailscale.com/types/ptr"
|
||||
"tailscale.com/util/set"
|
||||
"tailscale.com/wgengine"
|
||||
"tailscale.com/wgengine/filter"
|
||||
@ -879,3 +880,75 @@ func TestWatchNotificationsCallbacks(t *testing.T) {
|
||||
t.Fatalf("unexpected number of watchers in new LocalBackend, want: 0 got: %v", len(b.notifyWatchers))
|
||||
}
|
||||
}
|
||||
|
||||
// tests LocalBackend.updateNetmapDeltaLocked
|
||||
func TestUpdateNetmapDelta(t *testing.T) {
|
||||
var b LocalBackend
|
||||
if b.updateNetmapDeltaLocked(nil) {
|
||||
t.Errorf("updateNetmapDeltaLocked() = true, want false with nil netmap")
|
||||
}
|
||||
|
||||
b.netMap = &netmap.NetworkMap{}
|
||||
for i := 0; i < 5; i++ {
|
||||
b.netMap.Peers = append(b.netMap.Peers, (&tailcfg.Node{ID: (tailcfg.NodeID(i) + 1)}).View())
|
||||
}
|
||||
|
||||
someTime := time.Unix(123, 0)
|
||||
muts, ok := netmap.MutationsFromMapResponse(&tailcfg.MapResponse{
|
||||
PeersChangedPatch: []*tailcfg.PeerChange{
|
||||
{
|
||||
NodeID: 1,
|
||||
DERPRegion: 1,
|
||||
},
|
||||
{
|
||||
NodeID: 2,
|
||||
Online: ptr.To(true),
|
||||
},
|
||||
{
|
||||
NodeID: 3,
|
||||
Online: ptr.To(false),
|
||||
},
|
||||
{
|
||||
NodeID: 4,
|
||||
LastSeen: ptr.To(someTime),
|
||||
},
|
||||
},
|
||||
}, someTime)
|
||||
if !ok {
|
||||
t.Fatal("netmap.MutationsFromMapResponse failed")
|
||||
}
|
||||
|
||||
if !b.updateNetmapDeltaLocked(muts) {
|
||||
t.Fatalf("updateNetmapDeltaLocked() = false, want true with new netmap")
|
||||
}
|
||||
|
||||
wants := []*tailcfg.Node{
|
||||
{
|
||||
ID: 1,
|
||||
DERP: "", // unmodified by the delta
|
||||
},
|
||||
{
|
||||
ID: 2,
|
||||
Online: ptr.To(true),
|
||||
},
|
||||
{
|
||||
ID: 3,
|
||||
Online: ptr.To(false),
|
||||
},
|
||||
{
|
||||
ID: 4,
|
||||
LastSeen: ptr.To(someTime),
|
||||
},
|
||||
}
|
||||
for _, want := range wants {
|
||||
idx := b.netMap.PeerIndexByNodeID(want.ID)
|
||||
if idx == -1 {
|
||||
t.Errorf("ID %v not found in netmap", want.ID)
|
||||
continue
|
||||
}
|
||||
got := b.netMap.Peers[idx].AsStruct()
|
||||
if !reflect.DeepEqual(got, want) {
|
||||
t.Errorf("netmap.Peer %v wrong.\n got: %v\nwant: %v", want.ID, logger.AsJSON(got), logger.AsJSON(want))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1995,6 +1995,11 @@ type Oauth2Token struct {
|
||||
// new attempts at UPnP connections.
|
||||
NodeAttrDisableUPnP = "debug-disable-upnp"
|
||||
|
||||
// NodeAttrDisableDeltaUpdates makes the client not process updates via the
|
||||
// delta update mechanism and should instead treat all netmap changes as
|
||||
// "full" ones as tailscaled did in 1.48.x and earlier.
|
||||
NodeAttrDisableDeltaUpdates = "disable-delta-updates"
|
||||
|
||||
// NodeAttrRandomizeClientPort makes magicsock UDP bind to
|
||||
// :0 to get a random local port, ignoring any configured
|
||||
// fixed port.
|
||||
|
@ -8,6 +8,7 @@
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"net/netip"
|
||||
"sort"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
@ -15,6 +16,7 @@
|
||||
"tailscale.com/tka"
|
||||
"tailscale.com/types/key"
|
||||
"tailscale.com/types/views"
|
||||
"tailscale.com/util/cmpx"
|
||||
"tailscale.com/wgengine/filter"
|
||||
)
|
||||
|
||||
@ -124,6 +126,23 @@ func (nm *NetworkMap) PeerByTailscaleIP(ip netip.Addr) (peer tailcfg.NodeView, o
|
||||
return tailcfg.NodeView{}, false
|
||||
}
|
||||
|
||||
// PeerIndexByNodeID returns the index of the peer with the given nodeID
|
||||
// in nm.Peers, or -1 if nm is nil or not found.
|
||||
//
|
||||
// It assumes nm.Peers is sorted by Node.ID.
|
||||
func (nm *NetworkMap) PeerIndexByNodeID(nodeID tailcfg.NodeID) int {
|
||||
if nm == nil {
|
||||
return -1
|
||||
}
|
||||
idx, ok := sort.Find(len(nm.Peers), func(i int) int {
|
||||
return cmpx.Compare(nodeID, nm.Peers[i].ID())
|
||||
})
|
||||
if !ok {
|
||||
return -1
|
||||
}
|
||||
return idx
|
||||
}
|
||||
|
||||
// MagicDNSSuffix returns the domain's MagicDNS suffix (even if MagicDNS isn't
|
||||
// necessarily in use) of the provided Node.Name value.
|
||||
//
|
||||
|
@ -280,3 +280,31 @@ func TestConciseDiffFrom(t *testing.T) {
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestPeerIndexByNodeID(t *testing.T) {
|
||||
var nilPtr *NetworkMap
|
||||
if nilPtr.PeerIndexByNodeID(123) != -1 {
|
||||
t.Errorf("nil PeerIndexByNodeID should return -1")
|
||||
}
|
||||
var nm NetworkMap
|
||||
const min = 2
|
||||
const max = 10000
|
||||
const hole = max / 2
|
||||
for nid := tailcfg.NodeID(2); nid <= max; nid++ {
|
||||
if nid == hole {
|
||||
continue
|
||||
}
|
||||
nm.Peers = append(nm.Peers, (&tailcfg.Node{ID: nid}).View())
|
||||
}
|
||||
for want, nv := range nm.Peers {
|
||||
got := nm.PeerIndexByNodeID(nv.ID())
|
||||
if got != want {
|
||||
t.Errorf("PeerIndexByNodeID(%v) = %v; want %v", nv.ID(), got, want)
|
||||
}
|
||||
}
|
||||
for _, miss := range []tailcfg.NodeID{min - 1, hole, max + 1} {
|
||||
if got := nm.PeerIndexByNodeID(miss); got != -1 {
|
||||
t.Errorf("PeerIndexByNodeID(%v) = %v; want -1", miss, got)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
168
types/netmap/nodemut.go
Normal file
168
types/netmap/nodemut.go
Normal file
@ -0,0 +1,168 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package netmap
|
||||
|
||||
import (
|
||||
"net/netip"
|
||||
"reflect"
|
||||
"slices"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/util/cmpx"
|
||||
)
|
||||
|
||||
// NodeMutation is the common interface for types that describe
|
||||
// the change of a node's state.
|
||||
type NodeMutation interface {
|
||||
NodeIDBeingMutated() tailcfg.NodeID
|
||||
}
|
||||
|
||||
type mutatingNodeID tailcfg.NodeID
|
||||
|
||||
func (m mutatingNodeID) NodeIDBeingMutated() tailcfg.NodeID { return tailcfg.NodeID(m) }
|
||||
|
||||
// NodeMutationDERPHome is a NodeMutation that says a node
|
||||
// has changed its DERP home region.
|
||||
type NodeMutationDERPHome struct {
|
||||
mutatingNodeID
|
||||
DERPRegion int
|
||||
}
|
||||
|
||||
// NodeMutation is a NodeMutation that says a node's endpoints have changed.
|
||||
type NodeMutationEndpoints struct {
|
||||
mutatingNodeID
|
||||
Endpoints []netip.AddrPort
|
||||
}
|
||||
|
||||
// NodeMutationOnline is a NodeMutation that says a node is now online or
|
||||
// offline.
|
||||
type NodeMutationOnline struct {
|
||||
mutatingNodeID
|
||||
Online bool
|
||||
}
|
||||
|
||||
// NodeMutationLastSeen is a NodeMutation that says a node's LastSeen
|
||||
// value should be set to the current time.
|
||||
type NodeMutationLastSeen struct {
|
||||
mutatingNodeID
|
||||
LastSeen time.Time
|
||||
}
|
||||
|
||||
var peerChangeFields = sync.OnceValue(func() []reflect.StructField {
|
||||
var fields []reflect.StructField
|
||||
rt := reflect.TypeOf((*tailcfg.PeerChange)(nil)).Elem()
|
||||
for i := 0; i < rt.NumField(); i++ {
|
||||
fields = append(fields, rt.Field(i))
|
||||
}
|
||||
return fields
|
||||
})
|
||||
|
||||
// NodeMutationsFromPatch returns the NodeMutations that
|
||||
// p describes. If p describes something not yet supported
|
||||
// by a specific NodeMutation type, it returns (nil, false).
|
||||
func NodeMutationsFromPatch(p *tailcfg.PeerChange) (_ []NodeMutation, ok bool) {
|
||||
if p == nil || p.NodeID == 0 {
|
||||
return nil, false
|
||||
}
|
||||
var ret []NodeMutation
|
||||
rv := reflect.ValueOf(p).Elem()
|
||||
for i, sf := range peerChangeFields() {
|
||||
if rv.Field(i).IsZero() {
|
||||
continue
|
||||
}
|
||||
switch sf.Name {
|
||||
default:
|
||||
// Unhandled field.
|
||||
return nil, false
|
||||
case "NodeID":
|
||||
continue
|
||||
case "DERPRegion":
|
||||
ret = append(ret, NodeMutationDERPHome{mutatingNodeID(p.NodeID), p.DERPRegion})
|
||||
case "Endpoints":
|
||||
eps := make([]netip.AddrPort, len(p.Endpoints))
|
||||
for i, epStr := range p.Endpoints {
|
||||
var err error
|
||||
eps[i], err = netip.ParseAddrPort(epStr)
|
||||
if err != nil {
|
||||
return nil, false
|
||||
}
|
||||
}
|
||||
ret = append(ret, NodeMutationEndpoints{mutatingNodeID(p.NodeID), eps})
|
||||
case "Online":
|
||||
ret = append(ret, NodeMutationOnline{mutatingNodeID(p.NodeID), *p.Online})
|
||||
case "LastSeen":
|
||||
ret = append(ret, NodeMutationLastSeen{mutatingNodeID(p.NodeID), *p.LastSeen})
|
||||
}
|
||||
}
|
||||
return ret, true
|
||||
}
|
||||
|
||||
// MutationsFromMapResponse returns all the discrete node mutations described
|
||||
// by res. It returns ok=false if res contains any non-patch field as defined
|
||||
// by mapResponseContainsNonPatchFields.
|
||||
func MutationsFromMapResponse(res *tailcfg.MapResponse, now time.Time) (ret []NodeMutation, ok bool) {
|
||||
if now.IsZero() {
|
||||
now = time.Now()
|
||||
}
|
||||
if mapResponseContainsNonPatchFields(res) {
|
||||
return nil, false
|
||||
}
|
||||
// All that remains is PeersChangedPatch, OnlineChange, and LastSeenChange.
|
||||
|
||||
for _, p := range res.PeersChangedPatch {
|
||||
deltas, ok := NodeMutationsFromPatch(p)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
ret = append(ret, deltas...)
|
||||
}
|
||||
for nid, v := range res.OnlineChange {
|
||||
ret = append(ret, NodeMutationOnline{mutatingNodeID(nid), v})
|
||||
}
|
||||
for nid, v := range res.PeerSeenChange {
|
||||
if v {
|
||||
ret = append(ret, NodeMutationLastSeen{mutatingNodeID(nid), now})
|
||||
}
|
||||
}
|
||||
slices.SortStableFunc(ret, func(a, b NodeMutation) int {
|
||||
return cmpx.Compare(a.NodeIDBeingMutated(), b.NodeIDBeingMutated())
|
||||
})
|
||||
return ret, true
|
||||
}
|
||||
|
||||
// mapResponseContainsNonPatchFields reports whether res contains only "patch"
|
||||
// fields set (PeersChangedPatch primarily, but also including the legacy
|
||||
// PeerSeenChange and OnlineChange fields).
|
||||
//
|
||||
// It ignores any of the meta fields that are handled by PollNetMap before the
|
||||
// peer change handling gets involved.
|
||||
//
|
||||
// The purpose of this function is to ask whether this is a tricky enough
|
||||
// MapResponse to warrant a full netmap update. When this returns false, it
|
||||
// means the response can be handled incrementally, patching up the local state.
|
||||
func mapResponseContainsNonPatchFields(res *tailcfg.MapResponse) bool {
|
||||
return res.Node != nil ||
|
||||
res.DERPMap != nil ||
|
||||
res.DNSConfig != nil ||
|
||||
res.Domain != "" ||
|
||||
res.CollectServices != "" ||
|
||||
res.PacketFilter != nil ||
|
||||
res.UserProfiles != nil ||
|
||||
res.Health != nil ||
|
||||
res.SSHPolicy != nil ||
|
||||
res.TKAInfo != nil ||
|
||||
res.DomainDataPlaneAuditLogID != "" ||
|
||||
res.Debug != nil ||
|
||||
res.ControlDialPlan != nil ||
|
||||
res.ClientVersion != nil ||
|
||||
res.Peers != nil ||
|
||||
res.PeersRemoved != nil ||
|
||||
// PeersChanged is too coarse to be considered a patch. Also, we convert
|
||||
// PeersChanged to PeersChangedPatch in patchifyPeersChanged before this
|
||||
// function is called, so it should never be set anyway. But for
|
||||
// completedness, and for tests, check it too:
|
||||
res.PeersChanged != nil
|
||||
}
|
199
types/netmap/nodemut_test.go
Normal file
199
types/netmap/nodemut_test.go
Normal file
@ -0,0 +1,199 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package netmap
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/netip"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/ptr"
|
||||
)
|
||||
|
||||
// tests mapResponseContainsNonPatchFields
|
||||
func TestMapResponseContainsNonPatchFields(t *testing.T) {
|
||||
|
||||
// reflectNonzero returns a non-zero value of the given type.
|
||||
reflectNonzero := func(t reflect.Type) reflect.Value {
|
||||
|
||||
switch t.Kind() {
|
||||
case reflect.Bool:
|
||||
return reflect.ValueOf(true)
|
||||
case reflect.String:
|
||||
return reflect.ValueOf("foo").Convert(t)
|
||||
case reflect.Int64:
|
||||
return reflect.ValueOf(int64(1))
|
||||
case reflect.Slice:
|
||||
return reflect.MakeSlice(t, 1, 1)
|
||||
case reflect.Ptr:
|
||||
return reflect.New(t.Elem())
|
||||
case reflect.Map:
|
||||
return reflect.MakeMap(t)
|
||||
}
|
||||
panic(fmt.Sprintf("unhandled %v", t))
|
||||
}
|
||||
|
||||
rt := reflect.TypeOf(tailcfg.MapResponse{})
|
||||
for i := 0; i < rt.NumField(); i++ {
|
||||
f := rt.Field(i)
|
||||
|
||||
var want bool
|
||||
switch f.Name {
|
||||
case "MapSessionHandle", "Seq", "KeepAlive", "PingRequest", "PopBrowserURL", "ControlTime":
|
||||
// There are meta fields that apply to all MapResponse values.
|
||||
// They should be ignored.
|
||||
want = false
|
||||
case "PeersChangedPatch", "PeerSeenChange", "OnlineChange":
|
||||
// The actual three delta fields we care about handling.
|
||||
want = false
|
||||
default:
|
||||
// Everything else should be conseratively handled as a
|
||||
// non-delta field. We want it to return true so if
|
||||
// the field is not listed in the function being tested,
|
||||
// it'll return false and we'll fail this test.
|
||||
// This makes sure any new fields added to MapResponse
|
||||
// are accounted for here.
|
||||
want = true
|
||||
}
|
||||
|
||||
var v tailcfg.MapResponse
|
||||
rv := reflect.ValueOf(&v).Elem()
|
||||
rv.FieldByName(f.Name).Set(reflectNonzero(f.Type))
|
||||
|
||||
got := mapResponseContainsNonPatchFields(&v)
|
||||
if got != want {
|
||||
t.Errorf("field %q: got %v; want %v\nJSON: %v", f.Name, got, want, logger.AsJSON(v))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// tests MutationsFromMapResponse
|
||||
func TestMutationsFromMapResponse(t *testing.T) {
|
||||
someTime := time.Unix(123, 0)
|
||||
fromChanges := func(changes ...*tailcfg.PeerChange) *tailcfg.MapResponse {
|
||||
return &tailcfg.MapResponse{
|
||||
PeersChangedPatch: changes,
|
||||
}
|
||||
}
|
||||
muts := func(muts ...NodeMutation) []NodeMutation { return muts }
|
||||
tests := []struct {
|
||||
name string
|
||||
mr *tailcfg.MapResponse
|
||||
want []NodeMutation // nil means !ok, zero-length means none
|
||||
}{
|
||||
{
|
||||
name: "patch-ep",
|
||||
mr: fromChanges(&tailcfg.PeerChange{
|
||||
NodeID: 1,
|
||||
Endpoints: []string{"1.2.3.4:567"},
|
||||
}, &tailcfg.PeerChange{
|
||||
NodeID: 2,
|
||||
Endpoints: []string{"8.9.10.11:1234"},
|
||||
}),
|
||||
want: muts(
|
||||
NodeMutationEndpoints{1, []netip.AddrPort{netip.MustParseAddrPort("1.2.3.4:567")}},
|
||||
NodeMutationEndpoints{2, []netip.AddrPort{netip.MustParseAddrPort("8.9.10.11:1234")}},
|
||||
),
|
||||
},
|
||||
{
|
||||
name: "patch-derp",
|
||||
mr: fromChanges(&tailcfg.PeerChange{
|
||||
NodeID: 1,
|
||||
DERPRegion: 2,
|
||||
}),
|
||||
want: muts(NodeMutationDERPHome{1, 2}),
|
||||
},
|
||||
{
|
||||
name: "patch-online",
|
||||
mr: fromChanges(&tailcfg.PeerChange{
|
||||
NodeID: 1,
|
||||
Online: ptr.To(true),
|
||||
}),
|
||||
want: muts(NodeMutationOnline{1, true}),
|
||||
},
|
||||
{
|
||||
name: "patch-online-false",
|
||||
mr: fromChanges(&tailcfg.PeerChange{
|
||||
NodeID: 1,
|
||||
Online: ptr.To(false),
|
||||
}),
|
||||
want: muts(NodeMutationOnline{1, false}),
|
||||
},
|
||||
{
|
||||
name: "patch-lastseen",
|
||||
mr: fromChanges(&tailcfg.PeerChange{
|
||||
NodeID: 1,
|
||||
LastSeen: ptr.To(time.Unix(12345, 0)),
|
||||
}),
|
||||
want: muts(NodeMutationLastSeen{1, time.Unix(12345, 0)}),
|
||||
},
|
||||
{
|
||||
name: "legacy-online-change", // the old pre-Patch style
|
||||
mr: &tailcfg.MapResponse{
|
||||
OnlineChange: map[tailcfg.NodeID]bool{
|
||||
1: true,
|
||||
2: false,
|
||||
},
|
||||
},
|
||||
want: muts(
|
||||
NodeMutationOnline{1, true},
|
||||
NodeMutationOnline{2, false},
|
||||
),
|
||||
},
|
||||
{
|
||||
name: "legacy-lastseen-change", // the old pre-Patch style
|
||||
mr: &tailcfg.MapResponse{
|
||||
PeerSeenChange: map[tailcfg.NodeID]bool{
|
||||
1: true,
|
||||
},
|
||||
},
|
||||
want: muts(
|
||||
NodeMutationLastSeen{1, someTime},
|
||||
),
|
||||
},
|
||||
{
|
||||
name: "no-changes",
|
||||
mr: fromChanges(),
|
||||
want: make([]NodeMutation, 0), // non-nil to mean want ok but no changes
|
||||
},
|
||||
{
|
||||
name: "not-okay-patch-node-change",
|
||||
mr: &tailcfg.MapResponse{
|
||||
Node: &tailcfg.Node{}, // non-nil
|
||||
PeersChangedPatch: []*tailcfg.PeerChange{{
|
||||
NodeID: 1,
|
||||
DERPRegion: 2,
|
||||
}},
|
||||
},
|
||||
want: nil,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
got, gotOK := MutationsFromMapResponse(tt.mr, someTime)
|
||||
wantOK := tt.want != nil
|
||||
if gotOK != wantOK {
|
||||
t.Errorf("got ok=%v; want %v", gotOK, wantOK)
|
||||
} else if got == nil && gotOK {
|
||||
got = make([]NodeMutation, 0) // for cmd.Diff
|
||||
}
|
||||
if diff := cmp.Diff(tt.want, got,
|
||||
cmp.Comparer(func(a, b netip.Addr) bool { return a == b }),
|
||||
cmp.Comparer(func(a, b netip.AddrPort) bool { return a == b }),
|
||||
cmp.AllowUnexported(
|
||||
NodeMutationEndpoints{},
|
||||
NodeMutationDERPHome{},
|
||||
NodeMutationOnline{},
|
||||
NodeMutationLastSeen{},
|
||||
)); diff != "" {
|
||||
t.Errorf("wrong result (-want +got):\n%s", diff)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
@ -29,6 +29,7 @@
|
||||
"tailscale.com/tstime/mono"
|
||||
"tailscale.com/types/key"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/views"
|
||||
"tailscale.com/util/mak"
|
||||
"tailscale.com/util/ringbuffer"
|
||||
)
|
||||
@ -805,20 +806,38 @@ func (de *endpoint) updateFromNode(n tailcfg.NodeView, heartbeatDisabled bool) {
|
||||
de.derpAddr = newDerp
|
||||
}
|
||||
|
||||
de.setEndpointsLocked(addrPortsFromStringsView{n.Endpoints()})
|
||||
}
|
||||
|
||||
// addrPortsFromStringsView converts a view of AddrPort strings
|
||||
// to a view-like thing of netip.AddrPort.
|
||||
// TODO(bradfitz): change the type of tailcfg.Node.Endpoint.
|
||||
type addrPortsFromStringsView struct {
|
||||
views.Slice[string]
|
||||
}
|
||||
|
||||
func (a addrPortsFromStringsView) At(i int) netip.AddrPort {
|
||||
ap, _ := netip.ParseAddrPort(a.Slice.At(i))
|
||||
return ap // or the zero value on error
|
||||
}
|
||||
|
||||
func (de *endpoint) setEndpointsLocked(eps interface {
|
||||
LenIter() []struct{}
|
||||
At(i int) netip.AddrPort
|
||||
}) {
|
||||
for _, st := range de.endpointState {
|
||||
st.index = indexSentinelDeleted // assume deleted until updated in next loop
|
||||
}
|
||||
|
||||
var newIpps []netip.AddrPort
|
||||
for i := range n.Endpoints().LenIter() {
|
||||
epStr := n.Endpoints().At(i)
|
||||
for i := range eps.LenIter() {
|
||||
if i > math.MaxInt16 {
|
||||
// Seems unlikely.
|
||||
continue
|
||||
break
|
||||
}
|
||||
ipp, err := netip.ParseAddrPort(epStr)
|
||||
if err != nil {
|
||||
de.c.logf("magicsock: bogus netmap endpoint %q", epStr)
|
||||
ipp := eps.At(i)
|
||||
if !ipp.IsValid() {
|
||||
de.c.logf("magicsock: bogus netmap endpoint from %v", eps)
|
||||
continue
|
||||
}
|
||||
if st, ok := de.endpointState[ipp]; ok {
|
||||
@ -1214,3 +1233,9 @@ func (de *endpoint) resetLocked() {
|
||||
func (de *endpoint) numStopAndReset() int64 {
|
||||
return atomic.LoadInt64(&de.numStopAndResetAtomic)
|
||||
}
|
||||
|
||||
func (de *endpoint) setDERPHome(regionID uint16) {
|
||||
de.mu.Lock()
|
||||
defer de.mu.Unlock()
|
||||
de.derpAddr = netip.AddrPortFrom(tailcfg.DerpMagicIPAddr, uint16(regionID))
|
||||
}
|
||||
|
@ -2588,6 +2588,29 @@ func simpleDur(d time.Duration) time.Duration {
|
||||
return d.Round(time.Minute)
|
||||
}
|
||||
|
||||
// UpdateNetmapDelta implements controlclient.NetmapDeltaUpdater.
|
||||
func (c *Conn) UpdateNetmapDelta(muts []netmap.NodeMutation) (handled bool) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
for _, m := range muts {
|
||||
nodeID := m.NodeIDBeingMutated()
|
||||
ep, ok := c.peerMap.endpointForNodeID(nodeID)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
switch m := m.(type) {
|
||||
case netmap.NodeMutationDERPHome:
|
||||
ep.setDERPHome(uint16(m.DERPRegion))
|
||||
case netmap.NodeMutationEndpoints:
|
||||
ep.mu.Lock()
|
||||
ep.setEndpointsLocked(views.SliceOf(m.Endpoints))
|
||||
ep.mu.Unlock()
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// UpdateStatus implements the interface nede by ipnstate.StatusBuilder.
|
||||
//
|
||||
// This method adds in the magicsock-specific information only. Most
|
||||
|
Loading…
Reference in New Issue
Block a user