mirror of
https://github.com/tailscale/tailscale.git
synced 2025-08-14 23:17:29 +00:00
wgengine/magicsock,all: allocate peer relay over disco instead of PeerAPI (#16603)
Updates tailscale/corp#30583 Updates tailscale/corp#30534 Updates tailscale/corp#30557 Signed-off-by: Dylan Bargatze <dylan@tailscale.com> Signed-off-by: Jordan Whited <jordan@tailscale.com> Co-authored-by: Dylan Bargatze <dylan@tailscale.com>
This commit is contained in:
@@ -175,13 +175,15 @@ type Conn struct {
|
||||
|
||||
// These [eventbus.Subscriber] fields are solely accessed by
|
||||
// consumeEventbusTopics once initialized.
|
||||
pmSub *eventbus.Subscriber[portmapper.Mapping]
|
||||
filterSub *eventbus.Subscriber[FilterUpdate]
|
||||
nodeViewsSub *eventbus.Subscriber[NodeViewsUpdate]
|
||||
nodeMutsSub *eventbus.Subscriber[NodeMutationsUpdate]
|
||||
syncSub *eventbus.Subscriber[syncPoint]
|
||||
syncPub *eventbus.Publisher[syncPoint]
|
||||
subsDoneCh chan struct{} // closed when consumeEventbusTopics returns
|
||||
pmSub *eventbus.Subscriber[portmapper.Mapping]
|
||||
filterSub *eventbus.Subscriber[FilterUpdate]
|
||||
nodeViewsSub *eventbus.Subscriber[NodeViewsUpdate]
|
||||
nodeMutsSub *eventbus.Subscriber[NodeMutationsUpdate]
|
||||
syncSub *eventbus.Subscriber[syncPoint]
|
||||
syncPub *eventbus.Publisher[syncPoint]
|
||||
allocRelayEndpointPub *eventbus.Publisher[UDPRelayAllocReq]
|
||||
allocRelayEndpointSub *eventbus.Subscriber[UDPRelayAllocResp]
|
||||
subsDoneCh chan struct{} // closed when consumeEventbusTopics returns
|
||||
|
||||
// pconn4 and pconn6 are the underlying UDP sockets used to
|
||||
// send/receive packets for wireguard and other magicsock
|
||||
@@ -271,6 +273,14 @@ type Conn struct {
|
||||
// captureHook, if non-nil, is the pcap logging callback when capturing.
|
||||
captureHook syncs.AtomicValue[packet.CaptureCallback]
|
||||
|
||||
// hasPeerRelayServers is whether [relayManager] is configured with at least
|
||||
// one peer relay server via [relayManager.handleRelayServersSet]. It is
|
||||
// only accessed by [Conn.updateRelayServersSet], [endpoint.setDERPHome],
|
||||
// and [endpoint.discoverUDPRelayPathsLocked]. It exists to suppress
|
||||
// calls into [relayManager] leading to wasted work involving channel
|
||||
// operations and goroutine creation.
|
||||
hasPeerRelayServers atomic.Bool
|
||||
|
||||
// discoPrivate is the private naclbox key used for active
|
||||
// discovery traffic. It is always present, and immutable.
|
||||
discoPrivate key.DiscoPrivate
|
||||
@@ -567,6 +577,36 @@ func (s syncPoint) Signal() {
|
||||
close(s)
|
||||
}
|
||||
|
||||
// UDPRelayAllocReq represents a [*disco.AllocateUDPRelayEndpointRequest]
|
||||
// reception event. This is signaled over an [eventbus.Bus] from
|
||||
// [magicsock.Conn] towards [relayserver.extension].
|
||||
type UDPRelayAllocReq struct {
|
||||
// RxFromNodeKey is the unauthenticated (DERP server claimed src) node key
|
||||
// of the transmitting party, noted at disco message reception time over
|
||||
// DERP. This node key is unambiguously-aligned with RxFromDiscoKey being
|
||||
// that the disco message is received over DERP.
|
||||
RxFromNodeKey key.NodePublic
|
||||
// RxFromDiscoKey is the disco key of the transmitting party, noted and
|
||||
// authenticated at reception time.
|
||||
RxFromDiscoKey key.DiscoPublic
|
||||
// Message is the disco message.
|
||||
Message *disco.AllocateUDPRelayEndpointRequest
|
||||
}
|
||||
|
||||
// UDPRelayAllocResp represents a [*disco.AllocateUDPRelayEndpointResponse]
|
||||
// that is yet to be transmitted over DERP (or delivered locally if
|
||||
// ReqRxFromNodeKey is self). This is signaled over an [eventbus.Bus] from
|
||||
// [relayserver.extension] towards [magicsock.Conn].
|
||||
type UDPRelayAllocResp struct {
|
||||
// ReqRxFromNodeKey is copied from [UDPRelayAllocReq.RxFromNodeKey]. It
|
||||
// enables peer lookup leading up to transmission over DERP.
|
||||
ReqRxFromNodeKey key.NodePublic
|
||||
// ReqRxFromDiscoKey is copied from [UDPRelayAllocReq.RxFromDiscoKey].
|
||||
ReqRxFromDiscoKey key.DiscoPublic
|
||||
// Message is the disco message.
|
||||
Message *disco.AllocateUDPRelayEndpointResponse
|
||||
}
|
||||
|
||||
// newConn is the error-free, network-listening-side-effect-free based
|
||||
// of NewConn. Mostly for tests.
|
||||
func newConn(logf logger.Logf) *Conn {
|
||||
@@ -625,10 +665,40 @@ func (c *Conn) consumeEventbusTopics() {
|
||||
case syncPoint := <-c.syncSub.Events():
|
||||
c.dlogf("magicsock: received sync point after reconfig")
|
||||
syncPoint.Signal()
|
||||
case allocResp := <-c.allocRelayEndpointSub.Events():
|
||||
c.onUDPRelayAllocResp(allocResp)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Conn) onUDPRelayAllocResp(allocResp UDPRelayAllocResp) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
ep, ok := c.peerMap.endpointForNodeKey(allocResp.ReqRxFromNodeKey)
|
||||
if !ok {
|
||||
// If it's not a peer, it might be for self (we can peer relay through
|
||||
// ourselves), in which case we want to hand it down to [relayManager]
|
||||
// now versus taking a network round-trip through DERP.
|
||||
selfNodeKey := c.publicKeyAtomic.Load()
|
||||
if selfNodeKey.Compare(allocResp.ReqRxFromNodeKey) == 0 &&
|
||||
allocResp.ReqRxFromDiscoKey.Compare(c.discoPublic) == 0 {
|
||||
c.relayManager.handleRxDiscoMsg(c, allocResp.Message, selfNodeKey, allocResp.ReqRxFromDiscoKey, epAddr{})
|
||||
}
|
||||
return
|
||||
}
|
||||
disco := ep.disco.Load()
|
||||
if disco == nil {
|
||||
return
|
||||
}
|
||||
if disco.key.Compare(allocResp.ReqRxFromDiscoKey) != 0 {
|
||||
return
|
||||
}
|
||||
ep.mu.Lock()
|
||||
defer ep.mu.Unlock()
|
||||
derpAddr := ep.derpAddr
|
||||
go c.sendDiscoMessage(epAddr{ap: derpAddr}, ep.publicKey, disco.key, allocResp.Message, discoVerboseLog)
|
||||
}
|
||||
|
||||
// Synchronize waits for all [eventbus] events published
|
||||
// prior to this call to be processed by the receiver.
|
||||
func (c *Conn) Synchronize() {
|
||||
@@ -670,6 +740,8 @@ func NewConn(opts Options) (*Conn, error) {
|
||||
c.nodeMutsSub = eventbus.Subscribe[NodeMutationsUpdate](c.eventClient)
|
||||
c.syncSub = eventbus.Subscribe[syncPoint](c.eventClient)
|
||||
c.syncPub = eventbus.Publish[syncPoint](c.eventClient)
|
||||
c.allocRelayEndpointPub = eventbus.Publish[UDPRelayAllocReq](c.eventClient)
|
||||
c.allocRelayEndpointSub = eventbus.Subscribe[UDPRelayAllocResp](c.eventClient)
|
||||
c.subsDoneCh = make(chan struct{})
|
||||
go c.consumeEventbusTopics()
|
||||
}
|
||||
@@ -1847,6 +1919,24 @@ func (v *virtualNetworkID) get() uint32 {
|
||||
return v._vni & vniGetMask
|
||||
}
|
||||
|
||||
// sendDiscoAllocateUDPRelayEndpointRequest is primarily an alias for
|
||||
// sendDiscoMessage, but it will alternatively send m over the eventbus if dst
|
||||
// is a DERP IP:port, and dstKey is self. This saves a round-trip through DERP
|
||||
// when we are attempting to allocate on a self (in-process) peer relay server.
|
||||
func (c *Conn) sendDiscoAllocateUDPRelayEndpointRequest(dst epAddr, dstKey key.NodePublic, dstDisco key.DiscoPublic, allocReq *disco.AllocateUDPRelayEndpointRequest, logLevel discoLogLevel) (sent bool, err error) {
|
||||
isDERP := dst.ap.Addr() == tailcfg.DerpMagicIPAddr
|
||||
selfNodeKey := c.publicKeyAtomic.Load()
|
||||
if isDERP && dstKey.Compare(selfNodeKey) == 0 {
|
||||
c.allocRelayEndpointPub.Publish(UDPRelayAllocReq{
|
||||
RxFromNodeKey: selfNodeKey,
|
||||
RxFromDiscoKey: c.discoPublic,
|
||||
Message: allocReq,
|
||||
})
|
||||
return true, nil
|
||||
}
|
||||
return c.sendDiscoMessage(dst, dstKey, dstDisco, allocReq, logLevel)
|
||||
}
|
||||
|
||||
// sendDiscoMessage sends discovery message m to dstDisco at dst.
|
||||
//
|
||||
// If dst.ap is a DERP IP:port, then dstKey must be non-zero.
|
||||
@@ -2176,7 +2266,7 @@ func (c *Conn) handleDiscoMessage(msg []byte, src epAddr, shouldBeRelayHandshake
|
||||
c.logf("[unexpected] %T packets should not come from a relay server with Geneve control bit set", dm)
|
||||
return
|
||||
}
|
||||
c.relayManager.handleGeneveEncapDiscoMsg(c, challenge, di, src)
|
||||
c.relayManager.handleRxDiscoMsg(c, challenge, key.NodePublic{}, di.discoKey, src)
|
||||
return
|
||||
}
|
||||
|
||||
@@ -2201,7 +2291,7 @@ func (c *Conn) handleDiscoMessage(msg []byte, src epAddr, shouldBeRelayHandshake
|
||||
// If it's an unknown TxID, and it's Geneve-encapsulated, then
|
||||
// make [relayManager] aware. It might be in the middle of probing
|
||||
// src.
|
||||
c.relayManager.handleGeneveEncapDiscoMsg(c, dm, di, src)
|
||||
c.relayManager.handleRxDiscoMsg(c, dm, key.NodePublic{}, di.discoKey, src)
|
||||
}
|
||||
case *disco.CallMeMaybe, *disco.CallMeMaybeVia:
|
||||
var via *disco.CallMeMaybeVia
|
||||
@@ -2276,7 +2366,95 @@ func (c *Conn) handleDiscoMessage(msg []byte, src epAddr, shouldBeRelayHandshake
|
||||
len(cmm.MyNumber))
|
||||
go ep.handleCallMeMaybe(cmm)
|
||||
}
|
||||
case *disco.AllocateUDPRelayEndpointRequest, *disco.AllocateUDPRelayEndpointResponse:
|
||||
var resp *disco.AllocateUDPRelayEndpointResponse
|
||||
isResp := false
|
||||
msgType := "AllocateUDPRelayEndpointRequest"
|
||||
req, ok := dm.(*disco.AllocateUDPRelayEndpointRequest)
|
||||
if ok {
|
||||
metricRecvDiscoAllocUDPRelayEndpointRequest.Add(1)
|
||||
} else {
|
||||
metricRecvDiscoAllocUDPRelayEndpointResponse.Add(1)
|
||||
resp = dm.(*disco.AllocateUDPRelayEndpointResponse)
|
||||
msgType = "AllocateUDPRelayEndpointResponse"
|
||||
isResp = true
|
||||
}
|
||||
|
||||
if !isDERP {
|
||||
// These messages should only come via DERP.
|
||||
c.logf("[unexpected] %s packets should only come via DERP", msgType)
|
||||
return
|
||||
}
|
||||
nodeKey := derpNodeSrc
|
||||
ep, ok := c.peerMap.endpointForNodeKey(nodeKey)
|
||||
if !ok {
|
||||
c.logf("magicsock: disco: ignoring %s from %v; %v is unknown", msgType, sender.ShortString(), derpNodeSrc.ShortString())
|
||||
return
|
||||
}
|
||||
epDisco := ep.disco.Load()
|
||||
if epDisco == nil {
|
||||
return
|
||||
}
|
||||
if epDisco.key != di.discoKey {
|
||||
if isResp {
|
||||
metricRecvDiscoAllocUDPRelayEndpointResponseBadDisco.Add(1)
|
||||
} else {
|
||||
metricRecvDiscoAllocUDPRelayEndpointRequestBadDisco.Add(1)
|
||||
}
|
||||
c.logf("[unexpected] %s from peer via DERP whose netmap discokey != disco source", msgType)
|
||||
return
|
||||
}
|
||||
|
||||
if isResp {
|
||||
c.dlogf("[v1] magicsock: disco: %v<-%v (%v, %v) got %s, %d endpoints",
|
||||
c.discoShort, epDisco.short,
|
||||
ep.publicKey.ShortString(), derpStr(src.String()),
|
||||
msgType,
|
||||
len(resp.AddrPorts))
|
||||
c.relayManager.handleRxDiscoMsg(c, resp, nodeKey, di.discoKey, src)
|
||||
return
|
||||
} else if sender.Compare(req.ClientDisco[0]) != 0 && sender.Compare(req.ClientDisco[1]) != 0 {
|
||||
// An allocation request must contain the sender's disco key in
|
||||
// ClientDisco. One of the relay participants must be the sender.
|
||||
c.logf("magicsock: disco: %s from %v; %v does not contain sender's disco key",
|
||||
msgType, sender.ShortString(), derpNodeSrc.ShortString())
|
||||
return
|
||||
} else {
|
||||
c.dlogf("[v1] magicsock: disco: %v<-%v (%v, %v) got %s, for %d<->%d",
|
||||
c.discoShort, epDisco.short,
|
||||
ep.publicKey.ShortString(), derpStr(src.String()),
|
||||
msgType,
|
||||
req.ClientDisco[0], req.ClientDisco[1])
|
||||
}
|
||||
|
||||
if c.filt == nil {
|
||||
return
|
||||
}
|
||||
// Binary search of peers is O(log n) while c.mu is held.
|
||||
// TODO: We might be able to use ep.nodeAddr instead of all addresses,
|
||||
// or we might be able to release c.mu before doing this work. Keep it
|
||||
// simple and slow for now. c.peers.AsSlice is a copy. We may need to
|
||||
// write our own binary search for a [views.Slice].
|
||||
peerI, ok := slices.BinarySearchFunc(c.peers.AsSlice(), ep.nodeID, func(peer tailcfg.NodeView, target tailcfg.NodeID) int {
|
||||
if peer.ID() < target {
|
||||
return -1
|
||||
} else if peer.ID() > target {
|
||||
return 1
|
||||
}
|
||||
return 0
|
||||
})
|
||||
if !ok {
|
||||
// unexpected
|
||||
return
|
||||
}
|
||||
if !nodeHasCap(c.filt, c.peers.At(peerI), c.self, tailcfg.PeerCapabilityRelay) {
|
||||
return
|
||||
}
|
||||
c.allocRelayEndpointPub.Publish(UDPRelayAllocReq{
|
||||
RxFromDiscoKey: sender,
|
||||
RxFromNodeKey: nodeKey,
|
||||
Message: req,
|
||||
})
|
||||
}
|
||||
return
|
||||
}
|
||||
@@ -2337,7 +2515,7 @@ func (c *Conn) handlePingLocked(dm *disco.Ping, src epAddr, di *discoInfo, derpN
|
||||
// Geneve-encapsulated [disco.Ping] messages in the interest of
|
||||
// simplicity. It might be in the middle of probing src, so it must be
|
||||
// made aware.
|
||||
c.relayManager.handleGeneveEncapDiscoMsg(c, dm, di, src)
|
||||
c.relayManager.handleRxDiscoMsg(c, dm, key.NodePublic{}, di.discoKey, src)
|
||||
return
|
||||
}
|
||||
|
||||
@@ -2687,7 +2865,7 @@ func (c *Conn) SetProbeUDPLifetime(v bool) {
|
||||
// capVerIsRelayCapable returns true if version is relay client and server
|
||||
// capable, otherwise it returns false.
|
||||
func capVerIsRelayCapable(version tailcfg.CapabilityVersion) bool {
|
||||
return version >= 120
|
||||
return version >= 121
|
||||
}
|
||||
|
||||
// onFilterUpdate is called when a [FilterUpdate] is received over the
|
||||
@@ -2715,6 +2893,11 @@ func (c *Conn) onFilterUpdate(f FilterUpdate) {
|
||||
// peers are passed as args (vs c.mu-guarded fields) to enable callers to
|
||||
// release c.mu before calling as this is O(m * n) (we iterate all cap rules 'm'
|
||||
// in filt for every peer 'n').
|
||||
//
|
||||
// Calls to updateRelayServersSet must never run concurrent to
|
||||
// [endpoint.setDERPHome], otherwise [candidatePeerRelay] DERP home changes may
|
||||
// be missed from the perspective of [relayManager].
|
||||
//
|
||||
// TODO: Optimize this so that it's not O(m * n). This might involve:
|
||||
// 1. Changes to [filter.Filter], e.g. adding a CapsWithValues() to check for
|
||||
// a given capability instead of building and returning a map of all of
|
||||
@@ -2722,69 +2905,75 @@ func (c *Conn) onFilterUpdate(f FilterUpdate) {
|
||||
// 2. Moving this work upstream into [nodeBackend] or similar, and publishing
|
||||
// the computed result over the eventbus instead.
|
||||
func (c *Conn) updateRelayServersSet(filt *filter.Filter, self tailcfg.NodeView, peers views.Slice[tailcfg.NodeView]) {
|
||||
relayServers := make(set.Set[netip.AddrPort])
|
||||
relayServers := make(set.Set[candidatePeerRelay])
|
||||
nodes := append(peers.AsSlice(), self)
|
||||
for _, maybeCandidate := range nodes {
|
||||
peerAPI := peerAPIIfCandidateRelayServer(filt, self, maybeCandidate)
|
||||
if peerAPI.IsValid() {
|
||||
relayServers.Add(peerAPI)
|
||||
}
|
||||
}
|
||||
c.relayManager.handleRelayServersSet(relayServers)
|
||||
}
|
||||
|
||||
// peerAPIIfCandidateRelayServer returns the peer API address of maybeCandidate
|
||||
// if it is considered to be a candidate relay server upon evaluation against
|
||||
// filt and self, otherwise it returns a zero value. self and maybeCandidate
|
||||
// may be equal.
|
||||
func peerAPIIfCandidateRelayServer(filt *filter.Filter, self, maybeCandidate tailcfg.NodeView) netip.AddrPort {
|
||||
if filt == nil ||
|
||||
!self.Valid() ||
|
||||
!maybeCandidate.Valid() ||
|
||||
!maybeCandidate.Hostinfo().Valid() {
|
||||
return netip.AddrPort{}
|
||||
}
|
||||
if maybeCandidate.ID() != self.ID() && !capVerIsRelayCapable(maybeCandidate.Cap()) {
|
||||
// If maybeCandidate's [tailcfg.CapabilityVersion] is not relay-capable,
|
||||
// we skip it. If maybeCandidate happens to be self, then this check is
|
||||
// unnecessary as self is always capable from this point (the statically
|
||||
// compiled [tailcfg.CurrentCapabilityVersion]) forward.
|
||||
return netip.AddrPort{}
|
||||
}
|
||||
for _, maybeCandidatePrefix := range maybeCandidate.Addresses().All() {
|
||||
if !maybeCandidatePrefix.IsSingleIP() {
|
||||
if maybeCandidate.ID() != self.ID() && !capVerIsRelayCapable(maybeCandidate.Cap()) {
|
||||
// If maybeCandidate's [tailcfg.CapabilityVersion] is not relay-capable,
|
||||
// we skip it. If maybeCandidate happens to be self, then this check is
|
||||
// unnecessary as self is always capable from this point (the statically
|
||||
// compiled [tailcfg.CurrentCapabilityVersion]) forward.
|
||||
continue
|
||||
}
|
||||
maybeCandidateAddr := maybeCandidatePrefix.Addr()
|
||||
for _, selfPrefix := range self.Addresses().All() {
|
||||
if !selfPrefix.IsSingleIP() {
|
||||
if !nodeHasCap(filt, maybeCandidate, self, tailcfg.PeerCapabilityRelayTarget) {
|
||||
continue
|
||||
}
|
||||
relayServers.Add(candidatePeerRelay{
|
||||
nodeKey: maybeCandidate.Key(),
|
||||
discoKey: maybeCandidate.DiscoKey(),
|
||||
derpHomeRegionID: uint16(maybeCandidate.HomeDERP()),
|
||||
})
|
||||
}
|
||||
c.relayManager.handleRelayServersSet(relayServers)
|
||||
if len(relayServers) > 0 {
|
||||
c.hasPeerRelayServers.Store(true)
|
||||
} else {
|
||||
c.hasPeerRelayServers.Store(false)
|
||||
}
|
||||
}
|
||||
|
||||
// nodeHasCap returns true if src has cap on dst, otherwise it returns false.
|
||||
func nodeHasCap(filt *filter.Filter, src, dst tailcfg.NodeView, cap tailcfg.PeerCapability) bool {
|
||||
if filt == nil ||
|
||||
!src.Valid() ||
|
||||
!dst.Valid() {
|
||||
return false
|
||||
}
|
||||
for _, srcPrefix := range src.Addresses().All() {
|
||||
if !srcPrefix.IsSingleIP() {
|
||||
continue
|
||||
}
|
||||
srcAddr := srcPrefix.Addr()
|
||||
for _, dstPrefix := range dst.Addresses().All() {
|
||||
if !dstPrefix.IsSingleIP() {
|
||||
continue
|
||||
}
|
||||
selfAddr := selfPrefix.Addr()
|
||||
if selfAddr.BitLen() == maybeCandidateAddr.BitLen() { // same address family
|
||||
if filt.CapsWithValues(maybeCandidateAddr, selfAddr).HasCapability(tailcfg.PeerCapabilityRelayTarget) {
|
||||
for _, s := range maybeCandidate.Hostinfo().Services().All() {
|
||||
if maybeCandidateAddr.Is4() && s.Proto == tailcfg.PeerAPI4 ||
|
||||
maybeCandidateAddr.Is6() && s.Proto == tailcfg.PeerAPI6 {
|
||||
return netip.AddrPortFrom(maybeCandidateAddr, s.Port)
|
||||
}
|
||||
}
|
||||
return netip.AddrPort{} // no peerAPI
|
||||
} else {
|
||||
// [nodeBackend.peerCapsLocked] only returns/considers the
|
||||
// [tailcfg.PeerCapMap] between the passed src and the
|
||||
// _first_ host (/32 or /128) address for self. We are
|
||||
// consistent with that behavior here. If self and
|
||||
// maybeCandidate host addresses are of the same address
|
||||
// family they either have the capability or not. We do not
|
||||
// check against additional host addresses of the same
|
||||
// address family.
|
||||
return netip.AddrPort{}
|
||||
}
|
||||
dstAddr := dstPrefix.Addr()
|
||||
if dstAddr.BitLen() == srcAddr.BitLen() { // same address family
|
||||
// [nodeBackend.peerCapsLocked] only returns/considers the
|
||||
// [tailcfg.PeerCapMap] between the passed src and the _first_
|
||||
// host (/32 or /128) address for self. We are consistent with
|
||||
// that behavior here. If src and dst host addresses are of the
|
||||
// same address family they either have the capability or not.
|
||||
// We do not check against additional host addresses of the same
|
||||
// address family.
|
||||
return filt.CapsWithValues(srcAddr, dstAddr).HasCapability(cap)
|
||||
}
|
||||
}
|
||||
}
|
||||
return netip.AddrPort{}
|
||||
return false
|
||||
}
|
||||
|
||||
// candidatePeerRelay represents the identifiers and DERP home region ID for a
|
||||
// peer relay server.
|
||||
type candidatePeerRelay struct {
|
||||
nodeKey key.NodePublic
|
||||
discoKey key.DiscoPublic
|
||||
derpHomeRegionID uint16
|
||||
}
|
||||
|
||||
func (c *candidatePeerRelay) isValid() bool {
|
||||
return !c.nodeKey.IsZero() && !c.discoKey.IsZero()
|
||||
}
|
||||
|
||||
// onNodeViewsUpdate is called when a [NodeViewsUpdate] is received over the
|
||||
@@ -3792,18 +3981,22 @@ var (
|
||||
metricRecvDiscoBadKey = clientmetric.NewCounter("magicsock_disco_recv_bad_key")
|
||||
metricRecvDiscoBadParse = clientmetric.NewCounter("magicsock_disco_recv_bad_parse")
|
||||
|
||||
metricRecvDiscoUDP = clientmetric.NewCounter("magicsock_disco_recv_udp")
|
||||
metricRecvDiscoDERP = clientmetric.NewCounter("magicsock_disco_recv_derp")
|
||||
metricRecvDiscoPing = clientmetric.NewCounter("magicsock_disco_recv_ping")
|
||||
metricRecvDiscoPong = clientmetric.NewCounter("magicsock_disco_recv_pong")
|
||||
metricRecvDiscoCallMeMaybe = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe")
|
||||
metricRecvDiscoCallMeMaybeVia = clientmetric.NewCounter("magicsock_disco_recv_callmemaybevia")
|
||||
metricRecvDiscoCallMeMaybeBadNode = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_node")
|
||||
metricRecvDiscoCallMeMaybeViaBadNode = clientmetric.NewCounter("magicsock_disco_recv_callmemaybevia_bad_node")
|
||||
metricRecvDiscoCallMeMaybeBadDisco = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_disco")
|
||||
metricRecvDiscoCallMeMaybeViaBadDisco = clientmetric.NewCounter("magicsock_disco_recv_callmemaybevia_bad_disco")
|
||||
metricRecvDiscoDERPPeerNotHere = clientmetric.NewCounter("magicsock_disco_recv_derp_peer_not_here")
|
||||
metricRecvDiscoDERPPeerGoneUnknown = clientmetric.NewCounter("magicsock_disco_recv_derp_peer_gone_unknown")
|
||||
metricRecvDiscoUDP = clientmetric.NewCounter("magicsock_disco_recv_udp")
|
||||
metricRecvDiscoDERP = clientmetric.NewCounter("magicsock_disco_recv_derp")
|
||||
metricRecvDiscoPing = clientmetric.NewCounter("magicsock_disco_recv_ping")
|
||||
metricRecvDiscoPong = clientmetric.NewCounter("magicsock_disco_recv_pong")
|
||||
metricRecvDiscoCallMeMaybe = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe")
|
||||
metricRecvDiscoCallMeMaybeVia = clientmetric.NewCounter("magicsock_disco_recv_callmemaybevia")
|
||||
metricRecvDiscoCallMeMaybeBadNode = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_node")
|
||||
metricRecvDiscoCallMeMaybeViaBadNode = clientmetric.NewCounter("magicsock_disco_recv_callmemaybevia_bad_node")
|
||||
metricRecvDiscoCallMeMaybeBadDisco = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_disco")
|
||||
metricRecvDiscoCallMeMaybeViaBadDisco = clientmetric.NewCounter("magicsock_disco_recv_callmemaybevia_bad_disco")
|
||||
metricRecvDiscoAllocUDPRelayEndpointRequest = clientmetric.NewCounter("magicsock_disco_recv_alloc_udp_relay_endpoint_request")
|
||||
metricRecvDiscoAllocUDPRelayEndpointRequestBadDisco = clientmetric.NewCounter("magicsock_disco_recv_alloc_udp_relay_endpoint_request_bad_disco")
|
||||
metricRecvDiscoAllocUDPRelayEndpointResponseBadDisco = clientmetric.NewCounter("magicsock_disco_recv_alloc_udp_relay_endpoint_response_bad_disco")
|
||||
metricRecvDiscoAllocUDPRelayEndpointResponse = clientmetric.NewCounter("magicsock_disco_recv_alloc_udp_relay_endpoint_response")
|
||||
metricRecvDiscoDERPPeerNotHere = clientmetric.NewCounter("magicsock_disco_recv_derp_peer_not_here")
|
||||
metricRecvDiscoDERPPeerGoneUnknown = clientmetric.NewCounter("magicsock_disco_recv_derp_peer_gone_unknown")
|
||||
// metricDERPHomeChange is how many times our DERP home region DI has
|
||||
// changed from non-zero to a different non-zero.
|
||||
metricDERPHomeChange = clientmetric.NewCounter("derp_home_change")
|
||||
@@ -3985,6 +4178,22 @@ func (le *lazyEndpoint) FromPeer(peerPublicKey [32]byte) {
|
||||
}
|
||||
|
||||
// PeerRelays returns the current set of candidate peer relays.
|
||||
func (c *Conn) PeerRelays() set.Set[netip.AddrPort] {
|
||||
return c.relayManager.getServers()
|
||||
func (c *Conn) PeerRelays() set.Set[netip.Addr] {
|
||||
candidatePeerRelays := c.relayManager.getServers()
|
||||
servers := make(set.Set[netip.Addr], len(candidatePeerRelays))
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
for relay := range candidatePeerRelays {
|
||||
pi, ok := c.peerMap.byNodeKey[relay.nodeKey]
|
||||
if !ok {
|
||||
if c.self.Key().Compare(relay.nodeKey) == 0 {
|
||||
if c.self.Addresses().Len() > 0 {
|
||||
servers.Add(c.self.Addresses().At(0).Addr())
|
||||
}
|
||||
}
|
||||
continue
|
||||
}
|
||||
servers.Add(pi.ep.nodeAddr)
|
||||
}
|
||||
return servers
|
||||
}
|
||||
|
Reference in New Issue
Block a user