net/netmon, wgengine/userspace: purge ChangeDelta.Major and address TODOs (#17823)

updates tailscale/corp#33891

Addresses several older the TODO's in netmon.  This removes the 
Major flag precomputes the ChangeDelta state, rather than making
consumers of ChangeDeltas sort that out themselves.   We're also seeing
a lot of ChangeDelta's being flagged as "Major" when they are
not interesting, triggering rebinds in wgengine that are not needed.  This
cleans that up and adds a host of additional tests.

The dependencies are cleaned, notably removing dependency on netmon
itself for calculating what is interesting, and what is not.  This includes letting
individual platforms set a bespoke global "IsInterestingInterface"
function.  This is only used on Darwin.

RebindRequired now roughly follows how "Major" was historically
calculated but includes some additional checks for various
uninteresting events such as changes in interface addresses that
shouldn't trigger a rebind.  This significantly reduces thrashing (by
roughly half on Darwin clients which switching between nics).   The individual
values that we roll  into RebindRequired are also exposed so that
components consuming netmap.ChangeDelta can ask more
targeted questions.

Signed-off-by: Jonathan Nobels <jonathan@tailscale.com>
This commit is contained in:
Jonathan Nobels
2025-12-17 12:32:40 -05:00
committed by GitHub
parent 0fd1670a59
commit 3e89068792
19 changed files with 754 additions and 273 deletions

View File

@@ -138,12 +138,12 @@ func changeDeltaWatcher(ec *eventbus.Client, ctx context.Context, dump func(st *
case <-ec.Done():
return
case delta := <-changeSub.Events():
if !delta.Major {
log.Printf("Network monitor fired; not a major change")
if !delta.RebindLikelyRequired {
log.Printf("Network monitor fired; not a significant change")
return
}
log.Printf("Network monitor fired. New state:")
dump(delta.New)
dump(delta.CurrentState())
}
}
}

View File

@@ -296,7 +296,7 @@ type LocalBackend struct {
authURLTime time.Time // when the authURL was received from the control server; TODO(nickkhyl): move to nodeBackend
authActor ipnauth.Actor // an actor who called [LocalBackend.StartLoginInteractive] last, or nil; TODO(nickkhyl): move to nodeBackend
egg bool
prevIfState *netmon.State
interfaceState *netmon.State // latest network interface state or nil
peerAPIServer *peerAPIServer // or nil
peerAPIListeners []*peerAPIListener // TODO(nickkhyl): move to nodeBackend
loginFlags controlclient.LoginFlags
@@ -561,10 +561,16 @@ func NewLocalBackend(logf logger.Logf, logID logid.PublicID, sys *tsd.System, lo
b.e.SetStatusCallback(b.setWgengineStatus)
b.prevIfState = netMon.InterfaceState()
b.interfaceState = netMon.InterfaceState()
// Call our linkChange code once with the current state.
// Following changes are triggered via the eventbus.
b.linkChange(&netmon.ChangeDelta{New: netMon.InterfaceState()})
cd, err := netmon.NewChangeDelta(nil, b.interfaceState, false, netMon.TailscaleInterfaceName(), false)
if err != nil {
b.logf("[unexpected] setting initial netmon state failed: %v", err)
} else {
b.linkChange(cd)
}
if buildfeatures.HasPeerAPIServer {
if tunWrap, ok := b.sys.Tun.GetOK(); ok {
@@ -936,7 +942,7 @@ func (b *LocalBackend) pauseOrResumeControlClientLocked() {
if b.cc == nil {
return
}
networkUp := b.prevIfState.AnyInterfaceUp()
networkUp := b.interfaceState.AnyInterfaceUp()
pauseForNetwork := (b.state == ipn.Stopped && b.NetMap() != nil) || (!networkUp && !testenv.InTest() && !assumeNetworkUpdateForTest())
prefs := b.pm.CurrentPrefs()
@@ -963,24 +969,23 @@ func (b *LocalBackend) linkChange(delta *netmon.ChangeDelta) {
b.mu.Lock()
defer b.mu.Unlock()
ifst := delta.New
hadPAC := b.prevIfState.HasPAC()
b.prevIfState = ifst
b.interfaceState = delta.CurrentState()
b.pauseOrResumeControlClientLocked()
prefs := b.pm.CurrentPrefs()
if delta.Major && prefs.AutoExitNode().IsSet() {
if delta.RebindLikelyRequired && prefs.AutoExitNode().IsSet() {
b.refreshAutoExitNode = true
}
var needReconfig bool
// If the network changed and we're using an exit node and allowing LAN access, we may need to reconfigure.
if delta.Major && prefs.ExitNodeID() != "" && prefs.ExitNodeAllowLANAccess() {
if delta.RebindLikelyRequired && prefs.ExitNodeID() != "" && prefs.ExitNodeAllowLANAccess() {
b.logf("linkChange: in state %v; updating LAN routes", b.state)
needReconfig = true
}
// If the PAC-ness of the network changed, reconfig wireguard+route to add/remove subnets.
if hadPAC != ifst.HasPAC() {
b.logf("linkChange: in state %v; PAC changed from %v->%v", b.state, hadPAC, ifst.HasPAC())
if delta.HasPACOrProxyConfigChanged {
b.logf("linkChange: in state %v; PAC or proxyConfig changed; updating routes", b.state)
needReconfig = true
}
if needReconfig {
@@ -998,7 +1003,7 @@ func (b *LocalBackend) linkChange(delta *netmon.ChangeDelta) {
// If the local network configuration has changed, our filter may
// need updating to tweak default routes.
b.updateFilterLocked(prefs)
updateExitNodeUsageWarning(prefs, delta.New, b.health)
updateExitNodeUsageWarning(prefs, delta.CurrentState(), b.health)
if buildfeatures.HasPeerAPIServer {
cn := b.currentNode()
@@ -5059,7 +5064,7 @@ func (b *LocalBackend) authReconfigLocked() {
}
prefs := b.pm.CurrentPrefs()
hasPAC := b.prevIfState.HasPAC()
hasPAC := b.interfaceState.HasPAC()
disableSubnetsIfPAC := cn.SelfHasCap(tailcfg.NodeAttrDisableSubnetsIfPAC)
dohURL, dohURLOK := cn.exitNodeCanProxyDNS(prefs.ExitNodeID())
dcfg := cn.dnsConfigForNetmap(prefs, b.keyExpired, version.OS())
@@ -5310,7 +5315,7 @@ func (b *LocalBackend) initPeerAPIListenerLocked() {
var err error
skipListen := i > 0 && isNetstack
if !skipListen {
ln, err = ps.listen(a.Addr(), b.prevIfState)
ln, err = ps.listen(a.Addr(), b.interfaceState.TailscaleInterfaceIndex)
if err != nil {
if peerAPIListenAsync {
b.logf("[v1] possibly transient peerapi listen(%q) error, will try again on linkChange: %v", a.Addr(), err)

View File

@@ -41,7 +41,7 @@ import (
"tailscale.com/wgengine/filter"
)
var initListenConfig func(*net.ListenConfig, netip.Addr, *netmon.State, string) error
var initListenConfig func(config *net.ListenConfig, addr netip.Addr, tunIfIndex int) error
// peerDNSQueryHandler is implemented by tsdns.Resolver.
type peerDNSQueryHandler interface {
@@ -53,7 +53,7 @@ type peerAPIServer struct {
resolver peerDNSQueryHandler
}
func (s *peerAPIServer) listen(ip netip.Addr, ifState *netmon.State) (ln net.Listener, err error) {
func (s *peerAPIServer) listen(ip netip.Addr, tunIfIndex int) (ln net.Listener, err error) {
// Android for whatever reason often has problems creating the peerapi listener.
// But since we started intercepting it with netstack, it's not even important that
// we have a real kernel-level listener. So just create a dummy listener on Android
@@ -69,7 +69,7 @@ func (s *peerAPIServer) listen(ip netip.Addr, ifState *netmon.State) (ln net.Lis
// On iOS/macOS, this sets the lc.Control hook to
// setsockopt the interface index to bind to, to get
// out of the network sandbox.
if err := initListenConfig(&lc, ip, ifState, s.b.dialer.TUNName()); err != nil {
if err := initListenConfig(&lc, ip, tunIfIndex); err != nil {
return nil, err
}
if runtime.GOOS == "darwin" || runtime.GOOS == "ios" {

View File

@@ -6,11 +6,9 @@
package ipnlocal
import (
"fmt"
"net"
"net/netip"
"tailscale.com/net/netmon"
"tailscale.com/net/netns"
)
@@ -21,10 +19,6 @@ func init() {
// initListenConfigNetworkExtension configures nc for listening on IP
// through the iOS/macOS Network/System Extension (Packet Tunnel
// Provider) sandbox.
func initListenConfigNetworkExtension(nc *net.ListenConfig, ip netip.Addr, st *netmon.State, tunIfName string) error {
tunIf, ok := st.Interface[tunIfName]
if !ok {
return fmt.Errorf("no interface with name %q", tunIfName)
}
return netns.SetListenConfigInterfaceIndex(nc, tunIf.Index)
func initListenConfigNetworkExtension(nc *net.ListenConfig, ip netip.Addr, ifaceIndex int) error {
return netns.SetListenConfigInterfaceIndex(nc, ifaceIndex)
}

View File

@@ -171,7 +171,7 @@ func (s *localListener) Run() {
// required by the network sandbox to allow binding to
// a specific interface. Without this hook, the system
// chooses a default interface to bind to.
if err := initListenConfig(&lc, ip, s.b.prevIfState, s.b.dialer.TUNName()); err != nil {
if err := initListenConfig(&lc, ip, s.b.interfaceState.TailscaleInterfaceIndex); err != nil {
s.logf("localListener failed to init listen config %v, backing off: %v", s.ap, err)
s.bo.BackOff(s.ctx, err)
continue

View File

@@ -445,7 +445,7 @@ func (lg *Logger) internetUp() bool {
// [netmon.ChangeDelta] events to detect whether the Internet is expected to be
// reachable.
func (lg *Logger) onChangeDelta(delta *netmon.ChangeDelta) {
if delta.New.AnyInterfaceUp() {
if delta.AnyInterfaceUp() {
fmt.Fprintf(lg.stderr, "logtail: internet back up\n")
lg.networkIsUp.Set()
} else {
@@ -464,7 +464,7 @@ func (lg *Logger) awaitInternetUp(ctx context.Context) {
}
upc := make(chan bool, 1)
defer lg.netMonitor.RegisterChangeCallback(func(delta *netmon.ChangeDelta) {
if delta.New.AnyInterfaceUp() {
if delta.AnyInterfaceUp() {
select {
case upc <- true:
default:

View File

@@ -6,38 +6,43 @@ package netmon
import (
"context"
"sync"
"time"
"tailscale.com/types/logger"
"tailscale.com/util/eventbus"
)
const cooldownSeconds = 300
// LinkChangeLogLimiter returns a new [logger.Logf] that logs each unique
// format string to the underlying logger only once per major LinkChange event.
// format string to the underlying logger only once per major LinkChange event
// with a cooldownSeconds second cooldown.
//
// The logger stops tracking seen format strings when the provided context is
// done.
func LinkChangeLogLimiter(ctx context.Context, logf logger.Logf, nm *Monitor) logger.Logf {
var formatSeen sync.Map // map[string]bool
sub := eventbus.SubscribeFunc(nm.b, func(cd ChangeDelta) {
// If we're in a major change or a time jump, clear the seen map.
if cd.Major || cd.TimeJumped {
formatSeen.Clear()
var formatLastSeen sync.Map // map[string]int64
sub := eventbus.SubscribeFunc(nm.b, func(cd *ChangeDelta) {
// Any link changes that are flagged as likely require a rebind are
// interesting enough that we should log them.
if cd.RebindLikelyRequired {
formatLastSeen.Clear()
}
})
context.AfterFunc(ctx, sub.Close)
return func(format string, args ...any) {
// We only store 'true' in the map, so if it's present then it
// means we've already logged this format string.
_, loaded := formatSeen.LoadOrStore(format, true)
if loaded {
// TODO(andrew-d): we may still want to log this
// message every N minutes (1x/hour?) even if it's been
// seen, so that debugging doesn't require searching
// back in the logs for an unbounded amount of time.
//
// See: https://github.com/tailscale/tailscale/issues/13145
return
// get the current timestamp
now := time.Now().Unix()
lastSeen, ok := formatLastSeen.Load(format)
if ok {
// if we've seen this format string within the last cooldownSeconds, skip logging
if now-lastSeen.(int64) < cooldownSeconds {
return
}
}
// update the last seen timestamp for this format string
formatLastSeen.Store(format, now)
logf(format, args...)
}

View File

@@ -64,7 +64,14 @@ func syncTestLinkChangeLogLimiter(t *testing.T) {
// InjectEvent doesn't work because it's not a major event, so we
// instead inject the event ourselves.
injector := eventbustest.NewInjector(t, bus)
eventbustest.Inject(injector, ChangeDelta{Major: true})
cd, err := NewChangeDelta(nil, &State{}, true, "tailscale0", true)
if err != nil {
t.Fatal(err)
}
if cd.RebindLikelyRequired != true {
t.Fatalf("expected RebindLikelyRequired to be true, got false")
}
eventbustest.Inject(injector, cd)
synctest.Wait()
logf("hello %s", "world")

View File

@@ -7,10 +7,12 @@
package netmon
import (
"encoding/json"
"errors"
"fmt"
"log"
"net/netip"
"runtime"
"slices"
"sync"
"time"
@@ -45,12 +47,15 @@ type osMon interface {
// until the osMon is closed. After a Close, the returned
// error is ignored.
Receive() (message, error)
// IsInterestingInterface reports whether the provided interface should
// be considered for network change events.
IsInterestingInterface(iface string) bool
}
// IsInterestingInterface is the function used to determine whether
// a given interface name is interesting enough to pay attention to
// for network change monitoring purposes.
//
// If nil, all interfaces are considered interesting.
var IsInterestingInterface func(Interface, []netip.Prefix) bool
// Monitor represents a monitoring instance.
type Monitor struct {
logf logger.Logf
@@ -62,10 +67,6 @@ type Monitor struct {
stop chan struct{} // closed on Stop
static bool // static Monitor that doesn't actually monitor
// Things that must be set early, before use,
// and not change at runtime.
tsIfName string // tailscale interface name, if known/set ("tailscale0", "utun3", ...)
mu syncs.Mutex // guards all following fields
cbs set.HandleSet[ChangeFunc]
ifState *State
@@ -77,7 +78,8 @@ type Monitor struct {
goroutines sync.WaitGroup
wallTimer *time.Timer // nil until Started; re-armed AfterFunc per tick
lastWall time.Time
timeJumped bool // whether we need to send a changed=true after a big time jump
timeJumped bool // whether we need to send a changed=true after a big time jump
tsIfName string // tailscale interface name, if known/set ("tailscale0", "utun3", ...)
}
// ChangeFunc is a callback function registered with Monitor that's called when the
@@ -85,32 +87,225 @@ type Monitor struct {
type ChangeFunc func(*ChangeDelta)
// ChangeDelta describes the difference between two network states.
//
// Use NewChangeDelta to construct a delta and compute the cached fields.
type ChangeDelta struct {
// Old is the old interface state, if known.
// old is the old interface state, if known.
// It's nil if the old state is unknown.
// Do not mutate it.
Old *State
old *State
// New is the new network state.
// It is always non-nil.
// Do not mutate it.
New *State
// Major is our legacy boolean of whether the network changed in some major
// way.
//
// Deprecated: do not remove. As of 2023-08-23 we're in a renewed effort to
// remove it and ask specific qustions of ChangeDelta instead. Look at Old
// and New (or add methods to ChangeDelta) instead of using Major.
Major bool
new *State
// TimeJumped is whether there was a big jump in wall time since the last
// time we checked. This is a hint that a mobile sleeping device might have
// time we checked. This is a hint that a sleeping device might have
// come out of sleep.
TimeJumped bool
// TODO(bradfitz): add some lazy cached fields here as needed with methods
// on *ChangeDelta to let callers ask specific questions
// The tailscale interface name, e.g. "tailscale0", "utun3", etc. Not all
// platforms know this or set it. Copied from netmon.Monitor.tsIfName.
TailscaleIfaceName string
DefaultRouteInterface string
// Computed Fields
DefaultInterfaceChanged bool // whether default route interface changed
IsLessExpensive bool // whether new state's default interface is less expensive than old.
HasPACOrProxyConfigChanged bool // whether PAC/HTTP proxy config changed
InterfaceIPsChanged bool // whether any interface IPs changed in a meaningful way
AvailableProtocolsChanged bool // whether we have seen a change in available IPv4/IPv6
DefaultInterfaceMaybeViable bool // whether the default interface is potentially viable (has usable IPs, is up and is not the tunnel itself)
IsInitialState bool // whether this is the initial state (old == nil, new != nil)
// RebindLikelyRequired combines the various fields above to report whether this change likely requires us
// to rebind sockets. This is a very conservative estimate and covers a number ofcases where a rebind
// may not be strictly necessary. Consumers of the ChangeDelta should consider checking the individual fields
// above or the state of their sockets.
RebindLikelyRequired bool
}
// CurrentState returns the current (new) state after the change.
func (cd *ChangeDelta) CurrentState() *State {
return cd.new
}
// NewChangeDelta builds a ChangeDelta and eagerly computes the cached fields.
// forceViability, if true, forces DefaultInterfaceMaybeViable to be true regardless of the
// actual state of the default interface. This is useful in testing.
func NewChangeDelta(old, new *State, timeJumped bool, tsIfName string, forceViability bool) (*ChangeDelta, error) {
cd := ChangeDelta{
old: old,
new: new,
TimeJumped: timeJumped,
TailscaleIfaceName: tsIfName,
}
if cd.new == nil {
log.Printf("[unexpected] NewChangeDelta called with nil new state")
return nil, errors.New("new state cannot be nil")
} else if cd.old == nil && cd.new != nil {
cd.DefaultInterfaceChanged = cd.new.DefaultRouteInterface != ""
cd.IsLessExpensive = false
cd.HasPACOrProxyConfigChanged = true
cd.InterfaceIPsChanged = true
cd.IsInitialState = true
} else {
cd.AvailableProtocolsChanged = (cd.old.HaveV4 != cd.new.HaveV4) || (cd.old.HaveV6 != cd.new.HaveV6)
cd.DefaultInterfaceChanged = cd.old.DefaultRouteInterface != cd.new.DefaultRouteInterface
cd.IsLessExpensive = cd.old.IsExpensive && !cd.new.IsExpensive
cd.HasPACOrProxyConfigChanged = (cd.old.PAC != cd.new.PAC) || (cd.old.HTTPProxy != cd.new.HTTPProxy)
cd.InterfaceIPsChanged = cd.isInterestingInterfaceChange()
}
cd.DefaultRouteInterface = new.DefaultRouteInterface
defIf := new.Interface[cd.DefaultRouteInterface]
// The default interface is not viable if it is down or it is the Tailscale interface itself.
if !forceViability && (!defIf.IsUp() || cd.DefaultRouteInterface == tsIfName) {
cd.DefaultInterfaceMaybeViable = false
} else {
cd.DefaultInterfaceMaybeViable = true
}
// Compute rebind requirement. The default interface needs to be viable and
// one of the other conditions needs to be true.
cd.RebindLikelyRequired = (cd.old == nil ||
cd.TimeJumped ||
cd.DefaultInterfaceChanged ||
cd.InterfaceIPsChanged ||
cd.IsLessExpensive ||
cd.HasPACOrProxyConfigChanged ||
cd.AvailableProtocolsChanged) &&
cd.DefaultInterfaceMaybeViable
return &cd, nil
}
// StateDesc returns a description of the old and new states for logging.
func (cd *ChangeDelta) StateDesc() string {
return fmt.Sprintf("old: %v new: %v", cd.old, cd.new)
}
// InterfaceIPDisappeared reports whether the given IP address exists on any interface
// in the old state, but not in the new state.
func (cd *ChangeDelta) InterfaceIPDisappeared(ip netip.Addr) bool {
if cd.old == nil {
return false
}
if cd.new == nil && cd.old.HasIP(ip) {
return true
}
return cd.new.HasIP(ip) && !cd.old.HasIP(ip)
}
// AnyInterfaceUp reports whether any interfaces are up in the new state.
func (cd *ChangeDelta) AnyInterfaceUp() bool {
if cd.new == nil {
return false
}
for _, ifi := range cd.new.Interface {
if ifi.IsUp() {
return true
}
}
return false
}
// isInterestingInterfaceChange reports whether any interfaces have changed in a meaningful way.
// This excludes interfaces that are not interesting per IsInterestingInterface and
// filters out changes to interface IPs that that are uninteresting (e.g. link-local addresses).
func (cd *ChangeDelta) isInterestingInterfaceChange() bool {
// If there is no old state, everything is considered changed.
if cd.old == nil {
return true
}
// Compare interfaces in both directions. Old to new and new to old.
for iname, oldInterface := range cd.old.Interface {
if iname == cd.TailscaleIfaceName {
// Ignore changes in the Tailscale interface itself.
continue
}
oldIps := filterRoutableIPs(cd.old.InterfaceIPs[iname])
if IsInterestingInterface != nil && !IsInterestingInterface(oldInterface, oldIps) {
continue
}
// Old interfaces with no routable addresses are not interesting
if len(oldIps) == 0 {
continue
}
// The old interface doesn't exist in the new interface set and it has
// a global unicast IP. That's considered a change from the perspective
// of anything that may have been bound to it. If it didn't have a global
// unicast IP, it's not interesting.
newInterface, ok := cd.new.Interface[iname]
if !ok {
return true
}
newIps, ok := cd.new.InterfaceIPs[iname]
if !ok {
return true
}
newIps = filterRoutableIPs(newIps)
if !oldInterface.Equal(newInterface) || !prefixesEqual(oldIps, newIps) {
return true
}
}
for iname, newInterface := range cd.new.Interface {
if iname == cd.TailscaleIfaceName {
continue
}
newIps := filterRoutableIPs(cd.new.InterfaceIPs[iname])
if IsInterestingInterface != nil && !IsInterestingInterface(newInterface, newIps) {
continue
}
// New interfaces with no routable addresses are not interesting
if len(newIps) == 0 {
continue
}
oldInterface, ok := cd.old.Interface[iname]
if !ok {
return true
}
oldIps, ok := cd.old.InterfaceIPs[iname]
if !ok {
// Redundant but we can't dig up the "old" IPs for this interface.
return true
}
oldIps = filterRoutableIPs(oldIps)
// The interface's IPs, Name, MTU, etc have changed. This is definitely interesting.
if !newInterface.Equal(oldInterface) || !prefixesEqual(oldIps, newIps) {
return true
}
}
return false
}
func filterRoutableIPs(addrs []netip.Prefix) []netip.Prefix {
var filtered []netip.Prefix
for _, pfx := range addrs {
a := pfx.Addr()
// Skip link-local multicast addresses.
if a.IsLinkLocalMulticast() {
continue
}
if isUsableV4(a) || isUsableV6(a) {
filtered = append(filtered, pfx)
}
}
return filtered
}
// New instantiates and starts a monitoring instance.
@@ -174,9 +369,17 @@ func (m *Monitor) interfaceStateUncached() (*State, error) {
// This must be called only early in tailscaled startup before the monitor is
// used.
func (m *Monitor) SetTailscaleInterfaceName(ifName string) {
m.mu.Lock()
defer m.mu.Unlock()
m.tsIfName = ifName
}
func (m *Monitor) TailscaleInterfaceName() string {
m.mu.Lock()
defer m.mu.Unlock()
return m.tsIfName
}
// GatewayAndSelfIP returns the current network's default gateway, and
// the machine's default IP for that gateway.
//
@@ -344,17 +547,6 @@ func (m *Monitor) pump() {
}
}
// isInterestingInterface reports whether the provided interface should be
// considered when checking for network state changes.
// The ips parameter should be the IPs of the provided interface.
func (m *Monitor) isInterestingInterface(i Interface, ips []netip.Prefix) bool {
if !m.om.IsInterestingInterface(i.Name) {
return false
}
return true
}
// debounce calls the callback function with a delay between events
// and exits when a stop is issued.
func (m *Monitor) debounce() {
@@ -376,7 +568,10 @@ func (m *Monitor) debounce() {
select {
case <-m.stop:
return
case <-time.After(250 * time.Millisecond):
// 1s is reasonable debounce time for network changes. Events such as undocking a laptop
// or roaming onto wifi will often generate multiple events in quick succession as interfaces
// flap. We want to avoid spamming consumers of these events.
case <-time.After(1000 * time.Millisecond):
}
}
}
@@ -403,146 +598,51 @@ func (m *Monitor) handlePotentialChange(newState *State, forceCallbacks bool) {
return
}
delta := ChangeDelta{
Old: oldState,
New: newState,
TimeJumped: timeJumped,
delta, err := NewChangeDelta(oldState, newState, timeJumped, m.tsIfName, false)
if err != nil {
m.logf("[unexpected] error creating ChangeDelta: %v", err)
return
}
delta.Major = m.IsMajorChangeFrom(oldState, newState)
if delta.Major {
if delta.RebindLikelyRequired {
m.gwValid = false
if s1, s2 := oldState.String(), delta.New.String(); s1 == s2 {
m.logf("[unexpected] network state changed, but stringification didn't: %v", s1)
m.logf("[unexpected] old: %s", jsonSummary(oldState))
m.logf("[unexpected] new: %s", jsonSummary(newState))
}
}
m.ifState = newState
// See if we have a queued or new time jump signal.
if timeJumped {
m.resetTimeJumpedLocked()
if !delta.Major {
// Only log if it wasn't an interesting change.
m.logf("time jumped (probably wake from sleep); synthesizing major change event")
delta.Major = true
}
}
metricChange.Add(1)
if delta.Major {
if delta.RebindLikelyRequired {
metricChangeMajor.Add(1)
}
if delta.TimeJumped {
metricChangeTimeJump.Add(1)
}
m.changed.Publish(delta)
m.changed.Publish(*delta)
for _, cb := range m.cbs {
go cb(&delta)
go cb(delta)
}
}
// IsMajorChangeFrom reports whether the transition from s1 to s2 is
// a "major" change, where major roughly means it's worth tearing down
// a bunch of connections and rebinding.
//
// TODO(bradiftz): tigten this definition.
func (m *Monitor) IsMajorChangeFrom(s1, s2 *State) bool {
if s1 == nil && s2 == nil {
// reports whether a and b contain the same set of prefixes regardless of order.
func prefixesEqual(a, b []netip.Prefix) bool {
if len(a) != len(b) {
return false
}
if s1 == nil || s2 == nil {
return true
}
if s1.HaveV6 != s2.HaveV6 ||
s1.HaveV4 != s2.HaveV4 ||
s1.IsExpensive != s2.IsExpensive ||
s1.DefaultRouteInterface != s2.DefaultRouteInterface ||
s1.HTTPProxy != s2.HTTPProxy ||
s1.PAC != s2.PAC {
return true
}
for iname, i := range s1.Interface {
if iname == m.tsIfName {
// Ignore changes in the Tailscale interface itself.
continue
}
ips := s1.InterfaceIPs[iname]
if !m.isInterestingInterface(i, ips) {
continue
}
i2, ok := s2.Interface[iname]
if !ok {
return true
}
ips2, ok := s2.InterfaceIPs[iname]
if !ok {
return true
}
if !i.Equal(i2) || !prefixesMajorEqual(ips, ips2) {
return true
}
}
// Iterate over s2 in case there is a field in s2 that doesn't exist in s1
for iname, i := range s2.Interface {
if iname == m.tsIfName {
// Ignore changes in the Tailscale interface itself.
continue
}
ips := s2.InterfaceIPs[iname]
if !m.isInterestingInterface(i, ips) {
continue
}
i1, ok := s1.Interface[iname]
if !ok {
return true
}
ips1, ok := s1.InterfaceIPs[iname]
if !ok {
return true
}
if !i.Equal(i1) || !prefixesMajorEqual(ips, ips1) {
return true
}
}
return false
}
// prefixesMajorEqual reports whether a and b are equal after ignoring
// boring things like link-local, loopback, and multicast addresses.
func prefixesMajorEqual(a, b []netip.Prefix) bool {
// trim returns a subslice of p with link local unicast,
// loopback, and multicast prefixes removed from the front.
trim := func(p []netip.Prefix) []netip.Prefix {
for len(p) > 0 {
a := p[0].Addr()
if a.IsLinkLocalUnicast() || a.IsLoopback() || a.IsMulticast() {
p = p[1:]
continue
}
break
}
return p
}
for {
a = trim(a)
b = trim(b)
if len(a) == 0 || len(b) == 0 {
return len(a) == 0 && len(b) == 0
}
if a[0] != b[0] {
return false
}
a, b = a[1:], b[1:]
}
}
aa := make([]netip.Prefix, len(a))
bb := make([]netip.Prefix, len(b))
copy(aa, a)
copy(bb, b)
func jsonSummary(x any) any {
j, err := json.Marshal(x)
if err != nil {
return err
less := func(x, y netip.Prefix) int {
return x.Addr().Compare(y.Addr())
}
return j
slices.SortFunc(aa, less)
slices.SortFunc(bb, less)
return slices.Equal(aa, bb)
}
func wallTime() time.Time {

View File

@@ -16,6 +16,12 @@ import (
"tailscale.com/util/eventbus"
)
func init() {
IsInterestingInterface = func(iface Interface, prefixes []netip.Prefix) bool {
return isInterestingInterface(iface.Name)
}
}
const debugRouteMessages = false
// unspecifiedMessage is a minimal message implementation that should not
@@ -125,11 +131,10 @@ func addrType(addrs []route.Addr, rtaxType int) route.Addr {
return nil
}
func (m *darwinRouteMon) IsInterestingInterface(iface string) bool {
func isInterestingInterface(iface string) bool {
baseName := strings.TrimRight(iface, "0123456789")
switch baseName {
// TODO(maisem): figure out what this list should actually be.
case "llw", "awdl", "ipsec":
case "llw", "awdl", "ipsec", "gif", "XHC", "anpi", "lo", "utun":
return false
}
return true
@@ -137,7 +142,7 @@ func (m *darwinRouteMon) IsInterestingInterface(iface string) bool {
func (m *darwinRouteMon) skipInterfaceAddrMessage(msg *route.InterfaceAddrMessage) bool {
if la, ok := addrType(msg.Addrs, unix.RTAX_IFP).(*route.LinkAddr); ok {
if !m.IsInterestingInterface(la.Name) {
if !isInterestingInterface(la.Name) {
return true
}
}
@@ -150,6 +155,14 @@ func (m *darwinRouteMon) skipRouteMessage(msg *route.RouteMessage) bool {
// dst = fe80::b476:66ff:fe30:c8f6%15
return true
}
// We can skip route messages from uninteresting interfaces. We do this upstream
// against the InterfaceMonitor, but skipping them here avoids unnecessary work.
if la, ok := addrType(msg.Addrs, unix.RTAX_IFP).(*route.LinkAddr); ok {
if !isInterestingInterface(la.Name) {
return true
}
}
return false
}

View File

@@ -34,8 +34,6 @@ func newOSMon(_ *eventbus.Bus, logf logger.Logf, m *Monitor) (osMon, error) {
return &devdConn{conn}, nil
}
func (c *devdConn) IsInterestingInterface(iface string) bool { return true }
func (c *devdConn) Close() error {
return c.conn.Close()
}

View File

@@ -81,8 +81,6 @@ func newOSMon(bus *eventbus.Bus, logf logger.Logf, m *Monitor) (osMon, error) {
}, nil
}
func (c *nlConn) IsInterestingInterface(iface string) bool { return true }
func (c *nlConn) Close() error {
c.busClient.Close()
return c.conn.Close()

View File

@@ -8,6 +8,7 @@ import (
"net"
"net/netip"
"reflect"
"strings"
"sync/atomic"
"testing"
"time"
@@ -138,7 +139,7 @@ func TestMonitorMode(t *testing.T) {
n := 0
mon.RegisterChangeCallback(func(d *ChangeDelta) {
n++
t.Logf("cb: changed=%v, ifSt=%v", d.Major, d.New)
t.Logf("cb: changed=%v, ifSt=%v", d.RebindLikelyRequired, d.CurrentState())
})
mon.Start()
<-done
@@ -149,24 +150,22 @@ func TestMonitorMode(t *testing.T) {
mon.Start()
eventbustest.Expect(tw, func(event *ChangeDelta) (bool, error) {
n++
t.Logf("cb: changed=%v, ifSt=%v", event.Major, event.New)
t.Logf("cb: changed=%v, ifSt=%v", event.RebindLikelyRequired, event.CurrentState())
return false, nil // Return false, indicating we wanna look for more events
})
t.Logf("%v events", n)
}
}
// tests (*State).IsMajorChangeFrom
func TestIsMajorChangeFrom(t *testing.T) {
// tests (*ChangeDelta).RebindRequired
func TestRebindRequired(t *testing.T) {
// s1 cannot be nil by definition
tests := []struct {
name string
s1, s2 *State
want bool
name string
s1, s2 *State
tsIfName string
want bool
}{
{
name: "eq_nil",
want: false,
},
{
name: "nil_mix",
s2: new(State),
@@ -188,6 +187,110 @@ func TestIsMajorChangeFrom(t *testing.T) {
},
want: false,
},
{
name: "new-with-no-addr",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
"bar": {},
},
},
want: false,
},
{
name: "ignore-tailscale-interface-appearing",
tsIfName: "tailscale0",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
"tailscale0": {netip.MustParsePrefix("100.69.4.20/32")},
},
},
want: false,
},
{
name: "ignore-tailscale-interface-disappearing",
tsIfName: "tailscale0",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
"tailscale0": {netip.MustParsePrefix("100.69.4.20/32")},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
},
},
want: false,
},
{
name: "new-with-multicast-addr",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
"bar": {netip.MustParsePrefix("224.0.0.1/32")},
},
},
want: false,
},
{
name: "old-with-addr-dropped",
s1: &State{
DefaultRouteInterface: "bar",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
"bar": {netip.MustParsePrefix("192.168.0.1/32")},
},
},
s2: &State{
DefaultRouteInterface: "bar",
InterfaceIPs: map[string][]netip.Prefix{
"bar": {netip.MustParsePrefix("192.168.0.1/32")},
},
},
want: true,
},
{
name: "old-with-no-addr-dropped",
s1: &State{
DefaultRouteInterface: "bar",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {},
"bar": {netip.MustParsePrefix("192.168.0.1/16")},
},
},
s2: &State{
DefaultRouteInterface: "bar",
InterfaceIPs: map[string][]netip.Prefix{
"bar": {netip.MustParsePrefix("192.168.0.1/16")},
},
},
want: false,
},
{
name: "default-route-changed",
s1: &State{
@@ -221,6 +324,8 @@ func TestIsMajorChangeFrom(t *testing.T) {
want: true,
},
{
// (barnstar) TODO: ULA addresses are only useful in some contexts,
// so maybe this shouldn't trigger rebinds after all? Needs more thought.
name: "ipv6-ula-addressed-appeared",
s1: &State{
DefaultRouteInterface: "foo",
@@ -233,15 +338,147 @@ func TestIsMajorChangeFrom(t *testing.T) {
InterfaceIPs: map[string][]netip.Prefix{
"foo": {
netip.MustParsePrefix("10.0.1.2/16"),
// Brad saw this address coming & going on his home LAN, possibly
// via an Apple TV Thread routing advertisement? (Issue 9040)
netip.MustParsePrefix("fd15:bbfa:c583:4fce:f4fb:4ff:fe1a:4148/64"),
},
},
},
want: true, // TODO(bradfitz): want false (ignore the IPv6 ULA address on foo)
want: true,
},
{
// (barnstar) TODO: ULA addresses are only useful in some contexts,
// so maybe this shouldn't trigger rebinds after all? Needs more thought.
name: "ipv6-ula-addressed-disappeared",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {
netip.MustParsePrefix("10.0.1.2/16"),
netip.MustParsePrefix("fd15:bbfa:c583:4fce:f4fb:4ff:fe1a:4148/64"),
},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
},
},
want: true,
},
{
name: "ipv6-link-local-addressed-appeared",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {netip.MustParsePrefix("10.0.1.2/16")},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {
netip.MustParsePrefix("10.0.1.2/16"),
netip.MustParsePrefix("fe80::f242:25ff:fe64:b280/64"),
},
},
},
want: false,
},
{
name: "ipv6-addressed-changed",
s1: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {
netip.MustParsePrefix("10.0.1.2/16"),
netip.MustParsePrefix("2001::f242:25ff:fe64:b280/64"),
netip.MustParsePrefix("fe80::f242:25ff:fe64:b280/64"),
},
},
},
s2: &State{
DefaultRouteInterface: "foo",
InterfaceIPs: map[string][]netip.Prefix{
"foo": {
netip.MustParsePrefix("10.0.1.2/16"),
netip.MustParsePrefix("2001::beef:8bad:f00d:b280/64"),
netip.MustParsePrefix("fe80::f242:25ff:fe64:b280/64"),
},
},
},
want: true,
},
{
name: "have-addr-changed",
s1: &State{
HaveV6: false,
HaveV4: false,
},
s2: &State{
HaveV6: true,
HaveV4: true,
},
want: true,
},
{
name: "have-addr-unchanged",
s1: &State{
HaveV6: true,
HaveV4: true,
},
s2: &State{
HaveV6: true,
HaveV4: true,
},
want: false,
},
{
name: "new-is-less-expensive",
s1: &State{
IsExpensive: true,
},
s2: &State{
IsExpensive: false,
},
want: true,
},
{
name: "new-is-more-expensive",
s1: &State{
IsExpensive: false,
},
s2: &State{
IsExpensive: true,
},
want: false,
},
{
name: "uninteresting-interface-added",
s1: &State{
DefaultRouteInterface: "bar",
InterfaceIPs: map[string][]netip.Prefix{
"bar": {netip.MustParsePrefix("192.168.0.1/16")},
},
},
s2: &State{
DefaultRouteInterface: "bar",
InterfaceIPs: map[string][]netip.Prefix{
"bar": {netip.MustParsePrefix("192.168.0.1/16")},
"boring": {netip.MustParsePrefix("fd7a:115c:a1e0:ab12:4843:cd96:625e:13ce/64")},
},
},
want: false,
},
}
withIsInterestingInterface(t, func(ni Interface, pfxs []netip.Prefix) bool {
return !strings.HasPrefix(ni.Name, "boring")
})
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
// Populate dummy interfaces where missing.
@@ -258,16 +495,111 @@ func TestIsMajorChangeFrom(t *testing.T) {
}
}
var m Monitor
m.om = &testOSMon{
Interesting: func(name string) bool { return true },
cd, err := NewChangeDelta(tt.s1, tt.s2, false, tt.tsIfName, true)
if err != nil {
t.Fatalf("NewChangeDelta error: %v", err)
}
if got := m.IsMajorChangeFrom(tt.s1, tt.s2); got != tt.want {
t.Errorf("IsMajorChange = %v; want %v", got, tt.want)
_ = cd // in case we need it later
if got := cd.RebindLikelyRequired; got != tt.want {
t.Errorf("RebindRequired = %v; want %v", got, tt.want)
}
})
}
}
func withIsInterestingInterface(t *testing.T, fn func(Interface, []netip.Prefix) bool) {
t.Helper()
old := IsInterestingInterface
IsInterestingInterface = fn
t.Cleanup(func() { IsInterestingInterface = old })
}
func TestIncludesRoutableIP(t *testing.T) {
routable := []netip.Prefix{
netip.MustParsePrefix("1.2.3.4/32"),
netip.MustParsePrefix("10.0.0.1/24"), // RFC1918 IPv4 (private)
netip.MustParsePrefix("172.16.0.1/12"), // RFC1918 IPv4 (private)
netip.MustParsePrefix("192.168.1.1/24"), // RFC1918 IPv4 (private)
netip.MustParsePrefix("fd15:dead:beef::1/64"), // IPv6 ULA
netip.MustParsePrefix("2001:db8::1/64"), // global IPv6
}
nonRoutable := []netip.Prefix{
netip.MustParsePrefix("ff00::/8"), // multicast IPv6 (should be filtered)
netip.MustParsePrefix("fe80::1/64"), // link-local IPv6
netip.MustParsePrefix("::1/128"), // loopback IPv6
netip.MustParsePrefix("::/128"), // unspecified IPv6
netip.MustParsePrefix("224.0.0.1/32"), // multicast IPv4
netip.MustParsePrefix("127.0.0.1/32"), // loopback IPv4
}
got, want := filterRoutableIPs(
append(nonRoutable, routable...),
), routable
if !reflect.DeepEqual(got, want) {
t.Fatalf("filterRoutableIPs returned %v; want %v", got, want)
}
}
func TestPrefixesEqual(t *testing.T) {
tests := []struct {
name string
a, b []netip.Prefix
want bool
}{
{
name: "empty",
a: []netip.Prefix{},
b: []netip.Prefix{},
want: true,
},
{
name: "single-equal",
a: []netip.Prefix{netip.MustParsePrefix("10.0.0.1/24")},
b: []netip.Prefix{netip.MustParsePrefix("10.0.0.1/24")},
want: true,
},
{
name: "single-different",
a: []netip.Prefix{netip.MustParsePrefix("10.0.0.1/24")},
b: []netip.Prefix{netip.MustParsePrefix("10.0.0.2/24")},
want: false,
},
{
name: "unordered-equal",
a: []netip.Prefix{
netip.MustParsePrefix("10.0.0.1/24"),
netip.MustParsePrefix("10.0.2.1/24"),
},
b: []netip.Prefix{
netip.MustParsePrefix("10.0.2.1/24"),
netip.MustParsePrefix("10.0.0.1/24"),
},
want: true,
},
{
name: "subset",
a: []netip.Prefix{
netip.MustParsePrefix("10.0.2.1/24"),
},
b: []netip.Prefix{
netip.MustParsePrefix("10.0.2.1/24"),
netip.MustParsePrefix("10.0.0.1/24"),
},
want: false,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := prefixesEqual(tt.a, tt.b)
if got != tt.want {
t.Errorf("prefixesEqual(%v, %v) = %v; want %v", tt.a, tt.b, got, tt.want)
}
})
}
}
func TestForeachInterface(t *testing.T) {
tests := []struct {
name string
@@ -307,15 +639,3 @@ func TestForeachInterface(t *testing.T) {
})
}
}
type testOSMon struct {
osMon
Interesting func(name string) bool
}
func (m *testOSMon) IsInterestingInterface(name string) bool {
if m.Interesting == nil {
return true
}
return m.Interesting(name)
}

View File

@@ -74,8 +74,6 @@ func newOSMon(_ *eventbus.Bus, logf logger.Logf, pm *Monitor) (osMon, error) {
return m, nil
}
func (m *winMon) IsInterestingInterface(iface string) bool { return true }
func (m *winMon) Close() (ret error) {
m.cancel()
m.noDeadlockTicker.Stop()

View File

@@ -35,10 +35,6 @@ type pollingMon struct {
stop chan struct{}
}
func (pm *pollingMon) IsInterestingInterface(iface string) bool {
return true
}
func (pm *pollingMon) Close() error {
pm.closeOnce.Do(func() {
close(pm.stop)

View File

@@ -149,12 +149,28 @@ type Interface struct {
Desc string // extra description (used on Windows)
}
func (i Interface) IsLoopback() bool { return isLoopback(i.Interface) }
func (i Interface) IsUp() bool { return isUp(i.Interface) }
func (i Interface) IsLoopback() bool {
if i.Interface == nil {
return false
}
return isLoopback(i.Interface)
}
func (i Interface) IsUp() bool {
if i.Interface == nil {
return false
}
return isUp(i.Interface)
}
func (i Interface) Addrs() ([]net.Addr, error) {
if i.AltAddrs != nil {
return i.AltAddrs, nil
}
if i.Interface == nil {
return nil, nil
}
return i.Interface.Addrs()
}
@@ -271,6 +287,9 @@ type State struct {
// PAC is the URL to the Proxy Autoconfig URL, if applicable.
PAC string
// TailscaleInterfaceIndex is the index of the Tailscale interface
TailscaleInterfaceIndex int
}
func (s *State) String() string {
@@ -485,6 +504,16 @@ func getState(optTSInterfaceName string) (*State, error) {
ifUp := ni.IsUp()
s.Interface[ni.Name] = ni
s.InterfaceIPs[ni.Name] = append(s.InterfaceIPs[ni.Name], pfxs...)
// Skip uninteresting interfaces.
if IsInterestingInterface != nil && !IsInterestingInterface(ni, pfxs) {
return
}
if isTailscaleInterface(ni.Name, pfxs) {
s.TailscaleInterfaceIndex = ni.Index
}
if !ifUp || isTSInterfaceName || isTailscaleInterface(ni.Name, pfxs) {
return
}

View File

@@ -271,10 +271,10 @@ func setNetMon(netMon *netmon.Monitor) {
}
netMon.RegisterChangeCallback(func(delta *netmon.ChangeDelta) {
if !delta.Major {
if !delta.RebindLikelyRequired {
return
}
state := delta.New
state := delta.CurrentState()
ifName := state.DefaultRouteInterface
if ifName == "" {
return

View File

@@ -264,7 +264,7 @@ var (
func (d *Dialer) linkChanged(delta *netmon.ChangeDelta) {
// Track how often we see ChangeDeltas with no DefaultRouteInterface.
if delta.New.DefaultRouteInterface == "" {
if delta.DefaultRouteInterface == "" {
metricChangeDeltaNoDefaultRoute.Add(1)
}
@@ -294,22 +294,23 @@ func changeAffectsConn(delta *netmon.ChangeDelta, conn net.Conn) bool {
}
lip, rip := la.AddrPort().Addr(), ra.AddrPort().Addr()
if delta.Old == nil {
if delta.IsInitialState {
return false
}
if delta.Old.DefaultRouteInterface != delta.New.DefaultRouteInterface ||
delta.Old.HTTPProxy != delta.New.HTTPProxy {
if delta.DefaultInterfaceChanged ||
delta.HasPACOrProxyConfigChanged {
return true
}
// In a few cases, we don't have a new DefaultRouteInterface (e.g. on
// Android; see tailscale/corp#19124); if so, pessimistically assume
// Android and macOS/iOS; see tailscale/corp#19124); if so, pessimistically assume
// that all connections are affected.
if delta.New.DefaultRouteInterface == "" && runtime.GOOS != "plan9" {
if delta.DefaultRouteInterface == "" && runtime.GOOS != "plan9" {
return true
}
if !delta.New.HasIP(lip) && delta.Old.HasIP(lip) {
if delta.InterfaceIPDisappeared(lip) {
// Our interface with this source IP went away.
return true
}

View File

@@ -1349,20 +1349,18 @@ func (e *userspaceEngine) Done() <-chan struct{} {
}
func (e *userspaceEngine) linkChange(delta *netmon.ChangeDelta) {
changed := delta.Major // TODO(bradfitz): ask more specific questions?
cur := delta.New
up := cur.AnyInterfaceUp()
up := delta.AnyInterfaceUp()
if !up {
e.logf("LinkChange: all links down; pausing: %v", cur)
} else if changed {
e.logf("LinkChange: major, rebinding. New state: %v", cur)
e.logf("LinkChange: all links down; pausing: %v", delta.StateDesc())
} else if delta.RebindLikelyRequired {
e.logf("LinkChange: major, rebinding: %v", delta.StateDesc())
} else {
e.logf("[v1] LinkChange: minor")
}
e.health.SetAnyInterfaceUp(up)
e.magicConn.SetNetworkUp(up)
if !up || changed {
if !up || delta.RebindLikelyRequired {
if err := e.dns.FlushCaches(); err != nil {
e.logf("wgengine: dns flush failed after major link change: %v", err)
}
@@ -1372,9 +1370,20 @@ func (e *userspaceEngine) linkChange(delta *netmon.ChangeDelta) {
// suspend/resume or whenever NetworkManager is started, it
// nukes all systemd-resolved configs. So reapply our DNS
// config on major link change.
// TODO: explain why this is ncessary not just on Linux but also android
// and Apple platforms.
if changed {
//
// On Darwin (netext), we reapply the DNS config when the interface flaps
// because the change in interface can potentially change the nameservers
// for the forwarder. On Darwin netext clients, magicDNS is ~always the default
// resolver so having no nameserver to forward queries to (or one on a network we
// are not currently on) breaks DNS resolution system-wide. There are notable
// timing issues here with Darwin's network stack. It is not guaranteed that
// the forward resolver will be available immediately after the interface
// comes up. We leave it to the network extension to also poke magicDNS directly
// via [dns.Manager.RecompileDNSConfig] when it detects any change in the
// nameservers.
//
// TODO: On Android, Darwin-tailscaled, and openbsd, why do we need this?
if delta.RebindLikelyRequired && up {
switch runtime.GOOS {
case "linux", "android", "ios", "darwin", "openbsd":
e.wgLock.Lock()
@@ -1392,15 +1401,23 @@ func (e *userspaceEngine) linkChange(delta *netmon.ChangeDelta) {
}
}
e.magicConn.SetNetworkUp(up)
why := "link-change-minor"
if changed {
if delta.RebindLikelyRequired {
why = "link-change-major"
metricNumMajorChanges.Add(1)
e.magicConn.Rebind()
} else {
metricNumMinorChanges.Add(1)
}
e.magicConn.ReSTUN(why)
// If we're up and it's a minor change, just send a STUN ping
if up {
if delta.RebindLikelyRequired {
e.magicConn.Rebind()
}
e.magicConn.ReSTUN(why)
}
}
func (e *userspaceEngine) SetNetworkMap(nm *netmap.NetworkMap) {