net/portmapper: add NAT-PMP client, move port mapping service probing

* move probing out of netcheck into new net/portmapper package
* use PCP ANNOUNCE op codes for PCP discovery, rather than causing
  short-lived (sub-second) side effects with a 1-second-expiring map +
  delete.
* track when we heard things from the router so we can be less wasteful
  in querying the router's port mapping services in the future
* use portmapper from magicsock to map a public port

Fixes #1298
Fixes #1080
Fixes #1001
Updates #864

Signed-off-by: Brad Fitzpatrick <bradfitz@tailscale.com>
pull/1392/head
Brad Fitzpatrick 3 years ago committed by Brad Fitzpatrick
parent d038a5295d
commit c64bd587ae

@ -18,6 +18,7 @@ import (
"github.com/peterbourgon/ff/v2/ffcli"
"tailscale.com/derp/derpmap"
"tailscale.com/net/netcheck"
"tailscale.com/net/portmapper"
"tailscale.com/tailcfg"
"tailscale.com/types/logger"
)
@ -45,6 +46,7 @@ var netcheckArgs struct {
func runNetcheck(ctx context.Context, args []string) error {
c := &netcheck.Client{
UDPBindAddr: os.Getenv("TS_DEBUG_NETCHECK_UDP_BIND"),
PortMapper: portmapper.NewClient(logger.WithPrefix(log.Printf, "portmap: ")),
}
if netcheckArgs.verbose {
c.Logf = logger.WithPrefix(log.Printf, "netcheck: ")

@ -29,6 +29,7 @@ tailscale.com/cmd/tailscale dependencies: (generated by github.com/tailscale/dep
tailscale.com/net/netcheck from tailscale.com/cmd/tailscale/cli
tailscale.com/net/netns from tailscale.com/derp/derphttp+
tailscale.com/net/packet from tailscale.com/wgengine/filter
tailscale.com/net/portmapper from tailscale.com/net/netcheck+
tailscale.com/net/stun from tailscale.com/net/netcheck
tailscale.com/net/tlsdial from tailscale.com/derp/derphttp
tailscale.com/net/tsaddr from tailscale.com/net/interfaces

@ -94,6 +94,7 @@ tailscale.com/cmd/tailscaled dependencies: (generated by github.com/tailscale/de
tailscale.com/net/netns from tailscale.com/control/controlclient+
💣 tailscale.com/net/netstat from tailscale.com/ipn/ipnserver
tailscale.com/net/packet from tailscale.com/wgengine+
tailscale.com/net/portmapper from tailscale.com/net/netcheck+
tailscale.com/net/stun from tailscale.com/net/netcheck+
tailscale.com/net/tlsdial from tailscale.com/control/controlclient+
tailscale.com/net/tsaddr from tailscale.com/ipn/ipnlocal+
@ -113,6 +114,7 @@ tailscale.com/cmd/tailscaled dependencies: (generated by github.com/tailscale/de
tailscale.com/types/netmap from tailscale.com/control/controlclient+
tailscale.com/types/nettype from tailscale.com/wgengine/magicsock
tailscale.com/types/opt from tailscale.com/control/controlclient+
tailscale.com/types/pad32 from tailscale.com/wgengine/magicsock
tailscale.com/types/persist from tailscale.com/control/controlclient+
tailscale.com/types/preftype from tailscale.com/ipn+
tailscale.com/types/strbuilder from tailscale.com/net/packet

@ -8,9 +8,7 @@ package netcheck
import (
"bufio"
"context"
"crypto/rand"
"crypto/tls"
"encoding/binary"
"errors"
"fmt"
"io"
@ -25,11 +23,11 @@ import (
"time"
"github.com/tcnksm/go-httpstat"
"go4.org/mem"
"inet.af/netaddr"
"tailscale.com/derp/derphttp"
"tailscale.com/net/interfaces"
"tailscale.com/net/netns"
"tailscale.com/net/portmapper"
"tailscale.com/net/stun"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
@ -68,12 +66,6 @@ const (
// more aggressive than defaultActiveRetransmitTime. A few extra
// packets at startup is fine.
defaultInitialRetransmitTime = 100 * time.Millisecond
// portMapServiceProbeTimeout is the time we wait for port mapping
// services (UPnP, NAT-PMP, PCP) to respond before we give up and
// decide that they're not there. Since these services are on the
// same LAN as this machine and a single L3 hop away, we don't
// give them much time to respond.
portMapServiceProbeTimeout = 100 * time.Millisecond
)
type Report struct {
@ -160,6 +152,10 @@ type Client struct {
// It defaults to ":0".
UDPBindAddr string
// PortMapper, if non-nil, is used for portmap queries.
// If nil, portmap discovery is not done.
PortMapper *portmapper.Client // lazily initialized on first use
mu sync.Mutex // guards following
nextFull bool // do a full region scan, even if last != nil
prev map[time.Time]*Report // some previous reports
@ -219,8 +215,8 @@ func (c *Client) handleHairSTUNLocked(pkt []byte, src netaddr.IPPort) bool {
// (non-incremental) probe of all DERP regions.
func (c *Client) MakeNextReportFull() {
c.mu.Lock()
defer c.mu.Unlock()
c.nextFull = true
c.mu.Unlock()
}
func (c *Client) ReceiveSTUNPacket(pkt []byte, src netaddr.IPPort) {
@ -680,104 +676,20 @@ func (rs *reportState) setOptBool(b *opt.Bool, v bool) {
func (rs *reportState) probePortMapServices() {
defer rs.waitPortMap.Done()
gw, myIP, ok := interfaces.LikelyHomeRouterIP()
if !ok {
return
}
rs.setOptBool(&rs.report.UPnP, false)
rs.setOptBool(&rs.report.PMP, false)
rs.setOptBool(&rs.report.PCP, false)
port1900 := netaddr.IPPort{IP: gw, Port: 1900}.UDPAddr()
port5351 := netaddr.IPPort{IP: gw, Port: 5351}.UDPAddr()
rs.c.logf("[v1] probePortMapServices: me %v -> gw %v", myIP, gw)
// Create a UDP4 socket used just for querying for UPnP, NAT-PMP, and PCP.
uc, err := netns.Listener().ListenPacket(context.Background(), "udp4", ":0")
res, err := rs.c.PortMapper.Probe(context.Background())
if err != nil {
rs.c.logf("probePortMapServices: %v", err)
return
}
defer uc.Close()
tempPort := uc.LocalAddr().(*net.UDPAddr).Port
uc.SetReadDeadline(time.Now().Add(portMapServiceProbeTimeout))
// Send request packets for all three protocols.
uc.WriteTo(uPnPPacket, port1900)
uc.WriteTo(pmpPacket, port5351)
uc.WriteTo(pcpPacket(myIP, tempPort, false), port5351)
res := make([]byte, 1500)
sentPCPDelete := false
for {
n, addr, err := uc.ReadFrom(res)
if err != nil {
return
}
switch addr.(*net.UDPAddr).Port {
case 1900:
if mem.Contains(mem.B(res[:n]), mem.S(":InternetGatewayDevice:")) {
rs.setOptBool(&rs.report.UPnP, true)
}
case 5351:
if n == 12 && res[0] == 0x00 { // right length and version 0
rs.setOptBool(&rs.report.PMP, true)
}
if n == 60 && res[0] == 0x02 { // right length and version 2
rs.setOptBool(&rs.report.PCP, true)
if !sentPCPDelete {
sentPCPDelete = true
// And now delete the mapping.
// (PCP is the only protocol of the three that requires
// we cause a side effect to detect whether it's present,
// so we need to redo that side effect now.)
uc.WriteTo(pcpPacket(myIP, tempPort, true), port5351)
}
}
}
}
}
var pmpPacket = []byte{0, 0} // version 0, opcode 0 = "Public address request"
var uPnPPacket = []byte("M-SEARCH * HTTP/1.1\r\n" +
"HOST: 239.255.255.250:1900\r\n" +
"ST: ssdp:all\r\n" +
"MAN: \"ssdp:discover\"\r\n" +
"MX: 2\r\n\r\n")
var v4unspec, _ = netaddr.ParseIP("0.0.0.0")
// pcpPacket generates a PCP packet with a MAP opcode.
func pcpPacket(myIP netaddr.IP, mapToLocalPort int, delete bool) []byte {
const udpProtoNumber = 17
lifetimeSeconds := uint32(1)
if delete {
lifetimeSeconds = 0
}
const opMap = 1
// 24 byte header + 36 byte map opcode
pkt := make([]byte, (32+32+128)/8+(96+8+24+16+16+128)/8)
// The header (https://tools.ietf.org/html/rfc6887#section-7.1)
pkt[0] = 2 // version
pkt[1] = opMap
binary.BigEndian.PutUint32(pkt[4:8], lifetimeSeconds)
myIP16 := myIP.As16()
copy(pkt[8:], myIP16[:])
// The map opcode body (https://tools.ietf.org/html/rfc6887#section-11.1)
mapOp := pkt[24:]
rand.Read(mapOp[:12]) // 96 bit mappping nonce
mapOp[12] = udpProtoNumber
binary.BigEndian.PutUint16(mapOp[16:], uint16(mapToLocalPort))
v4unspec16 := v4unspec.As16()
copy(mapOp[20:], v4unspec16[:])
return pkt
rs.setOptBool(&rs.report.UPnP, res.UPnP)
rs.setOptBool(&rs.report.PMP, res.PMP)
rs.setOptBool(&rs.report.PCP, res.PCP)
}
func newReport() *Report {
@ -854,7 +766,7 @@ func (c *Client) GetReport(ctx context.Context, dm *tailcfg.DERPMap) (*Report, e
}
defer rs.pc4Hair.Close()
if !c.SkipExternalNetwork {
if !c.SkipExternalNetwork && c.PortMapper != nil {
rs.waitPortMap.Add(1)
go rs.probePortMapServices()
}
@ -927,7 +839,7 @@ func (c *Client) GetReport(ctx context.Context, dm *tailcfg.DERPMap) (*Report, e
rs.waitHairCheck(ctx)
c.vlogf("hairCheck done")
if !c.SkipExternalNetwork {
if !c.SkipExternalNetwork && c.PortMapper != nil {
rs.waitPortMap.Wait()
c.vlogf("portMap done")
}

@ -0,0 +1,611 @@
// Copyright (c) 2021 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 portmapper is a UDP port mapping client. It currently only does
// NAT-PMP, but will likely do UPnP and perhaps PCP later.
package portmapper
import (
"context"
"crypto/rand"
"encoding/binary"
"errors"
"fmt"
"io"
"net"
"sync"
"time"
"go4.org/mem"
"inet.af/netaddr"
"tailscale.com/net/interfaces"
"tailscale.com/net/netns"
"tailscale.com/types/logger"
)
// References:
//
// NAT-PMP: https://tools.ietf.org/html/rfc6886
// PCP: https://tools.ietf.org/html/rfc6887
// portMapServiceTimeout is the time we wait for port mapping
// services (UPnP, NAT-PMP, PCP) to respond before we give up and
// decide that they're not there. Since these services are on the
// same LAN as this machine and a single L3 hop away, we don't
// give them much time to respond.
const portMapServiceTimeout = 250 * time.Millisecond
// trustServiceStillAvailableDuration is how often we re-verify a port
// mapping service is available.
const trustServiceStillAvailableDuration = 10 * time.Minute
// Client is a port mapping client.
type Client struct {
logf logger.Logf
mu sync.Mutex // guards following, and all fields thereof
lastMyIP netaddr.IP
lastGW netaddr.IP
closed bool
lastProbe time.Time
pmpPubIP netaddr.IP // non-zero if known
pmpPubIPTime time.Time // time pmpPubIP last verified
pmpLastEpoch uint32
pcpSawTime time.Time // time we last saw PCP was available
uPnPSawTime time.Time // time we last saw UPnP was available
localPort uint16
pmpMapping *pmpMapping // non-nil if we have a PMP mapping
}
// pmpMapping is an already-created PMP mapping.
//
// All fields are immutable once created.
type pmpMapping struct {
gw netaddr.IP
external netaddr.IPPort
internal netaddr.IPPort
useUntil time.Time // the mapping's lifetime minus renewal interval
epoch uint32
}
// externalValid reports whether m.external is valid, with both its IP and Port populated.
func (m *pmpMapping) externalValid() bool {
return !m.external.IP.IsZero() && m.external.Port != 0
}
// release does a best effort fire-and-forget release of the PMP mapping m.
func (m *pmpMapping) release() {
uc, err := netns.Listener().ListenPacket(context.Background(), "udp4", ":0")
if err != nil {
return
}
defer uc.Close()
pkt := buildPMPRequestMappingPacket(m.internal.Port, m.external.Port, pmpMapLifetimeDelete)
uc.WriteTo(pkt, netaddr.IPPort{IP: m.gw, Port: pmpPort}.UDPAddr())
}
// NewClient returns a new portmapping client.
func NewClient(logf logger.Logf) *Client {
return &Client{
logf: logf,
}
}
// NoteNetworkDown should be called when the network has transitioned to a down state.
// It's too late to release port mappings at this point (the user might've just turned off
// their wifi), but we can make sure we invalidate mappings for later when the network
// comes back.
func (c *Client) NoteNetworkDown() {
c.mu.Lock()
defer c.mu.Unlock()
c.invalidateMappingsLocked(false)
}
func (c *Client) Close() error {
c.mu.Lock()
defer c.mu.Unlock()
if c.closed {
return nil
}
c.closed = true
c.invalidateMappingsLocked(true)
// TODO: close some future ever-listening UDP socket(s),
// waiting for multicast announcements from router.
return nil
}
// SetLocalPort updates the local port number to which we want to port
// map UDP traffic.
func (c *Client) SetLocalPort(localPort uint16) {
c.mu.Lock()
defer c.mu.Unlock()
if c.localPort == localPort {
return
}
c.localPort = localPort
c.invalidateMappingsLocked(true)
}
func (c *Client) gatewayAndSelfIP() (gw, myIP netaddr.IP, ok bool) {
gw, myIP, ok = interfaces.LikelyHomeRouterIP()
if !ok {
gw = netaddr.IP{}
myIP = netaddr.IP{}
}
c.mu.Lock()
defer c.mu.Unlock()
if gw != c.lastGW || myIP != c.lastMyIP || !ok {
c.lastMyIP = myIP
c.lastGW = gw
c.invalidateMappingsLocked(true)
}
return
}
func (c *Client) invalidateMappingsLocked(releaseOld bool) {
if c.pmpMapping != nil {
if releaseOld {
c.pmpMapping.release()
}
c.pmpMapping = nil
}
c.pmpPubIP = netaddr.IP{}
c.pmpPubIPTime = time.Time{}
c.pcpSawTime = time.Time{}
c.uPnPSawTime = time.Time{}
}
func (c *Client) sawPMPRecently() bool {
c.mu.Lock()
defer c.mu.Unlock()
return c.sawPMPRecentlyLocked()
}
func (c *Client) sawPMPRecentlyLocked() bool {
return !c.pmpPubIP.IsZero() && c.pmpPubIPTime.After(time.Now().Add(-trustServiceStillAvailableDuration))
}
func (c *Client) sawPCPRecently() bool {
c.mu.Lock()
defer c.mu.Unlock()
return c.pcpSawTime.After(time.Now().Add(-trustServiceStillAvailableDuration))
}
func (c *Client) sawUPnPRecently() bool {
c.mu.Lock()
defer c.mu.Unlock()
return c.uPnPSawTime.After(time.Now().Add(-trustServiceStillAvailableDuration))
}
// closeCloserOnContextDone starts a new goroutine to call c.Close
// if/when ctx becomes done.
// To stop the goroutine, call the returned stop func.
func closeCloserOnContextDone(ctx context.Context, c io.Closer) (stop func()) {
// Close uc on ctx being done.
ctxDone := ctx.Done()
if ctxDone == nil {
return func() {}
}
stopWaitDone := make(chan struct{})
go func() {
select {
case <-stopWaitDone:
case <-ctxDone:
c.Close()
}
}()
return func() { close(stopWaitDone) }
}
// NoMappingError is returned by CreateOrGetMapping when no NAT
// mapping could be returned.
type NoMappingError struct {
err error
}
func (nme NoMappingError) Unwrap() error { return nme.err }
func (nme NoMappingError) Error() string { return fmt.Sprintf("no NAT mapping available: %v", nme.err) }
// IsNoMappingError reports whether err is of type NoMappingError.
func IsNoMappingError(err error) bool {
_, ok := err.(NoMappingError)
return ok
}
var (
ErrNoPortMappingServices = errors.New("no port mapping services were found")
ErrGatewayNotFound = errors.New("failed to look gateway address")
)
// CreateOrGetMapping either creates a new mapping or returns a cached
// valid one.
//
// If no mapping is available, the error will be of type
// NoMappingError; see IsNoMappingError.
func (c *Client) CreateOrGetMapping(ctx context.Context) (external netaddr.IPPort, err error) {
gw, myIP, ok := c.gatewayAndSelfIP()
if !ok {
return netaddr.IPPort{}, NoMappingError{ErrGatewayNotFound}
}
c.mu.Lock()
localPort := c.localPort
m := &pmpMapping{
gw: gw,
internal: netaddr.IPPort{IP: myIP, Port: localPort},
}
// prevPort is the port we had most previously, if any. We try
// to ask for the same port. 0 means to give us any port.
var prevPort uint16
// Do we have an existing mapping that's valid?
now := time.Now()
if m := c.pmpMapping; m != nil {
if now.Before(m.useUntil) {
defer c.mu.Unlock()
return m.external, nil
}
// The mapping might still be valid, so just try to renew it.
prevPort = m.external.Port
}
// If we just did a Probe (e.g. via netchecker) but didn't
// find a PMP service, bail out early rather than probing
// again. Cuts down latency for most clients.
haveRecentPMP := c.sawPMPRecentlyLocked()
if haveRecentPMP {
m.external.IP = c.pmpPubIP
}
if c.lastProbe.After(now.Add(-5*time.Second)) && !haveRecentPMP {
c.mu.Unlock()
return netaddr.IPPort{}, NoMappingError{ErrNoPortMappingServices}
}
c.mu.Unlock()
uc, err := netns.Listener().ListenPacket(ctx, "udp4", ":0")
if err != nil {
return netaddr.IPPort{}, err
}
defer uc.Close()
uc.SetReadDeadline(time.Now().Add(portMapServiceTimeout))
defer closeCloserOnContextDone(ctx, uc)()
pmpAddr := netaddr.IPPort{IP: gw, Port: pmpPort}
pmpAddru := pmpAddr.UDPAddr()
// Ask for our external address if needed.
if m.external.IP.IsZero() {
if _, err := uc.WriteTo(pmpReqExternalAddrPacket, pmpAddru); err != nil {
return netaddr.IPPort{}, err
}
}
// And ask for a mapping.
pmpReqMapping := buildPMPRequestMappingPacket(localPort, prevPort, pmpMapLifetimeSec)
if _, err := uc.WriteTo(pmpReqMapping, pmpAddru); err != nil {
return netaddr.IPPort{}, err
}
res := make([]byte, 1500)
for {
n, srci, err := uc.ReadFrom(res)
if err != nil {
if ctx.Err() == context.Canceled {
return netaddr.IPPort{}, err
}
return netaddr.IPPort{}, NoMappingError{ErrNoPortMappingServices}
}
srcu := srci.(*net.UDPAddr)
src, ok := netaddr.FromStdAddr(srcu.IP, srcu.Port, srcu.Zone)
if !ok {
continue
}
if src == pmpAddr {
pres, ok := parsePMPResponse(res[:n])
if !ok {
c.logf("unexpected PMP response: % 02x", res[:n])
continue
}
if pres.ResultCode != 0 {
return netaddr.IPPort{}, NoMappingError{fmt.Errorf("PMP response Op=0x%x,Res=0x%x", pres.OpCode, pres.ResultCode)}
}
if pres.OpCode == pmpOpReply|pmpOpMapPublicAddr {
m.external.IP = pres.PublicAddr
}
if pres.OpCode == pmpOpReply|pmpOpMapUDP {
m.external.Port = pres.ExternalPort
d := time.Duration(pres.MappingValidSeconds) * time.Second
d /= 2 // renew in half the time
m.useUntil = time.Now().Add(d)
m.epoch = pres.SecondsSinceEpoch
}
}
if m.externalValid() {
c.mu.Lock()
defer c.mu.Unlock()
c.pmpMapping = m
return m.external, nil
}
}
}
type pmpResultCode uint16
// NAT-PMP constants.
const (
pmpPort = 5351
pmpMapLifetimeSec = 7200 // RFC recommended 2 hour map duration
pmpMapLifetimeDelete = 0 // 0 second lifetime deletes
pmpOpMapPublicAddr = 0
pmpOpMapUDP = 1
pmpOpReply = 0x80 // OR'd into request's op code on response
pmpCodeOK pmpResultCode = 0
pmpCodeUnsupportedVersion pmpResultCode = 1
pmpCodeNotAuthorized pmpResultCode = 2 // "e.g., box supports mapping, but user has turned feature off"
pmpCodeNetworkFailure pmpResultCode = 3 // "e.g., NAT box itself has not obtained a DHCP lease"
pmpCodeOutOfResources pmpResultCode = 4
pmpCodeUnsupportedOpcode pmpResultCode = 5
)
func buildPMPRequestMappingPacket(localPort, prevPort uint16, lifetimeSec uint32) (pkt []byte) {
pkt = make([]byte, 12)
pkt[1] = pmpOpMapUDP
binary.BigEndian.PutUint16(pkt[4:], localPort)
binary.BigEndian.PutUint16(pkt[6:], prevPort)
binary.BigEndian.PutUint32(pkt[8:], lifetimeSec)
return pkt
}
type pmpResponse struct {
OpCode uint8
ResultCode pmpResultCode
SecondsSinceEpoch uint32
// For Map ops:
MappingValidSeconds uint32
InternalPort uint16
ExternalPort uint16
// For public addr ops:
PublicAddr netaddr.IP
}
func parsePMPResponse(pkt []byte) (res pmpResponse, ok bool) {
if len(pkt) < 12 {
return
}
ver := pkt[0]
if ver != 0 {
return
}
res.OpCode = pkt[1]
res.ResultCode = pmpResultCode(binary.BigEndian.Uint16(pkt[2:]))
res.SecondsSinceEpoch = binary.BigEndian.Uint32(pkt[4:])
if res.OpCode == pmpOpReply|pmpOpMapUDP {
if len(pkt) != 16 {
return res, false
}
res.InternalPort = binary.BigEndian.Uint16(pkt[8:])
res.ExternalPort = binary.BigEndian.Uint16(pkt[10:])
res.MappingValidSeconds = binary.BigEndian.Uint32(pkt[12:])
}
if res.OpCode == pmpOpReply|pmpOpMapPublicAddr {
if len(pkt) != 12 {
return res, false
}
res.PublicAddr = netaddr.IPv4(pkt[8], pkt[9], pkt[10], pkt[11])
}
return res, true
}
type ProbeResult struct {
PCP bool
PMP bool
UPnP bool
}
// Probe returns a summary of which port mapping services are
// available on the network.
//
// If a probe has run recently and there haven't been any network changes since,
// the returned result might be server from the Client's cache, without
// sending any network traffic.
func (c *Client) Probe(ctx context.Context) (res ProbeResult, err error) {
gw, myIP, ok := c.gatewayAndSelfIP()
if !ok {
return res, ErrGatewayNotFound
}
defer func() {
if err == nil {
c.mu.Lock()
defer c.mu.Unlock()
c.lastProbe = time.Now()
}
}()
uc, err := netns.Listener().ListenPacket(context.Background(), "udp4", ":0")
if err != nil {
c.logf("ProbePCP: %v", err)
return res, err
}
defer uc.Close()
ctx, cancel := context.WithTimeout(ctx, 250*time.Millisecond)
defer cancel()
defer closeCloserOnContextDone(ctx, uc)()
pcpAddr := netaddr.IPPort{IP: gw, Port: pcpPort}.UDPAddr()
pmpAddr := netaddr.IPPort{IP: gw, Port: pmpPort}.UDPAddr()
upnpAddr := netaddr.IPPort{IP: gw, Port: upnpPort}.UDPAddr()
// Don't send probes to services that we recently learned (for
// the same gw/myIP) are available. See
// https://github.com/tailscale/tailscale/issues/1001
if c.sawPMPRecently() {
res.PMP = true
} else {
uc.WriteTo(pmpReqExternalAddrPacket, pmpAddr)
}
if c.sawPCPRecently() {
res.PCP = true
} else {
uc.WriteTo(pcpAnnounceRequest(myIP), pcpAddr)
}
if c.sawUPnPRecently() {
res.UPnP = true
} else {
uc.WriteTo(uPnPPacket, upnpAddr)
}
buf := make([]byte, 1500)
for {
if res.PCP && res.PMP && res.UPnP {
// Nothing more to discover.
return res, nil
}
n, addr, err := uc.ReadFrom(buf)
if err != nil {
if ctx.Err() == context.DeadlineExceeded {
err = nil
}
return res, err
}
port := addr.(*net.UDPAddr).Port
switch port {
case upnpPort:
if mem.Contains(mem.B(buf[:n]), mem.S(":InternetGatewayDevice:")) {
res.UPnP = true
c.mu.Lock()
c.uPnPSawTime = time.Now()
c.mu.Unlock()
}
case pcpPort: // same as pmpPort
if pres, ok := parsePCPResponse(buf[:n]); ok {
if pres.OpCode == pcpOpReply|pcpOpAnnounce && pres.ResultCode == pcpCodeOK {
c.logf("Got PCP response: epoch: %v", pres.Epoch)
res.PCP = true
c.mu.Lock()
c.pcpSawTime = time.Now()
c.mu.Unlock()
continue
}
c.logf("unexpected PCP probe response: %+v", pres)
}
if pres, ok := parsePMPResponse(buf[:n]); ok {
if pres.OpCode == pmpOpReply|pmpOpMapPublicAddr && pres.ResultCode == pmpCodeOK {
c.logf("Got PMP response; IP: %v, epoch: %v", pres.PublicAddr, pres.SecondsSinceEpoch)
res.PMP = true
c.mu.Lock()
c.pmpPubIP = pres.PublicAddr
c.pmpPubIPTime = time.Now()
c.pmpLastEpoch = pres.SecondsSinceEpoch
c.mu.Unlock()
continue
}
c.logf("unexpected PMP probe response: %+v", pres)
}
}
}
}
const (
pcpVersion = 2
pcpPort = 5351
pcpCodeOK = 0
pcpOpReply = 0x80 // OR'd into request's op code on response
pcpOpAnnounce = 0
pcpOpMap = 1
)
// pcpAnnounceRequest generates a PCP packet with an ANNOUNCE opcode.
func pcpAnnounceRequest(myIP netaddr.IP) []byte {
// See https://tools.ietf.org/html/rfc6887#section-7.1
pkt := make([]byte, 24)
pkt[0] = pcpVersion // version
pkt[1] = pcpOpAnnounce
myIP16 := myIP.As16()
copy(pkt[8:], myIP16[:])
return pkt
}
//lint:ignore U1000 moved this code from netcheck's old PCP probing; will be needed when we add PCP mapping
// pcpMapRequest generates a PCP packet with a MAP opcode.
func pcpMapRequest(myIP netaddr.IP, mapToLocalPort int, delete bool) []byte {
const udpProtoNumber = 17
lifetimeSeconds := uint32(1)
if delete {
lifetimeSeconds = 0
}
const opMap = 1
// 24 byte header + 36 byte map opcode
pkt := make([]byte, (32+32+128)/8+(96+8+24+16+16+128)/8)
// The header (https://tools.ietf.org/html/rfc6887#section-7.1)
pkt[0] = 2 // version
pkt[1] = opMap
binary.BigEndian.PutUint32(pkt[4:8], lifetimeSeconds)
myIP16 := myIP.As16()
copy(pkt[8:], myIP16[:])
// The map opcode body (https://tools.ietf.org/html/rfc6887#section-11.1)
mapOp := pkt[24:]
rand.Read(mapOp[:12]) // 96 bit mappping nonce
mapOp[12] = udpProtoNumber
binary.BigEndian.PutUint16(mapOp[16:], uint16(mapToLocalPort))
v4unspec := netaddr.MustParseIP("0.0.0.0")
v4unspec16 := v4unspec.As16()
copy(mapOp[20:], v4unspec16[:])
return pkt
}
type pcpResponse struct {
OpCode uint8
ResultCode uint8
Lifetime uint32
Epoch uint32
}
func parsePCPResponse(b []byte) (res pcpResponse, ok bool) {
if len(b) < 24 || b[0] != pcpVersion {
return
}
res.OpCode = b[1]
res.ResultCode = b[3]
res.Lifetime = binary.BigEndian.Uint32(b[4:])
res.Epoch = binary.BigEndian.Uint32(b[8:])
return res, true
}
const (
upnpPort = 1900
)
var uPnPPacket = []byte("M-SEARCH * HTTP/1.1\r\n" +
"HOST: 239.255.255.250:1900\r\n" +
"ST: ssdp:all\r\n" +
"MAN: \"ssdp:discover\"\r\n" +
"MX: 2\r\n\r\n")
var pmpReqExternalAddrPacket = []byte{0, 0} // version 0, opcode 0 = "Public address request"

@ -0,0 +1,54 @@
// Copyright (c) 2021 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 portmapper
import (
"context"
"os"
"strconv"
"testing"
"time"
)
func TestCreateOrGetMapping(t *testing.T) {
if v, _ := strconv.ParseBool(os.Getenv("HIT_NETWORK")); !v {
t.Skip("skipping test without HIT_NETWORK=1")
}
c := NewClient(t.Logf)
c.SetLocalPort(1234)
for i := 0; i < 2; i++ {
if i > 0 {
time.Sleep(100 * time.Millisecond)
}
ext, err := c.CreateOrGetMapping(context.Background())
t.Logf("Got: %v, %v", ext, err)
}
}
func TestClientProbe(t *testing.T) {
if v, _ := strconv.ParseBool(os.Getenv("HIT_NETWORK")); !v {
t.Skip("skipping test without HIT_NETWORK=1")
}
c := NewClient(t.Logf)
for i := 0; i < 2; i++ {
if i > 0 {
time.Sleep(100 * time.Millisecond)
}
res, err := c.Probe(context.Background())
t.Logf("Got: %+v, %v", res, err)
}
}
func TestClientProbeThenMap(t *testing.T) {
if v, _ := strconv.ParseBool(os.Getenv("HIT_NETWORK")); !v {
t.Skip("skipping test without HIT_NETWORK=1")
}
c := NewClient(t.Logf)
c.SetLocalPort(1234)
res, err := c.Probe(context.Background())
t.Logf("Probe: %+v, %v", res, err)
ext, err := c.CreateOrGetMapping(context.Background())
t.Logf("CreateOrGetMapping: %v, %v", ext, err)
}

@ -0,0 +1,9 @@
// Copyright (c) 2021 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 pad32 defines padding types that have width on only 32-bit platforms.
package pad32
// Four is 4 bytes of padding on 32-bit machines, else 0 bytes.
type Four [4 * (1 - ((^uint(0))>>32)&1)]byte

@ -43,6 +43,7 @@ import (
"tailscale.com/net/interfaces"
"tailscale.com/net/netcheck"
"tailscale.com/net/netns"
"tailscale.com/net/portmapper"
"tailscale.com/net/stun"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
@ -51,6 +52,7 @@ import (
"tailscale.com/types/logger"
"tailscale.com/types/netmap"
"tailscale.com/types/nettype"
"tailscale.com/types/pad32"
"tailscale.com/types/wgkey"
"tailscale.com/version"
"tailscale.com/wgengine/wgcfg"
@ -142,6 +144,10 @@ type Conn struct {
// conditions, including the closest DERP relay and NAT mappings.
netChecker *netcheck.Client
// portMapper is the NAT-PMP/PCP/UPnP prober/client, for requesting
// port mappings from NAT devices.
portMapper *portmapper.Client
// sendLogLimit is a rate limiter for errors logged in the (hot)
// packet sending codepath. It's so that, if magicsock gets into a
// bad state, we don't spam one error per wireguard packet being
@ -156,6 +162,7 @@ type Conn struct {
// derpRecvCh is used by ReceiveIPv4 to read DERP messages.
derpRecvCh chan derpReadResult
_ pad32.Four
// derpRecvCountAtomic is how many derpRecvCh sends are pending.
// It's incremented by runDerpReader whenever a DERP message
// arrives and decremented when they're read.
@ -475,6 +482,7 @@ func NewConn(opts Options) (*Conn, error) {
c.noteRecvActivity = opts.NoteRecvActivity
c.simulatedNetwork = opts.SimulatedNetwork
c.disableLegacy = opts.DisableLegacyNetworking
c.portMapper = portmapper.NewClient(logger.WithPrefix(c.logf, "portmapper: "))
if err := c.initialBind(); err != nil {
return nil, err
@ -486,7 +494,9 @@ func NewConn(opts Options) (*Conn, error) {
Logf: logger.WithPrefix(c.logf, "netcheck: "),
GetSTUNConn4: func() netcheck.STUNConn { return c.pconn4 },
SkipExternalNetwork: inTest(),
PortMapper: c.portMapper,
}
if c.pconn6 != nil {
c.netChecker.GetSTUNConn6 = func() netcheck.STUNConn { return c.pconn6 }
}
@ -1004,6 +1014,13 @@ func (c *Conn) determineEndpoints(ctx context.Context) (ipPorts []string, reason
}
}
if ext, err := c.portMapper.CreateOrGetMapping(ctx); err == nil {
c.logf("portmapper: using %v", ext)
addAddr(ext.String(), "portmap")
} else if !portmapper.IsNoMappingError(err) {
c.logf("portmapper: %v", err)
}
if nr.GlobalV4 != "" {
addAddr(nr.GlobalV4, "stun")
@ -1073,6 +1090,7 @@ func stringsEqual(x, y []string) bool {
return true
}
// LocalPort returns the current IPv4 listener's port number.
func (c *Conn) LocalPort() uint16 {
laddr := c.pconn4.LocalAddr()
return uint16(laddr.Port)
@ -2126,6 +2144,7 @@ func (c *Conn) SetNetworkUp(up bool) {
if up {
c.startDerpHomeConnectLocked()
} else {
c.portMapper.NoteNetworkDown()
c.closeAllDerpLocked("network-down")
}
}
@ -2432,6 +2451,7 @@ func (c *Conn) Close() error {
c.derpCleanupTimer.Stop()
}
c.stopPeriodicReSTUNTimerLocked()
c.portMapper.Close()
for _, ep := range c.endpointOfDisco {
ep.stopAndReset()
@ -2553,6 +2573,7 @@ func (c *Conn) initialBind() error {
if err := c.bind1(&c.pconn4, "udp4"); err != nil {
return err
}
c.portMapper.SetLocalPort(c.LocalPort())
if err := c.bind1(&c.pconn6, "udp6"); err != nil {
c.logf("magicsock: ignoring IPv6 bind failure: %v", err)
}
@ -2627,6 +2648,7 @@ func (c *Conn) Rebind() {
return
}
c.pconn4.Reset(packetConn.(*net.UDPConn))
c.portMapper.SetLocalPort(c.LocalPort())
c.mu.Lock()
c.closeAllDerpLocked("rebind")

@ -1384,17 +1384,21 @@ func stringifyConfig(cfg wgcfg.Config) string {
func Test32bitAlignment(t *testing.T) {
var de discoEndpoint
off := unsafe.Offsetof(de.lastRecvUnixAtomic)
if off%8 != 0 {
t.Fatalf("lastRecvUnixAtomic is not 8-byte aligned")
var c Conn
if off := unsafe.Offsetof(de.lastRecvUnixAtomic); off%8 != 0 {
t.Fatalf("discoEndpoint.lastRecvUnixAtomic is not 8-byte aligned")
}
if off := unsafe.Offsetof(c.derpRecvCountAtomic); off%8 != 0 {
t.Fatalf("Conn.derpRecvCountAtomic is not 8-byte aligned")
}
if !de.isFirstRecvActivityInAwhile() { // verify this doesn't panic on 32-bit
t.Error("expected true")
}
if de.isFirstRecvActivityInAwhile() {
t.Error("expected false on second call")
}
var c Conn
atomic.AddInt64(&c.derpRecvCountAtomic, 1)
}

Loading…
Cancel
Save