|  | @@ -9,7 +9,6 @@ import (
 | 
											
												
													
														|  |  	"sync"
 |  |  	"sync"
 | 
											
												
													
														|  |  	"sync/atomic"
 |  |  	"sync/atomic"
 | 
											
												
													
														|  |  	"time"
 |  |  	"time"
 | 
											
												
													
														|  | -	"unsafe"
 |  | 
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	"github.com/rcrowley/go-metrics"
 |  |  	"github.com/rcrowley/go-metrics"
 | 
											
												
													
														|  |  	"github.com/sirupsen/logrus"
 |  |  	"github.com/sirupsen/logrus"
 | 
											
										
											
												
													
														|  | @@ -49,29 +48,29 @@ type LightHouse struct {
 | 
											
												
													
														|  |  	// respond with.
 |  |  	// respond with.
 | 
											
												
													
														|  |  	// - When we are not a lighthouse, this filters which addresses we accept
 |  |  	// - When we are not a lighthouse, this filters which addresses we accept
 | 
											
												
													
														|  |  	// from lighthouses.
 |  |  	// from lighthouses.
 | 
											
												
													
														|  | -	atomicRemoteAllowList *RemoteAllowList
 |  | 
 | 
											
												
													
														|  | 
 |  | +	remoteAllowList atomic.Pointer[RemoteAllowList]
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// filters local addresses that we advertise to lighthouses
 |  |  	// filters local addresses that we advertise to lighthouses
 | 
											
												
													
														|  | -	atomicLocalAllowList *LocalAllowList
 |  | 
 | 
											
												
													
														|  | 
 |  | +	localAllowList atomic.Pointer[LocalAllowList]
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// used to trigger the HandshakeManager when we receive HostQueryReply
 |  |  	// used to trigger the HandshakeManager when we receive HostQueryReply
 | 
											
												
													
														|  |  	handshakeTrigger chan<- iputil.VpnIp
 |  |  	handshakeTrigger chan<- iputil.VpnIp
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	// atomicStaticList exists to avoid having a bool in each addrMap entry
 |  | 
 | 
											
												
													
														|  | 
 |  | +	// staticList exists to avoid having a bool in each addrMap entry
 | 
											
												
													
														|  |  	// since static should be rare
 |  |  	// since static should be rare
 | 
											
												
													
														|  | -	atomicStaticList  map[iputil.VpnIp]struct{}
 |  | 
 | 
											
												
													
														|  | -	atomicLighthouses map[iputil.VpnIp]struct{}
 |  | 
 | 
											
												
													
														|  | 
 |  | +	staticList  atomic.Pointer[map[iputil.VpnIp]struct{}]
 | 
											
												
													
														|  | 
 |  | +	lighthouses atomic.Pointer[map[iputil.VpnIp]struct{}]
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	atomicInterval  int64
 |  | 
 | 
											
												
													
														|  | 
 |  | +	interval        atomic.Int64
 | 
											
												
													
														|  |  	updateCancel    context.CancelFunc
 |  |  	updateCancel    context.CancelFunc
 | 
											
												
													
														|  |  	updateParentCtx context.Context
 |  |  	updateParentCtx context.Context
 | 
											
												
													
														|  |  	updateUdp       udp.EncWriter
 |  |  	updateUdp       udp.EncWriter
 | 
											
												
													
														|  |  	nebulaPort      uint32 // 32 bits because protobuf does not have a uint16
 |  |  	nebulaPort      uint32 // 32 bits because protobuf does not have a uint16
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -	atomicAdvertiseAddrs []netIpAndPort
 |  | 
 | 
											
												
													
														|  | 
 |  | +	advertiseAddrs atomic.Pointer[[]netIpAndPort]
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	// IP's of relays that can be used by peers to access me
 |  |  	// IP's of relays that can be used by peers to access me
 | 
											
												
													
														|  | -	atomicRelaysForMe []iputil.VpnIp
 |  | 
 | 
											
												
													
														|  | 
 |  | +	relaysForMe atomic.Pointer[[]iputil.VpnIp]
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	metrics           *MessageMetrics
 |  |  	metrics           *MessageMetrics
 | 
											
												
													
														|  |  	metricHolepunchTx metrics.Counter
 |  |  	metricHolepunchTx metrics.Counter
 | 
											
										
											
												
													
														|  | @@ -98,18 +97,20 @@ func NewLightHouseFromConfig(l *logrus.Logger, c *config.C, myVpnNet *net.IPNet,
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	ones, _ := myVpnNet.Mask.Size()
 |  |  	ones, _ := myVpnNet.Mask.Size()
 | 
											
												
													
														|  |  	h := LightHouse{
 |  |  	h := LightHouse{
 | 
											
												
													
														|  | -		amLighthouse:      amLighthouse,
 |  | 
 | 
											
												
													
														|  | -		myVpnIp:           iputil.Ip2VpnIp(myVpnNet.IP),
 |  | 
 | 
											
												
													
														|  | -		myVpnZeros:        iputil.VpnIp(32 - ones),
 |  | 
 | 
											
												
													
														|  | -		myVpnNet:          myVpnNet,
 |  | 
 | 
											
												
													
														|  | -		addrMap:           make(map[iputil.VpnIp]*RemoteList),
 |  | 
 | 
											
												
													
														|  | -		nebulaPort:        nebulaPort,
 |  | 
 | 
											
												
													
														|  | -		atomicLighthouses: make(map[iputil.VpnIp]struct{}),
 |  | 
 | 
											
												
													
														|  | -		atomicStaticList:  make(map[iputil.VpnIp]struct{}),
 |  | 
 | 
											
												
													
														|  | -		punchConn:         pc,
 |  | 
 | 
											
												
													
														|  | -		punchy:            p,
 |  | 
 | 
											
												
													
														|  | -		l:                 l,
 |  | 
 | 
											
												
													
														|  | -	}
 |  | 
 | 
											
												
													
														|  | 
 |  | +		amLighthouse: amLighthouse,
 | 
											
												
													
														|  | 
 |  | +		myVpnIp:      iputil.Ip2VpnIp(myVpnNet.IP),
 | 
											
												
													
														|  | 
 |  | +		myVpnZeros:   iputil.VpnIp(32 - ones),
 | 
											
												
													
														|  | 
 |  | +		myVpnNet:     myVpnNet,
 | 
											
												
													
														|  | 
 |  | +		addrMap:      make(map[iputil.VpnIp]*RemoteList),
 | 
											
												
													
														|  | 
 |  | +		nebulaPort:   nebulaPort,
 | 
											
												
													
														|  | 
 |  | +		punchConn:    pc,
 | 
											
												
													
														|  | 
 |  | +		punchy:       p,
 | 
											
												
													
														|  | 
 |  | +		l:            l,
 | 
											
												
													
														|  | 
 |  | +	}
 | 
											
												
													
														|  | 
 |  | +	lighthouses := make(map[iputil.VpnIp]struct{})
 | 
											
												
													
														|  | 
 |  | +	h.lighthouses.Store(&lighthouses)
 | 
											
												
													
														|  | 
 |  | +	staticList := make(map[iputil.VpnIp]struct{})
 | 
											
												
													
														|  | 
 |  | +	h.staticList.Store(&staticList)
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	if c.GetBool("stats.lighthouse_metrics", false) {
 |  |  	if c.GetBool("stats.lighthouse_metrics", false) {
 | 
											
												
													
														|  |  		h.metrics = newLighthouseMetrics()
 |  |  		h.metrics = newLighthouseMetrics()
 | 
											
										
											
												
													
														|  | @@ -137,31 +138,31 @@ func NewLightHouseFromConfig(l *logrus.Logger, c *config.C, myVpnNet *net.IPNet,
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetStaticHostList() map[iputil.VpnIp]struct{} {
 |  |  func (lh *LightHouse) GetStaticHostList() map[iputil.VpnIp]struct{} {
 | 
											
												
													
														|  | -	return *(*map[iputil.VpnIp]struct{})(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicStaticList))))
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return *lh.staticList.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetLighthouses() map[iputil.VpnIp]struct{} {
 |  |  func (lh *LightHouse) GetLighthouses() map[iputil.VpnIp]struct{} {
 | 
											
												
													
														|  | -	return *(*map[iputil.VpnIp]struct{})(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicLighthouses))))
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return *lh.lighthouses.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetRemoteAllowList() *RemoteAllowList {
 |  |  func (lh *LightHouse) GetRemoteAllowList() *RemoteAllowList {
 | 
											
												
													
														|  | -	return (*RemoteAllowList)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicRemoteAllowList))))
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return lh.remoteAllowList.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetLocalAllowList() *LocalAllowList {
 |  |  func (lh *LightHouse) GetLocalAllowList() *LocalAllowList {
 | 
											
												
													
														|  | -	return (*LocalAllowList)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicLocalAllowList))))
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return lh.localAllowList.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetAdvertiseAddrs() []netIpAndPort {
 |  |  func (lh *LightHouse) GetAdvertiseAddrs() []netIpAndPort {
 | 
											
												
													
														|  | -	return *(*[]netIpAndPort)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicAdvertiseAddrs))))
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return *lh.advertiseAddrs.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetRelaysForMe() []iputil.VpnIp {
 |  |  func (lh *LightHouse) GetRelaysForMe() []iputil.VpnIp {
 | 
											
												
													
														|  | -	return *(*[]iputil.VpnIp)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicRelaysForMe))))
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return *lh.relaysForMe.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) GetUpdateInterval() int64 {
 |  |  func (lh *LightHouse) GetUpdateInterval() int64 {
 | 
											
												
													
														|  | -	return atomic.LoadInt64(&lh.atomicInterval)
 |  | 
 | 
											
												
													
														|  | 
 |  | +	return lh.interval.Load()
 | 
											
												
													
														|  |  }
 |  |  }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  func (lh *LightHouse) reload(c *config.C, initial bool) error {
 |  |  func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
										
											
												
													
														|  | @@ -188,7 +189,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  			advAddrs = append(advAddrs, netIpAndPort{ip: fIp, port: fPort})
 |  |  			advAddrs = append(advAddrs, netIpAndPort{ip: fIp, port: fPort})
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicAdvertiseAddrs)), unsafe.Pointer(&advAddrs))
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lh.advertiseAddrs.Store(&advAddrs)
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  		if !initial {
 |  |  		if !initial {
 | 
											
												
													
														|  |  			lh.l.Info("lighthouse.advertise_addrs has changed")
 |  |  			lh.l.Info("lighthouse.advertise_addrs has changed")
 | 
											
										
											
												
													
														|  | @@ -196,10 +197,10 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  	if initial || c.HasChanged("lighthouse.interval") {
 |  |  	if initial || c.HasChanged("lighthouse.interval") {
 | 
											
												
													
														|  | -		atomic.StoreInt64(&lh.atomicInterval, int64(c.GetInt("lighthouse.interval", 10)))
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lh.interval.Store(int64(c.GetInt("lighthouse.interval", 10)))
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  		if !initial {
 |  |  		if !initial {
 | 
											
												
													
														|  | -			lh.l.Infof("lighthouse.interval changed to %v", lh.atomicInterval)
 |  | 
 | 
											
												
													
														|  | 
 |  | +			lh.l.Infof("lighthouse.interval changed to %v", lh.interval.Load())
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  			if lh.updateCancel != nil {
 |  |  			if lh.updateCancel != nil {
 | 
											
												
													
														|  |  				// May not always have a running routine
 |  |  				// May not always have a running routine
 | 
											
										
											
												
													
														|  | @@ -216,7 +217,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  			return util.NewContextualError("Invalid lighthouse.remote_allow_list", nil, err)
 |  |  			return util.NewContextualError("Invalid lighthouse.remote_allow_list", nil, err)
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicRemoteAllowList)), unsafe.Pointer(ral))
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lh.remoteAllowList.Store(ral)
 | 
											
												
													
														|  |  		if !initial {
 |  |  		if !initial {
 | 
											
												
													
														|  |  			//TODO: a diff will be annoyingly difficult
 |  |  			//TODO: a diff will be annoyingly difficult
 | 
											
												
													
														|  |  			lh.l.Info("lighthouse.remote_allow_list and/or lighthouse.remote_allow_ranges has changed")
 |  |  			lh.l.Info("lighthouse.remote_allow_list and/or lighthouse.remote_allow_ranges has changed")
 | 
											
										
											
												
													
														|  | @@ -229,7 +230,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  			return util.NewContextualError("Invalid lighthouse.local_allow_list", nil, err)
 |  |  			return util.NewContextualError("Invalid lighthouse.local_allow_list", nil, err)
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicLocalAllowList)), unsafe.Pointer(lal))
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lh.localAllowList.Store(lal)
 | 
											
												
													
														|  |  		if !initial {
 |  |  		if !initial {
 | 
											
												
													
														|  |  			//TODO: a diff will be annoyingly difficult
 |  |  			//TODO: a diff will be annoyingly difficult
 | 
											
												
													
														|  |  			lh.l.Info("lighthouse.local_allow_list has changed")
 |  |  			lh.l.Info("lighthouse.local_allow_list has changed")
 | 
											
										
											
												
													
														|  | @@ -244,7 +245,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  			return err
 |  |  			return err
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicStaticList)), unsafe.Pointer(&staticList))
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lh.staticList.Store(&staticList)
 | 
											
												
													
														|  |  		if !initial {
 |  |  		if !initial {
 | 
											
												
													
														|  |  			//TODO: we should remove any remote list entries for static hosts that were removed/modified?
 |  |  			//TODO: we should remove any remote list entries for static hosts that were removed/modified?
 | 
											
												
													
														|  |  			lh.l.Info("static_host_map has changed")
 |  |  			lh.l.Info("static_host_map has changed")
 | 
											
										
											
												
													
														|  | @@ -259,7 +260,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  			return err
 |  |  			return err
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -		atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicLighthouses)), unsafe.Pointer(&lhMap))
 |  | 
 | 
											
												
													
														|  | 
 |  | +		lh.lighthouses.Store(&lhMap)
 | 
											
												
													
														|  |  		if !initial {
 |  |  		if !initial {
 | 
											
												
													
														|  |  			//NOTE: we are not tearing down existing lighthouse connections because they might be used for non lighthouse traffic
 |  |  			//NOTE: we are not tearing down existing lighthouse connections because they might be used for non lighthouse traffic
 | 
											
												
													
														|  |  			lh.l.Info("lighthouse.hosts has changed")
 |  |  			lh.l.Info("lighthouse.hosts has changed")
 | 
											
										
											
												
													
														|  | @@ -274,7 +275,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  				lh.l.Info("Ignoring relays from config because am_relay is true")
 |  |  				lh.l.Info("Ignoring relays from config because am_relay is true")
 | 
											
												
													
														|  |  			}
 |  |  			}
 | 
											
												
													
														|  |  			relaysForMe := []iputil.VpnIp{}
 |  |  			relaysForMe := []iputil.VpnIp{}
 | 
											
												
													
														|  | -			atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicRelaysForMe)), unsafe.Pointer(&relaysForMe))
 |  | 
 | 
											
												
													
														|  | 
 |  | +			lh.relaysForMe.Store(&relaysForMe)
 | 
											
												
													
														|  |  		case false:
 |  |  		case false:
 | 
											
												
													
														|  |  			relaysForMe := []iputil.VpnIp{}
 |  |  			relaysForMe := []iputil.VpnIp{}
 | 
											
												
													
														|  |  			for _, v := range c.GetStringSlice("relay.relays", nil) {
 |  |  			for _, v := range c.GetStringSlice("relay.relays", nil) {
 | 
											
										
											
												
													
														|  | @@ -285,7 +286,7 @@ func (lh *LightHouse) reload(c *config.C, initial bool) error {
 | 
											
												
													
														|  |  					relaysForMe = append(relaysForMe, iputil.Ip2VpnIp(configRIP))
 |  |  					relaysForMe = append(relaysForMe, iputil.Ip2VpnIp(configRIP))
 | 
											
												
													
														|  |  				}
 |  |  				}
 | 
											
												
													
														|  |  			}
 |  |  			}
 | 
											
												
													
														|  | -			atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&lh.atomicRelaysForMe)), unsafe.Pointer(&relaysForMe))
 |  | 
 | 
											
												
													
														|  | 
 |  | +			lh.relaysForMe.Store(&relaysForMe)
 | 
											
												
													
														|  |  		}
 |  |  		}
 | 
											
												
													
														|  |  	}
 |  |  	}
 | 
											
												
													
														|  |  
 |  |  
 | 
											
										
											
												
													
														|  | @@ -460,7 +461,7 @@ func (lh *LightHouse) DeleteVpnIp(vpnIp iputil.VpnIp) {
 | 
											
												
													
														|  |  // AddStaticRemote adds a static host entry for vpnIp as ourselves as the owner
 |  |  // 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
 |  |  // 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
 |  |  // 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
 |  | 
 | 
											
												
													
														|  | 
 |  | +// NOTE: this function should not interact with any hot path objects, like lh.staticList, the caller should handle it
 | 
											
												
													
														|  |  func (lh *LightHouse) addStaticRemote(vpnIp iputil.VpnIp, toAddr *udp.Addr, staticList map[iputil.VpnIp]struct{}) {
 |  |  func (lh *LightHouse) addStaticRemote(vpnIp iputil.VpnIp, toAddr *udp.Addr, staticList map[iputil.VpnIp]struct{}) {
 | 
											
												
													
														|  |  	lh.Lock()
 |  |  	lh.Lock()
 | 
											
												
													
														|  |  	am := lh.unlockedGetRemoteList(vpnIp)
 |  |  	am := lh.unlockedGetRemoteList(vpnIp)
 |