|  | @@ -13,26 +13,11 @@ import (
 | 
											
												
													
														|  |  	"github.com/sirupsen/logrus"
 |  |  	"github.com/sirupsen/logrus"
 | 
											
												
													
														|  |  )
 |  |  )
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | 
 |  | +//TODO: if a lighthouse doesn't have an answer, clients AGGRESSIVELY REQUERY.. why? handshake manager and/or getOrHandshake?
 | 
											
												
													
														|  |  //TODO: nodes are roaming lighthouses, this is bad. How are they learning?
 |  |  //TODO: nodes are roaming lighthouses, this is bad. How are they learning?
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  var ErrHostNotKnown = errors.New("host not known")
 |  |  var ErrHostNotKnown = errors.New("host not known")
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -// The maximum number of ip addresses to store for a given vpnIp per address family
 |  | 
 | 
											
												
													
														|  | -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 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
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |  type LightHouse struct {
 |  |  type LightHouse struct {
 | 
											
												
													
														|  |  	//TODO: We need a timer wheel to kick out vpnIps that haven't reported in a long time
 |  |  	//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
 |  |  	sync.RWMutex //Because we concurrently read and write to our maps
 | 
											
										
											
												
													
														|  | @@ -42,7 +27,8 @@ type LightHouse struct {
 | 
											
												
													
														|  |  	punchConn    *udpConn
 |  |  	punchConn    *udpConn
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// Local cache of answers from light houses
 |  |  	// Local cache of answers from light houses
 | 
											
												
													
														|  | -	addrMap map[uint32]*ip4And6
 |  | 
 | 
											
												
													
														|  | 
 |  | +	// map of vpn Ip to answers
 | 
											
												
													
														|  | 
 |  | +	addrMap map[uint32]*RemoteList
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// filters remote addresses allowed for each host
 |  |  	// filters remote addresses allowed for each host
 | 
											
												
													
														|  |  	// - When we are a lighthouse, this filters what addresses we store and
 |  |  	// - When we are a lighthouse, this filters what addresses we store and
 | 
											
										
											
												
													
														|  | @@ -81,7 +67,7 @@ func NewLightHouse(l *logrus.Logger, amLighthouse bool, myVpnIpNet *net.IPNet, i
 | 
											
												
													
														|  |  		amLighthouse: amLighthouse,
 |  |  		amLighthouse: amLighthouse,
 | 
											
												
													
														|  |  		myVpnIp:      ip2int(myVpnIpNet.IP),
 |  |  		myVpnIp:      ip2int(myVpnIpNet.IP),
 | 
											
												
													
														|  |  		myVpnZeros:   uint32(32 - ones),
 |  |  		myVpnZeros:   uint32(32 - ones),
 | 
											
												
													
														|  | -		addrMap:      make(map[uint32]*ip4And6),
 |  | 
 | 
											
												
													
														|  | 
 |  | +		addrMap:      make(map[uint32]*RemoteList),
 | 
											
												
													
														|  |  		nebulaPort:   nebulaPort,
 |  |  		nebulaPort:   nebulaPort,
 | 
											
												
													
														|  |  		lighthouses:  make(map[uint32]struct{}),
 |  |  		lighthouses:  make(map[uint32]struct{}),
 | 
											
												
													
														|  |  		staticList:   make(map[uint32]struct{}),
 |  |  		staticList:   make(map[uint32]struct{}),
 | 
											
										
											
												
													
														|  | @@ -130,57 +116,79 @@ func (lh *LightHouse) ValidateLHStaticEntries() error {
 | 
											
												
													
														|  |  	return nil
 |  |  	return nil
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -func (lh *LightHouse) Query(ip uint32, f EncWriter) ([]*udpAddr, error) {
 |  | 
 | 
											
												
													
														|  | -	//TODO: we need to hold the lock through the next func
 |  | 
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) Query(ip uint32, f EncWriter) *RemoteList {
 | 
											
												
													
														|  |  	if !lh.IsLighthouseIP(ip) {
 |  |  	if !lh.IsLighthouseIP(ip) {
 | 
											
												
													
														|  |  		lh.QueryServer(ip, f)
 |  |  		lh.QueryServer(ip, f)
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  	lh.RLock()
 |  |  	lh.RLock()
 | 
											
												
													
														|  |  	if v, ok := lh.addrMap[ip]; ok {
 |  |  	if v, ok := lh.addrMap[ip]; ok {
 | 
											
												
													
														|  |  		lh.RUnlock()
 |  |  		lh.RUnlock()
 | 
											
												
													
														|  | -		return TransformLHReplyToUdpAddrs(v), nil
 |  | 
 | 
											
												
													
														|  | 
 |  | +		return v
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  	lh.RUnlock()
 |  |  	lh.RUnlock()
 | 
											
												
													
														|  | -	return nil, ErrHostNotKnown
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return nil
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  // This is asynchronous so no reply should be expected
 |  |  // This is asynchronous so no reply should be expected
 | 
											
												
													
														|  |  func (lh *LightHouse) QueryServer(ip uint32, f EncWriter) {
 |  |  func (lh *LightHouse) QueryServer(ip uint32, f EncWriter) {
 | 
											
												
													
														|  | -	if !lh.amLighthouse {
 |  | 
 | 
											
												
													
														|  | -		// Send a query to the lighthouses and hope for the best next time
 |  | 
 | 
											
												
													
														|  | -		query, err := proto.Marshal(NewLhQueryByInt(ip))
 |  | 
 | 
											
												
													
														|  | -		if err != nil {
 |  | 
 | 
											
												
													
														|  | -			lh.l.WithError(err).WithField("vpnIp", IntIp(ip)).Error("Failed to marshal lighthouse query payload")
 |  | 
 | 
											
												
													
														|  | -			return
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	if lh.amLighthouse {
 | 
											
												
													
														|  | 
 |  | +		return
 | 
											
												
													
														|  | 
 |  | +	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		lh.metricTx(NebulaMeta_HostQuery, int64(len(lh.lighthouses)))
 |  | 
 | 
											
												
													
														|  | -		nb := make([]byte, 12, 12)
 |  | 
 | 
											
												
													
														|  | -		out := make([]byte, mtu)
 |  | 
 | 
											
												
													
														|  | -		for n := range lh.lighthouses {
 |  | 
 | 
											
												
													
														|  | -			f.SendMessageToVpnIp(lightHouse, 0, n, query, nb, out)
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	if lh.IsLighthouseIP(ip) {
 | 
											
												
													
														|  | 
 |  | +		return
 | 
											
												
													
														|  | 
 |  | +	}
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +	// Send a query to the lighthouses and hope for the best next time
 | 
											
												
													
														|  | 
 |  | +	query, err := proto.Marshal(NewLhQueryByInt(ip))
 | 
											
												
													
														|  | 
 |  | +	if err != nil {
 | 
											
												
													
														|  | 
 |  | +		lh.l.WithError(err).WithField("vpnIp", IntIp(ip)).Error("Failed to marshal lighthouse query payload")
 | 
											
												
													
														|  | 
 |  | +		return
 | 
											
												
													
														|  | 
 |  | +	}
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +	lh.metricTx(NebulaMeta_HostQuery, int64(len(lh.lighthouses)))
 | 
											
												
													
														|  | 
 |  | +	nb := make([]byte, 12, 12)
 | 
											
												
													
														|  | 
 |  | +	out := make([]byte, mtu)
 | 
											
												
													
														|  | 
 |  | +	for n := range lh.lighthouses {
 | 
											
												
													
														|  | 
 |  | +		f.SendMessageToVpnIp(lightHouse, 0, n, query, nb, out)
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -func (lh *LightHouse) QueryCache(ip uint32) []*udpAddr {
 |  | 
 | 
											
												
													
														|  | -	//TODO: we need to hold the lock through the next func
 |  | 
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) QueryCache(ip uint32) *RemoteList {
 | 
											
												
													
														|  |  	lh.RLock()
 |  |  	lh.RLock()
 | 
											
												
													
														|  |  	if v, ok := lh.addrMap[ip]; ok {
 |  |  	if v, ok := lh.addrMap[ip]; ok {
 | 
											
												
													
														|  |  		lh.RUnlock()
 |  |  		lh.RUnlock()
 | 
											
												
													
														|  | -		return TransformLHReplyToUdpAddrs(v)
 |  | 
 | 
											
												
													
														|  | 
 |  | +		return v
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  	lh.RUnlock()
 |  |  	lh.RUnlock()
 | 
											
												
													
														|  | -	return nil
 |  | 
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +	lh.Lock()
 | 
											
												
													
														|  | 
 |  | +	defer lh.Unlock()
 | 
											
												
													
														|  | 
 |  | +	// Add an entry if we don't already have one
 | 
											
												
													
														|  | 
 |  | +	return lh.unlockedGetRemoteList(ip)
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -//
 |  | 
 | 
											
												
													
														|  | -func (lh *LightHouse) queryAndPrepMessage(ip uint32, f func(*ip4And6) (int, error)) (bool, int, error) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +// queryAndPrepMessage is a lock helper on RemoteList, assisting the caller to build a lighthouse message containing
 | 
											
												
													
														|  | 
 |  | +// details from the remote list. It looks for a hit in the addrMap and a hit in the RemoteList under the owner vpnIp
 | 
											
												
													
														|  | 
 |  | +// If one is found then f() is called with proper locking, f() must return result of n.MarshalTo()
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) queryAndPrepMessage(vpnIp uint32, f func(*cache) (int, error)) (bool, int, error) {
 | 
											
												
													
														|  |  	lh.RLock()
 |  |  	lh.RLock()
 | 
											
												
													
														|  | -	if v, ok := lh.addrMap[ip]; ok {
 |  | 
 | 
											
												
													
														|  | -		n, err := f(v)
 |  | 
 | 
											
												
													
														|  | 
 |  | +	// Do we have an entry in the main cache?
 | 
											
												
													
														|  | 
 |  | +	if v, ok := lh.addrMap[vpnIp]; ok {
 | 
											
												
													
														|  | 
 |  | +		// Swap lh lock for remote list lock
 | 
											
												
													
														|  | 
 |  | +		v.RLock()
 | 
											
												
													
														|  | 
 |  | +		defer v.RUnlock()
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  |  		lh.RUnlock()
 |  |  		lh.RUnlock()
 | 
											
												
													
														|  | -		return true, n, err
 |  | 
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +		// vpnIp should also be the owner here since we are a lighthouse.
 | 
											
												
													
														|  | 
 |  | +		c := v.cache[vpnIp]
 | 
											
												
													
														|  | 
 |  | +		// Make sure we have
 | 
											
												
													
														|  | 
 |  | +		if c != nil {
 | 
											
												
													
														|  | 
 |  | +			n, err := f(c)
 | 
											
												
													
														|  | 
 |  | +			return true, n, err
 | 
											
												
													
														|  | 
 |  | +		}
 | 
											
												
													
														|  | 
 |  | +		return false, 0, nil
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  	lh.RUnlock()
 |  |  	lh.RUnlock()
 | 
											
												
													
														|  |  	return false, 0, nil
 |  |  	return false, 0, nil
 | 
											
										
											
												
													
														|  | @@ -203,70 +211,47 @@ func (lh *LightHouse) DeleteVpnIP(vpnIP uint32) {
 | 
											
												
													
														|  |  	lh.Unlock()
 |  |  	lh.Unlock()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -// AddRemote is correct way for non LightHouse members to add an address. toAddr will be placed in the learned map
 |  | 
 | 
											
												
													
														|  | -// static means this is a static host entry from the config file, it should only be used on start up
 |  | 
 | 
											
												
													
														|  | -func (lh *LightHouse) AddRemote(vpnIP uint32, toAddr *udpAddr, static bool) {
 |  | 
 | 
											
												
													
														|  | -	if ipv4 := toAddr.IP.To4(); ipv4 != nil {
 |  | 
 | 
											
												
													
														|  | -		lh.addRemoteV4(vpnIP, NewIp4AndPort(ipv4, uint32(toAddr.Port)), static, true)
 |  | 
 | 
											
												
													
														|  | -	} else {
 |  | 
 | 
											
												
													
														|  | -		lh.addRemoteV6(vpnIP, NewIp6AndPort(toAddr.IP, uint32(toAddr.Port)), static, true)
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	//TODO: if we do not add due to a config filter we may end up not having any addresses here
 |  | 
 | 
											
												
													
														|  | -	if static {
 |  | 
 | 
											
												
													
														|  | -		lh.staticList[vpnIP] = struct{}{}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -// unlockedGetAddrs assumes you have the lh lock
 |  | 
 | 
											
												
													
														|  | -func (lh *LightHouse) unlockedGetAddrs(vpnIP uint32) *ip4And6 {
 |  | 
 | 
											
												
													
														|  | -	am, ok := lh.addrMap[vpnIP]
 |  | 
 | 
											
												
													
														|  | -	if !ok {
 |  | 
 | 
											
												
													
														|  | -		am = &ip4And6{}
 |  | 
 | 
											
												
													
														|  | -		lh.addrMap[vpnIP] = am
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -	return am
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -// addRemoteV4 is a lighthouse internal method that prepends a remote if it is allowed by the allow list and not duplicated
 |  | 
 | 
											
												
													
														|  | -func (lh *LightHouse) addRemoteV4(vpnIP uint32, to *Ip4AndPort, static bool, learned bool) {
 |  | 
 | 
											
												
													
														|  | -	// First we check if the sender thinks this is a static entry
 |  | 
 | 
											
												
													
														|  | -	// and do nothing if it is not, but should be considered static
 |  | 
 | 
											
												
													
														|  | -	if static == false {
 |  | 
 | 
											
												
													
														|  | -		if _, ok := lh.staticList[vpnIP]; ok {
 |  | 
 | 
											
												
													
														|  | -			return
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | 
 |  | +// AddStaticRemote adds a static host entry for vpnIp as ourselves as the owner
 | 
											
												
													
														|  | 
 |  | +// We are the owner because we don't want a lighthouse server to advertise for static hosts it was configured with
 | 
											
												
													
														|  | 
 |  | +// And we don't want a lighthouse query reply to interfere with our learned cache if we are a client
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) AddStaticRemote(vpnIp uint32, toAddr *udpAddr) {
 | 
											
												
													
														|  |  	lh.Lock()
 |  |  	lh.Lock()
 | 
											
												
													
														|  | -	defer lh.Unlock()
 |  | 
 | 
											
												
													
														|  | -	am := lh.unlockedGetAddrs(vpnIP)
 |  | 
 | 
											
												
													
														|  | 
 |  | +	am := lh.unlockedGetRemoteList(vpnIp)
 | 
											
												
													
														|  | 
 |  | +	am.Lock()
 | 
											
												
													
														|  | 
 |  | +	defer am.Unlock()
 | 
											
												
													
														|  | 
 |  | +	lh.Unlock()
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	if learned {
 |  | 
 | 
											
												
													
														|  | -		if !lh.unlockedShouldAddV4(am.learnedV4, to) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +	if ipv4 := toAddr.IP.To4(); ipv4 != nil {
 | 
											
												
													
														|  | 
 |  | +		to := NewIp4AndPort(ipv4, uint32(toAddr.Port))
 | 
											
												
													
														|  | 
 |  | +		if !lh.unlockedShouldAddV4(to) {
 | 
											
												
													
														|  |  			return
 |  |  			return
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  | -		am.learnedV4 = prependAndLimitV4(am.learnedV4, to)
 |  | 
 | 
											
												
													
														|  | 
 |  | +		am.unlockedPrependV4(lh.myVpnIp, to)
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  |  	} else {
 |  |  	} else {
 | 
											
												
													
														|  | -		if !lh.unlockedShouldAddV4(am.v4, to) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +		to := NewIp6AndPort(toAddr.IP, uint32(toAddr.Port))
 | 
											
												
													
														|  | 
 |  | +		if !lh.unlockedShouldAddV6(to) {
 | 
											
												
													
														|  |  			return
 |  |  			return
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  | -		am.v4 = prependAndLimitV4(am.v4, to)
 |  | 
 | 
											
												
													
														|  | 
 |  | +		am.unlockedPrependV6(lh.myVpnIp, to)
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +	// Mark it as static
 | 
											
												
													
														|  | 
 |  | +	lh.staticList[vpnIp] = struct{}{}
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -func prependAndLimitV4(cache []*Ip4AndPort, to *Ip4AndPort) []*Ip4AndPort {
 |  | 
 | 
											
												
													
														|  | -	cache = append(cache, nil)
 |  | 
 | 
											
												
													
														|  | -	copy(cache[1:], cache)
 |  | 
 | 
											
												
													
														|  | -	cache[0] = to
 |  | 
 | 
											
												
													
														|  | -	if len(cache) > MaxRemotes {
 |  | 
 | 
											
												
													
														|  | -		cache = cache[:maxAddrs]
 |  | 
 | 
											
												
													
														|  | 
 |  | +// unlockedGetRemoteList assumes you have the lh lock
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) unlockedGetRemoteList(vpnIP uint32) *RemoteList {
 | 
											
												
													
														|  | 
 |  | +	am, ok := lh.addrMap[vpnIP]
 | 
											
												
													
														|  | 
 |  | +	if !ok {
 | 
											
												
													
														|  | 
 |  | +		am = NewRemoteList()
 | 
											
												
													
														|  | 
 |  | +		lh.addrMap[vpnIP] = am
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  | -	return cache
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return am
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -// 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 {
 |  | 
 | 
											
												
													
														|  | 
 |  | +// unlockedShouldAddV4 checks if to is allowed by our allow list
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) unlockedShouldAddV4(to *Ip4AndPort) bool {
 | 
											
												
													
														|  |  	allow := lh.remoteAllowList.AllowIpV4(to.Ip)
 |  |  	allow := lh.remoteAllowList.AllowIpV4(to.Ip)
 | 
											
												
													
														|  |  	if lh.l.Level >= logrus.TraceLevel {
 |  |  	if lh.l.Level >= logrus.TraceLevel {
 | 
											
												
													
														|  |  		lh.l.WithField("remoteIp", IntIp(to.Ip)).WithField("allow", allow).Trace("remoteAllowList.Allow")
 |  |  		lh.l.WithField("remoteIp", IntIp(to.Ip)).WithField("allow", allow).Trace("remoteAllowList.Allow")
 | 
											
										
											
												
													
														|  | @@ -276,69 +261,21 @@ func (lh *LightHouse) unlockedShouldAddV4(am []*Ip4AndPort, to *Ip4AndPort) bool
 | 
											
												
													
														|  |  		return false
 |  |  		return false
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	for _, v := range am {
 |  | 
 | 
											
												
													
														|  | -		if v.Ip == to.Ip && v.Port == to.Port {
 |  | 
 | 
											
												
													
														|  | -			return false
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |  	return true
 |  |  	return true
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -// addRemoteV6 is a lighthouse internal method that prepends a remote if it is allowed by the allow list and not duplicated
 |  | 
 | 
											
												
													
														|  | -func (lh *LightHouse) addRemoteV6(vpnIP uint32, to *Ip6AndPort, static bool, learned bool) {
 |  | 
 | 
											
												
													
														|  | -	// First we check if the sender thinks this is a static entry
 |  | 
 | 
											
												
													
														|  | -	// and do nothing if it is not, but should be considered static
 |  | 
 | 
											
												
													
														|  | -	if static == false {
 |  | 
 | 
											
												
													
														|  | -		if _, ok := lh.staticList[vpnIP]; ok {
 |  | 
 | 
											
												
													
														|  | -			return
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	lh.Lock()
 |  | 
 | 
											
												
													
														|  | -	defer lh.Unlock()
 |  | 
 | 
											
												
													
														|  | -	am := lh.unlockedGetAddrs(vpnIP)
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	if learned {
 |  | 
 | 
											
												
													
														|  | -		if !lh.unlockedShouldAddV6(am.learnedV6, to) {
 |  | 
 | 
											
												
													
														|  | -			return
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -		am.learnedV6 = prependAndLimitV6(am.learnedV6, to)
 |  | 
 | 
											
												
													
														|  | -	} else {
 |  | 
 | 
											
												
													
														|  | -		if !lh.unlockedShouldAddV6(am.v6, to) {
 |  | 
 | 
											
												
													
														|  | -			return
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -		am.v6 = prependAndLimitV6(am.v6, to)
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -func prependAndLimitV6(cache []*Ip6AndPort, to *Ip6AndPort) []*Ip6AndPort {
 |  | 
 | 
											
												
													
														|  | -	cache = append(cache, nil)
 |  | 
 | 
											
												
													
														|  | -	copy(cache[1:], cache)
 |  | 
 | 
											
												
													
														|  | -	cache[0] = to
 |  | 
 | 
											
												
													
														|  | -	if len(cache) > MaxRemotes {
 |  | 
 | 
											
												
													
														|  | -		cache = cache[:maxAddrs]
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -	return cache
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -// 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 {
 |  | 
 | 
											
												
													
														|  | 
 |  | +// unlockedShouldAddV6 checks if to is allowed by our allow list
 | 
											
												
													
														|  | 
 |  | +func (lh *LightHouse) unlockedShouldAddV6(to *Ip6AndPort) bool {
 | 
											
												
													
														|  |  	allow := lh.remoteAllowList.AllowIpV6(to.Hi, to.Lo)
 |  |  	allow := lh.remoteAllowList.AllowIpV6(to.Hi, to.Lo)
 | 
											
												
													
														|  |  	if lh.l.Level >= logrus.TraceLevel {
 |  |  	if lh.l.Level >= logrus.TraceLevel {
 | 
											
												
													
														|  |  		lh.l.WithField("remoteIp", lhIp6ToIp(to)).WithField("allow", allow).Trace("remoteAllowList.Allow")
 |  |  		lh.l.WithField("remoteIp", lhIp6ToIp(to)).WithField("allow", allow).Trace("remoteAllowList.Allow")
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | 
 |  | +	// We don't check our vpn network here because nebula does not support ipv6 on the inside
 | 
											
												
													
														|  |  	if !allow {
 |  |  	if !allow {
 | 
											
												
													
														|  |  		return false
 |  |  		return false
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	for _, v := range am {
 |  | 
 | 
											
												
													
														|  | -		if v.Hi == to.Hi && v.Lo == to.Lo && v.Port == to.Port {
 |  | 
 | 
											
												
													
														|  | -			return false
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |  	return true
 |  |  	return true
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
										
											
												
													
														|  | @@ -349,13 +286,6 @@ func lhIp6ToIp(v *Ip6AndPort) net.IP {
 | 
											
												
													
														|  |  	return ip
 |  |  	return ip
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -func (lh *LightHouse) AddRemoteAndReset(vpnIP uint32, toIp *udpAddr) {
 |  | 
 | 
											
												
													
														|  | -	if lh.amLighthouse {
 |  | 
 | 
											
												
													
														|  | -		lh.DeleteVpnIP(vpnIP)
 |  | 
 | 
											
												
													
														|  | -		lh.AddRemote(vpnIP, toIp, false)
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |  func (lh *LightHouse) IsLighthouseIP(vpnIP uint32) bool {
 |  |  func (lh *LightHouse) IsLighthouseIP(vpnIP uint32) bool {
 | 
											
												
													
														|  |  	if _, ok := lh.lighthouses[vpnIP]; ok {
 |  |  	if _, ok := lh.lighthouses[vpnIP]; ok {
 | 
											
												
													
														|  |  		return true
 |  |  		return true
 | 
											
										
											
												
													
														|  | @@ -496,7 +426,6 @@ func (lhh *LightHouseHandler) resetMeta() *NebulaMeta {
 | 
											
												
													
														|  |  	return lhh.meta
 |  |  	return lhh.meta
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -//TODO: do we need c here?
 |  | 
 | 
											
												
													
														|  |  func (lhh *LightHouseHandler) HandleRequest(rAddr *udpAddr, vpnIp uint32, p []byte, w EncWriter) {
 |  |  func (lhh *LightHouseHandler) HandleRequest(rAddr *udpAddr, vpnIp uint32, p []byte, w EncWriter) {
 | 
											
												
													
														|  |  	n := lhh.resetMeta()
 |  |  	n := lhh.resetMeta()
 | 
											
												
													
														|  |  	err := n.Unmarshal(p)
 |  |  	err := n.Unmarshal(p)
 | 
											
										
											
												
													
														|  | @@ -544,13 +473,12 @@ func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp uint32, addr
 | 
											
												
													
														|  |  	//TODO: we can DRY this further
 |  |  	//TODO: we can DRY this further
 | 
											
												
													
														|  |  	reqVpnIP := n.Details.VpnIp
 |  |  	reqVpnIP := n.Details.VpnIp
 | 
											
												
													
														|  |  	//TODO: Maybe instead of marshalling into n we marshal into a new `r` to not nuke our current request data
 |  |  	//TODO: Maybe instead of marshalling into n we marshal into a new `r` to not nuke our current request data
 | 
											
												
													
														|  | -	//TODO: If we use a lock on cache we can avoid holding it on lh.addrMap and keep things moving better
 |  | 
 | 
											
												
													
														|  | -	found, ln, err := lhh.lh.queryAndPrepMessage(n.Details.VpnIp, func(cache *ip4And6) (int, error) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +	found, ln, err := lhh.lh.queryAndPrepMessage(n.Details.VpnIp, func(c *cache) (int, error) {
 | 
											
												
													
														|  |  		n = lhh.resetMeta()
 |  |  		n = lhh.resetMeta()
 | 
											
												
													
														|  |  		n.Type = NebulaMeta_HostQueryReply
 |  |  		n.Type = NebulaMeta_HostQueryReply
 | 
											
												
													
														|  |  		n.Details.VpnIp = reqVpnIP
 |  |  		n.Details.VpnIp = reqVpnIP
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		lhh.coalesceAnswers(cache, n)
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lhh.coalesceAnswers(c, n)
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  		return n.MarshalTo(lhh.pb)
 |  |  		return n.MarshalTo(lhh.pb)
 | 
											
												
													
														|  |  	})
 |  |  	})
 | 
											
										
											
												
													
														|  | @@ -568,12 +496,12 @@ func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp uint32, addr
 | 
											
												
													
														|  |  	w.SendMessageToVpnIp(lightHouse, 0, vpnIp, lhh.pb[:ln], lhh.nb, lhh.out[:0])
 |  |  	w.SendMessageToVpnIp(lightHouse, 0, vpnIp, lhh.pb[:ln], lhh.nb, lhh.out[:0])
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// This signals the other side to punch some zero byte udp packets
 |  |  	// This signals the other side to punch some zero byte udp packets
 | 
											
												
													
														|  | -	found, ln, err = lhh.lh.queryAndPrepMessage(vpnIp, func(cache *ip4And6) (int, error) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +	found, ln, err = lhh.lh.queryAndPrepMessage(vpnIp, func(c *cache) (int, error) {
 | 
											
												
													
														|  |  		n = lhh.resetMeta()
 |  |  		n = lhh.resetMeta()
 | 
											
												
													
														|  |  		n.Type = NebulaMeta_HostPunchNotification
 |  |  		n.Type = NebulaMeta_HostPunchNotification
 | 
											
												
													
														|  |  		n.Details.VpnIp = vpnIp
 |  |  		n.Details.VpnIp = vpnIp
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		lhh.coalesceAnswers(cache, n)
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lhh.coalesceAnswers(c, n)
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  		return n.MarshalTo(lhh.pb)
 |  |  		return n.MarshalTo(lhh.pb)
 | 
											
												
													
														|  |  	})
 |  |  	})
 | 
											
										
											
												
													
														|  | @@ -591,12 +519,24 @@ func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp uint32, addr
 | 
											
												
													
														|  |  	w.SendMessageToVpnIp(lightHouse, 0, reqVpnIP, lhh.pb[:ln], lhh.nb, lhh.out[:0])
 |  |  	w.SendMessageToVpnIp(lightHouse, 0, reqVpnIP, lhh.pb[:ln], lhh.nb, lhh.out[:0])
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -func (lhh *LightHouseHandler) coalesceAnswers(cache *ip4And6, n *NebulaMeta) {
 |  | 
 | 
											
												
													
														|  | -	n.Details.Ip4AndPorts = append(n.Details.Ip4AndPorts, cache.v4...)
 |  | 
 | 
											
												
													
														|  | -	n.Details.Ip4AndPorts = append(n.Details.Ip4AndPorts, cache.learnedV4...)
 |  | 
 | 
											
												
													
														|  | 
 |  | +func (lhh *LightHouseHandler) coalesceAnswers(c *cache, n *NebulaMeta) {
 | 
											
												
													
														|  | 
 |  | +	if c.v4 != nil {
 | 
											
												
													
														|  | 
 |  | +		if c.v4.learned != nil {
 | 
											
												
													
														|  | 
 |  | +			n.Details.Ip4AndPorts = append(n.Details.Ip4AndPorts, c.v4.learned)
 | 
											
												
													
														|  | 
 |  | +		}
 | 
											
												
													
														|  | 
 |  | +		if c.v4.reported != nil && len(c.v4.reported) > 0 {
 | 
											
												
													
														|  | 
 |  | +			n.Details.Ip4AndPorts = append(n.Details.Ip4AndPorts, c.v4.reported...)
 | 
											
												
													
														|  | 
 |  | +		}
 | 
											
												
													
														|  | 
 |  | +	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	n.Details.Ip6AndPorts = append(n.Details.Ip6AndPorts, cache.v6...)
 |  | 
 | 
											
												
													
														|  | -	n.Details.Ip6AndPorts = append(n.Details.Ip6AndPorts, cache.learnedV6...)
 |  | 
 | 
											
												
													
														|  | 
 |  | +	if c.v6 != nil {
 | 
											
												
													
														|  | 
 |  | +		if c.v6.learned != nil {
 | 
											
												
													
														|  | 
 |  | +			n.Details.Ip6AndPorts = append(n.Details.Ip6AndPorts, c.v6.learned)
 | 
											
												
													
														|  | 
 |  | +		}
 | 
											
												
													
														|  | 
 |  | +		if c.v6.reported != nil && len(c.v6.reported) > 0 {
 | 
											
												
													
														|  | 
 |  | +			n.Details.Ip6AndPorts = append(n.Details.Ip6AndPorts, c.v6.reported...)
 | 
											
												
													
														|  | 
 |  | +		}
 | 
											
												
													
														|  | 
 |  | +	}
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lhh *LightHouseHandler) handleHostQueryReply(n *NebulaMeta, vpnIp uint32) {
 |  |  func (lhh *LightHouseHandler) handleHostQueryReply(n *NebulaMeta, vpnIp uint32) {
 | 
											
										
											
												
													
														|  | @@ -604,14 +544,14 @@ func (lhh *LightHouseHandler) handleHostQueryReply(n *NebulaMeta, vpnIp uint32)
 | 
											
												
													
														|  |  		return
 |  |  		return
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	// We can't just slam the responses in as they may come from multiple lighthouses and we should coalesce the answers
 |  | 
 | 
											
												
													
														|  | -	for _, to := range n.Details.Ip4AndPorts {
 |  | 
 | 
											
												
													
														|  | -		lhh.lh.addRemoteV4(n.Details.VpnIp, to, false, false)
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	lhh.lh.Lock()
 | 
											
												
													
														|  | 
 |  | +	am := lhh.lh.unlockedGetRemoteList(n.Details.VpnIp)
 | 
											
												
													
														|  | 
 |  | +	am.Lock()
 | 
											
												
													
														|  | 
 |  | +	lhh.lh.Unlock()
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	for _, to := range n.Details.Ip6AndPorts {
 |  | 
 | 
											
												
													
														|  | -		lhh.lh.addRemoteV6(n.Details.VpnIp, to, false, false)
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	am.unlockedSetV4(vpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
 | 
											
												
													
														|  | 
 |  | +	am.unlockedSetV6(vpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
 | 
											
												
													
														|  | 
 |  | +	am.Unlock()
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// Non-blocking attempt to trigger, skip if it would block
 |  |  	// Non-blocking attempt to trigger, skip if it would block
 | 
											
												
													
														|  |  	select {
 |  |  	select {
 | 
											
										
											
												
													
														|  | @@ -637,35 +577,13 @@ func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	lhh.lh.Lock()
 |  |  	lhh.lh.Lock()
 | 
											
												
													
														|  | -	defer lhh.lh.Unlock()
 |  | 
 | 
											
												
													
														|  | -	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
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	// We don't accumulate addresses being told to us
 |  | 
 | 
											
												
													
														|  | -	am.v4 = am.v4[:0]
 |  | 
 | 
											
												
													
														|  | -	am.v6 = am.v6[:0]
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	for _, v := range n.Details.Ip4AndPorts {
 |  | 
 | 
											
												
													
														|  | -		if lhh.lh.unlockedShouldAddV4(am.v4, v) {
 |  | 
 | 
											
												
													
														|  | -			am.v4 = append(am.v4, v)
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	for _, v := range n.Details.Ip6AndPorts {
 |  | 
 | 
											
												
													
														|  | -		if lhh.lh.unlockedShouldAddV6(am.v6, v) {
 |  | 
 | 
											
												
													
														|  | -			am.v6 = append(am.v6, v)
 |  | 
 | 
											
												
													
														|  | -		}
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	am := lhh.lh.unlockedGetRemoteList(vpnIp)
 | 
											
												
													
														|  | 
 |  | +	am.Lock()
 | 
											
												
													
														|  | 
 |  | +	lhh.lh.Unlock()
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	// We prefer the first n addresses if we got too big
 |  | 
 | 
											
												
													
														|  | -	if len(am.v4) > MaxRemotes {
 |  | 
 | 
											
												
													
														|  | -		am.v4 = am.v4[:MaxRemotes]
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	if len(am.v6) > MaxRemotes {
 |  | 
 | 
											
												
													
														|  | -		am.v6 = am.v6[:MaxRemotes]
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	am.unlockedSetV4(vpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
 | 
											
												
													
														|  | 
 |  | +	am.unlockedSetV6(vpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
 | 
											
												
													
														|  | 
 |  | +	am.Unlock()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp uint32, w EncWriter) {
 |  |  func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp uint32, w EncWriter) {
 | 
											
										
											
												
													
														|  | @@ -716,33 +634,6 @@ func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp u
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -func TransformLHReplyToUdpAddrs(ips *ip4And6) []*udpAddr {
 |  | 
 | 
											
												
													
														|  | -	addrs := make([]*udpAddr, len(ips.v4)+len(ips.v6)+len(ips.learnedV4)+len(ips.learnedV6))
 |  | 
 | 
											
												
													
														|  | -	i := 0
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	for _, v := range ips.learnedV4 {
 |  | 
 | 
											
												
													
														|  | -		addrs[i] = NewUDPAddrFromLH4(v)
 |  | 
 | 
											
												
													
														|  | -		i++
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	for _, v := range ips.v4 {
 |  | 
 | 
											
												
													
														|  | -		addrs[i] = NewUDPAddrFromLH4(v)
 |  | 
 | 
											
												
													
														|  | -		i++
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	for _, v := range ips.learnedV6 {
 |  | 
 | 
											
												
													
														|  | -		addrs[i] = NewUDPAddrFromLH6(v)
 |  | 
 | 
											
												
													
														|  | -		i++
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	for _, v := range ips.v6 {
 |  | 
 | 
											
												
													
														|  | -		addrs[i] = NewUDPAddrFromLH6(v)
 |  | 
 | 
											
												
													
														|  | -		i++
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -	return addrs
 |  | 
 | 
											
												
													
														|  | -}
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |  // ipMaskContains checks if testIp is contained by ip after applying a cidr
 |  |  // ipMaskContains checks if testIp is contained by ip after applying a cidr
 | 
											
												
													
														|  |  // zeros is 32 - bits from net.IPMask.Size()
 |  |  // zeros is 32 - bits from net.IPMask.Size()
 | 
											
												
													
														|  |  func ipMaskContains(ip uint32, zeros uint32, testIp uint32) bool {
 |  |  func ipMaskContains(ip uint32, zeros uint32, testIp uint32) bool {
 |