|
@@ -7,6 +7,7 @@ import (
|
|
|
"encoding/binary"
|
|
|
"errors"
|
|
|
"net"
|
|
|
+ "sync"
|
|
|
"time"
|
|
|
|
|
|
"github.com/rcrowley/go-metrics"
|
|
@@ -42,15 +43,21 @@ type HandshakeConfig struct {
|
|
|
}
|
|
|
|
|
|
type HandshakeManager struct {
|
|
|
- pendingHostMap *HostMap
|
|
|
+ // Mutex for interacting with the vpnIps and indexes maps
|
|
|
+ sync.RWMutex
|
|
|
+
|
|
|
+ vpnIps map[iputil.VpnIp]*HostInfo
|
|
|
+ indexes map[uint32]*HostInfo
|
|
|
+
|
|
|
mainHostMap *HostMap
|
|
|
lightHouse *LightHouse
|
|
|
- outside *udp.Conn
|
|
|
+ outside udp.Conn
|
|
|
config HandshakeConfig
|
|
|
OutboundHandshakeTimer *LockingTimerWheel[iputil.VpnIp]
|
|
|
messageMetrics *MessageMetrics
|
|
|
metricInitiated metrics.Counter
|
|
|
metricTimedOut metrics.Counter
|
|
|
+ f *Interface
|
|
|
l *logrus.Logger
|
|
|
|
|
|
multiPort MultiPortConfig
|
|
@@ -60,9 +67,10 @@ type HandshakeManager struct {
|
|
|
trigger chan iputil.VpnIp
|
|
|
}
|
|
|
|
|
|
-func NewHandshakeManager(l *logrus.Logger, tunCidr *net.IPNet, preferredRanges []*net.IPNet, mainHostMap *HostMap, lightHouse *LightHouse, outside *udp.Conn, config HandshakeConfig) *HandshakeManager {
|
|
|
+func NewHandshakeManager(l *logrus.Logger, mainHostMap *HostMap, lightHouse *LightHouse, outside udp.Conn, config HandshakeConfig) *HandshakeManager {
|
|
|
return &HandshakeManager{
|
|
|
- pendingHostMap: NewHostMap(l, "pending", tunCidr, preferredRanges),
|
|
|
+ vpnIps: map[iputil.VpnIp]*HostInfo{},
|
|
|
+ indexes: map[uint32]*HostInfo{},
|
|
|
mainHostMap: mainHostMap,
|
|
|
lightHouse: lightHouse,
|
|
|
outside: outside,
|
|
@@ -76,7 +84,7 @@ func NewHandshakeManager(l *logrus.Logger, tunCidr *net.IPNet, preferredRanges [
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) Run(ctx context.Context, f EncWriter) {
|
|
|
+func (c *HandshakeManager) Run(ctx context.Context) {
|
|
|
clockSource := time.NewTicker(c.config.tryInterval)
|
|
|
defer clockSource.Stop()
|
|
|
|
|
@@ -85,27 +93,27 @@ func (c *HandshakeManager) Run(ctx context.Context, f EncWriter) {
|
|
|
case <-ctx.Done():
|
|
|
return
|
|
|
case vpnIP := <-c.trigger:
|
|
|
- c.handleOutbound(vpnIP, f, true)
|
|
|
+ c.handleOutbound(vpnIP, true)
|
|
|
case now := <-clockSource.C:
|
|
|
- c.NextOutboundHandshakeTimerTick(now, f)
|
|
|
+ c.NextOutboundHandshakeTimerTick(now)
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) NextOutboundHandshakeTimerTick(now time.Time, f EncWriter) {
|
|
|
+func (c *HandshakeManager) NextOutboundHandshakeTimerTick(now time.Time) {
|
|
|
c.OutboundHandshakeTimer.Advance(now)
|
|
|
for {
|
|
|
vpnIp, has := c.OutboundHandshakeTimer.Purge()
|
|
|
if !has {
|
|
|
break
|
|
|
}
|
|
|
- c.handleOutbound(vpnIp, f, false)
|
|
|
+ c.handleOutbound(vpnIp, false)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, lighthouseTriggered bool) {
|
|
|
- hostinfo, err := c.pendingHostMap.QueryVpnIp(vpnIp)
|
|
|
- if err != nil {
|
|
|
+func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, lighthouseTriggered bool) {
|
|
|
+ hostinfo := c.QueryVpnIp(vpnIp)
|
|
|
+ if hostinfo == nil {
|
|
|
return
|
|
|
}
|
|
|
hostinfo.Lock()
|
|
@@ -114,31 +122,34 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
// We may have raced to completion but now that we have a lock we should ensure we have not yet completed.
|
|
|
if hostinfo.HandshakeComplete {
|
|
|
// Ensure we don't exist in the pending hostmap anymore since we have completed
|
|
|
- c.pendingHostMap.DeleteHostInfo(hostinfo)
|
|
|
- return
|
|
|
- }
|
|
|
-
|
|
|
- // Check if we have a handshake packet to transmit yet
|
|
|
- if !hostinfo.HandshakeReady {
|
|
|
- // There is currently a slight race in getOrHandshake due to ConnectionState not being part of the HostInfo directly
|
|
|
- // Our hostinfo here was added to the pending map and the wheel may have ticked to us before we created ConnectionState
|
|
|
- c.OutboundHandshakeTimer.Add(vpnIp, c.config.tryInterval*time.Duration(hostinfo.HandshakeCounter))
|
|
|
+ c.DeleteHostInfo(hostinfo)
|
|
|
return
|
|
|
}
|
|
|
|
|
|
// If we are out of time, clean up
|
|
|
if hostinfo.HandshakeCounter >= c.config.retries {
|
|
|
- hostinfo.logger(c.l).WithField("udpAddrs", hostinfo.remotes.CopyAddrs(c.pendingHostMap.preferredRanges)).
|
|
|
+ hostinfo.logger(c.l).WithField("udpAddrs", hostinfo.remotes.CopyAddrs(c.mainHostMap.preferredRanges)).
|
|
|
WithField("initiatorIndex", hostinfo.localIndexId).
|
|
|
WithField("remoteIndex", hostinfo.remoteIndexId).
|
|
|
WithField("handshake", m{"stage": 1, "style": "ix_psk0"}).
|
|
|
WithField("durationNs", time.Since(hostinfo.handshakeStart).Nanoseconds()).
|
|
|
Info("Handshake timed out")
|
|
|
c.metricTimedOut.Inc(1)
|
|
|
- c.pendingHostMap.DeleteHostInfo(hostinfo)
|
|
|
+ c.DeleteHostInfo(hostinfo)
|
|
|
return
|
|
|
}
|
|
|
|
|
|
+ // Increment the counter to increase our delay, linear backoff
|
|
|
+ hostinfo.HandshakeCounter++
|
|
|
+
|
|
|
+ // Check if we have a handshake packet to transmit yet
|
|
|
+ if !hostinfo.HandshakeReady {
|
|
|
+ if !ixHandshakeStage0(c.f, hostinfo) {
|
|
|
+ c.OutboundHandshakeTimer.Add(vpnIp, c.config.tryInterval*time.Duration(hostinfo.HandshakeCounter))
|
|
|
+ return
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
// Get a remotes object if we don't already have one.
|
|
|
// This is mainly to protect us as this should never be the case
|
|
|
// NB ^ This comment doesn't jive. It's how the thing gets initialized.
|
|
@@ -147,7 +158,7 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
hostinfo.remotes = c.lightHouse.QueryCache(vpnIp)
|
|
|
}
|
|
|
|
|
|
- remotes := hostinfo.remotes.CopyAddrs(c.pendingHostMap.preferredRanges)
|
|
|
+ remotes := hostinfo.remotes.CopyAddrs(c.mainHostMap.preferredRanges)
|
|
|
remotesHaveChanged := !udp.AddrSlice(remotes).Equal(hostinfo.HandshakeLastRemotes)
|
|
|
|
|
|
// We only care about a lighthouse trigger if we have new remotes to send to.
|
|
@@ -166,15 +177,15 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
// If we only have 1 remote it is highly likely our query raced with the other host registered within the lighthouse
|
|
|
// Our vpnIp here has a tunnel with a lighthouse but has yet to send a host update packet there so we only know about
|
|
|
// the learned public ip for them. Query again to short circuit the promotion counter
|
|
|
- c.lightHouse.QueryServer(vpnIp, f)
|
|
|
+ c.lightHouse.QueryServer(vpnIp, c.f)
|
|
|
}
|
|
|
|
|
|
// Send the handshake to all known ips, stage 2 takes care of assigning the hostinfo.remote based on the first to reply
|
|
|
var sentTo []*udp.Addr
|
|
|
var sentMultiport bool
|
|
|
- hostinfo.remotes.ForEach(c.pendingHostMap.preferredRanges, func(addr *udp.Addr, _ bool) {
|
|
|
+ hostinfo.remotes.ForEach(c.mainHostMap.preferredRanges, func(addr *udp.Addr, _ bool) {
|
|
|
c.messageMetrics.Tx(header.Handshake, header.MessageSubType(hostinfo.HandshakePacket[0][1]), 1)
|
|
|
- err = c.outside.WriteTo(hostinfo.HandshakePacket[0], addr)
|
|
|
+ err := c.outside.WriteTo(hostinfo.HandshakePacket[0], addr)
|
|
|
if err != nil {
|
|
|
hostinfo.logger(c.l).WithField("udpAddr", addr).
|
|
|
WithField("initiatorIndex", hostinfo.localIndexId).
|
|
@@ -230,10 +241,10 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
if *relay == vpnIp || *relay == c.lightHouse.myVpnIp {
|
|
|
continue
|
|
|
}
|
|
|
- relayHostInfo, err := c.mainHostMap.QueryVpnIp(*relay)
|
|
|
- if err != nil || relayHostInfo.remote == nil {
|
|
|
- hostinfo.logger(c.l).WithError(err).WithField("relay", relay.String()).Info("Establish tunnel to relay target")
|
|
|
- f.Handshake(*relay)
|
|
|
+ relayHostInfo := c.mainHostMap.QueryVpnIp(*relay)
|
|
|
+ if relayHostInfo == nil || relayHostInfo.remote == nil {
|
|
|
+ hostinfo.logger(c.l).WithField("relay", relay.String()).Info("Establish tunnel to relay target")
|
|
|
+ c.f.Handshake(*relay)
|
|
|
continue
|
|
|
}
|
|
|
// Check the relay HostInfo to see if we already established a relay through it
|
|
@@ -241,7 +252,7 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
switch existingRelay.State {
|
|
|
case Established:
|
|
|
hostinfo.logger(c.l).WithField("relay", relay.String()).Info("Send handshake via relay")
|
|
|
- f.SendVia(relayHostInfo, existingRelay, hostinfo.HandshakePacket[0], make([]byte, 12), make([]byte, mtu), false)
|
|
|
+ c.f.SendVia(relayHostInfo, existingRelay, hostinfo.HandshakePacket[0], make([]byte, 12), make([]byte, mtu), false)
|
|
|
case Requested:
|
|
|
hostinfo.logger(c.l).WithField("relay", relay.String()).Info("Re-send CreateRelay request")
|
|
|
// Re-send the CreateRelay request, in case the previous one was lost.
|
|
@@ -258,7 +269,7 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
Error("Failed to marshal Control message to create relay")
|
|
|
} else {
|
|
|
// This must send over the hostinfo, not over hm.Hosts[ip]
|
|
|
- f.SendMessageToHostInfo(header.Control, 0, relayHostInfo, msg, make([]byte, 12), make([]byte, mtu))
|
|
|
+ c.f.SendMessageToHostInfo(header.Control, 0, relayHostInfo, msg, make([]byte, 12), make([]byte, mtu))
|
|
|
c.l.WithFields(logrus.Fields{
|
|
|
"relayFrom": c.lightHouse.myVpnIp,
|
|
|
"relayTo": vpnIp,
|
|
@@ -293,7 +304,7 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
WithError(err).
|
|
|
Error("Failed to marshal Control message to create relay")
|
|
|
} else {
|
|
|
- f.SendMessageToHostInfo(header.Control, 0, relayHostInfo, msg, make([]byte, 12), make([]byte, mtu))
|
|
|
+ c.f.SendMessageToHostInfo(header.Control, 0, relayHostInfo, msg, make([]byte, 12), make([]byte, mtu))
|
|
|
c.l.WithFields(logrus.Fields{
|
|
|
"relayFrom": c.lightHouse.myVpnIp,
|
|
|
"relayTo": vpnIp,
|
|
@@ -306,23 +317,78 @@ func (c *HandshakeManager) handleOutbound(vpnIp iputil.VpnIp, f EncWriter, light
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // Increment the counter to increase our delay, linear backoff
|
|
|
- hostinfo.HandshakeCounter++
|
|
|
-
|
|
|
// If a lighthouse triggered this attempt then we are still in the timer wheel and do not need to re-add
|
|
|
if !lighthouseTriggered {
|
|
|
c.OutboundHandshakeTimer.Add(vpnIp, c.config.tryInterval*time.Duration(hostinfo.HandshakeCounter))
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) AddVpnIp(vpnIp iputil.VpnIp, init func(*HostInfo)) *HostInfo {
|
|
|
- hostinfo, created := c.pendingHostMap.AddVpnIp(vpnIp, init)
|
|
|
+// GetOrHandshake will try to find a hostinfo with a fully formed tunnel or start a new handshake if one is not present
|
|
|
+// The 2nd argument will be true if the hostinfo is ready to transmit traffic
|
|
|
+func (hm *HandshakeManager) GetOrHandshake(vpnIp iputil.VpnIp, cacheCb func(*HostInfo)) (*HostInfo, bool) {
|
|
|
+ // Check the main hostmap and maintain a read lock if our host is not there
|
|
|
+ hm.mainHostMap.RLock()
|
|
|
+ if h, ok := hm.mainHostMap.Hosts[vpnIp]; ok {
|
|
|
+ hm.mainHostMap.RUnlock()
|
|
|
+ // Do not attempt promotion if you are a lighthouse
|
|
|
+ if !hm.lightHouse.amLighthouse {
|
|
|
+ h.TryPromoteBest(hm.mainHostMap.preferredRanges, hm.f)
|
|
|
+ }
|
|
|
+ return h, true
|
|
|
+ }
|
|
|
+
|
|
|
+ defer hm.mainHostMap.RUnlock()
|
|
|
+ return hm.StartHandshake(vpnIp, cacheCb), false
|
|
|
+}
|
|
|
+
|
|
|
+// StartHandshake will ensure a handshake is currently being attempted for the provided vpn ip
|
|
|
+func (hm *HandshakeManager) StartHandshake(vpnIp iputil.VpnIp, cacheCb func(*HostInfo)) *HostInfo {
|
|
|
+ hm.Lock()
|
|
|
+
|
|
|
+ if hostinfo, ok := hm.vpnIps[vpnIp]; ok {
|
|
|
+ // We are already trying to handshake with this vpn ip
|
|
|
+ if cacheCb != nil {
|
|
|
+ cacheCb(hostinfo)
|
|
|
+ }
|
|
|
+ hm.Unlock()
|
|
|
+ return hostinfo
|
|
|
+ }
|
|
|
+
|
|
|
+ hostinfo := &HostInfo{
|
|
|
+ vpnIp: vpnIp,
|
|
|
+ HandshakePacket: make(map[uint8][]byte, 0),
|
|
|
+ relayState: RelayState{
|
|
|
+ relays: map[iputil.VpnIp]struct{}{},
|
|
|
+ relayForByIp: map[iputil.VpnIp]*Relay{},
|
|
|
+ relayForByIdx: map[uint32]*Relay{},
|
|
|
+ },
|
|
|
+ }
|
|
|
+
|
|
|
+ hm.vpnIps[vpnIp] = hostinfo
|
|
|
+ hm.metricInitiated.Inc(1)
|
|
|
+ hm.OutboundHandshakeTimer.Add(vpnIp, hm.config.tryInterval)
|
|
|
|
|
|
- if created {
|
|
|
- c.OutboundHandshakeTimer.Add(vpnIp, c.config.tryInterval)
|
|
|
- c.metricInitiated.Inc(1)
|
|
|
+ if cacheCb != nil {
|
|
|
+ cacheCb(hostinfo)
|
|
|
}
|
|
|
|
|
|
+ // If this is a static host, we don't need to wait for the HostQueryReply
|
|
|
+ // We can trigger the handshake right now
|
|
|
+ _, doTrigger := hm.lightHouse.GetStaticHostList()[vpnIp]
|
|
|
+ if !doTrigger {
|
|
|
+ // Add any calculated remotes, and trigger early handshake if one found
|
|
|
+ doTrigger = hm.lightHouse.addCalculatedRemotes(vpnIp)
|
|
|
+ }
|
|
|
+
|
|
|
+ if doTrigger {
|
|
|
+ select {
|
|
|
+ case hm.trigger <- vpnIp:
|
|
|
+ default:
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ hm.Unlock()
|
|
|
+ hm.lightHouse.QueryServer(vpnIp, hm.f)
|
|
|
return hostinfo
|
|
|
}
|
|
|
|
|
@@ -344,10 +410,10 @@ var (
|
|
|
// ErrLocalIndexCollision if we already have an entry in the main or pending
|
|
|
// hostmap for the hostinfo.localIndexId.
|
|
|
func (c *HandshakeManager) CheckAndComplete(hostinfo *HostInfo, handshakePacket uint8, f *Interface) (*HostInfo, error) {
|
|
|
- c.pendingHostMap.Lock()
|
|
|
- defer c.pendingHostMap.Unlock()
|
|
|
c.mainHostMap.Lock()
|
|
|
defer c.mainHostMap.Unlock()
|
|
|
+ c.Lock()
|
|
|
+ defer c.Unlock()
|
|
|
|
|
|
// Check if we already have a tunnel with this vpn ip
|
|
|
existingHostInfo, found := c.mainHostMap.Hosts[hostinfo.vpnIp]
|
|
@@ -376,7 +442,7 @@ func (c *HandshakeManager) CheckAndComplete(hostinfo *HostInfo, handshakePacket
|
|
|
return existingIndex, ErrLocalIndexCollision
|
|
|
}
|
|
|
|
|
|
- existingIndex, found = c.pendingHostMap.Indexes[hostinfo.localIndexId]
|
|
|
+ existingIndex, found = c.indexes[hostinfo.localIndexId]
|
|
|
if found && existingIndex != hostinfo {
|
|
|
// We have a collision, but for a different hostinfo
|
|
|
return existingIndex, ErrLocalIndexCollision
|
|
@@ -398,47 +464,47 @@ func (c *HandshakeManager) CheckAndComplete(hostinfo *HostInfo, handshakePacket
|
|
|
// Complete is a simpler version of CheckAndComplete when we already know we
|
|
|
// won't have a localIndexId collision because we already have an entry in the
|
|
|
// pendingHostMap. An existing hostinfo is returned if there was one.
|
|
|
-func (c *HandshakeManager) Complete(hostinfo *HostInfo, f *Interface) {
|
|
|
- c.pendingHostMap.Lock()
|
|
|
- defer c.pendingHostMap.Unlock()
|
|
|
- c.mainHostMap.Lock()
|
|
|
- defer c.mainHostMap.Unlock()
|
|
|
+func (hm *HandshakeManager) Complete(hostinfo *HostInfo, f *Interface) {
|
|
|
+ hm.mainHostMap.Lock()
|
|
|
+ defer hm.mainHostMap.Unlock()
|
|
|
+ hm.Lock()
|
|
|
+ defer hm.Unlock()
|
|
|
|
|
|
- existingRemoteIndex, found := c.mainHostMap.RemoteIndexes[hostinfo.remoteIndexId]
|
|
|
+ existingRemoteIndex, found := hm.mainHostMap.RemoteIndexes[hostinfo.remoteIndexId]
|
|
|
if found && existingRemoteIndex != nil {
|
|
|
// We have a collision, but this can happen since we can't control
|
|
|
// the remote ID. Just log about the situation as a note.
|
|
|
- hostinfo.logger(c.l).
|
|
|
+ hostinfo.logger(hm.l).
|
|
|
WithField("remoteIndex", hostinfo.remoteIndexId).WithField("collision", existingRemoteIndex.vpnIp).
|
|
|
Info("New host shadows existing host remoteIndex")
|
|
|
}
|
|
|
|
|
|
// We need to remove from the pending hostmap first to avoid undoing work when after to the main hostmap.
|
|
|
- c.pendingHostMap.unlockedDeleteHostInfo(hostinfo)
|
|
|
- c.mainHostMap.unlockedAddHostInfo(hostinfo, f)
|
|
|
+ hm.unlockedDeleteHostInfo(hostinfo)
|
|
|
+ hm.mainHostMap.unlockedAddHostInfo(hostinfo, f)
|
|
|
}
|
|
|
|
|
|
-// AddIndexHostInfo generates a unique localIndexId for this HostInfo
|
|
|
+// allocateIndex generates a unique localIndexId for this HostInfo
|
|
|
// and adds it to the pendingHostMap. Will error if we are unable to generate
|
|
|
// a unique localIndexId
|
|
|
-func (c *HandshakeManager) AddIndexHostInfo(h *HostInfo) error {
|
|
|
- c.pendingHostMap.Lock()
|
|
|
- defer c.pendingHostMap.Unlock()
|
|
|
- c.mainHostMap.RLock()
|
|
|
- defer c.mainHostMap.RUnlock()
|
|
|
+func (hm *HandshakeManager) allocateIndex(h *HostInfo) error {
|
|
|
+ hm.mainHostMap.RLock()
|
|
|
+ defer hm.mainHostMap.RUnlock()
|
|
|
+ hm.Lock()
|
|
|
+ defer hm.Unlock()
|
|
|
|
|
|
for i := 0; i < 32; i++ {
|
|
|
- index, err := generateIndex(c.l)
|
|
|
+ index, err := generateIndex(hm.l)
|
|
|
if err != nil {
|
|
|
return err
|
|
|
}
|
|
|
|
|
|
- _, inPending := c.pendingHostMap.Indexes[index]
|
|
|
- _, inMain := c.mainHostMap.Indexes[index]
|
|
|
+ _, inPending := hm.indexes[index]
|
|
|
+ _, inMain := hm.mainHostMap.Indexes[index]
|
|
|
|
|
|
if !inMain && !inPending {
|
|
|
h.localIndexId = index
|
|
|
- c.pendingHostMap.Indexes[index] = h
|
|
|
+ hm.indexes[index] = h
|
|
|
return nil
|
|
|
}
|
|
|
}
|
|
@@ -446,22 +512,73 @@ func (c *HandshakeManager) AddIndexHostInfo(h *HostInfo) error {
|
|
|
return errors.New("failed to generate unique localIndexId")
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) addRemoteIndexHostInfo(index uint32, h *HostInfo) {
|
|
|
- c.pendingHostMap.addRemoteIndexHostInfo(index, h)
|
|
|
+func (c *HandshakeManager) DeleteHostInfo(hostinfo *HostInfo) {
|
|
|
+ c.Lock()
|
|
|
+ defer c.Unlock()
|
|
|
+ c.unlockedDeleteHostInfo(hostinfo)
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) DeleteHostInfo(hostinfo *HostInfo) {
|
|
|
- //l.Debugln("Deleting pending hostinfo :", hostinfo)
|
|
|
- c.pendingHostMap.DeleteHostInfo(hostinfo)
|
|
|
+func (c *HandshakeManager) unlockedDeleteHostInfo(hostinfo *HostInfo) {
|
|
|
+ delete(c.vpnIps, hostinfo.vpnIp)
|
|
|
+ if len(c.vpnIps) == 0 {
|
|
|
+ c.vpnIps = map[iputil.VpnIp]*HostInfo{}
|
|
|
+ }
|
|
|
+
|
|
|
+ delete(c.indexes, hostinfo.localIndexId)
|
|
|
+ if len(c.vpnIps) == 0 {
|
|
|
+ c.indexes = map[uint32]*HostInfo{}
|
|
|
+ }
|
|
|
+
|
|
|
+ if c.l.Level >= logrus.DebugLevel {
|
|
|
+ c.l.WithField("hostMap", m{"mapTotalSize": len(c.vpnIps),
|
|
|
+ "vpnIp": hostinfo.vpnIp, "indexNumber": hostinfo.localIndexId, "remoteIndexNumber": hostinfo.remoteIndexId}).
|
|
|
+ Debug("Pending hostmap hostInfo deleted")
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+func (c *HandshakeManager) QueryVpnIp(vpnIp iputil.VpnIp) *HostInfo {
|
|
|
+ c.RLock()
|
|
|
+ defer c.RUnlock()
|
|
|
+ return c.vpnIps[vpnIp]
|
|
|
+}
|
|
|
+
|
|
|
+func (c *HandshakeManager) QueryIndex(index uint32) *HostInfo {
|
|
|
+ c.RLock()
|
|
|
+ defer c.RUnlock()
|
|
|
+ return c.indexes[index]
|
|
|
}
|
|
|
|
|
|
-func (c *HandshakeManager) QueryIndex(index uint32) (*HostInfo, error) {
|
|
|
- return c.pendingHostMap.QueryIndex(index)
|
|
|
+func (c *HandshakeManager) GetPreferredRanges() []*net.IPNet {
|
|
|
+ return c.mainHostMap.preferredRanges
|
|
|
+}
|
|
|
+
|
|
|
+func (c *HandshakeManager) ForEachVpnIp(f controlEach) {
|
|
|
+ c.RLock()
|
|
|
+ defer c.RUnlock()
|
|
|
+
|
|
|
+ for _, v := range c.vpnIps {
|
|
|
+ f(v)
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+func (c *HandshakeManager) ForEachIndex(f controlEach) {
|
|
|
+ c.RLock()
|
|
|
+ defer c.RUnlock()
|
|
|
+
|
|
|
+ for _, v := range c.indexes {
|
|
|
+ f(v)
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
func (c *HandshakeManager) EmitStats() {
|
|
|
- c.pendingHostMap.EmitStats("pending")
|
|
|
- c.mainHostMap.EmitStats("main")
|
|
|
+ c.RLock()
|
|
|
+ hostLen := len(c.vpnIps)
|
|
|
+ indexLen := len(c.indexes)
|
|
|
+ c.RUnlock()
|
|
|
+
|
|
|
+ metrics.GetOrRegisterGauge("hostmap.pending.hosts", nil).Update(int64(hostLen))
|
|
|
+ metrics.GetOrRegisterGauge("hostmap.pending.indexes", nil).Update(int64(indexLen))
|
|
|
+ c.mainHostMap.EmitStats()
|
|
|
}
|
|
|
|
|
|
// Utility functions below
|