Switch most everything to netip in prep for ipv6 in the overlay (#1173)

This commit is contained in:
Nate Brown
2024-07-31 10:18:56 -05:00
committed by GitHub
parent 00458302ca
commit e264a0ff88
79 changed files with 1900 additions and 2682 deletions

View File

@@ -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
}