lighthouse.go 31 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130
  1. package nebula
  2. import (
  3. "context"
  4. "encoding/binary"
  5. "errors"
  6. "fmt"
  7. "net"
  8. "net/netip"
  9. "sync"
  10. "sync/atomic"
  11. "time"
  12. "github.com/rcrowley/go-metrics"
  13. "github.com/sirupsen/logrus"
  14. "github.com/slackhq/nebula/cidr"
  15. "github.com/slackhq/nebula/config"
  16. "github.com/slackhq/nebula/header"
  17. "github.com/slackhq/nebula/iputil"
  18. "github.com/slackhq/nebula/udp"
  19. "github.com/slackhq/nebula/util"
  20. )
  21. //TODO: if a lighthouse doesn't have an answer, clients AGGRESSIVELY REQUERY.. why? handshake manager and/or getOrHandshake?
  22. //TODO: nodes are roaming lighthouses, this is bad. How are they learning?
  23. var ErrHostNotKnown = errors.New("host not known")
  24. type netIpAndPort struct {
  25. ip net.IP
  26. port uint16
  27. }
  28. type LightHouse struct {
  29. //TODO: We need a timer wheel to kick out vpnIps that haven't reported in a long time
  30. sync.RWMutex //Because we concurrently read and write to our maps
  31. ctx context.Context
  32. amLighthouse bool
  33. myVpnIp iputil.VpnIp
  34. myVpnZeros iputil.VpnIp
  35. myVpnNet *net.IPNet
  36. punchConn udp.Conn
  37. punchy *Punchy
  38. // Local cache of answers from light houses
  39. // map of vpn Ip to answers
  40. addrMap map[iputil.VpnIp]*RemoteList
  41. // filters remote addresses allowed for each host
  42. // - When we are a lighthouse, this filters what addresses we store and
  43. // respond with.
  44. // - When we are not a lighthouse, this filters which addresses we accept
  45. // from lighthouses.
  46. remoteAllowList atomic.Pointer[RemoteAllowList]
  47. // filters local addresses that we advertise to lighthouses
  48. localAllowList atomic.Pointer[LocalAllowList]
  49. // used to trigger the HandshakeManager when we receive HostQueryReply
  50. handshakeTrigger chan<- iputil.VpnIp
  51. // staticList exists to avoid having a bool in each addrMap entry
  52. // since static should be rare
  53. staticList atomic.Pointer[map[iputil.VpnIp]struct{}]
  54. lighthouses atomic.Pointer[map[iputil.VpnIp]struct{}]
  55. interval atomic.Int64
  56. updateCancel context.CancelFunc
  57. ifce EncWriter
  58. nebulaPort uint32 // 32 bits because protobuf does not have a uint16
  59. advertiseAddrs atomic.Pointer[[]netIpAndPort]
  60. // IP's of relays that can be used by peers to access me
  61. relaysForMe atomic.Pointer[[]iputil.VpnIp]
  62. calculatedRemotes atomic.Pointer[cidr.Tree4] // Maps VpnIp to []*calculatedRemote
  63. metrics *MessageMetrics
  64. metricHolepunchTx metrics.Counter
  65. l *logrus.Logger
  66. }
  67. // NewLightHouseFromConfig will build a Lighthouse struct from the values provided in the config object
  68. // addrMap should be nil unless this is during a config reload
  69. func NewLightHouseFromConfig(ctx context.Context, l *logrus.Logger, c *config.C, myVpnNet *net.IPNet, pc udp.Conn, p *Punchy) (*LightHouse, error) {
  70. amLighthouse := c.GetBool("lighthouse.am_lighthouse", false)
  71. nebulaPort := uint32(c.GetInt("listen.port", 0))
  72. if amLighthouse && nebulaPort == 0 {
  73. return nil, util.NewContextualError("lighthouse.am_lighthouse enabled on node but no port number is set in config", nil, nil)
  74. }
  75. // If port is dynamic, discover it
  76. if nebulaPort == 0 && pc != nil {
  77. uPort, err := pc.LocalAddr()
  78. if err != nil {
  79. return nil, util.NewContextualError("Failed to get listening port", nil, err)
  80. }
  81. nebulaPort = uint32(uPort.Port)
  82. }
  83. ones, _ := myVpnNet.Mask.Size()
  84. h := LightHouse{
  85. ctx: ctx,
  86. amLighthouse: amLighthouse,
  87. myVpnIp: iputil.Ip2VpnIp(myVpnNet.IP),
  88. myVpnZeros: iputil.VpnIp(32 - ones),
  89. myVpnNet: myVpnNet,
  90. addrMap: make(map[iputil.VpnIp]*RemoteList),
  91. nebulaPort: nebulaPort,
  92. punchConn: pc,
  93. punchy: p,
  94. l: l,
  95. }
  96. lighthouses := make(map[iputil.VpnIp]struct{})
  97. h.lighthouses.Store(&lighthouses)
  98. staticList := make(map[iputil.VpnIp]struct{})
  99. h.staticList.Store(&staticList)
  100. if c.GetBool("stats.lighthouse_metrics", false) {
  101. h.metrics = newLighthouseMetrics()
  102. h.metricHolepunchTx = metrics.GetOrRegisterCounter("messages.tx.holepunch", nil)
  103. } else {
  104. h.metricHolepunchTx = metrics.NilCounter{}
  105. }
  106. err := h.reload(c, true)
  107. if err != nil {
  108. return nil, err
  109. }
  110. c.RegisterReloadCallback(func(c *config.C) {
  111. err := h.reload(c, false)
  112. switch v := err.(type) {
  113. case util.ContextualError:
  114. v.Log(l)
  115. case error:
  116. l.WithError(err).Error("failed to reload lighthouse")
  117. }
  118. })
  119. return &h, nil
  120. }
  121. func (lh *LightHouse) GetStaticHostList() map[iputil.VpnIp]struct{} {
  122. return *lh.staticList.Load()
  123. }
  124. func (lh *LightHouse) GetLighthouses() map[iputil.VpnIp]struct{} {
  125. return *lh.lighthouses.Load()
  126. }
  127. func (lh *LightHouse) GetRemoteAllowList() *RemoteAllowList {
  128. return lh.remoteAllowList.Load()
  129. }
  130. func (lh *LightHouse) GetLocalAllowList() *LocalAllowList {
  131. return lh.localAllowList.Load()
  132. }
  133. func (lh *LightHouse) GetAdvertiseAddrs() []netIpAndPort {
  134. return *lh.advertiseAddrs.Load()
  135. }
  136. func (lh *LightHouse) GetRelaysForMe() []iputil.VpnIp {
  137. return *lh.relaysForMe.Load()
  138. }
  139. func (lh *LightHouse) getCalculatedRemotes() *cidr.Tree4 {
  140. return lh.calculatedRemotes.Load()
  141. }
  142. func (lh *LightHouse) GetUpdateInterval() int64 {
  143. return lh.interval.Load()
  144. }
  145. func (lh *LightHouse) reload(c *config.C, initial bool) error {
  146. if initial || c.HasChanged("lighthouse.advertise_addrs") {
  147. rawAdvAddrs := c.GetStringSlice("lighthouse.advertise_addrs", []string{})
  148. advAddrs := make([]netIpAndPort, 0)
  149. for i, rawAddr := range rawAdvAddrs {
  150. fIp, fPort, err := udp.ParseIPAndPort(rawAddr)
  151. if err != nil {
  152. return util.NewContextualError("Unable to parse lighthouse.advertise_addrs entry", m{"addr": rawAddr, "entry": i + 1}, err)
  153. }
  154. if fPort == 0 {
  155. fPort = uint16(lh.nebulaPort)
  156. }
  157. if ip4 := fIp.To4(); ip4 != nil && lh.myVpnNet.Contains(fIp) {
  158. lh.l.WithField("addr", rawAddr).WithField("entry", i+1).
  159. Warn("Ignoring lighthouse.advertise_addrs report because it is within the nebula network range")
  160. continue
  161. }
  162. advAddrs = append(advAddrs, netIpAndPort{ip: fIp, port: fPort})
  163. }
  164. lh.advertiseAddrs.Store(&advAddrs)
  165. if !initial {
  166. lh.l.Info("lighthouse.advertise_addrs has changed")
  167. }
  168. }
  169. if initial || c.HasChanged("lighthouse.interval") {
  170. lh.interval.Store(int64(c.GetInt("lighthouse.interval", 10)))
  171. if !initial {
  172. lh.l.Infof("lighthouse.interval changed to %v", lh.interval.Load())
  173. if lh.updateCancel != nil {
  174. // May not always have a running routine
  175. lh.updateCancel()
  176. }
  177. lh.StartUpdateWorker()
  178. }
  179. }
  180. if initial || c.HasChanged("lighthouse.remote_allow_list") || c.HasChanged("lighthouse.remote_allow_ranges") {
  181. ral, err := NewRemoteAllowListFromConfig(c, "lighthouse.remote_allow_list", "lighthouse.remote_allow_ranges")
  182. if err != nil {
  183. return util.NewContextualError("Invalid lighthouse.remote_allow_list", nil, err)
  184. }
  185. lh.remoteAllowList.Store(ral)
  186. if !initial {
  187. //TODO: a diff will be annoyingly difficult
  188. lh.l.Info("lighthouse.remote_allow_list and/or lighthouse.remote_allow_ranges has changed")
  189. }
  190. }
  191. if initial || c.HasChanged("lighthouse.local_allow_list") {
  192. lal, err := NewLocalAllowListFromConfig(c, "lighthouse.local_allow_list")
  193. if err != nil {
  194. return util.NewContextualError("Invalid lighthouse.local_allow_list", nil, err)
  195. }
  196. lh.localAllowList.Store(lal)
  197. if !initial {
  198. //TODO: a diff will be annoyingly difficult
  199. lh.l.Info("lighthouse.local_allow_list has changed")
  200. }
  201. }
  202. if initial || c.HasChanged("lighthouse.calculated_remotes") {
  203. cr, err := NewCalculatedRemotesFromConfig(c, "lighthouse.calculated_remotes")
  204. if err != nil {
  205. return util.NewContextualError("Invalid lighthouse.calculated_remotes", nil, err)
  206. }
  207. lh.calculatedRemotes.Store(cr)
  208. if !initial {
  209. //TODO: a diff will be annoyingly difficult
  210. lh.l.Info("lighthouse.calculated_remotes has changed")
  211. }
  212. }
  213. //NOTE: many things will get much simpler when we combine static_host_map and lighthouse.hosts in config
  214. if initial || c.HasChanged("static_host_map") || c.HasChanged("static_map.cadence") || c.HasChanged("static_map.network") || c.HasChanged("static_map.lookup_timeout") {
  215. // Clean up. Entries still in the static_host_map will be re-built.
  216. // Entries no longer present must have their (possible) background DNS goroutines stopped.
  217. if existingStaticList := lh.staticList.Load(); existingStaticList != nil {
  218. lh.RLock()
  219. for staticVpnIp := range *existingStaticList {
  220. if am, ok := lh.addrMap[staticVpnIp]; ok && am != nil {
  221. am.hr.Cancel()
  222. }
  223. }
  224. lh.RUnlock()
  225. }
  226. // Build a new list based on current config.
  227. staticList := make(map[iputil.VpnIp]struct{})
  228. err := lh.loadStaticMap(c, lh.myVpnNet, staticList)
  229. if err != nil {
  230. return err
  231. }
  232. lh.staticList.Store(&staticList)
  233. if !initial {
  234. //TODO: we should remove any remote list entries for static hosts that were removed/modified?
  235. if c.HasChanged("static_host_map") {
  236. lh.l.Info("static_host_map has changed")
  237. }
  238. if c.HasChanged("static_map.cadence") {
  239. lh.l.Info("static_map.cadence has changed")
  240. }
  241. if c.HasChanged("static_map.network") {
  242. lh.l.Info("static_map.network has changed")
  243. }
  244. if c.HasChanged("static_map.lookup_timeout") {
  245. lh.l.Info("static_map.lookup_timeout has changed")
  246. }
  247. }
  248. }
  249. if initial || c.HasChanged("lighthouse.hosts") {
  250. lhMap := make(map[iputil.VpnIp]struct{})
  251. err := lh.parseLighthouses(c, lh.myVpnNet, lhMap)
  252. if err != nil {
  253. return err
  254. }
  255. lh.lighthouses.Store(&lhMap)
  256. if !initial {
  257. //NOTE: we are not tearing down existing lighthouse connections because they might be used for non lighthouse traffic
  258. lh.l.Info("lighthouse.hosts has changed")
  259. }
  260. }
  261. if initial || c.HasChanged("relay.relays") {
  262. switch c.GetBool("relay.am_relay", false) {
  263. case true:
  264. // Relays aren't allowed to specify other relays
  265. if len(c.GetStringSlice("relay.relays", nil)) > 0 {
  266. lh.l.Info("Ignoring relays from config because am_relay is true")
  267. }
  268. relaysForMe := []iputil.VpnIp{}
  269. lh.relaysForMe.Store(&relaysForMe)
  270. case false:
  271. relaysForMe := []iputil.VpnIp{}
  272. for _, v := range c.GetStringSlice("relay.relays", nil) {
  273. lh.l.WithField("relay", v).Info("Read relay from config")
  274. configRIP := net.ParseIP(v)
  275. if configRIP != nil {
  276. relaysForMe = append(relaysForMe, iputil.Ip2VpnIp(configRIP))
  277. }
  278. }
  279. lh.relaysForMe.Store(&relaysForMe)
  280. }
  281. }
  282. return nil
  283. }
  284. func (lh *LightHouse) parseLighthouses(c *config.C, tunCidr *net.IPNet, lhMap map[iputil.VpnIp]struct{}) error {
  285. lhs := c.GetStringSlice("lighthouse.hosts", []string{})
  286. if lh.amLighthouse && len(lhs) != 0 {
  287. lh.l.Warn("lighthouse.am_lighthouse enabled on node but upstream lighthouses exist in config")
  288. }
  289. for i, host := range lhs {
  290. ip := net.ParseIP(host)
  291. if ip == nil {
  292. return util.NewContextualError("Unable to parse lighthouse host entry", m{"host": host, "entry": i + 1}, nil)
  293. }
  294. if !tunCidr.Contains(ip) {
  295. return util.NewContextualError("lighthouse host is not in our subnet, invalid", m{"vpnIp": ip, "network": tunCidr.String()}, nil)
  296. }
  297. lhMap[iputil.Ip2VpnIp(ip)] = struct{}{}
  298. }
  299. if !lh.amLighthouse && len(lhMap) == 0 {
  300. lh.l.Warn("No lighthouse.hosts configured, this host will only be able to initiate tunnels with static_host_map entries")
  301. }
  302. staticList := lh.GetStaticHostList()
  303. for lhIP, _ := range lhMap {
  304. if _, ok := staticList[lhIP]; !ok {
  305. return fmt.Errorf("lighthouse %s does not have a static_host_map entry", lhIP)
  306. }
  307. }
  308. return nil
  309. }
  310. func getStaticMapCadence(c *config.C) (time.Duration, error) {
  311. cadence := c.GetString("static_map.cadence", "30s")
  312. d, err := time.ParseDuration(cadence)
  313. if err != nil {
  314. return 0, err
  315. }
  316. return d, nil
  317. }
  318. func getStaticMapLookupTimeout(c *config.C) (time.Duration, error) {
  319. lookupTimeout := c.GetString("static_map.lookup_timeout", "250ms")
  320. d, err := time.ParseDuration(lookupTimeout)
  321. if err != nil {
  322. return 0, err
  323. }
  324. return d, nil
  325. }
  326. func getStaticMapNetwork(c *config.C) (string, error) {
  327. network := c.GetString("static_map.network", "ip4")
  328. if network != "ip" && network != "ip4" && network != "ip6" {
  329. return "", fmt.Errorf("static_map.network must be one of ip, ip4, or ip6")
  330. }
  331. return network, nil
  332. }
  333. func (lh *LightHouse) loadStaticMap(c *config.C, tunCidr *net.IPNet, staticList map[iputil.VpnIp]struct{}) error {
  334. d, err := getStaticMapCadence(c)
  335. if err != nil {
  336. return err
  337. }
  338. network, err := getStaticMapNetwork(c)
  339. if err != nil {
  340. return err
  341. }
  342. lookup_timeout, err := getStaticMapLookupTimeout(c)
  343. if err != nil {
  344. return err
  345. }
  346. shm := c.GetMap("static_host_map", map[interface{}]interface{}{})
  347. i := 0
  348. for k, v := range shm {
  349. rip := net.ParseIP(fmt.Sprintf("%v", k))
  350. if rip == nil {
  351. return util.NewContextualError("Unable to parse static_host_map entry", m{"host": k, "entry": i + 1}, nil)
  352. }
  353. if !tunCidr.Contains(rip) {
  354. return util.NewContextualError("static_host_map key is not in our subnet, invalid", m{"vpnIp": rip, "network": tunCidr.String(), "entry": i + 1}, nil)
  355. }
  356. vpnIp := iputil.Ip2VpnIp(rip)
  357. vals, ok := v.([]interface{})
  358. if !ok {
  359. vals = []interface{}{v}
  360. }
  361. remoteAddrs := []string{}
  362. for _, v := range vals {
  363. remoteAddrs = append(remoteAddrs, fmt.Sprintf("%v", v))
  364. }
  365. err := lh.addStaticRemotes(i, d, network, lookup_timeout, vpnIp, remoteAddrs, staticList)
  366. if err != nil {
  367. return err
  368. }
  369. i++
  370. }
  371. return nil
  372. }
  373. func (lh *LightHouse) Query(ip iputil.VpnIp, f EncWriter) *RemoteList {
  374. if !lh.IsLighthouseIP(ip) {
  375. lh.QueryServer(ip, f)
  376. }
  377. lh.RLock()
  378. if v, ok := lh.addrMap[ip]; ok {
  379. lh.RUnlock()
  380. return v
  381. }
  382. lh.RUnlock()
  383. return nil
  384. }
  385. // This is asynchronous so no reply should be expected
  386. func (lh *LightHouse) QueryServer(ip iputil.VpnIp, f EncWriter) {
  387. if lh.amLighthouse {
  388. return
  389. }
  390. if lh.IsLighthouseIP(ip) {
  391. return
  392. }
  393. // Send a query to the lighthouses and hope for the best next time
  394. query, err := NewLhQueryByInt(ip).Marshal()
  395. if err != nil {
  396. lh.l.WithError(err).WithField("vpnIp", ip).Error("Failed to marshal lighthouse query payload")
  397. return
  398. }
  399. lighthouses := lh.GetLighthouses()
  400. lh.metricTx(NebulaMeta_HostQuery, int64(len(lighthouses)))
  401. nb := make([]byte, 12, 12)
  402. out := make([]byte, mtu)
  403. for n := range lighthouses {
  404. f.SendMessageToVpnIp(header.LightHouse, 0, n, query, nb, out)
  405. }
  406. }
  407. func (lh *LightHouse) QueryCache(ip iputil.VpnIp) *RemoteList {
  408. lh.RLock()
  409. if v, ok := lh.addrMap[ip]; ok {
  410. lh.RUnlock()
  411. return v
  412. }
  413. lh.RUnlock()
  414. lh.Lock()
  415. defer lh.Unlock()
  416. // Add an entry if we don't already have one
  417. return lh.unlockedGetRemoteList(ip)
  418. }
  419. // queryAndPrepMessage is a lock helper on RemoteList, assisting the caller to build a lighthouse message containing
  420. // details from the remote list. It looks for a hit in the addrMap and a hit in the RemoteList under the owner vpnIp
  421. // If one is found then f() is called with proper locking, f() must return result of n.MarshalTo()
  422. func (lh *LightHouse) queryAndPrepMessage(vpnIp iputil.VpnIp, f func(*cache) (int, error)) (bool, int, error) {
  423. lh.RLock()
  424. // Do we have an entry in the main cache?
  425. if v, ok := lh.addrMap[vpnIp]; ok {
  426. // Swap lh lock for remote list lock
  427. v.RLock()
  428. defer v.RUnlock()
  429. lh.RUnlock()
  430. // vpnIp should also be the owner here since we are a lighthouse.
  431. c := v.cache[vpnIp]
  432. // Make sure we have
  433. if c != nil {
  434. n, err := f(c)
  435. return true, n, err
  436. }
  437. return false, 0, nil
  438. }
  439. lh.RUnlock()
  440. return false, 0, nil
  441. }
  442. func (lh *LightHouse) DeleteVpnIp(vpnIp iputil.VpnIp) {
  443. // First we check the static mapping
  444. // and do nothing if it is there
  445. if _, ok := lh.GetStaticHostList()[vpnIp]; ok {
  446. return
  447. }
  448. lh.Lock()
  449. //l.Debugln(lh.addrMap)
  450. delete(lh.addrMap, vpnIp)
  451. if lh.l.Level >= logrus.DebugLevel {
  452. lh.l.Debugf("deleting %s from lighthouse.", vpnIp)
  453. }
  454. lh.Unlock()
  455. }
  456. // AddStaticRemote adds a static host entry for vpnIp as ourselves as the owner
  457. // We are the owner because we don't want a lighthouse server to advertise for static hosts it was configured with
  458. // And we don't want a lighthouse query reply to interfere with our learned cache if we are a client
  459. // NOTE: this function should not interact with any hot path objects, like lh.staticList, the caller should handle it
  460. 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 {
  461. lh.Lock()
  462. am := lh.unlockedGetRemoteList(vpnIp)
  463. am.Lock()
  464. defer am.Unlock()
  465. ctx := lh.ctx
  466. lh.Unlock()
  467. hr, err := NewHostnameResults(ctx, lh.l, d, network, timeout, toAddrs, func() {
  468. // This callback runs whenever the DNS hostname resolver finds a different set of IP's
  469. // in its resolution for hostnames.
  470. am.Lock()
  471. defer am.Unlock()
  472. am.shouldRebuild = true
  473. })
  474. if err != nil {
  475. return util.NewContextualError("Static host address could not be parsed", m{"vpnIp": vpnIp, "entry": i + 1}, err)
  476. }
  477. am.unlockedSetHostnamesResults(hr)
  478. for _, addrPort := range hr.GetIPs() {
  479. switch {
  480. case addrPort.Addr().Is4():
  481. to := NewIp4AndPortFromNetIP(addrPort.Addr(), addrPort.Port())
  482. if !lh.unlockedShouldAddV4(vpnIp, to) {
  483. continue
  484. }
  485. am.unlockedPrependV4(lh.myVpnIp, to)
  486. case addrPort.Addr().Is6():
  487. to := NewIp6AndPortFromNetIP(addrPort.Addr(), addrPort.Port())
  488. if !lh.unlockedShouldAddV6(vpnIp, to) {
  489. continue
  490. }
  491. am.unlockedPrependV6(lh.myVpnIp, to)
  492. }
  493. }
  494. // Mark it as static in the caller provided map
  495. staticList[vpnIp] = struct{}{}
  496. return nil
  497. }
  498. // addCalculatedRemotes adds any calculated remotes based on the
  499. // lighthouse.calculated_remotes configuration. It returns true if any
  500. // calculated remotes were added
  501. func (lh *LightHouse) addCalculatedRemotes(vpnIp iputil.VpnIp) bool {
  502. tree := lh.getCalculatedRemotes()
  503. if tree == nil {
  504. return false
  505. }
  506. value := tree.MostSpecificContains(vpnIp)
  507. if value == nil {
  508. return false
  509. }
  510. calculatedRemotes := value.([]*calculatedRemote)
  511. var calculated []*Ip4AndPort
  512. for _, cr := range calculatedRemotes {
  513. c := cr.Apply(vpnIp)
  514. if c != nil {
  515. calculated = append(calculated, c)
  516. }
  517. }
  518. lh.Lock()
  519. am := lh.unlockedGetRemoteList(vpnIp)
  520. am.Lock()
  521. defer am.Unlock()
  522. lh.Unlock()
  523. am.unlockedSetV4(lh.myVpnIp, vpnIp, calculated, lh.unlockedShouldAddV4)
  524. return len(calculated) > 0
  525. }
  526. // unlockedGetRemoteList assumes you have the lh lock
  527. func (lh *LightHouse) unlockedGetRemoteList(vpnIp iputil.VpnIp) *RemoteList {
  528. am, ok := lh.addrMap[vpnIp]
  529. if !ok {
  530. am = NewRemoteList(func(a netip.Addr) bool { return lh.shouldAdd(vpnIp, a) })
  531. lh.addrMap[vpnIp] = am
  532. }
  533. return am
  534. }
  535. func (lh *LightHouse) shouldAdd(vpnIp iputil.VpnIp, to netip.Addr) bool {
  536. switch {
  537. case to.Is4():
  538. ipBytes := to.As4()
  539. ip := iputil.Ip2VpnIp(ipBytes[:])
  540. allow := lh.GetRemoteAllowList().AllowIpV4(vpnIp, ip)
  541. if lh.l.Level >= logrus.TraceLevel {
  542. lh.l.WithField("remoteIp", vpnIp).WithField("allow", allow).Trace("remoteAllowList.Allow")
  543. }
  544. if !allow || ipMaskContains(lh.myVpnIp, lh.myVpnZeros, ip) {
  545. return false
  546. }
  547. case to.Is6():
  548. ipBytes := to.As16()
  549. hi := binary.BigEndian.Uint64(ipBytes[:8])
  550. lo := binary.BigEndian.Uint64(ipBytes[8:])
  551. allow := lh.GetRemoteAllowList().AllowIpV6(vpnIp, hi, lo)
  552. if lh.l.Level >= logrus.TraceLevel {
  553. lh.l.WithField("remoteIp", to).WithField("allow", allow).Trace("remoteAllowList.Allow")
  554. }
  555. // We don't check our vpn network here because nebula does not support ipv6 on the inside
  556. if !allow {
  557. return false
  558. }
  559. }
  560. return true
  561. }
  562. // unlockedShouldAddV4 checks if to is allowed by our allow list
  563. func (lh *LightHouse) unlockedShouldAddV4(vpnIp iputil.VpnIp, to *Ip4AndPort) bool {
  564. allow := lh.GetRemoteAllowList().AllowIpV4(vpnIp, iputil.VpnIp(to.Ip))
  565. if lh.l.Level >= logrus.TraceLevel {
  566. lh.l.WithField("remoteIp", vpnIp).WithField("allow", allow).Trace("remoteAllowList.Allow")
  567. }
  568. if !allow || ipMaskContains(lh.myVpnIp, lh.myVpnZeros, iputil.VpnIp(to.Ip)) {
  569. return false
  570. }
  571. return true
  572. }
  573. // unlockedShouldAddV6 checks if to is allowed by our allow list
  574. func (lh *LightHouse) unlockedShouldAddV6(vpnIp iputil.VpnIp, to *Ip6AndPort) bool {
  575. allow := lh.GetRemoteAllowList().AllowIpV6(vpnIp, to.Hi, to.Lo)
  576. if lh.l.Level >= logrus.TraceLevel {
  577. lh.l.WithField("remoteIp", lhIp6ToIp(to)).WithField("allow", allow).Trace("remoteAllowList.Allow")
  578. }
  579. // We don't check our vpn network here because nebula does not support ipv6 on the inside
  580. if !allow {
  581. return false
  582. }
  583. return true
  584. }
  585. func lhIp6ToIp(v *Ip6AndPort) net.IP {
  586. ip := make(net.IP, 16)
  587. binary.BigEndian.PutUint64(ip[:8], v.Hi)
  588. binary.BigEndian.PutUint64(ip[8:], v.Lo)
  589. return ip
  590. }
  591. func (lh *LightHouse) IsLighthouseIP(vpnIp iputil.VpnIp) bool {
  592. if _, ok := lh.GetLighthouses()[vpnIp]; ok {
  593. return true
  594. }
  595. return false
  596. }
  597. func NewLhQueryByInt(VpnIp iputil.VpnIp) *NebulaMeta {
  598. return &NebulaMeta{
  599. Type: NebulaMeta_HostQuery,
  600. Details: &NebulaMetaDetails{
  601. VpnIp: uint32(VpnIp),
  602. },
  603. }
  604. }
  605. func NewIp4AndPort(ip net.IP, port uint32) *Ip4AndPort {
  606. ipp := Ip4AndPort{Port: port}
  607. ipp.Ip = uint32(iputil.Ip2VpnIp(ip))
  608. return &ipp
  609. }
  610. func NewIp4AndPortFromNetIP(ip netip.Addr, port uint16) *Ip4AndPort {
  611. v4Addr := ip.As4()
  612. return &Ip4AndPort{
  613. Ip: binary.BigEndian.Uint32(v4Addr[:]),
  614. Port: uint32(port),
  615. }
  616. }
  617. func NewIp6AndPort(ip net.IP, port uint32) *Ip6AndPort {
  618. return &Ip6AndPort{
  619. Hi: binary.BigEndian.Uint64(ip[:8]),
  620. Lo: binary.BigEndian.Uint64(ip[8:]),
  621. Port: port,
  622. }
  623. }
  624. func NewIp6AndPortFromNetIP(ip netip.Addr, port uint16) *Ip6AndPort {
  625. ip6Addr := ip.As16()
  626. return &Ip6AndPort{
  627. Hi: binary.BigEndian.Uint64(ip6Addr[:8]),
  628. Lo: binary.BigEndian.Uint64(ip6Addr[8:]),
  629. Port: uint32(port),
  630. }
  631. }
  632. func NewUDPAddrFromLH4(ipp *Ip4AndPort) *udp.Addr {
  633. ip := ipp.Ip
  634. return udp.NewAddr(
  635. net.IPv4(byte(ip&0xff000000>>24), byte(ip&0x00ff0000>>16), byte(ip&0x0000ff00>>8), byte(ip&0x000000ff)),
  636. uint16(ipp.Port),
  637. )
  638. }
  639. func NewUDPAddrFromLH6(ipp *Ip6AndPort) *udp.Addr {
  640. return udp.NewAddr(lhIp6ToIp(ipp), uint16(ipp.Port))
  641. }
  642. func (lh *LightHouse) StartUpdateWorker() {
  643. interval := lh.GetUpdateInterval()
  644. if lh.amLighthouse || interval == 0 {
  645. return
  646. }
  647. clockSource := time.NewTicker(time.Second * time.Duration(interval))
  648. updateCtx, cancel := context.WithCancel(lh.ctx)
  649. lh.updateCancel = cancel
  650. go func() {
  651. defer clockSource.Stop()
  652. for {
  653. lh.SendUpdate()
  654. select {
  655. case <-updateCtx.Done():
  656. return
  657. case <-clockSource.C:
  658. continue
  659. }
  660. }
  661. }()
  662. }
  663. func (lh *LightHouse) SendUpdate() {
  664. var v4 []*Ip4AndPort
  665. var v6 []*Ip6AndPort
  666. for _, e := range lh.GetAdvertiseAddrs() {
  667. if ip := e.ip.To4(); ip != nil {
  668. v4 = append(v4, NewIp4AndPort(e.ip, uint32(e.port)))
  669. } else {
  670. v6 = append(v6, NewIp6AndPort(e.ip, uint32(e.port)))
  671. }
  672. }
  673. lal := lh.GetLocalAllowList()
  674. for _, e := range *localIps(lh.l, lal) {
  675. if ip4 := e.To4(); ip4 != nil && ipMaskContains(lh.myVpnIp, lh.myVpnZeros, iputil.Ip2VpnIp(ip4)) {
  676. continue
  677. }
  678. // Only add IPs that aren't my VPN/tun IP
  679. if ip := e.To4(); ip != nil {
  680. v4 = append(v4, NewIp4AndPort(e, lh.nebulaPort))
  681. } else {
  682. v6 = append(v6, NewIp6AndPort(e, lh.nebulaPort))
  683. }
  684. }
  685. var relays []uint32
  686. for _, r := range lh.GetRelaysForMe() {
  687. relays = append(relays, (uint32)(r))
  688. }
  689. m := &NebulaMeta{
  690. Type: NebulaMeta_HostUpdateNotification,
  691. Details: &NebulaMetaDetails{
  692. VpnIp: uint32(lh.myVpnIp),
  693. Ip4AndPorts: v4,
  694. Ip6AndPorts: v6,
  695. RelayVpnIp: relays,
  696. },
  697. }
  698. lighthouses := lh.GetLighthouses()
  699. lh.metricTx(NebulaMeta_HostUpdateNotification, int64(len(lighthouses)))
  700. nb := make([]byte, 12, 12)
  701. out := make([]byte, mtu)
  702. mm, err := m.Marshal()
  703. if err != nil {
  704. lh.l.WithError(err).Error("Error while marshaling for lighthouse update")
  705. return
  706. }
  707. for vpnIp := range lighthouses {
  708. lh.ifce.SendMessageToVpnIp(header.LightHouse, 0, vpnIp, mm, nb, out)
  709. }
  710. }
  711. type LightHouseHandler struct {
  712. lh *LightHouse
  713. nb []byte
  714. out []byte
  715. pb []byte
  716. meta *NebulaMeta
  717. l *logrus.Logger
  718. }
  719. func (lh *LightHouse) NewRequestHandler() *LightHouseHandler {
  720. lhh := &LightHouseHandler{
  721. lh: lh,
  722. nb: make([]byte, 12, 12),
  723. out: make([]byte, mtu),
  724. l: lh.l,
  725. pb: make([]byte, mtu),
  726. meta: &NebulaMeta{
  727. Details: &NebulaMetaDetails{},
  728. },
  729. }
  730. return lhh
  731. }
  732. func (lh *LightHouse) metricRx(t NebulaMeta_MessageType, i int64) {
  733. lh.metrics.Rx(header.MessageType(t), 0, i)
  734. }
  735. func (lh *LightHouse) metricTx(t NebulaMeta_MessageType, i int64) {
  736. lh.metrics.Tx(header.MessageType(t), 0, i)
  737. }
  738. // This method is similar to Reset(), but it re-uses the pointer structs
  739. // so that we don't have to re-allocate them
  740. func (lhh *LightHouseHandler) resetMeta() *NebulaMeta {
  741. details := lhh.meta.Details
  742. lhh.meta.Reset()
  743. // Keep the array memory around
  744. details.Ip4AndPorts = details.Ip4AndPorts[:0]
  745. details.Ip6AndPorts = details.Ip6AndPorts[:0]
  746. details.RelayVpnIp = details.RelayVpnIp[:0]
  747. lhh.meta.Details = details
  748. return lhh.meta
  749. }
  750. func lhHandleRequest(lhh *LightHouseHandler, f *Interface) udp.LightHouseHandlerFunc {
  751. return func(rAddr *udp.Addr, vpnIp iputil.VpnIp, p []byte) {
  752. lhh.HandleRequest(rAddr, vpnIp, p, f)
  753. }
  754. }
  755. func (lhh *LightHouseHandler) HandleRequest(rAddr *udp.Addr, vpnIp iputil.VpnIp, p []byte, w EncWriter) {
  756. n := lhh.resetMeta()
  757. err := n.Unmarshal(p)
  758. if err != nil {
  759. lhh.l.WithError(err).WithField("vpnIp", vpnIp).WithField("udpAddr", rAddr).
  760. Error("Failed to unmarshal lighthouse packet")
  761. //TODO: send recv_error?
  762. return
  763. }
  764. if n.Details == nil {
  765. lhh.l.WithField("vpnIp", vpnIp).WithField("udpAddr", rAddr).
  766. Error("Invalid lighthouse update")
  767. //TODO: send recv_error?
  768. return
  769. }
  770. lhh.lh.metricRx(n.Type, 1)
  771. switch n.Type {
  772. case NebulaMeta_HostQuery:
  773. lhh.handleHostQuery(n, vpnIp, rAddr, w)
  774. case NebulaMeta_HostQueryReply:
  775. lhh.handleHostQueryReply(n, vpnIp)
  776. case NebulaMeta_HostUpdateNotification:
  777. lhh.handleHostUpdateNotification(n, vpnIp, w)
  778. case NebulaMeta_HostMovedNotification:
  779. case NebulaMeta_HostPunchNotification:
  780. lhh.handleHostPunchNotification(n, vpnIp, w)
  781. case NebulaMeta_HostUpdateNotificationAck:
  782. // noop
  783. }
  784. }
  785. func (lhh *LightHouseHandler) handleHostQuery(n *NebulaMeta, vpnIp iputil.VpnIp, addr *udp.Addr, w EncWriter) {
  786. // Exit if we don't answer queries
  787. if !lhh.lh.amLighthouse {
  788. if lhh.l.Level >= logrus.DebugLevel {
  789. lhh.l.Debugln("I don't answer queries, but received from: ", addr)
  790. }
  791. return
  792. }
  793. //TODO: we can DRY this further
  794. reqVpnIp := n.Details.VpnIp
  795. //TODO: Maybe instead of marshalling into n we marshal into a new `r` to not nuke our current request data
  796. found, ln, err := lhh.lh.queryAndPrepMessage(iputil.VpnIp(n.Details.VpnIp), func(c *cache) (int, error) {
  797. n = lhh.resetMeta()
  798. n.Type = NebulaMeta_HostQueryReply
  799. n.Details.VpnIp = reqVpnIp
  800. lhh.coalesceAnswers(c, n)
  801. return n.MarshalTo(lhh.pb)
  802. })
  803. if !found {
  804. return
  805. }
  806. if err != nil {
  807. lhh.l.WithError(err).WithField("vpnIp", vpnIp).Error("Failed to marshal lighthouse host query reply")
  808. return
  809. }
  810. lhh.lh.metricTx(NebulaMeta_HostQueryReply, 1)
  811. w.SendMessageToVpnIp(header.LightHouse, 0, vpnIp, lhh.pb[:ln], lhh.nb, lhh.out[:0])
  812. // This signals the other side to punch some zero byte udp packets
  813. found, ln, err = lhh.lh.queryAndPrepMessage(vpnIp, func(c *cache) (int, error) {
  814. n = lhh.resetMeta()
  815. n.Type = NebulaMeta_HostPunchNotification
  816. n.Details.VpnIp = uint32(vpnIp)
  817. lhh.coalesceAnswers(c, n)
  818. return n.MarshalTo(lhh.pb)
  819. })
  820. if !found {
  821. return
  822. }
  823. if err != nil {
  824. lhh.l.WithError(err).WithField("vpnIp", vpnIp).Error("Failed to marshal lighthouse host was queried for")
  825. return
  826. }
  827. lhh.lh.metricTx(NebulaMeta_HostPunchNotification, 1)
  828. w.SendMessageToVpnIp(header.LightHouse, 0, iputil.VpnIp(reqVpnIp), lhh.pb[:ln], lhh.nb, lhh.out[:0])
  829. }
  830. func (lhh *LightHouseHandler) coalesceAnswers(c *cache, n *NebulaMeta) {
  831. if c.v4 != nil {
  832. if c.v4.learned != nil {
  833. n.Details.Ip4AndPorts = append(n.Details.Ip4AndPorts, c.v4.learned)
  834. }
  835. if c.v4.reported != nil && len(c.v4.reported) > 0 {
  836. n.Details.Ip4AndPorts = append(n.Details.Ip4AndPorts, c.v4.reported...)
  837. }
  838. }
  839. if c.v6 != nil {
  840. if c.v6.learned != nil {
  841. n.Details.Ip6AndPorts = append(n.Details.Ip6AndPorts, c.v6.learned)
  842. }
  843. if c.v6.reported != nil && len(c.v6.reported) > 0 {
  844. n.Details.Ip6AndPorts = append(n.Details.Ip6AndPorts, c.v6.reported...)
  845. }
  846. }
  847. if c.relay != nil {
  848. n.Details.RelayVpnIp = append(n.Details.RelayVpnIp, c.relay.relay...)
  849. }
  850. }
  851. func (lhh *LightHouseHandler) handleHostQueryReply(n *NebulaMeta, vpnIp iputil.VpnIp) {
  852. if !lhh.lh.IsLighthouseIP(vpnIp) {
  853. return
  854. }
  855. lhh.lh.Lock()
  856. am := lhh.lh.unlockedGetRemoteList(iputil.VpnIp(n.Details.VpnIp))
  857. am.Lock()
  858. lhh.lh.Unlock()
  859. certVpnIp := iputil.VpnIp(n.Details.VpnIp)
  860. am.unlockedSetV4(vpnIp, certVpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
  861. am.unlockedSetV6(vpnIp, certVpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
  862. am.unlockedSetRelay(vpnIp, certVpnIp, n.Details.RelayVpnIp)
  863. am.Unlock()
  864. // Non-blocking attempt to trigger, skip if it would block
  865. select {
  866. case lhh.lh.handshakeTrigger <- iputil.VpnIp(n.Details.VpnIp):
  867. default:
  868. }
  869. }
  870. func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp iputil.VpnIp, w EncWriter) {
  871. if !lhh.lh.amLighthouse {
  872. if lhh.l.Level >= logrus.DebugLevel {
  873. lhh.l.Debugln("I am not a lighthouse, do not take host updates: ", vpnIp)
  874. }
  875. return
  876. }
  877. //Simple check that the host sent this not someone else
  878. if n.Details.VpnIp != uint32(vpnIp) {
  879. if lhh.l.Level >= logrus.DebugLevel {
  880. lhh.l.WithField("vpnIp", vpnIp).WithField("answer", iputil.VpnIp(n.Details.VpnIp)).Debugln("Host sent invalid update")
  881. }
  882. return
  883. }
  884. lhh.lh.Lock()
  885. am := lhh.lh.unlockedGetRemoteList(vpnIp)
  886. am.Lock()
  887. lhh.lh.Unlock()
  888. certVpnIp := iputil.VpnIp(n.Details.VpnIp)
  889. am.unlockedSetV4(vpnIp, certVpnIp, n.Details.Ip4AndPorts, lhh.lh.unlockedShouldAddV4)
  890. am.unlockedSetV6(vpnIp, certVpnIp, n.Details.Ip6AndPorts, lhh.lh.unlockedShouldAddV6)
  891. am.unlockedSetRelay(vpnIp, certVpnIp, n.Details.RelayVpnIp)
  892. am.Unlock()
  893. n = lhh.resetMeta()
  894. n.Type = NebulaMeta_HostUpdateNotificationAck
  895. n.Details.VpnIp = uint32(vpnIp)
  896. ln, err := n.MarshalTo(lhh.pb)
  897. if err != nil {
  898. lhh.l.WithError(err).WithField("vpnIp", vpnIp).Error("Failed to marshal lighthouse host update ack")
  899. return
  900. }
  901. lhh.lh.metricTx(NebulaMeta_HostUpdateNotificationAck, 1)
  902. w.SendMessageToVpnIp(header.LightHouse, 0, vpnIp, lhh.pb[:ln], lhh.nb, lhh.out[:0])
  903. }
  904. func (lhh *LightHouseHandler) handleHostPunchNotification(n *NebulaMeta, vpnIp iputil.VpnIp, w EncWriter) {
  905. if !lhh.lh.IsLighthouseIP(vpnIp) {
  906. return
  907. }
  908. empty := []byte{0}
  909. punch := func(vpnPeer *udp.Addr) {
  910. if vpnPeer == nil {
  911. return
  912. }
  913. go func() {
  914. time.Sleep(lhh.lh.punchy.GetDelay())
  915. lhh.lh.metricHolepunchTx.Inc(1)
  916. lhh.lh.punchConn.WriteTo(empty, vpnPeer)
  917. }()
  918. if lhh.l.Level >= logrus.DebugLevel {
  919. //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))
  920. lhh.l.Debugf("Punching on %d for %s", vpnPeer.Port, iputil.VpnIp(n.Details.VpnIp))
  921. }
  922. }
  923. for _, a := range n.Details.Ip4AndPorts {
  924. punch(NewUDPAddrFromLH4(a))
  925. }
  926. for _, a := range n.Details.Ip6AndPorts {
  927. punch(NewUDPAddrFromLH6(a))
  928. }
  929. // This sends a nebula test packet to the host trying to contact us. In the case
  930. // of a double nat or other difficult scenario, this may help establish
  931. // a tunnel.
  932. if lhh.lh.punchy.GetRespond() {
  933. queryVpnIp := iputil.VpnIp(n.Details.VpnIp)
  934. go func() {
  935. time.Sleep(lhh.lh.punchy.GetRespondDelay())
  936. if lhh.l.Level >= logrus.DebugLevel {
  937. lhh.l.Debugf("Sending a nebula test packet to vpn ip %s", queryVpnIp)
  938. }
  939. //NOTE: we have to allocate a new output buffer here since we are spawning a new goroutine
  940. // for each punchBack packet. We should move this into a timerwheel or a single goroutine
  941. // managed by a channel.
  942. w.SendMessageToVpnIp(header.Test, header.TestRequest, queryVpnIp, []byte(""), make([]byte, 12, 12), make([]byte, mtu))
  943. }()
  944. }
  945. }
  946. // ipMaskContains checks if testIp is contained by ip after applying a cidr
  947. // zeros is 32 - bits from net.IPMask.Size()
  948. func ipMaskContains(ip iputil.VpnIp, zeros iputil.VpnIp, testIp iputil.VpnIp) bool {
  949. return (testIp^ip)>>zeros == 0
  950. }