mirror of
https://github.com/tailscale/tailscale.git
synced 2025-08-27 07:38:49 +00:00
net/netaddr: start migrating to net/netip via new netaddr adapter package
Updates #5162 Change-Id: Id7bdec303b25471f69d542f8ce43805328d56c12 Signed-off-by: Brad Fitzpatrick <bradfitz@tailscale.com>
This commit is contained in:

committed by
Brad Fitzpatrick

parent
7b1a91dfd3
commit
7eaf5e509f
@@ -29,7 +29,6 @@ import (
|
||||
|
||||
"go4.org/mem"
|
||||
"golang.zx2c4.com/wireguard/conn"
|
||||
"inet.af/netaddr"
|
||||
"tailscale.com/control/controlclient"
|
||||
"tailscale.com/derp"
|
||||
"tailscale.com/derp/derphttp"
|
||||
@@ -39,6 +38,7 @@ import (
|
||||
"tailscale.com/logtail/backoff"
|
||||
"tailscale.com/net/dnscache"
|
||||
"tailscale.com/net/interfaces"
|
||||
"tailscale.com/net/netaddr"
|
||||
"tailscale.com/net/netcheck"
|
||||
"tailscale.com/net/neterror"
|
||||
"tailscale.com/net/netns"
|
||||
@@ -55,7 +55,6 @@ import (
|
||||
"tailscale.com/types/nettype"
|
||||
"tailscale.com/util/clientmetric"
|
||||
"tailscale.com/util/mak"
|
||||
"tailscale.com/util/netconv"
|
||||
"tailscale.com/util/uniq"
|
||||
"tailscale.com/version"
|
||||
"tailscale.com/wgengine/monitor"
|
||||
@@ -890,7 +889,7 @@ func (c *Conn) Ping(peer *tailcfg.Node, res *ipnstate.PingResult, cb func(*ipnst
|
||||
return
|
||||
}
|
||||
if len(peer.Addresses) > 0 {
|
||||
res.NodeIP = peer.Addresses[0].IP().String()
|
||||
res.NodeIP = peer.Addresses[0].Addr().String()
|
||||
}
|
||||
res.NodeName = peer.Name // prefer DNS name
|
||||
if res.NodeName == "" {
|
||||
@@ -911,7 +910,7 @@ func (c *Conn) Ping(peer *tailcfg.Node, res *ipnstate.PingResult, cb func(*ipnst
|
||||
// c.mu must be held
|
||||
func (c *Conn) populateCLIPingResponseLocked(res *ipnstate.PingResult, latency time.Duration, ep netaddr.IPPort) {
|
||||
res.LatencySeconds = latency.Seconds()
|
||||
if ep.IP() != derpMagicIPAddr {
|
||||
if ep.Addr() != derpMagicIPAddr {
|
||||
res.Endpoint = ep.String()
|
||||
return
|
||||
}
|
||||
@@ -1049,7 +1048,7 @@ func (c *Conn) determineEndpoints(ctx context.Context) ([]tailcfg.Endpoint, erro
|
||||
return
|
||||
}
|
||||
addAddr := func(ipp netaddr.IPPort, et tailcfg.EndpointType) {
|
||||
if ipp.IsZero() || (debugOmitLocalAddresses && et == tailcfg.EndpointLocal) {
|
||||
if !ipp.IsValid() || (debugOmitLocalAddresses && et == tailcfg.EndpointLocal) {
|
||||
return
|
||||
}
|
||||
if _, ok := already[ipp]; !ok {
|
||||
@@ -1185,23 +1184,16 @@ var errDropDerpPacket = errors.New("too many DERP packets queued; dropping")
|
||||
|
||||
var errNoUDP = errors.New("no UDP available on platform")
|
||||
|
||||
var udpAddrPool = &sync.Pool{
|
||||
New: func() any { return new(net.UDPAddr) },
|
||||
}
|
||||
|
||||
// sendUDP sends UDP packet b to ipp.
|
||||
// See sendAddr's docs on the return value meanings.
|
||||
func (c *Conn) sendUDP(ipp netaddr.IPPort, b []byte) (sent bool, err error) {
|
||||
if runtime.GOOS == "js" {
|
||||
return false, errNoUDP
|
||||
}
|
||||
ua := udpAddrPool.Get().(*net.UDPAddr)
|
||||
sent, err = c.sendUDPStd(ipp.UDPAddrAt(ua), b)
|
||||
sent, err = c.sendUDPStd(ipp, b)
|
||||
if err != nil {
|
||||
metricSendUDPError.Add(1)
|
||||
} else {
|
||||
// Only return it to the pool on success; Issue 3122.
|
||||
udpAddrPool.Put(ua)
|
||||
if sent {
|
||||
metricSendUDP.Add(1)
|
||||
}
|
||||
@@ -1211,19 +1203,19 @@ func (c *Conn) sendUDP(ipp netaddr.IPPort, b []byte) (sent bool, err error) {
|
||||
|
||||
// sendUDP sends UDP packet b to addr.
|
||||
// See sendAddr's docs on the return value meanings.
|
||||
func (c *Conn) sendUDPStd(addr *net.UDPAddr, b []byte) (sent bool, err error) {
|
||||
func (c *Conn) sendUDPStd(addr netip.AddrPort, b []byte) (sent bool, err error) {
|
||||
switch {
|
||||
case addr.IP.To4() != nil:
|
||||
_, err = c.pconn4.WriteTo(b, addr)
|
||||
case addr.Addr().Is4():
|
||||
_, err = c.pconn4.WriteToUDPAddrPort(b, addr)
|
||||
if err != nil && (c.noV4.Get() || neterror.TreatAsLostUDP(err)) {
|
||||
return false, nil
|
||||
}
|
||||
case len(addr.IP) == net.IPv6len:
|
||||
case addr.Addr().Is6():
|
||||
if c.pconn6 == nil {
|
||||
// ignore IPv6 dest if we don't have an IPv6 address.
|
||||
return false, nil
|
||||
}
|
||||
_, err = c.pconn6.WriteTo(b, addr)
|
||||
_, err = c.pconn6.WriteToUDPAddrPort(b, addr)
|
||||
if err != nil && (c.noV6.Get() || neterror.TreatAsLostUDP(err)) {
|
||||
return false, nil
|
||||
}
|
||||
@@ -1244,7 +1236,7 @@ func (c *Conn) sendUDPStd(addr *net.UDPAddr, b []byte) (sent bool, err error) {
|
||||
// IPv6 address when the local machine doesn't have IPv6 support
|
||||
// returns (false, nil); it's not an error, but nothing was sent.
|
||||
func (c *Conn) sendAddr(addr netaddr.IPPort, pubKey key.NodePublic, b []byte) (sent bool, err error) {
|
||||
if addr.IP() != derpMagicIPAddr {
|
||||
if addr.Addr() != derpMagicIPAddr {
|
||||
return c.sendUDP(addr, b)
|
||||
}
|
||||
|
||||
@@ -1290,7 +1282,7 @@ const bufferedDerpWritesBeforeDrop = 32
|
||||
// If peer is non-zero, it can be used to find an active reverse
|
||||
// path, without using addr.
|
||||
func (c *Conn) derpWriteChanOfAddr(addr netaddr.IPPort, peer key.NodePublic) chan<- derpWriteRequest {
|
||||
if addr.IP() != derpMagicIPAddr {
|
||||
if addr.Addr() != derpMagicIPAddr {
|
||||
return nil
|
||||
}
|
||||
regionID := int(addr.Port())
|
||||
@@ -1795,7 +1787,7 @@ func (c *Conn) sendDiscoMessage(dst netaddr.IPPort, dstKey key.NodePublic, dstDi
|
||||
di := c.discoInfoLocked(dstDisco)
|
||||
c.mu.Unlock()
|
||||
|
||||
isDERP := dst.IP() == derpMagicIPAddr
|
||||
isDERP := dst.Addr() == derpMagicIPAddr
|
||||
if isDERP {
|
||||
metricSendDiscoDERP.Add(1)
|
||||
} else {
|
||||
@@ -1846,7 +1838,7 @@ func (c *Conn) sendDiscoMessage(dst netaddr.IPPort, dstKey key.NodePublic, dstDi
|
||||
// * nonce [24]byte
|
||||
// * naclbox of payload (see tailscale.com/disco package for inner payload format)
|
||||
//
|
||||
// For messages received over DERP, the src.IP() will be derpMagicIP (with
|
||||
// For messages received over DERP, the src.Addr() will be derpMagicIP (with
|
||||
// src.Port() being the region ID) and the derpNodeSrc will be the node key
|
||||
// it was received from at the DERP layer. derpNodeSrc is zero when received
|
||||
// over UDP.
|
||||
@@ -1932,7 +1924,7 @@ func (c *Conn) handleDiscoMessage(msg []byte, src netaddr.IPPort, derpNodeSrc ke
|
||||
return
|
||||
}
|
||||
|
||||
isDERP := src.IP() == derpMagicIPAddr
|
||||
isDERP := src.Addr() == derpMagicIPAddr
|
||||
if isDERP {
|
||||
metricRecvDiscoDERP.Add(1)
|
||||
} else {
|
||||
@@ -2024,7 +2016,7 @@ func (c *Conn) handlePingLocked(dm *disco.Ping, src netaddr.IPPort, di *discoInf
|
||||
likelyHeartBeat := src == di.lastPingFrom && time.Since(di.lastPingTime) < 5*time.Second
|
||||
di.lastPingFrom = src
|
||||
di.lastPingTime = time.Now()
|
||||
isDerp := src.IP() == derpMagicIPAddr
|
||||
isDerp := src.Addr() == derpMagicIPAddr
|
||||
|
||||
// If we can figure out with certainty which node key this disco
|
||||
// message is for, eagerly update our IP<>node and disco<>node
|
||||
@@ -2378,7 +2370,7 @@ func (c *Conn) SetNetworkMap(nm *netmap.NetworkMap) {
|
||||
|
||||
for _, a := range n.AllowedIPs {
|
||||
if a.IsSingleIP() {
|
||||
fmt.Fprintf(w, "aip=%v ", a.IP())
|
||||
fmt.Fprintf(w, "aip=%v ", a.Addr())
|
||||
} else {
|
||||
fmt.Fprintf(w, "aip=%v ", a)
|
||||
}
|
||||
@@ -2442,7 +2434,7 @@ func (c *Conn) maybeCloseDERPsOnRebind(okayLocalIPs []netaddr.IPPrefix) {
|
||||
c.closeOrReconectDERPLocked(regionID, "rebind-no-localaddr")
|
||||
continue
|
||||
}
|
||||
if !tsaddr.PrefixesContainsIP(okayLocalIPs, la.IP()) {
|
||||
if !tsaddr.PrefixesContainsIP(okayLocalIPs, la.Addr()) {
|
||||
c.closeOrReconectDERPLocked(regionID, "rebind-default-route-change")
|
||||
continue
|
||||
}
|
||||
@@ -2806,13 +2798,13 @@ func (c *Conn) initialBind() error {
|
||||
|
||||
// listenPacket opens a packet listener.
|
||||
// The network must be "udp4" or "udp6".
|
||||
func (c *Conn) listenPacket(network string, port uint16) (net.PacketConn, error) {
|
||||
func (c *Conn) listenPacket(network string, port uint16) (nettype.PacketConn, error) {
|
||||
ctx := context.Background() // unused without DNS name to resolve
|
||||
addr := net.JoinHostPort("", fmt.Sprint(port))
|
||||
if c.testOnlyPacketListener != nil {
|
||||
return c.testOnlyPacketListener.ListenPacket(ctx, network, addr)
|
||||
return nettype.MakePacketListenerWithNetIP(c.testOnlyPacketListener).ListenPacket(ctx, network, addr)
|
||||
}
|
||||
return netns.Listener(c.logf).ListenPacket(ctx, network, addr)
|
||||
return nettype.MakePacketListenerWithNetIP(netns.Listener(c.logf)).ListenPacket(ctx, network, addr)
|
||||
}
|
||||
|
||||
// bindSocket initializes rucPtr if necessary and binds a UDP socket to it.
|
||||
@@ -2854,7 +2846,7 @@ func (c *Conn) bindSocket(rucPtr **RebindingUDPConn, network string, curPortFate
|
||||
// Remove duplicates. (All duplicates are consecutive.)
|
||||
uniq.ModifySlice(&ports, func(i, j int) bool { return ports[i] == ports[j] })
|
||||
|
||||
var pconn net.PacketConn
|
||||
var pconn nettype.PacketConn
|
||||
for _, port := range ports {
|
||||
// Close the existing conn, in case it is sitting on the port we want.
|
||||
err := ruc.closeLocked()
|
||||
@@ -2943,7 +2935,7 @@ func (c *Conn) resetEndpointStates() {
|
||||
|
||||
// packIPPort packs an IPPort into the form wanted by WireGuard.
|
||||
func packIPPort(ua netaddr.IPPort) []byte {
|
||||
ip := ua.IP().Unmap()
|
||||
ip := ua.Addr().Unmap()
|
||||
a := ip.As16()
|
||||
ipb := a[:]
|
||||
if ip.Is4() {
|
||||
@@ -2983,11 +2975,11 @@ func (c *Conn) ParseEndpoint(nodeKeyStr string) (conn.Endpoint, error) {
|
||||
// Unix has no notion of re-binding a socket, so we swap it out for a new one.
|
||||
type RebindingUDPConn struct {
|
||||
mu sync.Mutex
|
||||
pconn net.PacketConn
|
||||
pconn nettype.PacketConn
|
||||
}
|
||||
|
||||
// currentConn returns c's current pconn.
|
||||
func (c *RebindingUDPConn) currentConn() net.PacketConn {
|
||||
func (c *RebindingUDPConn) currentConn() nettype.PacketConn {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
return c.pconn
|
||||
@@ -3021,9 +3013,7 @@ func (c *RebindingUDPConn) ReadFromNetaddr(b []byte) (n int, ipp netaddr.IPPort,
|
||||
// This lets us avoid allocations by calling ReadFromUDPAddrPort.
|
||||
// The non-*net.UDPConn case works, but it allocates.
|
||||
if udpConn, ok := pconn.(*net.UDPConn); ok {
|
||||
var ap netip.AddrPort
|
||||
n, ap, err = udpConn.ReadFromUDPAddrPort(b)
|
||||
ipp = netconv.AsIPPort(ap)
|
||||
n, ipp, err = udpConn.ReadFromUDPAddrPort(b)
|
||||
} else {
|
||||
var addr net.Addr
|
||||
n, addr, err = pconn.ReadFrom(b)
|
||||
@@ -3094,6 +3084,26 @@ func (c *RebindingUDPConn) WriteTo(b []byte, addr net.Addr) (int, error) {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *RebindingUDPConn) WriteToUDPAddrPort(b []byte, addr netip.AddrPort) (int, error) {
|
||||
for {
|
||||
c.mu.Lock()
|
||||
pconn := c.pconn
|
||||
c.mu.Unlock()
|
||||
|
||||
n, err := pconn.WriteToUDPAddrPort(b, addr)
|
||||
if err != nil {
|
||||
c.mu.Lock()
|
||||
pconn2 := c.pconn
|
||||
c.mu.Unlock()
|
||||
|
||||
if pconn != pconn2 {
|
||||
continue
|
||||
}
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
}
|
||||
|
||||
func newBlockForeverConn() *blockForeverConn {
|
||||
c := new(blockForeverConn)
|
||||
c.cond = sync.NewCond(&c.mu)
|
||||
@@ -3121,6 +3131,11 @@ func (c *blockForeverConn) WriteTo(p []byte, addr net.Addr) (n int, err error) {
|
||||
return len(p), nil
|
||||
}
|
||||
|
||||
func (c *blockForeverConn) WriteToUDPAddrPort(p []byte, addr netip.AddrPort) (int, error) {
|
||||
// Silently drop writes.
|
||||
return len(p), nil
|
||||
}
|
||||
|
||||
func (c *blockForeverConn) LocalAddr() net.Addr {
|
||||
// Return a *net.UDPAddr because lots of code assumes that it will.
|
||||
return new(net.UDPAddr)
|
||||
@@ -3153,11 +3168,11 @@ func simpleDur(d time.Duration) time.Duration {
|
||||
}
|
||||
|
||||
func sbPrintAddr(sb *strings.Builder, a netaddr.IPPort) {
|
||||
is6 := a.IP().Is6()
|
||||
is6 := a.Addr().Is6()
|
||||
if is6 {
|
||||
sb.WriteByte('[')
|
||||
}
|
||||
fmt.Fprintf(sb, "%s", a.IP())
|
||||
fmt.Fprintf(sb, "%s", a.Addr())
|
||||
if is6 {
|
||||
sb.WriteByte(']')
|
||||
}
|
||||
@@ -3197,8 +3212,8 @@ func (c *Conn) UpdateStatus(sb *ipnstate.StatusBuilder) {
|
||||
if !addr.IsSingleIP() {
|
||||
continue
|
||||
}
|
||||
sb.AddTailscaleIP(addr.IP())
|
||||
tailscaleIPs = append(tailscaleIPs, addr.IP())
|
||||
sb.AddTailscaleIP(addr.Addr())
|
||||
tailscaleIPs = append(tailscaleIPs, addr.Addr())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3236,7 +3251,7 @@ func (c *Conn) UpdateStatus(sb *ipnstate.StatusBuilder) {
|
||||
}
|
||||
|
||||
func ippDebugString(ua netaddr.IPPort) string {
|
||||
if ua.IP() == derpMagicIPAddr {
|
||||
if ua.Addr() == derpMagicIPAddr {
|
||||
return fmt.Sprintf("derp-%d", ua.Port())
|
||||
}
|
||||
return ua.String()
|
||||
@@ -3454,7 +3469,7 @@ func (de *endpoint) canP2P() bool {
|
||||
// de.mu must be held.
|
||||
func (de *endpoint) addrForSendLocked(now mono.Time) (udpAddr, derpAddr netaddr.IPPort) {
|
||||
udpAddr = de.bestAddr.IPPort
|
||||
if udpAddr.IsZero() || now.After(de.trustBestAddrUntil) {
|
||||
if !udpAddr.IsValid() || now.After(de.trustBestAddrUntil) {
|
||||
// We had a bestAddr but it expired so send both to it
|
||||
// and DERP.
|
||||
derpAddr = de.derpAddr
|
||||
@@ -3488,7 +3503,7 @@ func (de *endpoint) heartbeat() {
|
||||
|
||||
now := mono.Now()
|
||||
udpAddr, _ := de.addrForSendLocked(now)
|
||||
if !udpAddr.IsZero() {
|
||||
if udpAddr.IsValid() {
|
||||
// We have a preferred path. Ping that every 2 seconds.
|
||||
de.startPingLocked(udpAddr, now, pingHeartbeat)
|
||||
}
|
||||
@@ -3511,7 +3526,7 @@ func (de *endpoint) wantFullPingLocked(now mono.Time) bool {
|
||||
if !de.canP2P() {
|
||||
return false
|
||||
}
|
||||
if de.bestAddr.IsZero() || de.lastFullPing.IsZero() {
|
||||
if !de.bestAddr.IsValid() || de.lastFullPing.IsZero() {
|
||||
return true
|
||||
}
|
||||
if now.After(de.trustBestAddrUntil) {
|
||||
@@ -3543,10 +3558,10 @@ func (de *endpoint) cliPing(res *ipnstate.PingResult, cb func(*ipnstate.PingResu
|
||||
|
||||
now := mono.Now()
|
||||
udpAddr, derpAddr := de.addrForSendLocked(now)
|
||||
if !derpAddr.IsZero() {
|
||||
if derpAddr.IsValid() {
|
||||
de.startPingLocked(derpAddr, now, pingCLI)
|
||||
}
|
||||
if !udpAddr.IsZero() && now.Before(de.trustBestAddrUntil) {
|
||||
if udpAddr.IsValid() && now.Before(de.trustBestAddrUntil) {
|
||||
// Already have an active session, so just ping the address we're using.
|
||||
// Otherwise "tailscale ping" results to a node on the local network
|
||||
// can look like they're bouncing between, say 10.0.0.0/9 and the peer's
|
||||
@@ -3565,20 +3580,20 @@ func (de *endpoint) send(b []byte) error {
|
||||
|
||||
de.mu.Lock()
|
||||
udpAddr, derpAddr := de.addrForSendLocked(now)
|
||||
if de.canP2P() && (udpAddr.IsZero() || now.After(de.trustBestAddrUntil)) {
|
||||
if de.canP2P() && (!udpAddr.IsValid() || now.After(de.trustBestAddrUntil)) {
|
||||
de.sendPingsLocked(now, true)
|
||||
}
|
||||
de.noteActiveLocked()
|
||||
de.mu.Unlock()
|
||||
|
||||
if udpAddr.IsZero() && derpAddr.IsZero() {
|
||||
if !udpAddr.IsValid() && !derpAddr.IsValid() {
|
||||
return errors.New("no UDP or DERP addr")
|
||||
}
|
||||
var err error
|
||||
if !udpAddr.IsZero() {
|
||||
if udpAddr.IsValid() {
|
||||
_, err = de.c.sendAddr(udpAddr, de.publicKey, b)
|
||||
}
|
||||
if !derpAddr.IsZero() {
|
||||
if derpAddr.IsValid() {
|
||||
if ok, _ := de.c.sendAddr(derpAddr, de.publicKey, b); ok && err != nil {
|
||||
// UDP failed but DERP worked, so good enough:
|
||||
return nil
|
||||
@@ -3594,7 +3609,7 @@ func (de *endpoint) pingTimeout(txid stun.TxID) {
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
if debugDisco || de.bestAddr.IsZero() || mono.Now().After(de.trustBestAddrUntil) {
|
||||
if debugDisco || !de.bestAddr.IsValid() || mono.Now().After(de.trustBestAddrUntil) {
|
||||
de.c.logf("[v1] magicsock: disco: timeout waiting for pong %x from %v (%v, %v)", txid[:6], sp.to, de.publicKey.ShortString(), de.discoShort)
|
||||
}
|
||||
de.removeSentPingLocked(txid, sp)
|
||||
@@ -3710,7 +3725,7 @@ func (de *endpoint) sendPingsLocked(now mono.Time, sendCallMeMaybe bool) {
|
||||
de.startPingLocked(ep, now, pingDiscovery)
|
||||
}
|
||||
derpAddr := de.derpAddr
|
||||
if sentAny && sendCallMeMaybe && !derpAddr.IsZero() {
|
||||
if sentAny && sendCallMeMaybe && derpAddr.IsValid() {
|
||||
// Have our magicsock.Conn figure out its STUN endpoint (if
|
||||
// it doesn't know already) and then send a CallMeMaybe
|
||||
// message to our peer via DERP informing them that we've
|
||||
@@ -3822,7 +3837,7 @@ func (de *endpoint) handlePongConnLocked(m *disco.Pong, di *discoInfo, src netad
|
||||
de.mu.Lock()
|
||||
defer de.mu.Unlock()
|
||||
|
||||
isDerp := src.IP() == derpMagicIPAddr
|
||||
isDerp := src.Addr() == derpMagicIPAddr
|
||||
|
||||
sp, ok := de.sentPing[m.TxID]
|
||||
if !ok {
|
||||
@@ -3895,19 +3910,19 @@ func betterAddr(a, b addrLatency) bool {
|
||||
if a.IPPort == b.IPPort {
|
||||
return false
|
||||
}
|
||||
if b.IsZero() {
|
||||
if !b.IsValid() {
|
||||
return true
|
||||
}
|
||||
if a.IsZero() {
|
||||
if !a.IsValid() {
|
||||
return false
|
||||
}
|
||||
if a.IP().Is6() && b.IP().Is4() {
|
||||
if a.Addr().Is6() && b.Addr().Is4() {
|
||||
// Prefer IPv6 for being a bit more robust, as long as
|
||||
// the latencies are roughly equivalent.
|
||||
if a.latency/10*9 < b.latency {
|
||||
return true
|
||||
}
|
||||
} else if a.IP().Is4() && b.IP().Is6() {
|
||||
} else if a.Addr().Is4() && b.Addr().Is6() {
|
||||
if betterAddr(b, a) {
|
||||
return false
|
||||
}
|
||||
@@ -3952,7 +3967,7 @@ func (de *endpoint) handleCallMeMaybe(m *disco.CallMeMaybe) {
|
||||
}
|
||||
var newEPs []netaddr.IPPort
|
||||
for _, ep := range m.MyNumber {
|
||||
if ep.IP().Is6() && ep.IP().IsLinkLocalUnicast() {
|
||||
if ep.Addr().Is6() && ep.Addr().IsLinkLocalUnicast() {
|
||||
// We send these out, but ignore them for now.
|
||||
// TODO: teach the ping code to ping on all interfaces
|
||||
// for these.
|
||||
@@ -4010,7 +4025,7 @@ func (de *endpoint) populatePeerStatus(ps *ipnstate.PeerStatus) {
|
||||
ps.LastWrite = de.lastSend.WallTime()
|
||||
ps.Active = now.Sub(de.lastSend) < sessionActiveTimeout
|
||||
|
||||
if udpAddr, derpAddr := de.addrForSendLocked(now); !udpAddr.IsZero() && derpAddr.IsZero() {
|
||||
if udpAddr, derpAddr := de.addrForSendLocked(now); udpAddr.IsValid() && !derpAddr.IsValid() {
|
||||
ps.CurAddr = udpAddr.String()
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user