More LH cleanup (#429)

This commit is contained in:
Nathan Brown 2021-04-01 10:23:31 -05:00 committed by GitHub
parent 75f7bda0a4
commit 64d8e5aa96
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 246 additions and 248 deletions

View File

@ -127,7 +127,7 @@ proto: nebula.pb.go cert/cert.pb.go
nebula.pb.go: nebula.proto .FORCE
go build github.com/gogo/protobuf/protoc-gen-gogofaster
PATH="$(CURDIR):$(PATH)" protoc --gogofaster_out=. $<
PATH="$(CURDIR):$(PATH)" protoc --gogofaster_out=paths=source_relative:. $<
rm protoc-gen-gogofaster
cert/cert.pb.go: cert/cert.proto .FORCE

View File

@ -33,6 +33,34 @@ func (al *AllowList) Allow(ip net.IP) bool {
}
}
func (al *AllowList) AllowIpV4(ip uint32) bool {
if al == nil {
return true
}
result := al.cidrTree.MostSpecificContainsIpV4(ip)
switch v := result.(type) {
case bool:
return v
default:
panic(fmt.Errorf("invalid state, allowlist returned: %T %v", result, result))
}
}
func (al *AllowList) AllowIpV6(hi, lo uint64) bool {
if al == nil {
return true
}
result := al.cidrTree.MostSpecificContainsIpV6(hi, lo)
switch v := result.(type) {
case bool:
return v
default:
panic(fmt.Errorf("invalid state, allowlist returned: %T %v", result, result))
}
}
func (al *AllowList) AllowName(name string) bool {
if al == nil || len(al.nameRules) == 0 {
return true

View File

@ -5,6 +5,8 @@ import (
"net"
)
const startbit6 = uint64(1 << 63)
type CIDR6Tree struct {
root4 *CIDRNode
root6 *CIDRNode
@ -71,45 +73,6 @@ func (tree *CIDR6Tree) AddCIDR(cidr *net.IPNet, val interface{}) {
node.value = val
}
// Finds the first match, which may be the least specific
func (tree *CIDR6Tree) Contains(ip net.IP) (value interface{}) {
var node *CIDRNode
wholeIP, ipv4 := isIPV4(ip)
if ipv4 {
node = tree.root4
} else {
node = tree.root6
}
for i := 0; i < len(wholeIP); i += 4 {
ip := ip2int(wholeIP[i : i+4])
bit := startbit
for node != nil {
if node.value != nil {
return node.value
}
// Check if we have reached the end and the above return did not trigger, move to the next uint32 if available
if bit == 0 {
break
}
if ip&bit != 0 {
node = node.right
} else {
node = node.left
}
bit >>= 1
}
}
// Nothing found
return
}
// Finds the most specific match
func (tree *CIDR6Tree) MostSpecificContains(ip net.IP) (value interface{}) {
var node *CIDRNode
@ -147,23 +110,43 @@ func (tree *CIDR6Tree) MostSpecificContains(ip net.IP) (value interface{}) {
return value
}
// Finds the most specific match
func (tree *CIDR6Tree) Match(ip net.IP) (value interface{}) {
var node *CIDRNode
var bit uint32
func (tree *CIDR6Tree) MostSpecificContainsIpV4(ip uint32) (value interface{}) {
bit := startbit
node := tree.root4
wholeIP, ipv4 := isIPV4(ip)
if ipv4 {
node = tree.root4
} else {
node = tree.root6
for node != nil {
if node.value != nil {
value = node.value
}
if ip&bit != 0 {
node = node.right
} else {
node = node.left
}
bit >>= 1
}
for i := 0; i < len(wholeIP); i += 4 {
ip := ip2int(wholeIP[i : i+4])
bit = startbit
return value
}
func (tree *CIDR6Tree) MostSpecificContainsIpV6(hi, lo uint64) (value interface{}) {
ip := hi
node := tree.root6
for i := 0; i < 2; i++ {
bit := startbit6
for node != nil {
if node.value != nil {
value = node.value
}
if bit == 0 {
break
}
for node != nil && bit > 0 {
if ip&bit != 0 {
node = node.right
} else {
@ -172,10 +155,8 @@ func (tree *CIDR6Tree) Match(ip net.IP) (value interface{}) {
bit >>= 1
}
}
if bit == 0 && node != nil {
value = node.value
ip = lo
}
return value

View File

@ -7,52 +7,6 @@ import (
"github.com/stretchr/testify/assert"
)
func TestCIDR6Tree_Contains(t *testing.T) {
tree := NewCIDR6Tree()
tree.AddCIDR(getCIDR("1.0.0.0/8"), "1")
tree.AddCIDR(getCIDR("2.1.0.0/16"), "2")
tree.AddCIDR(getCIDR("3.1.1.0/24"), "3")
tree.AddCIDR(getCIDR("4.1.1.0/24"), "4a")
tree.AddCIDR(getCIDR("4.1.1.1/32"), "4b")
tree.AddCIDR(getCIDR("4.1.2.1/32"), "4c")
tree.AddCIDR(getCIDR("254.0.0.0/4"), "5")
tree.AddCIDR(getCIDR("2800::FFFF:1/128"), "a")
tree.AddCIDR(getCIDR("2800:1:2:3::0/64"), "b")
tests := []struct {
Result interface{}
IP string
}{
{"1", "1.0.0.0"},
{"1", "1.255.255.255"},
{"2", "2.1.0.0"},
{"2", "2.1.255.255"},
{"3", "3.1.1.0"},
{"3", "3.1.1.255"},
{"4a", "4.1.1.255"},
{"4a", "4.1.1.1"},
{"5", "240.0.0.0"},
{"5", "255.255.255.255"},
{nil, "239.0.0.0"},
{nil, "4.1.2.2"},
{"a", "2800::FFFF:1"},
{"b", "2800:1:2:3::1"},
{"b", "2800:1:2:3::6"},
}
for _, tt := range tests {
assert.Equal(t, tt.Result, tree.Contains(net.ParseIP(tt.IP)))
}
tree = NewCIDR6Tree()
tree.AddCIDR(getCIDR("1.1.1.1/0"), "cool")
tree.AddCIDR(getCIDR("::/0"), "cool6")
assert.Equal(t, "cool", tree.Contains(net.ParseIP("0.0.0.0")))
assert.Equal(t, "cool", tree.Contains(net.ParseIP("255.255.255.255")))
assert.Equal(t, "cool6", tree.Contains(net.ParseIP("::")))
assert.Equal(t, "cool6", tree.Contains(net.ParseIP("1:2:3:4:5:6:7:8")))
}
func TestCIDR6Tree_MostSpecificContains(t *testing.T) {
tree := NewCIDR6Tree()
tree.AddCIDR(getCIDR("1.0.0.0/8"), "1")
@ -97,38 +51,27 @@ func TestCIDR6Tree_MostSpecificContains(t *testing.T) {
tree.AddCIDR(getCIDR("::/0"), "cool6")
assert.Equal(t, "cool", tree.MostSpecificContains(net.ParseIP("0.0.0.0")))
assert.Equal(t, "cool", tree.MostSpecificContains(net.ParseIP("255.255.255.255")))
assert.Equal(t, "cool6", tree.Contains(net.ParseIP("::")))
assert.Equal(t, "cool6", tree.Contains(net.ParseIP("1:2:3:4:5:6:7:8")))
assert.Equal(t, "cool6", tree.MostSpecificContains(net.ParseIP("::")))
assert.Equal(t, "cool6", tree.MostSpecificContains(net.ParseIP("1:2:3:4:5:6:7:8")))
}
func TestCIDR6Tree_Match(t *testing.T) {
func TestCIDR6Tree_MostSpecificContainsIpV6(t *testing.T) {
tree := NewCIDR6Tree()
tree.AddCIDR(getCIDR("4.1.1.0/32"), "1a")
tree.AddCIDR(getCIDR("4.1.1.1/32"), "1b")
tree.AddCIDR(getCIDR("1:2:3:4:5:6:7:8/128"), "2a")
tree.AddCIDR(getCIDR("1:2:3:4:5:6:7:0/128"), "2b")
tree.AddCIDR(getCIDR("1:2:0:4:5:0:0:0/64"), "6a")
tree.AddCIDR(getCIDR("1:2:0:4:5:0:0:0/80"), "6b")
tree.AddCIDR(getCIDR("1:2:0:4:5:0:0:0/96"), "6c")
tests := []struct {
Result interface{}
IP string
}{
{"1a", "4.1.1.0"},
{"1b", "4.1.1.1"},
{nil, "4.1.1.2"},
{"2a", "1:2:3:4:5:6:7:8"},
{"2b", "1:2:3:4:5:6:7:0"},
{nil, "1:2:3:4:5:6:7:9"},
{"6a", "1:2:0:4:1:1:1:1"},
{"6b", "1:2:0:4:5:1:1:1"},
{"6c", "1:2:0:4:5:0:0:0"},
}
for _, tt := range tests {
assert.Equal(t, tt.Result, tree.Match(net.ParseIP(tt.IP)))
ip := NewIp6AndPort(net.ParseIP(tt.IP), 0)
assert.Equal(t, tt.Result, tree.MostSpecificContainsIpV6(ip.Hi, ip.Lo))
}
tree = NewCIDR6Tree()
tree.AddCIDR(getCIDR("1.1.1.1/0"), "cool")
tree.AddCIDR(getCIDR("1:2:3:4:5:6:7:8/0"), "cool6")
assert.Equal(t, "cool", tree.Contains(net.ParseIP("0.0.0.0")))
assert.Equal(t, "cool", tree.Contains(net.ParseIP("255.255.255.255")))
assert.Equal(t, "cool6", tree.Contains(net.ParseIP("::")))
assert.Equal(t, "cool6", tree.Contains(net.ParseIP("1:2:3:4:5:6:7:8")))
}

View File

@ -116,7 +116,6 @@ func (tree *CIDRTree) MostSpecificContains(ip uint32) (value interface{}) {
}
bit >>= 1
}
return value

View File

@ -29,7 +29,7 @@ func Test_NewConnectionManagerTest(t *testing.T) {
rawCertificateNoKey: []byte{},
}
lh := NewLightHouse(l, false, 0, []uint32{}, 1000, 0, &udpConn{}, false, 1, false)
lh := NewLightHouse(l, false, &net.IPNet{IP: net.IP{0, 0, 0, 0}, Mask: net.IPMask{0, 0, 0, 0}}, []uint32{}, 1000, 0, &udpConn{}, false, 1, false)
ifce := &Interface{
hostMap: hostMap,
inside: &Tun{},
@ -96,7 +96,7 @@ func Test_NewConnectionManagerTest2(t *testing.T) {
rawCertificateNoKey: []byte{},
}
lh := NewLightHouse(l, false, 0, []uint32{}, 1000, 0, &udpConn{}, false, 1, false)
lh := NewLightHouse(l, false, &net.IPNet{IP: net.IP{0, 0, 0, 0}, Mask: net.IPMask{0, 0, 0, 0}}, []uint32{}, 1000, 0, &udpConn{}, false, 1, false)
ifce := &Interface{
hostMap: hostMap,
inside: &Tun{},

View File

@ -31,7 +31,7 @@ type m map[string]interface{}
func newSimpleServer(caCrt *cert.NebulaCertificate, caKey []byte, name string, udpIp net.IP) (*nebula.Control, net.IP, *net.UDPAddr) {
l := NewTestLogger()
vpnIpNet := &net.IPNet{IP: make([]byte, len(udpIp)), Mask: net.IPMask{0, 0, 0, 0}}
vpnIpNet := &net.IPNet{IP: make([]byte, len(udpIp)), Mask: net.IPMask{255, 255, 255, 0}}
copy(vpnIpNet.IP, udpIp)
vpnIpNet.IP[1] += 128
udpAddr := net.UDPAddr{

View File

@ -20,7 +20,7 @@ func (f *Interface) consumeInsidePacket(packet []byte, fwPacket *FirewallPacket,
}
// Ignore packets from self to self
if fwPacket.RemoteIP == f.lightHouse.myIp {
if fwPacket.RemoteIP == f.myVpnIp {
return
}

View File

@ -61,6 +61,7 @@ type Interface struct {
createTime time.Time
lightHouse *LightHouse
localBroadcast uint32
myVpnIp uint32
dropLocalBroadcast bool
dropMulticast bool
udpBatchSize int
@ -115,6 +116,7 @@ func NewInterface(c *InterfaceConfig) (*Interface, error) {
writers: make([]*udpConn, c.routines),
readers: make([]io.ReadWriteCloser, c.routines),
caPool: c.caPool,
myVpnIp: ip2int(c.certState.certificate.Details.Ips[0].IP),
conntrackCacheTimeout: c.ConntrackCacheTimeout,

View File

@ -1,7 +1,7 @@
package nebula
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"net"
@ -13,9 +13,7 @@ import (
"github.com/sirupsen/logrus"
)
//TODO: if the pb code for ipv6 used a fixed data type we could save more work
//TODO: nodes are roaming lighthouses, this is bad. How are they learning?
//TODO: as a lh client, ignore any address within my nebula network?????
var ErrHostNotKnown = errors.New("host not known")
@ -25,11 +23,12 @@ const maxAddrs = 10
type ip4And6 struct {
//TODO: adding a lock here could allow us to release the lock on lh.addrMap quicker
// v4 and v6 store addresses that have been self reported by the client
// v4 and v6 store addresses that have been self reported by the client in a server or where all addresses are stored on a client
v4 []*Ip4AndPort
v6 []*Ip6AndPort
// Learned addresses are ones that a client does not know about but a lighthouse learned from as a result of the received packet
// This is only used if you are a lighthouse server
learnedV4 []*Ip4AndPort
learnedV6 []*Ip6AndPort
}
@ -38,7 +37,8 @@ type LightHouse struct {
//TODO: We need a timer wheel to kick out vpnIps that haven't reported in a long time
sync.RWMutex //Because we concurrently read and write to our maps
amLighthouse bool
myIp uint32
myVpnIp uint32
myVpnZeros uint32
punchConn *udpConn
// Local cache of answers from light houses
@ -75,10 +75,12 @@ type EncWriter interface {
SendMessageToVpnIp(t NebulaMessageType, st NebulaMessageSubType, vpnIp uint32, p, nb, out []byte)
}
func NewLightHouse(l *logrus.Logger, amLighthouse bool, myIp uint32, ips []uint32, interval int, nebulaPort uint32, pc *udpConn, punchBack bool, punchDelay time.Duration, metricsEnabled bool) *LightHouse {
func NewLightHouse(l *logrus.Logger, amLighthouse bool, myVpnIpNet *net.IPNet, ips []uint32, interval int, nebulaPort uint32, pc *udpConn, punchBack bool, punchDelay time.Duration, metricsEnabled bool) *LightHouse {
ones, _ := myVpnIpNet.Mask.Size()
h := LightHouse{
amLighthouse: amLighthouse,
myIp: myIp,
myVpnIp: ip2int(myVpnIpNet.IP),
myVpnZeros: uint32(32 - ones),
addrMap: make(map[uint32]*ip4And6),
nebulaPort: nebulaPort,
lighthouses: make(map[uint32]struct{}),
@ -216,16 +218,11 @@ func (lh *LightHouse) AddRemote(vpnIP uint32, toAddr *udpAddr, static bool) {
}
}
// unsafeGetAddrs assumes you have the lh lock
func (lh *LightHouse) unsafeGetAddrs(vpnIP uint32) *ip4And6 {
// unlockedGetAddrs assumes you have the lh lock
func (lh *LightHouse) unlockedGetAddrs(vpnIP uint32) *ip4And6 {
am, ok := lh.addrMap[vpnIP]
if !ok {
am = &ip4And6{
v4: make([]*Ip4AndPort, 0),
v6: make([]*Ip6AndPort, 0),
learnedV4: make([]*Ip4AndPort, 0),
learnedV6: make([]*Ip6AndPort, 0),
}
am = &ip4And6{}
lh.addrMap[vpnIP] = am
}
return am
@ -243,7 +240,7 @@ func (lh *LightHouse) addRemoteV4(vpnIP uint32, to *Ip4AndPort, static bool, lea
lh.Lock()
defer lh.Unlock()
am := lh.unsafeGetAddrs(vpnIP)
am := lh.unlockedGetAddrs(vpnIP)
if learned {
if !lh.unlockedShouldAddV4(am.learnedV4, to) {
@ -270,13 +267,12 @@ func prependAndLimitV4(cache []*Ip4AndPort, to *Ip4AndPort) []*Ip4AndPort {
// unlockedShouldAddV4 checks if to is allowed by our allow list and is not already present in the cache
func (lh *LightHouse) unlockedShouldAddV4(am []*Ip4AndPort, to *Ip4AndPort) bool {
ip := int2ip(to.Ip)
allow := lh.remoteAllowList.Allow(ip)
if lh.l.Level >= logrus.DebugLevel {
lh.l.WithField("remoteIp", ip).WithField("allow", allow).Debug("remoteAllowList.Allow")
allow := lh.remoteAllowList.AllowIpV4(to.Ip)
if lh.l.Level >= logrus.TraceLevel {
lh.l.WithField("remoteIp", IntIp(to.Ip)).WithField("allow", allow).Trace("remoteAllowList.Allow")
}
if !allow {
if !allow || ipMaskContains(lh.myVpnIp, lh.myVpnZeros, to.Ip) {
return false
}
@ -301,7 +297,7 @@ func (lh *LightHouse) addRemoteV6(vpnIP uint32, to *Ip6AndPort, static bool, lea
lh.Lock()
defer lh.Unlock()
am := lh.unsafeGetAddrs(vpnIP)
am := lh.unlockedGetAddrs(vpnIP)
if learned {
if !lh.unlockedShouldAddV6(am.learnedV6, to) {
@ -328,10 +324,9 @@ func prependAndLimitV6(cache []*Ip6AndPort, to *Ip6AndPort) []*Ip6AndPort {
// unlockedShouldAddV6 checks if to is allowed by our allow list and is not already present in the cache
func (lh *LightHouse) unlockedShouldAddV6(am []*Ip6AndPort, to *Ip6AndPort) bool {
ip := net.IP(to.Ip)
allow := lh.remoteAllowList.Allow(ip)
if lh.l.Level >= logrus.DebugLevel {
lh.l.WithField("remoteIp", ip).WithField("allow", allow).Debug("remoteAllowList.Allow")
allow := lh.remoteAllowList.AllowIpV6(to.Hi, to.Lo)
if lh.l.Level >= logrus.TraceLevel {
lh.l.WithField("remoteIp", lhIp6ToIp(to)).WithField("allow", allow).Trace("remoteAllowList.Allow")
}
if !allow {
@ -339,7 +334,7 @@ func (lh *LightHouse) unlockedShouldAddV6(am []*Ip6AndPort, to *Ip6AndPort) bool
}
for _, v := range am {
if bytes.Equal(v.Ip, to.Ip) && v.Port == to.Port {
if v.Hi == to.Hi && v.Lo == to.Lo && v.Port == to.Port {
return false
}
}
@ -347,6 +342,13 @@ func (lh *LightHouse) unlockedShouldAddV6(am []*Ip6AndPort, to *Ip6AndPort) bool
return true
}
func lhIp6ToIp(v *Ip6AndPort) net.IP {
ip := make(net.IP, 16)
binary.BigEndian.PutUint64(ip[:8], v.Hi)
binary.BigEndian.PutUint64(ip[8:], v.Lo)
return ip
}
func (lh *LightHouse) AddRemoteAndReset(vpnIP uint32, toIp *udpAddr) {
if lh.amLighthouse {
lh.DeleteVpnIP(vpnIP)
@ -377,10 +379,11 @@ func NewIp4AndPort(ip net.IP, port uint32) *Ip4AndPort {
}
func NewIp6AndPort(ip net.IP, port uint32) *Ip6AndPort {
ipp := Ip6AndPort{Port: port}
ipp.Ip = make([]byte, len(ip))
copy(ipp.Ip, ip)
return &ipp
return &Ip6AndPort{
Hi: binary.BigEndian.Uint64(ip[:8]),
Lo: binary.BigEndian.Uint64(ip[8:]),
Port: port,
}
}
func NewUDPAddrFromLH4(ipp *Ip4AndPort) *udpAddr {
@ -392,7 +395,7 @@ func NewUDPAddrFromLH4(ipp *Ip4AndPort) *udpAddr {
}
func NewUDPAddrFromLH6(ipp *Ip6AndPort) *udpAddr {
return NewUDPAddr(ipp.Ip, uint16(ipp.Port))
return NewUDPAddr(lhIp6ToIp(ipp), uint16(ipp.Port))
}
func (lh *LightHouse) LhUpdateWorker(f EncWriter) {
@ -411,7 +414,7 @@ func (lh *LightHouse) SendUpdate(f EncWriter) {
var v6 []*Ip6AndPort
for _, e := range *localIps(lh.l, lh.localAllowList) {
if ip2int(e) == lh.myIp {
if ip4 := e.To4(); ip4 != nil && ipMaskContains(lh.myVpnIp, lh.myVpnZeros, ip2int(ip4)) {
continue
}
@ -425,7 +428,7 @@ func (lh *LightHouse) SendUpdate(f EncWriter) {
m := &NebulaMeta{
Type: NebulaMeta_HostUpdateNotification,
Details: &NebulaMetaDetails{
VpnIp: lh.myIp,
VpnIp: lh.myVpnIp,
Ip4AndPorts: v4,
Ip6AndPorts: v6,
},
@ -434,14 +437,15 @@ func (lh *LightHouse) SendUpdate(f EncWriter) {
lh.metricTx(NebulaMeta_HostUpdateNotification, int64(len(lh.lighthouses)))
nb := make([]byte, 12, 12)
out := make([]byte, mtu)
for vpnIp := range lh.lighthouses {
mm, err := proto.Marshal(m)
if err != nil && lh.l.Level >= logrus.DebugLevel {
lh.l.Debugf("Invalid marshal to update")
}
//l.Error("LIGHTHOUSE PACKET SEND", mm)
f.SendMessageToVpnIp(lightHouse, 0, vpnIp, mm, nb, out)
mm, err := proto.Marshal(m)
if err != nil && lh.l.Level >= logrus.DebugLevel {
lh.l.WithError(err).Error("Error while marshaling for lighthouse update")
return
}
for vpnIp := range lh.lighthouses {
f.SendMessageToVpnIp(lightHouse, 0, vpnIp, mm, nb, out)
}
}
@ -634,10 +638,9 @@ func (lhh *LightHouseHandler) handleHostUpdateNotification(n *NebulaMeta, vpnIp
lhh.lh.Lock()
defer lhh.lh.Unlock()
am := lhh.lh.unsafeGetAddrs(vpnIp)
am := lhh.lh.unlockedGetAddrs(vpnIp)
//TODO: other note on a lock for am so we can release more quickly and lock our real unit of change which is far less contended
//TODO: we are not filtering by local or remote allowed addrs here, is this an ok change to make?
// We don't accumulate addresses being told to us
am.v4 = am.v4[:0]
@ -739,3 +742,9 @@ func TransformLHReplyToUdpAddrs(ips *ip4And6) []*udpAddr {
return addrs
}
// ipMaskContains checks if testIp is contained by ip after applying a cidr
// zeros is 32 - bits from net.IPMask.Size()
func ipMaskContains(ip uint32, zeros uint32, testIp uint32) bool {
return (testIp^ip)>>zeros == 0
}

View File

@ -48,7 +48,7 @@ func Test_lhStaticMapping(t *testing.T) {
udpServer, _ := NewListener(l, "0.0.0.0", 0, true)
meh := NewLightHouse(l, true, 1, []uint32{ip2int(lh1IP)}, 10, 10003, udpServer, false, 1, false)
meh := NewLightHouse(l, true, &net.IPNet{IP: net.IP{0, 0, 0, 1}, Mask: net.IPMask{0, 0, 0, 0}}, []uint32{ip2int(lh1IP)}, 10, 10003, udpServer, false, 1, false)
meh.AddRemote(ip2int(lh1IP), NewUDPAddr(lh1IP, uint16(4242)), true)
err := meh.ValidateLHStaticEntries()
assert.Nil(t, err)
@ -56,7 +56,7 @@ func Test_lhStaticMapping(t *testing.T) {
lh2 := "10.128.0.3"
lh2IP := net.ParseIP(lh2)
meh = NewLightHouse(l, true, 1, []uint32{ip2int(lh1IP), ip2int(lh2IP)}, 10, 10003, udpServer, false, 1, false)
meh = NewLightHouse(l, true, &net.IPNet{IP: net.IP{0, 0, 0, 1}, Mask: net.IPMask{0, 0, 0, 0}}, []uint32{ip2int(lh1IP), ip2int(lh2IP)}, 10, 10003, udpServer, false, 1, false)
meh.AddRemote(ip2int(lh1IP), NewUDPAddr(lh1IP, uint16(4242)), true)
err = meh.ValidateLHStaticEntries()
assert.EqualError(t, err, "Lighthouse 10.128.0.3 does not have a static_host_map entry")
@ -69,7 +69,7 @@ func BenchmarkLighthouseHandleRequest(b *testing.B) {
udpServer, _ := NewListener(l, "0.0.0.0", 0, true)
lh := NewLightHouse(l, true, 1, []uint32{ip2int(lh1IP)}, 10, 10003, udpServer, false, 1, false)
lh := NewLightHouse(l, true, &net.IPNet{IP: net.IP{0, 0, 0, 1}, Mask: net.IPMask{0, 0, 0, 0}}, []uint32{ip2int(lh1IP)}, 10, 10003, udpServer, false, 1, false)
hAddr := NewUDPAddrFromString("4.5.6.7:12345")
hAddr2 := NewUDPAddrFromString("4.5.6.7:12346")
@ -144,9 +144,8 @@ func TestLighthouse_Memory(t *testing.T) {
theirUdpAddr4 := &udpAddr{IP: net.ParseIP("24.15.0.3"), Port: 4242}
theirVpnIp := ip2int(net.ParseIP("10.128.0.3"))
lhIP := net.ParseIP("10.128.0.1")
udpServer, _ := NewListener(l, "0.0.0.0", 0, true)
lh := NewLightHouse(l, true, 1, []uint32{ip2int(lhIP)}, 10, 10003, udpServer, false, 1, false)
lh := NewLightHouse(l, true, &net.IPNet{IP: net.IP{10, 128, 0, 1}, Mask: net.IPMask{255, 255, 255, 0}}, []uint32{}, 10, 10003, udpServer, false, 1, false)
lhh := lh.NewRequestHandler()
// Test that my first update responds with just that
@ -173,7 +172,7 @@ func TestLighthouse_Memory(t *testing.T) {
r = newLHHostRequest(myUdpAddr0, myVpnIp, myVpnIp, lhh)
assertIp4InArray(t, r.msg.Details.Ip4AndPorts, myUdpAddr1, myUdpAddr4)
// Finally ensure proper ordering and limiting
// Ensure proper ordering and limiting
// Send 12 addrs, get 10 back, one removed on a dupe check the other by limiting
newLHHostUpdate(
myUdpAddr0,
@ -198,6 +197,14 @@ func TestLighthouse_Memory(t *testing.T) {
r.msg.Details.Ip4AndPorts,
myUdpAddr1, myUdpAddr2, myUdpAddr3, myUdpAddr4, myUdpAddr5, myUdpAddr6, myUdpAddr7, myUdpAddr8, myUdpAddr9, myUdpAddr10,
)
// Make sure we won't add ips in our vpn network
bad1 := &udpAddr{IP: net.ParseIP("10.128.0.99"), Port: 4242}
bad2 := &udpAddr{IP: net.ParseIP("10.128.0.100"), Port: 4242}
good := &udpAddr{IP: net.ParseIP("1.128.0.99"), Port: 4242}
newLHHostUpdate(myUdpAddr0, myVpnIp, []*udpAddr{bad1, bad2, good}, lhh)
r = newLHHostRequest(myUdpAddr0, myVpnIp, myVpnIp, lhh)
assertIp4InArray(t, r.msg.Details.Ip4AndPorts, good)
}
func newLHHostRequest(fromAddr *udpAddr, myVpnIp, queryVpnIp uint32, lhh *LightHouseHandler) testLhReply {
@ -254,7 +261,7 @@ func Test_lhRemoteAllowList(t *testing.T) {
udpServer, _ := NewListener(l, "0.0.0.0", 0, true)
lh := NewLightHouse(l, true, 1, []uint32{ip2int(lh1IP)}, 10, 10003, udpServer, false, 1, false)
lh := NewLightHouse(l, true, &net.IPNet{IP: net.IP{0, 0, 0, 1}, Mask: net.IPMask{255, 255, 255, 0}}, []uint32{ip2int(lh1IP)}, 10, 10003, udpServer, false, 1, false)
lh.SetRemoteAllowList(allowList)
// A disallowed ip should not enter the cache but we should end up with an empty entry in the addrMap
@ -306,6 +313,12 @@ func Test_lhRemoteAllowList(t *testing.T) {
)
}
func Test_ipMaskContains(t *testing.T) {
assert.True(t, ipMaskContains(ip2int(net.ParseIP("10.0.0.1")), 32-24, ip2int(net.ParseIP("10.0.0.255"))))
assert.False(t, ipMaskContains(ip2int(net.ParseIP("10.0.0.1")), 32-24, ip2int(net.ParseIP("10.0.1.1"))))
assert.True(t, ipMaskContains(ip2int(net.ParseIP("10.0.0.1")), 32, ip2int(net.ParseIP("10.0.1.1"))))
}
type testLhReply struct {
nebType NebulaMessageType
nebSubType NebulaMessageSubType

View File

@ -266,7 +266,7 @@ func Main(config *Config, configTest bool, buildVersion string, logger *logrus.L
lightHouse := NewLightHouse(
l,
amLighthouse,
ip2int(tunCidr.IP),
tunCidr,
lighthouseHosts,
//TODO: change to a duration
config.GetInt("lighthouse.interval", 10),

View File

@ -269,8 +269,9 @@ func (m *Ip4AndPort) GetPort() uint32 {
}
type Ip6AndPort struct {
Ip []byte `protobuf:"bytes,1,opt,name=Ip,proto3" json:"Ip,omitempty"`
Port uint32 `protobuf:"varint,2,opt,name=Port,proto3" json:"Port,omitempty"`
Hi uint64 `protobuf:"varint,1,opt,name=Hi,proto3" json:"Hi,omitempty"`
Lo uint64 `protobuf:"varint,2,opt,name=Lo,proto3" json:"Lo,omitempty"`
Port uint32 `protobuf:"varint,3,opt,name=Port,proto3" json:"Port,omitempty"`
}
func (m *Ip6AndPort) Reset() { *m = Ip6AndPort{} }
@ -306,11 +307,18 @@ func (m *Ip6AndPort) XXX_DiscardUnknown() {
var xxx_messageInfo_Ip6AndPort proto.InternalMessageInfo
func (m *Ip6AndPort) GetIp() []byte {
func (m *Ip6AndPort) GetHi() uint64 {
if m != nil {
return m.Ip
return m.Hi
}
return nil
return 0
}
func (m *Ip6AndPort) GetLo() uint64 {
if m != nil {
return m.Lo
}
return 0
}
func (m *Ip6AndPort) GetPort() uint32 {
@ -515,40 +523,43 @@ func init() {
func init() { proto.RegisterFile("nebula.proto", fileDescriptor_2d65afa7693df5ef) }
var fileDescriptor_2d65afa7693df5ef = []byte{
// 527 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x54, 0xbb, 0x8e, 0xda, 0x40,
0x14, 0x65, 0xc0, 0xc0, 0x72, 0x79, 0xac, 0x73, 0x93, 0x20, 0x6f, 0x0a, 0x6b, 0xe5, 0x22, 0xa2,
0x42, 0x11, 0xbb, 0x42, 0x69, 0x13, 0x52, 0x40, 0x01, 0x22, 0xd6, 0x26, 0x29, 0xa3, 0x59, 0x3c,
0x59, 0x2c, 0x60, 0xc6, 0xb2, 0x87, 0x68, 0xf9, 0x8b, 0x7c, 0x46, 0xba, 0xfc, 0x46, 0x8a, 0x14,
0x5b, 0xa4, 0x48, 0x19, 0xc1, 0x8f, 0x44, 0x33, 0x36, 0x36, 0x6b, 0x36, 0xdb, 0xdd, 0xc7, 0x39,
0x97, 0xc3, 0x99, 0x23, 0x43, 0x83, 0xb3, 0xeb, 0xf5, 0x92, 0x76, 0x83, 0x50, 0x48, 0x81, 0x95,
0xb8, 0x73, 0x7e, 0x15, 0x01, 0x26, 0xba, 0x1c, 0x33, 0x49, 0xb1, 0x07, 0xc6, 0xd5, 0x26, 0x60,
0x16, 0x39, 0x27, 0x9d, 0x56, 0xcf, 0xee, 0x26, 0x9c, 0x0c, 0xd1, 0x1d, 0xb3, 0x28, 0xa2, 0x37,
0x4c, 0xa1, 0x5c, 0x8d, 0xc5, 0x0b, 0xa8, 0xbe, 0x63, 0x92, 0xfa, 0xcb, 0xc8, 0x2a, 0x9e, 0x93,
0x4e, 0xbd, 0x77, 0x76, 0x4c, 0x4b, 0x00, 0xee, 0x1e, 0xe9, 0xfc, 0x26, 0x50, 0x3f, 0x38, 0x85,
0x27, 0x60, 0x4c, 0x04, 0x67, 0x66, 0x01, 0x9b, 0x50, 0x1b, 0x8a, 0x48, 0xbe, 0x5f, 0xb3, 0x70,
0x63, 0x12, 0x44, 0x68, 0xa5, 0xad, 0xcb, 0x82, 0xe5, 0xc6, 0x2c, 0xe2, 0x0b, 0x68, 0xab, 0xd9,
0x87, 0xc0, 0xa3, 0x92, 0x4d, 0x84, 0xf4, 0xbf, 0xf8, 0x33, 0x2a, 0x7d, 0xc1, 0xcd, 0x12, 0x9e,
0xc1, 0x73, 0xb5, 0x1b, 0x8b, 0xaf, 0xcc, 0xbb, 0xb7, 0x32, 0xf6, 0xab, 0xe9, 0x9a, 0xcf, 0xe6,
0xf7, 0x56, 0x65, 0x6c, 0x01, 0xa8, 0xd5, 0xa7, 0xb9, 0xa0, 0x2b, 0xdf, 0xac, 0xe0, 0x53, 0x38,
0xcd, 0xfa, 0xf8, 0x67, 0xab, 0x4a, 0xd9, 0x94, 0xca, 0xf9, 0x60, 0xce, 0x66, 0x0b, 0xf3, 0x44,
0x29, 0x4b, 0xdb, 0x18, 0x52, 0x73, 0x7e, 0x10, 0x78, 0x72, 0xf4, 0xaf, 0xf1, 0x19, 0x94, 0x3f,
0x06, 0x7c, 0x14, 0x68, 0x5b, 0x9b, 0x6e, 0xdc, 0xe0, 0x25, 0xd4, 0x47, 0xc1, 0xe5, 0x1b, 0xee,
0x4d, 0x45, 0x28, 0x95, 0x77, 0xa5, 0x4e, 0xbd, 0x87, 0x7b, 0xef, 0xb2, 0x95, 0x7b, 0x08, 0x8b,
0x59, 0xfd, 0x94, 0x65, 0xe4, 0x59, 0xfd, 0x03, 0x56, 0x0a, 0x43, 0x0b, 0xaa, 0x33, 0xb1, 0xe6,
0x92, 0x85, 0x56, 0x49, 0x6b, 0xd8, 0xb7, 0xce, 0x2b, 0x80, 0xec, 0x3c, 0xb6, 0xa0, 0x98, 0xca,
0x2c, 0x8e, 0x02, 0x44, 0x30, 0xd4, 0x5c, 0x3f, 0x6c, 0xd3, 0xd5, 0x75, 0xcc, 0xe8, 0x1f, 0x33,
0x1a, 0xff, 0x65, 0xdc, 0xee, 0x33, 0x36, 0xf5, 0xf9, 0xcd, 0xe3, 0x19, 0x53, 0x88, 0x07, 0x32,
0x86, 0x60, 0x5c, 0xf9, 0x2b, 0xa6, 0xaf, 0x1a, 0xae, 0xae, 0x1d, 0xe7, 0x28, 0x41, 0x8a, 0x6c,
0x16, 0xb0, 0x06, 0xe5, 0xf8, 0x3d, 0x88, 0xf3, 0x19, 0x4e, 0xe3, 0xbb, 0x43, 0xca, 0xbd, 0x68,
0x4e, 0x17, 0x0c, 0x5f, 0x67, 0x71, 0x25, 0x3a, 0xae, 0x39, 0x05, 0x29, 0x32, 0x9f, 0x59, 0x25,
0x62, 0xb8, 0xa2, 0x33, 0x2d, 0xa2, 0xe1, 0xea, 0xda, 0xf9, 0x4e, 0xa0, 0xfd, 0x30, 0x4f, 0xc1,
0x07, 0x2c, 0x94, 0x89, 0x37, 0xba, 0xc6, 0x97, 0xd0, 0x1a, 0x71, 0x5f, 0xfa, 0x54, 0x8a, 0x70,
0xc4, 0x3d, 0x76, 0x9b, 0xf8, 0x94, 0x9b, 0x2a, 0x9c, 0xcb, 0xa2, 0x40, 0x70, 0x8f, 0x25, 0xb8,
0xf8, 0xd9, 0x72, 0x53, 0x6c, 0x43, 0x65, 0x20, 0xc4, 0xc2, 0x67, 0x96, 0xa1, 0x9d, 0x49, 0xba,
0xd4, 0xaf, 0x72, 0xe6, 0xd7, 0x5b, 0xeb, 0xe7, 0xd6, 0x26, 0x77, 0x5b, 0x9b, 0xfc, 0xdd, 0xda,
0xe4, 0xdb, 0xce, 0x2e, 0xdc, 0xed, 0xec, 0xc2, 0x9f, 0x9d, 0x5d, 0xb8, 0xae, 0xe8, 0x6f, 0xc2,
0xc5, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x36, 0xec, 0xba, 0x62, 0x23, 0x04, 0x00, 0x00,
// 570 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x54, 0x41, 0x6f, 0xda, 0x4c,
0x10, 0x65, 0x8d, 0x21, 0xc9, 0x90, 0x10, 0x7f, 0xfb, 0xb5, 0x08, 0x7a, 0xb0, 0x22, 0x1f, 0x2a,
0x4e, 0xa4, 0x82, 0x08, 0xf5, 0xd8, 0x96, 0x1e, 0x40, 0x0a, 0x88, 0x5a, 0x69, 0x2b, 0xf5, 0x52,
0x2d, 0xf6, 0x16, 0xaf, 0x80, 0x5d, 0xd7, 0x5e, 0xaa, 0xf0, 0x2f, 0xfa, 0x33, 0x7a, 0xeb, 0xdf,
0xe8, 0xa1, 0x87, 0x1c, 0x7a, 0xe8, 0xb1, 0x82, 0x3f, 0x52, 0xed, 0xda, 0xd8, 0x04, 0xa2, 0xde,
0xe6, 0xcd, 0xbc, 0x37, 0x3b, 0x3c, 0x3f, 0x01, 0xa7, 0x9c, 0x4e, 0x96, 0x73, 0xd2, 0x0a, 0x23,
0x21, 0x05, 0x2e, 0x27, 0xc8, 0xf9, 0x69, 0x00, 0x8c, 0x74, 0x39, 0xa4, 0x92, 0xe0, 0x36, 0x98,
0x37, 0xab, 0x90, 0xd6, 0xd1, 0x05, 0x6a, 0x56, 0xdb, 0x76, 0x2b, 0xd5, 0xe4, 0x8c, 0xd6, 0x90,
0xc6, 0x31, 0x99, 0x52, 0xc5, 0x72, 0x35, 0x17, 0x77, 0xe0, 0xe8, 0x35, 0x95, 0x84, 0xcd, 0xe3,
0xba, 0x71, 0x81, 0x9a, 0x95, 0x76, 0xe3, 0x50, 0x96, 0x12, 0xdc, 0x2d, 0xd3, 0xf9, 0x85, 0xa0,
0xb2, 0xb3, 0x0a, 0x1f, 0x83, 0x39, 0x12, 0x9c, 0x5a, 0x05, 0x7c, 0x06, 0x27, 0x7d, 0x11, 0xcb,
0x37, 0x4b, 0x1a, 0xad, 0x2c, 0x84, 0x31, 0x54, 0x33, 0xe8, 0xd2, 0x70, 0xbe, 0xb2, 0x0c, 0xfc,
0x04, 0x6a, 0xaa, 0xf7, 0x36, 0xf4, 0x89, 0xa4, 0x23, 0x21, 0xd9, 0x27, 0xe6, 0x11, 0xc9, 0x04,
0xb7, 0x8a, 0xb8, 0x01, 0x8f, 0xd5, 0x6c, 0x28, 0xbe, 0x50, 0xff, 0xde, 0xc8, 0xdc, 0x8e, 0xc6,
0x4b, 0xee, 0x05, 0xf7, 0x46, 0x25, 0x5c, 0x05, 0x50, 0xa3, 0xf7, 0x81, 0x20, 0x0b, 0x66, 0x95,
0xf1, 0xff, 0x70, 0x9e, 0xe3, 0xe4, 0xd9, 0x23, 0x75, 0xd9, 0x98, 0xc8, 0xa0, 0x17, 0x50, 0x6f,
0x66, 0x1d, 0xab, 0xcb, 0x32, 0x98, 0x50, 0x4e, 0x9c, 0xef, 0x08, 0xfe, 0x3b, 0xf8, 0xd5, 0xf8,
0x11, 0x94, 0xde, 0x85, 0x7c, 0x10, 0x6a, 0x5b, 0xcf, 0xdc, 0x04, 0xe0, 0x2b, 0xa8, 0x0c, 0xc2,
0xab, 0x97, 0xdc, 0x1f, 0x8b, 0x48, 0x2a, 0xef, 0x8a, 0xcd, 0x4a, 0x1b, 0x6f, 0xbd, 0xcb, 0x47,
0xee, 0x2e, 0x2d, 0x51, 0x75, 0x33, 0x95, 0xb9, 0xaf, 0xea, 0xee, 0xa8, 0x32, 0x1a, 0xae, 0xc3,
0x91, 0x27, 0x96, 0x5c, 0xd2, 0xa8, 0x5e, 0xd4, 0x37, 0x6c, 0xa1, 0xf3, 0x0c, 0x20, 0x5f, 0x8f,
0xab, 0x60, 0x64, 0x67, 0x1a, 0x83, 0x10, 0x63, 0x30, 0x55, 0x5f, 0x7f, 0xd8, 0x33, 0x57, 0xd7,
0xce, 0x0b, 0xa5, 0xe8, 0xee, 0x28, 0xfa, 0x4c, 0x2b, 0x4c, 0xd7, 0xe8, 0x33, 0x85, 0xaf, 0x85,
0xe6, 0x9b, 0xae, 0x71, 0x2d, 0xb2, 0x0d, 0xc5, 0x9d, 0x0d, 0xb7, 0xdb, 0xcc, 0x8d, 0x19, 0x9f,
0xfe, 0x3b, 0x73, 0x8a, 0xf1, 0x40, 0xe6, 0x30, 0x98, 0x37, 0x6c, 0x41, 0xd3, 0x77, 0x74, 0xed,
0x38, 0x07, 0x89, 0x52, 0x62, 0xab, 0x80, 0x4f, 0xa0, 0x94, 0x7c, 0x1f, 0xe4, 0x7c, 0x84, 0xf3,
0x64, 0x6f, 0x9f, 0x70, 0x3f, 0x0e, 0xc8, 0x8c, 0xe2, 0xe7, 0x79, 0x7c, 0x91, 0x8e, 0xef, 0xde,
0x05, 0x19, 0x73, 0x3f, 0xc3, 0xea, 0x88, 0xfe, 0x82, 0x78, 0xfa, 0x88, 0x53, 0x57, 0xd7, 0xce,
0x37, 0x04, 0xb5, 0x87, 0x75, 0x8a, 0xde, 0xa3, 0x91, 0xd4, 0xaf, 0x9c, 0xba, 0xba, 0xc6, 0x4f,
0xa1, 0x3a, 0xe0, 0x4c, 0x32, 0x22, 0x45, 0x34, 0xe0, 0x3e, 0xbd, 0x4d, 0x9d, 0xde, 0xeb, 0x2a,
0x9e, 0x4b, 0xe3, 0x50, 0x70, 0x9f, 0xa6, 0xbc, 0xc4, 0xcf, 0xbd, 0x2e, 0xae, 0x41, 0xb9, 0x27,
0xc4, 0x8c, 0xd1, 0xba, 0xa9, 0x9d, 0x49, 0x51, 0xe6, 0x57, 0x29, 0xf7, 0xeb, 0x55, 0xe7, 0xc7,
0xda, 0x46, 0x77, 0x6b, 0x1b, 0xfd, 0x59, 0xdb, 0xe8, 0xeb, 0xc6, 0x2e, 0xdc, 0x6d, 0xec, 0xc2,
0xef, 0x8d, 0x5d, 0xf8, 0xd0, 0x98, 0x32, 0x19, 0x2c, 0x27, 0x2d, 0x4f, 0x2c, 0x2e, 0xe3, 0x39,
0xf1, 0x66, 0xc1, 0xe7, 0xcb, 0xc4, 0x93, 0x49, 0x59, 0xff, 0x7d, 0x74, 0xfe, 0x06, 0x00, 0x00,
0xff, 0xff, 0x20, 0x00, 0x2b, 0x46, 0x4e, 0x04, 0x00, 0x00,
}
func (m *NebulaMeta) Marshal() (dAtA []byte, err error) {
@ -708,14 +719,17 @@ func (m *Ip6AndPort) MarshalToSizedBuffer(dAtA []byte) (int, error) {
if m.Port != 0 {
i = encodeVarintNebula(dAtA, i, uint64(m.Port))
i--
dAtA[i] = 0x18
}
if m.Lo != 0 {
i = encodeVarintNebula(dAtA, i, uint64(m.Lo))
i--
dAtA[i] = 0x10
}
if len(m.Ip) > 0 {
i -= len(m.Ip)
copy(dAtA[i:], m.Ip)
i = encodeVarintNebula(dAtA, i, uint64(len(m.Ip)))
if m.Hi != 0 {
i = encodeVarintNebula(dAtA, i, uint64(m.Hi))
i--
dAtA[i] = 0xa
dAtA[i] = 0x8
}
return len(dAtA) - i, nil
}
@ -920,9 +934,11 @@ func (m *Ip6AndPort) Size() (n int) {
}
var l int
_ = l
l = len(m.Ip)
if l > 0 {
n += 1 + l + sovNebula(uint64(l))
if m.Hi != 0 {
n += 1 + sovNebula(uint64(m.Hi))
}
if m.Lo != 0 {
n += 1 + sovNebula(uint64(m.Lo))
}
if m.Port != 0 {
n += 1 + sovNebula(uint64(m.Port))
@ -1372,10 +1388,10 @@ func (m *Ip6AndPort) Unmarshal(dAtA []byte) error {
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Ip", wireType)
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Hi", wireType)
}
var byteLen int
m.Hi = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowNebula
@ -1385,27 +1401,31 @@ func (m *Ip6AndPort) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
byteLen |= int(b&0x7F) << shift
m.Hi |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
if byteLen < 0 {
return ErrInvalidLengthNebula
}
postIndex := iNdEx + byteLen
if postIndex < 0 {
return ErrInvalidLengthNebula
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Ip = append(m.Ip[:0], dAtA[iNdEx:postIndex]...)
if m.Ip == nil {
m.Ip = []byte{}
}
iNdEx = postIndex
case 2:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Lo", wireType)
}
m.Lo = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowNebula
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.Lo |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
case 3:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Port", wireType)
}

View File

@ -1,6 +1,8 @@
syntax = "proto3";
package nebula;
option go_package = "github.com/slackhq/nebula";
message NebulaMeta {
enum MessageType {
None = 0;
@ -33,8 +35,9 @@ message Ip4AndPort {
}
message Ip6AndPort {
bytes Ip = 1;
uint32 Port = 2;
uint64 Hi = 1;
uint64 Lo = 2;
uint32 Port = 3;
}
message NebulaPing {