derp, derphttp, magicsock: send new unknown peer frame when destination is unknown (#7552)

* wgengine/magicsock: add envknob to send CallMeMaybe to non-existent peer

For testing older client version responses to the PeerGone packet format change.

Updates #4326

Signed-off-by: Val <valerie@tailscale.com>

* derp: remove dead sclient struct member replaceLimiter

Leftover from an previous solution to the duplicate client problem.

Updates #2751

Signed-off-by: Val <valerie@tailscale.com>

* derp, derp/derphttp, wgengine/magicsock: add new PeerGone message type Not Here

Extend the PeerGone message type by adding a reason byte. Send a
PeerGone "Not Here" message when an endpoint sends a disco message to
a peer that this server has no record of.

Fixes #4326

Signed-off-by: Val <valerie@tailscale.com>

---------

Signed-off-by: Val <valerie@tailscale.com>
pull/7691/head
valscale 2 years ago committed by GitHub
parent 09d0b632d4
commit 74eb99aed1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

@ -60,7 +60,7 @@ tailscale.com/cmd/derper dependencies: (generated by github.com/tailscale/depawa
tailscale.com/tka from tailscale.com/client/tailscale+ tailscale.com/tka from tailscale.com/client/tailscale+
W tailscale.com/tsconst from tailscale.com/net/interfaces W tailscale.com/tsconst from tailscale.com/net/interfaces
💣 tailscale.com/tstime/mono from tailscale.com/tstime/rate 💣 tailscale.com/tstime/mono from tailscale.com/tstime/rate
tailscale.com/tstime/rate from tailscale.com/wgengine/filter tailscale.com/tstime/rate from tailscale.com/wgengine/filter+
tailscale.com/tsweb from tailscale.com/cmd/derper tailscale.com/tsweb from tailscale.com/cmd/derper
tailscale.com/types/dnstype from tailscale.com/tailcfg tailscale.com/types/dnstype from tailscale.com/tailcfg
tailscale.com/types/empty from tailscale.com/ipn tailscale.com/types/empty from tailscale.com/ipn

@ -92,7 +92,7 @@ tailscale.com/cmd/tailscale dependencies: (generated by github.com/tailscale/dep
tailscale.com/tka from tailscale.com/client/tailscale+ tailscale.com/tka from tailscale.com/client/tailscale+
W tailscale.com/tsconst from tailscale.com/net/interfaces W tailscale.com/tsconst from tailscale.com/net/interfaces
💣 tailscale.com/tstime/mono from tailscale.com/tstime/rate 💣 tailscale.com/tstime/mono from tailscale.com/tstime/rate
tailscale.com/tstime/rate from tailscale.com/wgengine/filter tailscale.com/tstime/rate from tailscale.com/wgengine/filter+
tailscale.com/types/dnstype from tailscale.com/tailcfg tailscale.com/types/dnstype from tailscale.com/tailcfg
tailscale.com/types/empty from tailscale.com/ipn tailscale.com/types/empty from tailscale.com/ipn
tailscale.com/types/ipproto from tailscale.com/net/flowtrack+ tailscale.com/types/ipproto from tailscale.com/net/flowtrack+

@ -267,7 +267,7 @@ tailscale.com/cmd/tailscaled dependencies: (generated by github.com/tailscale/de
W tailscale.com/tsconst from tailscale.com/net/interfaces W tailscale.com/tsconst from tailscale.com/net/interfaces
tailscale.com/tstime from tailscale.com/wgengine/magicsock tailscale.com/tstime from tailscale.com/wgengine/magicsock
💣 tailscale.com/tstime/mono from tailscale.com/net/tstun+ 💣 tailscale.com/tstime/mono from tailscale.com/net/tstun+
tailscale.com/tstime/rate from tailscale.com/wgengine/filter tailscale.com/tstime/rate from tailscale.com/wgengine/filter+
tailscale.com/tsweb from tailscale.com/cmd/tailscaled tailscale.com/tsweb from tailscale.com/cmd/tailscaled
tailscale.com/types/dnstype from tailscale.com/ipn/ipnlocal+ tailscale.com/types/dnstype from tailscale.com/ipn/ipnlocal+
tailscale.com/types/empty from tailscale.com/control/controlclient+ tailscale.com/types/empty from tailscale.com/control/controlclient+

@ -77,8 +77,11 @@ const (
// a previous sender is no longer connected. That is, if A // a previous sender is no longer connected. That is, if A
// sent to B, and then if A disconnects, the server sends // sent to B, and then if A disconnects, the server sends
// framePeerGone to B so B can forget that a reverse path // framePeerGone to B so B can forget that a reverse path
// exists on that connection to get back to A. // exists on that connection to get back to A. It is also sent
framePeerGone = frameType(0x08) // 32B pub key of peer that's gone // if A tries to send a CallMeMaybe to B and the server has no
// record of B (which currently would only happen if there was
// a bug).
framePeerGone = frameType(0x08) // 32B pub key of peer that's gone + 1 byte reason
// framePeerPresent is like framePeerGone, but for other // framePeerPresent is like framePeerGone, but for other
// members of the DERP region when they're meshed up together. // members of the DERP region when they're meshed up together.
@ -116,6 +119,15 @@ const (
frameRestarting = frameType(0x15) frameRestarting = frameType(0x15)
) )
// PeerGoneReasonType is a one byte reason code explaining why a
// server does not have a path to the requested destination.
type PeerGoneReasonType byte
const (
PeerGoneReasonDisconnected = PeerGoneReasonType(0x00) // peer disconnected from this server
PeerGoneReasonNotHere = PeerGoneReasonType(0x01) // server doesn't know about this peer, unexpected
)
var bin = binary.BigEndian var bin = binary.BigEndian
func writeUint32(bw *bufio.Writer, v uint32) error { func writeUint32(bw *bufio.Writer, v uint32) error {

@ -348,9 +348,12 @@ type ReceivedPacket struct {
func (ReceivedPacket) msg() {} func (ReceivedPacket) msg() {}
// PeerGoneMessage is a ReceivedMessage that indicates that the client // PeerGoneMessage is a ReceivedMessage that indicates that the client
// identified by the underlying public key had previously sent you a // identified by the underlying public key is not connected to this
// packet but has now disconnected from the server. // server.
type PeerGoneMessage key.NodePublic type PeerGoneMessage struct {
Peer key.NodePublic
Reason PeerGoneReasonType
}
func (PeerGoneMessage) msg() {} func (PeerGoneMessage) msg() {}
@ -524,7 +527,15 @@ func (c *Client) recvTimeout(timeout time.Duration) (m ReceivedMessage, err erro
c.logf("[unexpected] dropping short peerGone frame from DERP server") c.logf("[unexpected] dropping short peerGone frame from DERP server")
continue continue
} }
pg := PeerGoneMessage(key.NodePublicFromRaw32(mem.B(b[:keyLen]))) // Backward compatibility for the older peerGone without reason byte
reason := PeerGoneReasonDisconnected
if n > keyLen {
reason = PeerGoneReasonType(b[keyLen])
}
pg := PeerGoneMessage{
Peer: key.NodePublicFromRaw32(mem.B(b[:keyLen])),
Reason: reason,
}
return pg, nil return pg, nil
case framePeerPresent: case framePeerPresent:

@ -34,12 +34,12 @@ import (
"go4.org/mem" "go4.org/mem"
"golang.org/x/sync/errgroup" "golang.org/x/sync/errgroup"
"golang.org/x/time/rate"
"tailscale.com/client/tailscale" "tailscale.com/client/tailscale"
"tailscale.com/disco" "tailscale.com/disco"
"tailscale.com/envknob" "tailscale.com/envknob"
"tailscale.com/metrics" "tailscale.com/metrics"
"tailscale.com/syncs" "tailscale.com/syncs"
"tailscale.com/tstime/rate"
"tailscale.com/types/key" "tailscale.com/types/key"
"tailscale.com/types/logger" "tailscale.com/types/logger"
"tailscale.com/version" "tailscale.com/version"
@ -122,7 +122,8 @@ type Server struct {
_ align64 _ align64
packetsForwardedOut expvar.Int packetsForwardedOut expvar.Int
packetsForwardedIn expvar.Int packetsForwardedIn expvar.Int
peerGoneFrames expvar.Int // number of peer gone frames sent peerGoneDisconnectedFrames expvar.Int // number of peer disconnected frames sent
peerGoneNotHereFrames expvar.Int // number of peer not here frames sent
gotPing expvar.Int // number of ping frames from client gotPing expvar.Int // number of ping frames from client
sentPong expvar.Int // number of pong frames enqueued to client sentPong expvar.Int // number of pong frames enqueued to client
accepts expvar.Int accepts expvar.Int
@ -324,7 +325,8 @@ func NewServer(privateKey key.NodePrivate, logf logger.Logf) *Server {
s.packetsDroppedReasonCounters = []*expvar.Int{ s.packetsDroppedReasonCounters = []*expvar.Int{
s.packetsDroppedReason.Get("unknown_dest"), s.packetsDroppedReason.Get("unknown_dest"),
s.packetsDroppedReason.Get("unknown_dest_on_fwd"), s.packetsDroppedReason.Get("unknown_dest_on_fwd"),
s.packetsDroppedReason.Get("gone"), s.packetsDroppedReason.Get("gone_disconnected"),
s.packetsDroppedReason.Get("gone_not_here"),
s.packetsDroppedReason.Get("queue_head"), s.packetsDroppedReason.Get("queue_head"),
s.packetsDroppedReason.Get("queue_tail"), s.packetsDroppedReason.Get("queue_tail"),
s.packetsDroppedReason.Get("write_error"), s.packetsDroppedReason.Get("write_error"),
@ -615,13 +617,26 @@ func (s *Server) notePeerGoneFromRegionLocked(key key.NodePublic) {
} }
set.ForeachClient(func(peer *sclient) { set.ForeachClient(func(peer *sclient) {
if peer.connNum == connNum { if peer.connNum == connNum {
go peer.requestPeerGoneWrite(key) go peer.requestPeerGoneWrite(key, PeerGoneReasonDisconnected)
} }
}) })
} }
delete(s.sentTo, key) delete(s.sentTo, key)
} }
// requestPeerGoneWriteLimited sends a request to write a "peer gone"
// frame, but only in reply to a disco packet, and only if we haven't
// sent one recently.
func (c *sclient) requestPeerGoneWriteLimited(peer key.NodePublic, contents []byte, reason PeerGoneReasonType) {
if disco.LooksLikeDiscoWrapper(contents) != true {
return
}
if c.peerGoneLim.Allow() {
go c.requestPeerGoneWrite(peer, reason)
}
}
func (s *Server) addWatcher(c *sclient) { func (s *Server) addWatcher(c *sclient) {
if !c.canMesh { if !c.canMesh {
panic("invariant: addWatcher called without permissions") panic("invariant: addWatcher called without permissions")
@ -686,8 +701,9 @@ func (s *Server) accept(ctx context.Context, nc Conn, brw *bufio.ReadWriter, rem
sendQueue: make(chan pkt, perClientSendQueueDepth), sendQueue: make(chan pkt, perClientSendQueueDepth),
discoSendQueue: make(chan pkt, perClientSendQueueDepth), discoSendQueue: make(chan pkt, perClientSendQueueDepth),
sendPongCh: make(chan [8]byte, 1), sendPongCh: make(chan [8]byte, 1),
peerGone: make(chan key.NodePublic), peerGone: make(chan peerGoneMsg),
canMesh: clientInfo.MeshKey != "" && clientInfo.MeshKey == s.meshKey, canMesh: clientInfo.MeshKey != "" && clientInfo.MeshKey == s.meshKey,
peerGoneLim: rate.NewLimiter(rate.Every(time.Second), 3),
} }
if c.canMesh { if c.canMesh {
@ -887,6 +903,8 @@ func (c *sclient) handleFrameForwardPacket(ft frameType, fl uint32) error {
reason := dropReasonUnknownDestOnFwd reason := dropReasonUnknownDestOnFwd
if dstLen > 1 { if dstLen > 1 {
reason = dropReasonDupClient reason = dropReasonDupClient
} else {
c.requestPeerGoneWriteLimited(dstKey, contents, PeerGoneReasonNotHere)
} }
s.recordDrop(contents, srcKey, dstKey, reason) s.recordDrop(contents, srcKey, dstKey, reason)
return nil return nil
@ -952,6 +970,8 @@ func (c *sclient) handleFrameSendPacket(ft frameType, fl uint32) error {
reason := dropReasonUnknownDest reason := dropReasonUnknownDest
if dstLen > 1 { if dstLen > 1 {
reason = dropReasonDupClient reason = dropReasonDupClient
} else {
c.requestPeerGoneWriteLimited(dstKey, contents, PeerGoneReasonNotHere)
} }
s.recordDrop(contents, c.key, dstKey, reason) s.recordDrop(contents, c.key, dstKey, reason)
c.debug("SendPacket for %s, dropping with reason=%s", dstKey.ShortString(), reason) c.debug("SendPacket for %s, dropping with reason=%s", dstKey.ShortString(), reason)
@ -981,7 +1001,7 @@ type dropReason int
const ( const (
dropReasonUnknownDest dropReason = iota // unknown destination pubkey dropReasonUnknownDest dropReason = iota // unknown destination pubkey
dropReasonUnknownDestOnFwd // unknown destination pubkey on a derp-forwarded packet dropReasonUnknownDestOnFwd // unknown destination pubkey on a derp-forwarded packet
dropReasonGone // destination tailscaled disconnected before we could send dropReasonGoneDisconnected // destination tailscaled disconnected before we could send
dropReasonQueueHead // destination queue is full, dropped packet at queue head dropReasonQueueHead // destination queue is full, dropped packet at queue head
dropReasonQueueTail // destination queue is full, dropped packet at queue tail dropReasonQueueTail // destination queue is full, dropped packet at queue tail
dropReasonWriteError // OS write() failed dropReasonWriteError // OS write() failed
@ -1023,7 +1043,7 @@ func (c *sclient) sendPkt(dst *sclient, p pkt) error {
for attempt := 0; attempt < 3; attempt++ { for attempt := 0; attempt < 3; attempt++ {
select { select {
case <-dst.done: case <-dst.done:
s.recordDrop(p.bs, c.key, dstKey, dropReasonGone) s.recordDrop(p.bs, c.key, dstKey, dropReasonGoneDisconnected)
dst.debug("sendPkt attempt %d dropped, dst gone", attempt) dst.debug("sendPkt attempt %d dropped, dst gone", attempt)
return nil return nil
default: default:
@ -1052,11 +1072,14 @@ func (c *sclient) sendPkt(dst *sclient, p pkt) error {
} }
// requestPeerGoneWrite sends a request to write a "peer gone" frame // requestPeerGoneWrite sends a request to write a "peer gone" frame
// that the provided peer has disconnected. It blocks until either the // with an explanation of why it is gone. It blocks until either the
// write request is scheduled, or the client has closed. // write request is scheduled, or the client has closed.
func (c *sclient) requestPeerGoneWrite(peer key.NodePublic) { func (c *sclient) requestPeerGoneWrite(peer key.NodePublic, reason PeerGoneReasonType) {
select { select {
case c.peerGone <- peer: case c.peerGone <- peerGoneMsg{
peer: peer,
reason: reason,
}:
case <-c.done: case <-c.done:
} }
} }
@ -1270,25 +1293,20 @@ type sclient struct {
key key.NodePublic key key.NodePublic
info clientInfo info clientInfo
logf logger.Logf logf logger.Logf
done <-chan struct{} // closed when connection closes done <-chan struct{} // closed when connection closes
remoteAddr string // usually ip:port from net.Conn.RemoteAddr().String() remoteAddr string // usually ip:port from net.Conn.RemoteAddr().String()
remoteIPPort netip.AddrPort // zero if remoteAddr is not ip:port. remoteIPPort netip.AddrPort // zero if remoteAddr is not ip:port.
sendQueue chan pkt // packets queued to this client; never closed sendQueue chan pkt // packets queued to this client; never closed
discoSendQueue chan pkt // important packets queued to this client; never closed discoSendQueue chan pkt // important packets queued to this client; never closed
sendPongCh chan [8]byte // pong replies to send to the client; never closed sendPongCh chan [8]byte // pong replies to send to the client; never closed
peerGone chan key.NodePublic // write request that a previous sender has disconnected (not used by mesh peers) peerGone chan peerGoneMsg // write request that a peer is not at this server (not used by mesh peers)
meshUpdate chan struct{} // write request to write peerStateChange meshUpdate chan struct{} // write request to write peerStateChange
canMesh bool // clientInfo had correct mesh token for inter-region routing canMesh bool // clientInfo had correct mesh token for inter-region routing
isDup atomic.Bool // whether more than 1 sclient for key is connected isDup atomic.Bool // whether more than 1 sclient for key is connected
isDisabled atomic.Bool // whether sends to this peer are disabled due to active/active dups isDisabled atomic.Bool // whether sends to this peer are disabled due to active/active dups
debugLogging bool debugLogging bool
// replaceLimiter controls how quickly two connections with
// the same client key can kick each other off the server by
// taking over ownership of a key.
replaceLimiter *rate.Limiter
// Owned by run, not thread-safe. // Owned by run, not thread-safe.
br *bufio.Reader br *bufio.Reader
connectedAt time.Time connectedAt time.Time
@ -1304,6 +1322,11 @@ type sclient struct {
// the client for them to update their map of who's connected // the client for them to update their map of who's connected
// to this node. // to this node.
peerStateChange []peerConnState peerStateChange []peerConnState
// peerGoneLimiter limits how often the server will inform a
// client that it's trying to establish a direct connection
// through us with a peer we have no record of.
peerGoneLim *rate.Limiter
} }
// peerConnState represents whether a peer is connected to the server // peerConnState represents whether a peer is connected to the server
@ -1327,6 +1350,12 @@ type pkt struct {
bs []byte bs []byte
} }
// peerGoneMsg is a request to write a peerGone frame to an sclient
type peerGoneMsg struct {
peer key.NodePublic
reason PeerGoneReasonType
}
func (c *sclient) setPreferred(v bool) { func (c *sclient) setPreferred(v bool) {
if c.preferred == v { if c.preferred == v {
return return
@ -1381,9 +1410,9 @@ func (c *sclient) sendLoop(ctx context.Context) error {
for { for {
select { select {
case pkt := <-c.sendQueue: case pkt := <-c.sendQueue:
c.s.recordDrop(pkt.bs, pkt.src, c.key, dropReasonGone) c.s.recordDrop(pkt.bs, pkt.src, c.key, dropReasonGoneDisconnected)
case pkt := <-c.discoSendQueue: case pkt := <-c.discoSendQueue:
c.s.recordDrop(pkt.bs, pkt.src, c.key, dropReasonGone) c.s.recordDrop(pkt.bs, pkt.src, c.key, dropReasonGoneDisconnected)
default: default:
return return
} }
@ -1404,8 +1433,8 @@ func (c *sclient) sendLoop(ctx context.Context) error {
select { select {
case <-ctx.Done(): case <-ctx.Done():
return nil return nil
case peer := <-c.peerGone: case msg := <-c.peerGone:
werr = c.sendPeerGone(peer) werr = c.sendPeerGone(msg.peer, msg.reason)
continue continue
case <-c.meshUpdate: case <-c.meshUpdate:
werr = c.sendMeshUpdates() werr = c.sendMeshUpdates()
@ -1436,8 +1465,8 @@ func (c *sclient) sendLoop(ctx context.Context) error {
select { select {
case <-ctx.Done(): case <-ctx.Done():
return nil return nil
case peer := <-c.peerGone: case msg := <-c.peerGone:
werr = c.sendPeerGone(peer) werr = c.sendPeerGone(msg.peer, msg.reason)
case <-c.meshUpdate: case <-c.meshUpdate:
werr = c.sendMeshUpdates() werr = c.sendMeshUpdates()
continue continue
@ -1478,13 +1507,22 @@ func (c *sclient) sendPong(data [8]byte) error {
} }
// sendPeerGone sends a peerGone frame, without flushing. // sendPeerGone sends a peerGone frame, without flushing.
func (c *sclient) sendPeerGone(peer key.NodePublic) error { func (c *sclient) sendPeerGone(peer key.NodePublic, reason PeerGoneReasonType) error {
c.s.peerGoneFrames.Add(1) switch reason {
case PeerGoneReasonDisconnected:
c.s.peerGoneDisconnectedFrames.Add(1)
case PeerGoneReasonNotHere:
c.s.peerGoneNotHereFrames.Add(1)
}
c.setWriteDeadline() c.setWriteDeadline()
if err := writeFrameHeader(c.bw.bw(), framePeerGone, keyLen); err != nil { data := make([]byte, 0, keyLen+1)
data = peer.AppendTo(data)
data = append(data, byte(reason))
if err := writeFrameHeader(c.bw.bw(), framePeerGone, uint32(len(data))); err != nil {
return err return err
} }
_, err := c.bw.Write(peer.AppendTo(nil))
_, err := c.bw.Write(data)
return err return err
} }
@ -1515,7 +1553,7 @@ func (c *sclient) sendMeshUpdates() error {
if pcs.present { if pcs.present {
err = c.sendPeerPresent(pcs.peer) err = c.sendPeerPresent(pcs.peer)
} else { } else {
err = c.sendPeerGone(pcs.peer) err = c.sendPeerGone(pcs.peer, PeerGoneReasonDisconnected)
} }
if err != nil { if err != nil {
// Shouldn't happen, though, as we're writing // Shouldn't happen, though, as we're writing
@ -1756,7 +1794,8 @@ func (s *Server) ExpVar() expvar.Var {
m.Set("home_moves_out", &s.homeMovesOut) m.Set("home_moves_out", &s.homeMovesOut)
m.Set("got_ping", &s.gotPing) m.Set("got_ping", &s.gotPing)
m.Set("sent_pong", &s.sentPong) m.Set("sent_pong", &s.sentPong)
m.Set("peer_gone_frames", &s.peerGoneFrames) m.Set("peer_gone_disconnected_frames", &s.peerGoneDisconnectedFrames)
m.Set("peer_gone_not_here_frames", &s.peerGoneNotHereFrames)
m.Set("packets_forwarded_out", &s.packetsForwardedOut) m.Set("packets_forwarded_out", &s.packetsForwardedOut)
m.Set("packets_forwarded_in", &s.packetsForwardedIn) m.Set("packets_forwarded_in", &s.packetsForwardedIn)
m.Set("multiforwarder_created", &s.multiForwarderCreated) m.Set("multiforwarder_created", &s.multiForwarderCreated)

@ -25,6 +25,7 @@ import (
"go4.org/mem" "go4.org/mem"
"golang.org/x/time/rate" "golang.org/x/time/rate"
"tailscale.com/disco"
"tailscale.com/net/memnet" "tailscale.com/net/memnet"
"tailscale.com/types/key" "tailscale.com/types/key"
"tailscale.com/types/logger" "tailscale.com/types/logger"
@ -105,7 +106,8 @@ func TestSendRecv(t *testing.T) {
t.Logf("Connected client %d.", i) t.Logf("Connected client %d.", i)
} }
var peerGoneCount expvar.Int var peerGoneCountDisconnected expvar.Int
var peerGoneCountNotHere expvar.Int
t.Logf("Starting read loops") t.Logf("Starting read loops")
for i := 0; i < numClients; i++ { for i := 0; i < numClients; i++ {
@ -121,7 +123,14 @@ func TestSendRecv(t *testing.T) {
t.Errorf("unexpected message type %T", m) t.Errorf("unexpected message type %T", m)
continue continue
case PeerGoneMessage: case PeerGoneMessage:
peerGoneCount.Add(1) switch m.Reason {
case PeerGoneReasonDisconnected:
peerGoneCountDisconnected.Add(1)
case PeerGoneReasonNotHere:
peerGoneCountNotHere.Add(1)
default:
t.Errorf("unexpected PeerGone reason %v", m.Reason)
}
case ReceivedPacket: case ReceivedPacket:
if m.Source.IsZero() { if m.Source.IsZero() {
t.Errorf("zero Source address in ReceivedPacket") t.Errorf("zero Source address in ReceivedPacket")
@ -171,7 +180,19 @@ func TestSendRecv(t *testing.T) {
var got int64 var got int64
dl := time.Now().Add(5 * time.Second) dl := time.Now().Add(5 * time.Second)
for time.Now().Before(dl) { for time.Now().Before(dl) {
if got = peerGoneCount.Value(); got == want { if got = peerGoneCountDisconnected.Value(); got == want {
return
}
}
t.Errorf("peer gone count = %v; want %v", got, want)
}
wantUnknownPeers := func(want int64) {
t.Helper()
var got int64
dl := time.Now().Add(5 * time.Second)
for time.Now().Before(dl) {
if got = peerGoneCountNotHere.Value(); got == want {
return return
} }
} }
@ -194,6 +215,30 @@ func TestSendRecv(t *testing.T) {
recvNothing(0) recvNothing(0)
recvNothing(1) recvNothing(1)
// Send messages to a non-existent node
neKey := key.NewNode().Public()
msg4 := []byte("not a CallMeMaybe->unknown destination\n")
if err := clients[1].Send(neKey, msg4); err != nil {
t.Fatal(err)
}
wantUnknownPeers(0)
callMe := neKey.AppendTo([]byte(disco.Magic))
callMeHeader := make([]byte, disco.NonceLen)
callMe = append(callMe, callMeHeader...)
if err := clients[1].Send(neKey, callMe); err != nil {
t.Fatal(err)
}
wantUnknownPeers(1)
// PeerGoneNotHere is rate-limited to 3 times a second
for i := 0; i < 5; i++ {
if err := clients[1].Send(neKey, callMe); err != nil {
t.Fatal(err)
}
}
wantUnknownPeers(3)
wantActive(3, 0) wantActive(3, 0)
clients[0].NotePreferred(true) clients[0].NotePreferred(true)
wantActive(3, 1) wantActive(3, 1)
@ -595,10 +640,14 @@ func (tc *testClient) wantGone(t *testing.T, peer key.NodePublic) {
} }
switch m := m.(type) { switch m := m.(type) {
case PeerGoneMessage: case PeerGoneMessage:
got := key.NodePublic(m) got := key.NodePublic(m.Peer)
if peer != got { if peer != got {
t.Errorf("got gone message for %v; want gone for %v", tc.ts.keyName(got), tc.ts.keyName(peer)) t.Errorf("got gone message for %v; want gone for %v", tc.ts.keyName(got), tc.ts.keyName(peer))
} }
reason := m.Reason
if reason != PeerGoneReasonDisconnected {
t.Errorf("got gone message for reason %v; wanted %v", reason, PeerGoneReasonDisconnected)
}
default: default:
t.Fatalf("unexpected message type %T", m) t.Fatalf("unexpected message type %T", m)
} }

@ -128,7 +128,17 @@ func (c *Client) RunWatchConnectionLoop(ctx context.Context, ignoreServerKey key
case derp.PeerPresentMessage: case derp.PeerPresentMessage:
updatePeer(key.NodePublic(m), true) updatePeer(key.NodePublic(m), true)
case derp.PeerGoneMessage: case derp.PeerGoneMessage:
updatePeer(key.NodePublic(m), false) switch m.Reason {
case derp.PeerGoneReasonDisconnected:
// Normal case, log nothing
case derp.PeerGoneReasonNotHere:
logf("Recv: peer %s not connected to %s",
key.NodePublic(m.Peer).ShortString(), c.ServerPublicKey().ShortString())
default:
logf("Recv: peer %s not at server %s for unknown reason %v",
key.NodePublic(m.Peer).ShortString(), c.ServerPublicKey().ShortString(), m.Reason)
}
updatePeer(key.NodePublic(m.Peer), false)
default: default:
continue continue
} }

@ -13,16 +13,16 @@ func _() {
var x [1]struct{} var x [1]struct{}
_ = x[dropReasonUnknownDest-0] _ = x[dropReasonUnknownDest-0]
_ = x[dropReasonUnknownDestOnFwd-1] _ = x[dropReasonUnknownDestOnFwd-1]
_ = x[dropReasonGone-2] _ = x[dropReasonGoneDisconnected-2]
_ = x[dropReasonQueueHead-3] _ = x[dropReasonQueueHead-3]
_ = x[dropReasonQueueTail-4] _ = x[dropReasonQueueTail-4]
_ = x[dropReasonWriteError-5] _ = x[dropReasonWriteError-5]
_ = x[dropReasonDupClient-6] _ = x[dropReasonDupClient-6]
} }
const _dropReason_name = "UnknownDestUnknownDestOnFwdGoneQueueHeadQueueTailWriteErrorDupClient" const _dropReason_name = "UnknownDestUnknownDestOnFwdGoneDisconnectedQueueHeadQueueTailWriteErrorDupClient"
var _dropReason_index = [...]uint8{0, 11, 27, 31, 40, 49, 59, 68} var _dropReason_index = [...]uint8{0, 11, 27, 43, 52, 61, 71, 80}
func (i dropReason) String() string { func (i dropReason) String() string {
if i < 0 || i >= dropReason(len(_dropReason_index)-1) { if i < 0 || i >= dropReason(len(_dropReason_index)-1) {

@ -36,6 +36,10 @@ var (
// debugEnableSilentDisco disables the use of heartbeatTimer on the endpoint struct // debugEnableSilentDisco disables the use of heartbeatTimer on the endpoint struct
// and attempts to handle disco silently. See issue #540 for details. // and attempts to handle disco silently. See issue #540 for details.
debugEnableSilentDisco = envknob.RegisterBool("TS_DEBUG_ENABLE_SILENT_DISCO") debugEnableSilentDisco = envknob.RegisterBool("TS_DEBUG_ENABLE_SILENT_DISCO")
// DebugSendCallMeUnknownPeer sends a CallMeMaybe to a
// non-existent destination every time we send a real
// CallMeMaybe to test the PeerGoneNotHere logic.
debugSendCallMeUnknownPeer = envknob.RegisterBool("TS_DEBUG_SEND_CALLME_UNKNOWN_PEER")
) )
// inTest reports whether the running program is a test that set the // inTest reports whether the running program is a test that set the

@ -11,13 +11,13 @@ import "tailscale.com/types/opt"
// //
// They're inlinable and the linker can deadcode that's guarded by them to make // They're inlinable and the linker can deadcode that's guarded by them to make
// smaller binaries. // smaller binaries.
func debugDisco() bool { return false } func debugDisco() bool { return false }
func debugOmitLocalAddresses() bool { return false } func debugOmitLocalAddresses() bool { return false }
func logDerpVerbose() bool { return false } func logDerpVerbose() bool { return false }
func debugReSTUNStopOnIdle() bool { return false } func debugReSTUNStopOnIdle() bool { return false }
func debugAlwaysDERP() bool { return false } func debugAlwaysDERP() bool { return false }
func debugEnableSilentDisco() bool { return false } func debugEnableSilentDisco() bool { return false }
func debugUseDerpRouteEnv() string { return "" } func debugSendCallMeUnknownPeer() bool { return false }
func debugUseDerpRoute() opt.Bool { return "" } func debugUseDerpRouteEnv() string { return "" }
func debugUseDerpRoute() opt.Bool { return "" }
func inTest() bool { return false } func inTest() bool { return false }

@ -1727,7 +1727,19 @@ func (c *Conn) runDerpReader(ctx context.Context, derpFakeAddr netip.AddrPort, d
case derp.HealthMessage: case derp.HealthMessage:
health.SetDERPRegionHealth(regionID, m.Problem) health.SetDERPRegionHealth(regionID, m.Problem)
case derp.PeerGoneMessage: case derp.PeerGoneMessage:
c.removeDerpPeerRoute(key.NodePublic(m), regionID, dc) switch m.Reason {
case derp.PeerGoneReasonDisconnected:
// Do nothing.
case derp.PeerGoneReasonNotHere:
metricRecvDiscoDERPPeerNotHere.Add(1)
c.logf("[unexpected] magicsock: derp-%d does not know about peer %s, removing route",
regionID, key.NodePublic(m.Peer).ShortString())
default:
metricRecvDiscoDERPPeerGoneUnknown.Add(1)
c.logf("[unexpected] magicsock: derp-%d peer %s gone, reason %v, removing route",
regionID, key.NodePublic(m.Peer).ShortString(), m.Reason)
}
c.removeDerpPeerRoute(key.NodePublic(m.Peer), regionID, dc)
default: default:
// Ignore. // Ignore.
continue continue
@ -2381,6 +2393,12 @@ func (c *Conn) enqueueCallMeMaybe(derpAddr netip.AddrPort, de *endpoint) {
eps = append(eps, ep.Addr) eps = append(eps, ep.Addr)
} }
go de.c.sendDiscoMessage(derpAddr, de.publicKey, de.discoKey, &disco.CallMeMaybe{MyNumber: eps}, discoLog) go de.c.sendDiscoMessage(derpAddr, de.publicKey, de.discoKey, &disco.CallMeMaybe{MyNumber: eps}, discoLog)
if debugSendCallMeUnknownPeer() {
// Send a callMeMaybe packet to a non-existent peer
unknownKey := key.NewNode().Public()
c.logf("magicsock: sending CallMeMaybe to unknown peer per TS_DEBUG_SEND_CALLME_UNKNOWN_PEER")
go de.c.sendDiscoMessage(derpAddr, unknownKey, de.discoKey, &disco.CallMeMaybe{MyNumber: eps}, discoLog)
}
} }
// discoInfoLocked returns the previous or new discoInfo for k. // discoInfoLocked returns the previous or new discoInfo for k.
@ -4823,7 +4841,8 @@ var (
metricRecvDiscoCallMeMaybe = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe") metricRecvDiscoCallMeMaybe = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe")
metricRecvDiscoCallMeMaybeBadNode = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_node") metricRecvDiscoCallMeMaybeBadNode = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_node")
metricRecvDiscoCallMeMaybeBadDisco = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_disco") metricRecvDiscoCallMeMaybeBadDisco = clientmetric.NewCounter("magicsock_disco_recv_callmemaybe_bad_disco")
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 // metricDERPHomeChange is how many times our DERP home region DI has
// changed from non-zero to a different non-zero. // changed from non-zero to a different non-zero.
metricDERPHomeChange = clientmetric.NewCounter("derp_home_change") metricDERPHomeChange = clientmetric.NewCounter("derp_home_change")

Loading…
Cancel
Save