Functional loopback code

This commit is contained in:
Asim Aslam 2019-07-03 19:26:24 +01:00
parent 7008809eff
commit e54de56376
5 changed files with 313 additions and 137 deletions

View File

@ -45,7 +45,7 @@ type network struct {
// lease generates a new lease with a node id/address // lease generates a new lease with a node id/address
// TODO: use a consensus mechanism, pool or some deterministic // TODO: use a consensus mechanism, pool or some deterministic
// unique addressing method. // unique addressing method.
func (n *network) lease() *pb.Lease { func (n *network) lease(muid string) *pb.Lease {
// create the id // create the id
id := uuid.New().String() id := uuid.New().String()
// create a timestamp // create a timestamp
@ -62,6 +62,7 @@ func (n *network) lease() *pb.Lease {
Id: id, Id: id,
Timestamp: now, Timestamp: now,
Node: &pb.Node{ Node: &pb.Node{
Muid: muid,
Id: id, Id: id,
Address: address, Address: address,
Network: n.id, Network: n.id,

View File

@ -5,24 +5,30 @@ import (
"io" "io"
"sync" "sync"
"github.com/micro/go-micro/util/log"
gproto "github.com/golang/protobuf/proto" gproto "github.com/golang/protobuf/proto"
"github.com/google/uuid"
"github.com/micro/go-micro/codec" "github.com/micro/go-micro/codec"
"github.com/micro/go-micro/codec/proto"
pb "github.com/micro/go-micro/network/proto" pb "github.com/micro/go-micro/network/proto"
"github.com/micro/go-micro/transport" "github.com/micro/go-micro/transport"
"github.com/micro/go-micro/util/log"
) )
type link struct { type link struct {
// the embedded node // the embedded node
*node *node
closed chan bool
sync.RWMutex sync.RWMutex
// the link id // the link id
id string id string
// the send queue to the socket // the send queue to the socket
queue chan *Message sendQueue chan *Message
// the recv queue to the socket
recvQueue chan *Message
// codec we use to marshal things // codec we use to marshal things
codec codec.Marshaler codec codec.Marshaler
@ -39,16 +45,53 @@ type link struct {
weight int weight int
} }
var (
ErrLinkClosed = errors.New("link closed")
)
func newLink(n *node, sock transport.Socket, lease *pb.Lease) *link {
return &link{
id: uuid.New().String(),
closed: make(chan bool),
codec: &proto.Marshaler{},
node: n,
lease: lease,
socket: sock,
sendQueue: make(chan *Message, 128),
recvQueue: make(chan *Message, 128),
}
}
// link methods // link methods
// process processe messages on the send queue // process processes messages on the send queue.
// these are messages to be sent to the remote side.
func (l *link) process() { func (l *link) process() {
for { go func() {
select { for {
case m := <-l.queue: m := new(Message)
if err := l.send(m, nil); err != nil { if err := l.recv(m, nil); err != nil {
l.Close()
return return
} }
select {
case l.recvQueue <- m:
log.Debugf("%s processing recv", l.id)
case <-l.closed:
return
}
}
}()
for {
select {
case m := <-l.sendQueue:
if err := l.send(m, nil); err != nil {
l.Close()
return
}
case <-l.closed:
return
} }
} }
} }
@ -71,29 +114,49 @@ func (l *link) accept() error {
switch event { switch event {
// connect event // connect event
case "Connect": case "connect":
// process connect events from network.Connect() // process connect events from network.Connect()
// these are new connections to join the network // these are new connections to join the network
// decode the connection event // decode the connection event
conn := new(pb.Connect) conn := new(pb.Connect)
// expecting a connect message
if err := l.codec.Unmarshal(m.Body, conn); err != nil { if err := l.codec.Unmarshal(m.Body, conn); err != nil {
// skip error // skip error
continue continue
} }
// no micro id close the link
if len(conn.Muid) == 0 {
l.Close()
return errors.New("invalid muid " + conn.Muid)
}
// get the existing lease if it exists // get the existing lease if it exists
lease := conn.Lease lease := conn.Lease
// if there's no lease create a new one // if there's no lease create a new one
if lease == nil { if lease == nil {
// create a new lease/node // create a new lease/node
lease = l.node.network.lease() lease = l.node.network.lease(conn.Muid)
} }
// check if we connected to ourself
if conn.Muid == l.node.muid {
// check our own leasae
l.node.Lock()
if l.node.lease == nil {
l.node.lease = lease
}
l.node.Unlock()
}
// set the author to our own muid
lease.Author = l.node.muid
// send back a lease offer for the node // send back a lease offer for the node
if err := l.send(&Message{ if err := l.send(&Message{
Header: map[string]string{ Header: map[string]string{
"Micro-Method": "Lease", "Micro-Method": "lease",
}, },
}, lease); err != nil { }, lease); err != nil {
return err return err
@ -107,11 +170,10 @@ func (l *link) accept() error {
// we've connected // we've connected
// start processing the messages // start processing the messages
go l.process() go l.process()
return nil return nil
case "Close": case "close":
l.Close() l.Close()
return errors.New("connection closed") return io.EOF
default: default:
return errors.New("unknown method: " + event) return errors.New("unknown method: " + event)
} }
@ -132,9 +194,9 @@ func (l *link) connect() error {
// send a lease request // send a lease request
if err := l.send(&Message{ if err := l.send(&Message{
Header: map[string]string{ Header: map[string]string{
"Micro-Method": "Connect", "Micro-Method": "connect",
}, },
}, &pb.Connect{Lease: lease}); err != nil { }, &pb.Connect{Muid: l.node.muid, Lease: lease}); err != nil {
return err return err
} }
@ -151,7 +213,7 @@ func (l *link) connect() error {
// check the method // check the method
switch event { switch event {
case "Lease": case "lease":
// save the lease // save the lease
l.Lock() l.Lock()
l.lease = newLease l.lease = newLease
@ -159,10 +221,11 @@ func (l *link) connect() error {
// start processing the messages // start processing the messages
go l.process() go l.process()
case "Close": case "close":
l.socket.Close() l.Close()
return errors.New("connection closed") return io.EOF
default: default:
l.Close()
return errors.New("unable to attain lease") return errors.New("unable to attain lease")
} }
@ -190,13 +253,8 @@ func (l *link) send(m *Message, v interface{}) error {
tm.Body = b tm.Body = b
} }
log.Debugf("link %s sending %+v %+v\n", l.id, m, v)
// send via the transport socket // send via the transport socket
return l.socket.Send(&transport.Message{ return l.socket.Send(tm)
Header: m.Header,
Body: m.Body,
})
} }
// recv a message on the link // recv a message on the link
@ -207,12 +265,14 @@ func (l *link) recv(m *Message, v interface{}) error {
tm := new(transport.Message) tm := new(transport.Message)
log.Debugf("link %s attempting receiving", l.id)
// receive the transport message // receive the transport message
if err := l.socket.Recv(tm); err != nil { if err := l.socket.Recv(tm); err != nil {
return err return err
} }
log.Debugf("link %s receiving %+v %+v\n", l.id, tm, v) log.Debugf("link %s received %+v %+v\n", l.id, tm, v)
// set the message // set the message
m.Header = tm.Header m.Header = tm.Header
@ -235,12 +295,20 @@ func (l *link) recv(m *Message, v interface{}) error {
// Close the link // Close the link
func (l *link) Close() error { func (l *link) Close() error {
select {
case <-l.closed:
return nil
default:
close(l.closed)
}
// send a final close message // send a final close message
l.socket.Send(&transport.Message{ l.socket.Send(&transport.Message{
Header: map[string]string{ Header: map[string]string{
"Micro-Method": "Close", "Micro-Method": "close",
}, },
}) })
// close the socket // close the socket
return l.socket.Close() return l.socket.Close()
} }
@ -273,20 +341,27 @@ func (l *link) Length() int {
} }
func (l *link) Weight() int { func (l *link) Weight() int {
l.RLock() return len(l.sendQueue) + len(l.recvQueue)
defer l.RUnlock()
return l.weight
} }
// Accept accepts a message on the socket
func (l *link) Accept() (*Message, error) { func (l *link) Accept() (*Message, error) {
m := new(Message) select {
err := l.recv(m, nil) case <-l.closed:
if err != nil { return nil, ErrLinkClosed
return nil, err case m := <-l.recvQueue:
return m, nil
} }
return m, nil // never reach
return nil, nil
} }
// Send sends a message on the socket immediately
func (l *link) Send(m *Message) error { func (l *link) Send(m *Message) error {
return l.send(m, nil) select {
case <-l.closed:
return ErrLinkClosed
case l.sendQueue <- m:
}
return nil
} }

View File

@ -5,12 +5,12 @@ import (
"fmt" "fmt"
"net" "net"
"runtime/debug" "runtime/debug"
"sort"
"strconv" "strconv"
"sync" "sync"
"time" "time"
"github.com/google/uuid" "github.com/google/uuid"
"github.com/micro/go-micro/codec/proto"
"github.com/micro/go-micro/registry" "github.com/micro/go-micro/registry"
"github.com/micro/go-micro/transport" "github.com/micro/go-micro/transport"
"github.com/micro/go-micro/util/addr" "github.com/micro/go-micro/util/addr"
@ -22,17 +22,23 @@ import (
type node struct { type node struct {
*network *network
// closed channel // closed channel to close our connection to the network
closed chan bool closed chan bool
sync.RWMutex sync.RWMutex
// the node id // the nodes unique micro assigned mac address
muid string
// the node id registered in registry
id string id string
// address of this node // address of this node registered in registry
address string address string
// our network lease with our network id/address
lease *pb.Lease
// the node registry // the node registry
registry registry.Registry registry registry.Registry
@ -42,9 +48,13 @@ type node struct {
// the listener // the listener
listener transport.Listener listener transport.Listener
// connected records
// record.Address:true
connected map[string]bool
// leases for connections to us // leases for connections to us
// link id:link // link remote node:link
links map[string]*link links map[string][]*link
// messages received over links // messages received over links
recv chan *Message recv chan *Message
@ -57,8 +67,12 @@ type node struct {
func newNode(n *network) (*node, error) { func newNode(n *network) (*node, error) {
// create a new node // create a new node
node := &node{ node := &node{
// this nodes unique micro assigned mac address
muid: fmt.Sprintf("%s-%s", n.id, uuid.New().String()),
// map of connected records
connected: make(map[string]bool),
// the links // the links
links: make(map[string]*link), links: make(map[string][]*link),
// closed channel // closed channel
closed: make(chan bool), closed: make(chan bool),
// set the nodes network // set the nodes network
@ -113,17 +127,6 @@ func newNode(n *network) (*node, error) {
// forwards to every link we have // forwards to every link we have
go node.process() go node.process()
// lookup the network to see if there's any nodes
records := n.lookup(node.registry)
// assuming if there are no records, we are the first
// we set ourselves a lease. should we actually do this?
if len(records) == 0 {
// set your own node id
lease := n.lease()
node.id = lease.Node.Id
}
var port int var port int
// TODO: this should be an overlay address // TODO: this should be an overlay address
// ideally received via some dhcp style broadcast // ideally received via some dhcp style broadcast
@ -140,6 +143,9 @@ func newNode(n *network) (*node, error) {
// set the address // set the address
addr, _ := addr.Extract(host) addr, _ := addr.Extract(host)
// used to register in registry for network resolution
// separate to our lease on the network itself
node.id = uuid.New().String()
node.address = fmt.Sprintf("%s:%d", addr, port) node.address = fmt.Sprintf("%s:%d", addr, port)
// register self with the registry using network: prefix // register self with the registry using network: prefix
@ -169,15 +175,19 @@ func newNode(n *network) (*node, error) {
link := <-linkChan link := <-linkChan
// process this link // process this link
log.Debugf("connect managing link %s", link.id)
go node.manage(link) go node.manage(link)
go func() { go func() {
// process any further new links for {
select { // process any further new links
case l := <-linkChan: select {
go node.manage(l) case l := <-linkChan:
case <-node.closed: log.Debugf("Managing new link %s", l.id)
return go node.manage(l)
case <-node.closed:
return
}
} }
}() }()
@ -203,18 +213,7 @@ func (n *node) accept(l transport.Listener) error {
}() }()
// create a new link // create a new link
link := &link{ link := newLink(n, sock, nil)
// link has a unique id
id: uuid.New().String(),
// proto marshaler
codec: proto.Marshaler{},
// link has a socket
socket: sock,
// for generating leases,
node: n,
// the send queue,
queue: make(chan *Message, 128),
}
log.Debugf("Accepting connection from %s", link.socket.Remote()) log.Debugf("Accepting connection from %s", link.socket.Remote())
@ -222,24 +221,44 @@ func (n *node) accept(l transport.Listener) error {
// the remote end will send "Connect" // the remote end will send "Connect"
// and we will return a "Lease" // and we will return a "Lease"
if err := link.accept(); err != nil { if err := link.accept(); err != nil {
log.Debugf("Error accepting connection %v", err)
return return
} }
log.Debugf("Accepted link from %s", link.socket.Remote()) log.Debugf("Accepted link from %s", link.socket.Remote())
// save with the remote address as the key // save with the muid as the key
// where we attempt to connect to nodes // where we attempt to connect to nodes
// we do not connect to the same thing // we do not connect to the same thing
n.Lock()
n.links[link.socket.Remote()] = link // TODO: figure out why this is an issue
n.Unlock() // When we receive a connection from ourself
// we can't maintain the two links separately
// so we don't save it. It's basically some
// weird loopback issue because its our own socket.
if n.muid != link.lease.Node.Muid {
n.Lock()
// get the links
links := n.links[link.lease.Node.Muid]
// append to the current links
links = append(links, link)
// save the links with muid as the key
n.links[link.lease.Node.Muid] = links
n.Unlock()
}
// manage the link for its lifetime // manage the link for its lifetime
log.Debugf("managing the link now %s", link.id)
n.manage(link) n.manage(link)
}) })
} }
// processes the send queue // processes the sends the messages from n.Send into the queue of
// each link. If multiple links exist for a muid it should only
// send on link to figure it out.
// If we connected to a record and that link goes down we should
// also remove it from the n.connected map.
func (n *node) process() { func (n *node) process() {
for { for {
select { select {
@ -251,29 +270,51 @@ func (n *node) process() {
// queue the message on each link // queue the message on each link
// TODO: more than likely use proxy // TODO: more than likely use proxy
n.RLock() n.RLock()
for _, l := range n.links { // range over all the links
l.queue <- m for _, links := range n.links {
if len(links) == 0 {
continue
}
// sort the links by weight
sort.Slice(links, func(i, j int) bool {
return links[i].Weight() < links[j].Weight()
})
// queue the message
log.Debugf("sending on link %s", links[0].id)
links[0].Send(m)
} }
n.RUnlock() n.RUnlock()
} }
} }
} }
// Manage manages the link for its lifetime. It should ideally throw
// away the link in the n.links map if there's any issues or total disconnection
// it should look at link.Status.
// If we connected to a record and that link goes down we should
// also remove it from the n.connected map.
func (n *node) manage(l *link) { func (n *node) manage(l *link) {
// now process inbound messages on the link // now process inbound messages on the link
// assumption is this handles everything else // assumption is this handles everything else
for { for {
// get a message on the link // the send side uses a link queue but the receive side immediately sends it
m := new(Message) // ideally we should probably have an internal queue on that side as well
if err := l.recv(m, nil); err != nil { // so we can judge link saturation both ways.
m, err := l.Accept()
if err != nil {
log.Debugf("Error accepting message on link %s: %v", l.id, err)
// ??? // ???
return return
} }
// if the node connection is closed bail out
select { select {
case <-n.closed: case <-n.closed:
return return
// send to the recv channel e.g node.Accept() // send to the network recv channel e.g node.Accept()
case n.recv <- m: case n.recv <- m:
} }
} }
@ -301,18 +342,15 @@ func (n *node) connect(linkChan chan *link) {
n.RLock() n.RLock()
// only start processing if we have less than 3 links // only start processing if we have less than 3 links
if len(n.links) > 2 { conns := len(n.links)
if conns > 2 {
n.RUnlock() n.RUnlock()
continue continue
} }
// get a list of link addresses so we don't reconnect // get a list of link addresses so we don't reconnect
// to the ones we're already connected to // to the ones we're already connected to
nodes := map[string]bool{} connected := n.connected
for addr, _ := range n.links {
// id is the lookup address used to connect
nodes[addr] = true
}
// unlock our read lock // unlock our read lock
n.RUnlock() n.RUnlock()
@ -329,11 +367,17 @@ func (n *node) connect(linkChan chan *link) {
// while still connecting to the global network // while still connecting to the global network
for _, record := range records { for _, record := range records {
// skip existing connections // skip existing connections
if nodes[record.Address] { if connected[record.Address] {
log.Debugf("Skipping connection to %s", record.Address) log.Tracef("Skipping connection to %s", record.Address)
continue continue
} }
// check how many connections we have
if conns > 2 {
log.Debugf("Made enough connections")
break
}
// attempt to connect and create a link // attempt to connect and create a link
log.Debugf("Dialing connection to %s", record.Address) log.Debugf("Dialing connection to %s", record.Address)
@ -346,13 +390,7 @@ func (n *node) connect(linkChan chan *link) {
} }
// create a new link with the lease and socket // create a new link with the lease and socket
link := &link{ link := newLink(n, sock, lease)
codec: &proto.Marshaler{},
id: uuid.New().String(),
lease: lease,
socket: sock,
queue: make(chan *Message, 128),
}
log.Debugf("Connecting link to %s", record.Address) log.Debugf("Connecting link to %s", record.Address)
@ -363,7 +401,6 @@ func (n *node) connect(linkChan chan *link) {
// first connect will not have a lease so we get one with node id/address // first connect will not have a lease so we get one with node id/address
if err := link.connect(); err != nil { if err := link.connect(); err != nil {
// shit // shit
link.Close()
continue continue
} }
@ -375,23 +412,39 @@ func (n *node) connect(linkChan chan *link) {
// we may have to expire the lease // we may have to expire the lease
lease = link.lease lease = link.lease
// save the new link // save the new link
n.links[link.socket.Remote()] = link // get existing links using the lease author
links := n.links[lease.Author]
// append to the links
links = append(links, link)
// save the links using the author
n.links[lease.Author] = links
n.Unlock()
// update number of connections
conns++
// save the connection
n.Lock()
n.connected[record.Address] = true
n.Unlock() n.Unlock()
// drop this down the link channel to the network // drop this down the link channel to the network
// so it can manage the links // so it can manage the links
select { linkChan <- link
case linkChan <- link:
// we don't wait for anyone
default:
}
} }
} }
} }
} }
func (n *node) Address() string { func (n *node) Address() string {
return n.address n.RLock()
defer n.RUnlock()
// we have no address yet
if n.lease == nil {
return ""
}
// return node address in the lease
return n.lease.Node.Address
} }
// Close shutdowns all the links and closes the listener // Close shutdowns all the links and closes the listener
@ -400,15 +453,19 @@ func (n *node) Close() error {
case <-n.closed: case <-n.closed:
return nil return nil
default: default:
// mark as closed // mark as closed, we're now useless and there's no coming back
close(n.closed) close(n.closed)
// shutdown all the links // shutdown all the links
n.Lock() n.Lock()
for id, link := range n.links { for muid, links := range n.links {
link.Close() for _, link := range links {
delete(n.links, id) link.Close()
}
delete(n.links, muid)
} }
// reset connected
n.connected = nil
n.Unlock() n.Unlock()
// deregister self // deregister self
@ -444,6 +501,15 @@ func (n *node) Accept() (*Message, error) {
return nil, nil return nil, nil
} }
func (n *node) Id() string {
n.RLock()
defer n.RUnlock()
if n.lease == nil {
return ""
}
return n.lease.Node.Id
}
func (n *node) Network() string { func (n *node) Network() string {
return n.network.id return n.network.id
} }

View File

@ -22,10 +22,12 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
// A connect message is for connecting to the network // A connect message is for connecting to the network
type Connect struct { type Connect struct {
// the unique muid (mac) address of the node
Muid string `protobuf:"bytes,1,opt,name=muid,proto3" json:"muid,omitempty"`
// Lease specifies an existing lease to indicate // Lease specifies an existing lease to indicate
// we don't need a new address, we just want to // we don't need a new address, we just want to
// establish a link. // establish a link.
Lease *Lease `protobuf:"bytes,1,opt,name=lease,proto3" json:"lease,omitempty"` Lease *Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -56,6 +58,13 @@ func (m *Connect) XXX_DiscardUnknown() {
var xxx_messageInfo_Connect proto.InternalMessageInfo var xxx_messageInfo_Connect proto.InternalMessageInfo
func (m *Connect) GetMuid() string {
if m != nil {
return m.Muid
}
return ""
}
func (m *Connect) GetLease() *Lease { func (m *Connect) GetLease() *Lease {
if m != nil { if m != nil {
return m.Lease return m.Lease
@ -63,14 +72,16 @@ func (m *Connect) GetLease() *Lease {
return nil return nil
} }
// A lease is returned to anyone attempting to connect. // A lease is returned to anyone attempting to connect with a valid muid.
type Lease struct { type Lease struct {
// unique lease id // unique lease id
Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
// timestamp of lease // timestamp of lease
Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
// author is the muid of the author
Author string `protobuf:"bytes,3,opt,name=author,proto3" json:"author,omitempty"`
// the node // the node
Node *Node `protobuf:"bytes,3,opt,name=node,proto3" json:"node,omitempty"` Node *Node `protobuf:"bytes,4,opt,name=node,proto3" json:"node,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -115,6 +126,13 @@ func (m *Lease) GetTimestamp() int64 {
return 0 return 0
} }
func (m *Lease) GetAuthor() string {
if m != nil {
return m.Author
}
return ""
}
func (m *Lease) GetNode() *Node { func (m *Lease) GetNode() *Node {
if m != nil { if m != nil {
return m.Node return m.Node
@ -124,9 +142,10 @@ func (m *Lease) GetNode() *Node {
// A node is the network node // A node is the network node
type Node struct { type Node struct {
Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` Muid string `protobuf:"bytes,1,opt,name=muid,proto3" json:"muid,omitempty"`
Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` Id string `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
Network string `protobuf:"bytes,3,opt,name=network,proto3" json:"network,omitempty"` Address string `protobuf:"bytes,3,opt,name=address,proto3" json:"address,omitempty"`
Network string `protobuf:"bytes,4,opt,name=network,proto3" json:"network,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -157,6 +176,13 @@ func (m *Node) XXX_DiscardUnknown() {
var xxx_messageInfo_Node proto.InternalMessageInfo var xxx_messageInfo_Node proto.InternalMessageInfo
func (m *Node) GetMuid() string {
if m != nil {
return m.Muid
}
return ""
}
func (m *Node) GetId() string { func (m *Node) GetId() string {
if m != nil { if m != nil {
return m.Id return m.Id
@ -189,18 +215,20 @@ func init() {
} }
var fileDescriptor_4daa91d05ddc28b6 = []byte{ var fileDescriptor_4daa91d05ddc28b6 = []byte{
// 203 bytes of a gzipped FileDescriptorProto // 229 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x90, 0xc1, 0x4a, 0xc4, 0x30, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0xc1, 0x6a, 0xc3, 0x30,
0x10, 0x86, 0x69, 0x77, 0xd7, 0x25, 0x23, 0x88, 0xe4, 0xa0, 0x39, 0x78, 0x58, 0x7a, 0x2a, 0x85, 0x0c, 0x86, 0x49, 0x9a, 0xb6, 0x44, 0x83, 0x31, 0x74, 0xe8, 0x7c, 0xd8, 0xa1, 0xe4, 0x54, 0x0a,
0xa6, 0xa0, 0x17, 0xef, 0xde, 0x44, 0x3c, 0xe4, 0x0d, 0x62, 0x33, 0x94, 0xa0, 0xcd, 0x94, 0x24, 0x75, 0x61, 0x7b, 0x84, 0x5d, 0xcb, 0x0e, 0x7e, 0x80, 0x41, 0x56, 0x8b, 0x2e, 0x6c, 0xb6, 0x8a,
0xe0, 0xeb, 0x4b, 0xa7, 0xad, 0x82, 0xbd, 0xe5, 0x9b, 0xf9, 0xf3, 0x7f, 0x30, 0xd0, 0x8c, 0xbe, 0xed, 0x32, 0xf6, 0xf6, 0x23, 0x8a, 0x93, 0xc1, 0xd6, 0x9b, 0x3e, 0xe9, 0xf7, 0xff, 0xcb, 0x82,
0x8f, 0xd4, 0x0d, 0xd4, 0x2e, 0x8f, 0x80, 0xf9, 0x9b, 0xe2, 0x67, 0x37, 0x45, 0xca, 0xbf, 0xa4, 0xad, 0xeb, 0x8e, 0x81, 0xf7, 0x27, 0xde, 0x0d, 0x85, 0xa7, 0xf4, 0xc5, 0xe1, 0x63, 0x7f, 0x0e,
0x99, 0xe4, 0xed, 0x40, 0x9a, 0x53, 0x7a, 0x9d, 0x57, 0xcf, 0x70, 0x7e, 0xa1, 0x10, 0xb0, 0xcf, 0x9c, 0x26, 0xd2, 0x42, 0x78, 0x77, 0x62, 0x2d, 0x2a, 0x9d, 0xfb, 0xcd, 0x01, 0x96, 0xcf, 0xec,
0xb2, 0x85, 0xd3, 0x17, 0xda, 0x84, 0xaa, 0xb8, 0x14, 0xf5, 0xf5, 0xe3, 0xbd, 0xfe, 0x1f, 0xd6, 0x3d, 0x1d, 0x13, 0x22, 0x54, 0xee, 0xd2, 0x59, 0x55, 0xac, 0x8b, 0x4d, 0x6d, 0xa4, 0xc6, 0x1d,
0x6f, 0xf3, 0xda, 0x2c, 0xa9, 0xca, 0xc2, 0x89, 0x59, 0xde, 0x40, 0xe9, 0x1d, 0x7f, 0x12, 0xa6, 0xcc, 0x3f, 0xa9, 0x8d, 0xa4, 0xca, 0x75, 0xb1, 0xb9, 0x79, 0xbc, 0xd7, 0x7f, 0x0d, 0xf4, 0xa1,
0xf4, 0x4e, 0x3e, 0x80, 0xc8, 0x7e, 0xc4, 0x94, 0xed, 0x38, 0xa9, 0xf2, 0x52, 0xd4, 0x07, 0xf3, 0x1f, 0x9b, 0x41, 0xd5, 0x7c, 0xc3, 0x5c, 0x18, 0x6f, 0xa1, 0x9c, 0x9c, 0xca, 0xce, 0xe2, 0x03,
0x37, 0x90, 0x0d, 0x1c, 0x03, 0x39, 0x54, 0x07, 0x96, 0xdc, 0xed, 0x25, 0xef, 0xe4, 0xd0, 0x70, 0xd4, 0xa9, 0x73, 0x14, 0x53, 0xeb, 0xce, 0xe2, 0x35, 0x33, 0xbf, 0x0d, 0x5c, 0xc1, 0xa2, 0xbd,
0xa6, 0x7a, 0x85, 0xe3, 0x4c, 0x3b, 0x83, 0x82, 0xb3, 0x75, 0x2e, 0x62, 0x4a, 0xdc, 0x2f, 0xcc, 0xa4, 0x77, 0x0e, 0x6a, 0x26, 0x2f, 0x32, 0xe1, 0x16, 0x2a, 0xcf, 0x96, 0x54, 0x25, 0xe1, 0xab,
0x86, 0xf3, 0x66, 0xed, 0x61, 0x81, 0x30, 0x1b, 0x7e, 0x5c, 0xf1, 0x05, 0x9e, 0x7e, 0x02, 0x00, 0xff, 0xe1, 0x2f, 0x6c, 0xc9, 0x88, 0xa6, 0x79, 0x85, 0xaa, 0xa7, 0xab, 0xbf, 0x18, 0xb6, 0x29,
0x00, 0xff, 0xff, 0xf3, 0x2b, 0x3d, 0x08, 0x2f, 0x01, 0x00, 0x00, 0xa7, 0x6d, 0x14, 0x2c, 0x5b, 0x6b, 0x03, 0xc5, 0x98, 0x03, 0x47, 0xec, 0x27, 0xd9, 0x5b, 0x42,
0x6b, 0x33, 0xe2, 0xdb, 0x42, 0x2e, 0xf8, 0xf4, 0x13, 0x00, 0x00, 0xff, 0xff, 0xba, 0xcc, 0x46,
0x1e, 0x6f, 0x01, 0x00, 0x00,
} }

View File

@ -4,25 +4,31 @@ package go.micro.network;
// A connect message is for connecting to the network // A connect message is for connecting to the network
message Connect { message Connect {
// the unique muid (mac) address of the node
string muid = 1;
// Lease specifies an existing lease to indicate // Lease specifies an existing lease to indicate
// we don't need a new address, we just want to // we don't need a new address, we just want to
// establish a link. // establish a link.
Lease lease = 1; Lease lease = 2;
} }
// A lease is returned to anyone attempting to connect. // A lease is returned to anyone attempting to connect with a valid muid.
message Lease { message Lease {
// unique lease id // unique lease id
string id = 1; string id = 1;
// timestamp of lease // timestamp of lease
int64 timestamp = 2; int64 timestamp = 2;
// author is the muid of the author
string author = 3;
// the node // the node
Node node = 3; Node node = 4;
} }
// A node is the network node // A node is the network node
message Node { message Node {
string id = 1; string muid = 1;
string address = 2; string id = 2;
string network = 3; string address = 3;
string network = 4;
} }