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")
}
b.netMap = &netmap.NetworkMap{
b.currentNode().SetNetMap(&netmap.NetworkMap{
DNS: tailcfg.DNSConfig{
ExitNodeFilteredSet: []string{
".ts.net",
"some.exact.bad",
},
},
}
})
tests := []struct {
name string
want bool

View File

@ -4,7 +4,6 @@
package ipnlocal
import (
"cmp"
"fmt"
"os"
"slices"
@ -26,26 +25,14 @@ const (
// enabled. This is currently based on checking for the drive:share node
// attribute.
func (b *LocalBackend) DriveSharingEnabled() bool {
b.mu.Lock()
defer b.mu.Unlock()
return b.driveSharingEnabledLocked()
}
func (b *LocalBackend) driveSharingEnabledLocked() bool {
return b.netMap != nil && b.netMap.SelfNode.HasCap(tailcfg.NodeAttrsTaildriveShare)
return b.currentNode().SelfHasCap(tailcfg.NodeAttrsTaildriveShare)
}
// DriveAccessEnabled reports whether accessing Taildrive shares on remote nodes
// is enabled. This is currently based on checking for the drive:access node
// attribute.
func (b *LocalBackend) DriveAccessEnabled() bool {
b.mu.Lock()
defer b.mu.Unlock()
return b.driveAccessEnabledLocked()
}
func (b *LocalBackend) driveAccessEnabledLocked() bool {
return b.netMap != nil && b.netMap.SelfNode.HasCap(tailcfg.NodeAttrsTaildriveAccess)
return b.currentNode().SelfHasCap(tailcfg.NodeAttrsTaildriveAccess)
}
// 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.
func (b *LocalBackend) driveNotifyCurrentSharesLocked() {
var shares views.SliceView[*drive.Share, drive.ShareView]
if b.driveSharingEnabledLocked() {
if b.DriveSharingEnabled() {
// Only populate shares if sharing is enabled.
shares = b.pm.prefs.DriveShares()
}
@ -310,12 +297,12 @@ func (b *LocalBackend) updateDrivePeersLocked(nm *netmap.NetworkMap) {
}
var driveRemotes []*drive.Remote
if b.driveAccessEnabledLocked() {
if b.DriveAccessEnabled() {
// Only populate peers if access is enabled, otherwise leave blank.
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 {
@ -330,23 +317,20 @@ func (b *LocalBackend) driveRemotesFromPeers(nm *netmap.NetworkMap) []*drive.Rem
// Peers are available to Taildrive if:
// - They are online
// - They are allowed to share at least one folder with us
b.mu.Lock()
latestNetMap := b.netMap
b.mu.Unlock()
idx, found := slices.BinarySearchFunc(latestNetMap.Peers, peerID, func(candidate tailcfg.NodeView, id tailcfg.NodeID) int {
return cmp.Compare(candidate.ID(), id)
})
if !found {
cn := b.currentNode()
peer, ok := cn.PeerByID(peerID)
if !ok {
return false
}
peer := latestNetMap.Peers[idx]
// Exclude offline peers.
// TODO(oxtoacart): for some reason, this correctly
// catches when a node goes from offline to online,
// 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() {
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.
addresses := peer.Addresses()
for _, p := range addresses.All() {
capsMap := b.PeerCaps(p.Addr())
if capsMap.HasCapability(tailcfg.PeerCapabilityTaildriveSharer) {
if cn.PeerHasCap(p.Addr(), tailcfg.PeerCapabilityTaildriveSharer) {
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
func TestUpdateNetmapDelta(t *testing.T) {
b := newTestLocalBackend(t)
if b.updateNetmapDeltaLocked(nil) {
if b.currentNode().UpdateNetmapDelta(nil) {
t.Errorf("updateNetmapDeltaLocked() = true, want false with nil netmap")
}
b.netMap = &netmap.NetworkMap{}
nm := &netmap.NetworkMap{}
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)
muts, ok := netmap.MutationsFromMapResponse(&tailcfg.MapResponse{
@ -955,7 +955,7 @@ func TestUpdateNetmapDelta(t *testing.T) {
t.Fatal("netmap.MutationsFromMapResponse failed")
}
if !b.updateNetmapDeltaLocked(muts) {
if !b.currentNode().UpdateNetmapDelta(muts) {
t.Fatalf("updateNetmapDeltaLocked() = false, want true with new netmap")
}
@ -978,9 +978,9 @@ func TestUpdateNetmapDelta(t *testing.T) {
},
}
for _, want := range wants {
gotv, ok := b.peers[want.ID]
gotv, ok := b.currentNode().PeerByID(want.ID)
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
}
got := gotv.AsStruct()
@ -1398,7 +1398,7 @@ func TestCoveredRouteRangeNoDefault(t *testing.T) {
func TestReconfigureAppConnector(t *testing.T) {
b := newTestBackend(t)
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs)
b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
if b.appConnector != nil {
t.Fatal("unexpected app connector")
}
@ -1411,7 +1411,7 @@ func TestReconfigureAppConnector(t *testing.T) {
},
AppConnectorSet: true,
})
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs)
b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
if b.appConnector == nil {
t.Fatal("expected app connector")
}
@ -1422,15 +1422,19 @@ func TestReconfigureAppConnector(t *testing.T) {
"connectors": ["tag:example"]
}`
b.netMap.SelfNode = (&tailcfg.Node{
Name: "example.ts.net",
Tags: []string{"tag:example"},
CapMap: (tailcfg.NodeCapMap)(map[tailcfg.NodeCapability][]tailcfg.RawMessage{
"tailscale.com/app-connectors": {tailcfg.RawMessage(appCfg)},
}),
}).View()
nm := &netmap.NetworkMap{
SelfNode: (&tailcfg.Node{
Name: "example.ts.net",
Tags: []string{"tag:example"},
CapMap: (tailcfg.NodeCapMap)(map[tailcfg.NodeCapability][]tailcfg.RawMessage{
"tailscale.com/app-connectors": {tailcfg.RawMessage(appCfg)},
}),
}).View(),
}
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs)
b.currentNode().SetNetMap(nm)
b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
b.appConnector.Wait(context.Background())
want := []string{"example.com"}
@ -1450,7 +1454,7 @@ func TestReconfigureAppConnector(t *testing.T) {
},
AppConnectorSet: true,
})
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs)
b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
if b.appConnector != nil {
t.Fatal("expected no app connector")
}
@ -1482,7 +1486,7 @@ func TestBackfillAppConnectorRoutes(t *testing.T) {
}); err != nil {
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.
ip := netip.MustParseAddr("1.2.3.4")
@ -1503,7 +1507,7 @@ func TestBackfillAppConnectorRoutes(t *testing.T) {
// Mimic b.authReconfigure for the app connector bits.
b.mu.Lock()
b.reconfigAppConnectorLocked(b.netMap, b.pm.prefs)
b.reconfigAppConnectorLocked(b.NetMap(), b.pm.prefs)
b.mu.Unlock()
b.readvertiseAppConnectorRoutes()
@ -1819,7 +1823,7 @@ func TestSetExitNodeIDPolicy(t *testing.T) {
}
pm := must.Get(newProfileManager(new(mem.Store), t.Logf, new(health.Tracker)))
pm.prefs = test.prefs.View()
b.netMap = test.nm
b.currentNode().SetNetMap(test.nm)
b.pm = pm
b.lastSuggestedExitNode = test.lastSuggestedExitNode
@ -1946,8 +1950,7 @@ func TestUpdateNetmapDeltaAutoExitNode(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
b := newTestLocalBackend(t)
b.netMap = tt.netmap
b.updatePeersFromNetmapLocked(b.netMap)
b.currentNode().SetNetMap(tt.netmap)
b.lastSuggestedExitNode = tt.lastSuggestedExitNode
b.sys.MagicSock.Get().SetLastNetcheckReportForTest(b.ctx, tt.report)
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(),
Peers: []tailcfg.NodeView{
peer1,
peer2,
},
DERPMap: defaultDERPMap,
}
})
b.lastSuggestedExitNode = peer1.StableID()
b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct())
if eid := b.Prefs().ExitNodeID(); eid != peer1.StableID() {
@ -2137,7 +2140,7 @@ func TestSetControlClientStatusAutoExitNode(t *testing.T) {
syspolicy.ExitNodeID, "auto:any",
))
syspolicy.MustRegisterStoreForTest(t, "TestStore", setting.DeviceScope, policyStore)
b.netMap = nm
b.currentNode().SetNetMap(nm)
b.lastSuggestedExitNode = peer1.StableID()
b.sys.MagicSock.Get().SetLastNetcheckReportForTest(b.ctx, report)
b.SetPrefsForTest(b.pm.CurrentPrefs().AsStruct())
@ -3068,9 +3071,11 @@ func TestDriveManageShares(t *testing.T) {
b.driveSetSharesLocked(tt.existing)
}
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}
b.netMap.SelfNode = self.View()
nm.SelfNode = self.View()
b.currentNode().SetNetMap(nm)
b.sys.Set(driveimpl.NewFileSystemForRemote(b.logf))
}
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)
if 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
nodeKeySignature := &tka.NodeKeySignature{}
if b.netMap != nil {
selfAuthorized = b.tka.authority.NodeKeyAuthorized(b.netMap.SelfNode.Key(), b.netMap.SelfNode.KeySignature().AsSlice()) == nil
if err := nodeKeySignature.Unserialize(b.netMap.SelfNode.KeySignature().AsSlice()); err != nil {
nm := b.currentNode().NetMap()
if nm != 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)
}
}
@ -539,9 +540,9 @@ func (b *LocalBackend) NetworkLockStatus() *ipnstate.NetworkLockStatus {
}
var visible []*ipnstate.TKAPeer
if b.netMap != nil {
visible = make([]*ipnstate.TKAPeer, len(b.netMap.Peers))
for i, p := range b.netMap.Peers {
if nm != nil {
visible = make([]*ipnstate.TKAPeer, len(nm.Peers))
for i, p := range nm.Peers {
s := tkaStateFromPeer(p)
visible[i] = &s
}
@ -702,12 +703,10 @@ func (b *LocalBackend) NetworkLockForceLocalDisable() error {
id1, id2 := b.tka.authority.StateIDs()
stateID := fmt.Sprintf("%d:%d", id1, id2)
cn := b.currentNode()
newPrefs := b.pm.CurrentPrefs().AsStruct().Clone() // .Persist should always be initialized here.
newPrefs.Persist.DisallowedTKAStateIDs = append(newPrefs.Persist.DisallowedTKAStateIDs, stateID)
if err := b.pm.SetPrefs(newPrefs.View(), ipn.NetworkProfile{
MagicDNSName: b.netMap.MagicDNSSuffix(),
DomainName: b.netMap.DomainName(),
}); err != nil {
if err := b.pm.SetPrefs(newPrefs.View(), cn.NetworkProfile()); err != nil {
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
// IPv6 equivalent) to make this work (see updateFilterLocked
// in LocalBackend).
f := b.filterAtomic.Load()
f := b.currentNode().filter()
if f == nil {
return false
}

View File

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

View File

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

View File

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

View File

@ -179,23 +179,32 @@ func (b *LocalBackend) HasCapFileSharing() bool {
func (b *LocalBackend) FileTargets() ([]*apitype.FileTarget, error) {
var ret []*apitype.FileTarget
b.mu.Lock()
b.mu.Lock() // for b.{state,capFileSharing}
defer b.mu.Unlock()
nm := b.netMap
cn := b.currentNode()
nm := cn.NetMap()
self := cn.SelfUserID()
if b.state != ipn.Running || nm == nil {
return nil, errors.New("not connected to the tailnet")
}
if !b.capFileSharing {
return nil, errors.New("file sharing not enabled by Tailscale admin")
}
for _, p := range b.peers {
if !b.peerIsTaildropTargetLocked(p) {
continue
peers := cn.AppendMatchingPeers(nil, func(p tailcfg.NodeView) bool {
if !p.Valid() || p.Hostinfo().OS() == "tvOS" {
return false
}
if p.Hostinfo().OS() == "tvOS" {
continue
if self != p.User() {
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 == "" {
continue
}
@ -214,7 +223,9 @@ func (b *LocalBackend) taildropTargetStatus(p tailcfg.NodeView) ipnstate.Taildro
if b.state != ipn.Running {
return ipnstate.TaildropTargetIpnStateNotRunning
}
if b.netMap == nil {
cn := b.currentNode()
nm := cn.NetMap()
if nm == nil {
return ipnstate.TaildropTargetNoNetmapAvailable
}
if !b.capFileSharing {
@ -228,10 +239,10 @@ func (b *LocalBackend) taildropTargetStatus(p tailcfg.NodeView) ipnstate.Taildro
if !p.Valid() {
return ipnstate.TaildropTargetNoPeerInfo
}
if b.netMap.User() != p.User() {
if nm.User() != p.User() {
// Different user must have the explicit file sharing target capability
if p.Addresses().Len() == 0 ||
!b.peerHasCapLocked(p.Addresses().At(0).Addr(), tailcfg.PeerCapabilityFileSharingTarget) {
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 ipnstate.TaildropTargetOwnedByOtherUser
}
}
@ -239,32 +250,12 @@ func (b *LocalBackend) taildropTargetStatus(p tailcfg.NodeView) ipnstate.Taildro
if p.Hostinfo().OS() == "tvOS" {
return ipnstate.TaildropTargetUnsupportedOS
}
if peerAPIBase(b.netMap, p) == "" {
if !cn.PeerHasPeerAPI(p) {
return ipnstate.TaildropTargetNoPeerAPI
}
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
// sends an ipn.Notify with the full list of outgoingFiles.
func (b *LocalBackend) UpdateOutgoingFiles(updates map[string]*ipn.OutgoingFile) {

View File

@ -13,7 +13,6 @@ import (
"tailscale.com/tailcfg"
"tailscale.com/tstest/deptest"
"tailscale.com/types/netmap"
"tailscale.com/util/mak"
)
func TestFileTargets(t *testing.T) {
@ -23,7 +22,7 @@ func TestFileTargets(t *testing.T) {
t.Errorf("before connect: got %q; want %q", got, want)
}
b.netMap = new(netmap.NetworkMap)
b.currentNode().SetNetMap(new(netmap.NetworkMap))
_, err = b.FileTargets()
if got, want := fmt.Sprint(err), "not connected to the tailnet"; 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))
}
var peerMap map[tailcfg.NodeID]tailcfg.NodeView
mak.NonNil(&peerMap)
var nodeID tailcfg.NodeID
nodeID = 1234
peer := &tailcfg.Node{
ID: 1234,
Hostinfo: (&tailcfg.Hostinfo{OS: "tvOS"}).View(),
nm := &netmap.NetworkMap{
Peers: []tailcfg.NodeView{
(&tailcfg.Node{
ID: 1234,
Hostinfo: (&tailcfg.Hostinfo{OS: "tvOS"}).View(),
}).View(),
},
}
peerMap[nodeID] = peer.View()
b.peers = peerMap
b.currentNode().SetNetMap(nm)
got, err = b.FileTargets()
if err != nil {
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
// route local traffic when using kernel networking mode.
func (b *LocalBackend) updateWebClientListenersLocked() {
if b.netMap == nil {
nm := b.currentNode().NetMap()
if nm == nil {
return
}
addrs := b.netMap.GetAddresses()
addrs := nm.GetAddresses()
for _, pfx := range addrs.All() {
addrPort := netip.AddrPortFrom(pfx.Addr(), webClientPort)
if _, ok := b.webClientListeners[addrPort]; ok {