ipn/ipnlocal: add localNodeContext with netmap-related fields and methods

Updates #12614

Signed-off-by: Nick Khyl <nickk@tailscale.com>
This commit is contained in:
Nick Khyl 2025-04-24 21:54:48 -05:00 committed by Nick Khyl
parent f701d39ba4
commit 8b72dd7873
13 changed files with 648 additions and 351 deletions

View File

@ -382,14 +382,14 @@ func TestAllowExitNodeDNSProxyToServeName(t *testing.T) {
t.Fatal("unexpected true on backend with nil NetMap") t.Fatal("unexpected true on backend with nil NetMap")
} }
b.netMap = &netmap.NetworkMap{ b.currentNode().SetNetMap(&netmap.NetworkMap{
DNS: tailcfg.DNSConfig{ DNS: tailcfg.DNSConfig{
ExitNodeFilteredSet: []string{ ExitNodeFilteredSet: []string{
".ts.net", ".ts.net",
"some.exact.bad", "some.exact.bad",
}, },
}, },
} })
tests := []struct { tests := []struct {
name string name string
want bool want bool

View File

@ -4,7 +4,6 @@
package ipnlocal package ipnlocal
import ( import (
"cmp"
"fmt" "fmt"
"os" "os"
"slices" "slices"
@ -26,26 +25,14 @@ const (
// enabled. This is currently based on checking for the drive:share node // enabled. This is currently based on checking for the drive:share node
// attribute. // attribute.
func (b *LocalBackend) DriveSharingEnabled() bool { func (b *LocalBackend) DriveSharingEnabled() bool {
b.mu.Lock() return b.currentNode().SelfHasCap(tailcfg.NodeAttrsTaildriveShare)
defer b.mu.Unlock()
return b.driveSharingEnabledLocked()
}
func (b *LocalBackend) driveSharingEnabledLocked() bool {
return b.netMap != nil && b.netMap.SelfNode.HasCap(tailcfg.NodeAttrsTaildriveShare)
} }
// DriveAccessEnabled reports whether accessing Taildrive shares on remote nodes // DriveAccessEnabled reports whether accessing Taildrive shares on remote nodes
// is enabled. This is currently based on checking for the drive:access node // is enabled. This is currently based on checking for the drive:access node
// attribute. // attribute.
func (b *LocalBackend) DriveAccessEnabled() bool { func (b *LocalBackend) DriveAccessEnabled() bool {
b.mu.Lock() return b.currentNode().SelfHasCap(tailcfg.NodeAttrsTaildriveAccess)
defer b.mu.Unlock()
return b.driveAccessEnabledLocked()
}
func (b *LocalBackend) driveAccessEnabledLocked() bool {
return b.netMap != nil && b.netMap.SelfNode.HasCap(tailcfg.NodeAttrsTaildriveAccess)
} }
// DriveSetServerAddr tells Taildrive to use the given address for connecting // DriveSetServerAddr tells Taildrive to use the given address for connecting
@ -266,7 +253,7 @@ func (b *LocalBackend) driveNotifyShares(shares views.SliceView[*drive.Share, dr
// shares has changed since the last notification. // shares has changed since the last notification.
func (b *LocalBackend) driveNotifyCurrentSharesLocked() { func (b *LocalBackend) driveNotifyCurrentSharesLocked() {
var shares views.SliceView[*drive.Share, drive.ShareView] var shares views.SliceView[*drive.Share, drive.ShareView]
if b.driveSharingEnabledLocked() { if b.DriveSharingEnabled() {
// Only populate shares if sharing is enabled. // Only populate shares if sharing is enabled.
shares = b.pm.prefs.DriveShares() shares = b.pm.prefs.DriveShares()
} }
@ -310,12 +297,12 @@ func (b *LocalBackend) updateDrivePeersLocked(nm *netmap.NetworkMap) {
} }
var driveRemotes []*drive.Remote var driveRemotes []*drive.Remote
if b.driveAccessEnabledLocked() { if b.DriveAccessEnabled() {
// Only populate peers if access is enabled, otherwise leave blank. // Only populate peers if access is enabled, otherwise leave blank.
driveRemotes = b.driveRemotesFromPeers(nm) driveRemotes = b.driveRemotesFromPeers(nm)
} }
fs.SetRemotes(b.netMap.Domain, driveRemotes, b.newDriveTransport()) fs.SetRemotes(nm.Domain, driveRemotes, b.newDriveTransport())
} }
func (b *LocalBackend) driveRemotesFromPeers(nm *netmap.NetworkMap) []*drive.Remote { func (b *LocalBackend) driveRemotesFromPeers(nm *netmap.NetworkMap) []*drive.Remote {
@ -330,23 +317,20 @@ func (b *LocalBackend) driveRemotesFromPeers(nm *netmap.NetworkMap) []*drive.Rem
// Peers are available to Taildrive if: // Peers are available to Taildrive if:
// - They are online // - They are online
// - They are allowed to share at least one folder with us // - They are allowed to share at least one folder with us
b.mu.Lock() cn := b.currentNode()
latestNetMap := b.netMap peer, ok := cn.PeerByID(peerID)
b.mu.Unlock() if !ok {
idx, found := slices.BinarySearchFunc(latestNetMap.Peers, peerID, func(candidate tailcfg.NodeView, id tailcfg.NodeID) int {
return cmp.Compare(candidate.ID(), id)
})
if !found {
return false return false
} }
peer := latestNetMap.Peers[idx]
// Exclude offline peers. // Exclude offline peers.
// TODO(oxtoacart): for some reason, this correctly // TODO(oxtoacart): for some reason, this correctly
// catches when a node goes from offline to online, // catches when a node goes from offline to online,
// but not the other way around... // but not the other way around...
// TODO(oxtoacart,nickkhyl): the reason was probably
// that we were using netmap.Peers instead of b.peers.
// The netmap.Peers slice is not updated in all cases.
// It should be fixed now that we use PeerByIDOk.
if !peer.Online().Get() { if !peer.Online().Get() {
return false return false
} }
@ -354,8 +338,7 @@ func (b *LocalBackend) driveRemotesFromPeers(nm *netmap.NetworkMap) []*drive.Rem
// Check that the peer is allowed to share with us. // Check that the peer is allowed to share with us.
addresses := peer.Addresses() addresses := peer.Addresses()
for _, p := range addresses.All() { for _, p := range addresses.All() {
capsMap := b.PeerCaps(p.Addr()) if cn.PeerHasCap(p.Addr(), tailcfg.PeerCapabilityTaildriveSharer) {
if capsMap.HasCapability(tailcfg.PeerCapabilityTaildriveSharer) {
return true return true
} }
} }

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,207 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
package ipnlocal
import (
"net/netip"
"sync"
"sync/atomic"
"go4.org/netipx"
"tailscale.com/ipn"
"tailscale.com/tailcfg"
"tailscale.com/types/key"
"tailscale.com/types/logger"
"tailscale.com/types/netmap"
"tailscale.com/util/slicesx"
"tailscale.com/wgengine/filter"
)
// localNodeContext holds the [LocalBackend]'s context tied to a local node (usually the current one).
//
// Its exported methods are safe for concurrent use, but the struct is not a snapshot of state at a given moment;
// its state can change between calls. For example, asking for the same value (e.g., netmap or prefs) twice
// may return different results. Returned values are immutable and safe for concurrent use.
//
// If both the [LocalBackend]'s internal mutex and the [localNodeContext] mutex must be held at the same time,
// the [LocalBackend] mutex must be acquired first. See the comment on the [LocalBackend] field for more details.
//
// Two pointers to different [localNodeContext] instances represent different local nodes.
// However, there's currently a bug where a new [localNodeContext] might not be created
// during an implicit node switch (see tailscale/corp#28014).
// In the future, we might want to include at least the following in this struct (in addition to the current fields).
// However, not everything should be exported or otherwise made available to the outside world (e.g. [ipnext] extensions,
// peer API handlers, etc.).
// - [ipn.State]: when the LocalBackend switches to a different [localNodeContext], it can update the state of the old one.
// - [ipn.LoginProfileView] and [ipn.Prefs]: we should update them when the [profileManager] reports changes to them.
// In the future, [profileManager] (and the corresponding methods of the [LocalBackend]) can be made optional,
// and something else could be used to set them once or update them as needed.
// - [tailcfg.HostinfoView]: it includes certain fields that are tied to the current profile/node/prefs. We should also
// update to build it once instead of mutating it in twelvety different places.
// - [filter.Filter] (normal and jailed, along with the filterHash): the localNodeContext could have a method to (re-)build
// the filter for the current netmap/prefs (see [LocalBackend.updateFilterLocked]), and it needs to track the current
// filters and their hash.
// - Fields related to a requested or required (re-)auth: authURL, authURLTime, authActor, keyExpired, etc.
// - [controlclient.Client]/[*controlclient.Auto]: the current control client. It is ties to a node identity.
// - [tkaState]: it is tied to the current profile / node.
// - Fields related to scheduled node expiration: nmExpiryTimer, numClientStatusCalls, [expiryManager].
//
// It should not include any fields used by specific features that don't belong in [LocalBackend].
// Even if they're tied to the local node, instead of moving them here, we should extract the entire feature
// into a separate package and have it install proper hooks.
type localNodeContext struct {
// filterAtomic is a stateful packet filter. Immutable once created, but can be
// replaced with a new one.
filterAtomic atomic.Pointer[filter.Filter]
// TODO(nickkhyl): maybe use sync.RWMutex?
mu sync.Mutex // protects the following fields
// NetMap is the most recently set full netmap from the controlclient.
// It can't be mutated in place once set. Because it can't be mutated in place,
// delta updates from the control server don't apply to it. Instead, use
// the peers map to get up-to-date information on the state of peers.
// In general, avoid using the netMap.Peers slice. We'd like it to go away
// as of 2023-09-17.
// TODO(nickkhyl): make it an atomic pointer to avoid the need for a mutex?
netMap *netmap.NetworkMap
// peers is the set of current peers and their current values after applying
// delta node mutations as they come in (with mu held). The map values can be
// given out to callers, but the map itself can be mutated in place (with mu held)
// and must not escape the [localNodeContext].
peers map[tailcfg.NodeID]tailcfg.NodeView
// nodeByAddr maps nodes' own addresses (excluding subnet routes) to node IDs.
// It is mutated in place (with mu held) and must not escape the [localNodeContext].
nodeByAddr map[netip.Addr]tailcfg.NodeID
}
func newLocalNodeContext() *localNodeContext {
cn := &localNodeContext{}
// Default filter blocks everything and logs nothing.
noneFilter := filter.NewAllowNone(logger.Discard, &netipx.IPSet{})
cn.filterAtomic.Store(noneFilter)
return cn
}
func (c *localNodeContext) Self() tailcfg.NodeView {
c.mu.Lock()
defer c.mu.Unlock()
if c.netMap == nil {
return tailcfg.NodeView{}
}
return c.netMap.SelfNode
}
func (c *localNodeContext) SelfUserID() tailcfg.UserID {
self := c.Self()
if !self.Valid() {
return 0
}
return self.User()
}
// SelfHasCap reports whether the specified capability was granted to the self node in the most recent netmap.
func (c *localNodeContext) SelfHasCap(wantCap tailcfg.NodeCapability) bool {
return c.SelfHasCapOr(wantCap, false)
}
// SelfHasCapOr is like [localNodeContext.SelfHasCap], but returns the specified default value
// if the netmap is not available yet.
func (c *localNodeContext) SelfHasCapOr(wantCap tailcfg.NodeCapability, def bool) bool {
c.mu.Lock()
defer c.mu.Unlock()
if c.netMap == nil {
return def
}
return c.netMap.AllCaps.Contains(wantCap)
}
func (c *localNodeContext) NetworkProfile() ipn.NetworkProfile {
c.mu.Lock()
defer c.mu.Unlock()
return ipn.NetworkProfile{
// These are ok to call with nil netMap.
MagicDNSName: c.netMap.MagicDNSSuffix(),
DomainName: c.netMap.DomainName(),
}
}
// TODO(nickkhyl): update it to return a [tailcfg.DERPMapView]?
func (c *localNodeContext) DERPMap() *tailcfg.DERPMap {
c.mu.Lock()
defer c.mu.Unlock()
if c.netMap == nil {
return nil
}
return c.netMap.DERPMap
}
func (c *localNodeContext) NodeByAddr(ip netip.Addr) (_ tailcfg.NodeID, ok bool) {
c.mu.Lock()
defer c.mu.Unlock()
nid, ok := c.nodeByAddr[ip]
return nid, ok
}
func (c *localNodeContext) NodeByKey(k key.NodePublic) (_ tailcfg.NodeID, ok bool) {
c.mu.Lock()
defer c.mu.Unlock()
if c.netMap == nil {
return 0, false
}
if self := c.netMap.SelfNode; self.Valid() && self.Key() == k {
return self.ID(), true
}
// TODO(bradfitz,nickkhyl): add nodeByKey like nodeByAddr instead of walking peers.
for _, n := range c.peers {
if n.Key() == k {
return n.ID(), true
}
}
return 0, false
}
func (c *localNodeContext) PeerByID(id tailcfg.NodeID) (_ tailcfg.NodeView, ok bool) {
c.mu.Lock()
defer c.mu.Unlock()
n, ok := c.peers[id]
return n, ok
}
func (c *localNodeContext) UserByID(id tailcfg.UserID) (_ tailcfg.UserProfileView, ok bool) {
c.mu.Lock()
nm := c.netMap
c.mu.Unlock()
if nm == nil {
return tailcfg.UserProfileView{}, false
}
u, ok := nm.UserProfiles[id]
return u, ok
}
// Peers returns all the current peers in an undefined order.
func (c *localNodeContext) Peers() []tailcfg.NodeView {
c.mu.Lock()
defer c.mu.Unlock()
return slicesx.MapValues(c.peers)
}
// unlockedNodesPermitted reports whether any peer with theUnsignedPeerAPIOnly bool set true has any of its allowed IPs
// in the specified packet filter.
//
// TODO(nickkhyl): It is here temporarily until we can move the whole [LocalBackend.updateFilterLocked] here,
// but change it so it builds and returns a filter for the current netmap/prefs instead of re-configuring the engine filter.
// Something like (*localNodeContext).RebuildFilters() (filter, jailedFilter *filter.Filter, changed bool) perhaps?
func (c *localNodeContext) unlockedNodesPermitted(packetFilter []filter.Match) bool {
c.mu.Lock()
defer c.mu.Unlock()
return packetFilterPermitsUnlockedNodes(c.peers, packetFilter)
}
func (c *localNodeContext) filter() *filter.Filter {
return c.filterAtomic.Load()
}

View File

@ -920,15 +920,15 @@ func TestWatchNotificationsCallbacks(t *testing.T) {
// tests LocalBackend.updateNetmapDeltaLocked // tests LocalBackend.updateNetmapDeltaLocked
func TestUpdateNetmapDelta(t *testing.T) { func TestUpdateNetmapDelta(t *testing.T) {
b := newTestLocalBackend(t) b := newTestLocalBackend(t)
if b.updateNetmapDeltaLocked(nil) { if b.currentNode().UpdateNetmapDelta(nil) {
t.Errorf("updateNetmapDeltaLocked() = true, want false with nil netmap") t.Errorf("updateNetmapDeltaLocked() = true, want false with nil netmap")
} }
b.netMap = &netmap.NetworkMap{} nm := &netmap.NetworkMap{}
for i := range 5 { for i := range 5 {
b.netMap.Peers = append(b.netMap.Peers, (&tailcfg.Node{ID: (tailcfg.NodeID(i) + 1)}).View()) nm.Peers = append(nm.Peers, (&tailcfg.Node{ID: (tailcfg.NodeID(i) + 1)}).View())
} }
b.updatePeersFromNetmapLocked(b.netMap) b.currentNode().SetNetMap(nm)
someTime := time.Unix(123, 0) someTime := time.Unix(123, 0)
muts, ok := netmap.MutationsFromMapResponse(&tailcfg.MapResponse{ muts, ok := netmap.MutationsFromMapResponse(&tailcfg.MapResponse{
@ -955,7 +955,7 @@ func TestUpdateNetmapDelta(t *testing.T) {
t.Fatal("netmap.MutationsFromMapResponse failed") t.Fatal("netmap.MutationsFromMapResponse failed")
} }
if !b.updateNetmapDeltaLocked(muts) { if !b.currentNode().UpdateNetmapDelta(muts) {
t.Fatalf("updateNetmapDeltaLocked() = false, want true with new netmap") t.Fatalf("updateNetmapDeltaLocked() = false, want true with new netmap")
} }
@ -978,9 +978,9 @@ func TestUpdateNetmapDelta(t *testing.T) {
}, },
} }
for _, want := range wants { for _, want := range wants {
gotv, ok := b.peers[want.ID] gotv, ok := b.currentNode().PeerByID(want.ID)
if !ok { if !ok {
t.Errorf("netmap.Peer %v missing from b.peers", want.ID) t.Errorf("netmap.Peer %v missing from b.profile.Peers", want.ID)
continue continue
} }
got := gotv.AsStruct() got := gotv.AsStruct()
@ -1398,7 +1398,7 @@ func TestCoveredRouteRangeNoDefault(t *testing.T) {
func TestReconfigureAppConnector(t *testing.T) { func TestReconfigureAppConnector(t *testing.T) {
b := newTestBackend(t) b := newTestBackend(t)
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs) b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
if b.appConnector != nil { if b.appConnector != nil {
t.Fatal("unexpected app connector") t.Fatal("unexpected app connector")
} }
@ -1411,7 +1411,7 @@ func TestReconfigureAppConnector(t *testing.T) {
}, },
AppConnectorSet: true, AppConnectorSet: true,
}) })
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs) b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
if b.appConnector == nil { if b.appConnector == nil {
t.Fatal("expected app connector") t.Fatal("expected app connector")
} }
@ -1422,15 +1422,19 @@ func TestReconfigureAppConnector(t *testing.T) {
"connectors": ["tag:example"] "connectors": ["tag:example"]
}` }`
b.netMap.SelfNode = (&tailcfg.Node{ nm := &netmap.NetworkMap{
Name: "example.ts.net", SelfNode: (&tailcfg.Node{
Tags: []string{"tag:example"}, Name: "example.ts.net",
CapMap: (tailcfg.NodeCapMap)(map[tailcfg.NodeCapability][]tailcfg.RawMessage{ Tags: []string{"tag:example"},
"tailscale.com/app-connectors": {tailcfg.RawMessage(appCfg)}, CapMap: (tailcfg.NodeCapMap)(map[tailcfg.NodeCapability][]tailcfg.RawMessage{
}), "tailscale.com/app-connectors": {tailcfg.RawMessage(appCfg)},
}).View() }),
}).View(),
}
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs) b.currentNode().SetNetMap(nm)
b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
b.appConnector.Wait(context.Background()) b.appConnector.Wait(context.Background())
want := []string{"example.com"} want := []string{"example.com"}
@ -1450,7 +1454,7 @@ func TestReconfigureAppConnector(t *testing.T) {
}, },
AppConnectorSet: true, AppConnectorSet: true,
}) })
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs) b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
if b.appConnector != nil { if b.appConnector != nil {
t.Fatal("expected no app connector") t.Fatal("expected no app connector")
} }
@ -1482,7 +1486,7 @@ func TestBackfillAppConnectorRoutes(t *testing.T) {
}); err != nil { }); err != nil {
t.Fatal(err) t.Fatal(err)
} }
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs) b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
// Smoke check that AdvertiseRoutes doesn't have the test IP. // Smoke check that AdvertiseRoutes doesn't have the test IP.
ip := netip.MustParseAddr("1.2.3.4") ip := netip.MustParseAddr("1.2.3.4")
@ -1503,7 +1507,7 @@ func TestBackfillAppConnectorRoutes(t *testing.T) {
// Mimic b.authReconfigure for the app connector bits. // Mimic b.authReconfigure for the app connector bits.
b.mu.Lock() b.mu.Lock()
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs) b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
b.mu.Unlock() b.mu.Unlock()
b.readvertiseAppConnectorRoutes() b.readvertiseAppConnectorRoutes()
@ -1819,7 +1823,7 @@ func TestSetExitNodeIDPolicy(t *testing.T) {
} }
pm := must.Get(newProfileManager(new(mem.Store), t.Logf, new(health.Tracker))) pm := must.Get(newProfileManager(new(mem.Store), t.Logf, new(health.Tracker)))
pm.prefs = test.prefs.View() pm.prefs = test.prefs.View()
b.netMap = test.nm b.currentNode().SetNetMap(test.nm)
b.pm = pm b.pm = pm
b.lastSuggestedExitNode = test.lastSuggestedExitNode b.lastSuggestedExitNode = test.lastSuggestedExitNode
@ -1946,8 +1950,7 @@ func TestUpdateNetmapDeltaAutoExitNode(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
b := newTestLocalBackend(t) b := newTestLocalBackend(t)
b.netMap = tt.netmap b.currentNode().SetNetMap(tt.netmap)
b.updatePeersFromNetmapLocked(b.netMap)
b.lastSuggestedExitNode = tt.lastSuggestedExitNode b.lastSuggestedExitNode = tt.lastSuggestedExitNode
b.sys.MagicSock.Get().SetLastNetcheckReportForTest(b.ctx, tt.report) b.sys.MagicSock.Get().SetLastNetcheckReportForTest(b.ctx, tt.report)
b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct()) b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct())
@ -2065,14 +2068,14 @@ func TestAutoExitNodeSetNetInfoCallback(t *testing.T) {
}, },
}, },
} }
b.netMap = &netmap.NetworkMap{ b.currentNode().SetNetMap(&netmap.NetworkMap{
SelfNode: selfNode.View(), SelfNode: selfNode.View(),
Peers: []tailcfg.NodeView{ Peers: []tailcfg.NodeView{
peer1, peer1,
peer2, peer2,
}, },
DERPMap: defaultDERPMap, DERPMap: defaultDERPMap,
} })
b.lastSuggestedExitNode = peer1.StableID() b.lastSuggestedExitNode = peer1.StableID()
b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct()) b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct())
if eid := b.Prefs().ExitNodeID(); eid != peer1.StableID() { if eid := b.Prefs().ExitNodeID(); eid != peer1.StableID() {
@ -2137,7 +2140,7 @@ func TestSetControlClientStatusAutoExitNode(t *testing.T) {
syspolicy.ExitNodeID, "auto:any", syspolicy.ExitNodeID, "auto:any",
)) ))
syspolicy.MustRegisterStoreForTest(t, "TestStore", setting.DeviceScope, policyStore) syspolicy.MustRegisterStoreForTest(t, "TestStore", setting.DeviceScope, policyStore)
b.netMap = nm b.currentNode().SetNetMap(nm)
b.lastSuggestedExitNode = peer1.StableID() b.lastSuggestedExitNode = peer1.StableID()
b.sys.MagicSock.Get().SetLastNetcheckReportForTest(b.ctx, report) b.sys.MagicSock.Get().SetLastNetcheckReportForTest(b.ctx, report)
b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct()) b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct())
@ -3068,9 +3071,11 @@ func TestDriveManageShares(t *testing.T) {
b.driveSetSharesLocked(tt.existing) b.driveSetSharesLocked(tt.existing)
} }
if !tt.disabled { if !tt.disabled {
self := b.netMap.SelfNode.AsStruct() nm := ptr.To(*b.currentNode().NetMap())
self := nm.SelfNode.AsStruct()
self.CapMap = tailcfg.NodeCapMap{tailcfg.NodeAttrsTaildriveShare: nil} self.CapMap = tailcfg.NodeCapMap{tailcfg.NodeAttrsTaildriveShare: nil}
b.netMap.SelfNode = self.View() nm.SelfNode = self.View()
b.currentNode().SetNetMap(nm)
b.sys.Set(driveimpl.NewFileSystemForRemote(b.logf)) b.sys.Set(driveimpl.NewFileSystemForRemote(b.logf))
} }
b.mu.Unlock() b.mu.Unlock()
@ -5323,7 +5328,7 @@ func TestSrcCapPacketFilter(t *testing.T) {
}}, }},
}) })
f := lb.GetFilterForTest() f := lb.currentNode().GetFilterForTest()
res := f.Check(netip.MustParseAddr("2.2.2.2"), netip.MustParseAddr("1.1.1.1"), 22, ipproto.TCP) res := f.Check(netip.MustParseAddr("2.2.2.2"), netip.MustParseAddr("1.1.1.1"), 22, ipproto.TCP)
if res != filter.Accept { if res != filter.Accept {
t.Errorf("Check(2.2.2.2, ...) = %s, want %s", res, filter.Accept) t.Errorf("Check(2.2.2.2, ...) = %s, want %s", res, filter.Accept)

View File

@ -516,9 +516,10 @@ func (b *LocalBackend) NetworkLockStatus() *ipnstate.NetworkLockStatus {
var selfAuthorized bool var selfAuthorized bool
nodeKeySignature := &tka.NodeKeySignature{} nodeKeySignature := &tka.NodeKeySignature{}
if b.netMap != nil { nm := b.currentNode().NetMap()
selfAuthorized = b.tka.authority.NodeKeyAuthorized(b.netMap.SelfNode.Key(), b.netMap.SelfNode.KeySignature().AsSlice()) == nil if nm != nil {
if err := nodeKeySignature.Unserialize(b.netMap.SelfNode.KeySignature().AsSlice()); err != nil { selfAuthorized = b.tka.authority.NodeKeyAuthorized(nm.SelfNode.Key(), nm.SelfNode.KeySignature().AsSlice()) == nil
if err := nodeKeySignature.Unserialize(nm.SelfNode.KeySignature().AsSlice()); err != nil {
b.logf("failed to decode self node key signature: %v", err) b.logf("failed to decode self node key signature: %v", err)
} }
} }
@ -539,9 +540,9 @@ func (b *LocalBackend) NetworkLockStatus() *ipnstate.NetworkLockStatus {
} }
var visible []*ipnstate.TKAPeer var visible []*ipnstate.TKAPeer
if b.netMap != nil { if nm != nil {
visible = make([]*ipnstate.TKAPeer, len(b.netMap.Peers)) visible = make([]*ipnstate.TKAPeer, len(nm.Peers))
for i, p := range b.netMap.Peers { for i, p := range nm.Peers {
s := tkaStateFromPeer(p) s := tkaStateFromPeer(p)
visible[i] = &s visible[i] = &s
} }
@ -702,12 +703,10 @@ func (b *LocalBackend) NetworkLockForceLocalDisable() error {
id1, id2 := b.tka.authority.StateIDs() id1, id2 := b.tka.authority.StateIDs()
stateID := fmt.Sprintf("%d:%d", id1, id2) stateID := fmt.Sprintf("%d:%d", id1, id2)
cn := b.currentNode()
newPrefs := b.pm.CurrentPrefs().AsStruct().Clone() // .Persist should always be initialized here. newPrefs := b.pm.CurrentPrefs().AsStruct().Clone() // .Persist should always be initialized here.
newPrefs.Persist.DisallowedTKAStateIDs = append(newPrefs.Persist.DisallowedTKAStateIDs, stateID) newPrefs.Persist.DisallowedTKAStateIDs = append(newPrefs.Persist.DisallowedTKAStateIDs, stateID)
if err := b.pm.SetPrefs(newPrefs.View(), ipn.NetworkProfile{ if err := b.pm.SetPrefs(newPrefs.View(), cn.NetworkProfile()); err != nil {
MagicDNSName: b.netMap.MagicDNSSuffix(),
DomainName: b.netMap.DomainName(),
}); err != nil {
return fmt.Errorf("saving prefs: %w", err) return fmt.Errorf("saving prefs: %w", err)
} }

View File

@ -770,7 +770,7 @@ func (h *peerAPIHandler) replyToDNSQueries() bool {
// but an app connector explicitly adds 0.0.0.0/32 (and the // but an app connector explicitly adds 0.0.0.0/32 (and the
// IPv6 equivalent) to make this work (see updateFilterLocked // IPv6 equivalent) to make this work (see updateFilterLocked
// in LocalBackend). // in LocalBackend).
f := b.filterAtomic.Load() f := b.currentNode().filter()
if f == nil { if f == nil {
return false return false
} }

View File

@ -162,9 +162,9 @@ func TestHandlePeerAPI(t *testing.T) {
lb := &LocalBackend{ lb := &LocalBackend{
logf: e.logBuf.Logf, logf: e.logBuf.Logf,
capFileSharing: tt.capSharing, capFileSharing: tt.capSharing,
netMap: &netmap.NetworkMap{SelfNode: selfNode.View()},
clock: &tstest.Clock{}, clock: &tstest.Clock{},
} }
lb.currentNode().SetNetMap(&netmap.NetworkMap{SelfNode: selfNode.View()})
e.ph = &peerAPIHandler{ e.ph = &peerAPIHandler{
isSelf: tt.isSelf, isSelf: tt.isSelf,
selfNode: selfNode.View(), selfNode: selfNode.View(),

View File

@ -232,7 +232,7 @@ func (b *LocalBackend) updateServeTCPPortNetMapAddrListenersLocked(ports []uint1
} }
} }
nm := b.netMap nm := b.NetMap()
if nm == nil { if nm == nil {
b.logf("netMap is nil") b.logf("netMap is nil")
return return
@ -282,7 +282,7 @@ func (b *LocalBackend) setServeConfigLocked(config *ipn.ServeConfig, etag string
} }
} }
nm := b.netMap nm := b.NetMap()
if nm == nil { if nm == nil {
return errors.New("netMap is nil") return errors.New("netMap is nil")
} }

View File

@ -320,7 +320,7 @@ func TestServeConfigServices(t *testing.T) {
t.Fatal(err) t.Fatal(err)
} }
b.netMap = &netmap.NetworkMap{ b.currentNode().SetNetMap(&netmap.NetworkMap{
SelfNode: (&tailcfg.Node{ SelfNode: (&tailcfg.Node{
Name: "example.ts.net", Name: "example.ts.net",
CapMap: tailcfg.NodeCapMap{ CapMap: tailcfg.NodeCapMap{
@ -334,7 +334,7 @@ func TestServeConfigServices(t *testing.T) {
ProfilePicURL: "https://example.com/photo.jpg", ProfilePicURL: "https://example.com/photo.jpg",
}).View(), }).View(),
}, },
} })
tests := []struct { tests := []struct {
name string name string
@ -902,7 +902,7 @@ func newTestBackend(t *testing.T) *LocalBackend {
pm.currentProfile = (&ipn.LoginProfile{ID: "id0"}).View() pm.currentProfile = (&ipn.LoginProfile{ID: "id0"}).View()
b.pm = pm b.pm = pm
b.netMap = &netmap.NetworkMap{ b.currentNode().SetNetMap(&netmap.NetworkMap{
SelfNode: (&tailcfg.Node{ SelfNode: (&tailcfg.Node{
Name: "example.ts.net", Name: "example.ts.net",
}).View(), }).View(),
@ -913,24 +913,26 @@ func newTestBackend(t *testing.T) *LocalBackend {
ProfilePicURL: "https://example.com/photo.jpg", ProfilePicURL: "https://example.com/photo.jpg",
}).View(), }).View(),
}, },
} Peers: []tailcfg.NodeView{
b.peers = map[tailcfg.NodeID]tailcfg.NodeView{ (&tailcfg.Node{
152: (&tailcfg.Node{ ID: 152,
ID: 152, ComputedName: "some-peer",
ComputedName: "some-peer", User: tailcfg.UserID(1),
User: tailcfg.UserID(1), Addresses: []netip.Prefix{
}).View(), netip.MustParsePrefix("100.150.151.152/32"),
153: (&tailcfg.Node{ },
ID: 153, }).View(),
ComputedName: "some-tagged-peer", (&tailcfg.Node{
Tags: []string{"tag:server", "tag:test"}, ID: 153,
User: tailcfg.UserID(1), ComputedName: "some-tagged-peer",
}).View(), Tags: []string{"tag:server", "tag:test"},
} User: tailcfg.UserID(1),
b.nodeByAddr = map[netip.Addr]tailcfg.NodeID{ Addresses: []netip.Prefix{
netip.MustParseAddr("100.150.151.152"): 152, netip.MustParsePrefix("100.150.151.153/32"),
netip.MustParseAddr("100.150.151.153"): 153, },
} }).View(),
},
})
return b return b
} }

View File

@ -179,23 +179,32 @@ func (b *LocalBackend) HasCapFileSharing() bool {
func (b *LocalBackend) FileTargets() ([]*apitype.FileTarget, error) { func (b *LocalBackend) FileTargets() ([]*apitype.FileTarget, error) {
var ret []*apitype.FileTarget var ret []*apitype.FileTarget
b.mu.Lock() b.mu.Lock() // for b.{state,capFileSharing}
defer b.mu.Unlock() defer b.mu.Unlock()
nm := b.netMap cn := b.currentNode()
nm := cn.NetMap()
self := cn.SelfUserID()
if b.state != ipn.Running || nm == nil { if b.state != ipn.Running || nm == nil {
return nil, errors.New("not connected to the tailnet") return nil, errors.New("not connected to the tailnet")
} }
if !b.capFileSharing { if !b.capFileSharing {
return nil, errors.New("file sharing not enabled by Tailscale admin") return nil, errors.New("file sharing not enabled by Tailscale admin")
} }
for _, p := range b.peers { peers := cn.AppendMatchingPeers(nil, func(p tailcfg.NodeView) bool {
if !b.peerIsTaildropTargetLocked(p) { if !p.Valid() || p.Hostinfo().OS() == "tvOS" {
continue return false
} }
if p.Hostinfo().OS() == "tvOS" { if self != p.User() {
continue return false
} }
peerAPI := peerAPIBase(b.netMap, p) if p.Addresses().Len() != 0 && cn.PeerHasCap(p.Addresses().At(0).Addr(), tailcfg.PeerCapabilityFileSharingTarget) {
// Explicitly noted in the netmap ACL caps as a target.
return true
}
return false
})
for _, p := range peers {
peerAPI := cn.PeerAPIBase(p)
if peerAPI == "" { if peerAPI == "" {
continue continue
} }
@ -214,7 +223,9 @@ func (b *LocalBackend) taildropTargetStatus(p tailcfg.NodeView) ipnstate.Taildro
if b.state != ipn.Running { if b.state != ipn.Running {
return ipnstate.TaildropTargetIpnStateNotRunning return ipnstate.TaildropTargetIpnStateNotRunning
} }
if b.netMap == nil { cn := b.currentNode()
nm := cn.NetMap()
if nm == nil {
return ipnstate.TaildropTargetNoNetmapAvailable return ipnstate.TaildropTargetNoNetmapAvailable
} }
if !b.capFileSharing { if !b.capFileSharing {
@ -228,10 +239,10 @@ func (b *LocalBackend) taildropTargetStatus(p tailcfg.NodeView) ipnstate.Taildro
if !p.Valid() { if !p.Valid() {
return ipnstate.TaildropTargetNoPeerInfo return ipnstate.TaildropTargetNoPeerInfo
} }
if b.netMap.User() != p.User() { if nm.User() != p.User() {
// Different user must have the explicit file sharing target capability // Different user must have the explicit file sharing target capability
if p.Addresses().Len() == 0 || if p.Addresses().Len() == 0 || !cn.PeerHasCap(p.Addresses().At(0).Addr(), tailcfg.PeerCapabilityFileSharingTarget) {
!b.peerHasCapLocked(p.Addresses().At(0).Addr(), tailcfg.PeerCapabilityFileSharingTarget) { // Explicitly noted in the netmap ACL caps as a target.
return ipnstate.TaildropTargetOwnedByOtherUser return ipnstate.TaildropTargetOwnedByOtherUser
} }
} }
@ -239,32 +250,12 @@ func (b *LocalBackend) taildropTargetStatus(p tailcfg.NodeView) ipnstate.Taildro
if p.Hostinfo().OS() == "tvOS" { if p.Hostinfo().OS() == "tvOS" {
return ipnstate.TaildropTargetUnsupportedOS return ipnstate.TaildropTargetUnsupportedOS
} }
if peerAPIBase(b.netMap, p) == "" { if !cn.PeerHasPeerAPI(p) {
return ipnstate.TaildropTargetNoPeerAPI return ipnstate.TaildropTargetNoPeerAPI
} }
return ipnstate.TaildropTargetAvailable return ipnstate.TaildropTargetAvailable
} }
// peerIsTaildropTargetLocked reports whether p is a valid Taildrop file
// recipient from this node according to its ownership and the capabilities in
// the netmap.
//
// b.mu must be locked.
func (b *LocalBackend) peerIsTaildropTargetLocked(p tailcfg.NodeView) bool {
if b.netMap == nil || !p.Valid() {
return false
}
if b.netMap.User() == p.User() {
return true
}
if p.Addresses().Len() > 0 &&
b.peerHasCapLocked(p.Addresses().At(0).Addr(), tailcfg.PeerCapabilityFileSharingTarget) {
// Explicitly noted in the netmap ACL caps as a target.
return true
}
return false
}
// UpdateOutgoingFiles updates b.outgoingFiles to reflect the given updates and // UpdateOutgoingFiles updates b.outgoingFiles to reflect the given updates and
// sends an ipn.Notify with the full list of outgoingFiles. // sends an ipn.Notify with the full list of outgoingFiles.
func (b *LocalBackend) UpdateOutgoingFiles(updates map[string]*ipn.OutgoingFile) { func (b *LocalBackend) UpdateOutgoingFiles(updates map[string]*ipn.OutgoingFile) {

View File

@ -13,7 +13,6 @@ import (
"tailscale.com/tailcfg" "tailscale.com/tailcfg"
"tailscale.com/tstest/deptest" "tailscale.com/tstest/deptest"
"tailscale.com/types/netmap" "tailscale.com/types/netmap"
"tailscale.com/util/mak"
) )
func TestFileTargets(t *testing.T) { func TestFileTargets(t *testing.T) {
@ -23,7 +22,7 @@ func TestFileTargets(t *testing.T) {
t.Errorf("before connect: got %q; want %q", got, want) t.Errorf("before connect: got %q; want %q", got, want)
} }
b.netMap = new(netmap.NetworkMap) b.currentNode().SetNetMap(new(netmap.NetworkMap))
_, err = b.FileTargets() _, err = b.FileTargets()
if got, want := fmt.Sprint(err), "not connected to the tailnet"; got != want { if got, want := fmt.Sprint(err), "not connected to the tailnet"; got != want {
t.Errorf("non-running netmap: got %q; want %q", got, want) t.Errorf("non-running netmap: got %q; want %q", got, want)
@ -44,16 +43,15 @@ func TestFileTargets(t *testing.T) {
t.Fatalf("unexpected %d peers", len(got)) t.Fatalf("unexpected %d peers", len(got))
} }
var peerMap map[tailcfg.NodeID]tailcfg.NodeView nm := &netmap.NetworkMap{
mak.NonNil(&peerMap) Peers: []tailcfg.NodeView{
var nodeID tailcfg.NodeID (&tailcfg.Node{
nodeID = 1234 ID: 1234,
peer := &tailcfg.Node{ Hostinfo: (&tailcfg.Hostinfo{OS: "tvOS"}).View(),
ID: 1234, }).View(),
Hostinfo: (&tailcfg.Hostinfo{OS: "tvOS"}).View(), },
} }
peerMap[nodeID] = peer.View() b.currentNode().SetNetMap(nm)
b.peers = peerMap
got, err = b.FileTargets() got, err = b.FileTargets()
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)

View File

@ -116,11 +116,12 @@ func (b *LocalBackend) handleWebClientConn(c net.Conn) error {
// for each of the local device's Tailscale IP addresses. This is needed to properly // for each of the local device's Tailscale IP addresses. This is needed to properly
// route local traffic when using kernel networking mode. // route local traffic when using kernel networking mode.
func (b *LocalBackend) updateWebClientListenersLocked() { func (b *LocalBackend) updateWebClientListenersLocked() {
if b.netMap == nil { nm := b.currentNode().NetMap()
if nm == nil {
return return
} }
addrs := b.netMap.GetAddresses() addrs := nm.GetAddresses()
for _, pfx := range addrs.All() { for _, pfx := range addrs.All() {
addrPort := netip.AddrPortFrom(pfx.Addr(), webClientPort) addrPort := netip.AddrPortFrom(pfx.Addr(), webClientPort)
if _, ok := b.webClientListeners[addrPort]; ok { if _, ok := b.webClientListeners[addrPort]; ok {