Merge pull request #882 from micro/link-state
A few changes for the network / tunnel link state
This commit is contained in:
		| @@ -677,19 +677,19 @@ func (n *network) getHopCount(rtr string) int { | |||||||
|  |  | ||||||
| 	// the route origin is our peer | 	// the route origin is our peer | ||||||
| 	if _, ok := n.peers[rtr]; ok { | 	if _, ok := n.peers[rtr]; ok { | ||||||
| 		return 2 | 		return 10 | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// the route origin is the peer of our peer | 	// the route origin is the peer of our peer | ||||||
| 	for _, peer := range n.peers { | 	for _, peer := range n.peers { | ||||||
| 		for id := range peer.peers { | 		for id := range peer.peers { | ||||||
| 			if rtr == id { | 			if rtr == id { | ||||||
| 				return 3 | 				return 100 | ||||||
| 			} | 			} | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| 	// otherwise we are three hops away | 	// otherwise we are three hops away | ||||||
| 	return 4 | 	return 1000 | ||||||
| } | } | ||||||
|  |  | ||||||
| // getRouteMetric calculates router metric and returns it | // getRouteMetric calculates router metric and returns it | ||||||
| @@ -721,11 +721,15 @@ func (n *network) getRouteMetric(router string, gateway string, link string) int | |||||||
| 		// make sure length is non-zero | 		// make sure length is non-zero | ||||||
| 		length := link.Length() | 		length := link.Length() | ||||||
| 		if length == 0 { | 		if length == 0 { | ||||||
| 			length = 10e10 | 			log.Debugf("Link length is 0 %v %v", link, link.Length()) | ||||||
|  | 			length = 10e9 | ||||||
| 		} | 		} | ||||||
| 		return (delay * length * int64(hops)) / 10e9 | 		log.Debugf("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) | 	log.Debugf("Network failed to find a link to gateway: %s", gateway) | ||||||
|  |  | ||||||
| 	return math.MaxInt64 | 	return math.MaxInt64 | ||||||
| } | } | ||||||
|  |  | ||||||
| @@ -783,12 +787,18 @@ func (n *network) processCtrlChan(listener tunnel.Listener) { | |||||||
| 					} | 					} | ||||||
| 					// calculate route metric and add to the advertised metric | 					// calculate route metric and add to the advertised metric | ||||||
| 					// we need to make sure we do not overflow math.MaxInt64 | 					// we need to make sure we do not overflow math.MaxInt64 | ||||||
| 					log.Debugf("Network metric for router %s and gateway %s", event.Route.Router, event.Route.Gateway) | 					metric := n.getRouteMetric(event.Route.Router, event.Route.Gateway, event.Route.Link) | ||||||
| 					if metric := n.getRouteMetric(event.Route.Router, event.Route.Gateway, event.Route.Link); metric != math.MaxInt64 { | 					log.Debugf("Network metric for router %s and gateway %s: %v", event.Route.Router, event.Route.Gateway, metric) | ||||||
| 						route.Metric += metric |  | ||||||
|  | 					// check we don't overflow max int 64 | ||||||
|  | 					if d := route.Metric + metric; d > math.MaxInt64 || d <= 0 { | ||||||
|  | 						// set to max int64 if we overflow | ||||||
|  | 						route.Metric = math.MaxInt64 | ||||||
| 					} else { | 					} else { | ||||||
| 						route.Metric = metric | 						// set the combined value of metrics otherwise | ||||||
|  | 						route.Metric = d | ||||||
| 					} | 					} | ||||||
|  |  | ||||||
| 					// create router event | 					// create router event | ||||||
| 					e := &router.Event{ | 					e := &router.Event{ | ||||||
| 						Type:      router.EventType(event.Type), | 						Type:      router.EventType(event.Type), | ||||||
|   | |||||||
| @@ -19,6 +19,7 @@ import ( | |||||||
| 	"github.com/micro/go-micro/proxy" | 	"github.com/micro/go-micro/proxy" | ||||||
| 	"github.com/micro/go-micro/router" | 	"github.com/micro/go-micro/router" | ||||||
| 	"github.com/micro/go-micro/server" | 	"github.com/micro/go-micro/server" | ||||||
|  | 	"github.com/micro/go-micro/util/log" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| // Proxy will transparently proxy requests to an endpoint. | // Proxy will transparently proxy requests to an endpoint. | ||||||
| @@ -294,6 +295,8 @@ func (p *Proxy) ServeRequest(ctx context.Context, req server.Request, rsp server | |||||||
| 			continue | 			continue | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
|  | 		log.Debugf("Proxy using route %+v\n", route) | ||||||
|  |  | ||||||
| 		// set the address to call | 		// set the address to call | ||||||
| 		addresses := toNodes([]router.Route{route}) | 		addresses := toNodes([]router.Route{route}) | ||||||
| 		opts = append(opts, client.WithAddress(addresses...)) | 		opts = append(opts, client.WithAddress(addresses...)) | ||||||
|   | |||||||
| @@ -90,6 +90,7 @@ func (t *tun) getSession(channel, session string) (*session, bool) { | |||||||
| 	return s, ok | 	return s, ok | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // delSession deletes a session if it exists | ||||||
| func (t *tun) delSession(channel, session string) { | func (t *tun) delSession(channel, session string) { | ||||||
| 	t.Lock() | 	t.Lock() | ||||||
| 	delete(t.sessions, channel+session) | 	delete(t.sessions, channel+session) | ||||||
| @@ -146,6 +147,9 @@ func (t *tun) newSessionId() string { | |||||||
| 	return uuid.New().String() | 	return uuid.New().String() | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // announce will send a message to the link to tell the other side of a channel mapping we have. | ||||||
|  | // This usually happens if someone calls Dial and sends a discover message but otherwise we | ||||||
|  | // periodically send these messages to asynchronously manage channel mappings. | ||||||
| func (t *tun) announce(channel, session string, link *link) { | func (t *tun) announce(channel, session string, link *link) { | ||||||
| 	// create the "announce" response message for a discover request | 	// create the "announce" response message for a discover request | ||||||
| 	msg := &transport.Message{ | 	msg := &transport.Message{ | ||||||
| @@ -206,7 +210,7 @@ func (t *tun) monitor() { | |||||||
| 			// check the link status and purge dead links | 			// check the link status and purge dead links | ||||||
| 			for node, link := range t.links { | 			for node, link := range t.links { | ||||||
| 				// check link status | 				// check link status | ||||||
| 				switch link.Status() { | 				switch link.State() { | ||||||
| 				case "closed": | 				case "closed": | ||||||
| 					delLinks = append(delLinks, node) | 					delLinks = append(delLinks, node) | ||||||
| 				case "error": | 				case "error": | ||||||
| @@ -303,8 +307,16 @@ func (t *tun) process() { | |||||||
|  |  | ||||||
| 			// build the list of links ot send to | 			// build the list of links ot send to | ||||||
| 			for node, link := range t.links { | 			for node, link := range t.links { | ||||||
|  | 				// get the values we need | ||||||
|  | 				link.RLock() | ||||||
|  | 				id := link.id | ||||||
|  | 				connected := link.connected | ||||||
|  | 				loopback := link.loopback | ||||||
|  | 				_, exists := link.channels[msg.channel] | ||||||
|  | 				link.RUnlock() | ||||||
|  |  | ||||||
| 				// if the link is not connected skip it | 				// if the link is not connected skip it | ||||||
| 				if !link.connected { | 				if !connected { | ||||||
| 					log.Debugf("Link for node %s not connected", node) | 					log.Debugf("Link for node %s not connected", node) | ||||||
| 					err = errors.New("link not connected") | 					err = errors.New("link not connected") | ||||||
| 					continue | 					continue | ||||||
| @@ -313,32 +325,29 @@ func (t *tun) process() { | |||||||
| 				// if the link was a loopback accepted connection | 				// if the link was a loopback accepted connection | ||||||
| 				// and the message is being sent outbound via | 				// and the message is being sent outbound via | ||||||
| 				// a dialled connection don't use this link | 				// a dialled connection don't use this link | ||||||
| 				if link.loopback && msg.outbound { | 				if loopback && msg.outbound { | ||||||
| 					err = errors.New("link is loopback") | 					err = errors.New("link is loopback") | ||||||
| 					continue | 					continue | ||||||
| 				} | 				} | ||||||
|  |  | ||||||
| 				// if the message was being returned by the loopback listener | 				// if the message was being returned by the loopback listener | ||||||
| 				// send it back up the loopback link only | 				// send it back up the loopback link only | ||||||
| 				if msg.loopback && !link.loopback { | 				if msg.loopback && !loopback { | ||||||
| 					err = errors.New("link is not loopback") | 					err = errors.New("link is not loopback") | ||||||
| 					continue | 					continue | ||||||
| 				} | 				} | ||||||
|  |  | ||||||
| 				// check the multicast mappings | 				// check the multicast mappings | ||||||
| 				if msg.mode == Multicast { | 				if msg.mode == Multicast { | ||||||
| 					link.RLock() |  | ||||||
| 					_, ok := link.channels[msg.channel] |  | ||||||
| 					link.RUnlock() |  | ||||||
| 					// channel mapping not found in link | 					// channel mapping not found in link | ||||||
| 					if !ok { | 					if !exists { | ||||||
| 						continue | 						continue | ||||||
| 					} | 					} | ||||||
| 				} else { | 				} else { | ||||||
| 					// if we're picking the link check the id | 					// if we're picking the link check the id | ||||||
| 					// this is where we explicitly set the link | 					// this is where we explicitly set the link | ||||||
| 					// in a message received via the listen method | 					// in a message received via the listen method | ||||||
| 					if len(msg.link) > 0 && link.id != msg.link { | 					if len(msg.link) > 0 && id != msg.link { | ||||||
| 						err = errors.New("link not found") | 						err = errors.New("link not found") | ||||||
| 						continue | 						continue | ||||||
| 					} | 					} | ||||||
| @@ -422,6 +431,12 @@ func (t *tun) listen(link *link) { | |||||||
|  |  | ||||||
| 	// let us know if its a loopback | 	// let us know if its a loopback | ||||||
| 	var loopback bool | 	var loopback bool | ||||||
|  | 	var connected bool | ||||||
|  |  | ||||||
|  | 	// set the connected value | ||||||
|  | 	link.RLock() | ||||||
|  | 	connected = link.connected | ||||||
|  | 	link.RUnlock() | ||||||
|  |  | ||||||
| 	for { | 	for { | ||||||
| 		// process anything via the net interface | 		// process anything via the net interface | ||||||
| @@ -451,7 +466,7 @@ func (t *tun) listen(link *link) { | |||||||
|  |  | ||||||
| 		// if its not connected throw away the link | 		// if its not connected throw away the link | ||||||
| 		// the first message we process needs to be connect | 		// the first message we process needs to be connect | ||||||
| 		if !link.connected && mtype != "connect" { | 		if !connected && mtype != "connect" { | ||||||
| 			log.Debugf("Tunnel link %s not connected", link.id) | 			log.Debugf("Tunnel link %s not connected", link.id) | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
| @@ -461,7 +476,8 @@ func (t *tun) listen(link *link) { | |||||||
| 			log.Debugf("Tunnel link %s received connect message", link.Remote()) | 			log.Debugf("Tunnel link %s received connect message", link.Remote()) | ||||||
|  |  | ||||||
| 			link.Lock() | 			link.Lock() | ||||||
| 			// are we connecting to ourselves? |  | ||||||
|  | 			// check if we're connecting to ourselves? | ||||||
| 			if id == t.id { | 			if id == t.id { | ||||||
| 				link.loopback = true | 				link.loopback = true | ||||||
| 				loopback = true | 				loopback = true | ||||||
| @@ -471,6 +487,8 @@ func (t *tun) listen(link *link) { | |||||||
| 			link.id = link.Remote() | 			link.id = link.Remote() | ||||||
| 			// set as connected | 			// set as connected | ||||||
| 			link.connected = true | 			link.connected = true | ||||||
|  | 			connected = true | ||||||
|  |  | ||||||
| 			link.Unlock() | 			link.Unlock() | ||||||
|  |  | ||||||
| 			// save the link once connected | 			// save the link once connected | ||||||
| @@ -494,9 +512,7 @@ func (t *tun) listen(link *link) { | |||||||
|  |  | ||||||
| 			// the entire listener was closed so remove it from the mapping | 			// the entire listener was closed so remove it from the mapping | ||||||
| 			if sessionId == "listener" { | 			if sessionId == "listener" { | ||||||
| 				link.Lock() | 				link.delChannel(channel) | ||||||
| 				delete(link.channels, channel) |  | ||||||
| 				link.Unlock() |  | ||||||
| 				continue | 				continue | ||||||
| 			} | 			} | ||||||
|  |  | ||||||
| @@ -510,10 +526,8 @@ func (t *tun) listen(link *link) { | |||||||
| 			// otherwise its a session mapping of sorts | 			// otherwise its a session mapping of sorts | ||||||
| 		case "keepalive": | 		case "keepalive": | ||||||
| 			log.Debugf("Tunnel link %s received keepalive", link.Remote()) | 			log.Debugf("Tunnel link %s received keepalive", link.Remote()) | ||||||
| 			link.Lock() |  | ||||||
| 			// save the keepalive | 			// save the keepalive | ||||||
| 			link.lastKeepAlive = time.Now() | 			link.keepalive() | ||||||
| 			link.Unlock() |  | ||||||
| 			continue | 			continue | ||||||
| 		// a new connection dialled outbound | 		// a new connection dialled outbound | ||||||
| 		case "open": | 		case "open": | ||||||
| @@ -540,11 +554,7 @@ func (t *tun) listen(link *link) { | |||||||
| 			channels := strings.Split(channel, ",") | 			channels := strings.Split(channel, ",") | ||||||
|  |  | ||||||
| 			// update mapping in the link | 			// update mapping in the link | ||||||
| 			link.Lock() | 			link.setChannel(channels...) | ||||||
| 			for _, channel := range channels { |  | ||||||
| 				link.channels[channel] = time.Now() |  | ||||||
| 			} |  | ||||||
| 			link.Unlock() |  | ||||||
|  |  | ||||||
| 			// this was an announcement not intended for anything | 			// this was an announcement not intended for anything | ||||||
| 			if sessionId == "listener" || sessionId == "" { | 			if sessionId == "listener" || sessionId == "" { | ||||||
| @@ -746,8 +756,13 @@ func (t *tun) setupLink(node string) (*link, error) { | |||||||
| 	} | 	} | ||||||
| 	log.Debugf("Tunnel connected to %s", node) | 	log.Debugf("Tunnel connected to %s", node) | ||||||
|  |  | ||||||
|  | 	// create a new link | ||||||
|  | 	link := newLink(c) | ||||||
|  | 	// set link id to remote side | ||||||
|  | 	link.id = c.Remote() | ||||||
|  |  | ||||||
| 	// send the first connect message | 	// send the first connect message | ||||||
| 	if err := c.Send(&transport.Message{ | 	if err := link.Send(&transport.Message{ | ||||||
| 		Header: map[string]string{ | 		Header: map[string]string{ | ||||||
| 			"Micro-Tunnel":       "connect", | 			"Micro-Tunnel":       "connect", | ||||||
| 			"Micro-Tunnel-Id":    t.id, | 			"Micro-Tunnel-Id":    t.id, | ||||||
| @@ -757,10 +772,6 @@ func (t *tun) setupLink(node string) (*link, error) { | |||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// create a new link |  | ||||||
| 	link := newLink(c) |  | ||||||
| 	// set link id to remote side |  | ||||||
| 	link.id = c.Remote() |  | ||||||
| 	// we made the outbound connection | 	// we made the outbound connection | ||||||
| 	// and sent the connect message | 	// and sent the connect message | ||||||
| 	link.connected = true | 	link.connected = true | ||||||
| @@ -903,6 +914,53 @@ func (t *tun) close() error { | |||||||
| 	return t.listener.Close() | 	return t.listener.Close() | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // pickLink will pick the best link based on connectivity, delay, rate and length | ||||||
|  | func (t *tun) pickLink(links []*link) *link { | ||||||
|  | 	var metric float64 | ||||||
|  | 	var chosen *link | ||||||
|  |  | ||||||
|  | 	// find the best link | ||||||
|  | 	for i, link := range links { | ||||||
|  | 		// don't use disconnected or errored links | ||||||
|  | 		if link.State() != "connected" { | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// get the link state info | ||||||
|  | 		d := float64(link.Delay()) | ||||||
|  | 		l := float64(link.Length()) | ||||||
|  | 		r := link.Rate() | ||||||
|  |  | ||||||
|  | 		// metric = delay x length x rate | ||||||
|  | 		m := d * l * r | ||||||
|  |  | ||||||
|  | 		// first link so just and go | ||||||
|  | 		if i == 0 { | ||||||
|  | 			metric = m | ||||||
|  | 			chosen = link | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// we found a better metric | ||||||
|  | 		if m < metric { | ||||||
|  | 			metric = m | ||||||
|  | 			chosen = link | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// if there's no link we're just going to mess around | ||||||
|  | 	if chosen == nil { | ||||||
|  | 		i := rand.Intn(len(links)) | ||||||
|  | 		return links[i] | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// we chose the link with; | ||||||
|  | 	// the lowest delay e.g least messages queued | ||||||
|  | 	// the lowest rate e.g the least messages flowing | ||||||
|  | 	// the lowest length e.g the smallest roundtrip time | ||||||
|  | 	return chosen | ||||||
|  | } | ||||||
|  |  | ||||||
| func (t *tun) Address() string { | func (t *tun) Address() string { | ||||||
| 	t.RLock() | 	t.RLock() | ||||||
| 	defer t.RUnlock() | 	defer t.RUnlock() | ||||||
| @@ -966,42 +1024,32 @@ func (t *tun) Dial(channel string, opts ...DialOption) (Session, error) { | |||||||
| 	c.mode = options.Mode | 	c.mode = options.Mode | ||||||
| 	// set the dial timeout | 	// set the dial timeout | ||||||
| 	c.timeout = options.Timeout | 	c.timeout = options.Timeout | ||||||
| 	// get the current time |  | ||||||
| 	now := time.Now() |  | ||||||
|  |  | ||||||
| 	after := func() time.Duration { | 	var links []*link | ||||||
| 		d := time.Since(now) |  | ||||||
| 		// dial timeout minus time since |  | ||||||
| 		wait := options.Timeout - d |  | ||||||
| 		if wait < time.Duration(0) { |  | ||||||
| 			return time.Duration(0) |  | ||||||
| 		} |  | ||||||
| 		return wait |  | ||||||
| 	} |  | ||||||
|  |  | ||||||
| 	var links []string |  | ||||||
| 	// did we measure the rtt | 	// did we measure the rtt | ||||||
| 	var measured bool | 	var measured bool | ||||||
|  |  | ||||||
| 	// non multicast so we need to find the link |  | ||||||
| 	t.RLock() | 	t.RLock() | ||||||
|  |  | ||||||
|  | 	// non multicast so we need to find the link | ||||||
| 	for _, link := range t.links { | 	for _, link := range t.links { | ||||||
| 		// use the link specified it its available | 		// use the link specified it its available | ||||||
| 		if id := options.Link; len(id) > 0 && link.id != id { | 		if id := options.Link; len(id) > 0 && link.id != id { | ||||||
| 			continue | 			continue | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
| 		link.RLock() | 		// get the channel | ||||||
| 		_, ok := link.channels[channel] | 		lastMapped := link.getChannel(channel) | ||||||
| 		link.RUnlock() |  | ||||||
|  |  | ||||||
| 		// we have at least one channel mapping | 		// we have at least one channel mapping | ||||||
| 		if ok { | 		if !lastMapped.IsZero() { | ||||||
|  | 			links = append(links, link) | ||||||
| 			c.discovered = true | 			c.discovered = true | ||||||
| 			links = append(links, link.id) |  | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	t.RUnlock() | 	t.RUnlock() | ||||||
|  |  | ||||||
| 	// link not found | 	// link not found | ||||||
| 	if len(links) == 0 && len(options.Link) > 0 { | 	if len(links) == 0 && len(options.Link) > 0 { | ||||||
| 		// delete session and return error | 		// delete session and return error | ||||||
| @@ -1014,9 +1062,9 @@ func (t *tun) Dial(channel string, opts ...DialOption) (Session, error) { | |||||||
| 	// TODO: pick the link efficiently based | 	// TODO: pick the link efficiently based | ||||||
| 	// on link status and saturation. | 	// on link status and saturation. | ||||||
| 	if c.discovered && c.mode == Unicast { | 	if c.discovered && c.mode == Unicast { | ||||||
| 		// set the link | 		// pickLink will pick the best link | ||||||
| 		i := rand.Intn(len(links)) | 		link := t.pickLink(links) | ||||||
| 		c.link = links[i] | 		c.link = link.id | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// shit fuck | 	// shit fuck | ||||||
| @@ -1024,57 +1072,8 @@ func (t *tun) Dial(channel string, opts ...DialOption) (Session, error) { | |||||||
| 		// piggy back roundtrip | 		// piggy back roundtrip | ||||||
| 		nowRTT := time.Now() | 		nowRTT := time.Now() | ||||||
|  |  | ||||||
| 		// create a new discovery message for this channel | 		// attempt to discover the link | ||||||
| 		msg := c.newMessage("discover") | 		err := c.Discover() | ||||||
| 		msg.mode = Broadcast |  | ||||||
| 		msg.outbound = true |  | ||||||
| 		msg.link = "" |  | ||||||
|  |  | ||||||
| 		// send the discovery message |  | ||||||
| 		t.send <- msg |  | ||||||
|  |  | ||||||
| 		select { |  | ||||||
| 		case <-time.After(after()): |  | ||||||
| 			t.delSession(c.channel, c.session) |  | ||||||
| 			log.Debugf("Tunnel deleting session %s %s: %v", c.session, c.channel, ErrDialTimeout) |  | ||||||
| 			return nil, ErrDialTimeout |  | ||||||
| 		case err := <-c.errChan: |  | ||||||
| 			if err != nil { |  | ||||||
| 				t.delSession(c.channel, c.session) |  | ||||||
| 				log.Debugf("Tunnel deleting session %s %s: %v", c.session, c.channel, err) |  | ||||||
| 				return nil, err |  | ||||||
| 			} |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		var err error |  | ||||||
|  |  | ||||||
| 		// set a dialTimeout |  | ||||||
| 		dialTimeout := after() |  | ||||||
|  |  | ||||||
| 		// set a shorter delay for multicast |  | ||||||
| 		if c.mode != Unicast { |  | ||||||
| 			// shorten this |  | ||||||
| 			dialTimeout = time.Millisecond * 500 |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		// wait for announce |  | ||||||
| 		select { |  | ||||||
| 		case msg := <-c.recv: |  | ||||||
| 			if msg.typ != "announce" { |  | ||||||
| 				err = ErrDiscoverChan |  | ||||||
| 			} |  | ||||||
| 		case <-time.After(dialTimeout): |  | ||||||
| 			err = ErrDialTimeout |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		// if its multicast just go ahead because this is best effort |  | ||||||
| 		if c.mode != Unicast { |  | ||||||
| 			c.discovered = true |  | ||||||
| 			c.accepted = true |  | ||||||
| 			return c, nil |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		// otherwise return an error |  | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			t.delSession(c.channel, c.session) | 			t.delSession(c.channel, c.session) | ||||||
| 			log.Debugf("Tunnel deleting session %s %s: %v", c.session, c.channel, err) | 			log.Debugf("Tunnel deleting session %s %s: %v", c.session, c.channel, err) | ||||||
| @@ -1095,34 +1094,34 @@ func (t *tun) Dial(channel string, opts ...DialOption) (Session, error) { | |||||||
| 			// set measured to true | 			// set measured to true | ||||||
| 			measured = true | 			measured = true | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
| 		// set discovered to true |  | ||||||
| 		c.discovered = true |  | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// a unicast session so we call "open" and wait for an "accept" | 	// a unicast session so we call "open" and wait for an "accept" | ||||||
|  |  | ||||||
| 	// reset now in case we use it | 	// reset now in case we use it | ||||||
| 	now = time.Now() | 	now := time.Now() | ||||||
|  |  | ||||||
| 	// try to open the session | 	// try to open the session | ||||||
| 	err := c.Open() | 	if err := c.Open(); err != nil { | ||||||
| 	if err != nil { |  | ||||||
| 		// delete the session | 		// delete the session | ||||||
| 		t.delSession(c.channel, c.session) | 		t.delSession(c.channel, c.session) | ||||||
| 		log.Debugf("Tunnel deleting session %s %s: %v", c.session, c.channel, err) | 		log.Debugf("Tunnel deleting session %s %s: %v", c.session, c.channel, err) | ||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
|  | 	// set time take to open | ||||||
|  | 	d := time.Since(now) | ||||||
|  |  | ||||||
| 	// if we haven't measured the roundtrip do it now | 	// if we haven't measured the roundtrip do it now | ||||||
| 	if !measured && c.mode == Unicast { | 	if !measured && c.mode == Unicast { | ||||||
| 		// set the link time | 		// set the link time | ||||||
| 		t.RLock() | 		t.RLock() | ||||||
| 		link, ok := t.links[c.link] | 		link, ok := t.links[c.link] | ||||||
| 		t.RUnlock() | 		t.RUnlock() | ||||||
|  |  | ||||||
| 		if ok { | 		if ok { | ||||||
| 			// set the rountrip time | 			// set the rountrip time | ||||||
| 			link.setRTT(time.Since(now)) | 			link.setRTT(d) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
|   | |||||||
							
								
								
									
										151
									
								
								tunnel/link.go
									
									
									
									
									
								
							
							
						
						
									
										151
									
								
								tunnel/link.go
									
									
									
									
									
								
							| @@ -1,12 +1,14 @@ | |||||||
| package tunnel | package tunnel | ||||||
|  |  | ||||||
| import ( | import ( | ||||||
|  | 	"bytes" | ||||||
| 	"io" | 	"io" | ||||||
| 	"sync" | 	"sync" | ||||||
| 	"time" | 	"time" | ||||||
|  |  | ||||||
| 	"github.com/google/uuid" | 	"github.com/google/uuid" | ||||||
| 	"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 { | ||||||
| @@ -15,9 +17,11 @@ type link struct { | |||||||
| 	sync.RWMutex | 	sync.RWMutex | ||||||
| 	// stops the link | 	// stops the link | ||||||
| 	closed chan bool | 	closed chan bool | ||||||
| 	// send queue | 	// link state channel for testing link | ||||||
|  | 	state chan *packet | ||||||
|  | 	// send queue for sending packets | ||||||
| 	sendQueue chan *packet | 	sendQueue chan *packet | ||||||
| 	// receive queue | 	// receive queue for receiving packets | ||||||
| 	recvQueue chan *packet | 	recvQueue chan *packet | ||||||
| 	// unique id of this link e.g uuid | 	// unique id of this link e.g uuid | ||||||
| 	// which we define for ourselves | 	// which we define for ourselves | ||||||
| @@ -56,21 +60,49 @@ type packet struct { | |||||||
| 	err error | 	err error | ||||||
| } | } | ||||||
|  |  | ||||||
|  | var ( | ||||||
|  | 	// the 4 byte 0 packet sent to determine the link state | ||||||
|  | 	linkRequest = []byte{0, 0, 0, 0} | ||||||
|  | 	// the 4 byte 1 filled packet sent to determine link state | ||||||
|  | 	linkResponse = []byte{1, 1, 1, 1} | ||||||
|  | ) | ||||||
|  |  | ||||||
| func newLink(s transport.Socket) *link { | func newLink(s transport.Socket) *link { | ||||||
| 	l := &link{ | 	l := &link{ | ||||||
| 		Socket:        s, | 		Socket:        s, | ||||||
| 		id:            uuid.New().String(), | 		id:            uuid.New().String(), | ||||||
| 		lastKeepAlive: time.Now(), | 		lastKeepAlive: time.Now(), | ||||||
| 		closed:        make(chan bool), |  | ||||||
| 		channels:      make(map[string]time.Time), | 		channels:      make(map[string]time.Time), | ||||||
|  | 		closed:        make(chan bool), | ||||||
|  | 		state:         make(chan *packet, 64), | ||||||
| 		sendQueue:     make(chan *packet, 128), | 		sendQueue:     make(chan *packet, 128), | ||||||
| 		recvQueue:     make(chan *packet, 128), | 		recvQueue:     make(chan *packet, 128), | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
|  | 	// process inbound/outbound packets | ||||||
| 	go l.process() | 	go l.process() | ||||||
| 	go l.expiry() | 	// manage the link state | ||||||
|  | 	go l.manage() | ||||||
|  |  | ||||||
| 	return l | 	return l | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // setRate sets the bits per second rate as a float64 | ||||||
|  | func (l *link) setRate(bits int64, delta time.Duration) { | ||||||
|  | 	// rate of send in bits per nanosecond | ||||||
|  | 	rate := float64(bits) / float64(delta.Nanoseconds()) | ||||||
|  |  | ||||||
|  | 	// default the rate if its zero | ||||||
|  | 	if l.rate == 0 { | ||||||
|  | 		// rate per second | ||||||
|  | 		l.rate = rate * 1e9 | ||||||
|  | 	} else { | ||||||
|  | 		// set new rate per second | ||||||
|  | 		l.rate = 0.8*l.rate + 0.2*(rate*1e9) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // setRTT sets a nanosecond based moving average roundtrip time for the link | ||||||
| func (l *link) setRTT(d time.Duration) { | func (l *link) setRTT(d time.Duration) { | ||||||
| 	l.Lock() | 	l.Lock() | ||||||
| 	defer l.Unlock() | 	defer l.Unlock() | ||||||
| @@ -86,6 +118,33 @@ func (l *link) setRTT(d time.Duration) { | |||||||
| 	l.length = int64(length) | 	l.length = int64(length) | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (l *link) delChannel(ch string) { | ||||||
|  | 	l.Lock() | ||||||
|  | 	delete(l.channels, ch) | ||||||
|  | 	l.Unlock() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l *link) getChannel(ch string) time.Time { | ||||||
|  | 	l.RLock() | ||||||
|  | 	defer l.RUnlock() | ||||||
|  | 	return l.channels[ch] | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l *link) setChannel(channels ...string) { | ||||||
|  | 	l.Lock() | ||||||
|  | 	for _, ch := range channels { | ||||||
|  | 		l.channels[ch] = time.Now() | ||||||
|  | 	} | ||||||
|  | 	l.Unlock() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // set the keepalive time | ||||||
|  | func (l *link) keepalive() { | ||||||
|  | 	l.Lock() | ||||||
|  | 	l.lastKeepAlive = time.Now() | ||||||
|  | 	l.Unlock() | ||||||
|  | } | ||||||
|  |  | ||||||
| // process deals with the send queue | // process deals with the send queue | ||||||
| func (l *link) process() { | func (l *link) process() { | ||||||
| 	// receive messages | 	// receive messages | ||||||
| @@ -101,8 +160,22 @@ func (l *link) process() { | |||||||
|  |  | ||||||
| 			// process new received message | 			// process new received message | ||||||
|  |  | ||||||
|  | 			pk := &packet{message: m, err: err} | ||||||
|  |  | ||||||
|  | 			// this is our link state packet | ||||||
|  | 			if m.Header["Micro-Method"] == "link" { | ||||||
|  | 				// process link state message | ||||||
| 				select { | 				select { | ||||||
| 			case l.recvQueue <- &packet{message: m, err: err}: | 				case l.state <- pk: | ||||||
|  | 				default: | ||||||
|  | 				} | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			// process all messages as is | ||||||
|  |  | ||||||
|  | 			select { | ||||||
|  | 			case l.recvQueue <- pk: | ||||||
| 			case <-l.closed: | 			case <-l.closed: | ||||||
| 				return | 				return | ||||||
| 			} | 			} | ||||||
| @@ -122,15 +195,53 @@ func (l *link) process() { | |||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
| // watches the channel expiry | // manage manages the link state including rtt packets and channel mapping expiry | ||||||
| func (l *link) expiry() { | func (l *link) manage() { | ||||||
|  | 	// tick over every minute to expire and fire rtt packets | ||||||
| 	t := time.NewTicker(time.Minute) | 	t := time.NewTicker(time.Minute) | ||||||
| 	defer t.Stop() | 	defer t.Stop() | ||||||
|  |  | ||||||
|  | 	// used to send link state packets | ||||||
|  | 	send := func(b []byte) error { | ||||||
|  | 		return l.Send(&transport.Message{ | ||||||
|  | 			Header: map[string]string{ | ||||||
|  | 				"Micro-Method": "link", | ||||||
|  | 			}, Body: b, | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// set time now | ||||||
|  | 	now := time.Now() | ||||||
|  |  | ||||||
|  | 	// send the initial rtt request packet | ||||||
|  | 	send(linkRequest) | ||||||
|  |  | ||||||
| 	for { | 	for { | ||||||
| 		select { | 		select { | ||||||
|  | 		// exit if closed | ||||||
| 		case <-l.closed: | 		case <-l.closed: | ||||||
| 			return | 			return | ||||||
|  | 		// process link state rtt packets | ||||||
|  | 		case p := <-l.state: | ||||||
|  | 			if p.err != nil { | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  | 			// check the type of message | ||||||
|  | 			switch { | ||||||
|  | 			case bytes.Compare(p.message.Body, linkRequest) == 0: | ||||||
|  | 				log.Tracef("Link %s received link request %v", l.id, p.message.Body) | ||||||
|  | 				// send response | ||||||
|  | 				if err := send(linkResponse); err != nil { | ||||||
|  | 					l.Lock() | ||||||
|  | 					l.errCount++ | ||||||
|  | 					l.Unlock() | ||||||
|  | 				} | ||||||
|  | 			case bytes.Compare(p.message.Body, linkResponse) == 0: | ||||||
|  | 				// set round trip time | ||||||
|  | 				d := time.Since(now) | ||||||
|  | 				log.Tracef("Link %s received link response in %v", p.message.Body, d) | ||||||
|  | 				l.setRTT(d) | ||||||
|  | 			} | ||||||
| 		case <-t.C: | 		case <-t.C: | ||||||
| 			// drop any channel mappings older than 2 minutes | 			// drop any channel mappings older than 2 minutes | ||||||
| 			var kill []string | 			var kill []string | ||||||
| @@ -155,6 +266,10 @@ func (l *link) expiry() { | |||||||
| 				delete(l.channels, ch) | 				delete(l.channels, ch) | ||||||
| 			} | 			} | ||||||
| 			l.Unlock() | 			l.Unlock() | ||||||
|  |  | ||||||
|  | 			// fire off a link state rtt packet | ||||||
|  | 			now = time.Now() | ||||||
|  | 			send(linkRequest) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| @@ -185,7 +300,6 @@ func (l *link) Delay() int64 { | |||||||
| func (l *link) Rate() float64 { | func (l *link) Rate() float64 { | ||||||
| 	l.RLock() | 	l.RLock() | ||||||
| 	defer l.RUnlock() | 	defer l.RUnlock() | ||||||
|  |  | ||||||
| 	return l.rate | 	return l.rate | ||||||
| } | } | ||||||
|  |  | ||||||
| @@ -194,7 +308,6 @@ func (l *link) Rate() float64 { | |||||||
| func (l *link) Length() int64 { | func (l *link) Length() int64 { | ||||||
| 	l.RLock() | 	l.RLock() | ||||||
| 	defer l.RUnlock() | 	defer l.RUnlock() | ||||||
|  |  | ||||||
| 	return l.length | 	return l.length | ||||||
| } | } | ||||||
|  |  | ||||||
| @@ -278,23 +391,11 @@ func (l *link) Send(m *transport.Message) error { | |||||||
|  |  | ||||||
| 	// calculate based on data | 	// calculate based on data | ||||||
| 	if dataSent > 0 { | 	if dataSent > 0 { | ||||||
| 		// measure time taken |  | ||||||
| 		delta := time.Since(now) |  | ||||||
|  |  | ||||||
| 		// bit sent | 		// bit sent | ||||||
| 		bits := dataSent * 1024 | 		bits := dataSent * 1024 | ||||||
|  |  | ||||||
| 		// rate of send in bits per nanosecond | 		// set the rate | ||||||
| 		rate := float64(bits) / float64(delta.Nanoseconds()) | 		l.setRate(int64(bits), time.Since(now)) | ||||||
|  |  | ||||||
| 		// default the rate if its zero |  | ||||||
| 		if l.rate == 0 { |  | ||||||
| 			// rate per second |  | ||||||
| 			l.rate = rate * 1e9 |  | ||||||
| 		} else { |  | ||||||
| 			// set new rate per second |  | ||||||
| 			l.rate = 0.8*l.rate + 0.2*(rate*1e9) |  | ||||||
| 		} |  | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	return nil | 	return nil | ||||||
| @@ -325,8 +426,8 @@ func (l *link) Recv(m *transport.Message) error { | |||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
| // Status can return connected, closed, error | // State can return connected, closed, error | ||||||
| func (l *link) Status() string { | func (l *link) State() string { | ||||||
| 	select { | 	select { | ||||||
| 	case <-l.closed: | 	case <-l.closed: | ||||||
| 		return "closed" | 		return "closed" | ||||||
|   | |||||||
| @@ -106,6 +106,109 @@ func (s *session) newMessage(typ string) *message { | |||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // waitFor waits for the message type required until the timeout specified | ||||||
|  | func (s *session) waitFor(msgType string, timeout time.Duration) (*message, error) { | ||||||
|  | 	now := time.Now() | ||||||
|  |  | ||||||
|  | 	after := func() time.Duration { | ||||||
|  | 		d := time.Since(now) | ||||||
|  | 		// dial timeout minus time since | ||||||
|  | 		wait := timeout - d | ||||||
|  |  | ||||||
|  | 		if wait < time.Duration(0) { | ||||||
|  | 			return time.Duration(0) | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		return wait | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// wait for the message type | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case msg := <-s.recv: | ||||||
|  | 			// ignore what we don't want | ||||||
|  | 			if msg.typ != msgType { | ||||||
|  | 				log.Debugf("Tunnel received non %s message in waiting for %s", msg.typ, msgType) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  | 			// got the message | ||||||
|  | 			return msg, nil | ||||||
|  | 		case <-time.After(after()): | ||||||
|  | 			return nil, ErrDialTimeout | ||||||
|  | 		case <-s.closed: | ||||||
|  | 			return nil, io.EOF | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Discover attempts to discover the link for a specific channel | ||||||
|  | func (s *session) Discover() error { | ||||||
|  | 	// create a new discovery message for this channel | ||||||
|  | 	msg := s.newMessage("discover") | ||||||
|  | 	msg.mode = Broadcast | ||||||
|  | 	msg.outbound = true | ||||||
|  | 	msg.link = "" | ||||||
|  |  | ||||||
|  | 	// send the discovery message | ||||||
|  | 	s.send <- msg | ||||||
|  |  | ||||||
|  | 	// set time now | ||||||
|  | 	now := time.Now() | ||||||
|  |  | ||||||
|  | 	after := func() time.Duration { | ||||||
|  | 		d := time.Since(now) | ||||||
|  | 		// dial timeout minus time since | ||||||
|  | 		wait := s.timeout - d | ||||||
|  | 		if wait < time.Duration(0) { | ||||||
|  | 			return time.Duration(0) | ||||||
|  | 		} | ||||||
|  | 		return wait | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// wait to hear back about the sent message | ||||||
|  | 	select { | ||||||
|  | 	case <-time.After(after()): | ||||||
|  | 		return ErrDialTimeout | ||||||
|  | 	case err := <-s.errChan: | ||||||
|  | 		if err != nil { | ||||||
|  | 			return err | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	var err error | ||||||
|  |  | ||||||
|  | 	// set a new dialTimeout | ||||||
|  | 	dialTimeout := after() | ||||||
|  |  | ||||||
|  | 	// set a shorter delay for multicast | ||||||
|  | 	if s.mode != Unicast { | ||||||
|  | 		// shorten this | ||||||
|  | 		dialTimeout = time.Millisecond * 500 | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// wait for announce | ||||||
|  | 	_, err = s.waitFor("announce", dialTimeout) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// if its multicast just go ahead because this is best effort | ||||||
|  | 	if s.mode != Unicast { | ||||||
|  | 		s.discovered = true | ||||||
|  | 		s.accepted = true | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// set discovered | ||||||
|  | 	s.discovered = true | ||||||
|  |  | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
| // Open will fire the open message for the session. This is called by the dialler. | // Open will fire the open message for the session. This is called by the dialler. | ||||||
| func (s *session) Open() error { | func (s *session) Open() error { | ||||||
| 	// create a new message | 	// create a new message | ||||||
| @@ -131,21 +234,15 @@ func (s *session) Open() error { | |||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// now wait for the accept | 	// now wait for the accept | ||||||
| 	select { | 	msg, err := s.waitFor("accept", s.timeout) | ||||||
| 	case msg = <-s.recv: | 	if err != nil { | ||||||
| 		if msg.typ != "accept" { | 		return err | ||||||
| 			log.Debugf("Received non accept message in Open %s", msg.typ) |  | ||||||
| 			return errors.New("failed to connect") |  | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// set to accepted | 	// set to accepted | ||||||
| 	s.accepted = true | 	s.accepted = true | ||||||
| 	// set link | 	// set link | ||||||
| 	s.link = msg.link | 	s.link = msg.link | ||||||
| 	case <-time.After(s.timeout): |  | ||||||
| 		return ErrDialTimeout |  | ||||||
| 	case <-s.closed: |  | ||||||
| 		return io.EOF |  | ||||||
| 	} |  | ||||||
|  |  | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|   | |||||||
| @@ -63,8 +63,8 @@ type Link interface { | |||||||
| 	Length() int64 | 	Length() int64 | ||||||
| 	// Current transfer rate as bits per second (lower is better) | 	// Current transfer rate as bits per second (lower is better) | ||||||
| 	Rate() float64 | 	Rate() float64 | ||||||
| 	// Status of the link e.g connected/closed | 	// State of the link e.g connected/closed | ||||||
| 	Status() string | 	State() string | ||||||
| 	// honours transport socket | 	// honours transport socket | ||||||
| 	transport.Socket | 	transport.Socket | ||||||
| } | } | ||||||
|   | |||||||
		Reference in New Issue
	
	Block a user