mirror of
https://github.com/slackhq/nebula.git
synced 2025-11-22 08:24:25 +01:00
Switch most everything to netip in prep for ipv6 in the overlay (#1173)
This commit is contained in:
400
lighthouse.go
400
lighthouse.go
@@ -7,16 +7,16 @@ import (
|
||||
"fmt"
|
||||
"net"
|
||||
"net/netip"
|
||||
"strconv"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/gaissmai/bart"
|
||||
"github.com/rcrowley/go-metrics"
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/slackhq/nebula/cidr"
|
||||
"github.com/slackhq/nebula/config"
|
||||
"github.com/slackhq/nebula/header"
|
||||
"github.com/slackhq/nebula/iputil"
|
||||
"github.com/slackhq/nebula/udp"
|
||||
"github.com/slackhq/nebula/util"
|
||||
)
|
||||
@@ -26,25 +26,18 @@ import (
|
||||
|
||||
var ErrHostNotKnown = errors.New("host not known")
|
||||
|
||||
type netIpAndPort struct {
|
||||
ip net.IP
|
||||
port uint16
|
||||
}
|
||||
|
||||
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
|
||||
ctx context.Context
|
||||
amLighthouse bool
|
||||
myVpnIp iputil.VpnIp
|
||||
myVpnZeros iputil.VpnIp
|
||||
myVpnNet *net.IPNet
|
||||
myVpnNet netip.Prefix
|
||||
punchConn udp.Conn
|
||||
punchy *Punchy
|
||||
|
||||
// Local cache of answers from light houses
|
||||
// map of vpn Ip to answers
|
||||
addrMap map[iputil.VpnIp]*RemoteList
|
||||
addrMap map[netip.Addr]*RemoteList
|
||||
|
||||
// filters remote addresses allowed for each host
|
||||
// - When we are a lighthouse, this filters what addresses we store and
|
||||
@@ -57,26 +50,26 @@ type LightHouse struct {
|
||||
localAllowList atomic.Pointer[LocalAllowList]
|
||||
|
||||
// used to trigger the HandshakeManager when we receive HostQueryReply
|
||||
handshakeTrigger chan<- iputil.VpnIp
|
||||
handshakeTrigger chan<- netip.Addr
|
||||
|
||||
// staticList exists to avoid having a bool in each addrMap entry
|
||||
// since static should be rare
|
||||
staticList atomic.Pointer[map[iputil.VpnIp]struct{}]
|
||||
lighthouses atomic.Pointer[map[iputil.VpnIp]struct{}]
|
||||
staticList atomic.Pointer[map[netip.Addr]struct{}]
|
||||
lighthouses atomic.Pointer[map[netip.Addr]struct{}]
|
||||
|
||||
interval atomic.Int64
|
||||
updateCancel context.CancelFunc
|
||||
ifce EncWriter
|
||||
nebulaPort uint32 // 32 bits because protobuf does not have a uint16
|
||||
|
||||
advertiseAddrs atomic.Pointer[[]netIpAndPort]
|
||||
advertiseAddrs atomic.Pointer[[]netip.AddrPort]
|
||||
|
||||
// IP's of relays that can be used by peers to access me
|
||||
relaysForMe atomic.Pointer[[]iputil.VpnIp]
|
||||
relaysForMe atomic.Pointer[[]netip.Addr]
|
||||
|
||||
queryChan chan iputil.VpnIp
|
||||
queryChan chan netip.Addr
|
||||
|
||||
calculatedRemotes atomic.Pointer[cidr.Tree4[[]*calculatedRemote]] // Maps VpnIp to []*calculatedRemote
|
||||
calculatedRemotes atomic.Pointer[bart.Table[[]*calculatedRemote]] // Maps VpnIp to []*calculatedRemote
|
||||
|
||||
metrics *MessageMetrics
|
||||
metricHolepunchTx metrics.Counter
|
||||
@@ -85,7 +78,7 @@ type LightHouse struct {
|
||||
|
||||
// NewLightHouseFromConfig will build a Lighthouse struct from the values provided in the config object
|
||||
// addrMap should be nil unless this is during a config reload
|
||||
func NewLightHouseFromConfig(ctx context.Context, l *logrus.Logger, c *config.C, myVpnNet *net.IPNet, pc udp.Conn, p *Punchy) (*LightHouse, error) {
|
||||
func NewLightHouseFromConfig(ctx context.Context, l *logrus.Logger, c *config.C, myVpnNet netip.Prefix, pc udp.Conn, p *Punchy) (*LightHouse, error) {
|
||||
amLighthouse := c.GetBool("lighthouse.am_lighthouse", false)
|
||||
nebulaPort := uint32(c.GetInt("listen.port", 0))
|
||||
if amLighthouse && nebulaPort == 0 {
|
||||
@@ -98,26 +91,23 @@ func NewLightHouseFromConfig(ctx context.Context, l *logrus.Logger, c *config.C,
|
||||
if err != nil {
|
||||
return nil, util.NewContextualError("Failed to get listening port", nil, err)
|
||||
}
|
||||
nebulaPort = uint32(uPort.Port)
|
||||
nebulaPort = uint32(uPort.Port())
|
||||
}
|
||||
|
||||
ones, _ := myVpnNet.Mask.Size()
|
||||
h := LightHouse{
|
||||
ctx: ctx,
|
||||
amLighthouse: amLighthouse,
|
||||
myVpnIp: iputil.Ip2VpnIp(myVpnNet.IP),
|
||||
myVpnZeros: iputil.VpnIp(32 - ones),
|
||||
myVpnNet: myVpnNet,
|
||||
addrMap: make(map[iputil.VpnIp]*RemoteList),
|
||||
addrMap: make(map[netip.Addr]*RemoteList),
|
||||
nebulaPort: nebulaPort,
|
||||
punchConn: pc,
|
||||
punchy: p,
|
||||
queryChan: make(chan iputil.VpnIp, c.GetUint32("handshakes.query_buffer", 64)),
|
||||
queryChan: make(chan netip.Addr, c.GetUint32("handshakes.query_buffer", 64)),
|
||||
l: l,
|
||||
}
|
||||
lighthouses := make(map[iputil.VpnIp]struct{})
|
||||
lighthouses := make(map[netip.Addr]struct{})
|
||||
h.lighthouses.Store(&lighthouses)
|
||||
staticList := make(map[iputil.VpnIp]struct{})
|
||||
staticList := make(map[netip.Addr]struct{})
|
||||
h.staticList.Store(&staticList)
|
||||
|
||||
if c.GetBool("stats.lighthouse_metrics", false) {
|
||||
@@ -147,11 +137,11 @@ func NewLightHouseFromConfig(ctx context.Context, l *logrus.Logger, c *config.C,
|
||||
return &h, nil
|
||||
}
|
||||
|
||||
func (lh *LightHouse) GetStaticHostList() map[iputil.VpnIp]struct{} {
|
||||
func (lh *LightHouse) GetStaticHostList() map[netip.Addr]struct{} {
|
||||
return *lh.staticList.Load()
|
||||
}
|
||||
|
||||
func (lh *LightHouse) GetLighthouses() map[iputil.VpnIp]struct{} {
|
||||
func (lh *LightHouse) GetLighthouses() map[netip.Addr]struct{} {
|
||||
return *lh.lighthouses.Load()
|
||||
}
|
||||
|
||||
@@ -163,15 +153,15 @@ func (lh *LightHouse) GetLocalAllowList() *LocalAllowList {
|
||||
return lh.localAllowList.Load()
|
||||
}
|
||||
|
||||
func (lh *LightHouse) GetAdvertiseAddrs() []netIpAndPort {
|
||||
func (lh *LightHouse) GetAdvertiseAddrs() []netip.AddrPort {
|
||||
return *lh.advertiseAddrs.Load()
|
||||
}
|
||||
|
||||
func (lh *LightHouse) GetRelaysForMe() []iputil.VpnIp {
|
||||
func (lh *LightHouse) GetRelaysForMe() []netip.Addr {
|
||||
return *lh.relaysForMe.Load()
|
||||
}
|
||||
|
||||
func (lh *LightHouse) getCalculatedRemotes() *cidr.Tree4[[]*calculatedRemote] {
|
||||
func (lh *LightHouse) getCalculatedRemotes() *bart.Table[[]*calculatedRemote] {
|
||||
return lh.calculatedRemotes.Load()
|
||||
}
|
||||
|
||||
@@ -182,25 +172,40 @@ func (lh *LightHouse) GetUpdateInterval() int64 {
|
||||
func (lh *LightHouse) reload(c *config.C, initial bool) error {
|
||||
if initial || c.HasChanged("lighthouse.advertise_addrs") {
|
||||
rawAdvAddrs := c.GetStringSlice("lighthouse.advertise_addrs", []string{})
|
||||
advAddrs := make([]netIpAndPort, 0)
|
||||
advAddrs := make([]netip.AddrPort, 0)
|
||||
|
||||
for i, rawAddr := range rawAdvAddrs {
|
||||
fIp, fPort, err := udp.ParseIPAndPort(rawAddr)
|
||||
host, sport, err := net.SplitHostPort(rawAddr)
|
||||
if err != nil {
|
||||
return util.NewContextualError("Unable to parse lighthouse.advertise_addrs entry", m{"addr": rawAddr, "entry": i + 1}, err)
|
||||
}
|
||||
|
||||
if fPort == 0 {
|
||||
fPort = uint16(lh.nebulaPort)
|
||||
ips, err := net.DefaultResolver.LookupNetIP(context.Background(), "ip", host)
|
||||
if err != nil {
|
||||
return util.NewContextualError("Unable to lookup lighthouse.advertise_addrs entry", m{"addr": rawAddr, "entry": i + 1}, err)
|
||||
}
|
||||
if len(ips) == 0 {
|
||||
return util.NewContextualError("Unable to lookup lighthouse.advertise_addrs entry", m{"addr": rawAddr, "entry": i + 1}, nil)
|
||||
}
|
||||
|
||||
if ip4 := fIp.To4(); ip4 != nil && lh.myVpnNet.Contains(fIp) {
|
||||
port, err := strconv.Atoi(sport)
|
||||
if err != nil {
|
||||
return util.NewContextualError("Unable to parse port in lighthouse.advertise_addrs entry", m{"addr": rawAddr, "entry": i + 1}, err)
|
||||
}
|
||||
|
||||
if port == 0 {
|
||||
port = int(lh.nebulaPort)
|
||||
}
|
||||
|
||||
//TODO: we could technically insert all returned ips instead of just the first one if a dns lookup was used
|
||||
ip := ips[0].Unmap()
|
||||
if lh.myVpnNet.Contains(ip) {
|
||||
lh.l.WithField("addr", rawAddr).WithField("entry", i+1).
|
||||
Warn("Ignoring lighthouse.advertise_addrs report because it is within the nebula network range")
|
||||
continue
|
||||
}
|
||||
|
||||
advAddrs = append(advAddrs, netIpAndPort{ip: fIp, port: fPort})
|
||||
advAddrs = append(advAddrs, netip.AddrPortFrom(ip, uint16(port)))
|
||||
}
|
||||
|
||||
lh.advertiseAddrs.Store(&advAddrs)
|
||||
@@ -278,8 +283,8 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
|
||||
lh.RUnlock()
|
||||
}
|
||||
// Build a new list based on current config.
|
||||
staticList := make(map[iputil.VpnIp]struct{})
|
||||
err := lh.loadStaticMap(c, lh.myVpnNet, staticList)
|
||||
staticList := make(map[netip.Addr]struct{})
|
||||
err := lh.loadStaticMap(c, staticList)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -303,8 +308,8 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
|
||||
}
|
||||
|
||||
if initial || c.HasChanged("lighthouse.hosts") {
|
||||
lhMap := make(map[iputil.VpnIp]struct{})
|
||||
err := lh.parseLighthouses(c, lh.myVpnNet, lhMap)
|
||||
lhMap := make(map[netip.Addr]struct{})
|
||||
err := lh.parseLighthouses(c, lhMap)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -323,16 +328,17 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
|
||||
if len(c.GetStringSlice("relay.relays", nil)) > 0 {
|
||||
lh.l.Info("Ignoring relays from config because am_relay is true")
|
||||
}
|
||||
relaysForMe := []iputil.VpnIp{}
|
||||
relaysForMe := []netip.Addr{}
|
||||
lh.relaysForMe.Store(&relaysForMe)
|
||||
case false:
|
||||
relaysForMe := []iputil.VpnIp{}
|
||||
relaysForMe := []netip.Addr{}
|
||||
for _, v := range c.GetStringSlice("relay.relays", nil) {
|
||||
lh.l.WithField("relay", v).Info("Read relay from config")
|
||||
|
||||
configRIP := net.ParseIP(v)
|
||||
if configRIP != nil {
|
||||
relaysForMe = append(relaysForMe, iputil.Ip2VpnIp(configRIP))
|
||||
configRIP, err := netip.ParseAddr(v)
|
||||
//TODO: We could print the error here
|
||||
if err == nil {
|
||||
relaysForMe = append(relaysForMe, configRIP)
|
||||
}
|
||||
}
|
||||
lh.relaysForMe.Store(&relaysForMe)
|
||||
@@ -342,21 +348,21 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lh *LightHouse) parseLighthouses(c *config.C, tunCidr *net.IPNet, lhMap map[iputil.VpnIp]struct{}) error {
|
||||
func (lh *LightHouse) parseLighthouses(c *config.C, lhMap map[netip.Addr]struct{}) error {
|
||||
lhs := c.GetStringSlice("lighthouse.hosts", []string{})
|
||||
if lh.amLighthouse && len(lhs) != 0 {
|
||||
lh.l.Warn("lighthouse.am_lighthouse enabled on node but upstream lighthouses exist in config")
|
||||
}
|
||||
|
||||
for i, host := range lhs {
|
||||
ip := net.ParseIP(host)
|
||||
if ip == nil {
|
||||
return util.NewContextualError("Unable to parse lighthouse host entry", m{"host": host, "entry": i + 1}, nil)
|
||||
ip, err := netip.ParseAddr(host)
|
||||
if err != nil {
|
||||
return util.NewContextualError("Unable to parse lighthouse host entry", m{"host": host, "entry": i + 1}, err)
|
||||
}
|
||||
if !tunCidr.Contains(ip) {
|
||||
return util.NewContextualError("lighthouse host is not in our subnet, invalid", m{"vpnIp": ip, "network": tunCidr.String()}, nil)
|
||||
if !lh.myVpnNet.Contains(ip) {
|
||||
return util.NewContextualError("lighthouse host is not in our subnet, invalid", m{"vpnIp": ip, "network": lh.myVpnNet}, nil)
|
||||
}
|
||||
lhMap[iputil.Ip2VpnIp(ip)] = struct{}{}
|
||||
lhMap[ip] = struct{}{}
|
||||
}
|
||||
|
||||
if !lh.amLighthouse && len(lhMap) == 0 {
|
||||
@@ -399,7 +405,7 @@ func getStaticMapNetwork(c *config.C) (string, error) {
|
||||
return network, nil
|
||||
}
|
||||
|
||||
func (lh *LightHouse) loadStaticMap(c *config.C, tunCidr *net.IPNet, staticList map[iputil.VpnIp]struct{}) error {
|
||||
func (lh *LightHouse) loadStaticMap(c *config.C, staticList map[netip.Addr]struct{}) error {
|
||||
d, err := getStaticMapCadence(c)
|
||||
if err != nil {
|
||||
return err
|
||||
@@ -410,7 +416,7 @@ func (lh *LightHouse) loadStaticMap(c *config.C, tunCidr *net.IPNet, staticList
|
||||
return err
|
||||
}
|
||||
|
||||
lookup_timeout, err := getStaticMapLookupTimeout(c)
|
||||
lookupTimeout, err := getStaticMapLookupTimeout(c)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -419,16 +425,15 @@ func (lh *LightHouse) loadStaticMap(c *config.C, tunCidr *net.IPNet, staticList
|
||||
i := 0
|
||||
|
||||
for k, v := range shm {
|
||||
rip := net.ParseIP(fmt.Sprintf("%v", k))
|
||||
if rip == nil {
|
||||
return util.NewContextualError("Unable to parse static_host_map entry", m{"host": k, "entry": i + 1}, nil)
|
||||
vpnIp, err := netip.ParseAddr(fmt.Sprintf("%v", k))
|
||||
if err != nil {
|
||||
return util.NewContextualError("Unable to parse static_host_map entry", m{"host": k, "entry": i + 1}, err)
|
||||
}
|
||||
|
||||
if !tunCidr.Contains(rip) {
|
||||
return util.NewContextualError("static_host_map key is not in our subnet, invalid", m{"vpnIp": rip, "network": tunCidr.String(), "entry": i + 1}, nil)
|
||||
if !lh.myVpnNet.Contains(vpnIp) {
|
||||
return util.NewContextualError("static_host_map key is not in our subnet, invalid", m{"vpnIp": vpnIp, "network": lh.myVpnNet, "entry": i + 1}, nil)
|
||||
}
|
||||
|
||||
vpnIp := iputil.Ip2VpnIp(rip)
|
||||
vals, ok := v.([]interface{})
|
||||
if !ok {
|
||||
vals = []interface{}{v}
|
||||
@@ -438,7 +443,7 @@ func (lh *LightHouse) loadStaticMap(c *config.C, tunCidr *net.IPNet, staticList
|
||||
remoteAddrs = append(remoteAddrs, fmt.Sprintf("%v", v))
|
||||
}
|
||||
|
||||
err := lh.addStaticRemotes(i, d, network, lookup_timeout, vpnIp, remoteAddrs, staticList)
|
||||
err = lh.addStaticRemotes(i, d, network, lookupTimeout, vpnIp, remoteAddrs, staticList)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -448,7 +453,7 @@ func (lh *LightHouse) loadStaticMap(c *config.C, tunCidr *net.IPNet, staticList
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lh *LightHouse) Query(ip iputil.VpnIp) *RemoteList {
|
||||
func (lh *LightHouse) Query(ip netip.Addr) *RemoteList {
|
||||
if !lh.IsLighthouseIP(ip) {
|
||||
lh.QueryServer(ip)
|
||||
}
|
||||
@@ -462,7 +467,7 @@ func (lh *LightHouse) Query(ip iputil.VpnIp) *RemoteList {
|
||||
}
|
||||
|
||||
// QueryServer is asynchronous so no reply should be expected
|
||||
func (lh *LightHouse) QueryServer(ip iputil.VpnIp) {
|
||||
func (lh *LightHouse) QueryServer(ip netip.Addr) {
|
||||
// Don't put lighthouse ips in the query channel because we can't query lighthouses about lighthouses
|
||||
if lh.amLighthouse || lh.IsLighthouseIP(ip) {
|
||||
return
|
||||
@@ -471,7 +476,7 @@ func (lh *LightHouse) QueryServer(ip iputil.VpnIp) {
|
||||
lh.queryChan <- ip
|
||||
}
|
||||
|
||||
func (lh *LightHouse) QueryCache(ip iputil.VpnIp) *RemoteList {
|
||||
func (lh *LightHouse) QueryCache(ip netip.Addr) *RemoteList {
|
||||
lh.RLock()
|
||||
if v, ok := lh.addrMap[ip]; ok {
|
||||
lh.RUnlock()
|
||||
@@ -488,7 +493,7 @@ func (lh *LightHouse) QueryCache(ip iputil.VpnIp) *RemoteList {
|
||||
// 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 iputil.VpnIp, f func(*cache) (int, error)) (bool, int, error) {
|
||||
func (lh *LightHouse) queryAndPrepMessage(vpnIp netip.Addr, f func(*cache) (int, error)) (bool, int, error) {
|
||||
lh.RLock()
|
||||
// Do we have an entry in the main cache?
|
||||
if v, ok := lh.addrMap[vpnIp]; ok {
|
||||
@@ -511,7 +516,7 @@ func (lh *LightHouse) queryAndPrepMessage(vpnIp iputil.VpnIp, f func(*cache) (in
|
||||
return false, 0, nil
|
||||
}
|
||||
|
||||
func (lh *LightHouse) DeleteVpnIp(vpnIp iputil.VpnIp) {
|
||||
func (lh *LightHouse) DeleteVpnIp(vpnIp netip.Addr) {
|
||||
// First we check the static mapping
|
||||
// and do nothing if it is there
|
||||
if _, ok := lh.GetStaticHostList()[vpnIp]; ok {
|
||||
@@ -532,7 +537,7 @@ func (lh *LightHouse) DeleteVpnIp(vpnIp iputil.VpnIp) {
|
||||
// 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
|
||||
// NOTE: this function should not interact with any hot path objects, like lh.staticList, the caller should handle it
|
||||
func (lh *LightHouse) addStaticRemotes(i int, d time.Duration, network string, timeout time.Duration, vpnIp iputil.VpnIp, toAddrs []string, staticList map[iputil.VpnIp]struct{}) error {
|
||||
func (lh *LightHouse) addStaticRemotes(i int, d time.Duration, network string, timeout time.Duration, vpnIp netip.Addr, toAddrs []string, staticList map[netip.Addr]struct{}) error {
|
||||
lh.Lock()
|
||||
am := lh.unlockedGetRemoteList(vpnIp)
|
||||
am.Lock()
|
||||
@@ -553,20 +558,14 @@ func (lh *LightHouse) addStaticRemotes(i int, d time.Duration, network string, t
|
||||
am.unlockedSetHostnamesResults(hr)
|
||||
|
||||
for _, addrPort := range hr.GetIPs() {
|
||||
|
||||
if !lh.shouldAdd(vpnIp, addrPort.Addr()) {
|
||||
continue
|
||||
}
|
||||
switch {
|
||||
case addrPort.Addr().Is4():
|
||||
to := NewIp4AndPortFromNetIP(addrPort.Addr(), addrPort.Port())
|
||||
if !lh.unlockedShouldAddV4(vpnIp, to) {
|
||||
continue
|
||||
}
|
||||
am.unlockedPrependV4(lh.myVpnIp, to)
|
||||
am.unlockedPrependV4(lh.myVpnNet.Addr(), NewIp4AndPortFromNetIP(addrPort.Addr(), addrPort.Port()))
|
||||
case addrPort.Addr().Is6():
|
||||
to := NewIp6AndPortFromNetIP(addrPort.Addr(), addrPort.Port())
|
||||
if !lh.unlockedShouldAddV6(vpnIp, to) {
|
||||
continue
|
||||
}
|
||||
am.unlockedPrependV6(lh.myVpnIp, to)
|
||||
am.unlockedPrependV6(lh.myVpnNet.Addr(), NewIp6AndPortFromNetIP(addrPort.Addr(), addrPort.Port()))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -578,12 +577,12 @@ func (lh *LightHouse) addStaticRemotes(i int, d time.Duration, network string, t
|
||||
// addCalculatedRemotes adds any calculated remotes based on the
|
||||
// lighthouse.calculated_remotes configuration. It returns true if any
|
||||
// calculated remotes were added
|
||||
func (lh *LightHouse) addCalculatedRemotes(vpnIp iputil.VpnIp) bool {
|
||||
func (lh *LightHouse) addCalculatedRemotes(vpnIp netip.Addr) bool {
|
||||
tree := lh.getCalculatedRemotes()
|
||||
if tree == nil {
|
||||
return false
|
||||
}
|
||||
ok, calculatedRemotes := tree.MostSpecificContains(vpnIp)
|
||||
calculatedRemotes, ok := tree.Lookup(vpnIp)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
@@ -602,13 +601,13 @@ func (lh *LightHouse) addCalculatedRemotes(vpnIp iputil.VpnIp) bool {
|
||||
defer am.Unlock()
|
||||
lh.Unlock()
|
||||
|
||||
am.unlockedSetV4(lh.myVpnIp, vpnIp, calculated, lh.unlockedShouldAddV4)
|
||||
am.unlockedSetV4(lh.myVpnNet.Addr(), vpnIp, calculated, lh.unlockedShouldAddV4)
|
||||
|
||||
return len(calculated) > 0
|
||||
}
|
||||
|
||||
// unlockedGetRemoteList assumes you have the lh lock
|
||||
func (lh *LightHouse) unlockedGetRemoteList(vpnIp iputil.VpnIp) *RemoteList {
|
||||
func (lh *LightHouse) unlockedGetRemoteList(vpnIp netip.Addr) *RemoteList {
|
||||
am, ok := lh.addrMap[vpnIp]
|
||||
if !ok {
|
||||
am = NewRemoteList(func(a netip.Addr) bool { return lh.shouldAdd(vpnIp, a) })
|
||||
@@ -617,44 +616,27 @@ func (lh *LightHouse) unlockedGetRemoteList(vpnIp iputil.VpnIp) *RemoteList {
|
||||
return am
|
||||
}
|
||||
|
||||
func (lh *LightHouse) shouldAdd(vpnIp iputil.VpnIp, to netip.Addr) bool {
|
||||
switch {
|
||||
case to.Is4():
|
||||
ipBytes := to.As4()
|
||||
ip := iputil.Ip2VpnIp(ipBytes[:])
|
||||
allow := lh.GetRemoteAllowList().AllowIpV4(vpnIp, ip)
|
||||
if lh.l.Level >= logrus.TraceLevel {
|
||||
lh.l.WithField("remoteIp", vpnIp).WithField("allow", allow).Trace("remoteAllowList.Allow")
|
||||
}
|
||||
if !allow || ipMaskContains(lh.myVpnIp, lh.myVpnZeros, ip) {
|
||||
return false
|
||||
}
|
||||
case to.Is6():
|
||||
ipBytes := to.As16()
|
||||
|
||||
hi := binary.BigEndian.Uint64(ipBytes[:8])
|
||||
lo := binary.BigEndian.Uint64(ipBytes[8:])
|
||||
allow := lh.GetRemoteAllowList().AllowIpV6(vpnIp, hi, lo)
|
||||
if lh.l.Level >= logrus.TraceLevel {
|
||||
lh.l.WithField("remoteIp", 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 {
|
||||
return false
|
||||
}
|
||||
func (lh *LightHouse) shouldAdd(vpnIp netip.Addr, to netip.Addr) bool {
|
||||
allow := lh.GetRemoteAllowList().Allow(vpnIp, to)
|
||||
if lh.l.Level >= logrus.TraceLevel {
|
||||
lh.l.WithField("remoteIp", vpnIp).WithField("allow", allow).Trace("remoteAllowList.Allow")
|
||||
}
|
||||
if !allow || lh.myVpnNet.Contains(to) {
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// unlockedShouldAddV4 checks if to is allowed by our allow list
|
||||
func (lh *LightHouse) unlockedShouldAddV4(vpnIp iputil.VpnIp, to *Ip4AndPort) bool {
|
||||
allow := lh.GetRemoteAllowList().AllowIpV4(vpnIp, iputil.VpnIp(to.Ip))
|
||||
func (lh *LightHouse) unlockedShouldAddV4(vpnIp netip.Addr, to *Ip4AndPort) bool {
|
||||
ip := AddrPortFromIp4AndPort(to)
|
||||
allow := lh.GetRemoteAllowList().Allow(vpnIp, ip.Addr())
|
||||
if lh.l.Level >= logrus.TraceLevel {
|
||||
lh.l.WithField("remoteIp", vpnIp).WithField("allow", allow).Trace("remoteAllowList.Allow")
|
||||
}
|
||||
|
||||
if !allow || ipMaskContains(lh.myVpnIp, lh.myVpnZeros, iputil.VpnIp(to.Ip)) {
|
||||
if !allow || lh.myVpnNet.Contains(ip.Addr()) {
|
||||
return false
|
||||
}
|
||||
|
||||
@@ -662,14 +644,14 @@ func (lh *LightHouse) unlockedShouldAddV4(vpnIp iputil.VpnIp, to *Ip4AndPort) bo
|
||||
}
|
||||
|
||||
// unlockedShouldAddV6 checks if to is allowed by our allow list
|
||||
func (lh *LightHouse) unlockedShouldAddV6(vpnIp iputil.VpnIp, to *Ip6AndPort) bool {
|
||||
allow := lh.GetRemoteAllowList().AllowIpV6(vpnIp, to.Hi, to.Lo)
|
||||
func (lh *LightHouse) unlockedShouldAddV6(vpnIp netip.Addr, to *Ip6AndPort) bool {
|
||||
ip := AddrPortFromIp6AndPort(to)
|
||||
allow := lh.GetRemoteAllowList().Allow(vpnIp, ip.Addr())
|
||||
if lh.l.Level >= logrus.TraceLevel {
|
||||
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 || lh.myVpnNet.Contains(ip.Addr()) {
|
||||
return false
|
||||
}
|
||||
|
||||
@@ -683,26 +665,39 @@ func lhIp6ToIp(v *Ip6AndPort) net.IP {
|
||||
return ip
|
||||
}
|
||||
|
||||
func (lh *LightHouse) IsLighthouseIP(vpnIp iputil.VpnIp) bool {
|
||||
func (lh *LightHouse) IsLighthouseIP(vpnIp netip.Addr) bool {
|
||||
if _, ok := lh.GetLighthouses()[vpnIp]; ok {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func NewLhQueryByInt(VpnIp iputil.VpnIp) *NebulaMeta {
|
||||
func NewLhQueryByInt(vpnIp netip.Addr) *NebulaMeta {
|
||||
if vpnIp.Is6() {
|
||||
//TODO: need to support ipv6
|
||||
panic("ipv6 is not yet supported")
|
||||
}
|
||||
|
||||
b := vpnIp.As4()
|
||||
return &NebulaMeta{
|
||||
Type: NebulaMeta_HostQuery,
|
||||
Details: &NebulaMetaDetails{
|
||||
VpnIp: uint32(VpnIp),
|
||||
VpnIp: binary.BigEndian.Uint32(b[:]),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func NewIp4AndPort(ip net.IP, port uint32) *Ip4AndPort {
|
||||
ipp := Ip4AndPort{Port: port}
|
||||
ipp.Ip = uint32(iputil.Ip2VpnIp(ip))
|
||||
return &ipp
|
||||
func AddrPortFromIp4AndPort(ip *Ip4AndPort) netip.AddrPort {
|
||||
b := [4]byte{}
|
||||
binary.BigEndian.PutUint32(b[:], ip.Ip)
|
||||
return netip.AddrPortFrom(netip.AddrFrom4(b), uint16(ip.Port))
|
||||
}
|
||||
|
||||
func AddrPortFromIp6AndPort(ip *Ip6AndPort) netip.AddrPort {
|
||||
b := [16]byte{}
|
||||
binary.BigEndian.PutUint64(b[:8], ip.Hi)
|
||||
binary.BigEndian.PutUint64(b[8:], ip.Lo)
|
||||
return netip.AddrPortFrom(netip.AddrFrom16(b), uint16(ip.Port))
|
||||
}
|
||||
|
||||
func NewIp4AndPortFromNetIP(ip netip.Addr, port uint16) *Ip4AndPort {
|
||||
@@ -713,14 +708,7 @@ func NewIp4AndPortFromNetIP(ip netip.Addr, port uint16) *Ip4AndPort {
|
||||
}
|
||||
}
|
||||
|
||||
func NewIp6AndPort(ip net.IP, port uint32) *Ip6AndPort {
|
||||
return &Ip6AndPort{
|
||||
Hi: binary.BigEndian.Uint64(ip[:8]),
|
||||
Lo: binary.BigEndian.Uint64(ip[8:]),
|
||||
Port: port,
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: IPV6-WORK we can delete some more of these
|
||||
func NewIp6AndPortFromNetIP(ip netip.Addr, port uint16) *Ip6AndPort {
|
||||
ip6Addr := ip.As16()
|
||||
return &Ip6AndPort{
|
||||
@@ -729,17 +717,6 @@ func NewIp6AndPortFromNetIP(ip netip.Addr, port uint16) *Ip6AndPort {
|
||||
Port: uint32(port),
|
||||
}
|
||||
}
|
||||
func NewUDPAddrFromLH4(ipp *Ip4AndPort) *udp.Addr {
|
||||
ip := ipp.Ip
|
||||
return udp.NewAddr(
|
||||
net.IPv4(byte(ip&0xff000000>>24), byte(ip&0x00ff0000>>16), byte(ip&0x0000ff00>>8), byte(ip&0x000000ff)),
|
||||
uint16(ipp.Port),
|
||||
)
|
||||
}
|
||||
|
||||
func NewUDPAddrFromLH6(ipp *Ip6AndPort) *udp.Addr {
|
||||
return udp.NewAddr(lhIp6ToIp(ipp), uint16(ipp.Port))
|
||||
}
|
||||
|
||||
func (lh *LightHouse) startQueryWorker() {
|
||||
if lh.amLighthouse {
|
||||
@@ -761,7 +738,7 @@ func (lh *LightHouse) startQueryWorker() {
|
||||
}()
|
||||
}
|
||||
|
||||
func (lh *LightHouse) innerQueryServer(ip iputil.VpnIp, nb, out []byte) {
|
||||
func (lh *LightHouse) innerQueryServer(ip netip.Addr, nb, out []byte) {
|
||||
if lh.IsLighthouseIP(ip) {
|
||||
return
|
||||
}
|
||||
@@ -812,36 +789,41 @@ func (lh *LightHouse) SendUpdate() {
|
||||
var v6 []*Ip6AndPort
|
||||
|
||||
for _, e := range lh.GetAdvertiseAddrs() {
|
||||
if ip := e.ip.To4(); ip != nil {
|
||||
v4 = append(v4, NewIp4AndPort(e.ip, uint32(e.port)))
|
||||
if e.Addr().Is4() {
|
||||
v4 = append(v4, NewIp4AndPortFromNetIP(e.Addr(), e.Port()))
|
||||
} else {
|
||||
v6 = append(v6, NewIp6AndPort(e.ip, uint32(e.port)))
|
||||
v6 = append(v6, NewIp6AndPortFromNetIP(e.Addr(), e.Port()))
|
||||
}
|
||||
}
|
||||
|
||||
lal := lh.GetLocalAllowList()
|
||||
for _, e := range *localIps(lh.l, lal) {
|
||||
if ip4 := e.To4(); ip4 != nil && ipMaskContains(lh.myVpnIp, lh.myVpnZeros, iputil.Ip2VpnIp(ip4)) {
|
||||
for _, e := range localIps(lh.l, lal) {
|
||||
if lh.myVpnNet.Contains(e) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Only add IPs that aren't my VPN/tun IP
|
||||
if ip := e.To4(); ip != nil {
|
||||
v4 = append(v4, NewIp4AndPort(e, lh.nebulaPort))
|
||||
if e.Is4() {
|
||||
v4 = append(v4, NewIp4AndPortFromNetIP(e, uint16(lh.nebulaPort)))
|
||||
} else {
|
||||
v6 = append(v6, NewIp6AndPort(e, lh.nebulaPort))
|
||||
v6 = append(v6, NewIp6AndPortFromNetIP(e, uint16(lh.nebulaPort)))
|
||||
}
|
||||
}
|
||||
|
||||
var relays []uint32
|
||||
for _, r := range lh.GetRelaysForMe() {
|
||||
relays = append(relays, (uint32)(r))
|
||||
//TODO: IPV6-WORK both relays and vpnip need ipv6 support
|
||||
b := r.As4()
|
||||
relays = append(relays, binary.BigEndian.Uint32(b[:]))
|
||||
}
|
||||
|
||||
//TODO: IPV6-WORK both relays and vpnip need ipv6 support
|
||||
b := lh.myVpnNet.Addr().As4()
|
||||
|
||||
m := &NebulaMeta{
|
||||
Type: NebulaMeta_HostUpdateNotification,
|
||||
Details: &NebulaMetaDetails{
|
||||
VpnIp: uint32(lh.myVpnIp),
|
||||
VpnIp: binary.BigEndian.Uint32(b[:]),
|
||||
Ip4AndPorts: v4,
|
||||
Ip6AndPorts: v6,
|
||||
RelayVpnIp: relays,
|
||||
@@ -913,12 +895,12 @@ func (lhh *LightHouseHandler) resetMeta() *NebulaMeta {
|
||||
}
|
||||
|
||||
func lhHandleRequest(lhh *LightHouseHandler, f *Interface) udp.LightHouseHandlerFunc {
|
||||
return func(rAddr *udp.Addr, vpnIp iputil.VpnIp, p []byte) {
|
||||
return func(rAddr netip.AddrPort, vpnIp netip.Addr, p []byte) {
|
||||
lhh.HandleRequest(rAddr, vpnIp, p, f)
|
||||
}
|
||||
}
|
||||
|
||||
func (lhh *LightHouseHandler) HandleRequest(rAddr *udp.Addr, vpnIp iputil.VpnIp, p []byte, w EncWriter) {
|
||||
func (lhh *LightHouseHandler) HandleRequest(rAddr netip.AddrPort, vpnIp netip.Addr, p []byte, w EncWriter) {
|
||||
n := lhh.resetMeta()
|
||||
err := n.Unmarshal(p)
|
||||
if err != nil {
|
||||
@@ -956,7 +938,7 @@ func (lhh *LightHouseHandler) HandleRequest(rAddr *udp.Addr, vpnIp iputil.VpnIp,
|
||||
}
|
||||
}
|
||||
|
||||
func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp iputil.VpnIp, addr *udp.Addr, w EncWriter) {
|
||||
func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp netip.Addr, addr netip.AddrPort, w EncWriter) {
|
||||
// Exit if we don't answer queries
|
||||
if !lhh.lh.amLighthouse {
|
||||
if lhh.l.Level >= logrus.DebugLevel {
|
||||
@@ -967,8 +949,14 @@ func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp iputil.VpnIp,
|
||||
|
||||
//TODO: we can DRY this further
|
||||
reqVpnIp := n.Details.VpnIp
|
||||
|
||||
//TODO: IPV6-WORK
|
||||
b := [4]byte{}
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.VpnIp)
|
||||
queryVpnIp := netip.AddrFrom4(b)
|
||||
|
||||
//TODO: Maybe instead of marshalling into n we marshal into a new `r` to not nuke our current request data
|
||||
found, ln, err := lhh.lh.queryAndPrepMessage(iputil.VpnIp(n.Details.VpnIp), func(c *cache) (int, error) {
|
||||
found, ln, err := lhh.lh.queryAndPrepMessage(queryVpnIp, func(c *cache) (int, error) {
|
||||
n = lhh.resetMeta()
|
||||
n.Type = NebulaMeta_HostQueryReply
|
||||
n.Details.VpnIp = reqVpnIp
|
||||
@@ -994,8 +982,9 @@ func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp iputil.VpnIp,
|
||||
found, ln, err = lhh.lh.queryAndPrepMessage(vpnIp, func(c *cache) (int, error) {
|
||||
n = lhh.resetMeta()
|
||||
n.Type = NebulaMeta_HostPunchNotification
|
||||
n.Details.VpnIp = uint32(vpnIp)
|
||||
|
||||
//TODO: IPV6-WORK
|
||||
b = vpnIp.As4()
|
||||
n.Details.VpnIp = binary.BigEndian.Uint32(b[:])
|
||||
lhh.coalesceAnswers(c, n)
|
||||
|
||||
return n.MarshalTo(lhh.pb)
|
||||
@@ -1011,7 +1000,11 @@ func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp iputil.VpnIp,
|
||||
}
|
||||
|
||||
lhh.lh.metricTx(NebulaMeta_HostPunchNotification, 1)
|
||||
w.SendMessageToVpnIp(header.LightHouse, 0, iputil.VpnIp(reqVpnIp), lhh.pb[:ln], lhh.nb, lhh.out[:0])
|
||||
|
||||
//TODO: IPV6-WORK
|
||||
binary.BigEndian.PutUint32(b[:], reqVpnIp)
|
||||
sendTo := netip.AddrFrom4(b)
|
||||
w.SendMessageToVpnIp(header.LightHouse, 0, sendTo, lhh.pb[:ln], lhh.nb, lhh.out[:0])
|
||||
}
|
||||
|
||||
func (lhh *LightHouseHandler) coalesceAnswers(c *cache, n *NebulaMeta) {
|
||||
@@ -1034,34 +1027,52 @@ func (lhh *LightHouseHandler) coalesceAnswers(c *cache, n *NebulaMeta) {
|
||||
}
|
||||
|
||||
if c.relay != nil {
|
||||
n.Details.RelayVpnIp = append(n.Details.RelayVpnIp, c.relay.relay...)
|
||||
//TODO: IPV6-WORK
|
||||
relays := make([]uint32, len(c.relay.relay))
|
||||
b := [4]byte{}
|
||||
for i, _ := range relays {
|
||||
b = c.relay.relay[i].As4()
|
||||
relays[i] = binary.BigEndian.Uint32(b[:])
|
||||
}
|
||||
n.Details.RelayVpnIp = append(n.Details.RelayVpnIp, relays...)
|
||||
}
|
||||
}
|
||||
|
||||
func (lhh *LightHouseHandler) handleHostQueryReply(n *NebulaMeta, vpnIp iputil.VpnIp) {
|
||||
func (lhh *LightHouseHandler) handleHostQueryReply(n *NebulaMeta, vpnIp netip.Addr) {
|
||||
if !lhh.lh.IsLighthouseIP(vpnIp) {
|
||||
return
|
||||
}
|
||||
|
||||
lhh.lh.Lock()
|
||||
am := lhh.lh.unlockedGetRemoteList(iputil.VpnIp(n.Details.VpnIp))
|
||||
//TODO: IPV6-WORK
|
||||
b := [4]byte{}
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.VpnIp)
|
||||
certVpnIp := netip.AddrFrom4(b)
|
||||
am := lhh.lh.unlockedGetRemoteList(certVpnIp)
|
||||
am.Lock()
|
||||
lhh.lh.Unlock()
|
||||
|
||||
certVpnIp := iputil.VpnIp(n.Details.VpnIp)
|
||||
//TODO: IPV6-WORK
|
||||
am.unlockedSetV4(vpnIp, certVpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
|
||||
am.unlockedSetV6(vpnIp, certVpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
|
||||
am.unlockedSetRelay(vpnIp, certVpnIp, n.Details.RelayVpnIp)
|
||||
|
||||
//TODO: IPV6-WORK
|
||||
relays := make([]netip.Addr, len(n.Details.RelayVpnIp))
|
||||
for i, _ := range n.Details.RelayVpnIp {
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.RelayVpnIp[i])
|
||||
relays[i] = netip.AddrFrom4(b)
|
||||
}
|
||||
am.unlockedSetRelay(vpnIp, certVpnIp, relays)
|
||||
am.Unlock()
|
||||
|
||||
// Non-blocking attempt to trigger, skip if it would block
|
||||
select {
|
||||
case lhh.lh.handshakeTrigger <- iputil.VpnIp(n.Details.VpnIp):
|
||||
case lhh.lh.handshakeTrigger <- certVpnIp:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp iputil.VpnIp, w EncWriter) {
|
||||
func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp netip.Addr, w EncWriter) {
|
||||
if !lhh.lh.amLighthouse {
|
||||
if lhh.l.Level >= logrus.DebugLevel {
|
||||
lhh.l.Debugln("I am not a lighthouse, do not take host updates: ", vpnIp)
|
||||
@@ -1070,9 +1081,13 @@ func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp
|
||||
}
|
||||
|
||||
//Simple check that the host sent this not someone else
|
||||
if n.Details.VpnIp != uint32(vpnIp) {
|
||||
//TODO: IPV6-WORK
|
||||
b := [4]byte{}
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.VpnIp)
|
||||
detailsVpnIp := netip.AddrFrom4(b)
|
||||
if detailsVpnIp != vpnIp {
|
||||
if lhh.l.Level >= logrus.DebugLevel {
|
||||
lhh.l.WithField("vpnIp", vpnIp).WithField("answer", iputil.VpnIp(n.Details.VpnIp)).Debugln("Host sent invalid update")
|
||||
lhh.l.WithField("vpnIp", vpnIp).WithField("answer", detailsVpnIp).Debugln("Host sent invalid update")
|
||||
}
|
||||
return
|
||||
}
|
||||
@@ -1082,15 +1097,24 @@ func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp
|
||||
am.Lock()
|
||||
lhh.lh.Unlock()
|
||||
|
||||
certVpnIp := iputil.VpnIp(n.Details.VpnIp)
|
||||
am.unlockedSetV4(vpnIp, certVpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
|
||||
am.unlockedSetV6(vpnIp, certVpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
|
||||
am.unlockedSetRelay(vpnIp, certVpnIp, n.Details.RelayVpnIp)
|
||||
am.unlockedSetV4(vpnIp, detailsVpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
|
||||
am.unlockedSetV6(vpnIp, detailsVpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
|
||||
|
||||
//TODO: IPV6-WORK
|
||||
relays := make([]netip.Addr, len(n.Details.RelayVpnIp))
|
||||
for i, _ := range n.Details.RelayVpnIp {
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.RelayVpnIp[i])
|
||||
relays[i] = netip.AddrFrom4(b)
|
||||
}
|
||||
am.unlockedSetRelay(vpnIp, detailsVpnIp, relays)
|
||||
am.Unlock()
|
||||
|
||||
n = lhh.resetMeta()
|
||||
n.Type = NebulaMeta_HostUpdateNotificationAck
|
||||
n.Details.VpnIp = uint32(vpnIp)
|
||||
|
||||
//TODO: IPV6-WORK
|
||||
vpnIpB := vpnIp.As4()
|
||||
n.Details.VpnIp = binary.BigEndian.Uint32(vpnIpB[:])
|
||||
ln, err := n.MarshalTo(lhh.pb)
|
||||
|
||||
if err != nil {
|
||||
@@ -1102,14 +1126,14 @@ func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp
|
||||
w.SendMessageToVpnIp(header.LightHouse, 0, vpnIp, lhh.pb[:ln], lhh.nb, lhh.out[:0])
|
||||
}
|
||||
|
||||
func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp iputil.VpnIp, w EncWriter) {
|
||||
func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp netip.Addr, w EncWriter) {
|
||||
if !lhh.lh.IsLighthouseIP(vpnIp) {
|
||||
return
|
||||
}
|
||||
|
||||
empty := []byte{0}
|
||||
punch := func(vpnPeer *udp.Addr) {
|
||||
if vpnPeer == nil {
|
||||
punch := func(vpnPeer netip.AddrPort) {
|
||||
if !vpnPeer.IsValid() {
|
||||
return
|
||||
}
|
||||
|
||||
@@ -1121,23 +1145,29 @@ func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp i
|
||||
|
||||
if lhh.l.Level >= logrus.DebugLevel {
|
||||
//TODO: lacking the ip we are actually punching on, old: l.Debugf("Punching %s on %d for %s", IntIp(a.Ip), a.Port, IntIp(n.Details.VpnIp))
|
||||
lhh.l.Debugf("Punching on %d for %s", vpnPeer.Port, iputil.VpnIp(n.Details.VpnIp))
|
||||
//TODO: IPV6-WORK, make this debug line not suck
|
||||
b := [4]byte{}
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.VpnIp)
|
||||
lhh.l.Debugf("Punching on %d for %v", vpnPeer.Port(), netip.AddrFrom4(b))
|
||||
}
|
||||
}
|
||||
|
||||
for _, a := range n.Details.Ip4AndPorts {
|
||||
punch(NewUDPAddrFromLH4(a))
|
||||
punch(AddrPortFromIp4AndPort(a))
|
||||
}
|
||||
|
||||
for _, a := range n.Details.Ip6AndPorts {
|
||||
punch(NewUDPAddrFromLH6(a))
|
||||
punch(AddrPortFromIp6AndPort(a))
|
||||
}
|
||||
|
||||
// This sends a nebula test packet to the host trying to contact us. In the case
|
||||
// of a double nat or other difficult scenario, this may help establish
|
||||
// a tunnel.
|
||||
if lhh.lh.punchy.GetRespond() {
|
||||
queryVpnIp := iputil.VpnIp(n.Details.VpnIp)
|
||||
//TODO: IPV6-WORK
|
||||
b := [4]byte{}
|
||||
binary.BigEndian.PutUint32(b[:], n.Details.VpnIp)
|
||||
queryVpnIp := netip.AddrFrom4(b)
|
||||
go func() {
|
||||
time.Sleep(lhh.lh.punchy.GetRespondDelay())
|
||||
if lhh.l.Level >= logrus.DebugLevel {
|
||||
@@ -1150,9 +1180,3 @@ func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp i
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
// ipMaskContains checks if testIp is contained by ip after applying a cidr.
|
||||
// zeros is 32 - bits from net.IPMask.Size()
|
||||
func ipMaskContains(ip iputil.VpnIp, zeros iputil.VpnIp, testIp iputil.VpnIp) bool {
|
||||
return (testIp^ip)>>zeros == 0
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user