mirror of
https://github.com/tailscale/tailscale.git
synced 2025-12-01 09:32:08 +00:00
all: move network monitoring from wgengine/monitor to net/netmon
We're using it in more and more places, and it's not really specific to our use of Wireguard (and does more just link/interface monitoring). Also removes the separate interface we had for it in sockstats -- it's a small enough package (we already pull in all of its dependencies via other paths) that it's not worth the extra complexity. Updates #7621 Updates #7850 Signed-off-by: Mihai Parparita <mihai@tailscale.com>
This commit is contained in:
committed by
Mihai Parparita
parent
3ede3aafe4
commit
4722f7e322
@@ -39,10 +39,10 @@ func setupWGTest(b *testing.B, logf logger.Logf, traf *TrafficGen, a1, a2 netip.
|
||||
traf: traf,
|
||||
}
|
||||
e1, err := wgengine.NewUserspaceEngine(l1, wgengine.Config{
|
||||
Router: router.NewFake(l1),
|
||||
LinkMonitor: nil,
|
||||
ListenPort: 0,
|
||||
Tun: t1,
|
||||
Router: router.NewFake(l1),
|
||||
NetMon: nil,
|
||||
ListenPort: 0,
|
||||
Tun: t1,
|
||||
})
|
||||
if err != nil {
|
||||
log.Fatalf("e1 init: %v", err)
|
||||
@@ -63,10 +63,10 @@ func setupWGTest(b *testing.B, logf logger.Logf, traf *TrafficGen, a1, a2 netip.
|
||||
traf: traf,
|
||||
}
|
||||
e2, err := wgengine.NewUserspaceEngine(l2, wgengine.Config{
|
||||
Router: router.NewFake(l2),
|
||||
LinkMonitor: nil,
|
||||
ListenPort: 0,
|
||||
Tun: t2,
|
||||
Router: router.NewFake(l2),
|
||||
NetMon: nil,
|
||||
ListenPort: 0,
|
||||
Tun: t2,
|
||||
})
|
||||
if err != nil {
|
||||
log.Fatalf("e2 init: %v", err)
|
||||
|
||||
@@ -47,6 +47,7 @@ import (
|
||||
"tailscale.com/net/netaddr"
|
||||
"tailscale.com/net/netcheck"
|
||||
"tailscale.com/net/neterror"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/net/netns"
|
||||
"tailscale.com/net/packet"
|
||||
"tailscale.com/net/portmapper"
|
||||
@@ -69,7 +70,6 @@ import (
|
||||
"tailscale.com/util/uniq"
|
||||
"tailscale.com/version"
|
||||
"tailscale.com/wgengine/capture"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
const (
|
||||
@@ -279,7 +279,7 @@ type Conn struct {
|
||||
idleFunc func() time.Duration // nil means unknown
|
||||
testOnlyPacketListener nettype.PacketListener
|
||||
noteRecvActivity func(key.NodePublic) // or nil, see Options.NoteRecvActivity
|
||||
linkMon *monitor.Mon // or nil
|
||||
netMon *netmon.Monitor // or nil
|
||||
|
||||
// ================================================================
|
||||
// No locking required to access these fields, either because
|
||||
@@ -573,9 +573,9 @@ type Options struct {
|
||||
// not hold Conn.mu while calling it.
|
||||
NoteRecvActivity func(key.NodePublic)
|
||||
|
||||
// LinkMonitor is the link monitor to use.
|
||||
// NetMon is the network monitor to use.
|
||||
// With one, the portmapper won't be used.
|
||||
LinkMonitor *monitor.Mon
|
||||
NetMon *netmon.Monitor
|
||||
}
|
||||
|
||||
func (o *Options) logf() logger.Logf {
|
||||
@@ -643,10 +643,10 @@ func NewConn(opts Options) (*Conn, error) {
|
||||
c.testOnlyPacketListener = opts.TestOnlyPacketListener
|
||||
c.noteRecvActivity = opts.NoteRecvActivity
|
||||
c.portMapper = portmapper.NewClient(logger.WithPrefix(c.logf, "portmapper: "), nil, c.onPortMapChanged)
|
||||
if opts.LinkMonitor != nil {
|
||||
c.portMapper.SetGatewayLookupFunc(opts.LinkMonitor.GatewayAndSelfIP)
|
||||
if opts.NetMon != nil {
|
||||
c.portMapper.SetGatewayLookupFunc(opts.NetMon.GatewayAndSelfIP)
|
||||
}
|
||||
c.linkMon = opts.LinkMonitor
|
||||
c.netMon = opts.NetMon
|
||||
|
||||
if err := c.rebind(keepCurrentPort); err != nil {
|
||||
return nil, err
|
||||
@@ -3369,8 +3369,8 @@ func (c *Conn) Rebind() {
|
||||
}
|
||||
|
||||
var ifIPs []netip.Prefix
|
||||
if c.linkMon != nil {
|
||||
st := c.linkMon.InterfaceState()
|
||||
if c.netMon != nil {
|
||||
st := c.netMon.InterfaceState()
|
||||
defIf := st.DefaultRouteInterface
|
||||
ifIPs = st.InterfaceIPs[defIf]
|
||||
c.logf("Rebind; defIf=%q, ips=%v", defIf, ifIPs)
|
||||
|
||||
@@ -1,390 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
// Package monitor provides facilities for monitoring network
|
||||
// interface and route changes. It primarily exists to know when
|
||||
// portable devices move between different networks.
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"net/netip"
|
||||
"runtime"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"tailscale.com/net/interfaces"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/util/set"
|
||||
)
|
||||
|
||||
// pollWallTimeInterval is how often we check the time to check
|
||||
// for big jumps in wall (non-monotonic) time as a backup mechanism
|
||||
// to get notified of a sleeping device waking back up.
|
||||
// Usually there are also minor network change events on wake that let
|
||||
// us check the wall time sooner than this.
|
||||
const pollWallTimeInterval = 15 * time.Second
|
||||
|
||||
// message represents a message returned from an osMon.
|
||||
type message interface {
|
||||
// Ignore is whether we should ignore this message.
|
||||
ignore() bool
|
||||
}
|
||||
|
||||
// osMon is the interface that each operating system-specific
|
||||
// implementation of the link monitor must implement.
|
||||
type osMon interface {
|
||||
Close() error
|
||||
|
||||
// Receive returns a new network interface change message. It
|
||||
// should block until there's either something to return, or
|
||||
// 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
|
||||
}
|
||||
|
||||
// Mon represents a monitoring instance.
|
||||
type Mon struct {
|
||||
logf logger.Logf
|
||||
om osMon // nil means not supported on this platform
|
||||
change chan struct{}
|
||||
stop chan struct{} // closed on Stop
|
||||
|
||||
mu sync.Mutex // guards all following fields
|
||||
cbs set.HandleSet[interfaces.ChangeFunc]
|
||||
ruleDelCB set.HandleSet[RuleDeleteCallback]
|
||||
ifState *interfaces.State
|
||||
gwValid bool // whether gw and gwSelfIP are valid
|
||||
gw netip.Addr // our gateway's IP
|
||||
gwSelfIP netip.Addr // our own IP address (that corresponds to gw)
|
||||
started bool
|
||||
closed bool
|
||||
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
|
||||
}
|
||||
|
||||
// New instantiates and starts a monitoring instance.
|
||||
// The returned monitor is inactive until it's started by the Start method.
|
||||
// Use RegisterChangeCallback to get notified of network changes.
|
||||
func New(logf logger.Logf) (*Mon, error) {
|
||||
logf = logger.WithPrefix(logf, "monitor: ")
|
||||
m := &Mon{
|
||||
logf: logf,
|
||||
change: make(chan struct{}, 1),
|
||||
stop: make(chan struct{}),
|
||||
lastWall: wallTime(),
|
||||
}
|
||||
st, err := m.interfaceStateUncached()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
m.ifState = st
|
||||
|
||||
m.om, err = newOSMon(logf, m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if m.om == nil {
|
||||
return nil, errors.New("newOSMon returned nil, nil")
|
||||
}
|
||||
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// InterfaceState returns the latest snapshot of the machine's network
|
||||
// interfaces.
|
||||
//
|
||||
// The returned value is owned by Mon; it must not be modified.
|
||||
func (m *Mon) InterfaceState() *interfaces.State {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
return m.ifState
|
||||
}
|
||||
|
||||
func (m *Mon) interfaceStateUncached() (*interfaces.State, error) {
|
||||
return interfaces.GetState()
|
||||
}
|
||||
|
||||
// GatewayAndSelfIP returns the current network's default gateway, and
|
||||
// the machine's default IP for that gateway.
|
||||
//
|
||||
// It's the same as interfaces.LikelyHomeRouterIP, but it caches the
|
||||
// result until the monitor detects a network change.
|
||||
func (m *Mon) GatewayAndSelfIP() (gw, myIP netip.Addr, ok bool) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
if m.gwValid {
|
||||
return m.gw, m.gwSelfIP, true
|
||||
}
|
||||
gw, myIP, ok = interfaces.LikelyHomeRouterIP()
|
||||
if ok {
|
||||
m.gw, m.gwSelfIP, m.gwValid = gw, myIP, true
|
||||
}
|
||||
return gw, myIP, ok
|
||||
}
|
||||
|
||||
// RegisterChangeCallback adds callback to the set of parties to be
|
||||
// notified (in their own goroutine) when the network state changes.
|
||||
// To remove this callback, call unregister (or close the monitor).
|
||||
func (m *Mon) RegisterChangeCallback(callback interfaces.ChangeFunc) (unregister func()) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
handle := m.cbs.Add(callback)
|
||||
return func() {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
delete(m.cbs, handle)
|
||||
}
|
||||
}
|
||||
|
||||
// RuleDeleteCallback is a callback when a Linux IP policy routing
|
||||
// rule is deleted. The table is the table number (52, 253, 354) and
|
||||
// priority is the priority order number (for Tailscale rules
|
||||
// currently: 5210, 5230, 5250, 5270)
|
||||
type RuleDeleteCallback func(table uint8, priority uint32)
|
||||
|
||||
// RegisterRuleDeleteCallback adds callback to the set of parties to be
|
||||
// notified (in their own goroutine) when a Linux ip rule is deleted.
|
||||
// To remove this callback, call unregister (or close the monitor).
|
||||
func (m *Mon) RegisterRuleDeleteCallback(callback RuleDeleteCallback) (unregister func()) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
handle := m.ruleDelCB.Add(callback)
|
||||
return func() {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
delete(m.ruleDelCB, handle)
|
||||
}
|
||||
}
|
||||
|
||||
// Start starts the monitor.
|
||||
// A monitor can only be started & closed once.
|
||||
func (m *Mon) Start() {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
if m.started || m.closed {
|
||||
return
|
||||
}
|
||||
m.started = true
|
||||
|
||||
if shouldMonitorTimeJump {
|
||||
m.wallTimer = time.AfterFunc(pollWallTimeInterval, m.pollWallTime)
|
||||
}
|
||||
|
||||
if m.om == nil {
|
||||
return
|
||||
}
|
||||
m.goroutines.Add(2)
|
||||
go m.pump()
|
||||
go m.debounce()
|
||||
}
|
||||
|
||||
// Close closes the monitor.
|
||||
func (m *Mon) Close() error {
|
||||
m.mu.Lock()
|
||||
if m.closed {
|
||||
m.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
m.closed = true
|
||||
close(m.stop)
|
||||
|
||||
if m.wallTimer != nil {
|
||||
m.wallTimer.Stop()
|
||||
}
|
||||
|
||||
var err error
|
||||
if m.om != nil {
|
||||
err = m.om.Close()
|
||||
}
|
||||
|
||||
started := m.started
|
||||
m.mu.Unlock()
|
||||
|
||||
if started {
|
||||
m.goroutines.Wait()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// InjectEvent forces the monitor to pretend there was a network
|
||||
// change and re-check the state of the network. Any registered
|
||||
// ChangeFunc callbacks will be called within the event coalescing
|
||||
// period (under a fraction of a second).
|
||||
func (m *Mon) InjectEvent() {
|
||||
select {
|
||||
case m.change <- struct{}{}:
|
||||
default:
|
||||
// Another change signal is already
|
||||
// buffered. Debounce will wake up soon
|
||||
// enough.
|
||||
}
|
||||
}
|
||||
|
||||
func (m *Mon) stopped() bool {
|
||||
select {
|
||||
case <-m.stop:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// pump continuously retrieves messages from the connection, notifying
|
||||
// the change channel of changes, and stopping when a stop is issued.
|
||||
func (m *Mon) pump() {
|
||||
defer m.goroutines.Done()
|
||||
for !m.stopped() {
|
||||
msg, err := m.om.Receive()
|
||||
if err != nil {
|
||||
if m.stopped() {
|
||||
return
|
||||
}
|
||||
// Keep retrying while we're not closed.
|
||||
m.logf("error from link monitor: %v", err)
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
if rdm, ok := msg.(ipRuleDeletedMessage); ok {
|
||||
m.notifyRuleDeleted(rdm)
|
||||
continue
|
||||
}
|
||||
if msg.ignore() {
|
||||
continue
|
||||
}
|
||||
m.InjectEvent()
|
||||
}
|
||||
}
|
||||
|
||||
func (m *Mon) notifyRuleDeleted(rdm ipRuleDeletedMessage) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
for _, cb := range m.ruleDelCB {
|
||||
go cb(rdm.table, rdm.priority)
|
||||
}
|
||||
}
|
||||
|
||||
// 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 *Mon) isInterestingInterface(i interfaces.Interface, ips []netip.Prefix) bool {
|
||||
return m.om.IsInterestingInterface(i.Name) && interfaces.UseInterestingInterfaces(i, ips)
|
||||
}
|
||||
|
||||
// debounce calls the callback function with a delay between events
|
||||
// and exits when a stop is issued.
|
||||
func (m *Mon) debounce() {
|
||||
defer m.goroutines.Done()
|
||||
for {
|
||||
select {
|
||||
case <-m.stop:
|
||||
return
|
||||
case <-m.change:
|
||||
}
|
||||
|
||||
if curState, err := m.interfaceStateUncached(); err != nil {
|
||||
m.logf("interfaces.State: %v", err)
|
||||
} else {
|
||||
m.mu.Lock()
|
||||
|
||||
oldState := m.ifState
|
||||
changed := !curState.EqualFiltered(oldState, m.isInterestingInterface, interfaces.UseInterestingIPs)
|
||||
if changed {
|
||||
m.gwValid = false
|
||||
m.ifState = curState
|
||||
|
||||
if s1, s2 := oldState.String(), curState.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(curState))
|
||||
}
|
||||
}
|
||||
// See if we have a queued or new time jump signal.
|
||||
if shouldMonitorTimeJump && m.checkWallTimeAdvanceLocked() {
|
||||
m.resetTimeJumpedLocked()
|
||||
if !changed {
|
||||
// Only log if it wasn't an interesting change.
|
||||
m.logf("time jumped (probably wake from sleep); synthesizing major change event")
|
||||
changed = true
|
||||
}
|
||||
}
|
||||
for _, cb := range m.cbs {
|
||||
go cb(changed, m.ifState)
|
||||
}
|
||||
m.mu.Unlock()
|
||||
}
|
||||
|
||||
select {
|
||||
case <-m.stop:
|
||||
return
|
||||
case <-time.After(250 * time.Millisecond):
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func jsonSummary(x any) any {
|
||||
j, err := json.Marshal(x)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return j
|
||||
}
|
||||
|
||||
func wallTime() time.Time {
|
||||
// From time package's docs: "The canonical way to strip a
|
||||
// monotonic clock reading is to use t = t.Round(0)."
|
||||
return time.Now().Round(0)
|
||||
}
|
||||
|
||||
func (m *Mon) pollWallTime() {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
if m.closed {
|
||||
return
|
||||
}
|
||||
if m.checkWallTimeAdvanceLocked() {
|
||||
m.InjectEvent()
|
||||
}
|
||||
m.wallTimer.Reset(pollWallTimeInterval)
|
||||
}
|
||||
|
||||
// shouldMonitorTimeJump is whether we keep a regular periodic timer running in
|
||||
// the background watching for jumps in wall time.
|
||||
//
|
||||
// We don't do this on mobile platforms for battery reasons, and because these
|
||||
// platforms don't really sleep in the same way.
|
||||
const shouldMonitorTimeJump = runtime.GOOS != "android" && runtime.GOOS != "ios"
|
||||
|
||||
// checkWallTimeAdvanceLocked reports whether wall time jumped more than 150% of
|
||||
// pollWallTimeInterval, indicating we probably just came out of sleep. Once a
|
||||
// time jump is detected it must be reset by calling resetTimeJumpedLocked.
|
||||
func (m *Mon) checkWallTimeAdvanceLocked() bool {
|
||||
if !shouldMonitorTimeJump {
|
||||
panic("unreachable") // if callers are correct
|
||||
}
|
||||
now := wallTime()
|
||||
if now.Sub(m.lastWall) > pollWallTimeInterval*3/2 {
|
||||
m.timeJumped = true // it is reset by debounce.
|
||||
}
|
||||
m.lastWall = now
|
||||
return m.timeJumped
|
||||
}
|
||||
|
||||
// resetTimeJumpedLocked consumes the signal set by checkWallTimeAdvanceLocked.
|
||||
func (m *Mon) resetTimeJumpedLocked() {
|
||||
m.timeJumped = false
|
||||
}
|
||||
|
||||
type ipRuleDeletedMessage struct {
|
||||
table uint8
|
||||
priority uint32
|
||||
}
|
||||
|
||||
func (ipRuleDeletedMessage) ignore() bool { return true }
|
||||
@@ -1,225 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/netip"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"golang.org/x/net/route"
|
||||
"golang.org/x/sys/unix"
|
||||
"tailscale.com/net/netaddr"
|
||||
"tailscale.com/types/logger"
|
||||
)
|
||||
|
||||
const debugRouteMessages = false
|
||||
|
||||
// unspecifiedMessage is a minimal message implementation that should not
|
||||
// be ignored. In general, OS-specific implementations should use better
|
||||
// types and avoid this if they can.
|
||||
type unspecifiedMessage struct{}
|
||||
|
||||
func (unspecifiedMessage) ignore() bool { return false }
|
||||
|
||||
func newOSMon(logf logger.Logf, _ *Mon) (osMon, error) {
|
||||
fd, err := unix.Socket(unix.AF_ROUTE, unix.SOCK_RAW, 0)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &darwinRouteMon{
|
||||
logf: logf,
|
||||
fd: fd,
|
||||
}, nil
|
||||
}
|
||||
|
||||
type darwinRouteMon struct {
|
||||
logf logger.Logf
|
||||
fd int // AF_ROUTE socket
|
||||
buf [2 << 10]byte
|
||||
closeOnce sync.Once
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) Close() error {
|
||||
var err error
|
||||
m.closeOnce.Do(func() {
|
||||
err = unix.Close(m.fd)
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) Receive() (message, error) {
|
||||
for {
|
||||
n, err := unix.Read(m.fd, m.buf[:])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
msgs, err := route.ParseRIB(route.RIBTypeRoute, m.buf[:n])
|
||||
if err != nil {
|
||||
if debugRouteMessages {
|
||||
m.logf("read %d bytes (% 02x), failed to parse RIB: %v", n, m.buf[:n], err)
|
||||
}
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
if len(msgs) == 0 {
|
||||
if debugRouteMessages {
|
||||
m.logf("read %d bytes with no messages (% 02x)", n, m.buf[:n])
|
||||
}
|
||||
continue
|
||||
}
|
||||
nSkip := 0
|
||||
for _, msg := range msgs {
|
||||
if m.skipMessage(msg) {
|
||||
nSkip++
|
||||
}
|
||||
}
|
||||
if debugRouteMessages {
|
||||
m.logf("read %d bytes, %d messages (%d skipped)", n, len(msgs), nSkip)
|
||||
if nSkip < len(msgs) {
|
||||
m.logMessages(msgs)
|
||||
}
|
||||
}
|
||||
if nSkip == len(msgs) {
|
||||
continue
|
||||
}
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) skipMessage(msg route.Message) bool {
|
||||
switch msg := msg.(type) {
|
||||
case *route.InterfaceMulticastAddrMessage:
|
||||
return true
|
||||
case *route.InterfaceAddrMessage:
|
||||
return m.skipInterfaceAddrMessage(msg)
|
||||
case *route.RouteMessage:
|
||||
return m.skipRouteMessage(msg)
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// addrType returns addrs[rtaxType], if that (the route address type) exists,
|
||||
// else it returns nil.
|
||||
//
|
||||
// The RTAX_* constants at https://github.com/apple/darwin-xnu/blob/main/bsd/net/route.h
|
||||
// for what each address index represents.
|
||||
func addrType(addrs []route.Addr, rtaxType int) route.Addr {
|
||||
if len(addrs) > rtaxType {
|
||||
return addrs[rtaxType]
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) 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":
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
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) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) skipRouteMessage(msg *route.RouteMessage) bool {
|
||||
if ip := ipOfAddr(addrType(msg.Addrs, unix.RTAX_DST)); ip.IsLinkLocalUnicast() {
|
||||
// Skip those like:
|
||||
// dst = fe80::b476:66ff:fe30:c8f6%15
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) logMessages(msgs []route.Message) {
|
||||
for i, msg := range msgs {
|
||||
switch msg := msg.(type) {
|
||||
default:
|
||||
m.logf(" [%d] %T", i, msg)
|
||||
case *route.InterfaceAddrMessage:
|
||||
m.logf(" [%d] InterfaceAddrMessage: ver=%d, type=%v, flags=0x%x, idx=%v",
|
||||
i, msg.Version, msg.Type, msg.Flags, msg.Index)
|
||||
m.logAddrs(msg.Addrs)
|
||||
case *route.InterfaceMulticastAddrMessage:
|
||||
m.logf(" [%d] InterfaceMulticastAddrMessage: ver=%d, type=%v, flags=0x%x, idx=%v",
|
||||
i, msg.Version, msg.Type, msg.Flags, msg.Index)
|
||||
m.logAddrs(msg.Addrs)
|
||||
case *route.RouteMessage:
|
||||
m.logf(" [%d] RouteMessage: ver=%d, type=%v, flags=0x%x, idx=%v, id=%v, seq=%v, err=%v",
|
||||
i, msg.Version, msg.Type, msg.Flags, msg.Index, msg.ID, msg.Seq, msg.Err)
|
||||
m.logAddrs(msg.Addrs)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (m *darwinRouteMon) logAddrs(addrs []route.Addr) {
|
||||
for i, a := range addrs {
|
||||
if a == nil {
|
||||
continue
|
||||
}
|
||||
m.logf(" %v = %v", rtaxName(i), fmtAddr(a))
|
||||
}
|
||||
}
|
||||
|
||||
// ipOfAddr returns the route.Addr (possibly nil) as a netip.Addr
|
||||
// (possibly zero).
|
||||
func ipOfAddr(a route.Addr) netip.Addr {
|
||||
switch a := a.(type) {
|
||||
case *route.Inet4Addr:
|
||||
return netaddr.IPv4(a.IP[0], a.IP[1], a.IP[2], a.IP[3])
|
||||
case *route.Inet6Addr:
|
||||
ip := netip.AddrFrom16(a.IP)
|
||||
if a.ZoneID != 0 {
|
||||
ip = ip.WithZone(fmt.Sprint(a.ZoneID)) // TODO: look up net.InterfaceByIndex? but it might be changing?
|
||||
}
|
||||
return ip
|
||||
}
|
||||
return netip.Addr{}
|
||||
}
|
||||
|
||||
func fmtAddr(a route.Addr) any {
|
||||
if a == nil {
|
||||
return nil
|
||||
}
|
||||
if ip := ipOfAddr(a); ip.IsValid() {
|
||||
return ip
|
||||
}
|
||||
switch a := a.(type) {
|
||||
case *route.LinkAddr:
|
||||
return fmt.Sprintf("[LinkAddr idx=%v name=%q addr=%x]", a.Index, a.Name, a.Addr)
|
||||
default:
|
||||
return fmt.Sprintf("%T: %+v", a, a)
|
||||
}
|
||||
}
|
||||
|
||||
// See https://github.com/apple/darwin-xnu/blob/main/bsd/net/route.h
|
||||
func rtaxName(i int) string {
|
||||
switch i {
|
||||
case unix.RTAX_DST:
|
||||
return "dst"
|
||||
case unix.RTAX_GATEWAY:
|
||||
return "gateway"
|
||||
case unix.RTAX_NETMASK:
|
||||
return "netmask"
|
||||
case unix.RTAX_GENMASK:
|
||||
return "genmask"
|
||||
case unix.RTAX_IFP: // "interface name sockaddr present"
|
||||
return "IFP"
|
||||
case unix.RTAX_IFA: // "interface addr sockaddr present"
|
||||
return "IFA"
|
||||
case unix.RTAX_AUTHOR:
|
||||
return "author"
|
||||
case unix.RTAX_BRD:
|
||||
return "BRD"
|
||||
}
|
||||
return fmt.Sprint(i)
|
||||
}
|
||||
@@ -1,27 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"golang.org/x/net/route"
|
||||
)
|
||||
|
||||
func TestIssue1416RIB(t *testing.T) {
|
||||
const ribHex = `32 00 05 10 30 00 00 00 00 00 00 00 04 00 00 00 14 12 04 00 06 03 06 00 65 6e 30 ac 87 a3 19 7f 82 00 00 00 0e 12 00 00 00 00 06 00 91 e0 f0 01 00 00`
|
||||
rtmMsg, err := hex.DecodeString(strings.ReplaceAll(ribHex, " ", ""))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
msgs, err := route.ParseRIB(route.RIBTypeRoute, rtmMsg)
|
||||
if err != nil {
|
||||
t.Logf("ParseRIB: %v", err)
|
||||
t.Skip("skipping on known failure; see https://github.com/tailscale/tailscale/issues/1416")
|
||||
t.Fatal(err)
|
||||
}
|
||||
t.Logf("Got: %#v", msgs)
|
||||
}
|
||||
@@ -1,56 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"fmt"
|
||||
"net"
|
||||
"strings"
|
||||
|
||||
"tailscale.com/types/logger"
|
||||
)
|
||||
|
||||
// unspecifiedMessage is a minimal message implementation that should not
|
||||
// be ignored. In general, OS-specific implementations should use better
|
||||
// types and avoid this if they can.
|
||||
type unspecifiedMessage struct{}
|
||||
|
||||
func (unspecifiedMessage) ignore() bool { return false }
|
||||
|
||||
// devdConn implements osMon using devd(8).
|
||||
type devdConn struct {
|
||||
conn net.Conn
|
||||
}
|
||||
|
||||
func newOSMon(logf logger.Logf, m *Mon) (osMon, error) {
|
||||
conn, err := net.Dial("unixpacket", "/var/run/devd.seqpacket.pipe")
|
||||
if err != nil {
|
||||
logf("devd dial error: %v, falling back to polling method", err)
|
||||
return newPollingMon(logf, m)
|
||||
}
|
||||
return &devdConn{conn}, nil
|
||||
}
|
||||
|
||||
func (c *devdConn) IsInterestingInterface(iface string) bool { return true }
|
||||
|
||||
func (c *devdConn) Close() error {
|
||||
return c.conn.Close()
|
||||
}
|
||||
|
||||
func (c *devdConn) Receive() (message, error) {
|
||||
for {
|
||||
msg, err := bufio.NewReader(c.conn).ReadString('\n')
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("reading devd socket: %v", err)
|
||||
}
|
||||
// Only return messages related to the network subsystem.
|
||||
if !strings.Contains(msg, "system=IFNET") {
|
||||
continue
|
||||
}
|
||||
// TODO: this is where the devd-specific message would
|
||||
// get converted into a "standard" event message and returned.
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
}
|
||||
@@ -1,290 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
//go:build !android
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"net"
|
||||
"net/netip"
|
||||
"time"
|
||||
|
||||
"github.com/jsimonetti/rtnetlink"
|
||||
"github.com/mdlayher/netlink"
|
||||
"golang.org/x/sys/unix"
|
||||
"tailscale.com/envknob"
|
||||
"tailscale.com/net/tsaddr"
|
||||
"tailscale.com/types/logger"
|
||||
)
|
||||
|
||||
var debugNetlinkMessages = envknob.RegisterBool("TS_DEBUG_NETLINK")
|
||||
|
||||
// unspecifiedMessage is a minimal message implementation that should not
|
||||
// be ignored. In general, OS-specific implementations should use better
|
||||
// types and avoid this if they can.
|
||||
type unspecifiedMessage struct{}
|
||||
|
||||
func (unspecifiedMessage) ignore() bool { return false }
|
||||
|
||||
// nlConn wraps a *netlink.Conn and returns a monitor.Message
|
||||
// instead of a netlink.Message. Currently, messages are discarded,
|
||||
// but down the line, when messages trigger different logic depending
|
||||
// on the type of event, this provides the capability of handling
|
||||
// each architecture-specific message in a generic fashion.
|
||||
type nlConn struct {
|
||||
logf logger.Logf
|
||||
conn *netlink.Conn
|
||||
buffered []netlink.Message
|
||||
|
||||
// addrCache maps interface indices to a set of addresses, and is
|
||||
// used to suppress duplicate RTM_NEWADDR messages. It is populated
|
||||
// by RTM_NEWADDR messages and de-populated by RTM_DELADDR. See
|
||||
// issue #4282.
|
||||
addrCache map[uint32]map[netip.Addr]bool
|
||||
}
|
||||
|
||||
func newOSMon(logf logger.Logf, m *Mon) (osMon, error) {
|
||||
conn, err := netlink.Dial(unix.NETLINK_ROUTE, &netlink.Config{
|
||||
// Routes get us most of the events of interest, but we need
|
||||
// address as well to cover things like DHCP deciding to give
|
||||
// us a new address upon renewal - routing wouldn't change,
|
||||
// but all reachability would.
|
||||
Groups: unix.RTMGRP_IPV4_IFADDR | unix.RTMGRP_IPV6_IFADDR |
|
||||
unix.RTMGRP_IPV4_ROUTE | unix.RTMGRP_IPV6_ROUTE |
|
||||
unix.RTMGRP_IPV4_RULE, // no IPV6_RULE in x/sys/unix
|
||||
})
|
||||
if err != nil {
|
||||
// Google Cloud Run does not implement NETLINK_ROUTE RTMGRP support
|
||||
logf("monitor_linux: AF_NETLINK RTMGRP failed, falling back to polling")
|
||||
return newPollingMon(logf, m)
|
||||
}
|
||||
return &nlConn{logf: logf, conn: conn, addrCache: make(map[uint32]map[netip.Addr]bool)}, nil
|
||||
}
|
||||
|
||||
func (c *nlConn) IsInterestingInterface(iface string) bool { return true }
|
||||
|
||||
func (c *nlConn) Close() error { return c.conn.Close() }
|
||||
|
||||
func (c *nlConn) Receive() (message, error) {
|
||||
if len(c.buffered) == 0 {
|
||||
var err error
|
||||
c.buffered, err = c.conn.Receive()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(c.buffered) == 0 {
|
||||
// Unexpected. Not seen in wild, but sleep defensively.
|
||||
time.Sleep(time.Second)
|
||||
return ignoreMessage{}, nil
|
||||
}
|
||||
}
|
||||
msg := c.buffered[0]
|
||||
c.buffered = c.buffered[1:]
|
||||
|
||||
// See https://github.com/torvalds/linux/blob/master/include/uapi/linux/rtnetlink.h
|
||||
// And https://man7.org/linux/man-pages/man7/rtnetlink.7.html
|
||||
switch msg.Header.Type {
|
||||
case unix.RTM_NEWADDR, unix.RTM_DELADDR:
|
||||
var rmsg rtnetlink.AddressMessage
|
||||
if err := rmsg.UnmarshalBinary(msg.Data); err != nil {
|
||||
c.logf("failed to parse type %v: %v", msg.Header.Type, err)
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
|
||||
nip := netaddrIP(rmsg.Attributes.Address)
|
||||
|
||||
if debugNetlinkMessages() {
|
||||
typ := "RTM_NEWADDR"
|
||||
if msg.Header.Type == unix.RTM_DELADDR {
|
||||
typ = "RTM_DELADDR"
|
||||
}
|
||||
|
||||
// label attributes are seemingly only populated for ipv4 addresses in the wild.
|
||||
label := rmsg.Attributes.Label
|
||||
if label == "" {
|
||||
itf, err := net.InterfaceByIndex(int(rmsg.Index))
|
||||
if err == nil {
|
||||
label = itf.Name
|
||||
}
|
||||
}
|
||||
|
||||
c.logf("%s: %s(%d) %s / %s", typ, label, rmsg.Index, rmsg.Attributes.Address, rmsg.Attributes.Local)
|
||||
}
|
||||
|
||||
addrs := c.addrCache[rmsg.Index]
|
||||
|
||||
// Ignore duplicate RTM_NEWADDR messages using c.addrCache to
|
||||
// detect them. See nlConn.addrcache and issue #4282.
|
||||
if msg.Header.Type == unix.RTM_NEWADDR {
|
||||
if addrs == nil {
|
||||
addrs = make(map[netip.Addr]bool)
|
||||
c.addrCache[rmsg.Index] = addrs
|
||||
}
|
||||
|
||||
if addrs[nip] {
|
||||
if debugNetlinkMessages() {
|
||||
c.logf("ignored duplicate RTM_NEWADDR for %s", nip)
|
||||
}
|
||||
return ignoreMessage{}, nil
|
||||
}
|
||||
|
||||
addrs[nip] = true
|
||||
} else { // msg.Header.Type == unix.RTM_DELADDR
|
||||
if addrs != nil {
|
||||
delete(addrs, nip)
|
||||
}
|
||||
|
||||
if len(addrs) == 0 {
|
||||
delete(c.addrCache, rmsg.Index)
|
||||
}
|
||||
}
|
||||
|
||||
nam := &newAddrMessage{
|
||||
IfIndex: rmsg.Index,
|
||||
Addr: nip,
|
||||
Delete: msg.Header.Type == unix.RTM_DELADDR,
|
||||
}
|
||||
if debugNetlinkMessages() {
|
||||
c.logf("%+v", nam)
|
||||
}
|
||||
return nam, nil
|
||||
case unix.RTM_NEWROUTE, unix.RTM_DELROUTE:
|
||||
typeStr := "RTM_NEWROUTE"
|
||||
if msg.Header.Type == unix.RTM_DELROUTE {
|
||||
typeStr = "RTM_DELROUTE"
|
||||
}
|
||||
var rmsg rtnetlink.RouteMessage
|
||||
if err := rmsg.UnmarshalBinary(msg.Data); err != nil {
|
||||
c.logf("%s: failed to parse: %v", typeStr, err)
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
src := netaddrIPPrefix(rmsg.Attributes.Src, rmsg.SrcLength)
|
||||
dst := netaddrIPPrefix(rmsg.Attributes.Dst, rmsg.DstLength)
|
||||
gw := netaddrIP(rmsg.Attributes.Gateway)
|
||||
|
||||
if msg.Header.Type == unix.RTM_NEWROUTE &&
|
||||
(rmsg.Attributes.Table == 255 || rmsg.Attributes.Table == 254) &&
|
||||
(dst.Addr().IsMulticast() || dst.Addr().IsLinkLocalUnicast()) {
|
||||
|
||||
if debugNetlinkMessages() {
|
||||
c.logf("%s ignored", typeStr)
|
||||
}
|
||||
|
||||
// Normal Linux route changes on new interface coming up; don't log or react.
|
||||
return ignoreMessage{}, nil
|
||||
}
|
||||
|
||||
if rmsg.Table == tsTable && dst.IsSingleIP() {
|
||||
// Don't log. Spammy and normal to see a bunch of these on start-up,
|
||||
// which we make ourselves.
|
||||
} else if tsaddr.IsTailscaleIP(dst.Addr()) {
|
||||
// Verbose only.
|
||||
c.logf("%s: [v1] src=%v, dst=%v, gw=%v, outif=%v, table=%v", typeStr,
|
||||
condNetAddrPrefix(src), condNetAddrPrefix(dst), condNetAddrIP(gw),
|
||||
rmsg.Attributes.OutIface, rmsg.Attributes.Table)
|
||||
} else {
|
||||
c.logf("%s: src=%v, dst=%v, gw=%v, outif=%v, table=%v", typeStr,
|
||||
condNetAddrPrefix(src), condNetAddrPrefix(dst), condNetAddrIP(gw),
|
||||
rmsg.Attributes.OutIface, rmsg.Attributes.Table)
|
||||
}
|
||||
if msg.Header.Type == unix.RTM_DELROUTE {
|
||||
// Just logging it for now.
|
||||
// (Debugging https://github.com/tailscale/tailscale/issues/643)
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
|
||||
nrm := &newRouteMessage{
|
||||
Table: rmsg.Table,
|
||||
Src: src,
|
||||
Dst: dst,
|
||||
Gateway: gw,
|
||||
}
|
||||
if debugNetlinkMessages() {
|
||||
c.logf("%+v", nrm)
|
||||
}
|
||||
return nrm, nil
|
||||
case unix.RTM_NEWRULE:
|
||||
// Probably ourselves adding it.
|
||||
return ignoreMessage{}, nil
|
||||
case unix.RTM_DELRULE:
|
||||
// For https://github.com/tailscale/tailscale/issues/1591 where
|
||||
// systemd-networkd deletes our rules.
|
||||
var rmsg rtnetlink.RouteMessage
|
||||
err := rmsg.UnmarshalBinary(msg.Data)
|
||||
if err != nil {
|
||||
c.logf("ip rule deleted; failed to parse netlink message: %v", err)
|
||||
} else {
|
||||
c.logf("ip rule deleted: %+v", rmsg)
|
||||
// On `ip -4 rule del pref 5210 table main`, logs:
|
||||
// monitor: ip rule deleted: {Family:2 DstLength:0 SrcLength:0 Tos:0 Table:254 Protocol:0 Scope:0 Type:1 Flags:0 Attributes:{Dst:<nil> Src:<nil> Gateway:<nil> OutIface:0 Priority:5210 Table:254 Mark:4294967295 Expires:<nil> Metrics:<nil> Multipath:[]}}
|
||||
}
|
||||
rdm := ipRuleDeletedMessage{
|
||||
table: rmsg.Table,
|
||||
priority: rmsg.Attributes.Priority,
|
||||
}
|
||||
if debugNetlinkMessages() {
|
||||
c.logf("%+v", rdm)
|
||||
}
|
||||
return rdm, nil
|
||||
case unix.RTM_NEWLINK, unix.RTM_DELLINK:
|
||||
// This is an unhandled message, but don't print an error.
|
||||
// See https://github.com/tailscale/tailscale/issues/6806
|
||||
return unspecifiedMessage{}, nil
|
||||
default:
|
||||
c.logf("unhandled netlink msg type %+v, %q", msg.Header, msg.Data)
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
}
|
||||
|
||||
func netaddrIP(std net.IP) netip.Addr {
|
||||
ip, _ := netip.AddrFromSlice(std)
|
||||
return ip.Unmap()
|
||||
}
|
||||
|
||||
func netaddrIPPrefix(std net.IP, bits uint8) netip.Prefix {
|
||||
ip, _ := netip.AddrFromSlice(std)
|
||||
return netip.PrefixFrom(ip.Unmap(), int(bits))
|
||||
}
|
||||
|
||||
func condNetAddrPrefix(ipp netip.Prefix) string {
|
||||
if !ipp.Addr().IsValid() {
|
||||
return ""
|
||||
}
|
||||
return ipp.String()
|
||||
}
|
||||
|
||||
func condNetAddrIP(ip netip.Addr) string {
|
||||
if !ip.IsValid() {
|
||||
return ""
|
||||
}
|
||||
return ip.String()
|
||||
}
|
||||
|
||||
// newRouteMessage is a message for a new route being added.
|
||||
type newRouteMessage struct {
|
||||
Src, Dst netip.Prefix
|
||||
Gateway netip.Addr
|
||||
Table uint8
|
||||
}
|
||||
|
||||
const tsTable = 52
|
||||
|
||||
func (m *newRouteMessage) ignore() bool {
|
||||
return m.Table == tsTable || tsaddr.IsTailscaleIP(m.Dst.Addr())
|
||||
}
|
||||
|
||||
// newAddrMessage is a message for a new address being added.
|
||||
type newAddrMessage struct {
|
||||
Delete bool
|
||||
Addr netip.Addr
|
||||
IfIndex uint32 // interface index
|
||||
}
|
||||
|
||||
func (m *newAddrMessage) ignore() bool {
|
||||
return tsaddr.IsTailscaleIP(m.Addr)
|
||||
}
|
||||
|
||||
type ignoreMessage struct{}
|
||||
|
||||
func (ignoreMessage) ignore() bool { return true }
|
||||
@@ -1,99 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"net"
|
||||
"net/netip"
|
||||
"testing"
|
||||
|
||||
"github.com/jsimonetti/rtnetlink"
|
||||
"github.com/mdlayher/netlink"
|
||||
"golang.org/x/sys/unix"
|
||||
)
|
||||
|
||||
func newAddrMsg(iface uint32, addr string, typ netlink.HeaderType) netlink.Message {
|
||||
ip := net.ParseIP(addr)
|
||||
if ip == nil {
|
||||
panic("newAddrMsg: invalid addr: " + addr)
|
||||
}
|
||||
|
||||
addrMsg := rtnetlink.AddressMessage{
|
||||
Index: iface,
|
||||
Attributes: &rtnetlink.AddressAttributes{
|
||||
Address: ip,
|
||||
},
|
||||
}
|
||||
|
||||
b, err := addrMsg.MarshalBinary()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
return netlink.Message{
|
||||
Header: netlink.Header{Type: typ},
|
||||
Data: b,
|
||||
}
|
||||
}
|
||||
|
||||
// See issue #4282 and nlConn.addrCache.
|
||||
func TestIgnoreDuplicateNEWADDR(t *testing.T) {
|
||||
mustReceive := func(c *nlConn) message {
|
||||
msg, err := c.Receive()
|
||||
if err != nil {
|
||||
t.Fatalf("mustReceive: unwanted error: %s", err)
|
||||
}
|
||||
return msg
|
||||
}
|
||||
|
||||
t.Run("suppress duplicate NEWADDRs", func(t *testing.T) {
|
||||
c := nlConn{
|
||||
buffered: []netlink.Message{
|
||||
newAddrMsg(1, "192.168.0.5", unix.RTM_NEWADDR),
|
||||
newAddrMsg(1, "192.168.0.5", unix.RTM_NEWADDR),
|
||||
},
|
||||
addrCache: make(map[uint32]map[netip.Addr]bool),
|
||||
}
|
||||
|
||||
msg := mustReceive(&c)
|
||||
if _, ok := msg.(*newAddrMessage); !ok {
|
||||
t.Fatalf("want newAddrMessage, got %T %v", msg, msg)
|
||||
}
|
||||
|
||||
msg = mustReceive(&c)
|
||||
if _, ok := msg.(ignoreMessage); !ok {
|
||||
t.Fatalf("want ignoreMessage, got %T %v", msg, msg)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("do not suppress after DELADDR", func(t *testing.T) {
|
||||
c := nlConn{
|
||||
buffered: []netlink.Message{
|
||||
newAddrMsg(1, "192.168.0.5", unix.RTM_NEWADDR),
|
||||
newAddrMsg(1, "192.168.0.5", unix.RTM_DELADDR),
|
||||
newAddrMsg(1, "192.168.0.5", unix.RTM_NEWADDR),
|
||||
},
|
||||
addrCache: make(map[uint32]map[netip.Addr]bool),
|
||||
}
|
||||
|
||||
msg := mustReceive(&c)
|
||||
if _, ok := msg.(*newAddrMessage); !ok {
|
||||
t.Fatalf("want newAddrMessage, got %T %v", msg, msg)
|
||||
}
|
||||
|
||||
msg = mustReceive(&c)
|
||||
if m, ok := msg.(*newAddrMessage); !ok {
|
||||
t.Fatalf("want newAddrMessage, got %T %v", msg, msg)
|
||||
} else {
|
||||
if !m.Delete {
|
||||
t.Fatalf("want delete, got %#v", m)
|
||||
}
|
||||
}
|
||||
|
||||
msg = mustReceive(&c)
|
||||
if _, ok := msg.(*newAddrMessage); !ok {
|
||||
t.Fatalf("want newAddrMessage, got %T %v", msg, msg)
|
||||
}
|
||||
})
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
//go:build (!linux && !freebsd && !windows && !darwin) || android
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"tailscale.com/types/logger"
|
||||
)
|
||||
|
||||
func newOSMon(logf logger.Logf, m *Mon) (osMon, error) {
|
||||
return newPollingMon(logf, m)
|
||||
}
|
||||
|
||||
// unspecifiedMessage is a minimal message implementation that should not
|
||||
// be ignored. In general, OS-specific implementations should use better
|
||||
// types and avoid this if they can.
|
||||
type unspecifiedMessage struct{}
|
||||
|
||||
func (unspecifiedMessage) ignore() bool { return false }
|
||||
@@ -1,112 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"tailscale.com/net/interfaces"
|
||||
)
|
||||
|
||||
func TestMonitorStartClose(t *testing.T) {
|
||||
mon, err := New(t.Logf)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
mon.Start()
|
||||
if err := mon.Close(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMonitorJustClose(t *testing.T) {
|
||||
mon, err := New(t.Logf)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := mon.Close(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMonitorInjectEvent(t *testing.T) {
|
||||
mon, err := New(t.Logf)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer mon.Close()
|
||||
got := make(chan bool, 1)
|
||||
mon.RegisterChangeCallback(func(changed bool, state *interfaces.State) {
|
||||
select {
|
||||
case got <- true:
|
||||
default:
|
||||
}
|
||||
})
|
||||
mon.Start()
|
||||
mon.InjectEvent()
|
||||
select {
|
||||
case <-got:
|
||||
// Pass.
|
||||
case <-time.After(5 * time.Second):
|
||||
t.Fatal("timeout waiting for callback")
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
monitor = flag.String("monitor", "", `go into monitor mode like 'route monitor'; test never terminates. Value can be either "raw" or "callback"`)
|
||||
monitorDuration = flag.Duration("monitor-duration", 0, "if non-zero, how long to run TestMonitorMode. Zero means forever.")
|
||||
)
|
||||
|
||||
func TestMonitorMode(t *testing.T) {
|
||||
switch *monitor {
|
||||
case "":
|
||||
t.Skip("skipping non-test without --monitor")
|
||||
case "raw", "callback":
|
||||
default:
|
||||
t.Skipf(`invalid --monitor value: must be "raw" or "callback"`)
|
||||
}
|
||||
mon, err := New(t.Logf)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
switch *monitor {
|
||||
case "raw":
|
||||
var closed atomic.Bool
|
||||
if *monitorDuration != 0 {
|
||||
t := time.AfterFunc(*monitorDuration, func() {
|
||||
closed.Store(true)
|
||||
mon.Close()
|
||||
})
|
||||
defer t.Stop()
|
||||
}
|
||||
for {
|
||||
msg, err := mon.om.Receive()
|
||||
if closed.Load() {
|
||||
return
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
t.Logf("msg: %#v", msg)
|
||||
}
|
||||
case "callback":
|
||||
var done <-chan time.Time
|
||||
if *monitorDuration != 0 {
|
||||
t := time.NewTimer(*monitorDuration)
|
||||
defer t.Stop()
|
||||
done = t.C
|
||||
}
|
||||
n := 0
|
||||
mon.RegisterChangeCallback(func(changed bool, st *interfaces.State) {
|
||||
n++
|
||||
t.Logf("cb: changed=%v, ifSt=%v", changed, st)
|
||||
})
|
||||
mon.Start()
|
||||
<-done
|
||||
t.Logf("%v callbacks", n)
|
||||
}
|
||||
}
|
||||
@@ -1,161 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"golang.zx2c4.com/wireguard/windows/tunnel/winipcfg"
|
||||
"tailscale.com/net/tsaddr"
|
||||
"tailscale.com/types/logger"
|
||||
)
|
||||
|
||||
var (
|
||||
errClosed = errors.New("closed")
|
||||
)
|
||||
|
||||
type eventMessage struct {
|
||||
eventType string
|
||||
}
|
||||
|
||||
func (eventMessage) ignore() bool { return false }
|
||||
|
||||
type winMon struct {
|
||||
logf logger.Logf
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
messagec chan eventMessage
|
||||
addressChangeCallback *winipcfg.UnicastAddressChangeCallback
|
||||
routeChangeCallback *winipcfg.RouteChangeCallback
|
||||
|
||||
mu sync.Mutex
|
||||
lastLog time.Time // time we last logged about any windows change event
|
||||
|
||||
// noDeadlockTicker exists just to have something scheduled as
|
||||
// far as the Go runtime is concerned. Otherwise "tailscaled
|
||||
// debug --monitor" thinks it's deadlocked with nothing to do,
|
||||
// as Go's runtime doesn't know about callbacks registered with
|
||||
// Windows.
|
||||
noDeadlockTicker *time.Ticker
|
||||
}
|
||||
|
||||
func newOSMon(logf logger.Logf, _ *Mon) (osMon, error) {
|
||||
m := &winMon{
|
||||
logf: logf,
|
||||
messagec: make(chan eventMessage, 1),
|
||||
noDeadlockTicker: time.NewTicker(5000 * time.Hour), // arbitrary
|
||||
}
|
||||
m.ctx, m.cancel = context.WithCancel(context.Background())
|
||||
|
||||
var err error
|
||||
m.addressChangeCallback, err = winipcfg.RegisterUnicastAddressChangeCallback(m.unicastAddressChanged)
|
||||
if err != nil {
|
||||
m.logf("winipcfg.RegisterUnicastAddressChangeCallback error: %v", err)
|
||||
m.cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.routeChangeCallback, err = winipcfg.RegisterRouteChangeCallback(m.routeChanged)
|
||||
if err != nil {
|
||||
m.addressChangeCallback.Unregister()
|
||||
m.logf("winipcfg.RegisterRouteChangeCallback error: %v", err)
|
||||
m.cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (m *winMon) IsInterestingInterface(iface string) bool { return true }
|
||||
|
||||
func (m *winMon) Close() (ret error) {
|
||||
m.cancel()
|
||||
m.noDeadlockTicker.Stop()
|
||||
|
||||
if m.addressChangeCallback != nil {
|
||||
if err := m.addressChangeCallback.Unregister(); err != nil {
|
||||
m.logf("addressChangeCallback.Unregister error: %v", err)
|
||||
ret = err
|
||||
} else {
|
||||
m.addressChangeCallback = nil
|
||||
}
|
||||
}
|
||||
|
||||
if m.routeChangeCallback != nil {
|
||||
if err := m.routeChangeCallback.Unregister(); err != nil {
|
||||
m.logf("routeChangeCallback.Unregister error: %v", err)
|
||||
ret = err
|
||||
} else {
|
||||
m.routeChangeCallback = nil
|
||||
}
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (m *winMon) Receive() (message, error) {
|
||||
if m.ctx.Err() != nil {
|
||||
m.logf("Receive call on closed monitor")
|
||||
return nil, errClosed
|
||||
}
|
||||
|
||||
t0 := time.Now()
|
||||
|
||||
select {
|
||||
case msg := <-m.messagec:
|
||||
now := time.Now()
|
||||
m.mu.Lock()
|
||||
sinceLast := now.Sub(m.lastLog)
|
||||
m.lastLog = now
|
||||
m.mu.Unlock()
|
||||
// If it's either been awhile since we last logged
|
||||
// anything, or if this some route/addr that's not
|
||||
// about a Tailscale IP ("ts" prefix), then log. This
|
||||
// is mainly limited to suppress the flood about our own
|
||||
// route updates after connecting to a large tailnet
|
||||
// and all the IPv4 /32 routes.
|
||||
if sinceLast > 5*time.Second || !strings.HasPrefix(msg.eventType, "ts") {
|
||||
m.logf("got windows change event after %v: evt=%s", time.Since(t0).Round(time.Millisecond), msg.eventType)
|
||||
}
|
||||
return msg, nil
|
||||
case <-m.ctx.Done():
|
||||
return nil, errClosed
|
||||
}
|
||||
}
|
||||
|
||||
// unicastAddressChanged is the callback we register with Windows to call when unicast address changes.
|
||||
func (m *winMon) unicastAddressChanged(_ winipcfg.MibNotificationType, row *winipcfg.MibUnicastIPAddressRow) {
|
||||
what := "addr"
|
||||
if ip := row.Address.Addr(); ip.IsValid() && tsaddr.IsTailscaleIP(ip.Unmap()) {
|
||||
what = "tsaddr"
|
||||
}
|
||||
|
||||
// start a goroutine to finish our work, to return to Windows out of this callback
|
||||
go m.somethingChanged(what)
|
||||
}
|
||||
|
||||
// routeChanged is the callback we register with Windows to call when route changes.
|
||||
func (m *winMon) routeChanged(_ winipcfg.MibNotificationType, row *winipcfg.MibIPforwardRow2) {
|
||||
what := "route"
|
||||
ip := row.DestinationPrefix.Prefix().Addr().Unmap()
|
||||
if ip.IsValid() && tsaddr.IsTailscaleIP(ip) {
|
||||
what = "tsroute"
|
||||
}
|
||||
// start a goroutine to finish our work, to return to Windows out of this callback
|
||||
go m.somethingChanged(what)
|
||||
}
|
||||
|
||||
// somethingChanged gets called from OS callbacks whenever address or route changes.
|
||||
func (m *winMon) somethingChanged(evt string) {
|
||||
select {
|
||||
case <-m.ctx.Done():
|
||||
return
|
||||
case m.messagec <- eventMessage{eventType: evt}:
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -1,91 +0,0 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
//go:build !windows && !darwin
|
||||
|
||||
package monitor
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"os"
|
||||
"runtime"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"tailscale.com/net/interfaces"
|
||||
"tailscale.com/types/logger"
|
||||
)
|
||||
|
||||
func newPollingMon(logf logger.Logf, m *Mon) (osMon, error) {
|
||||
return &pollingMon{
|
||||
logf: logf,
|
||||
m: m,
|
||||
stop: make(chan struct{}),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// pollingMon is a bad but portable implementation of the link monitor
|
||||
// that works by polling the interface state every 10 seconds, in lieu
|
||||
// of anything to subscribe to.
|
||||
type pollingMon struct {
|
||||
logf logger.Logf
|
||||
m *Mon
|
||||
|
||||
closeOnce sync.Once
|
||||
stop chan struct{}
|
||||
}
|
||||
|
||||
func (pm *pollingMon) IsInterestingInterface(iface string) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (pm *pollingMon) Close() error {
|
||||
pm.closeOnce.Do(func() {
|
||||
close(pm.stop)
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pm *pollingMon) isCloudRun() bool {
|
||||
// https: //cloud.google.com/run/docs/reference/container-contract#env-vars
|
||||
if os.Getenv("K_REVISION") == "" || os.Getenv("K_CONFIGURATION") == "" ||
|
||||
os.Getenv("K_SERVICE") == "" || os.Getenv("PORT") == "" {
|
||||
return false
|
||||
}
|
||||
vers, err := os.ReadFile("/proc/version")
|
||||
if err != nil {
|
||||
pm.logf("Failed to read /proc/version: %v", err)
|
||||
return false
|
||||
}
|
||||
return string(bytes.TrimSpace(vers)) == "Linux version 4.4.0 #1 SMP Sun Jan 10 15:06:54 PST 2016"
|
||||
}
|
||||
|
||||
func (pm *pollingMon) Receive() (message, error) {
|
||||
d := 10 * time.Second
|
||||
if runtime.GOOS == "android" {
|
||||
// We'll have Android notify the link monitor to wake up earlier,
|
||||
// so this can go very slowly there, to save battery.
|
||||
// https://github.com/tailscale/tailscale/issues/1427
|
||||
d = 10 * time.Minute
|
||||
}
|
||||
if pm.isCloudRun() {
|
||||
// Cloud Run routes never change at runtime. the containers are killed within
|
||||
// 15 minutes by default, set the interval long enough to be effectively infinite.
|
||||
pm.logf("monitor polling: Cloud Run detected, reduce polling interval to 24h")
|
||||
d = 24 * time.Hour
|
||||
}
|
||||
ticker := time.NewTicker(d)
|
||||
defer ticker.Stop()
|
||||
base := pm.m.InterfaceState()
|
||||
for {
|
||||
if cur, err := pm.m.interfaceStateUncached(); err == nil && !cur.EqualFiltered(base, interfaces.UseInterestingInterfaces, interfaces.UseInterestingIPs) {
|
||||
return unspecifiedMessage{}, nil
|
||||
}
|
||||
select {
|
||||
case <-ticker.C:
|
||||
case <-pm.stop:
|
||||
return nil, errors.New("stopped")
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -10,9 +10,9 @@ import (
|
||||
"reflect"
|
||||
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/preftype"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
// Router is responsible for managing the system network stack.
|
||||
@@ -34,11 +34,11 @@ type Router interface {
|
||||
// New returns a new Router for the current platform, using the
|
||||
// provided tun device.
|
||||
//
|
||||
// If linkMon is nil, it's not used. It's currently (2021-07-20) only
|
||||
// If netMon is nil, it's not used. It's currently (2021-07-20) only
|
||||
// used on Linux in some situations.
|
||||
func New(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
func New(logf logger.Logf, tundev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
logf = logger.WithPrefix(logf, "router: ")
|
||||
return newUserspaceRouter(logf, tundev, linkMon)
|
||||
return newUserspaceRouter(logf, tundev, netMon)
|
||||
}
|
||||
|
||||
// Cleanup restores the system network configuration to its original state
|
||||
|
||||
@@ -5,12 +5,12 @@ package router
|
||||
|
||||
import (
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
return newUserspaceBSDRouter(logf, tundev, linkMon)
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
return newUserspaceBSDRouter(logf, tundev, netMon)
|
||||
}
|
||||
|
||||
func cleanup(logger.Logf, string) {
|
||||
|
||||
@@ -10,11 +10,11 @@ import (
|
||||
"runtime"
|
||||
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
func newUserspaceRouter(logf logger.Logf, tunDev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
func newUserspaceRouter(logf logger.Logf, tunDev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
return nil, fmt.Errorf("unsupported OS %q", runtime.GOOS)
|
||||
}
|
||||
|
||||
|
||||
@@ -5,8 +5,8 @@ package router
|
||||
|
||||
import (
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
// For now this router only supports the userspace WireGuard implementations.
|
||||
@@ -14,8 +14,8 @@ import (
|
||||
// Work is currently underway for an in-kernel FreeBSD implementation of wireguard
|
||||
// https://svnweb.freebsd.org/base?view=revision&revision=357986
|
||||
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
return newUserspaceBSDRouter(logf, tundev, linkMon)
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
return newUserspaceBSDRouter(logf, tundev, netMon)
|
||||
}
|
||||
|
||||
func cleanup(logf logger.Logf, interfaceName string) {
|
||||
|
||||
@@ -24,12 +24,12 @@ import (
|
||||
"golang.org/x/sys/unix"
|
||||
"golang.org/x/time/rate"
|
||||
"tailscale.com/envknob"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/net/tsaddr"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/preftype"
|
||||
"tailscale.com/util/multierr"
|
||||
"tailscale.com/version/distro"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
const (
|
||||
@@ -94,8 +94,8 @@ type linuxRouter struct {
|
||||
closed atomic.Bool
|
||||
logf func(fmt string, args ...any)
|
||||
tunname string
|
||||
linkMon *monitor.Mon
|
||||
unregLinkMon func()
|
||||
netMon *netmon.Monitor
|
||||
unregNetMon func()
|
||||
addrs map[netip.Prefix]bool
|
||||
routes map[netip.Prefix]bool
|
||||
localRoutes map[netip.Prefix]bool
|
||||
@@ -121,7 +121,7 @@ type linuxRouter struct {
|
||||
cmd commandRunner
|
||||
}
|
||||
|
||||
func newUserspaceRouter(logf logger.Logf, tunDev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
func newUserspaceRouter(logf logger.Logf, tunDev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
tunname, err := tunDev.Name()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -156,15 +156,15 @@ func newUserspaceRouter(logf logger.Logf, tunDev tun.Device, linkMon *monitor.Mo
|
||||
ambientCapNetAdmin: useAmbientCaps(),
|
||||
}
|
||||
|
||||
return newUserspaceRouterAdvanced(logf, tunname, linkMon, ipt4, ipt6, cmd, supportsV6, supportsV6NAT)
|
||||
return newUserspaceRouterAdvanced(logf, tunname, netMon, ipt4, ipt6, cmd, supportsV6, supportsV6NAT)
|
||||
}
|
||||
|
||||
func newUserspaceRouterAdvanced(logf logger.Logf, tunname string, linkMon *monitor.Mon, netfilter4, netfilter6 netfilterRunner, cmd commandRunner, supportsV6, supportsV6NAT bool) (Router, error) {
|
||||
func newUserspaceRouterAdvanced(logf logger.Logf, tunname string, netMon *netmon.Monitor, netfilter4, netfilter6 netfilterRunner, cmd commandRunner, supportsV6, supportsV6NAT bool) (Router, error) {
|
||||
r := &linuxRouter{
|
||||
logf: logf,
|
||||
tunname: tunname,
|
||||
netfilterMode: netfilterOff,
|
||||
linkMon: linkMon,
|
||||
netMon: netMon,
|
||||
|
||||
v6Available: supportsV6,
|
||||
v6NATAvailable: supportsV6NAT,
|
||||
@@ -336,8 +336,8 @@ func (r *linuxRouter) useIPCommand() bool {
|
||||
return !ok
|
||||
}
|
||||
|
||||
// onIPRuleDeleted is the callback from the link monitor for when an IP policy
|
||||
// rule is deleted. See Issue 1591.
|
||||
// onIPRuleDeleted is the callback from the network monitor for when an IP
|
||||
// policy rule is deleted. See Issue 1591.
|
||||
//
|
||||
// If an ip rule is deleted (with pref number 52xx, as Tailscale sets), then
|
||||
// set a timer to restore our rules, in case they were deleted. The timer lets
|
||||
@@ -372,8 +372,8 @@ func (r *linuxRouter) onIPRuleDeleted(table uint8, priority uint32) {
|
||||
}
|
||||
|
||||
func (r *linuxRouter) Up() error {
|
||||
if r.unregLinkMon == nil && r.linkMon != nil {
|
||||
r.unregLinkMon = r.linkMon.RegisterRuleDeleteCallback(r.onIPRuleDeleted)
|
||||
if r.unregNetMon == nil && r.netMon != nil {
|
||||
r.unregNetMon = r.netMon.RegisterRuleDeleteCallback(r.onIPRuleDeleted)
|
||||
}
|
||||
if err := r.addIPRules(); err != nil {
|
||||
return fmt.Errorf("adding IP rules: %w", err)
|
||||
@@ -390,8 +390,8 @@ func (r *linuxRouter) Up() error {
|
||||
|
||||
func (r *linuxRouter) Close() error {
|
||||
r.closed.Store(true)
|
||||
if r.unregLinkMon != nil {
|
||||
r.unregLinkMon()
|
||||
if r.unregNetMon != nil {
|
||||
r.unregNetMon()
|
||||
}
|
||||
if err := r.downInterface(); err != nil {
|
||||
return err
|
||||
|
||||
@@ -21,9 +21,9 @@ import (
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"github.com/vishvananda/netlink"
|
||||
"golang.org/x/exp/slices"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/tstest"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
func TestRouterStates(t *testing.T) {
|
||||
@@ -320,7 +320,7 @@ ip route add throw 192.168.0.0/24 table 52` + basic,
|
||||
},
|
||||
}
|
||||
|
||||
mon, err := monitor.New(logger.Discard)
|
||||
mon, err := netmon.New(logger.Discard)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@@ -659,7 +659,7 @@ func createTestTUN(t *testing.T) tun.Device {
|
||||
|
||||
type linuxTest struct {
|
||||
tun tun.Device
|
||||
mon *monitor.Mon
|
||||
mon *netmon.Monitor
|
||||
r *linuxRouter
|
||||
logOutput tstest.MemLogger
|
||||
}
|
||||
@@ -684,7 +684,7 @@ func newLinuxRootTest(t *testing.T) *linuxTest {
|
||||
|
||||
logf := lt.logOutput.Logf
|
||||
|
||||
mon, err := monitor.New(logger.Discard)
|
||||
mon, err := netmon.New(logger.Discard)
|
||||
if err != nil {
|
||||
lt.Close()
|
||||
t.Fatal(err)
|
||||
|
||||
@@ -12,8 +12,8 @@ import (
|
||||
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"go4.org/netipx"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
// For now this router only supports the WireGuard userspace implementation.
|
||||
@@ -22,14 +22,14 @@ import (
|
||||
|
||||
type openbsdRouter struct {
|
||||
logf logger.Logf
|
||||
linkMon *monitor.Mon
|
||||
netMon *netmon.Monitor
|
||||
tunname string
|
||||
local4 netip.Prefix
|
||||
local6 netip.Prefix
|
||||
routes map[netip.Prefix]struct{}
|
||||
}
|
||||
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
tunname, err := tundev.Name()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -37,7 +37,7 @@ func newUserspaceRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mo
|
||||
|
||||
return &openbsdRouter{
|
||||
logf: logf,
|
||||
linkMon: linkMon,
|
||||
netMon: netMon,
|
||||
tunname: tunname,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -14,21 +14,21 @@ import (
|
||||
|
||||
"github.com/tailscale/wireguard-go/tun"
|
||||
"go4.org/netipx"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/net/tsaddr"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/version"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
type userspaceBSDRouter struct {
|
||||
logf logger.Logf
|
||||
linkMon *monitor.Mon
|
||||
netMon *netmon.Monitor
|
||||
tunname string
|
||||
local []netip.Prefix
|
||||
routes map[netip.Prefix]bool
|
||||
}
|
||||
|
||||
func newUserspaceBSDRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
func newUserspaceBSDRouter(logf logger.Logf, tundev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
tunname, err := tundev.Name()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -36,7 +36,7 @@ func newUserspaceBSDRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor
|
||||
|
||||
return &userspaceBSDRouter{
|
||||
logf: logf,
|
||||
linkMon: linkMon,
|
||||
netMon: netMon,
|
||||
tunname: tunname,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -22,19 +22,19 @@ import (
|
||||
"golang.zx2c4.com/wireguard/windows/tunnel/winipcfg"
|
||||
"tailscale.com/logtail/backoff"
|
||||
"tailscale.com/net/dns"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
)
|
||||
|
||||
type winRouter struct {
|
||||
logf func(fmt string, args ...any)
|
||||
linkMon *monitor.Mon // may be nil
|
||||
netMon *netmon.Monitor // may be nil
|
||||
nativeTun *tun.NativeTun
|
||||
routeChangeCallback *winipcfg.RouteChangeCallback
|
||||
firewall *firewallTweaker
|
||||
}
|
||||
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mon) (Router, error) {
|
||||
func newUserspaceRouter(logf logger.Logf, tundev tun.Device, netMon *netmon.Monitor) (Router, error) {
|
||||
nativeTun := tundev.(*tun.NativeTun)
|
||||
luid := winipcfg.LUID(nativeTun.LUID())
|
||||
guid, err := luid.GUID()
|
||||
@@ -44,7 +44,7 @@ func newUserspaceRouter(logf logger.Logf, tundev tun.Device, linkMon *monitor.Mo
|
||||
|
||||
return &winRouter{
|
||||
logf: logf,
|
||||
linkMon: linkMon,
|
||||
netMon: netMon,
|
||||
nativeTun: nativeTun,
|
||||
firewall: &firewallTweaker{
|
||||
logf: logger.WithPrefix(logf, "firewall: "),
|
||||
|
||||
@@ -28,6 +28,7 @@ import (
|
||||
"tailscale.com/net/dns/resolver"
|
||||
"tailscale.com/net/flowtrack"
|
||||
"tailscale.com/net/interfaces"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/net/packet"
|
||||
"tailscale.com/net/sockstats"
|
||||
"tailscale.com/net/tsaddr"
|
||||
@@ -48,7 +49,6 @@ import (
|
||||
"tailscale.com/wgengine/capture"
|
||||
"tailscale.com/wgengine/filter"
|
||||
"tailscale.com/wgengine/magicsock"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
"tailscale.com/wgengine/netlog"
|
||||
"tailscale.com/wgengine/router"
|
||||
"tailscale.com/wgengine/wgcfg"
|
||||
@@ -84,21 +84,21 @@ const statusPollInterval = 1 * time.Minute
|
||||
const networkLoggerUploadTimeout = 5 * time.Second
|
||||
|
||||
type userspaceEngine struct {
|
||||
logf logger.Logf
|
||||
wgLogger *wglog.Logger //a wireguard-go logging wrapper
|
||||
reqCh chan struct{}
|
||||
waitCh chan struct{} // chan is closed when first Close call completes; contrast with closing bool
|
||||
timeNow func() mono.Time
|
||||
tundev *tstun.Wrapper
|
||||
wgdev *device.Device
|
||||
router router.Router
|
||||
confListenPort uint16 // original conf.ListenPort
|
||||
dns *dns.Manager
|
||||
magicConn *magicsock.Conn
|
||||
linkMon *monitor.Mon
|
||||
linkMonOwned bool // whether we created linkMon (and thus need to close it)
|
||||
linkMonUnregister func() // unsubscribes from changes; used regardless of linkMonOwned
|
||||
birdClient BIRDClient // or nil
|
||||
logf logger.Logf
|
||||
wgLogger *wglog.Logger //a wireguard-go logging wrapper
|
||||
reqCh chan struct{}
|
||||
waitCh chan struct{} // chan is closed when first Close call completes; contrast with closing bool
|
||||
timeNow func() mono.Time
|
||||
tundev *tstun.Wrapper
|
||||
wgdev *device.Device
|
||||
router router.Router
|
||||
confListenPort uint16 // original conf.ListenPort
|
||||
dns *dns.Manager
|
||||
magicConn *magicsock.Conn
|
||||
netMon *netmon.Monitor
|
||||
netMonOwned bool // whether we created netMon (and thus need to close it)
|
||||
netMonUnregister func() // unsubscribes from changes; used regardless of netMonOwned
|
||||
birdClient BIRDClient // or nil
|
||||
|
||||
testMaybeReconfigHook func() // for tests; if non-nil, fires if maybeReconfigWireguardLocked called
|
||||
|
||||
@@ -199,9 +199,9 @@ type Config struct {
|
||||
// If nil, a fake OSConfigurator that does nothing is used.
|
||||
DNS dns.OSConfigurator
|
||||
|
||||
// LinkMonitor optionally provides an existing link monitor to re-use.
|
||||
// If nil, a new link monitor is created.
|
||||
LinkMonitor *monitor.Mon
|
||||
// NetMon optionally provides an existing network monitor to re-use.
|
||||
// If nil, a new network monitor is created.
|
||||
NetMon *netmon.Monitor
|
||||
|
||||
// Dialer is the dialer to use for outbound connections.
|
||||
// If nil, a new Dialer is created
|
||||
@@ -316,34 +316,34 @@ func NewUserspaceEngine(logf logger.Logf, conf Config) (_ Engine, reterr error)
|
||||
e.isLocalAddr.Store(tsaddr.NewContainsIPFunc(nil))
|
||||
e.isDNSIPOverTailscale.Store(tsaddr.NewContainsIPFunc(nil))
|
||||
|
||||
if conf.LinkMonitor != nil {
|
||||
e.linkMon = conf.LinkMonitor
|
||||
if conf.NetMon != nil {
|
||||
e.netMon = conf.NetMon
|
||||
} else {
|
||||
mon, err := monitor.New(logf)
|
||||
mon, err := netmon.New(logf)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
closePool.add(mon)
|
||||
e.linkMon = mon
|
||||
e.linkMonOwned = true
|
||||
e.netMon = mon
|
||||
e.netMonOwned = true
|
||||
}
|
||||
|
||||
tunName, _ := conf.Tun.Name()
|
||||
conf.Dialer.SetTUNName(tunName)
|
||||
conf.Dialer.SetLinkMonitor(e.linkMon)
|
||||
e.dns = dns.NewManager(logf, conf.DNS, e.linkMon, conf.Dialer, fwdDNSLinkSelector{e, tunName})
|
||||
conf.Dialer.SetNetMon(e.netMon)
|
||||
e.dns = dns.NewManager(logf, conf.DNS, e.netMon, conf.Dialer, fwdDNSLinkSelector{e, tunName})
|
||||
|
||||
// TODO: there's probably a better place for this
|
||||
sockstats.SetLinkMonitor(e.linkMon)
|
||||
sockstats.SetNetMon(e.netMon)
|
||||
|
||||
logf("link state: %+v", e.linkMon.InterfaceState())
|
||||
logf("link state: %+v", e.netMon.InterfaceState())
|
||||
|
||||
unregisterMonWatch := e.linkMon.RegisterChangeCallback(func(changed bool, st *interfaces.State) {
|
||||
unregisterMonWatch := e.netMon.RegisterChangeCallback(func(changed bool, st *interfaces.State) {
|
||||
tshttpproxy.InvalidateCache()
|
||||
e.linkChange(changed, st)
|
||||
})
|
||||
closePool.addFunc(unregisterMonWatch)
|
||||
e.linkMonUnregister = unregisterMonWatch
|
||||
e.netMonUnregister = unregisterMonWatch
|
||||
|
||||
endpointsFn := func(endpoints []tailcfg.Endpoint) {
|
||||
e.mu.Lock()
|
||||
@@ -359,7 +359,7 @@ func NewUserspaceEngine(logf logger.Logf, conf Config) (_ Engine, reterr error)
|
||||
DERPActiveFunc: e.RequestStatus,
|
||||
IdleFunc: e.tundev.IdleDuration,
|
||||
NoteRecvActivity: e.noteRecvActivity,
|
||||
LinkMonitor: e.linkMon,
|
||||
NetMon: e.netMon,
|
||||
}
|
||||
|
||||
var err error
|
||||
@@ -368,7 +368,7 @@ func NewUserspaceEngine(logf logger.Logf, conf Config) (_ Engine, reterr error)
|
||||
return nil, fmt.Errorf("wgengine: %v", err)
|
||||
}
|
||||
closePool.add(e.magicConn)
|
||||
e.magicConn.SetNetworkUp(e.linkMon.InterfaceState().AnyInterfaceUp())
|
||||
e.magicConn.SetNetworkUp(e.netMon.InterfaceState().AnyInterfaceUp())
|
||||
|
||||
tsTUNDev.SetDiscoKey(e.magicConn.DiscoPublicKey())
|
||||
|
||||
@@ -455,8 +455,8 @@ func NewUserspaceEngine(logf logger.Logf, conf Config) (_ Engine, reterr error)
|
||||
if err := e.router.Set(nil); err != nil {
|
||||
return nil, fmt.Errorf("router.Set(nil): %w", err)
|
||||
}
|
||||
e.logf("Starting link monitor...")
|
||||
e.linkMon.Start()
|
||||
e.logf("Starting network monitor...")
|
||||
e.netMon.Start()
|
||||
|
||||
e.logf("Engine created.")
|
||||
return e, nil
|
||||
@@ -1094,9 +1094,9 @@ func (e *userspaceEngine) Close() {
|
||||
r := bufio.NewReader(strings.NewReader(""))
|
||||
e.wgdev.IpcSetOperation(r)
|
||||
e.magicConn.Close()
|
||||
e.linkMonUnregister()
|
||||
if e.linkMonOwned {
|
||||
e.linkMon.Close()
|
||||
e.netMonUnregister()
|
||||
if e.netMonOwned {
|
||||
e.netMon.Close()
|
||||
}
|
||||
e.dns.Down()
|
||||
e.router.Close()
|
||||
@@ -1119,15 +1119,15 @@ func (e *userspaceEngine) Wait() {
|
||||
<-e.waitCh
|
||||
}
|
||||
|
||||
func (e *userspaceEngine) GetLinkMonitor() *monitor.Mon {
|
||||
return e.linkMon
|
||||
func (e *userspaceEngine) GetNetMon() *netmon.Monitor {
|
||||
return e.netMon
|
||||
}
|
||||
|
||||
// LinkChange signals a network change event. It's currently
|
||||
// (2021-03-03) only called on Android. On other platforms, linkMon
|
||||
// (2021-03-03) only called on Android. On other platforms, netMon
|
||||
// generates link change events for us.
|
||||
func (e *userspaceEngine) LinkChange(_ bool) {
|
||||
e.linkMon.InjectEvent()
|
||||
e.netMon.InjectEvent()
|
||||
}
|
||||
|
||||
func (e *userspaceEngine) linkChange(changed bool, cur *interfaces.State) {
|
||||
|
||||
@@ -18,6 +18,7 @@ import (
|
||||
"tailscale.com/ipn/ipnstate"
|
||||
"tailscale.com/net/dns"
|
||||
"tailscale.com/net/dns/resolver"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/net/tstun"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/types/key"
|
||||
@@ -25,7 +26,6 @@ import (
|
||||
"tailscale.com/wgengine/capture"
|
||||
"tailscale.com/wgengine/filter"
|
||||
"tailscale.com/wgengine/magicsock"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
"tailscale.com/wgengine/router"
|
||||
"tailscale.com/wgengine/wgcfg"
|
||||
)
|
||||
@@ -126,8 +126,8 @@ func (e *watchdogEngine) watchdog(name string, fn func()) {
|
||||
func (e *watchdogEngine) Reconfig(cfg *wgcfg.Config, routerCfg *router.Config, dnsCfg *dns.Config, debug *tailcfg.Debug) error {
|
||||
return e.watchdogErr("Reconfig", func() error { return e.wrap.Reconfig(cfg, routerCfg, dnsCfg, debug) })
|
||||
}
|
||||
func (e *watchdogEngine) GetLinkMonitor() *monitor.Mon {
|
||||
return e.wrap.GetLinkMonitor()
|
||||
func (e *watchdogEngine) GetNetMon() *netmon.Monitor {
|
||||
return e.wrap.GetNetMon()
|
||||
}
|
||||
func (e *watchdogEngine) GetFilter() *filter.Filter {
|
||||
return e.wrap.GetFilter()
|
||||
|
||||
@@ -10,12 +10,12 @@ import (
|
||||
|
||||
"tailscale.com/ipn/ipnstate"
|
||||
"tailscale.com/net/dns"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/types/key"
|
||||
"tailscale.com/types/netmap"
|
||||
"tailscale.com/wgengine/capture"
|
||||
"tailscale.com/wgengine/filter"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
"tailscale.com/wgengine/router"
|
||||
"tailscale.com/wgengine/wgcfg"
|
||||
)
|
||||
@@ -92,8 +92,8 @@ type Engine interface {
|
||||
// WireGuard status changes.
|
||||
SetStatusCallback(StatusCallback)
|
||||
|
||||
// GetLinkMonitor returns the link monitor.
|
||||
GetLinkMonitor() *monitor.Mon
|
||||
// GetNetMon returns the network monitor.
|
||||
GetNetMon() *netmon.Monitor
|
||||
|
||||
// RequestStatus requests a WireGuard status update right
|
||||
// away, sent to the callback registered via SetStatusCallback.
|
||||
@@ -119,7 +119,7 @@ type Engine interface {
|
||||
//
|
||||
// Deprecated: don't use this method. It was removed shortly
|
||||
// before the Tailscale 1.6 release when we remembered that
|
||||
// Android doesn't use the Linux-based link monitor and has
|
||||
// Android doesn't use the Linux-based network monitor and has
|
||||
// its own mechanism that uses LinkChange. Android is the only
|
||||
// caller of this method now. Don't add more.
|
||||
LinkChange(isExpensive bool)
|
||||
|
||||
Reference in New Issue
Block a user