lighthouse.go 33 KB

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