mirror of
https://github.com/coder/coder.git
synced 2025-07-23 21:32:07 +00:00
feat: use tailnet v2 API for coordination (#11638)
This one is huge, and I'm sorry. The problem is that once I change `tailnet.Conn` to start doing v2 behavior, I kind of have to change it everywhere, including in CoderSDK (CLI), the agent, wsproxy, and ServerTailnet. There is still a bit more cleanup to do, and I need to add code so that when we lose connection to the Coordinator, we mark all peers as LOST, but that will be in a separate PR since this is big enough!
This commit is contained in:
519
tailnet/conn.go
519
tailnet/conn.go
@ -3,48 +3,40 @@ package tailnet
|
||||
import (
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/netip"
|
||||
"os"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cenkalti/backoff/v4"
|
||||
"github.com/google/uuid"
|
||||
"go4.org/netipx"
|
||||
"golang.org/x/xerrors"
|
||||
"gvisor.dev/gvisor/pkg/tcpip"
|
||||
"gvisor.dev/gvisor/pkg/tcpip/adapters/gonet"
|
||||
"tailscale.com/envknob"
|
||||
"tailscale.com/ipn/ipnstate"
|
||||
"tailscale.com/net/connstats"
|
||||
"tailscale.com/net/dns"
|
||||
"tailscale.com/net/netmon"
|
||||
"tailscale.com/net/netns"
|
||||
"tailscale.com/net/tsdial"
|
||||
"tailscale.com/net/tstun"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/tsd"
|
||||
"tailscale.com/types/ipproto"
|
||||
"tailscale.com/types/key"
|
||||
tslogger "tailscale.com/types/logger"
|
||||
"tailscale.com/types/netlogtype"
|
||||
"tailscale.com/types/netmap"
|
||||
"tailscale.com/wgengine"
|
||||
"tailscale.com/wgengine/filter"
|
||||
"tailscale.com/wgengine/magicsock"
|
||||
"tailscale.com/wgengine/netstack"
|
||||
"tailscale.com/wgengine/router"
|
||||
"tailscale.com/wgengine/wgcfg/nmcfg"
|
||||
|
||||
"cdr.dev/slog"
|
||||
"github.com/coder/coder/v2/coderd/database/dbtime"
|
||||
"github.com/coder/coder/v2/cryptorand"
|
||||
"github.com/coder/coder/v2/tailnet/proto"
|
||||
)
|
||||
|
||||
var ErrConnClosed = xerrors.New("connection closed")
|
||||
@ -128,42 +120,6 @@ func NewConn(options *Options) (conn *Conn, err error) {
|
||||
}
|
||||
|
||||
nodePrivateKey := key.NewNode()
|
||||
nodePublicKey := nodePrivateKey.Public()
|
||||
|
||||
netMap := &netmap.NetworkMap{
|
||||
DERPMap: options.DERPMap,
|
||||
NodeKey: nodePublicKey,
|
||||
PrivateKey: nodePrivateKey,
|
||||
Addresses: options.Addresses,
|
||||
PacketFilter: []filter.Match{{
|
||||
// Allow any protocol!
|
||||
IPProto: []ipproto.Proto{ipproto.TCP, ipproto.UDP, ipproto.ICMPv4, ipproto.ICMPv6, ipproto.SCTP},
|
||||
// Allow traffic sourced from anywhere.
|
||||
Srcs: []netip.Prefix{
|
||||
netip.PrefixFrom(netip.AddrFrom4([4]byte{}), 0),
|
||||
netip.PrefixFrom(netip.AddrFrom16([16]byte{}), 0),
|
||||
},
|
||||
// Allow traffic to route anywhere.
|
||||
Dsts: []filter.NetPortRange{
|
||||
{
|
||||
Net: netip.PrefixFrom(netip.AddrFrom4([4]byte{}), 0),
|
||||
Ports: filter.PortRange{
|
||||
First: 0,
|
||||
Last: 65535,
|
||||
},
|
||||
},
|
||||
{
|
||||
Net: netip.PrefixFrom(netip.AddrFrom16([16]byte{}), 0),
|
||||
Ports: filter.PortRange{
|
||||
First: 0,
|
||||
Last: 65535,
|
||||
},
|
||||
},
|
||||
},
|
||||
Caps: []filter.CapMatch{},
|
||||
}},
|
||||
}
|
||||
|
||||
var nodeID tailcfg.NodeID
|
||||
|
||||
// If we're provided with a UUID, use it to populate our node ID.
|
||||
@ -177,14 +133,6 @@ func NewConn(options *Options) (conn *Conn, err error) {
|
||||
nodeID = tailcfg.NodeID(uid)
|
||||
}
|
||||
|
||||
// This is used by functions below to identify the node via key
|
||||
netMap.SelfNode = &tailcfg.Node{
|
||||
ID: nodeID,
|
||||
Key: nodePublicKey,
|
||||
Addresses: options.Addresses,
|
||||
AllowedIPs: options.Addresses,
|
||||
}
|
||||
|
||||
wireguardMonitor, err := netmon.New(Logger(options.Logger.Named("net.wgmonitor")))
|
||||
if err != nil {
|
||||
return nil, xerrors.Errorf("create wireguard link monitor: %w", err)
|
||||
@ -243,7 +191,6 @@ func NewConn(options *Options) (conn *Conn, err error) {
|
||||
if err != nil {
|
||||
return nil, xerrors.Errorf("set node private key: %w", err)
|
||||
}
|
||||
netMap.SelfNode.DiscoKey = magicConn.DiscoPublicKey()
|
||||
|
||||
netStack, err := netstack.Create(
|
||||
Logger(options.Logger.Named("net.netstack")),
|
||||
@ -262,44 +209,46 @@ func NewConn(options *Options) (conn *Conn, err error) {
|
||||
}
|
||||
netStack.ProcessLocalIPs = true
|
||||
wireguardEngine = wgengine.NewWatchdog(wireguardEngine)
|
||||
wireguardEngine.SetDERPMap(options.DERPMap)
|
||||
netMapCopy := *netMap
|
||||
options.Logger.Debug(context.Background(), "updating network map")
|
||||
wireguardEngine.SetNetworkMap(&netMapCopy)
|
||||
|
||||
localIPSet := netipx.IPSetBuilder{}
|
||||
for _, addr := range netMap.Addresses {
|
||||
localIPSet.AddPrefix(addr)
|
||||
}
|
||||
localIPs, _ := localIPSet.IPSet()
|
||||
logIPSet := netipx.IPSetBuilder{}
|
||||
logIPs, _ := logIPSet.IPSet()
|
||||
wireguardEngine.SetFilter(filter.New(
|
||||
netMap.PacketFilter,
|
||||
localIPs,
|
||||
logIPs,
|
||||
cfgMaps := newConfigMaps(
|
||||
options.Logger,
|
||||
wireguardEngine,
|
||||
nodeID,
|
||||
nodePrivateKey,
|
||||
magicConn.DiscoPublicKey(),
|
||||
)
|
||||
cfgMaps.setAddresses(options.Addresses)
|
||||
cfgMaps.setDERPMap(DERPMapToProto(options.DERPMap))
|
||||
cfgMaps.setBlockEndpoints(options.BlockEndpoints)
|
||||
|
||||
nodeUp := newNodeUpdater(
|
||||
options.Logger,
|
||||
nil,
|
||||
Logger(options.Logger.Named("net.packet-filter")),
|
||||
))
|
||||
nodeID,
|
||||
nodePrivateKey.Public(),
|
||||
magicConn.DiscoPublicKey(),
|
||||
)
|
||||
nodeUp.setAddresses(options.Addresses)
|
||||
nodeUp.setBlockEndpoints(options.BlockEndpoints)
|
||||
wireguardEngine.SetStatusCallback(nodeUp.setStatus)
|
||||
wireguardEngine.SetNetInfoCallback(nodeUp.setNetInfo)
|
||||
magicConn.SetDERPForcedWebsocketCallback(nodeUp.setDERPForcedWebsocket)
|
||||
|
||||
server := &Conn{
|
||||
blockEndpoints: options.BlockEndpoints,
|
||||
derpForceWebSockets: options.DERPForceWebSockets,
|
||||
closed: make(chan struct{}),
|
||||
logger: options.Logger,
|
||||
magicConn: magicConn,
|
||||
dialer: dialer,
|
||||
listeners: map[listenKey]*listener{},
|
||||
peerMap: map[tailcfg.NodeID]*tailcfg.Node{},
|
||||
lastDERPForcedWebSockets: map[int]string{},
|
||||
tunDevice: sys.Tun.Get(),
|
||||
netMap: netMap,
|
||||
netStack: netStack,
|
||||
wireguardMonitor: wireguardMonitor,
|
||||
closed: make(chan struct{}),
|
||||
logger: options.Logger,
|
||||
magicConn: magicConn,
|
||||
dialer: dialer,
|
||||
listeners: map[listenKey]*listener{},
|
||||
tunDevice: sys.Tun.Get(),
|
||||
netStack: netStack,
|
||||
wireguardMonitor: wireguardMonitor,
|
||||
wireguardRouter: &router.Config{
|
||||
LocalAddrs: netMap.Addresses,
|
||||
LocalAddrs: options.Addresses,
|
||||
},
|
||||
wireguardEngine: wireguardEngine,
|
||||
configMaps: cfgMaps,
|
||||
nodeUpdater: nodeUp,
|
||||
}
|
||||
defer func() {
|
||||
if err != nil {
|
||||
@ -307,52 +256,6 @@ func NewConn(options *Options) (conn *Conn, err error) {
|
||||
}
|
||||
}()
|
||||
|
||||
wireguardEngine.SetStatusCallback(func(s *wgengine.Status, err error) {
|
||||
server.logger.Debug(context.Background(), "wireguard status", slog.F("status", s), slog.Error(err))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
server.lastMutex.Lock()
|
||||
if s.AsOf.Before(server.lastStatus) {
|
||||
// Don't process outdated status!
|
||||
server.lastMutex.Unlock()
|
||||
return
|
||||
}
|
||||
server.lastStatus = s.AsOf
|
||||
if endpointsEqual(s.LocalAddrs, server.lastEndpoints) {
|
||||
// No need to update the node if nothing changed!
|
||||
server.lastMutex.Unlock()
|
||||
return
|
||||
}
|
||||
server.lastEndpoints = append([]tailcfg.Endpoint{}, s.LocalAddrs...)
|
||||
server.lastMutex.Unlock()
|
||||
server.sendNode()
|
||||
})
|
||||
|
||||
wireguardEngine.SetNetInfoCallback(func(ni *tailcfg.NetInfo) {
|
||||
server.logger.Debug(context.Background(), "netinfo callback", slog.F("netinfo", ni))
|
||||
server.lastMutex.Lock()
|
||||
if reflect.DeepEqual(server.lastNetInfo, ni) {
|
||||
server.lastMutex.Unlock()
|
||||
return
|
||||
}
|
||||
server.lastNetInfo = ni.Clone()
|
||||
server.lastMutex.Unlock()
|
||||
server.sendNode()
|
||||
})
|
||||
|
||||
magicConn.SetDERPForcedWebsocketCallback(func(region int, reason string) {
|
||||
server.logger.Debug(context.Background(), "derp forced websocket", slog.F("region", region), slog.F("reason", reason))
|
||||
server.lastMutex.Lock()
|
||||
if server.lastDERPForcedWebSockets[region] == reason {
|
||||
server.lastMutex.Unlock()
|
||||
return
|
||||
}
|
||||
server.lastDERPForcedWebSockets[region] = reason
|
||||
server.lastMutex.Unlock()
|
||||
server.sendNode()
|
||||
})
|
||||
|
||||
netStack.GetTCPHandlerForFlow = server.forwardTCP
|
||||
|
||||
err = netStack.Start(nil)
|
||||
@ -389,16 +292,14 @@ func IPFromUUID(uid uuid.UUID) netip.Addr {
|
||||
|
||||
// Conn is an actively listening Wireguard connection.
|
||||
type Conn struct {
|
||||
mutex sync.Mutex
|
||||
closed chan struct{}
|
||||
logger slog.Logger
|
||||
blockEndpoints bool
|
||||
derpForceWebSockets bool
|
||||
mutex sync.Mutex
|
||||
closed chan struct{}
|
||||
logger slog.Logger
|
||||
|
||||
dialer *tsdial.Dialer
|
||||
tunDevice *tstun.Wrapper
|
||||
peerMap map[tailcfg.NodeID]*tailcfg.Node
|
||||
netMap *netmap.NetworkMap
|
||||
configMaps *configMaps
|
||||
nodeUpdater *nodeUpdater
|
||||
netStack *netstack.Impl
|
||||
magicConn *magicsock.Conn
|
||||
wireguardMonitor *netmon.Monitor
|
||||
@ -406,17 +307,6 @@ type Conn struct {
|
||||
wireguardEngine wgengine.Engine
|
||||
listeners map[listenKey]*listener
|
||||
|
||||
lastMutex sync.Mutex
|
||||
nodeSending bool
|
||||
nodeChanged bool
|
||||
// It's only possible to store these values via status functions,
|
||||
// so the values must be stored for retrieval later on.
|
||||
lastStatus time.Time
|
||||
lastEndpoints []tailcfg.Endpoint
|
||||
lastDERPForcedWebSockets map[int]string
|
||||
lastNetInfo *tailcfg.NetInfo
|
||||
nodeCallback func(node *Node)
|
||||
|
||||
trafficStats *connstats.Statistics
|
||||
}
|
||||
|
||||
@ -425,57 +315,30 @@ func (c *Conn) MagicsockSetDebugLoggingEnabled(enabled bool) {
|
||||
}
|
||||
|
||||
func (c *Conn) SetAddresses(ips []netip.Prefix) error {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
c.netMap.Addresses = ips
|
||||
|
||||
netMapCopy := *c.netMap
|
||||
c.logger.Debug(context.Background(), "updating network map")
|
||||
c.wireguardEngine.SetNetworkMap(&netMapCopy)
|
||||
err := c.reconfig()
|
||||
if err != nil {
|
||||
return xerrors.Errorf("reconfig: %w", err)
|
||||
}
|
||||
|
||||
c.configMaps.setAddresses(ips)
|
||||
c.nodeUpdater.setAddresses(ips)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Conn) Addresses() []netip.Prefix {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
return c.netMap.Addresses
|
||||
}
|
||||
|
||||
func (c *Conn) SetNodeCallback(callback func(node *Node)) {
|
||||
c.lastMutex.Lock()
|
||||
c.nodeCallback = callback
|
||||
c.lastMutex.Unlock()
|
||||
c.sendNode()
|
||||
c.nodeUpdater.setCallback(callback)
|
||||
}
|
||||
|
||||
// SetDERPMap updates the DERPMap of a connection.
|
||||
func (c *Conn) SetDERPMap(derpMap *tailcfg.DERPMap) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
c.logger.Debug(context.Background(), "updating derp map", slog.F("derp_map", derpMap))
|
||||
c.wireguardEngine.SetDERPMap(derpMap)
|
||||
c.netMap.DERPMap = derpMap
|
||||
netMapCopy := *c.netMap
|
||||
c.logger.Debug(context.Background(), "updating network map")
|
||||
c.wireguardEngine.SetNetworkMap(&netMapCopy)
|
||||
c.configMaps.setDERPMap(DERPMapToProto(derpMap))
|
||||
}
|
||||
|
||||
func (c *Conn) SetDERPForceWebSockets(v bool) {
|
||||
c.logger.Info(context.Background(), "setting DERP Force Websockets", slog.F("force_derp_websockets", v))
|
||||
c.magicConn.SetDERPForceWebsockets(v)
|
||||
}
|
||||
|
||||
// SetBlockEndpoints sets whether or not to block P2P endpoints. This setting
|
||||
// SetBlockEndpoints sets whether to block P2P endpoints. This setting
|
||||
// will only apply to new peers.
|
||||
func (c *Conn) SetBlockEndpoints(blockEndpoints bool) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
c.blockEndpoints = blockEndpoints
|
||||
c.configMaps.setBlockEndpoints(blockEndpoints)
|
||||
c.nodeUpdater.setBlockEndpoints(blockEndpoints)
|
||||
}
|
||||
|
||||
// SetDERPRegionDialer updates the dialer to use for connecting to DERP regions.
|
||||
@ -483,186 +346,24 @@ func (c *Conn) SetDERPRegionDialer(dialer func(ctx context.Context, region *tail
|
||||
c.magicConn.SetDERPRegionDialer(dialer)
|
||||
}
|
||||
|
||||
// UpdateNodes connects with a set of peers. This can be constantly updated,
|
||||
// and peers will continually be reconnected as necessary. If replacePeers is
|
||||
// true, all peers will be removed before adding the new ones.
|
||||
//
|
||||
//nolint:revive // Complains about replacePeers.
|
||||
func (c *Conn) UpdateNodes(nodes []*Node, replacePeers bool) error {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
// UpdatePeers connects with a set of peers. This can be constantly updated,
|
||||
// and peers will continually be reconnected as necessary.
|
||||
func (c *Conn) UpdatePeers(updates []*proto.CoordinateResponse_PeerUpdate) error {
|
||||
if c.isClosed() {
|
||||
return ErrConnClosed
|
||||
}
|
||||
|
||||
status := c.Status()
|
||||
if replacePeers {
|
||||
c.netMap.Peers = []*tailcfg.Node{}
|
||||
c.peerMap = map[tailcfg.NodeID]*tailcfg.Node{}
|
||||
}
|
||||
for _, peer := range c.netMap.Peers {
|
||||
peerStatus, ok := status.Peer[peer.Key]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
// If this peer was added in the last 5 minutes, assume it
|
||||
// could still be active.
|
||||
if time.Since(peer.Created) < 5*time.Minute {
|
||||
continue
|
||||
}
|
||||
// We double-check that it's safe to remove by ensuring no
|
||||
// handshake has been sent in the past 5 minutes as well. Connections that
|
||||
// are actively exchanging IP traffic will handshake every 2 minutes.
|
||||
if time.Since(peerStatus.LastHandshake) < 5*time.Minute {
|
||||
continue
|
||||
}
|
||||
|
||||
c.logger.Debug(context.Background(), "removing peer, last handshake >5m ago",
|
||||
slog.F("peer", peer.Key), slog.F("last_handshake", peerStatus.LastHandshake),
|
||||
)
|
||||
delete(c.peerMap, peer.ID)
|
||||
}
|
||||
|
||||
for _, node := range nodes {
|
||||
// If no preferred DERP is provided, we can't reach the node.
|
||||
if node.PreferredDERP == 0 {
|
||||
c.logger.Debug(context.Background(), "no preferred DERP, skipping node", slog.F("node", node))
|
||||
continue
|
||||
}
|
||||
c.logger.Debug(context.Background(), "adding node", slog.F("node", node))
|
||||
|
||||
peerStatus, ok := status.Peer[node.Key]
|
||||
peerNode := &tailcfg.Node{
|
||||
ID: node.ID,
|
||||
Created: time.Now(),
|
||||
Key: node.Key,
|
||||
DiscoKey: node.DiscoKey,
|
||||
Addresses: node.Addresses,
|
||||
AllowedIPs: node.AllowedIPs,
|
||||
Endpoints: node.Endpoints,
|
||||
DERP: fmt.Sprintf("%s:%d", tailcfg.DerpMagicIP, node.PreferredDERP),
|
||||
Hostinfo: (&tailcfg.Hostinfo{}).View(),
|
||||
// Starting KeepAlive messages at the initialization of a connection
|
||||
// causes a race condition. If we handshake before the peer has our
|
||||
// node, we'll have wait for 5 seconds before trying again. Ideally,
|
||||
// the first handshake starts when the user first initiates a
|
||||
// connection to the peer. After a successful connection we enable
|
||||
// keep alives to persist the connection and keep it from becoming
|
||||
// idle. SSH connections don't send send packets while idle, so we
|
||||
// use keep alives to avoid random hangs while we set up the
|
||||
// connection again after inactivity.
|
||||
KeepAlive: ok && peerStatus.Active,
|
||||
}
|
||||
if c.blockEndpoints {
|
||||
peerNode.Endpoints = nil
|
||||
}
|
||||
c.peerMap[node.ID] = peerNode
|
||||
}
|
||||
|
||||
c.netMap.Peers = make([]*tailcfg.Node, 0, len(c.peerMap))
|
||||
for _, peer := range c.peerMap {
|
||||
c.netMap.Peers = append(c.netMap.Peers, peer.Clone())
|
||||
}
|
||||
|
||||
netMapCopy := *c.netMap
|
||||
c.logger.Debug(context.Background(), "updating network map")
|
||||
c.wireguardEngine.SetNetworkMap(&netMapCopy)
|
||||
err := c.reconfig()
|
||||
if err != nil {
|
||||
return xerrors.Errorf("reconfig: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// PeerSelector is used to select a peer from within a Tailnet.
|
||||
type PeerSelector struct {
|
||||
ID tailcfg.NodeID
|
||||
IP netip.Prefix
|
||||
}
|
||||
|
||||
func (c *Conn) RemovePeer(selector PeerSelector) (deleted bool, err error) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
if c.isClosed() {
|
||||
return false, ErrConnClosed
|
||||
}
|
||||
|
||||
deleted = false
|
||||
for _, peer := range c.peerMap {
|
||||
if peer.ID == selector.ID {
|
||||
delete(c.peerMap, peer.ID)
|
||||
deleted = true
|
||||
break
|
||||
}
|
||||
|
||||
for _, peerIP := range peer.Addresses {
|
||||
if peerIP.Bits() == selector.IP.Bits() && peerIP.Addr().Compare(selector.IP.Addr()) == 0 {
|
||||
delete(c.peerMap, peer.ID)
|
||||
deleted = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !deleted {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
c.netMap.Peers = make([]*tailcfg.Node, 0, len(c.peerMap))
|
||||
for _, peer := range c.peerMap {
|
||||
c.netMap.Peers = append(c.netMap.Peers, peer.Clone())
|
||||
}
|
||||
|
||||
netMapCopy := *c.netMap
|
||||
c.logger.Debug(context.Background(), "updating network map")
|
||||
c.wireguardEngine.SetNetworkMap(&netMapCopy)
|
||||
err = c.reconfig()
|
||||
if err != nil {
|
||||
return false, xerrors.Errorf("reconfig: %w", err)
|
||||
}
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (c *Conn) reconfig() error {
|
||||
cfg, err := nmcfg.WGCfg(c.netMap, Logger(c.logger.Named("net.wgconfig")), netmap.AllowSingleHosts, "")
|
||||
if err != nil {
|
||||
return xerrors.Errorf("update wireguard config: %w", err)
|
||||
}
|
||||
|
||||
err = c.wireguardEngine.Reconfig(cfg, c.wireguardRouter, &dns.Config{}, &tailcfg.Debug{})
|
||||
if err != nil {
|
||||
if c.isClosed() {
|
||||
return nil
|
||||
}
|
||||
if errors.Is(err, wgengine.ErrNoChanges) {
|
||||
return nil
|
||||
}
|
||||
return xerrors.Errorf("reconfig: %w", err)
|
||||
}
|
||||
|
||||
c.configMaps.updatePeers(updates)
|
||||
return nil
|
||||
}
|
||||
|
||||
// NodeAddresses returns the addresses of a node from the NetworkMap.
|
||||
func (c *Conn) NodeAddresses(publicKey key.NodePublic) ([]netip.Prefix, bool) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
for _, node := range c.netMap.Peers {
|
||||
if node.Key == publicKey {
|
||||
return node.Addresses, true
|
||||
}
|
||||
}
|
||||
return nil, false
|
||||
return c.configMaps.nodeAddresses(publicKey)
|
||||
}
|
||||
|
||||
// Status returns the current ipnstate of a connection.
|
||||
func (c *Conn) Status() *ipnstate.Status {
|
||||
sb := &ipnstate.StatusBuilder{WantPeers: true}
|
||||
c.wireguardEngine.UpdateStatus(sb)
|
||||
return sb.Status()
|
||||
return c.configMaps.status()
|
||||
}
|
||||
|
||||
// Ping sends a ping to the Wireguard engine.
|
||||
@ -689,16 +390,9 @@ func (c *Conn) Ping(ctx context.Context, ip netip.Addr) (time.Duration, bool, *i
|
||||
|
||||
// DERPMap returns the currently set DERP mapping.
|
||||
func (c *Conn) DERPMap() *tailcfg.DERPMap {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
return c.netMap.DERPMap
|
||||
}
|
||||
|
||||
// BlockEndpoints returns whether or not P2P is blocked.
|
||||
func (c *Conn) BlockEndpoints() bool {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
return c.blockEndpoints
|
||||
c.configMaps.L.Lock()
|
||||
defer c.configMaps.L.Unlock()
|
||||
return c.configMaps.derpMapLocked()
|
||||
}
|
||||
|
||||
// AwaitReachable pings the provided IP continually until the
|
||||
@ -759,6 +453,9 @@ func (c *Conn) Closed() <-chan struct{} {
|
||||
|
||||
// Close shuts down the Wireguard connection.
|
||||
func (c *Conn) Close() error {
|
||||
c.logger.Info(context.Background(), "closing tailnet Conn")
|
||||
c.configMaps.close()
|
||||
c.nodeUpdater.close()
|
||||
c.mutex.Lock()
|
||||
select {
|
||||
case <-c.closed:
|
||||
@ -808,91 +505,11 @@ func (c *Conn) isClosed() bool {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Conn) sendNode() {
|
||||
c.lastMutex.Lock()
|
||||
defer c.lastMutex.Unlock()
|
||||
if c.nodeSending {
|
||||
c.nodeChanged = true
|
||||
return
|
||||
}
|
||||
node := c.selfNode()
|
||||
// Conn.UpdateNodes will skip any nodes that don't have the PreferredDERP
|
||||
// set to non-zero, since we cannot reach nodes without DERP for discovery.
|
||||
// Therefore, there is no point in sending the node without this, and we can
|
||||
// save ourselves from churn in the tailscale/wireguard layer.
|
||||
if node.PreferredDERP == 0 {
|
||||
c.logger.Debug(context.Background(), "skipped sending node; no PreferredDERP", slog.F("node", node))
|
||||
return
|
||||
}
|
||||
nodeCallback := c.nodeCallback
|
||||
if nodeCallback == nil {
|
||||
return
|
||||
}
|
||||
c.nodeSending = true
|
||||
go func() {
|
||||
c.logger.Debug(context.Background(), "sending node", slog.F("node", node))
|
||||
nodeCallback(node)
|
||||
c.lastMutex.Lock()
|
||||
c.nodeSending = false
|
||||
if c.nodeChanged {
|
||||
c.nodeChanged = false
|
||||
c.lastMutex.Unlock()
|
||||
c.sendNode()
|
||||
return
|
||||
}
|
||||
c.lastMutex.Unlock()
|
||||
}()
|
||||
}
|
||||
|
||||
// Node returns the last node that was sent to the node callback.
|
||||
func (c *Conn) Node() *Node {
|
||||
c.lastMutex.Lock()
|
||||
defer c.lastMutex.Unlock()
|
||||
return c.selfNode()
|
||||
}
|
||||
|
||||
func (c *Conn) selfNode() *Node {
|
||||
endpoints := make([]string, 0, len(c.lastEndpoints))
|
||||
for _, addr := range c.lastEndpoints {
|
||||
endpoints = append(endpoints, addr.Addr.String())
|
||||
}
|
||||
var preferredDERP int
|
||||
var derpLatency map[string]float64
|
||||
derpForcedWebsocket := make(map[int]string, 0)
|
||||
if c.lastNetInfo != nil {
|
||||
preferredDERP = c.lastNetInfo.PreferredDERP
|
||||
derpLatency = c.lastNetInfo.DERPLatency
|
||||
|
||||
if c.derpForceWebSockets {
|
||||
// We only need to store this for a single region, since this is
|
||||
// mostly used for debugging purposes and doesn't actually have a
|
||||
// code purpose.
|
||||
derpForcedWebsocket[preferredDERP] = "DERP is configured to always fallback to WebSockets"
|
||||
} else {
|
||||
for k, v := range c.lastDERPForcedWebSockets {
|
||||
derpForcedWebsocket[k] = v
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
node := &Node{
|
||||
ID: c.netMap.SelfNode.ID,
|
||||
AsOf: dbtime.Now(),
|
||||
Key: c.netMap.SelfNode.Key,
|
||||
Addresses: c.netMap.SelfNode.Addresses,
|
||||
AllowedIPs: c.netMap.SelfNode.AllowedIPs,
|
||||
DiscoKey: c.magicConn.DiscoPublicKey(),
|
||||
Endpoints: endpoints,
|
||||
PreferredDERP: preferredDERP,
|
||||
DERPLatency: derpLatency,
|
||||
DERPForcedWebsocket: derpForcedWebsocket,
|
||||
}
|
||||
c.mutex.Lock()
|
||||
if c.blockEndpoints {
|
||||
node.Endpoints = nil
|
||||
}
|
||||
c.mutex.Unlock()
|
||||
return node
|
||||
c.nodeUpdater.L.Lock()
|
||||
defer c.nodeUpdater.L.Unlock()
|
||||
return c.nodeUpdater.nodeLocked()
|
||||
}
|
||||
|
||||
// This and below is taken _mostly_ verbatim from Tailscale:
|
||||
@ -1056,15 +673,3 @@ func Logger(logger slog.Logger) tslogger.Logf {
|
||||
logger.Debug(context.Background(), fmt.Sprintf(format, args...))
|
||||
})
|
||||
}
|
||||
|
||||
func endpointsEqual(x, y []tailcfg.Endpoint) bool {
|
||||
if len(x) != len(y) {
|
||||
return false
|
||||
}
|
||||
for i := range x {
|
||||
if x[i] != y[i] {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
Reference in New Issue
Block a user