wgengine/magicsock: use NodeKey, not DiscoKey, as the trigger for lazy reconfig.

Updates #2752

Signed-off-by: David Anderson <danderson@tailscale.com>
pull/2773/head
David Anderson 3 years ago committed by Dave Anderson
parent 4c27e2fa22
commit dfd978f0f2

@ -210,7 +210,7 @@ type Conn struct {
derpActiveFunc func() derpActiveFunc func()
idleFunc func() time.Duration // nil means unknown idleFunc func() time.Duration // nil means unknown
testOnlyPacketListener nettype.PacketListener testOnlyPacketListener nettype.PacketListener
noteRecvActivity func(tailcfg.DiscoKey) // or nil, see Options.NoteRecvActivity noteRecvActivity func(tailcfg.NodeKey) // or nil, see Options.NoteRecvActivity
// ================================================================ // ================================================================
// No locking required to access these fields, either because // No locking required to access these fields, either because
@ -449,18 +449,17 @@ type Options struct {
// Only used by tests. // Only used by tests.
TestOnlyPacketListener nettype.PacketListener TestOnlyPacketListener nettype.PacketListener
// NoteRecvActivity, if provided, is a func for magicsock to // NoteRecvActivity, if provided, is a func for magicsock to call
// call whenever it receives a packet from a a // whenever it receives a packet from a a peer if it's been more
// discovery-capable peer if it's been more than ~10 seconds // than ~10 seconds since the last one. (10 seconds is somewhat
// since the last one. (10 seconds is somewhat arbitrary; the // arbitrary; the sole user just doesn't need or want it called on
// sole user just doesn't need or want it called on every // every packet, just every minute or two for Wireguard timeouts,
// packet, just every minute or two for Wireguard timeouts, // and 10 seconds seems like a good trade-off between often enough
// and 10 seconds seems like a good trade-off between often // and not too often.)
// enough and not too often.) The provided func is called // The provided func is likely to call back into
// while holding userspaceEngine.wgLock and likely calls // Conn.ParseEndpoint, which acquires Conn.mu. As such, you should
// Conn.ParseEndpoint, which acquires Conn.mu. As such, you // not hold Conn.mu while calling it.
// should not hold Conn.mu while calling it. NoteRecvActivity func(tailcfg.NodeKey)
NoteRecvActivity func(tailcfg.DiscoKey)
// LinkMonitor is the link monitor to use. // LinkMonitor is the link monitor to use.
// With one, the portmapper won't be used. // With one, the portmapper won't be used.
@ -1534,18 +1533,6 @@ func (c *Conn) runDerpWriter(ctx context.Context, dc *derphttp.Client, ch <-chan
} }
} }
// noteRecvActivityFromEndpoint calls the c.noteRecvActivity hook if
// e is a discovery-capable peer and this is the first receive activity
// it's got in awhile (in last 10 seconds).
//
// This should be called whenever a packet arrives from e.
func (c *Conn) noteRecvActivityFromEndpoint(e conn.Endpoint) {
de, ok := e.(*endpoint)
if ok && c.noteRecvActivity != nil && de.isFirstRecvActivityInAwhile() {
c.noteRecvActivity(de.discoKey)
}
}
// receiveIPv6 receives a UDP IPv6 packet. It is called by wireguard-go. // receiveIPv6 receives a UDP IPv6 packet. It is called by wireguard-go.
func (c *Conn) receiveIPv6(b []byte) (int, conn.Endpoint, error) { func (c *Conn) receiveIPv6(b []byte) (int, conn.Endpoint, error) {
health.ReceiveIPv6.Enter() health.ReceiveIPv6.Enter()
@ -1580,7 +1567,7 @@ func (c *Conn) receiveIPv4(b []byte) (n int, ep conn.Endpoint, err error) {
// //
// ok is whether this read should be reported up to wireguard-go (our // ok is whether this read should be reported up to wireguard-go (our
// caller). // caller).
func (c *Conn) receiveIP(b []byte, ipp netaddr.IPPort, cache *ippEndpointCache) (ep conn.Endpoint, ok bool) { func (c *Conn) receiveIP(b []byte, ipp netaddr.IPPort, cache *ippEndpointCache) (ep *endpoint, ok bool) {
if stun.Is(b) { if stun.Is(b) {
c.stunReceiveFunc.Load().(func([]byte, netaddr.IPPort))(b, ipp) c.stunReceiveFunc.Load().(func([]byte, netaddr.IPPort))(b, ipp)
return nil, false return nil, false
@ -1608,7 +1595,7 @@ func (c *Conn) receiveIP(b []byte, ipp netaddr.IPPort, cache *ippEndpointCache)
cache.gen = de.numStopAndReset() cache.gen = de.numStopAndReset()
ep = de ep = de
} }
c.noteRecvActivityFromEndpoint(ep) ep.noteRecvActivity()
return ep, true return ep, true
} }
@ -1662,7 +1649,7 @@ func (c *Conn) processDERPReadResult(dm derpReadResult, b []byte) (n int, ep *en
return 0, nil return 0, nil
} }
c.noteRecvActivityFromEndpoint(ep) ep.noteRecvActivity()
return n, ep return n, ep
} }
@ -3131,17 +3118,18 @@ func (de *endpoint) initFakeUDPAddr() {
de.fakeWGAddr = netaddr.IPPortFrom(netaddr.IPFrom16(addr), 12345) de.fakeWGAddr = netaddr.IPPortFrom(netaddr.IPFrom16(addr), 12345)
} }
// isFirstRecvActivityInAwhile notes that receive activity has occurred for this // noteRecvActivity records receive activity on de, and invokes
// endpoint and reports whether it's been at least 10 seconds since the last // Conn.noteRecvActivity no more than once every 10s.
// receive activity (including having never received from this peer before). func (de *endpoint) noteRecvActivity() {
func (de *endpoint) isFirstRecvActivityInAwhile() bool { if de.c.noteRecvActivity == nil {
return
}
now := mono.Now() now := mono.Now()
elapsed := now.Sub(de.lastRecv.LoadAtomic()) elapsed := now.Sub(de.lastRecv.LoadAtomic())
if elapsed > 10*time.Second { if elapsed > 10*time.Second {
de.lastRecv.StoreAtomic(now) de.lastRecv.StoreAtomic(now)
return true de.c.noteRecvActivity(de.publicKey)
} }
return false
} }
// String exists purely so wireguard-go internals can log.Printf("%v") // String exists purely so wireguard-go internals can log.Printf("%v")

@ -1117,17 +1117,26 @@ func TestDiscoStringLogRace(t *testing.T) {
} }
func Test32bitAlignment(t *testing.T) { func Test32bitAlignment(t *testing.T) {
var de endpoint // Need an associated conn with non-nil noteRecvActivity to
// trigger interesting work on the atomics in endpoint.
called := 0
de := endpoint{
c: &Conn{
noteRecvActivity: func(tailcfg.NodeKey) { called++ },
},
}
if off := unsafe.Offsetof(de.lastRecv); off%8 != 0 { if off := unsafe.Offsetof(de.lastRecv); off%8 != 0 {
t.Fatalf("endpoint.lastRecv is not 8-byte aligned") t.Fatalf("endpoint.lastRecv is not 8-byte aligned")
} }
if !de.isFirstRecvActivityInAwhile() { // verify this doesn't panic on 32-bit de.noteRecvActivity() // verify this doesn't panic on 32-bit
t.Error("expected true") if called != 1 {
t.Fatal("expected call to noteRecvActivity")
} }
if de.isFirstRecvActivityInAwhile() { de.noteRecvActivity()
t.Error("expected false on second call") if called != 1 {
t.Error("expected no second call to noteRecvActivity")
} }
} }

@ -115,8 +115,8 @@ type userspaceEngine struct {
lastEngineSigFull deephash.Sum // of full wireguard config lastEngineSigFull deephash.Sum // of full wireguard config
lastEngineSigTrim deephash.Sum // of trimmed wireguard config lastEngineSigTrim deephash.Sum // of trimmed wireguard config
lastDNSConfig *dns.Config lastDNSConfig *dns.Config
recvActivityAt map[tailcfg.DiscoKey]mono.Time recvActivityAt map[tailcfg.NodeKey]mono.Time
trimmedDisco map[tailcfg.DiscoKey]bool // set of disco keys of peers currently excluded from wireguard config trimmedNodes map[tailcfg.NodeKey]bool // set of node keys of peers currently excluded from wireguard config
sentActivityAt map[netaddr.IP]*mono.Time // value is accessed atomically sentActivityAt map[netaddr.IP]*mono.Time // value is accessed atomically
destIPActivityFuncs map[netaddr.IP]func() destIPActivityFuncs map[netaddr.IP]func()
statusBufioReader *bufio.Reader // reusable for UAPI statusBufioReader *bufio.Reader // reusable for UAPI
@ -321,7 +321,7 @@ func NewUserspaceEngine(logf logger.Logf, conf Config) (_ Engine, reterr error)
EndpointsFunc: endpointsFn, EndpointsFunc: endpointsFn,
DERPActiveFunc: e.RequestStatus, DERPActiveFunc: e.RequestStatus,
IdleFunc: e.tundev.IdleDuration, IdleFunc: e.tundev.IdleDuration,
NoteRecvActivity: e.noteReceiveActivity, NoteRecvActivity: e.noteRecvActivity,
LinkMonitor: e.linkMon, LinkMonitor: e.linkMon,
} }
@ -533,19 +533,14 @@ func forceFullWireguardConfig(numPeers int) bool {
// isTrimmablePeer reports whether p is a peer that we can trim out of the // isTrimmablePeer reports whether p is a peer that we can trim out of the
// network map. // network map.
// //
// We can only trim peers that both a) support discovery (because we // For implementation simplificy, we can only trim peers that have
// know who they are when we receive their data and don't need to rely // only non-subnet AllowedIPs (an IPv4 /32 or IPv6 /128), which is the
// on wireguard-go figuring it out) and b) for implementation // common case for most peers. Subnet router nodes will just always be
// simplicity, have only non-subnet AllowedIPs (an IPv4 /32 or IPv6 // created in the wireguard-go config.
// /128), which is the common case for most peers. Subnet router nodes
// will just always be created in the wireguard-go config.
func isTrimmablePeer(p *wgcfg.Peer, numPeers int) bool { func isTrimmablePeer(p *wgcfg.Peer, numPeers int) bool {
if forceFullWireguardConfig(numPeers) { if forceFullWireguardConfig(numPeers) {
return false return false
} }
if p.Endpoints.DiscoKey.IsZero() {
return false
}
// AllowedIPs must all be single IPs, not subnets. // AllowedIPs must all be single IPs, not subnets.
for _, aip := range p.AllowedIPs { for _, aip := range p.AllowedIPs {
@ -556,22 +551,22 @@ func isTrimmablePeer(p *wgcfg.Peer, numPeers int) bool {
return true return true
} }
// noteReceiveActivity is called by magicsock when a packet has been received // noteRecvActivity is called by magicsock when a packet has been
// by the peer using discovery key dk. Magicsock calls this no more than // received for the peer with node key nk. Magicsock calls this no
// every 10 seconds for a given peer. // more than every 10 seconds for a given peer.
func (e *userspaceEngine) noteReceiveActivity(dk tailcfg.DiscoKey) { func (e *userspaceEngine) noteRecvActivity(nk tailcfg.NodeKey) {
e.wgLock.Lock() e.wgLock.Lock()
defer e.wgLock.Unlock() defer e.wgLock.Unlock()
if _, ok := e.recvActivityAt[dk]; !ok { if _, ok := e.recvActivityAt[nk]; !ok {
// Not a trimmable peer we care about tracking. (See isTrimmablePeer) // Not a trimmable peer we care about tracking. (See isTrimmablePeer)
if e.trimmedDisco[dk] { if e.trimmedNodes[nk] {
e.logf("wgengine: [unexpected] noteReceiveActivity called on idle discokey %v that's not in recvActivityAt", dk.ShortString()) e.logf("wgengine: [unexpected] noteReceiveActivity called on idle node %v that's not in recvActivityAt", nk.ShortString())
} }
return return
} }
now := e.timeNow() now := e.timeNow()
e.recvActivityAt[dk] = now e.recvActivityAt[nk] = now
// As long as there's activity, periodically poll the engine to get // As long as there's activity, periodically poll the engine to get
// stats for the far away side effect of // stats for the far away side effect of
@ -591,18 +586,18 @@ func (e *userspaceEngine) noteReceiveActivity(dk tailcfg.DiscoKey) {
// lazyPeerIdleThreshold without the divide by 2, but // lazyPeerIdleThreshold without the divide by 2, but
// maybeReconfigWireguardLocked is cheap enough to call every // maybeReconfigWireguardLocked is cheap enough to call every
// couple minutes (just not on every packet). // couple minutes (just not on every packet).
if e.trimmedDisco[dk] { if e.trimmedNodes[nk] {
e.logf("wgengine: idle peer %v now active, reconfiguring wireguard", dk.ShortString()) e.logf("wgengine: idle peer %v now active, reconfiguring wireguard", nk.ShortString())
e.maybeReconfigWireguardLocked(nil) e.maybeReconfigWireguardLocked(nil)
} }
} }
// isActiveSince reports whether the peer identified by (dk, ip) has // isActiveSinceLocked reports whether the peer identified by (nk, ip)
// had a packet sent to or received from it since t. // has had a packet sent to or received from it since t.
// //
// e.wgLock must be held. // e.wgLock must be held.
func (e *userspaceEngine) isActiveSince(dk tailcfg.DiscoKey, ip netaddr.IP, t mono.Time) bool { func (e *userspaceEngine) isActiveSinceLocked(nk tailcfg.NodeKey, ip netaddr.IP, t mono.Time) bool {
if e.recvActivityAt[dk].After(t) { if e.recvActivityAt[nk].After(t) {
return true return true
} }
timePtr, ok := e.sentActivityAt[ip] timePtr, ok := e.sentActivityAt[ip]
@ -618,7 +613,7 @@ func (e *userspaceEngine) isActiveSince(dk tailcfg.DiscoKey, ip netaddr.IP, t mo
// If discoChanged is nil or empty, this extra removal step isn't done. // If discoChanged is nil or empty, this extra removal step isn't done.
// //
// e.wgLock must be held. // e.wgLock must be held.
func (e *userspaceEngine) maybeReconfigWireguardLocked(discoChanged map[key.Public]bool) error { func (e *userspaceEngine) maybeReconfigWireguardLocked(discoChanged map[tailcfg.NodeKey]bool) error {
if hook := e.testMaybeReconfigHook; hook != nil { if hook := e.testMaybeReconfigHook; hook != nil {
hook() hook()
return nil return nil
@ -640,58 +635,58 @@ func (e *userspaceEngine) maybeReconfigWireguardLocked(discoChanged map[key.Publ
activeCutoff := e.timeNow().Add(-lazyPeerIdleThreshold) activeCutoff := e.timeNow().Add(-lazyPeerIdleThreshold)
// Not all peers can be trimmed from the network map (see // Not all peers can be trimmed from the network map (see
// isTrimmablePeer). For those are are trimmable, keep track // isTrimmablePeer). For those are are trimmable, keep track of
// of their DiscoKey and Tailscale IPs. These are the ones // their NodeKey and Tailscale IPs. These are the ones we'll need
// we'll need to install tracking hooks for to watch their // to install tracking hooks for to watch their send/receive
// send/receive activity. // activity.
trackDisco := make([]tailcfg.DiscoKey, 0, len(full.Peers)) trackNodes := make([]tailcfg.NodeKey, 0, len(full.Peers))
trackIPs := make([]netaddr.IP, 0, len(full.Peers)) trackIPs := make([]netaddr.IP, 0, len(full.Peers))
trimmedDisco := map[tailcfg.DiscoKey]bool{} // TODO: don't re-alloc this map each time trimmedNodes := map[tailcfg.NodeKey]bool{} // TODO: don't re-alloc this map each time
needRemoveStep := false needRemoveStep := false
for i := range full.Peers { for i := range full.Peers {
p := &full.Peers[i] p := &full.Peers[i]
nk := tailcfg.NodeKey(p.PublicKey)
if !isTrimmablePeer(p, len(full.Peers)) { if !isTrimmablePeer(p, len(full.Peers)) {
min.Peers = append(min.Peers, *p) min.Peers = append(min.Peers, *p)
if discoChanged[key.Public(p.PublicKey)] { if discoChanged[nk] {
needRemoveStep = true needRemoveStep = true
} }
continue continue
} }
dk := p.Endpoints.DiscoKey trackNodes = append(trackNodes, nk)
trackDisco = append(trackDisco, dk)
recentlyActive := false recentlyActive := false
for _, cidr := range p.AllowedIPs { for _, cidr := range p.AllowedIPs {
trackIPs = append(trackIPs, cidr.IP()) trackIPs = append(trackIPs, cidr.IP())
recentlyActive = recentlyActive || e.isActiveSince(dk, cidr.IP(), activeCutoff) recentlyActive = recentlyActive || e.isActiveSinceLocked(nk, cidr.IP(), activeCutoff)
} }
if recentlyActive { if recentlyActive {
min.Peers = append(min.Peers, *p) min.Peers = append(min.Peers, *p)
if discoChanged[key.Public(p.PublicKey)] { if discoChanged[tailcfg.NodeKey(p.PublicKey)] {
needRemoveStep = true needRemoveStep = true
} }
} else { } else {
trimmedDisco[dk] = true trimmedNodes[tailcfg.NodeKey(p.PublicKey)] = true
} }
} }
e.lastNMinPeers = len(min.Peers) e.lastNMinPeers = len(min.Peers)
if !deephash.Update(&e.lastEngineSigTrim, &min, trimmedDisco, trackDisco, trackIPs) { if !deephash.Update(&e.lastEngineSigTrim, &min, trimmedNodes, trackNodes, trackIPs) {
// No changes // No changes
return nil return nil
} }
e.trimmedDisco = trimmedDisco e.trimmedNodes = trimmedNodes
e.updateActivityMapsLocked(trackDisco, trackIPs) e.updateActivityMapsLocked(trackNodes, trackIPs)
if needRemoveStep { if needRemoveStep {
minner := min minner := min
minner.Peers = nil minner.Peers = nil
numRemove := 0 numRemove := 0
for _, p := range min.Peers { for _, p := range min.Peers {
if discoChanged[key.Public(p.PublicKey)] { if discoChanged[tailcfg.NodeKey(p.PublicKey)] {
numRemove++ numRemove++
continue continue
} }
@ -719,17 +714,17 @@ func (e *userspaceEngine) maybeReconfigWireguardLocked(discoChanged map[key.Publ
// as given to wireguard-go. // as given to wireguard-go.
// //
// e.wgLock must be held. // e.wgLock must be held.
func (e *userspaceEngine) updateActivityMapsLocked(trackDisco []tailcfg.DiscoKey, trackIPs []netaddr.IP) { func (e *userspaceEngine) updateActivityMapsLocked(trackNodes []tailcfg.NodeKey, trackIPs []netaddr.IP) {
// Generate the new map of which discokeys we want to track // Generate the new map of which nodekeys we want to track
// receive times for. // receive times for.
mr := map[tailcfg.DiscoKey]mono.Time{} // TODO: only recreate this if set of keys changed mr := map[tailcfg.NodeKey]mono.Time{} // TODO: only recreate this if set of keys changed
for _, dk := range trackDisco { for _, nk := range trackNodes {
// Preserve old times in the new map, but also // Preserve old times in the new map, but also
// populate map entries for new trackDisco values with // populate map entries for new trackNodes values with
// time.Time{} zero values. (Only entries in this map // time.Time{} zero values. (Only entries in this map
// are tracked, so the Time zero values allow it to be // are tracked, so the Time zero values allow it to be
// tracked later) // tracked later)
mr[dk] = e.recvActivityAt[dk] mr[nk] = e.recvActivityAt[nk]
} }
e.recvActivityAt = mr e.recvActivityAt = mr
@ -844,12 +839,12 @@ func (e *userspaceEngine) Reconfig(cfg *wgcfg.Config, routerCfg *router.Config,
// If so, we need to update the wireguard-go/device.Device in two phases: // If so, we need to update the wireguard-go/device.Device in two phases:
// once without the node which has restarted, to clear its wireguard session key, // once without the node which has restarted, to clear its wireguard session key,
// and a second time with it. // and a second time with it.
discoChanged := make(map[key.Public]bool) discoChanged := make(map[tailcfg.NodeKey]bool)
{ {
prevEP := make(map[key.Public]tailcfg.DiscoKey) prevEP := make(map[tailcfg.NodeKey]tailcfg.DiscoKey)
for i := range e.lastCfgFull.Peers { for i := range e.lastCfgFull.Peers {
if p := &e.lastCfgFull.Peers[i]; !p.Endpoints.DiscoKey.IsZero() { if p := &e.lastCfgFull.Peers[i]; !p.Endpoints.DiscoKey.IsZero() {
prevEP[key.Public(p.PublicKey)] = p.Endpoints.DiscoKey prevEP[tailcfg.NodeKey(p.PublicKey)] = p.Endpoints.DiscoKey
} }
} }
for i := range cfg.Peers { for i := range cfg.Peers {
@ -857,7 +852,7 @@ func (e *userspaceEngine) Reconfig(cfg *wgcfg.Config, routerCfg *router.Config,
if p.Endpoints.DiscoKey.IsZero() { if p.Endpoints.DiscoKey.IsZero() {
continue continue
} }
pub := key.Public(p.PublicKey) pub := tailcfg.NodeKey(p.PublicKey)
if old, ok := prevEP[pub]; ok && old != p.Endpoints.DiscoKey { if old, ok := prevEP[pub]; ok && old != p.Endpoints.DiscoKey {
discoChanged[pub] = true discoChanged[pub] = true
e.logf("wgengine: Reconfig: %s changed from %q to %q", pub.ShortString(), old, p.Endpoints) e.logf("wgengine: Reconfig: %s changed from %q to %q", pub.ShortString(), old, p.Endpoints)

@ -17,6 +17,8 @@ import (
"tailscale.com/tailcfg" "tailscale.com/tailcfg"
"tailscale.com/tstime/mono" "tailscale.com/tstime/mono"
"tailscale.com/types/key" "tailscale.com/types/key"
"tailscale.com/types/netmap"
"tailscale.com/types/wgkey"
"tailscale.com/wgengine/router" "tailscale.com/wgengine/router"
"tailscale.com/wgengine/wgcfg" "tailscale.com/wgengine/wgcfg"
) )
@ -36,20 +38,20 @@ func TestNoteReceiveActivity(t *testing.T) {
} }
e := &userspaceEngine{ e := &userspaceEngine{
timeNow: func() mono.Time { return now }, timeNow: func() mono.Time { return now },
recvActivityAt: map[tailcfg.DiscoKey]mono.Time{}, recvActivityAt: map[tailcfg.NodeKey]mono.Time{},
logf: func(format string, a ...interface{}) { logf: func(format string, a ...interface{}) {
fmt.Fprintf(&logBuf, format, a...) fmt.Fprintf(&logBuf, format, a...)
}, },
tundev: new(tstun.Wrapper), tundev: new(tstun.Wrapper),
testMaybeReconfigHook: func() { confc <- true }, testMaybeReconfigHook: func() { confc <- true },
trimmedDisco: map[tailcfg.DiscoKey]bool{}, trimmedNodes: map[tailcfg.NodeKey]bool{},
} }
ra := e.recvActivityAt ra := e.recvActivityAt
dk := tailcfg.DiscoKey(key.NewPrivate().Public()) nk := tailcfg.NodeKey(key.NewPrivate().Public())
// Activity on an untracked key should do nothing. // Activity on an untracked key should do nothing.
e.noteReceiveActivity(dk) e.noteRecvActivity(nk)
if len(ra) != 0 { if len(ra) != 0 {
t.Fatalf("unexpected growth in map: now has %d keys; want 0", len(ra)) t.Fatalf("unexpected growth in map: now has %d keys; want 0", len(ra))
} }
@ -58,12 +60,12 @@ func TestNoteReceiveActivity(t *testing.T) {
} }
// Now track it, but don't mark it trimmed, so shouldn't update. // Now track it, but don't mark it trimmed, so shouldn't update.
ra[dk] = 0 ra[nk] = 0
e.noteReceiveActivity(dk) e.noteRecvActivity(nk)
if len(ra) != 1 { if len(ra) != 1 {
t.Fatalf("unexpected growth in map: now has %d keys; want 1", len(ra)) t.Fatalf("unexpected growth in map: now has %d keys; want 1", len(ra))
} }
if got := ra[dk]; got != now { if got := ra[nk]; got != now {
t.Fatalf("time in map = %v; want %v", got, now) t.Fatalf("time in map = %v; want %v", got, now)
} }
if gotConf() { if gotConf() {
@ -71,12 +73,12 @@ func TestNoteReceiveActivity(t *testing.T) {
} }
// Now mark it trimmed and expect an update. // Now mark it trimmed and expect an update.
e.trimmedDisco[dk] = true e.trimmedNodes[nk] = true
e.noteReceiveActivity(dk) e.noteRecvActivity(nk)
if len(ra) != 1 { if len(ra) != 1 {
t.Fatalf("unexpected growth in map: now has %d keys; want 1", len(ra)) t.Fatalf("unexpected growth in map: now has %d keys; want 1", len(ra))
} }
if got := ra[dk]; got != now { if got := ra[nk]; got != now {
t.Fatalf("time in map = %v; want %v", got, now) t.Fatalf("time in map = %v; want %v", got, now)
} }
if !gotConf() { if !gotConf() {
@ -94,38 +96,47 @@ func TestUserspaceEngineReconfig(t *testing.T) {
routerCfg := &router.Config{} routerCfg := &router.Config{}
for _, discoHex := range []string{ for _, nodeHex := range []string{
"aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa", "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
"bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb", "bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb",
} { } {
nm := &netmap.NetworkMap{
Peers: []*tailcfg.Node{
&tailcfg.Node{
Key: nkFromHex(nodeHex),
},
},
}
cfg := &wgcfg.Config{ cfg := &wgcfg.Config{
Peers: []wgcfg.Peer{ Peers: []wgcfg.Peer{
{ {
PublicKey: wgkey.Key(nkFromHex(nodeHex)),
AllowedIPs: []netaddr.IPPrefix{ AllowedIPs: []netaddr.IPPrefix{
netaddr.IPPrefixFrom(netaddr.IPv4(100, 100, 99, 1), 32), netaddr.IPPrefixFrom(netaddr.IPv4(100, 100, 99, 1), 32),
}, },
Endpoints: wgcfg.Endpoints{DiscoKey: dkFromHex(discoHex)}, Endpoints: wgcfg.Endpoints{PublicKey: wgkey.Key(nkFromHex(nodeHex))},
}, },
}, },
} }
e.SetNetworkMap(nm)
err = e.Reconfig(cfg, routerCfg, &dns.Config{}, nil) err = e.Reconfig(cfg, routerCfg, &dns.Config{}, nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
wantRecvAt := map[tailcfg.DiscoKey]mono.Time{ wantRecvAt := map[tailcfg.NodeKey]mono.Time{
dkFromHex(discoHex): 0, nkFromHex(nodeHex): 0,
} }
if got := ue.recvActivityAt; !reflect.DeepEqual(got, wantRecvAt) { if got := ue.recvActivityAt; !reflect.DeepEqual(got, wantRecvAt) {
t.Errorf("wrong recvActivityAt\n got: %v\nwant: %v\n", got, wantRecvAt) t.Errorf("wrong recvActivityAt\n got: %v\nwant: %v\n", got, wantRecvAt)
} }
wantTrimmedDisco := map[tailcfg.DiscoKey]bool{ wantTrimmedNodes := map[tailcfg.NodeKey]bool{
dkFromHex(discoHex): true, nkFromHex(nodeHex): true,
} }
if got := ue.trimmedDisco; !reflect.DeepEqual(got, wantTrimmedDisco) { if got := ue.trimmedNodes; !reflect.DeepEqual(got, wantTrimmedNodes) {
t.Errorf("wrong wantTrimmedDisco\n got: %v\nwant: %v\n", got, wantTrimmedDisco) t.Errorf("wrong wantTrimmedNodes\n got: %v\nwant: %v\n", got, wantTrimmedNodes)
} }
} }
} }
@ -153,14 +164,14 @@ func TestUserspaceEnginePortReconfig(t *testing.T) {
defer ue.Close() defer ue.Close()
startingPort := ue.magicConn.LocalPort() startingPort := ue.magicConn.LocalPort()
discoKey := dkFromHex("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa") nodeKey := nkFromHex("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa")
cfg := &wgcfg.Config{ cfg := &wgcfg.Config{
Peers: []wgcfg.Peer{ Peers: []wgcfg.Peer{
{ {
AllowedIPs: []netaddr.IPPrefix{ AllowedIPs: []netaddr.IPPrefix{
netaddr.IPPrefixFrom(netaddr.IPv4(100, 100, 99, 1), 32), netaddr.IPPrefixFrom(netaddr.IPv4(100, 100, 99, 1), 32),
}, },
Endpoints: wgcfg.Endpoints{DiscoKey: discoKey}, Endpoints: wgcfg.Endpoints{PublicKey: wgkey.Key(nodeKey)},
}, },
}, },
} }
@ -195,7 +206,7 @@ func TestUserspaceEnginePortReconfig(t *testing.T) {
} }
} }
func dkFromHex(hex string) tailcfg.DiscoKey { func nkFromHex(hex string) tailcfg.NodeKey {
if len(hex) != 64 { if len(hex) != 64 {
panic(fmt.Sprintf("%q is len %d; want 64", hex, len(hex))) panic(fmt.Sprintf("%q is len %d; want 64", hex, len(hex)))
} }
@ -203,7 +214,7 @@ func dkFromHex(hex string) tailcfg.DiscoKey {
if err != nil { if err != nil {
panic(fmt.Sprintf("%q is not hex: %v", hex, err)) panic(fmt.Sprintf("%q is not hex: %v", hex, err))
} }
return tailcfg.DiscoKey(k) return tailcfg.NodeKey(k)
} }
// an experiment to see if genLocalAddrFunc was worth it. As of Go // an experiment to see if genLocalAddrFunc was worth it. As of Go

Loading…
Cancel
Save