ipn/ipnlocal: determine client-side-reachability by pinging

PR #17367 introduced the `client-side-reachability` node attribute
that switches the node to completely ignore the online signal from
control. This was to work around the times when the control plane
determines that a node is not online, while the node is still able to
connect to its peers. However, this breaks high-availability failover,
since nodes that are actually offline are never treated that way.

This PR makes nodeBackend.PeerIsReachable actually test reachability
by disco-pinging the peer node.

In the future, the client itself should collect reachability data from
active Wireguard flows.

Updates #17366
Updates tailscale/corp#30379
Updates tailscale/corp#32686
Updates tailscale/corp#33033

Signed-off-by: Simon Law <sfllaw@tailscale.com>
sfllaw/tailscale-ping-for-client-side-reachability
Simon Law 2 months ago
parent 5555bbcd22
commit c386bc9e9c
No known key found for this signature in database
GPG Key ID: B83D1EE07548341D

@ -499,7 +499,7 @@ func NewLocalBackend(logf logger.Logf, logID logid.PublicID, sys *tsd.System, lo
needsCaptiveDetection: make(chan bool), needsCaptiveDetection: make(chan bool),
} }
nb := newNodeBackend(ctx, b.logf, b.sys.Bus.Get()) nb := newNodeBackend(ctx, b.logf, b.sys.Bus.Get(), b)
b.currentNodeAtomic.Store(nb) b.currentNodeAtomic.Store(nb)
nb.ready() nb.ready()
@ -629,7 +629,7 @@ func (b *LocalBackend) currentNode() *nodeBackend {
if v := b.currentNodeAtomic.Load(); v != nil || !testenv.InTest() { if v := b.currentNodeAtomic.Load(); v != nil || !testenv.InTest() {
return v return v
} }
v := newNodeBackend(cmp.Or(b.ctx, context.Background()), b.logf, b.sys.Bus.Get()) v := newNodeBackend(cmp.Or(b.ctx, context.Background()), b.logf, b.sys.Bus.Get(), b)
if b.currentNodeAtomic.CompareAndSwap(nil, v) { if b.currentNodeAtomic.CompareAndSwap(nil, v) {
v.ready() v.ready()
} }
@ -6750,7 +6750,7 @@ func (b *LocalBackend) resetForProfileChangeLockedOnEntry(unlock unlockOnce) err
// down, so no need to do any work. // down, so no need to do any work.
return nil return nil
} }
newNode := newNodeBackend(b.ctx, b.logf, b.sys.Bus.Get()) newNode := newNodeBackend(b.ctx, b.logf, b.sys.Bus.Get(), b)
if oldNode := b.currentNodeAtomic.Swap(newNode); oldNode != nil { if oldNode := b.currentNodeAtomic.Swap(newNode); oldNode != nil {
oldNode.shutdown(errNodeContextChanged) oldNode.shutdown(errNodeContextChanged)
} }

@ -4920,7 +4920,7 @@ func TestSuggestExitNode(t *testing.T) {
allowList = set.SetOf(tt.allowPolicy) allowList = set.SetOf(tt.allowPolicy)
} }
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
defer nb.shutdown(errShutdown) defer nb.shutdown(errShutdown)
nb.SetNetMap(tt.netMap) nb.SetNetMap(tt.netMap)
@ -5373,7 +5373,7 @@ func TestSuggestExitNodeTrafficSteering(t *testing.T) {
tt.netMap.AllCaps = set.SetOf(slices.Collect(caps)) tt.netMap.AllCaps = set.SetOf(slices.Collect(caps))
} }
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
defer nb.shutdown(errShutdown) defer nb.shutdown(errShutdown)
nb.SetNetMap(tt.netMap) nb.SetNetMap(tt.netMap)

@ -6,8 +6,10 @@ package ipnlocal
import ( import (
"cmp" "cmp"
"context" "context"
"fmt"
"net/netip" "net/netip"
"slices" "slices"
"strings"
"sync" "sync"
"sync/atomic" "sync/atomic"
@ -70,6 +72,8 @@ type nodeBackend struct {
ctx context.Context // canceled by [nodeBackend.shutdown] ctx context.Context // canceled by [nodeBackend.shutdown]
ctxCancel context.CancelCauseFunc // cancels ctx ctxCancel context.CancelCauseFunc // cancels ctx
pinger Pinger
// filterAtomic is a stateful packet filter. Immutable once created, but can be // filterAtomic is a stateful packet filter. Immutable once created, but can be
// replaced with a new one. // replaced with a new one.
filterAtomic atomic.Pointer[filter.Filter] filterAtomic atomic.Pointer[filter.Filter]
@ -106,12 +110,13 @@ type nodeBackend struct {
nodeByAddr map[netip.Addr]tailcfg.NodeID nodeByAddr map[netip.Addr]tailcfg.NodeID
} }
func newNodeBackend(ctx context.Context, logf logger.Logf, bus *eventbus.Bus) *nodeBackend { func newNodeBackend(ctx context.Context, logf logger.Logf, bus *eventbus.Bus, pinger Pinger) *nodeBackend {
ctx, ctxCancel := context.WithCancelCause(ctx) ctx, ctxCancel := context.WithCancelCause(ctx)
nb := &nodeBackend{ nb := &nodeBackend{
logf: logf, logf: logf,
ctx: ctx, ctx: ctx,
ctxCancel: ctxCancel, ctxCancel: ctxCancel,
pinger: pinger,
eventClient: bus.Client("ipnlocal.nodeBackend"), eventClient: bus.Client("ipnlocal.nodeBackend"),
readyCh: make(chan struct{}), readyCh: make(chan struct{}),
} }
@ -381,19 +386,68 @@ func (nb *nodeBackend) PeerIsReachable(ctx context.Context, p tailcfg.NodeView)
// This node can always reach itself. // This node can always reach itself.
return true return true
} }
return nb.peerIsReachable(ctx, p) res, err := nb.peerIsReachable(ctx, p)
if err != nil {
nb.logf("peer reachability: %s", err)
}
return res
} }
func (nb *nodeBackend) peerIsReachable(ctx context.Context, p tailcfg.NodeView) bool { // peerIsReachable will only return a [context.DeadlineExceeded] error if ctx
// TODO(sfllaw): The following does not actually test for client-side // was cancelled by its deadline passing, not if an active probe times out.
// reachability. This would require a mechanism that tracks whether the func (nb *nodeBackend) peerIsReachable(ctx context.Context, p tailcfg.NodeView) (bool, error) {
// current node can actually reach this peer, either because they are // When the [Pinger] is missing, fall back on the control plane.
// already communicating or because they can ping each other. if nb.pinger == nil {
// online := p.Online().Get()
// Instead, it makes the client ignore p.Online completely. nb.logf("peer reachable: missing pinger")
return online, nil
}
var addr netip.Addr
for _, a := range p.Addresses().All() {
if !a.IsSingleIP() {
continue
}
addr = a.Addr()
break
}
if !addr.IsValid() {
return false, fmt.Errorf("peer %s (%v) has no IP addresses: %s", p.Name(), p.ID(), p.Addresses())
}
// Wireguard-only nodes cannot be disco-pinged, so we trust the control
// plane.
// //
// TODO(sfllaw): We could try to initiate a Wireguard session and see if
// a response comes back. ICMP ping is also an option, but there might
// be false negatives if ICMP is blocked.
if p.IsWireGuardOnly() {
return p.Online().Get(), nil
}
// Disco ping the peer node to determine if it is actually reachable.
// See tailscale/corp#32686. // See tailscale/corp#32686.
return true //
// TODO(sfllaw): If there is already an active Wireguard session to the
// peer, then we can avoid active probes and return early.
res, err := nb.pinger.Ping(ctx, addr, tailcfg.PingDisco, 0)
if err != nil {
// Encountered a non-ping error, ping failures would be reported
// in res.Err. This is likely to happen when ctx is cancelled.
return false, fmt.Errorf("aborted ping for peer %s (%v) at %s: %w", p.Name(), p.ID(), addr, err)
}
if res.Err != "" {
if res.IsLocalIP {
// Nodes can always reach themselves.
return true, nil
}
if strings.Contains(res.Err, context.DeadlineExceeded.Error()) {
// Ping has timed out: this is not an error.
return false, nil
}
return false, fmt.Errorf("failed to ping peer %s (%v) at %s: %s", p.Name(), p.ID(), addr, err)
}
return true, nil
} }
func nodeIP(n tailcfg.NodeView, pred func(netip.Addr) bool) netip.Addr { func nodeIP(n tailcfg.NodeView, pred func(netip.Addr) bool) netip.Addr {

@ -6,9 +6,12 @@ package ipnlocal
import ( import (
"context" "context"
"errors" "errors"
"fmt"
"net/netip"
"testing" "testing"
"time" "time"
"tailscale.com/ipn/ipnstate"
"tailscale.com/tailcfg" "tailscale.com/tailcfg"
"tailscale.com/tstest" "tailscale.com/tstest"
"tailscale.com/types/netmap" "tailscale.com/types/netmap"
@ -17,7 +20,7 @@ import (
) )
func TestNodeBackendReadiness(t *testing.T) { func TestNodeBackendReadiness(t *testing.T) {
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
// The node backend is not ready until [nodeBackend.ready] is called, // The node backend is not ready until [nodeBackend.ready] is called,
// and [nodeBackend.Wait] should fail with [context.DeadlineExceeded]. // and [nodeBackend.Wait] should fail with [context.DeadlineExceeded].
@ -48,7 +51,7 @@ func TestNodeBackendReadiness(t *testing.T) {
} }
func TestNodeBackendShutdown(t *testing.T) { func TestNodeBackendShutdown(t *testing.T) {
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
shutdownCause := errors.New("test shutdown") shutdownCause := errors.New("test shutdown")
@ -86,7 +89,7 @@ func TestNodeBackendShutdown(t *testing.T) {
} }
func TestNodeBackendReadyAfterShutdown(t *testing.T) { func TestNodeBackendReadyAfterShutdown(t *testing.T) {
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
shutdownCause := errors.New("test shutdown") shutdownCause := errors.New("test shutdown")
nb.shutdown(shutdownCause) nb.shutdown(shutdownCause)
@ -98,7 +101,7 @@ func TestNodeBackendReadyAfterShutdown(t *testing.T) {
func TestNodeBackendParentContextCancellation(t *testing.T) { func TestNodeBackendParentContextCancellation(t *testing.T) {
ctx, cancelCtx := context.WithCancel(context.Background()) ctx, cancelCtx := context.WithCancel(context.Background())
nb := newNodeBackend(ctx, tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(ctx, tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
cancelCtx() cancelCtx()
@ -115,7 +118,7 @@ func TestNodeBackendParentContextCancellation(t *testing.T) {
} }
func TestNodeBackendConcurrentReadyAndShutdown(t *testing.T) { func TestNodeBackendConcurrentReadyAndShutdown(t *testing.T) {
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New(), nil)
// Calling [nodeBackend.ready] and [nodeBackend.shutdown] concurrently // Calling [nodeBackend.ready] and [nodeBackend.shutdown] concurrently
// should not cause issues, and [nodeBackend.Wait] should unblock, // should not cause issues, and [nodeBackend.Wait] should unblock,
@ -127,6 +130,17 @@ func TestNodeBackendConcurrentReadyAndShutdown(t *testing.T) {
} }
func TestNodeBackendReachability(t *testing.T) { func TestNodeBackendReachability(t *testing.T) {
addrs := []netip.Prefix{netip.MustParsePrefix("100.64.0.1/32")}
defaults := func(n tailcfg.Node) tailcfg.Node {
if n.ID == 0 {
n.ID = 1234
}
if n.Name == "" {
n.Name = "exit-node.example.ts.net"
}
return n
}
for _, tc := range []struct { for _, tc := range []struct {
name string name string
@ -139,54 +153,191 @@ func TestNodeBackendReachability(t *testing.T) {
// peer node. // peer node.
cap bool cap bool
// Peer defines the peer node.
peer tailcfg.Node peer tailcfg.Node
// Ping sets how the peer node responds to pings:
// pingTimedOut: peer is unreachable
// pingSuccess: peer responds to pings
// pingLocalhost: peer is the same as the self node
ping mockPinger
want bool want bool
}{ }{
{
name: "disabled/nil",
cap: false,
peer: defaults(tailcfg.Node{
Online: nil,
}),
want: false,
},
{ {
name: "disabled/offline", name: "disabled/offline",
cap: false, cap: false,
peer: tailcfg.Node{ peer: defaults(tailcfg.Node{
Online: ptr.To(false), Online: ptr.To(false),
}, }),
want: false, want: false,
}, },
{ {
name: "disabled/online", name: "disabled/online",
cap: false, cap: false,
peer: tailcfg.Node{ peer: defaults(tailcfg.Node{
Online: ptr.To(true), Online: ptr.To(true),
}, }),
want: true, want: true,
}, },
{
name: "enabled/no_ip",
cap: true,
ping: pingTimedOut,
peer: defaults(tailcfg.Node{
Online: ptr.To(false),
Addresses: nil,
}),
want: false,
},
{ {
name: "enabled/offline", name: "enabled/offline",
cap: true, cap: true,
peer: tailcfg.Node{ peer: defaults(tailcfg.Node{
Online: ptr.To(false), Online: ptr.To(false),
Addresses: addrs,
}),
ping: pingTimedOut,
want: false,
}, },
{
name: "enabled/offline_but_pingable",
cap: true,
peer: defaults(tailcfg.Node{
Online: ptr.To(false),
Addresses: addrs,
}),
ping: pingSuccess,
want: true, want: true,
}, },
{ {
name: "enabled/online", name: "enabled/online",
cap: true, cap: true,
peer: tailcfg.Node{ peer: defaults(tailcfg.Node{
Online: ptr.To(true),
Addresses: addrs,
}),
ping: pingSuccess,
want: true,
},
{
name: "enabled/online_but_unpingable",
cap: true,
peer: defaults(tailcfg.Node{
Online: ptr.To(true), Online: ptr.To(true),
Addresses: addrs,
}),
ping: pingTimedOut,
want: false,
},
{
name: "enabled/offline_localhost",
cap: true,
peer: defaults(tailcfg.Node{
Online: ptr.To(false),
Addresses: addrs,
}),
ping: pingLocalhost,
want: true,
}, },
{
name: "enabled/online_localhost",
cap: true,
peer: defaults(tailcfg.Node{
Online: ptr.To(true),
Addresses: addrs,
}),
ping: pingLocalhost,
want: true, want: true,
}, },
{
name: "enabled/offline_but_cancelled",
cap: true,
peer: defaults(tailcfg.Node{
Online: ptr.To(false),
Addresses: addrs,
}),
ping: pingCancelled,
want: false,
},
{
name: "enabled/online_but_cancelled",
cap: true,
peer: defaults(tailcfg.Node{
Online: ptr.To(true),
Addresses: addrs,
}),
ping: pingCancelled,
want: false,
},
} { } {
t.Run(tc.name, func(t *testing.T) { t.Run(tc.name, func(t *testing.T) {
nb := newNodeBackend(t.Context(), tstest.WhileTestRunningLogger(t), eventbus.New()) ctx := t.Context()
nb := newNodeBackend(ctx, tstest.WhileTestRunningLogger(t), eventbus.New(), mockPinger(tc.ping))
nb.netMap = &netmap.NetworkMap{} nb.netMap = &netmap.NetworkMap{}
if tc.cap { if tc.cap {
nb.netMap.AllCaps.Make() nb.netMap.AllCaps.Make()
nb.netMap.AllCaps.Add(tailcfg.NodeAttrClientSideReachability) nb.netMap.AllCaps.Add(tailcfg.NodeAttrClientSideReachability)
} }
got := nb.PeerIsReachable(t.Context(), tc.peer.View()) if tc.ping == pingCancelled {
c, cancel := context.WithCancelCause(ctx)
ctx = c
cancel(fmt.Errorf("subtest: %q", tc.name))
}
got := nb.PeerIsReachable(ctx, tc.peer.View())
if got != tc.want { if got != tc.want {
t.Errorf("got %v, want %v", got, tc.want) t.Errorf("got %v, want %v", got, tc.want)
} }
}) })
} }
} }
type mockPinger int
const (
pingTimedOut mockPinger = iota
pingSuccess
pingLocalhost
pingCancelled
)
func (p mockPinger) Ping(ctx context.Context, ip netip.Addr, pingType tailcfg.PingType, size int) (*ipnstate.PingResult, error) {
select {
case <-ctx.Done():
return nil, ctx.Err()
default:
}
res := &ipnstate.PingResult{
IP: ip.String(),
NodeIP: ip.String(),
}
switch p {
case pingTimedOut:
ctx, cancel := context.WithTimeout(ctx, 0)
defer cancel()
<-ctx.Done()
res.Err = ctx.Err().Error()
return res, nil
case pingLocalhost:
res.Err = fmt.Sprintf("%v is local Tailscale IP", ip)
res.IsLocalIP = true
case pingSuccess:
res.LatencySeconds = 1
default:
panic(fmt.Sprintf("unknown mockPinger %v", p))
}
return res, nil
}

@ -17,6 +17,12 @@ import (
"tailscale.com/tailcfg" "tailscale.com/tailcfg"
) )
// Pinger is the [LocalBackend.Ping] method.
type Pinger interface {
// Ping is a request to do a ping with the peer handling the given IP.
Ping(ctx context.Context, ip netip.Addr, pingType tailcfg.PingType, size int) (*ipnstate.PingResult, error)
}
func (b *LocalBackend) Ping(ctx context.Context, ip netip.Addr, pingType tailcfg.PingType, size int) (*ipnstate.PingResult, error) { func (b *LocalBackend) Ping(ctx context.Context, ip netip.Addr, pingType tailcfg.PingType, size int) (*ipnstate.PingResult, error) {
if pingType == tailcfg.PingPeerAPI { if pingType == tailcfg.PingPeerAPI {
t0 := b.clock.Now() t0 := b.clock.Now()

@ -175,7 +175,8 @@ type CapabilityVersion int
// - 127: 2025-09-19: can handle C2N /debug/netmap. // - 127: 2025-09-19: can handle C2N /debug/netmap.
// - 128: 2025-10-02: can handle C2N /debug/health. // - 128: 2025-10-02: can handle C2N /debug/health.
// - 129: 2025-10-04: Fixed sleep/wake deadlock in magicsock when using peer relay (PR #17449) // - 129: 2025-10-04: Fixed sleep/wake deadlock in magicsock when using peer relay (PR #17449)
const CurrentCapabilityVersion CapabilityVersion = 129 // - 130: 2025-10-06: Client will determine whether it can reach an exit node when [NodeAttrClientSideReachability] is set.
const CurrentCapabilityVersion CapabilityVersion = 130
// ID is an integer ID for a user, node, or login allocated by the // ID is an integer ID for a user, node, or login allocated by the
// control plane. // control plane.

Loading…
Cancel
Save