derp/derpmap: new DERP config package, merge netcheck into magicsock more

Fixes #153
Updates #162
Updates #163

Signed-off-by: Brad Fitzpatrick <bradfitz@tailscale.com>
pull/169/head
Brad Fitzpatrick 4 years ago committed by Brad Fitzpatrick
parent bd0e20f351
commit 39c0ae1dba

@ -11,22 +11,42 @@ import (
"sort"
"time"
"tailscale.com/derp/derpmap"
"tailscale.com/net/dnscache"
"tailscale.com/netcheck"
)
func runNetcheck(ctx context.Context, args []string) error {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
report, err := netcheck.GetReport(ctx, log.Printf)
c := &netcheck.Client{
DERP: derpmap.Prod(),
Logf: log.Printf,
DNSCache: dnscache.Get(),
}
report, err := c.GetReport(ctx)
if err != nil {
log.Fatalf("netcheck: %v", err)
}
fmt.Printf("\nReport:\n")
fmt.Printf("\t* UDP: %v\n", report.UDP)
fmt.Printf("\t* IPv6: %v\n", report.IPv6)
if report.GlobalV4 != "" {
fmt.Printf("\t* IPv4: yes, %v\n", report.GlobalV4)
} else {
fmt.Printf("\t* IPv4: (no addr found)\n")
}
if report.GlobalV6 != "" {
fmt.Printf("\t* IPv6: yes, %v\n", report.GlobalV6)
} else if report.IPv6 {
fmt.Printf("\t* IPv6: (no addr found)\n")
} else {
fmt.Printf("\t* IPv6: no\n")
}
fmt.Printf("\t* MappingVariesByDestIP: %v\n", report.MappingVariesByDestIP)
fmt.Printf("\t* HairPinning: %v\n", report.HairPinning)
fmt.Printf("\t* Nearest DERP: %v (%v)\n", report.PreferredDERP, netcheck.DERPNodeLocation(report.PreferredDERP))
fmt.Printf("\t* Nearest DERP: %v (%v)\n", report.PreferredDERP, c.DERP.LocationOfID(report.PreferredDERP))
fmt.Printf("\t* DERP latency:\n")
var ss []string
for s := range report.DERPLatency {

@ -0,0 +1,138 @@
// Copyright (c) 2020 Tailscale Inc & AUTHORS All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// Package derpmap contains information about Tailscale.com's production DERP nodes.
package derpmap
import (
"fmt"
)
// World is a set of DERP server.
type World struct {
servers []*Server
ids []int
byID map[int]*Server
stun4 []string
stun6 []string
}
func (w *World) IDs() []int { return w.ids }
func (w *World) STUN4() []string { return w.stun4 }
func (w *World) STUN6() []string { return w.stun6 }
func (w *World) ServerByID(id int) *Server { return w.byID[id] }
// LocationOfID returns the geographic name of a node, if present.
func (w *World) LocationOfID(id int) string {
if s, ok := w.byID[id]; ok {
return s.Geo
}
return ""
}
func (w *World) NodeIDOfSTUNServer(server string) int {
// TODO: keep reverse map? Small enough to not matter for now.
for _, s := range w.servers {
if s.STUN4 == server || s.STUN6 == server {
return s.ID
}
}
return 0
}
// Prod returns the production DERP nodes.
func Prod() *World {
return prod
}
func NewTestWorld(stun ...string) *World {
w := &World{}
for i, s := range stun {
w.add(&Server{
ID: i + 1,
Geo: fmt.Sprintf("Testopolis-%d", i+1),
STUN4: s,
})
}
return w
}
func NewTestWorldWith(servers ...*Server) *World {
w := &World{}
for _, s := range servers {
w.add(s)
}
return w
}
var prod = new(World) // ... a dazzling place I never knew
func addProd(id int, geo string) {
prod.add(&Server{
ID: id,
Geo: geo,
HostHTTPS: fmt.Sprintf("derp%v.tailscale.com", id),
STUN4: fmt.Sprintf("derp%v.tailscale.com:3478", id),
STUN6: fmt.Sprintf("derp%v-v6.tailscale.com:3478", id),
})
}
func (w *World) add(s *Server) {
if s.ID == 0 {
panic("ID required")
}
if _, dup := w.byID[s.ID]; dup {
panic("duplicate prod server")
}
if w.byID == nil {
w.byID = make(map[int]*Server)
}
w.byID[s.ID] = s
w.ids = append(w.ids, s.ID)
w.servers = append(w.servers, s)
if s.STUN4 != "" {
w.stun4 = append(w.stun4, s.STUN4)
}
if s.STUN6 != "" {
w.stun6 = append(w.stun6, s.STUN6)
}
}
func init() {
addProd(1, "New York")
addProd(2, "San Francisco")
addProd(3, "Singapore")
addProd(4, "Frankfurt")
}
// Server is configuration for a DERP server.
type Server struct {
ID int
// HostHTTPS is the HTTPS hostname.
HostHTTPS string
// STUN4 is the host:port of the IPv4 STUN server on this DERP
// node. Required.
STUN4 string
// STUN6 optionally provides the IPv6 host:port of the STUN
// server on the DERP node.
// It should be an IPv6-only address for now. (We currently make lazy
// assumptions that the server names are unique.)
STUN6 string
// Geo is a human-readable geographic region name of this server.
Geo string
}
func (s *Server) String() string {
if s == nil {
return "<nil *derpmap.Server>"
}
if s.Geo != "" {
return fmt.Sprintf("%v (%v)", s.HostHTTPS, s.Geo)
}
return s.HostHTTPS
}

@ -29,11 +29,8 @@ func TestIPN(t *testing.T) {
// Turn off STUN for the test to make it hermitic.
// TODO(crawshaw): add a test that runs against a local STUN server.
origDefaultSTUN := magicsock.DefaultSTUN
magicsock.DefaultSTUN = nil
defer func() {
magicsock.DefaultSTUN = origDefaultSTUN
}()
magicsock.DisableSTUNForTesting = true
defer func() { magicsock.DisableSTUNForTesting = false }()
// TODO(apenwarr): Make resource checks actually pass.
// They don't right now, because (at least) wgengine doesn't fully

@ -7,15 +7,17 @@ package netcheck
import (
"context"
"errors"
"fmt"
"io"
"log"
"net"
"strconv"
"strings"
"sync"
"time"
"golang.org/x/sync/errgroup"
"tailscale.com/derp/derpmap"
"tailscale.com/interfaces"
"tailscale.com/net/dnscache"
"tailscale.com/stun"
@ -32,6 +34,9 @@ type Report struct {
PreferredDERP int // or 0 for unknown
DERPLatency map[string]time.Duration // keyed by STUN host:port
GlobalV4 string // ip:port of global IPv4
GlobalV6 string // [ip]:port of global IPv6 // TODO
// TODO: update Clone when adding new fields
}
@ -49,111 +54,193 @@ func (r *Report) Clone() *Report {
return &r2
}
const derpNodes = 4 // [1,4] contiguous, at present
// Client generates a netcheck Report.
type Client struct {
// DERP is the DERP world to use.
DERP *derpmap.World
// DNSCache optionally specifies a DNSCache to use.
// If nil, a DNS cache is not used.
DNSCache *dnscache.Resolver
// Logf optionally specifies where to log to.
Logf logger.Logf
GetSTUNConn4 func() STUNConn
GetSTUNConn6 func() STUNConn
var derpLoc = map[int]string{
1: "New York",
2: "San Francsico",
3: "Singapore",
4: "Frankfurt",
s4 *stunner.Stunner
s6 *stunner.Stunner
}
func DERPNodeLocation(id int) string { return derpLoc[id] }
// STUNConn is the interface required by the netcheck Client when
// reusing an existing UDP connection.
type STUNConn interface {
WriteTo([]byte, net.Addr) (int, error)
ReadFrom([]byte) (int, net.Addr, error)
}
func GetReport(ctx context.Context, logf logger.Logf) (*Report, error) {
func (c *Client) logf(format string, a ...interface{}) {
if c.Logf != nil {
c.Logf(format, a...)
} else {
log.Printf(format, a...)
}
}
var stunServers []string
var stunServers6 []string
for i := 1; i <= derpNodes; i++ {
stunServers = append(stunServers, fmt.Sprintf("derp%v.tailscale.com:3478", i))
stunServers6 = append(stunServers6, fmt.Sprintf("derp%v-v6.tailscale.com:3478", i))
func (c *Client) ReceiveSTUNPacket(pkt []byte, src *net.UDPAddr) {
var st *stunner.Stunner
if src == nil || src.IP == nil {
panic("bogus src")
}
if src.IP.To4() != nil {
st = c.s4
} else {
st = c.s6
}
if st != nil {
st.Receive(pkt, src)
}
}
// GetReport gets a report.
//
// It may not be called concurrently with itself.
func (c *Client) GetReport(ctx context.Context) (*Report, error) {
// Mask user context with ours that we guarantee to cancel so
// we can depend on it being closed in goroutines later.
// (User ctx might be context.Background, etc)
ctx, cancel := context.WithCancel(ctx)
defer cancel()
defer func() {
c.s4 = nil
c.s6 = nil
}()
if c.DERP == nil {
return nil, errors.New("netcheck: GetReport: Client.DERP is nil")
}
stuns4 := c.DERP.STUN4()
stuns6 := c.DERP.STUN6()
if len(stuns4) == 0 {
// TODO: make this work? if we ever need it
// to. Requirement for self-hosted Tailscale might be
// to run a DERP+STUN server co-resident with the
// Control server.
return nil, errors.New("netcheck: GetReport: no STUN servers, no Report")
}
for _, s := range stuns4 {
if _, _, err := net.SplitHostPort(s); err != nil {
return nil, fmt.Errorf("netcheck: GetReport: bogus STUN4 server %q", s)
}
}
for _, s := range stuns6 {
if _, _, err := net.SplitHostPort(s); err != nil {
return nil, fmt.Errorf("netcheck: GetReport: bogus STUN6 server %q", s)
}
}
closeOnCtx := func(c io.Closer) {
<-ctx.Done()
c.Close()
}
v6, err := interfaces.HaveIPv6GlobalAddress()
v6iface, err := interfaces.HaveIPv6GlobalAddress()
if err != nil {
logf("interfaces: %v", err)
c.logf("interfaces: %v", err)
}
var (
mu sync.Mutex
ret = &Report{
DERPLatency: map[string]time.Duration{},
}
gotIP = map[string]string{} // server -> IP
gotIPHair = map[string]string{} // server -> IP for second UDP4 for hairpinning
gotIP4 string
gotEP = map[string]string{} // server -> ipPort
gotEPHair = map[string]string{} // server -> ipPort for second UDP4 for hairpinning
gotEP4 string
bestDerpLatency time.Duration
)
add := func(server, ip string, d time.Duration) {
logf("%s says we are %s (in %v)", server, ip, d)
add := func(server, ipPort string, d time.Duration) {
c.logf("%s says we are %s (in %v)", server, ipPort, d)
ua, err := net.ResolveUDPAddr("udp", ipPort)
if err != nil {
c.logf("[unexpected] STUN addr %q", ipPort)
return
}
isV6 := ua.IP.To4() == nil
mu.Lock()
defer mu.Unlock()
ret.UDP = true
ret.DERPLatency[server] = d
if strings.Contains(server, "-v6") {
if isV6 {
ret.IPv6 = true
ret.GlobalV6 = ipPort
// TODO: track MappingVariesByDestIP for IPv6
// too? Would be sad if so, but who knows.
} else {
// IPv4
if gotIP4 == "" {
gotIP4 = ip
if gotEP4 == "" {
gotEP4 = ipPort
ret.GlobalV4 = ipPort
} else {
if gotIP4 != ip {
if gotEP4 != ipPort {
ret.MappingVariesByDestIP.Set(true)
} else if ret.MappingVariesByDestIP == "" {
ret.MappingVariesByDestIP.Set(false)
}
}
}
gotIP[server] = ip
gotEP[server] = ipPort
if ret.PreferredDERP == 0 {
ret.PreferredDERP = derpIndexOfSTUNHostPort(server)
if ret.PreferredDERP == 0 || d < bestDerpLatency {
bestDerpLatency = d
ret.PreferredDERP = c.DERP.NodeIDOfSTUNServer(server)
}
}
addHair := func(server, ip string, d time.Duration) {
addHair := func(server, ipPort string, d time.Duration) {
mu.Lock()
defer mu.Unlock()
gotIPHair[server] = ip
gotEPHair[server] = ipPort
}
var pc4, pc6 net.PacketConn
var pc4, pc6 STUNConn
pc4, err = net.ListenPacket("udp4", ":0")
if err != nil {
logf("udp4: %v", err)
return nil, err
if f := c.GetSTUNConn4; f != nil {
pc4 = f()
} else {
u4, err := net.ListenPacket("udp4", ":0")
if err != nil {
c.logf("udp4: %v", err)
return nil, err
}
pc4 = u4
go closeOnCtx(u4)
}
go closeOnCtx(pc4)
// And a second UDP4 socket to check hairpinning.
pc4Hair, err := net.ListenPacket("udp4", ":0")
if err != nil {
logf("udp4: %v", err)
c.logf("udp4: %v", err)
return nil, err
}
go closeOnCtx(pc4Hair)
if v6 {
pc6, err = net.ListenPacket("udp6", ":0")
if err != nil {
logf("udp6: %v", err)
v6 = false
if v6iface {
if f := c.GetSTUNConn6; f != nil {
pc6 = f()
} else {
go closeOnCtx(pc6)
u6, err := net.ListenPacket("udp6", ":0")
if err != nil {
c.logf("udp6: %v", err)
} else {
pc6 = u6
go closeOnCtx(u6)
}
}
}
reader := func(s *stunner.Stunner, pc net.PacketConn, maxReads int) {
reader := func(s *stunner.Stunner, pc STUNConn, maxReads int) {
var buf [64 << 10]byte
for i := 0; i < maxReads; i++ {
n, addr, err := pc.ReadFrom(buf[:])
@ -161,12 +248,12 @@ func GetReport(ctx context.Context, logf logger.Logf) (*Report, error) {
if ctx.Err() != nil {
return
}
logf("ReadFrom: %v", err)
c.logf("ReadFrom: %v", err)
return
}
ua, ok := addr.(*net.UDPAddr)
if !ok {
logf("ReadFrom: unexpected addr %T", addr)
c.logf("ReadFrom: unexpected addr %T", addr)
continue
}
s.Receive(buf[:n], ua)
@ -180,34 +267,40 @@ func GetReport(ctx context.Context, logf logger.Logf) (*Report, error) {
s4 := &stunner.Stunner{
Send: pc4.WriteTo,
Endpoint: add,
Servers: stunServers,
Logf: logf,
Servers: stuns4,
Logf: c.logf,
DNSCache: dnscache.Get(),
}
c.s4 = s4
grp.Go(func() error { return s4.Run(ctx) })
go reader(s4, pc4, unlimited)
if c.GetSTUNConn4 == nil {
go reader(s4, pc4, unlimited)
}
s4Hair := &stunner.Stunner{
Send: pc4Hair.WriteTo,
Endpoint: addHair,
Servers: stunServers,
Logf: logf,
Servers: stuns4,
Logf: c.logf,
DNSCache: dnscache.Get(),
}
grp.Go(func() error { return s4Hair.Run(ctx) })
go reader(s4Hair, pc4Hair, 2)
if v6 {
if pc6 != nil {
s6 := &stunner.Stunner{
Endpoint: add,
Send: pc6.WriteTo,
Servers: stunServers6,
Logf: logf,
Servers: stuns6,
Logf: c.logf,
OnlyIPv6: true,
DNSCache: dnscache.Get(),
}
c.s6 = s6
grp.Go(func() error { return s6.Run(ctx) })
go reader(s6, pc6, unlimited)
if c.GetSTUNConn6 == nil {
go reader(s6, pc6, unlimited)
}
}
err = grp.Wait()
@ -220,7 +313,7 @@ func GetReport(ctx context.Context, logf logger.Logf) (*Report, error) {
// Check hairpinning.
if ret.MappingVariesByDestIP == "false" {
hairIPStr, hairPortStr, _ := net.SplitHostPort(gotIPHair["derp1.tailscale.com:3478"])
hairIPStr, hairPortStr, _ := net.SplitHostPort(gotEPHair["derp1.tailscale.com:3478"])
hairIP := net.ParseIP(hairIPStr)
hairPort, _ := strconv.Atoi(hairPortStr)
if hairIP != nil && hairPort != 0 {
@ -241,14 +334,3 @@ func GetReport(ctx context.Context, logf logger.Logf) (*Report, error) {
return ret.Clone(), nil
}
// derpIndexOfSTUNHostPort extracts the derp indes from a STUN host:port like
// "derp1-v6.tailscale.com:3478" or "derp2.tailscale.com:3478".
// It returns 0 on unexpected input.
func derpIndexOfSTUNHostPort(hp string) int {
hp = strings.TrimSuffix(hp, ".tailscale.com:3478")
hp = strings.TrimSuffix(hp, "-v6")
hp = strings.TrimPrefix(hp, "derp")
n, _ := strconv.Atoi(hp)
return n // 0 on error is okay
}

@ -94,7 +94,7 @@ func (s *Stunner) logf(format string, args ...interface{}) {
// Receive delivers a STUN packet to the stunner.
func (s *Stunner) Receive(p []byte, fromAddr *net.UDPAddr) {
if !stun.Is(p) {
s.logf("stunner: received non-STUN packet")
s.logf("[unexpected] stunner: received non-STUN packet")
return
}
now := time.Now()
@ -131,6 +131,9 @@ func (s *Stunner) resolver() *net.Resolver {
func (s *Stunner) Run(ctx context.Context) error {
s.sessions = map[string]*session{}
for _, server := range s.Servers {
if _, _, err := net.SplitHostPort(server); err != nil {
return fmt.Errorf("Stunner.Run: invalid server %q (in Server list %q)", server, s.Servers)
}
sctx, cancel := context.WithCancel(ctx)
s.sessions[server] = &session{
ctx: sctx,
@ -155,11 +158,18 @@ func (s *Stunner) Run(ctx context.Context) error {
func (s *Stunner) runServer(ctx context.Context, server string) {
session := s.sessions[server]
// If we're using a DNS cache, prime the cache before doing
// any quick timeouts (100ms, etc) so the timeout doesn't
// apply to the first DNS lookup.
if s.DNSCache != nil {
_, _ = s.DNSCache.LookupIP(ctx, server)
}
for i, d := range retryDurations {
ctx, cancel := context.WithTimeout(ctx, d)
err := s.sendSTUN(ctx, server)
if err != nil {
s.logf("stunner: %s: %v", server, err)
s.logf("stunner: sendSTUN(%q): %v", server, err)
}
select {
@ -194,13 +204,13 @@ func (s *Stunner) sendSTUN(ctx context.Context, server string) error {
if s.DNSCache != nil {
ip, err := s.DNSCache.LookupIP(ctx, host)
if err != nil {
return fmt.Errorf("lookup ip addr: %v", err)
return fmt.Errorf("lookup ip addr from cache (%q): %v", host, err)
}
ipAddrs = []net.IPAddr{{IP: ip}}
} else {
ipAddrs, err = s.resolver().LookupIPAddr(ctx, host)
if err != nil {
return fmt.Errorf("lookup ip addr: %v", err)
return fmt.Errorf("lookup ip addr (%q): %v", host, err)
}
}
for _, ipAddr := range ipAddrs {

@ -1,63 +0,0 @@
// Copyright 2019 Tailscale & AUTHORS. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package magicsock
import (
"fmt"
"net"
)
// DerpMagicIP is a fake WireGuard endpoint IP address that means
// to use DERP. When used, the port number of the WireGuard endpoint
// is the DERP server number to use.
//
// Mnemonic: 3.3.40 are numbers above the keys D, E, R, P.
const DerpMagicIP = "127.3.3.40"
var derpMagicIP = net.ParseIP(DerpMagicIP).To4()
var (
derpHostOfIndex = map[int]string{} // node ID index (fake port number) -> hostname
derpIndexOfHost = map[string]int{} // derpHostOfIndex reversed
derpNodeID []int
)
const (
derpUSNY = 1
derpUSSF = 2
derpSG = 3
derpDE = 4
)
func init() {
addDerper(derpUSNY, "derp1.tailscale.com")
addDerper(derpUSSF, "derp2.tailscale.com")
addDerper(derpSG, "derp3.tailscale.com")
addDerper(derpDE, "derp4.tailscale.com")
}
func addDerper(i int, host string) {
if other, dup := derpHostOfIndex[i]; dup {
panic(fmt.Sprintf("duplicate DERP index %v (host %q and %q)", i, other, host))
}
if other, dup := derpIndexOfHost[host]; dup {
panic(fmt.Sprintf("duplicate DERP host %q (index %v and %v)", host, other, i))
}
derpHostOfIndex[i] = host
derpIndexOfHost[host] = i
derpNodeID = append(derpNodeID, i)
}
// derpHost returns the hostname of a DERP server index (a fake port
// number used with derpMagicIP).
func derpHost(i int) string {
if h, ok := derpHostOfIndex[i]; ok {
return h
}
if 1 <= i && i <= 64<<10 {
return fmt.Sprintf("derp%v.tailscale.com", i)
}
return ""
}

@ -30,11 +30,11 @@ import (
"golang.org/x/time/rate"
"tailscale.com/derp"
"tailscale.com/derp/derphttp"
"tailscale.com/derp/derpmap"
"tailscale.com/interfaces"
"tailscale.com/net/dnscache"
"tailscale.com/netcheck"
"tailscale.com/stun"
"tailscale.com/stunner"
"tailscale.com/tailcfg"
"tailscale.com/types/key"
"tailscale.com/types/logger"
@ -46,11 +46,13 @@ import (
type Conn struct {
pconn *RebindingUDPConn
pconnPort uint16
stunServers []string
startEpUpdate chan struct{} // send to trigger endpoint update
epFunc func(endpoints []string)
logf logger.Logf
sendLogLimit *rate.Limiter
derps *derpmap.World
netChecker *netcheck.Client
goroutines sync.WaitGroup
// bufferedIPv4From and bufferedIPv4Packet are owned by
// ReceiveIPv4, and used when both a DERP and IPv4 packet arrive
@ -95,6 +97,15 @@ type Conn struct {
derpTLSConfig *tls.Config // normally nil; used by tests
}
// DerpMagicIP is a fake WireGuard endpoint IP address that means
// to use DERP. When used, the port number of the WireGuard endpoint
// is the DERP server number to use.
//
// Mnemonic: 3.3.40 are numbers above the keys D, E, R, P.
const DerpMagicIP = "127.3.3.40"
var derpMagicIP = net.ParseIP(DerpMagicIP).To4()
// activeDerp contains fields for an active DERP connection.
type activeDerp struct {
c *derphttp.Client
@ -114,10 +125,7 @@ type udpAddr struct {
// The current default (zero) means to auto-select a random free port.
const DefaultPort = 0
var DefaultSTUN = []string{
"stun.l.google.com:19302",
"stun3.l.google.com:19302",
}
var DisableSTUNForTesting bool
// Options contains options for Listen.
type Options struct {
@ -127,6 +135,8 @@ type Options struct {
// Zero means to pick one automatically.
Port uint16
// STUN, if non-empty, specifies alternate STUN servers for testing.
// If empty, the production DERP servers are used.
STUN []string
// EndpointsFunc optionally provides a func to be called when
@ -178,7 +188,6 @@ func Listen(opts Options) (*Conn, error) {
pconn: new(RebindingUDPConn),
pconnPort: opts.Port,
sendLogLimit: rate.NewLimiter(rate.Every(1*time.Minute), 1),
stunServers: append([]string{}, opts.STUN...),
startEpUpdate: make(chan struct{}, 1),
connCtx: connCtx,
connCtxCancel: connCtxCancel,
@ -190,11 +199,27 @@ func Listen(opts Options) (*Conn, error) {
derpRecvCh: make(chan derpReadResult),
udpRecvCh: make(chan udpReadResult),
derpTLSConfig: opts.derpTLSConfig,
derps: derpmap.Prod(),
}
if len(opts.STUN) > 0 {
c.derps = derpmap.NewTestWorld(opts.STUN...)
}
c.netChecker = &netcheck.Client{
DERP: c.derps,
Logf: logger.WithPrefix(c.logf, "netcheck: "),
GetSTUNConn4: func() netcheck.STUNConn { return c.pconn },
// TODO: add GetSTUNConn6 once Conn has a pconn6
}
c.ignoreSTUNPackets()
c.pconn.Reset(packetConn.(*net.UDPConn))
c.reSTUN()
go c.epUpdate(connCtx)
c.goroutines.Add(1)
go func() {
defer c.goroutines.Done()
c.epUpdate(connCtx)
}()
return c, nil
}
@ -244,10 +269,14 @@ func (c *Conn) epUpdate(ctx context.Context) {
go func() {
defer close(lastDone)
c.updateNetInfo() // best effort
c.cleanStaleDerp()
endpoints, err := c.determineEndpoints(epCtx)
netReport, err := c.updateNetInfo(epCtx)
if err != nil {
c.logf("magicsock.Conn: updateNetInfo failed: %v", err)
return
}
endpoints, err := c.determineEndpoints(epCtx, netReport)
if err != nil {
c.logf("magicsock.Conn: endpoint update failed: %v", err)
// TODO(crawshaw): are there any conditions under which
@ -258,36 +287,25 @@ func (c *Conn) epUpdate(ctx context.Context) {
return
}
lastEndpoints = endpoints
// TODO(bradfiz): get nearestDerp back to ipn for a HostInfo update
c.epFunc(endpoints)
}()
}
}
func (c *Conn) hasExternalSTUN() bool {
for _, hp := range c.stunServers {
if strings.Contains(hp, ".com:") {
// matches stun.l.google.com:19302 or derp\d+.tailscale.com:nnnn
return true
}
}
return false
}
func (c *Conn) updateNetInfo() {
logf := logger.WithPrefix(c.logf, "updateNetInfo: ")
if !c.hasExternalSTUN() {
logf("skipping in non-production mode")
return
func (c *Conn) updateNetInfo(ctx context.Context) (*netcheck.Report, error) {
if DisableSTUNForTesting {
return nil, errors.New("STUN disabled for testing")
}
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
ctx, cancel := context.WithTimeout(ctx, 2*time.Second)
defer cancel()
report, err := netcheck.GetReport(ctx, logf)
c.stunReceiveFunc.Store(c.netChecker.ReceiveSTUNPacket)
defer c.ignoreSTUNPackets()
report, err := c.netChecker.GetReport(ctx)
if err != nil {
logf("GetReport: %v", err)
return
return nil, err
}
ni := &tailcfg.NetInfo{
@ -314,6 +332,7 @@ func (c *Conn) updateNetInfo() {
// TODO: set link type
c.callNetInfoCallback(ni)
return report, nil
}
var processStartUnixNano = time.Now().UnixNano()
@ -331,14 +350,15 @@ func (c *Conn) pickDERPFallback() int {
return c.myDerp
}
if len(derpNodeID) == 0 {
ids := c.derps.IDs()
if len(ids) == 0 {
// No DERP nodes registered.
return 0
}
h := fnv.New64()
h.Write([]byte(fmt.Sprintf("%p/%d", c, processStartUnixNano))) // arbitrary
return derpNodeID[rand.New(rand.NewSource(int64(h.Sum64()))).Intn(len(derpNodeID))]
return ids[rand.New(rand.NewSource(int64(h.Sum64()))).Intn(len(ids))]
}
// callNetInfoCallback calls the NetInfo callback (if previously
@ -385,7 +405,7 @@ func (c *Conn) setNearestDERP(derpNum int) (wantDERP bool) {
return true
}
c.myDerp = derpNum
c.logf("home DERP server is now %s", derpHost(derpNum))
c.logf("home DERP server is now %v, %v", derpNum, c.derps.ServerByID(derpNum))
for i, ad := range c.activeDerp {
go ad.c.NotePreferred(i == c.myDerp)
}
@ -398,35 +418,24 @@ func (c *Conn) setNearestDERP(derpNum int) (wantDERP bool) {
// determineEndpoints returns the machine's endpoint addresses. It
// does a STUN lookup to determine its public address.
func (c *Conn) determineEndpoints(ctx context.Context) (ipPorts []string, err error) {
var (
alreadyMu sync.Mutex
already = make(map[string]bool) // endpoint -> true
)
var eps []string // unique endpoints
func (c *Conn) determineEndpoints(ctx context.Context, nr *netcheck.Report) (ipPorts []string, err error) {
already := make(map[string]bool) // endpoint -> true
var eps []string // unique endpoints
addAddr := func(s, reason string) {
c.logf("magicsock: found local %s (%s)\n", s, reason)
alreadyMu.Lock()
defer alreadyMu.Unlock()
if !already[s] {
already[s] = true
eps = append(eps, s)
}
}
s := &stunner.Stunner{
Send: c.pconn.WriteTo,
Endpoint: func(server, endpoint string, d time.Duration) { addAddr(endpoint, "stun") },
Servers: c.stunServers,
Logf: c.logf,
if nr.GlobalV4 != "" {
addAddr(nr.GlobalV4, "stun")
}
c.stunReceiveFunc.Store(s.Receive)
if err := s.Run(ctx); err != nil {
return nil, err
const tailControlDoesIPv6 = false // TODO: when IPv6 filtering/splitting is enabled in tailcontrol
if nr.GlobalV6 != "" && tailControlDoesIPv6 {
addAddr(nr.GlobalV6, "stun")
}
c.ignoreSTUNPackets()
@ -615,7 +624,9 @@ func (c *Conn) Send(b []byte, ep conn.Endpoint) error {
ret = err
}
if err != nil && addr != roamAddr && c.sendLogLimit.Allow() {
c.logf("magicsock: Conn.Send(%v): %v", addr, err)
if c.connCtx.Err() == nil { // don't log if we're closed
c.logf("magicsock: Conn.Send(%v): %v", addr, err)
}
}
}
if success {
@ -686,14 +697,15 @@ func (c *Conn) derpWriteChanOfAddr(addr *net.UDPAddr) chan<- derpWriteRequest {
if c.activeDerp == nil {
c.activeDerp = make(map[int]activeDerp)
}
host := derpHost(addr.Port)
if host == "" {
derpSrv := c.derps.ServerByID(addr.Port)
if derpSrv == nil || derpSrv.HostHTTPS == "" {
return nil
}
// TODO(bradfitz): don't hold derpMu here. It's slow. Release first and use singleflight to dial+re-lock to add.
dc, err := derphttp.NewClient(c.privateKey, "https://"+host+"/derp", c.logf)
dc, err := derphttp.NewClient(c.privateKey, "https://"+derpSrv.HostHTTPS+"/derp", c.logf)
if err != nil {
c.logf("derphttp.NewClient: port %d, host %q invalid? err: %v", addr.Port, host, err)
c.logf("derphttp.NewClient: port %d, host %q invalid? err: %v", addr.Port, derpSrv.HostHTTPS, err)
return nil
}
dc.NotePreferred(c.myDerp == addr.Port)
@ -852,6 +864,34 @@ type udpReadResult struct {
// immediate cancellation of network operations.
var aLongTimeAgo = time.Unix(233431200, 0)
// awaitUDP4 reads a single IPv4 UDP packet (or an error) and sends it
// to c.udpRecvCh, skipping over (but handling) any STUN replies.
func (c *Conn) awaitUDP4(b []byte) {
for {
n, pAddr, err := c.pconn.ReadFrom(b)
if err != nil {
select {
case c.udpRecvCh <- udpReadResult{err: err}:
case <-c.donec():
}
return
}
addr := pAddr.(*net.UDPAddr)
if stun.Is(b[:n]) {
c.stunReceiveFunc.Load().(func([]byte, *net.UDPAddr))(b, addr)
continue
}
addr.IP = addr.IP.To4()
select {
case c.udpRecvCh <- udpReadResult{n: n, addr: addr}:
case <-c.donec():
}
return
}
}
func (c *Conn) ReceiveIPv4(b []byte) (n int, ep conn.Endpoint, addr *net.UDPAddr, err error) {
// First, process any buffered packet from earlier.
if addr := c.bufferedIPv4From; addr != nil {
@ -859,31 +899,7 @@ func (c *Conn) ReceiveIPv4(b []byte) (n int, ep conn.Endpoint, addr *net.UDPAddr
return copy(b, c.bufferedIPv4Packet), c.findEndpoint(addr), addr, nil
}
go func() {
// Read a packet, and process any STUN packets before returning.
for {
n, pAddr, err := c.pconn.ReadFrom(b)
if err != nil {
select {
case c.udpRecvCh <- udpReadResult{err: err}:
case <-c.donec():
}
return
}
if stun.Is(b[:n]) {
c.stunReceiveFunc.Load().(func([]byte, *net.UDPAddr))(b, addr)
continue
}
addr := pAddr.(*net.UDPAddr)
addr.IP = addr.IP.To4()
select {
case c.udpRecvCh <- udpReadResult{n: n, addr: addr}:
case <-c.donec():
}
return
}
}()
go c.awaitUDP4(b)
// Once the above goroutine has started, it owns b until it writes
// to udpRecvCh. The code below must not access b until it's
@ -1052,7 +1068,9 @@ func (c *Conn) Close() error {
c.closeAllDerpLocked()
c.derpMu.Unlock()
return c.pconn.Close()
err := c.pconn.Close()
c.goroutines.Wait()
return err
}
func (c *Conn) reSTUN() {

@ -25,6 +25,7 @@ import (
"github.com/tailscale/wireguard-go/wgcfg"
"tailscale.com/derp"
"tailscale.com/derp/derphttp"
"tailscale.com/derp/derpmap"
"tailscale.com/stun"
"tailscale.com/types/key"
"tailscale.com/types/logger"
@ -46,8 +47,9 @@ func TestListen(t *testing.T) {
port := pickPort(t)
conn, err := Listen(Options{
Port: port,
STUN: []string{stunAddr.String()},
STUN: []string{stunAddr},
EndpointsFunc: epFunc,
Logf: t.Logf,
})
if err != nil {
t.Fatal(err)
@ -101,11 +103,9 @@ func TestDerpIPConstant(t *testing.T) {
}
func TestPickDERPFallback(t *testing.T) {
if len(derpNodeID) == 0 {
t.Fatal("no DERP nodes registered; this test needs an update after DERP node runtime discovery")
c := &Conn{
derps: derpmap.Prod(),
}
c := new(Conn)
a := c.pickDERPFallback()
if a == 0 {
t.Fatalf("pickDERPFallback returned 0")
@ -123,7 +123,7 @@ func TestPickDERPFallback(t *testing.T) {
// distribution over nodes works.
got := map[int]int{}
for i := 0; i < 50; i++ {
c = new(Conn)
c = &Conn{derps: derpmap.Prod()}
got[c.pickDERPFallback()]++
}
t.Logf("distribution: %v", got)
@ -145,7 +145,7 @@ type stunStats struct {
readIPv6 int
}
func serveSTUN(t *testing.T) (addr net.Addr, cleanupFn func()) {
func serveSTUN(t *testing.T) (addr string, cleanupFn func()) {
t.Helper()
// TODO(crawshaw): use stats to test re-STUN logic
@ -156,8 +156,11 @@ func serveSTUN(t *testing.T) (addr net.Addr, cleanupFn func()) {
t.Fatalf("failed to open STUN listener: %v", err)
}
stunAddr := pc.LocalAddr().String()
stunAddr = strings.Replace(stunAddr, "0.0.0.0:", "localhost:", 1)
go runSTUN(t, pc, &stats)
return pc.LocalAddr(), func() { pc.Close() }
return stunAddr, func() { pc.Close() }
}
func runSTUN(t *testing.T, pc net.PacketConn, stats *stunStats) {
@ -273,20 +276,6 @@ func runDERP(t *testing.T) (s *derp.Server, addr string, cleanupFn func()) {
return s, addr, cleanupFn
}
func stashDerpers() (cleanupFn func()) {
origDerpHostOfIndex := derpHostOfIndex
origDerpIndexOfHost := derpIndexOfHost
origDerpNodeID := derpNodeID
derpHostOfIndex = map[int]string{}
derpIndexOfHost = map[string]int{}
derpNodeID = nil
return func() {
derpHostOfIndex = origDerpHostOfIndex
derpIndexOfHost = origDerpIndexOfHost
derpNodeID = origDerpNodeID
}
}
// devLogger returns a wireguard-go device.Logger that writes
// wireguard logs to the test logger.
func devLogger(t *testing.T, prefix string) *device.Logger {
@ -310,6 +299,7 @@ func devLogger(t *testing.T, prefix string) *device.Logger {
func TestDeviceStartStop(t *testing.T) {
conn, err := Listen(Options{
EndpointsFunc: func(eps []string) {},
Logf: t.Logf,
})
if err != nil {
t.Fatal(err)
@ -332,16 +322,21 @@ func TestTwoDevicePing(t *testing.T) {
// (Do it now, or derpHost will try to connect to derp1.tailscale.com.)
derpServer, derpAddr, derpCleanupFn := runDERP(t)
defer derpCleanupFn()
defer stashDerpers()()
addDerper(1, derpAddr)
stunAddr, stunCleanupFn := serveSTUN(t)
defer stunCleanupFn()
derps := derpmap.NewTestWorldWith(&derpmap.Server{
ID: 1,
HostHTTPS: derpAddr,
STUN4: stunAddr,
Geo: "Testopolis",
})
epCh1 := make(chan []string, 16)
conn1, err := Listen(Options{
STUN: []string{stunAddr.String()},
Logf: logger.WithPrefix(t.Logf, "conn1: "),
STUN: []string{stunAddr},
EndpointsFunc: func(eps []string) {
epCh1 <- eps
},
@ -350,11 +345,13 @@ func TestTwoDevicePing(t *testing.T) {
if err != nil {
t.Fatal(err)
}
conn1.derps = derps
defer conn1.Close()
epCh2 := make(chan []string, 16)
conn2, err := Listen(Options{
STUN: []string{stunAddr.String()},
Logf: logger.WithPrefix(t.Logf, "conn2: "),
STUN: []string{stunAddr},
EndpointsFunc: func(eps []string) {
epCh2 <- eps
},
@ -363,6 +360,7 @@ func TestTwoDevicePing(t *testing.T) {
if err != nil {
t.Fatal(err)
}
conn2.derps = derps
defer conn2.Close()
ports := []uint16{conn1.LocalPort(), conn2.LocalPort()}

@ -119,7 +119,6 @@ func newUserspaceEngineAdvanced(logf logger.Logf, tundev tun.Device, routerGen R
}
magicsockOpts := magicsock.Options{
Port: listenPort,
STUN: magicsock.DefaultSTUN,
EndpointsFunc: endpointsFn,
}
e.magicConn, err = magicsock.Listen(magicsockOpts)

Loading…
Cancel
Save