More LH cleanup (#429)

This commit is contained in:
Nathan Brown
2021-04-01 10:23:31 -05:00
committed by GitHub
parent 75f7bda0a4
commit 64d8e5aa96
14 changed files with 246 additions and 248 deletions

View File

@ -1,7 +1,7 @@
package nebula
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"net"
@ -13,9 +13,7 @@ import (
"github.com/sirupsen/logrus"
)
//TODO: if the pb code for ipv6 used a fixed data type we could save more work
//TODO: nodes are roaming lighthouses, this is bad. How are they learning?
//TODO: as a lh client, ignore any address within my nebula network?????
var ErrHostNotKnown = errors.New("host not known")
@ -25,11 +23,12 @@ const maxAddrs = 10
type ip4And6 struct {
//TODO: adding a lock here could allow us to release the lock on lh.addrMap quicker
// v4 and v6 store addresses that have been self reported by the client
// v4 and v6 store addresses that have been self reported by the client in a server or where all addresses are stored on a client
v4 []*Ip4AndPort
v6 []*Ip6AndPort
// Learned addresses are ones that a client does not know about but a lighthouse learned from as a result of the received packet
// This is only used if you are a lighthouse server
learnedV4 []*Ip4AndPort
learnedV6 []*Ip6AndPort
}
@ -38,7 +37,8 @@ type LightHouse struct {
//TODO: We need a timer wheel to kick out vpnIps that haven't reported in a long time
sync.RWMutex //Because we concurrently read and write to our maps
amLighthouse bool
myIp uint32
myVpnIp uint32
myVpnZeros uint32
punchConn *udpConn
// Local cache of answers from light houses
@ -75,10 +75,12 @@ type EncWriter interface {
SendMessageToVpnIp(t NebulaMessageType, st NebulaMessageSubType, vpnIp uint32, p, nb, out []byte)
}
func NewLightHouse(l *logrus.Logger, amLighthouse bool, myIp uint32, ips []uint32, interval int, nebulaPort uint32, pc *udpConn, punchBack bool, punchDelay time.Duration, metricsEnabled bool) *LightHouse {
func NewLightHouse(l *logrus.Logger, amLighthouse bool, myVpnIpNet *net.IPNet, ips []uint32, interval int, nebulaPort uint32, pc *udpConn, punchBack bool, punchDelay time.Duration, metricsEnabled bool) *LightHouse {
ones, _ := myVpnIpNet.Mask.Size()
h := LightHouse{
amLighthouse: amLighthouse,
myIp: myIp,
myVpnIp: ip2int(myVpnIpNet.IP),
myVpnZeros: uint32(32 - ones),
addrMap: make(map[uint32]*ip4And6),
nebulaPort: nebulaPort,
lighthouses: make(map[uint32]struct{}),
@ -216,16 +218,11 @@ func (lh *LightHouse) AddRemote(vpnIP uint32, toAddr *udpAddr, static bool) {
}
}
// unsafeGetAddrs assumes you have the lh lock
func (lh *LightHouse) unsafeGetAddrs(vpnIP uint32) *ip4And6 {
// unlockedGetAddrs assumes you have the lh lock
func (lh *LightHouse) unlockedGetAddrs(vpnIP uint32) *ip4And6 {
am, ok := lh.addrMap[vpnIP]
if !ok {
am = &ip4And6{
v4: make([]*Ip4AndPort, 0),
v6: make([]*Ip6AndPort, 0),
learnedV4: make([]*Ip4AndPort, 0),
learnedV6: make([]*Ip6AndPort, 0),
}
am = &ip4And6{}
lh.addrMap[vpnIP] = am
}
return am
@ -243,7 +240,7 @@ func (lh *LightHouse) addRemoteV4(vpnIP uint32, to *Ip4AndPort, static bool, lea
lh.Lock()
defer lh.Unlock()
am := lh.unsafeGetAddrs(vpnIP)
am := lh.unlockedGetAddrs(vpnIP)
if learned {
if !lh.unlockedShouldAddV4(am.learnedV4, to) {
@ -270,13 +267,12 @@ func prependAndLimitV4(cache []*Ip4AndPort, to *Ip4AndPort) []*Ip4AndPort {
// unlockedShouldAddV4 checks if to is allowed by our allow list and is not already present in the cache
func (lh *LightHouse) unlockedShouldAddV4(am []*Ip4AndPort, to *Ip4AndPort) bool {
ip := int2ip(to.Ip)
allow := lh.remoteAllowList.Allow(ip)
if lh.l.Level >= logrus.DebugLevel {
lh.l.WithField("remoteIp", ip).WithField("allow", allow).Debug("remoteAllowList.Allow")
allow := lh.remoteAllowList.AllowIpV4(to.Ip)
if lh.l.Level >= logrus.TraceLevel {
lh.l.WithField("remoteIp", IntIp(to.Ip)).WithField("allow", allow).Trace("remoteAllowList.Allow")
}
if !allow {
if !allow || ipMaskContains(lh.myVpnIp, lh.myVpnZeros, to.Ip) {
return false
}
@ -301,7 +297,7 @@ func (lh *LightHouse) addRemoteV6(vpnIP uint32, to *Ip6AndPort, static bool, lea
lh.Lock()
defer lh.Unlock()
am := lh.unsafeGetAddrs(vpnIP)
am := lh.unlockedGetAddrs(vpnIP)
if learned {
if !lh.unlockedShouldAddV6(am.learnedV6, to) {
@ -328,10 +324,9 @@ func prependAndLimitV6(cache []*Ip6AndPort, to *Ip6AndPort) []*Ip6AndPort {
// unlockedShouldAddV6 checks if to is allowed by our allow list and is not already present in the cache
func (lh *LightHouse) unlockedShouldAddV6(am []*Ip6AndPort, to *Ip6AndPort) bool {
ip := net.IP(to.Ip)
allow := lh.remoteAllowList.Allow(ip)
if lh.l.Level >= logrus.DebugLevel {
lh.l.WithField("remoteIp", ip).WithField("allow", allow).Debug("remoteAllowList.Allow")
allow := lh.remoteAllowList.AllowIpV6(to.Hi, to.Lo)
if lh.l.Level >= logrus.TraceLevel {
lh.l.WithField("remoteIp", lhIp6ToIp(to)).WithField("allow", allow).Trace("remoteAllowList.Allow")
}
if !allow {
@ -339,7 +334,7 @@ func (lh *LightHouse) unlockedShouldAddV6(am []*Ip6AndPort, to *Ip6AndPort) bool
}
for _, v := range am {
if bytes.Equal(v.Ip, to.Ip) && v.Port == to.Port {
if v.Hi == to.Hi && v.Lo == to.Lo && v.Port == to.Port {
return false
}
}
@ -347,6 +342,13 @@ func (lh *LightHouse) unlockedShouldAddV6(am []*Ip6AndPort, to *Ip6AndPort) bool
return true
}
func lhIp6ToIp(v *Ip6AndPort) net.IP {
ip := make(net.IP, 16)
binary.BigEndian.PutUint64(ip[:8], v.Hi)
binary.BigEndian.PutUint64(ip[8:], v.Lo)
return ip
}
func (lh *LightHouse) AddRemoteAndReset(vpnIP uint32, toIp *udpAddr) {
if lh.amLighthouse {
lh.DeleteVpnIP(vpnIP)
@ -377,10 +379,11 @@ func NewIp4AndPort(ip net.IP, port uint32) *Ip4AndPort {
}
func NewIp6AndPort(ip net.IP, port uint32) *Ip6AndPort {
ipp := Ip6AndPort{Port: port}
ipp.Ip = make([]byte, len(ip))
copy(ipp.Ip, ip)
return &ipp
return &Ip6AndPort{
Hi: binary.BigEndian.Uint64(ip[:8]),
Lo: binary.BigEndian.Uint64(ip[8:]),
Port: port,
}
}
func NewUDPAddrFromLH4(ipp *Ip4AndPort) *udpAddr {
@ -392,7 +395,7 @@ func NewUDPAddrFromLH4(ipp *Ip4AndPort) *udpAddr {
}
func NewUDPAddrFromLH6(ipp *Ip6AndPort) *udpAddr {
return NewUDPAddr(ipp.Ip, uint16(ipp.Port))
return NewUDPAddr(lhIp6ToIp(ipp), uint16(ipp.Port))
}
func (lh *LightHouse) LhUpdateWorker(f EncWriter) {
@ -411,7 +414,7 @@ func (lh *LightHouse) SendUpdate(f EncWriter) {
var v6 []*Ip6AndPort
for _, e := range *localIps(lh.l, lh.localAllowList) {
if ip2int(e) == lh.myIp {
if ip4 := e.To4(); ip4 != nil && ipMaskContains(lh.myVpnIp, lh.myVpnZeros, ip2int(ip4)) {
continue
}
@ -425,7 +428,7 @@ func (lh *LightHouse) SendUpdate(f EncWriter) {
m := &NebulaMeta{
Type: NebulaMeta_HostUpdateNotification,
Details: &NebulaMetaDetails{
VpnIp: lh.myIp,
VpnIp: lh.myVpnIp,
Ip4AndPorts: v4,
Ip6AndPorts: v6,
},
@ -434,14 +437,15 @@ func (lh *LightHouse) SendUpdate(f EncWriter) {
lh.metricTx(NebulaMeta_HostUpdateNotification, int64(len(lh.lighthouses)))
nb := make([]byte, 12, 12)
out := make([]byte, mtu)
for vpnIp := range lh.lighthouses {
mm, err := proto.Marshal(m)
if err != nil && lh.l.Level >= logrus.DebugLevel {
lh.l.Debugf("Invalid marshal to update")
}
//l.Error("LIGHTHOUSE PACKET SEND", mm)
f.SendMessageToVpnIp(lightHouse, 0, vpnIp, mm, nb, out)
mm, err := proto.Marshal(m)
if err != nil && lh.l.Level >= logrus.DebugLevel {
lh.l.WithError(err).Error("Error while marshaling for lighthouse update")
return
}
for vpnIp := range lh.lighthouses {
f.SendMessageToVpnIp(lightHouse, 0, vpnIp, mm, nb, out)
}
}
@ -634,10 +638,9 @@ func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp
lhh.lh.Lock()
defer lhh.lh.Unlock()
am := lhh.lh.unsafeGetAddrs(vpnIp)
am := lhh.lh.unlockedGetAddrs(vpnIp)
//TODO: other note on a lock for am so we can release more quickly and lock our real unit of change which is far less contended
//TODO: we are not filtering by local or remote allowed addrs here, is this an ok change to make?
// We don't accumulate addresses being told to us
am.v4 = am.v4[:0]
@ -739,3 +742,9 @@ func TransformLHReplyToUdpAddrs(ips *ip4And6) []*udpAddr {
return addrs
}
// ipMaskContains checks if testIp is contained by ip after applying a cidr
// zeros is 32 - bits from net.IPMask.Size()
func ipMaskContains(ip uint32, zeros uint32, testIp uint32) bool {
return (testIp^ip)>>zeros == 0
}