Move a few things around

This commit is contained in:
Asim Aslam 2019-12-09 22:56:26 +00:00
parent 55ab44c8be
commit b29da80539
2 changed files with 425 additions and 406 deletions

View File

@ -74,7 +74,7 @@ type network struct {
// whether we've discovered by the network
discovered chan bool
// solicted checks whether routes were solicited by one node
solicited chan string
solicited chan *node
}
// message is network message
@ -167,7 +167,7 @@ func newNetwork(opts ...Option) Network {
tunClient: make(map[string]transport.Client),
peerLinks: make(map[string]tunnel.Link),
discovered: make(chan bool, 1),
solicited: make(chan string, 1),
solicited: make(chan *node, 1),
}
network.node.network = network
@ -202,6 +202,144 @@ func (n *network) Name() string {
return n.options.Name
}
// acceptNetConn accepts connections from NetworkChannel
func (n *network) acceptNetConn(l tunnel.Listener, recv chan *message) {
var i int
for {
// accept a connection
conn, err := l.Accept()
if err != nil {
sleep := backoff.Do(i)
log.Debugf("Network tunnel [%s] accept error: %v, backing off for %v", ControlChannel, err, sleep)
time.Sleep(sleep)
if i > 5 {
i = 0
}
i++
continue
}
select {
case <-n.closed:
if err := conn.Close(); err != nil {
log.Debugf("Network tunnel [%s] failed to close connection: %v", NetworkChannel, err)
}
return
default:
// go handle NetworkChannel connection
go n.handleNetConn(conn, recv)
}
}
}
// acceptCtrlConn accepts connections from ControlChannel
func (n *network) acceptCtrlConn(l tunnel.Listener, recv chan *message) {
var i int
for {
// accept a connection
conn, err := l.Accept()
if err != nil {
sleep := backoff.Do(i)
log.Debugf("Network tunnel [%s] accept error: %v, backing off for %v", ControlChannel, err, sleep)
time.Sleep(sleep)
if i > 5 {
// reset the counter
i = 0
}
i++
continue
}
select {
case <-n.closed:
if err := conn.Close(); err != nil {
log.Debugf("Network tunnel [%s] failed to close connection: %v", ControlChannel, err)
}
return
default:
// go handle ControlChannel connection
go n.handleCtrlConn(conn, recv)
}
}
}
// handleCtrlConn handles ControlChannel connections
// advertise advertises routes to the network
func (n *network) advertise(advertChan <-chan *router.Advert) {
hasher := fnv.New64()
for {
select {
// process local adverts and randomly fire them at other nodes
case advert := <-advertChan:
// create a proto advert
var events []*pbRtr.Event
for _, event := range advert.Events {
// the routes service address
address := event.Route.Address
// only hash the address if we're advertising our own local routes
if event.Route.Router == advert.Id {
// hash the service before advertising it
hasher.Reset()
// routes for multiple instances of a service will be collapsed here.
// TODO: once we store labels in the table this may need to change
// to include the labels in case they differ but highly unlikely
hasher.Write([]byte(event.Route.Service + n.node.Address()))
address = fmt.Sprintf("%d", hasher.Sum64())
}
// calculate route metric to advertise
metric := n.getRouteMetric(event.Route.Router, event.Route.Gateway, event.Route.Link)
// NOTE: we override Gateway, Link and Address here
route := &pbRtr.Route{
Service: event.Route.Service,
Address: address,
Gateway: n.node.Address(),
Network: event.Route.Network,
Router: event.Route.Router,
Link: DefaultLink,
Metric: metric,
}
e := &pbRtr.Event{
Type: pbRtr.EventType(event.Type),
Timestamp: event.Timestamp.UnixNano(),
Route: route,
}
events = append(events, e)
}
msg := &pbRtr.Advert{
Id: advert.Id,
Type: pbRtr.AdvertType(advert.Type),
Timestamp: advert.Timestamp.UnixNano(),
Events: events,
}
// send the advert to all on the control channel
// since its not a solicitation
if advert.Type != router.Solicitation {
if err := n.sendMsg("advert", ControlChannel, msg); err != nil {
log.Debugf("Network failed to advertise routes: %v", err)
}
continue
}
// it's a solication, someone asked for it
// so we're going to pick off the node and send it
select {
case peer := <-n.solicited:
// someone requested the route
n.sendTo("advert", ControlChannel, peer, msg)
default:
// send to all since we can't get anything
n.sendMsg("advert", ControlChannel, msg)
}
case <-n.closed:
return
}
}
}
func (n *network) initNodes(startup bool) {
nodes, err := n.resolveNodes()
if err != nil && !startup {
@ -306,32 +444,257 @@ func (n *network) handleNetConn(s tunnel.Session, msg chan *message) {
}
}
// acceptNetConn accepts connections from NetworkChannel
func (n *network) acceptNetConn(l tunnel.Listener, recv chan *message) {
var i int
func (n *network) handleCtrlConn(s tunnel.Session, msg chan *message) {
for {
// accept a connection
conn, err := l.Accept()
if err != nil {
sleep := backoff.Do(i)
log.Debugf("Network tunnel [%s] accept error: %v, backing off for %v", ControlChannel, err, sleep)
time.Sleep(sleep)
if i > 5 {
i = 0
m := new(transport.Message)
if err := s.Recv(m); err != nil {
log.Debugf("Network tunnel [%s] receive error: %v", ControlChannel, err)
switch err {
case io.EOF, tunnel.ErrReadTimeout:
s.Close()
return
}
i++
continue
}
// check if peer is set
peer := m.Header["Micro-Peer"]
// check who the message is intended for
if len(peer) > 0 && peer != n.options.Id {
continue
}
select {
case msg <- &message{
msg: m,
session: s,
}:
case <-n.closed:
return
}
}
}
// getHopCount queries network graph and returns hop count for given router
// - Routes for local services have hop count 1
// - Routes with ID of adjacent nodes have hop count 2
// - Routes by peers of the advertiser have hop count 3
// - Routes beyond node neighbourhood have hop count 4
func (n *network) getHopCount(rtr string) int {
// make sure node.peers are not modified
n.node.RLock()
defer n.node.RUnlock()
// we are the origin of the route
if rtr == n.options.Id {
return 1
}
// the route origin is our peer
if _, ok := n.peers[rtr]; ok {
return 10
}
// the route origin is the peer of our peer
for _, peer := range n.peers {
for id := range peer.peers {
if rtr == id {
return 100
}
}
}
// otherwise we are three hops away
return 1000
}
// getRouteMetric calculates router metric and returns it
// Route metric is calculated based on link status and route hopd count
func (n *network) getRouteMetric(router string, gateway string, link string) int64 {
// set the route metric
n.RLock()
defer n.RUnlock()
// local links are marked as 1
if link == "local" && gateway == "" {
return 1
}
// local links from other gateways as 2
if link == "local" && gateway != "" {
return 2
}
log.Tracef("Network looking up %s link to gateway: %s", link, gateway)
// attempt to find link based on gateway address
lnk, ok := n.peerLinks[gateway]
if !ok {
log.Debugf("Network failed to find a link to gateway: %s", gateway)
// no link found so infinite metric returned
return math.MaxInt64
}
// calculating metric
delay := lnk.Delay()
hops := n.getHopCount(router)
length := lnk.Length()
// make sure delay is non-zero
if delay == 0 {
delay = 1
}
// make sure length is non-zero
if length == 0 {
log.Debugf("Link length is 0 %v %v", link, lnk.Length())
length = 10e9
}
log.Tracef("Network calculated metric %v delay %v length %v distance %v", (delay*length*int64(hops))/10e6, delay, length, hops)
return (delay * length * int64(hops)) / 10e6
}
// processCtrlChan processes messages received on ControlChannel
func (n *network) processCtrlChan(listener tunnel.Listener) {
defer listener.Close()
// receive control message queue
recv := make(chan *message, 128)
// accept ControlChannel cconnections
go n.acceptCtrlConn(listener, recv)
for {
select {
case m := <-recv:
// switch on type of message and take action
switch m.msg.Header["Micro-Method"] {
case "advert":
pbRtrAdvert := &pbRtr.Advert{}
if err := proto.Unmarshal(m.msg.Body, pbRtrAdvert); err != nil {
log.Debugf("Network fail to unmarshal advert message: %v", err)
continue
}
// don't process your own messages
if pbRtrAdvert.Id == n.options.Id {
continue
}
log.Debugf("Network received advert message from: %s", pbRtrAdvert.Id)
// loookup advertising node in our peer topology
advertNode := n.node.GetPeerNode(pbRtrAdvert.Id)
if advertNode == nil {
// if we can't find the node in our topology (MaxDepth) we skipp prcessing adverts
log.Debugf("Network skipping advert message from unknown peer: %s", pbRtrAdvert.Id)
continue
}
var events []*router.Event
for _, event := range pbRtrAdvert.Events {
// we know the advertising node is not the origin of the route
if pbRtrAdvert.Id != event.Route.Router {
// if the origin router is not the advertising node peer
// we can't rule out potential routing loops so we bail here
if peer := advertNode.GetPeerNode(event.Route.Router); peer == nil {
log.Debugf("Network skipping advert message from peer: %s", pbRtrAdvert.Id)
continue
}
}
route := router.Route{
Service: event.Route.Service,
Address: event.Route.Address,
Gateway: event.Route.Gateway,
Network: event.Route.Network,
Router: event.Route.Router,
Link: event.Route.Link,
Metric: event.Route.Metric,
}
// calculate route metric and add to the advertised metric
// we need to make sure we do not overflow math.MaxInt64
metric := n.getRouteMetric(event.Route.Router, event.Route.Gateway, event.Route.Link)
log.Tracef("Network metric for router %s and gateway %s: %v", event.Route.Router, event.Route.Gateway, metric)
// check we don't overflow max int 64
if d := route.Metric + metric; d <= 0 {
// set to max int64 if we overflow
route.Metric = math.MaxInt64
} else {
// set the combined value of metrics otherwise
route.Metric = d
}
// create router event
e := &router.Event{
Type: router.EventType(event.Type),
Timestamp: time.Unix(0, pbRtrAdvert.Timestamp),
Route: route,
}
events = append(events, e)
}
// if no events are eligible for processing continue
if len(events) == 0 {
log.Tracef("Network no events to be processed by router: %s", n.options.Id)
continue
}
// create an advert and process it
advert := &router.Advert{
Id: pbRtrAdvert.Id,
Type: router.AdvertType(pbRtrAdvert.Type),
Timestamp: time.Unix(0, pbRtrAdvert.Timestamp),
TTL: time.Duration(pbRtrAdvert.Ttl),
Events: events,
}
log.Debugf("Network router %s processing advert: %s", n.Id(), advert.Id)
if err := n.router.Process(advert); err != nil {
log.Debugf("Network failed to process advert %s: %v", advert.Id, err)
}
case "solicit":
pbRtrSolicit := &pbRtr.Solicit{}
if err := proto.Unmarshal(m.msg.Body, pbRtrSolicit); err != nil {
log.Debugf("Network fail to unmarshal solicit message: %v", err)
continue
}
log.Debugf("Network received solicit message from: %s", pbRtrSolicit.Id)
// ignore solicitation when requested by you
if pbRtrSolicit.Id == n.options.Id {
continue
}
log.Debugf("Network router flushing routes for: %s", pbRtrSolicit.Id)
// advertise all the routes when a new node has connected
if err := n.router.Solicit(); err != nil {
log.Debugf("Network failed to solicit routes: %s", err)
}
peer := &node{
id: pbRtrSolicit.Id,
link: m.msg.Header["Micro-Link"],
}
// specify that someone solicited the route
select {
case n.solicited <- peer:
default:
// don't block
}
}
case <-n.closed:
if err := conn.Close(); err != nil {
log.Debugf("Network tunnel [%s] failed to close connection: %v", NetworkChannel, err)
}
return
default:
// go handle NetworkChannel connection
go n.handleNetConn(conn, recv)
}
}
}
@ -371,13 +734,14 @@ func (n *network) processNetChan(listener tunnel.Listener) {
peer := &node{
id: pbNetConnect.Node.Id,
address: pbNetConnect.Node.Address,
link: m.msg.Header["Micro-Link"],
peers: make(map[string]*node),
lastSeen: now,
}
// update peer links
if err := n.updatePeerLinks(pbNetConnect.Node.Address, m); err != nil {
if err := n.updatePeerLinks(peer); err != nil {
log.Debugf("Network failed updating peer links: %s", err)
}
@ -385,7 +749,7 @@ func (n *network) processNetChan(listener tunnel.Listener) {
if err := n.node.AddPeer(peer); err == ErrPeerExists {
log.Debugf("Network peer exists, refreshing: %s", peer.id)
// update lastSeen time for the existing node
if err := n.RefreshPeer(peer.id, now); err != nil {
if err := n.RefreshPeer(peer.id, peer.link, now); err != nil {
log.Debugf("Network failed refreshing peer %s: %v", peer.id, err)
}
}
@ -396,10 +760,9 @@ func (n *network) processNetChan(listener tunnel.Listener) {
// get node peers down to MaxDepth encoded in protobuf
msg := PeersToProto(n.node, MaxDepth)
node := pbNetConnect.Node.Id
// advertise yourself to the network
if err := n.sendTo("peer", NetworkChannel, node, msg); err != nil {
if err := n.sendTo("peer", NetworkChannel, peer, msg); err != nil {
log.Debugf("Network failed to advertise peers: %v", err)
}
@ -410,7 +773,7 @@ func (n *network) processNetChan(listener tunnel.Listener) {
// specify that we're soliciting
select {
case n.solicited <- node:
case n.solicited <- peer:
default:
// don't block
}
@ -434,13 +797,14 @@ func (n *network) processNetChan(listener tunnel.Listener) {
peer := &node{
id: pbNetPeer.Node.Id,
address: pbNetPeer.Node.Address,
link: m.msg.Header["Micro-Link"],
peers: make(map[string]*node),
lastSeen: now,
}
// update peer links
if err := n.updatePeerLinks(pbNetPeer.Node.Address, m); err != nil {
if err := n.updatePeerLinks(peer); err != nil {
log.Debugf("Network failed updating peer links: %s", err)
}
@ -450,10 +814,8 @@ func (n *network) processNetChan(listener tunnel.Listener) {
Id: n.options.Id,
}
node := pbNetPeer.Node.Id
// only solicit this peer
if err := n.sendTo("solicit", ControlChannel, node, msg); err != nil {
if err := n.sendTo("solicit", ControlChannel, peer, msg); err != nil {
log.Debugf("Network failed to send solicit message: %s", err)
}
@ -467,7 +829,7 @@ func (n *network) processNetChan(listener tunnel.Listener) {
log.Debugf("Network peer exists, refreshing: %s", pbNetPeer.Node.Id)
// update lastSeen time for the peer
if err := n.RefreshPeer(pbNetPeer.Node.Id, now); err != nil {
if err := n.RefreshPeer(pbNetPeer.Node.Id, peer.link, now); err != nil {
log.Debugf("Network failed refreshing peer %s: %v", pbNetPeer.Node.Id, err)
}
@ -630,26 +992,43 @@ func (n *network) manage() {
}
}
func (n *network) sendConnect() {
// send connect message to NetworkChannel
// NOTE: in theory we could do this as soon as
// Dial to NetworkChannel succeeds, but instead
// we initialize all other node resources first
msg := &pbNet.Connect{
Node: &pbNet.Node{
Id: n.node.id,
Address: n.node.address,
},
}
if err := n.sendMsg("connect", NetworkChannel, msg); err != nil {
log.Debugf("Network failed to send connect message: %s", err)
}
}
// sendTo sends a message to a specific node as a one off.
// we need this because when links die, we have no discovery info,
// and sending to an existing multicast link doesn't immediately work
func (n *network) sendTo(method, channel, peer string, msg proto.Message) error {
func (n *network) sendTo(method, channel string, peer *node, msg proto.Message) error {
body, err := proto.Marshal(msg)
if err != nil {
return err
}
c, err := n.tunnel.Dial(channel, tunnel.DialMode(tunnel.Multicast))
c, err := n.tunnel.Dial(channel, tunnel.DialMode(tunnel.Multicast), tunnel.DialLink(peer.link))
if err != nil {
return err
}
defer c.Close()
log.Debugf("Network sending %s message from: %s to %s", method, n.options.Id, peer)
log.Debugf("Network sending %s message from: %s to %s", method, n.options.Id, peer.id)
return c.Send(&transport.Message{
Header: map[string]string{
"Micro-Method": method,
"Micro-Peer": peer,
"Micro-Peer": peer.id,
},
Body: body,
})
@ -682,11 +1061,11 @@ func (n *network) sendMsg(method, channel string, msg proto.Message) error {
}
// updatePeerLinks updates link for a given peer
func (n *network) updatePeerLinks(peerAddr string, m *message) error {
func (n *network) updatePeerLinks(peer *node) error {
n.Lock()
defer n.Unlock()
linkId := m.msg.Header["Micro-Link"]
linkId := peer.link
log.Tracef("Network looking up link %s in the peer links", linkId)
@ -705,386 +1084,21 @@ func (n *network) updatePeerLinks(peerAddr string, m *message) error {
}
// if the peerLink is found in the returned links update peerLinks
log.Tracef("Network updating peer links for peer %s", peerAddr)
log.Tracef("Network updating peer links for peer %s", peer.address)
// add peerLink to the peerLinks map
if link, ok := n.peerLinks[peerAddr]; ok {
if link, ok := n.peerLinks[peer.address]; ok {
// if the existing has better Length then the new, replace it
if link.Length() < peerLink.Length() {
n.peerLinks[peerAddr] = peerLink
n.peerLinks[peer.address] = peerLink
}
} else {
n.peerLinks[peerAddr] = peerLink
n.peerLinks[peer.address] = peerLink
}
return nil
}
// handleCtrlConn handles ControlChannel connections
func (n *network) handleCtrlConn(s tunnel.Session, msg chan *message) {
for {
m := new(transport.Message)
if err := s.Recv(m); err != nil {
log.Debugf("Network tunnel [%s] receive error: %v", ControlChannel, err)
switch err {
case io.EOF, tunnel.ErrReadTimeout:
s.Close()
return
}
continue
}
// check if peer is set
peer := m.Header["Micro-Peer"]
// check who the message is intended for
if len(peer) > 0 && peer != n.options.Id {
continue
}
select {
case msg <- &message{
msg: m,
session: s,
}:
case <-n.closed:
return
}
}
}
// acceptCtrlConn accepts connections from ControlChannel
func (n *network) acceptCtrlConn(l tunnel.Listener, recv chan *message) {
var i int
for {
// accept a connection
conn, err := l.Accept()
if err != nil {
sleep := backoff.Do(i)
log.Debugf("Network tunnel [%s] accept error: %v, backing off for %v", ControlChannel, err, sleep)
time.Sleep(sleep)
if i > 5 {
// reset the counter
i = 0
}
i++
continue
}
select {
case <-n.closed:
if err := conn.Close(); err != nil {
log.Debugf("Network tunnel [%s] failed to close connection: %v", ControlChannel, err)
}
return
default:
// go handle ControlChannel connection
go n.handleCtrlConn(conn, recv)
}
}
}
// getHopCount queries network graph and returns hop count for given router
// - Routes for local services have hop count 1
// - Routes with ID of adjacent nodes have hop count 2
// - Routes by peers of the advertiser have hop count 3
// - Routes beyond node neighbourhood have hop count 4
func (n *network) getHopCount(rtr string) int {
// make sure node.peers are not modified
n.node.RLock()
defer n.node.RUnlock()
// we are the origin of the route
if rtr == n.options.Id {
return 1
}
// the route origin is our peer
if _, ok := n.peers[rtr]; ok {
return 10
}
// the route origin is the peer of our peer
for _, peer := range n.peers {
for id := range peer.peers {
if rtr == id {
return 100
}
}
}
// otherwise we are three hops away
return 1000
}
// getRouteMetric calculates router metric and returns it
// Route metric is calculated based on link status and route hopd count
func (n *network) getRouteMetric(router string, gateway string, link string) int64 {
// set the route metric
n.RLock()
defer n.RUnlock()
if link == "local" && gateway == "" {
return 1
}
if link == "local" && gateway != "" {
return 2
}
log.Tracef("Network looking up %s link to gateway: %s", link, gateway)
if link, ok := n.peerLinks[gateway]; ok {
// maka sure delay is non-zero
delay := link.Delay()
if delay == 0 {
delay = 1
}
// get the route hop count
hops := n.getHopCount(router)
// make sure length is non-zero
length := link.Length()
if length == 0 {
log.Debugf("Link length is 0 %v %v", link, link.Length())
length = 10e9
}
log.Tracef("Network calculated metric %v delay %v length %v distance %v", (delay*length*int64(hops))/10e6, delay, length, hops)
return (delay * length * int64(hops)) / 10e6
}
log.Debugf("Network failed to find a link to gateway: %s", gateway)
return math.MaxInt64
}
// processCtrlChan processes messages received on ControlChannel
func (n *network) processCtrlChan(listener tunnel.Listener) {
defer listener.Close()
// receive control message queue
recv := make(chan *message, 128)
// accept ControlChannel cconnections
go n.acceptCtrlConn(listener, recv)
for {
select {
case m := <-recv:
// switch on type of message and take action
switch m.msg.Header["Micro-Method"] {
case "advert":
pbRtrAdvert := &pbRtr.Advert{}
if err := proto.Unmarshal(m.msg.Body, pbRtrAdvert); err != nil {
log.Debugf("Network fail to unmarshal advert message: %v", err)
continue
}
// don't process your own messages
if pbRtrAdvert.Id == n.options.Id {
continue
}
log.Debugf("Network received advert message from: %s", pbRtrAdvert.Id)
// loookup advertising node in our peer topology
advertNode := n.node.GetPeerNode(pbRtrAdvert.Id)
if advertNode == nil {
// if we can't find the node in our topology (MaxDepth) we skipp prcessing adverts
log.Debugf("Network skipping advert message from unknown peer: %s", pbRtrAdvert.Id)
continue
}
var events []*router.Event
for _, event := range pbRtrAdvert.Events {
// we know the advertising node is not the origin of the route
if pbRtrAdvert.Id != event.Route.Router {
// if the origin router is not the advertising node peer
// we can't rule out potential routing loops so we bail here
if peer := advertNode.GetPeerNode(event.Route.Router); peer == nil {
log.Debugf("Network skipping advert message from peer: %s", pbRtrAdvert.Id)
continue
}
}
route := router.Route{
Service: event.Route.Service,
Address: event.Route.Address,
Gateway: event.Route.Gateway,
Network: event.Route.Network,
Router: event.Route.Router,
Link: event.Route.Link,
Metric: event.Route.Metric,
}
// calculate route metric and add to the advertised metric
// we need to make sure we do not overflow math.MaxInt64
metric := n.getRouteMetric(event.Route.Router, event.Route.Gateway, event.Route.Link)
log.Tracef("Network metric for router %s and gateway %s: %v", event.Route.Router, event.Route.Gateway, metric)
// check we don't overflow max int 64
if d := route.Metric + metric; d <= 0 {
// set to max int64 if we overflow
route.Metric = math.MaxInt64
} else {
// set the combined value of metrics otherwise
route.Metric = d
}
// create router event
e := &router.Event{
Type: router.EventType(event.Type),
Timestamp: time.Unix(0, pbRtrAdvert.Timestamp),
Route: route,
}
events = append(events, e)
}
// if no events are eligible for processing continue
if len(events) == 0 {
log.Tracef("Network no events to be processed by router: %s", n.options.Id)
continue
}
// create an advert and process it
advert := &router.Advert{
Id: pbRtrAdvert.Id,
Type: router.AdvertType(pbRtrAdvert.Type),
Timestamp: time.Unix(0, pbRtrAdvert.Timestamp),
TTL: time.Duration(pbRtrAdvert.Ttl),
Events: events,
}
log.Debugf("Network router %s processing advert: %s", n.Id(), advert.Id)
if err := n.router.Process(advert); err != nil {
log.Debugf("Network failed to process advert %s: %v", advert.Id, err)
}
case "solicit":
pbRtrSolicit := &pbRtr.Solicit{}
if err := proto.Unmarshal(m.msg.Body, pbRtrSolicit); err != nil {
log.Debugf("Network fail to unmarshal solicit message: %v", err)
continue
}
log.Debugf("Network received solicit message from: %s", pbRtrSolicit.Id)
// ignore solicitation when requested by you
if pbRtrSolicit.Id == n.options.Id {
continue
}
log.Debugf("Network router flushing routes for: %s", pbRtrSolicit.Id)
// advertise all the routes when a new node has connected
if err := n.router.Solicit(); err != nil {
log.Debugf("Network failed to solicit routes: %s", err)
}
// specify that someone solicited the route
select {
case n.solicited <- pbRtrSolicit.Id:
default:
// don't block
}
}
case <-n.closed:
return
}
}
}
// advertise advertises routes to the network
func (n *network) advertise(advertChan <-chan *router.Advert) {
hasher := fnv.New64()
for {
select {
// process local adverts and randomly fire them at other nodes
case advert := <-advertChan:
// create a proto advert
var events []*pbRtr.Event
for _, event := range advert.Events {
// the routes service address
address := event.Route.Address
// only hash the address if we're advertising our own local routes
if event.Route.Router == advert.Id {
// hash the service before advertising it
hasher.Reset()
// routes for multiple instances of a service will be collapsed here.
// TODO: once we store labels in the table this may need to change
// to include the labels in case they differ but highly unlikely
hasher.Write([]byte(event.Route.Service + n.node.Address()))
address = fmt.Sprintf("%d", hasher.Sum64())
}
// calculate route metric to advertise
metric := n.getRouteMetric(event.Route.Router, event.Route.Gateway, event.Route.Link)
// NOTE: we override Gateway, Link and Address here
route := &pbRtr.Route{
Service: event.Route.Service,
Address: address,
Gateway: n.node.Address(),
Network: event.Route.Network,
Router: event.Route.Router,
Link: DefaultLink,
Metric: metric,
}
e := &pbRtr.Event{
Type: pbRtr.EventType(event.Type),
Timestamp: event.Timestamp.UnixNano(),
Route: route,
}
events = append(events, e)
}
msg := &pbRtr.Advert{
Id: advert.Id,
Type: pbRtr.AdvertType(advert.Type),
Timestamp: advert.Timestamp.UnixNano(),
Events: events,
}
// send the advert to all on the control channel
// since its not a solicitation
if advert.Type != router.Solicitation {
if err := n.sendMsg("advert", ControlChannel, msg); err != nil {
log.Debugf("Network failed to advertise routes: %v", err)
}
continue
}
// it's a solication, someone asked for it
// so we're going to pick off the node and send it
select {
case node := <-n.solicited:
// someone requested the route
n.sendTo("advert", ControlChannel, node, msg)
default:
// send to all since we can't get anything
n.sendMsg("advert", ControlChannel, msg)
}
case <-n.closed:
return
}
}
}
func (n *network) sendConnect() {
// send connect message to NetworkChannel
// NOTE: in theory we could do this as soon as
// Dial to NetworkChannel succeeds, but instead
// we initialize all other node resources first
msg := &pbNet.Connect{
Node: &pbNet.Node{
Id: n.node.id,
Address: n.node.address,
},
}
if err := n.sendMsg("connect", NetworkChannel, msg); err != nil {
log.Debugf("Network failed to send connect message: %s", err)
}
}
// connect will wait for a link to be established and send the connect
// message. We're trying to ensure convergence pretty quickly. So we want
// to hear back. In the case we become completely disconnected we'll

View File

@ -28,6 +28,8 @@ type node struct {
id string
// address is node address
address string
// link on which we communicate with the peer
link string
// peers are nodes with direct link to this node
peers map[string]*node
// network returns the node network
@ -127,7 +129,7 @@ func (n *node) UpdatePeer(peer *node) error {
// RefreshPeer updates node timestamp
// It returns false if the peer has not been found.
func (n *node) RefreshPeer(id string, now time.Time) error {
func (n *node) RefreshPeer(id, link string, now time.Time) error {
n.Lock()
defer n.Unlock()
@ -136,6 +138,9 @@ func (n *node) RefreshPeer(id string, now time.Time) error {
return ErrPeerNotFound
}
// set peer link
peer.link = link
if peer.lastSeen.Before(now) {
peer.lastSeen = now
}