Move proxy/router
This commit is contained in:
713
router/default.go
Normal file
713
router/default.go
Normal file
@@ -0,0 +1,713 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/micro/go-micro/registry"
|
||||
)
|
||||
|
||||
const (
|
||||
// AdvertiseEventsTick is time interval in which the router advertises route updates
|
||||
AdvertiseEventsTick = 5 * time.Second
|
||||
// AdvertiseTableTick is time interval in which router advertises all routes found in routing table
|
||||
AdvertiseTableTick = 1 * time.Minute
|
||||
// AdvertiseFlushTick is time the yet unconsumed advertisements are flush i.e. discarded
|
||||
AdvertiseFlushTick = 15 * time.Second
|
||||
// AdvertSuppress is advert suppression threshold
|
||||
AdvertSuppress = 2000.0
|
||||
// AdvertRecover is advert recovery threshold
|
||||
AdvertRecover = 750.0
|
||||
// DefaultAdvertTTL is default advertisement TTL
|
||||
DefaultAdvertTTL = 1 * time.Minute
|
||||
// DeletePenalty penalises route deletion
|
||||
DeletePenalty = 1000.0
|
||||
// UpdatePenalty penalises route updates
|
||||
UpdatePenalty = 500.0
|
||||
// PenaltyHalfLife is the time the advert penalty decays to half its value
|
||||
PenaltyHalfLife = 2.0
|
||||
// MaxSuppressTime defines time after which the suppressed advert is deleted
|
||||
MaxSuppressTime = 5 * time.Minute
|
||||
)
|
||||
|
||||
var (
|
||||
// PenaltyDecay is a coefficient which controls the speed the advert penalty decays
|
||||
PenaltyDecay = math.Log(2) / PenaltyHalfLife
|
||||
)
|
||||
|
||||
// router implements default router
|
||||
type router struct {
|
||||
sync.RWMutex
|
||||
// embed the table
|
||||
table *table
|
||||
opts Options
|
||||
status Status
|
||||
exit chan struct{}
|
||||
errChan chan error
|
||||
eventChan chan *Event
|
||||
advertWg *sync.WaitGroup
|
||||
wg *sync.WaitGroup
|
||||
|
||||
// advert subscribers
|
||||
subscribers map[string]chan *Advert
|
||||
}
|
||||
|
||||
// newRouter creates new router and returns it
|
||||
func newRouter(opts ...Option) Router {
|
||||
// get default options
|
||||
options := DefaultOptions()
|
||||
|
||||
// apply requested options
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
r := &router{
|
||||
table: newTable(),
|
||||
opts: options,
|
||||
status: Status{Code: Stopped, Error: nil},
|
||||
advertWg: &sync.WaitGroup{},
|
||||
wg: &sync.WaitGroup{},
|
||||
subscribers: make(map[string]chan *Advert),
|
||||
}
|
||||
|
||||
go r.run()
|
||||
|
||||
return r
|
||||
}
|
||||
|
||||
// Init initializes router with given options
|
||||
func (r *router) Init(opts ...Option) error {
|
||||
for _, o := range opts {
|
||||
o(&r.opts)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Options returns router options
|
||||
func (r *router) Options() Options {
|
||||
return r.opts
|
||||
}
|
||||
|
||||
func (r *router) Table() Table {
|
||||
return r.table
|
||||
}
|
||||
|
||||
// manageRoute applies action on a given route
|
||||
func (r *router) manageRoute(route Route, action string) error {
|
||||
switch action {
|
||||
case "create":
|
||||
if err := r.table.Create(route); err != nil && err != ErrDuplicateRoute {
|
||||
return fmt.Errorf("failed adding route for service %s: %s", route.Service, err)
|
||||
}
|
||||
case "update":
|
||||
if err := r.table.Update(route); err != nil && err != ErrDuplicateRoute {
|
||||
return fmt.Errorf("failed updating route for service %s: %s", route.Service, err)
|
||||
}
|
||||
case "delete":
|
||||
if err := r.table.Delete(route); err != nil && err != ErrRouteNotFound {
|
||||
return fmt.Errorf("failed deleting route for service %s: %s", route.Service, err)
|
||||
}
|
||||
default:
|
||||
return fmt.Errorf("failed to manage route for service %s. Unknown action: %s", route.Service, action)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// manageServiceRoutes applies action to all routes of the service.
|
||||
// It returns error of the action fails with error.
|
||||
func (r *router) manageServiceRoutes(service *registry.Service, action string) error {
|
||||
// action is the routing table action
|
||||
action = strings.ToLower(action)
|
||||
|
||||
// take route action on each service node
|
||||
for _, node := range service.Nodes {
|
||||
route := Route{
|
||||
Service: service.Name,
|
||||
Address: node.Address,
|
||||
Gateway: "",
|
||||
Network: r.opts.Network,
|
||||
Link: DefaultLink,
|
||||
Metric: DefaultLocalMetric,
|
||||
}
|
||||
|
||||
if err := r.manageRoute(route, action); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// manageRegistryRoutes applies action to all routes of each service found in the registry.
|
||||
// It returns error if either the services failed to be listed or the routing table action fails.
|
||||
func (r *router) manageRegistryRoutes(reg registry.Registry, action string) error {
|
||||
services, err := reg.ListServices()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed listing services: %v", err)
|
||||
}
|
||||
|
||||
// add each service node as a separate route
|
||||
for _, service := range services {
|
||||
// get the service to retrieve all its info
|
||||
srvs, err := reg.GetService(service.Name)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
// manage the routes for all returned services
|
||||
for _, srv := range srvs {
|
||||
if err := r.manageServiceRoutes(srv, action); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// watchRegistry watches registry and updates routing table based on the received events.
|
||||
// It returns error if either the registry watcher fails with error or if the routing table update fails.
|
||||
func (r *router) watchRegistry(w registry.Watcher) error {
|
||||
// wait in the background for the router to stop
|
||||
// when the router stops, stop the watcher and exit
|
||||
r.wg.Add(1)
|
||||
|
||||
exit := make(chan bool)
|
||||
|
||||
defer func() {
|
||||
// close the exit channel when the go routine finishes
|
||||
close(exit)
|
||||
r.wg.Done()
|
||||
}()
|
||||
|
||||
go func() {
|
||||
defer w.Stop()
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
case <-exit:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
var watchErr error
|
||||
|
||||
for {
|
||||
res, err := w.Next()
|
||||
if err != nil {
|
||||
if err != registry.ErrWatcherStopped {
|
||||
watchErr = err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
if err := r.manageServiceRoutes(res.Service, res.Action); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return watchErr
|
||||
}
|
||||
|
||||
// watchTable watches routing table entries and either adds or deletes locally registered service to/from network registry
|
||||
// It returns error if the locally registered services either fails to be added/deleted to/from network registry.
|
||||
func (r *router) watchTable(w Watcher) error {
|
||||
// wait in the background for the router to stop
|
||||
// when the router stops, stop the watcher and exit
|
||||
r.wg.Add(1)
|
||||
exit := make(chan bool)
|
||||
|
||||
defer func() {
|
||||
// close the exit channel when the go routine finishes
|
||||
close(exit)
|
||||
r.wg.Done()
|
||||
}()
|
||||
|
||||
go func() {
|
||||
defer w.Stop()
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
case <-exit:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
var watchErr error
|
||||
|
||||
for {
|
||||
event, err := w.Next()
|
||||
if err != nil {
|
||||
if err != ErrWatcherStopped {
|
||||
watchErr = err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
close(r.eventChan)
|
||||
return nil
|
||||
case r.eventChan <- event:
|
||||
}
|
||||
}
|
||||
|
||||
// close event channel on error
|
||||
close(r.eventChan)
|
||||
|
||||
return watchErr
|
||||
}
|
||||
|
||||
// publishAdvert publishes router advert to advert channel
|
||||
// NOTE: this might cease to be a dedicated method in the future
|
||||
func (r *router) publishAdvert(advType AdvertType, events []*Event) {
|
||||
defer r.advertWg.Done()
|
||||
|
||||
a := &Advert{
|
||||
Id: r.opts.Id,
|
||||
Type: advType,
|
||||
TTL: DefaultAdvertTTL,
|
||||
Timestamp: time.Now(),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
r.RLock()
|
||||
for _, sub := range r.subscribers {
|
||||
// check the exit chan first
|
||||
select {
|
||||
case <-r.exit:
|
||||
r.RUnlock()
|
||||
return
|
||||
default:
|
||||
}
|
||||
|
||||
// now send the message
|
||||
select {
|
||||
case sub <- a:
|
||||
default:
|
||||
}
|
||||
}
|
||||
r.RUnlock()
|
||||
}
|
||||
|
||||
// advertiseTable advertises the whole routing table to the network
|
||||
func (r *router) advertiseTable() error {
|
||||
// create table advertisement ticker
|
||||
ticker := time.NewTicker(AdvertiseTableTick)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
// list routing table routes to announce
|
||||
routes, err := r.table.List()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed listing routes: %s", err)
|
||||
}
|
||||
// collect all the added routes before we attempt to add default gateway
|
||||
events := make([]*Event, len(routes))
|
||||
for i, route := range routes {
|
||||
event := &Event{
|
||||
Type: Update,
|
||||
Timestamp: time.Now(),
|
||||
Route: route,
|
||||
}
|
||||
events[i] = event
|
||||
}
|
||||
|
||||
// advertise all routes as Update events to subscribers
|
||||
if len(events) > 0 {
|
||||
r.advertWg.Add(1)
|
||||
go r.publishAdvert(RouteUpdate, events)
|
||||
}
|
||||
case <-r.exit:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// routeAdvert contains a list of route events to be advertised
|
||||
type routeAdvert struct {
|
||||
events []*Event
|
||||
// lastUpdate records the time of the last advert update
|
||||
lastUpdate time.Time
|
||||
// penalty is current advert penalty
|
||||
penalty float64
|
||||
// isSuppressed flags the advert suppression
|
||||
isSuppressed bool
|
||||
// suppressTime records the time interval the advert has been suppressed for
|
||||
suppressTime time.Time
|
||||
}
|
||||
|
||||
// advertiseEvents advertises routing table events
|
||||
// It suppresses unhealthy flapping events and advertises healthy events upstream.
|
||||
func (r *router) advertiseEvents() error {
|
||||
// ticker to periodically scan event for advertising
|
||||
ticker := time.NewTicker(AdvertiseEventsTick)
|
||||
defer ticker.Stop()
|
||||
|
||||
// advertMap is a map of advert events
|
||||
advertMap := make(map[uint64]*routeAdvert)
|
||||
|
||||
// routing table watcher
|
||||
tableWatcher, err := r.Watch()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed creating routing table watcher: %v", err)
|
||||
}
|
||||
|
||||
r.wg.Add(1)
|
||||
go func() {
|
||||
defer r.wg.Done()
|
||||
select {
|
||||
case r.errChan <- r.watchTable(tableWatcher):
|
||||
case <-r.exit:
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
var events []*Event
|
||||
// collect all events which are not flapping
|
||||
for key, advert := range advertMap {
|
||||
// decay the event penalty
|
||||
delta := time.Since(advert.lastUpdate).Seconds()
|
||||
advert.penalty = advert.penalty * math.Exp(-delta*PenaltyDecay)
|
||||
|
||||
// suppress/recover the event based on its penalty level
|
||||
switch {
|
||||
case advert.penalty > AdvertSuppress && !advert.isSuppressed:
|
||||
advert.isSuppressed = true
|
||||
advert.suppressTime = time.Now()
|
||||
case advert.penalty < AdvertRecover && advert.isSuppressed:
|
||||
advert.isSuppressed = false
|
||||
}
|
||||
|
||||
// max suppression time threshold has been reached, delete the advert
|
||||
if advert.isSuppressed {
|
||||
if time.Since(advert.suppressTime) > MaxSuppressTime {
|
||||
delete(advertMap, key)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
if !advert.isSuppressed {
|
||||
for _, event := range advert.events {
|
||||
e := new(Event)
|
||||
*e = *event
|
||||
events = append(events, e)
|
||||
// delete the advert from the advertMap
|
||||
delete(advertMap, key)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// advertise all Update events to subscribers
|
||||
if len(events) > 0 {
|
||||
r.advertWg.Add(1)
|
||||
go r.publishAdvert(RouteUpdate, events)
|
||||
}
|
||||
case e := <-r.eventChan:
|
||||
// if event is nil, continue
|
||||
if e == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// determine the event penalty
|
||||
var penalty float64
|
||||
switch e.Type {
|
||||
case Update:
|
||||
penalty = UpdatePenalty
|
||||
case Delete:
|
||||
penalty = DeletePenalty
|
||||
}
|
||||
|
||||
// check if we have already registered the route
|
||||
// we use the route hash as advertMap key
|
||||
hash := e.Route.Hash()
|
||||
advert, ok := advertMap[hash]
|
||||
if !ok {
|
||||
events := []*Event{e}
|
||||
advert = &routeAdvert{
|
||||
events: events,
|
||||
penalty: penalty,
|
||||
lastUpdate: time.Now(),
|
||||
}
|
||||
advertMap[hash] = advert
|
||||
continue
|
||||
}
|
||||
|
||||
// attempt to squash last two events if possible
|
||||
lastEvent := advert.events[len(advert.events)-1]
|
||||
if lastEvent.Type == e.Type {
|
||||
advert.events[len(advert.events)-1] = e
|
||||
} else {
|
||||
advert.events = append(advert.events, e)
|
||||
}
|
||||
|
||||
// update event penalty and recorded timestamp
|
||||
advert.lastUpdate = time.Now()
|
||||
advert.penalty += penalty
|
||||
case <-r.exit:
|
||||
// first wait for the advertiser to finish
|
||||
r.advertWg.Wait()
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// watchErrors watches router errors and takes appropriate actions
|
||||
func (r *router) watchErrors() {
|
||||
var err error
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
case err = <-r.errChan:
|
||||
}
|
||||
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
if r.status.Code != Stopped {
|
||||
// notify all goroutines to finish
|
||||
close(r.exit)
|
||||
|
||||
// drain the advertise channel only if advertising
|
||||
if r.status.Code == Advertising {
|
||||
// drain the event channel
|
||||
for range r.eventChan {
|
||||
}
|
||||
}
|
||||
|
||||
// mark the router as Stopped and set its Error to nil
|
||||
r.status = Status{Code: Stopped, Error: nil}
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
r.status = Status{Code: Error, Error: err}
|
||||
}
|
||||
}
|
||||
|
||||
// Run runs the router.
|
||||
func (r *router) run() {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
switch r.status.Code {
|
||||
case Stopped, Error:
|
||||
// add all local service routes into the routing table
|
||||
if err := r.manageRegistryRoutes(r.opts.Registry, "create"); err != nil {
|
||||
r.status = Status{Code: Error, Error: fmt.Errorf("failed adding registry routes: %s", err)}
|
||||
return
|
||||
}
|
||||
|
||||
// add default gateway into routing table
|
||||
if r.opts.Gateway != "" {
|
||||
// note, the only non-default value is the gateway
|
||||
route := Route{
|
||||
Service: "*",
|
||||
Address: "*",
|
||||
Gateway: r.opts.Gateway,
|
||||
Network: "*",
|
||||
Metric: DefaultLocalMetric,
|
||||
}
|
||||
if err := r.table.Create(route); err != nil {
|
||||
r.status = Status{Code: Error, Error: fmt.Errorf("failed adding default gateway route: %s", err)}
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// create error and exit channels
|
||||
r.errChan = make(chan error, 1)
|
||||
r.exit = make(chan struct{})
|
||||
|
||||
// registry watcher
|
||||
regWatcher, err := r.opts.Registry.Watch()
|
||||
if err != nil {
|
||||
r.status = Status{Code: Error, Error: fmt.Errorf("failed creating registry watcher: %v", err)}
|
||||
return
|
||||
}
|
||||
|
||||
r.wg.Add(1)
|
||||
go func() {
|
||||
defer r.wg.Done()
|
||||
select {
|
||||
case r.errChan <- r.watchRegistry(regWatcher):
|
||||
case <-r.exit:
|
||||
}
|
||||
}()
|
||||
|
||||
// watch for errors and cleanup
|
||||
r.wg.Add(1)
|
||||
go func() {
|
||||
defer r.wg.Done()
|
||||
r.watchErrors()
|
||||
}()
|
||||
|
||||
// mark router as Running and set its Error to nil
|
||||
r.status = Status{Code: Running, Error: nil}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// Advertise stars advertising the routes to the network and returns the advertisements channel to consume from.
|
||||
// If the router is already advertising it returns the channel to consume from.
|
||||
// It returns error if either the router is not running or if the routing table fails to list the routes to advertise.
|
||||
func (r *router) Advertise() (<-chan *Advert, error) {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
switch r.status.Code {
|
||||
case Advertising:
|
||||
advertChan := make(chan *Advert)
|
||||
r.subscribers[uuid.New().String()] = advertChan
|
||||
return advertChan, nil
|
||||
case Running:
|
||||
// list routing table routes to announce
|
||||
routes, err := r.table.List()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed listing routes: %s", err)
|
||||
}
|
||||
// collect all the added routes before we attempt to add default gateway
|
||||
events := make([]*Event, len(routes))
|
||||
for i, route := range routes {
|
||||
event := &Event{
|
||||
Type: Create,
|
||||
Timestamp: time.Now(),
|
||||
Route: route,
|
||||
}
|
||||
events[i] = event
|
||||
}
|
||||
|
||||
// create event channels
|
||||
r.eventChan = make(chan *Event)
|
||||
|
||||
// advertise your presence
|
||||
r.advertWg.Add(1)
|
||||
go r.publishAdvert(Announce, events)
|
||||
|
||||
r.wg.Add(1)
|
||||
go func() {
|
||||
defer r.wg.Done()
|
||||
select {
|
||||
case r.errChan <- r.advertiseEvents():
|
||||
case <-r.exit:
|
||||
}
|
||||
}()
|
||||
|
||||
r.advertWg.Add(1)
|
||||
go func() {
|
||||
defer r.advertWg.Done()
|
||||
// advertise the whole routing table
|
||||
select {
|
||||
case r.errChan <- r.advertiseTable():
|
||||
case <-r.exit:
|
||||
}
|
||||
}()
|
||||
|
||||
// mark router as Running and set its Error to nil
|
||||
r.status = Status{Code: Advertising, Error: nil}
|
||||
|
||||
// create advert channel
|
||||
advertChan := make(chan *Advert)
|
||||
r.subscribers[uuid.New().String()] = advertChan
|
||||
|
||||
return advertChan, nil
|
||||
case Stopped:
|
||||
return nil, fmt.Errorf("not running")
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("error: %s", r.status.Error)
|
||||
}
|
||||
|
||||
// Process updates the routing table using the advertised values
|
||||
func (r *router) Process(a *Advert) error {
|
||||
// NOTE: event sorting might not be necessary
|
||||
// copy update events intp new slices
|
||||
events := make([]*Event, len(a.Events))
|
||||
copy(events, a.Events)
|
||||
// sort events by timestamp
|
||||
sort.Slice(events, func(i, j int) bool {
|
||||
return events[i].Timestamp.Before(events[j].Timestamp)
|
||||
})
|
||||
|
||||
for _, event := range events {
|
||||
// create a copy of the route
|
||||
route := event.Route
|
||||
action := event.Type
|
||||
if err := r.manageRoute(route, fmt.Sprintf("%s", action)); err != nil {
|
||||
return fmt.Errorf("failed applying action %s to routing table: %s", action, err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *router) Lookup(q Query) ([]Route, error) {
|
||||
return r.table.Query(q)
|
||||
}
|
||||
|
||||
func (r *router) Watch(opts ...WatchOption) (Watcher, error) {
|
||||
return r.table.Watch(opts...)
|
||||
}
|
||||
|
||||
// Status returns router status
|
||||
func (r *router) Status() Status {
|
||||
r.RLock()
|
||||
defer r.RUnlock()
|
||||
|
||||
// make a copy of the status
|
||||
status := r.status
|
||||
|
||||
return status
|
||||
}
|
||||
|
||||
// Stop stops the router
|
||||
func (r *router) Stop() error {
|
||||
r.Lock()
|
||||
// only close the channel if the router is running and/or advertising
|
||||
if r.status.Code == Running || r.status.Code == Advertising {
|
||||
// notify all goroutines to finish
|
||||
close(r.exit)
|
||||
|
||||
// drain the advertise channel only if advertising
|
||||
if r.status.Code == Advertising {
|
||||
// drain the event channel
|
||||
for range r.eventChan {
|
||||
}
|
||||
}
|
||||
|
||||
// close advert subscribers
|
||||
for id, sub := range r.subscribers {
|
||||
// close the channel
|
||||
close(sub)
|
||||
|
||||
// delete the subscriber
|
||||
delete(r.subscribers, id)
|
||||
}
|
||||
|
||||
// mark the router as Stopped and set its Error to nil
|
||||
r.status = Status{Code: Stopped, Error: nil}
|
||||
}
|
||||
r.Unlock()
|
||||
|
||||
// wait for all goroutines to finish
|
||||
r.wg.Wait()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// String prints debugging information about router
|
||||
func (r *router) String() string {
|
||||
return "default"
|
||||
}
|
180
router/handler/router.go
Normal file
180
router/handler/router.go
Normal file
@@ -0,0 +1,180 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/micro/go-micro/errors"
|
||||
"github.com/micro/go-micro/router"
|
||||
pb "github.com/micro/go-micro/router/proto"
|
||||
)
|
||||
|
||||
// Router implements router handler
|
||||
type Router struct {
|
||||
Router router.Router
|
||||
}
|
||||
|
||||
// Lookup looks up routes in the routing table and returns them
|
||||
func (r *Router) Lookup(ctx context.Context, req *pb.LookupRequest, resp *pb.LookupResponse) error {
|
||||
query := router.NewQuery(
|
||||
router.QueryService(req.Query.Service),
|
||||
)
|
||||
|
||||
routes, err := r.Router.Lookup(query)
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to lookup routes: %v", err)
|
||||
}
|
||||
|
||||
var respRoutes []*pb.Route
|
||||
for _, route := range routes {
|
||||
respRoute := &pb.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int64(route.Metric),
|
||||
}
|
||||
respRoutes = append(respRoutes, respRoute)
|
||||
}
|
||||
|
||||
resp.Routes = respRoutes
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *Router) Advertise(ctx context.Context, req *pb.AdvertiseRequest, stream pb.Router_AdvertiseStream) error {
|
||||
advertChan, err := r.Router.Advertise()
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to get adverts: %v", err)
|
||||
}
|
||||
|
||||
for advert := range advertChan {
|
||||
var events []*pb.Event
|
||||
for _, event := range advert.Events {
|
||||
route := &pb.Route{
|
||||
Service: event.Route.Service,
|
||||
Address: event.Route.Address,
|
||||
Gateway: event.Route.Gateway,
|
||||
Network: event.Route.Network,
|
||||
Link: event.Route.Link,
|
||||
Metric: int64(event.Route.Metric),
|
||||
}
|
||||
e := &pb.Event{
|
||||
Type: pb.EventType(event.Type),
|
||||
Timestamp: event.Timestamp.UnixNano(),
|
||||
Route: route,
|
||||
}
|
||||
events = append(events, e)
|
||||
}
|
||||
|
||||
advert := &pb.Advert{
|
||||
Id: advert.Id,
|
||||
Type: pb.AdvertType(advert.Type),
|
||||
Timestamp: advert.Timestamp.UnixNano(),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
// send the advert
|
||||
err := stream.Send(advert)
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "error sending message %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *Router) Process(ctx context.Context, req *pb.Advert, rsp *pb.ProcessResponse) error {
|
||||
events := make([]*router.Event, len(req.Events))
|
||||
for i, event := range req.Events {
|
||||
route := router.Route{
|
||||
Service: event.Route.Service,
|
||||
Address: event.Route.Address,
|
||||
Gateway: event.Route.Gateway,
|
||||
Network: event.Route.Network,
|
||||
Link: event.Route.Link,
|
||||
Metric: int(event.Route.Metric),
|
||||
}
|
||||
|
||||
events[i] = &router.Event{
|
||||
Type: router.EventType(event.Type),
|
||||
Timestamp: time.Unix(0, event.Timestamp),
|
||||
Route: route,
|
||||
}
|
||||
}
|
||||
|
||||
advert := &router.Advert{
|
||||
Id: req.Id,
|
||||
Type: router.AdvertType(req.Type),
|
||||
Timestamp: time.Unix(0, req.Timestamp),
|
||||
TTL: time.Duration(req.Ttl),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
if err := r.Router.Process(advert); err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "error publishing advert: %v", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *Router) Status(ctx context.Context, req *pb.Request, rsp *pb.StatusResponse) error {
|
||||
status := r.Router.Status()
|
||||
|
||||
rsp.Status = &pb.Status{
|
||||
Code: status.Code.String(),
|
||||
}
|
||||
|
||||
if status.Error != nil {
|
||||
rsp.Status.Error = status.Error.Error()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Watch streans routing table events
|
||||
func (r *Router) Watch(ctx context.Context, req *pb.WatchRequest, stream pb.Router_WatchStream) error {
|
||||
watcher, err := r.Router.Watch()
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed creating event watcher: %v", err)
|
||||
}
|
||||
|
||||
defer stream.Close()
|
||||
|
||||
for {
|
||||
event, err := watcher.Next()
|
||||
if err == router.ErrWatcherStopped {
|
||||
break
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "error watching events: %v", err)
|
||||
}
|
||||
|
||||
route := &pb.Route{
|
||||
Service: event.Route.Service,
|
||||
Address: event.Route.Address,
|
||||
Gateway: event.Route.Gateway,
|
||||
Network: event.Route.Network,
|
||||
Link: event.Route.Link,
|
||||
Metric: int64(event.Route.Metric),
|
||||
}
|
||||
|
||||
tableEvent := &pb.Event{
|
||||
Type: pb.EventType(event.Type),
|
||||
Timestamp: event.Timestamp.UnixNano(),
|
||||
Route: route,
|
||||
}
|
||||
|
||||
if err := stream.Send(tableEvent); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
114
router/handler/table.go
Normal file
114
router/handler/table.go
Normal file
@@ -0,0 +1,114 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/micro/go-micro/errors"
|
||||
"github.com/micro/go-micro/router"
|
||||
pb "github.com/micro/go-micro/router/proto"
|
||||
)
|
||||
|
||||
type Table struct {
|
||||
Router router.Router
|
||||
}
|
||||
|
||||
func (t *Table) Create(ctx context.Context, route *pb.Route, resp *pb.CreateResponse) error {
|
||||
err := t.Router.Table().Create(router.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int(route.Metric),
|
||||
})
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to create route: %s", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Table) Update(ctx context.Context, route *pb.Route, resp *pb.UpdateResponse) error {
|
||||
err := t.Router.Table().Update(router.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int(route.Metric),
|
||||
})
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to update route: %s", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Table) Delete(ctx context.Context, route *pb.Route, resp *pb.DeleteResponse) error {
|
||||
err := t.Router.Table().Delete(router.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int(route.Metric),
|
||||
})
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to delete route: %s", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// List returns all routes in the routing table
|
||||
func (t *Table) List(ctx context.Context, req *pb.Request, resp *pb.ListResponse) error {
|
||||
routes, err := t.Router.Table().List()
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to list routes: %s", err)
|
||||
}
|
||||
|
||||
var respRoutes []*pb.Route
|
||||
for _, route := range routes {
|
||||
respRoute := &pb.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int64(route.Metric),
|
||||
}
|
||||
respRoutes = append(respRoutes, respRoute)
|
||||
}
|
||||
|
||||
resp.Routes = respRoutes
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Table) Query(ctx context.Context, req *pb.QueryRequest, resp *pb.QueryResponse) error {
|
||||
query := router.NewQuery(
|
||||
router.QueryService(req.Query.Service),
|
||||
)
|
||||
|
||||
routes, err := t.Router.Table().Query(query)
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.router", "failed to lookup routes: %s", err)
|
||||
}
|
||||
|
||||
var respRoutes []*pb.Route
|
||||
for _, route := range routes {
|
||||
respRoute := &pb.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int64(route.Metric),
|
||||
}
|
||||
respRoutes = append(respRoutes, respRoute)
|
||||
}
|
||||
|
||||
resp.Routes = respRoutes
|
||||
|
||||
return nil
|
||||
}
|
75
router/options.go
Normal file
75
router/options.go
Normal file
@@ -0,0 +1,75 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"github.com/google/uuid"
|
||||
"github.com/micro/go-micro/client"
|
||||
"github.com/micro/go-micro/registry"
|
||||
)
|
||||
|
||||
// Options are router options
|
||||
type Options struct {
|
||||
// Id is router id
|
||||
Id string
|
||||
// Address is router address
|
||||
Address string
|
||||
// Gateway is network gateway
|
||||
Gateway string
|
||||
// Network is network address
|
||||
Network string
|
||||
// Registry is the local registry
|
||||
Registry registry.Registry
|
||||
// Client for calling router
|
||||
Client client.Client
|
||||
}
|
||||
|
||||
// Id sets Router Id
|
||||
func Id(id string) Option {
|
||||
return func(o *Options) {
|
||||
o.Id = id
|
||||
}
|
||||
}
|
||||
|
||||
// Address sets router service address
|
||||
func Address(a string) Option {
|
||||
return func(o *Options) {
|
||||
o.Address = a
|
||||
}
|
||||
}
|
||||
|
||||
// Client to call router service
|
||||
func Client(c client.Client) Option {
|
||||
return func(o *Options) {
|
||||
o.Client = c
|
||||
}
|
||||
}
|
||||
|
||||
// Gateway sets network gateway
|
||||
func Gateway(g string) Option {
|
||||
return func(o *Options) {
|
||||
o.Gateway = g
|
||||
}
|
||||
}
|
||||
|
||||
// Network sets router network
|
||||
func Network(n string) Option {
|
||||
return func(o *Options) {
|
||||
o.Network = n
|
||||
}
|
||||
}
|
||||
|
||||
// Registry sets the local registry
|
||||
func Registry(r registry.Registry) Option {
|
||||
return func(o *Options) {
|
||||
o.Registry = r
|
||||
}
|
||||
}
|
||||
|
||||
// DefaultOptions returns router default options
|
||||
func DefaultOptions() Options {
|
||||
return Options{
|
||||
Id: uuid.New().String(),
|
||||
Address: DefaultAddress,
|
||||
Network: DefaultNetwork,
|
||||
Registry: registry.DefaultRegistry,
|
||||
}
|
||||
}
|
416
router/proto/router.micro.go
Normal file
416
router/proto/router.micro.go
Normal file
@@ -0,0 +1,416 @@
|
||||
// Code generated by protoc-gen-micro. DO NOT EDIT.
|
||||
// source: micro/go-micro/router/proto/router.proto
|
||||
|
||||
package go_micro_router
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
math "math"
|
||||
)
|
||||
|
||||
import (
|
||||
context "context"
|
||||
client "github.com/micro/go-micro/client"
|
||||
server "github.com/micro/go-micro/server"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ context.Context
|
||||
var _ client.Option
|
||||
var _ server.Option
|
||||
|
||||
// Client API for Router service
|
||||
|
||||
type RouterService interface {
|
||||
Lookup(ctx context.Context, in *LookupRequest, opts ...client.CallOption) (*LookupResponse, error)
|
||||
Watch(ctx context.Context, in *WatchRequest, opts ...client.CallOption) (Router_WatchService, error)
|
||||
Advertise(ctx context.Context, in *Request, opts ...client.CallOption) (Router_AdvertiseService, error)
|
||||
Process(ctx context.Context, in *Advert, opts ...client.CallOption) (*ProcessResponse, error)
|
||||
Status(ctx context.Context, in *Request, opts ...client.CallOption) (*StatusResponse, error)
|
||||
}
|
||||
|
||||
type routerService struct {
|
||||
c client.Client
|
||||
name string
|
||||
}
|
||||
|
||||
func NewRouterService(name string, c client.Client) RouterService {
|
||||
if c == nil {
|
||||
c = client.NewClient()
|
||||
}
|
||||
if len(name) == 0 {
|
||||
name = "go.micro.router"
|
||||
}
|
||||
return &routerService{
|
||||
c: c,
|
||||
name: name,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *routerService) Lookup(ctx context.Context, in *LookupRequest, opts ...client.CallOption) (*LookupResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Router.Lookup", in)
|
||||
out := new(LookupResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *routerService) Watch(ctx context.Context, in *WatchRequest, opts ...client.CallOption) (Router_WatchService, error) {
|
||||
req := c.c.NewRequest(c.name, "Router.Watch", &WatchRequest{})
|
||||
stream, err := c.c.Stream(ctx, req, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := stream.Send(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &routerServiceWatch{stream}, nil
|
||||
}
|
||||
|
||||
type Router_WatchService interface {
|
||||
SendMsg(interface{}) error
|
||||
RecvMsg(interface{}) error
|
||||
Close() error
|
||||
Recv() (*Event, error)
|
||||
}
|
||||
|
||||
type routerServiceWatch struct {
|
||||
stream client.Stream
|
||||
}
|
||||
|
||||
func (x *routerServiceWatch) Close() error {
|
||||
return x.stream.Close()
|
||||
}
|
||||
|
||||
func (x *routerServiceWatch) SendMsg(m interface{}) error {
|
||||
return x.stream.Send(m)
|
||||
}
|
||||
|
||||
func (x *routerServiceWatch) RecvMsg(m interface{}) error {
|
||||
return x.stream.Recv(m)
|
||||
}
|
||||
|
||||
func (x *routerServiceWatch) Recv() (*Event, error) {
|
||||
m := new(Event)
|
||||
err := x.stream.Recv(m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *routerService) Advertise(ctx context.Context, in *Request, opts ...client.CallOption) (Router_AdvertiseService, error) {
|
||||
req := c.c.NewRequest(c.name, "Router.Advertise", &Request{})
|
||||
stream, err := c.c.Stream(ctx, req, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := stream.Send(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &routerServiceAdvertise{stream}, nil
|
||||
}
|
||||
|
||||
type Router_AdvertiseService interface {
|
||||
SendMsg(interface{}) error
|
||||
RecvMsg(interface{}) error
|
||||
Close() error
|
||||
Recv() (*Advert, error)
|
||||
}
|
||||
|
||||
type routerServiceAdvertise struct {
|
||||
stream client.Stream
|
||||
}
|
||||
|
||||
func (x *routerServiceAdvertise) Close() error {
|
||||
return x.stream.Close()
|
||||
}
|
||||
|
||||
func (x *routerServiceAdvertise) SendMsg(m interface{}) error {
|
||||
return x.stream.Send(m)
|
||||
}
|
||||
|
||||
func (x *routerServiceAdvertise) RecvMsg(m interface{}) error {
|
||||
return x.stream.Recv(m)
|
||||
}
|
||||
|
||||
func (x *routerServiceAdvertise) Recv() (*Advert, error) {
|
||||
m := new(Advert)
|
||||
err := x.stream.Recv(m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *routerService) Process(ctx context.Context, in *Advert, opts ...client.CallOption) (*ProcessResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Router.Process", in)
|
||||
out := new(ProcessResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *routerService) Status(ctx context.Context, in *Request, opts ...client.CallOption) (*StatusResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Router.Status", in)
|
||||
out := new(StatusResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// Server API for Router service
|
||||
|
||||
type RouterHandler interface {
|
||||
Lookup(context.Context, *LookupRequest, *LookupResponse) error
|
||||
Watch(context.Context, *WatchRequest, Router_WatchStream) error
|
||||
Advertise(context.Context, *Request, Router_AdvertiseStream) error
|
||||
Process(context.Context, *Advert, *ProcessResponse) error
|
||||
Status(context.Context, *Request, *StatusResponse) error
|
||||
}
|
||||
|
||||
func RegisterRouterHandler(s server.Server, hdlr RouterHandler, opts ...server.HandlerOption) error {
|
||||
type router interface {
|
||||
Lookup(ctx context.Context, in *LookupRequest, out *LookupResponse) error
|
||||
Watch(ctx context.Context, stream server.Stream) error
|
||||
Advertise(ctx context.Context, stream server.Stream) error
|
||||
Process(ctx context.Context, in *Advert, out *ProcessResponse) error
|
||||
Status(ctx context.Context, in *Request, out *StatusResponse) error
|
||||
}
|
||||
type Router struct {
|
||||
router
|
||||
}
|
||||
h := &routerHandler{hdlr}
|
||||
return s.Handle(s.NewHandler(&Router{h}, opts...))
|
||||
}
|
||||
|
||||
type routerHandler struct {
|
||||
RouterHandler
|
||||
}
|
||||
|
||||
func (h *routerHandler) Lookup(ctx context.Context, in *LookupRequest, out *LookupResponse) error {
|
||||
return h.RouterHandler.Lookup(ctx, in, out)
|
||||
}
|
||||
|
||||
func (h *routerHandler) Watch(ctx context.Context, stream server.Stream) error {
|
||||
m := new(WatchRequest)
|
||||
if err := stream.Recv(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return h.RouterHandler.Watch(ctx, m, &routerWatchStream{stream})
|
||||
}
|
||||
|
||||
type Router_WatchStream interface {
|
||||
SendMsg(interface{}) error
|
||||
RecvMsg(interface{}) error
|
||||
Close() error
|
||||
Send(*Event) error
|
||||
}
|
||||
|
||||
type routerWatchStream struct {
|
||||
stream server.Stream
|
||||
}
|
||||
|
||||
func (x *routerWatchStream) Close() error {
|
||||
return x.stream.Close()
|
||||
}
|
||||
|
||||
func (x *routerWatchStream) SendMsg(m interface{}) error {
|
||||
return x.stream.Send(m)
|
||||
}
|
||||
|
||||
func (x *routerWatchStream) RecvMsg(m interface{}) error {
|
||||
return x.stream.Recv(m)
|
||||
}
|
||||
|
||||
func (x *routerWatchStream) Send(m *Event) error {
|
||||
return x.stream.Send(m)
|
||||
}
|
||||
|
||||
func (h *routerHandler) Advertise(ctx context.Context, stream server.Stream) error {
|
||||
m := new(Request)
|
||||
if err := stream.Recv(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return h.RouterHandler.Advertise(ctx, m, &routerAdvertiseStream{stream})
|
||||
}
|
||||
|
||||
type Router_AdvertiseStream interface {
|
||||
SendMsg(interface{}) error
|
||||
RecvMsg(interface{}) error
|
||||
Close() error
|
||||
Send(*Advert) error
|
||||
}
|
||||
|
||||
type routerAdvertiseStream struct {
|
||||
stream server.Stream
|
||||
}
|
||||
|
||||
func (x *routerAdvertiseStream) Close() error {
|
||||
return x.stream.Close()
|
||||
}
|
||||
|
||||
func (x *routerAdvertiseStream) SendMsg(m interface{}) error {
|
||||
return x.stream.Send(m)
|
||||
}
|
||||
|
||||
func (x *routerAdvertiseStream) RecvMsg(m interface{}) error {
|
||||
return x.stream.Recv(m)
|
||||
}
|
||||
|
||||
func (x *routerAdvertiseStream) Send(m *Advert) error {
|
||||
return x.stream.Send(m)
|
||||
}
|
||||
|
||||
func (h *routerHandler) Process(ctx context.Context, in *Advert, out *ProcessResponse) error {
|
||||
return h.RouterHandler.Process(ctx, in, out)
|
||||
}
|
||||
|
||||
func (h *routerHandler) Status(ctx context.Context, in *Request, out *StatusResponse) error {
|
||||
return h.RouterHandler.Status(ctx, in, out)
|
||||
}
|
||||
|
||||
// Client API for Table service
|
||||
|
||||
type TableService interface {
|
||||
Create(ctx context.Context, in *Route, opts ...client.CallOption) (*CreateResponse, error)
|
||||
Delete(ctx context.Context, in *Route, opts ...client.CallOption) (*DeleteResponse, error)
|
||||
Update(ctx context.Context, in *Route, opts ...client.CallOption) (*UpdateResponse, error)
|
||||
List(ctx context.Context, in *Request, opts ...client.CallOption) (*ListResponse, error)
|
||||
Query(ctx context.Context, in *QueryRequest, opts ...client.CallOption) (*QueryResponse, error)
|
||||
}
|
||||
|
||||
type tableService struct {
|
||||
c client.Client
|
||||
name string
|
||||
}
|
||||
|
||||
func NewTableService(name string, c client.Client) TableService {
|
||||
if c == nil {
|
||||
c = client.NewClient()
|
||||
}
|
||||
if len(name) == 0 {
|
||||
name = "go.micro.router"
|
||||
}
|
||||
return &tableService{
|
||||
c: c,
|
||||
name: name,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *tableService) Create(ctx context.Context, in *Route, opts ...client.CallOption) (*CreateResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Table.Create", in)
|
||||
out := new(CreateResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *tableService) Delete(ctx context.Context, in *Route, opts ...client.CallOption) (*DeleteResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Table.Delete", in)
|
||||
out := new(DeleteResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *tableService) Update(ctx context.Context, in *Route, opts ...client.CallOption) (*UpdateResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Table.Update", in)
|
||||
out := new(UpdateResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *tableService) List(ctx context.Context, in *Request, opts ...client.CallOption) (*ListResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Table.List", in)
|
||||
out := new(ListResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *tableService) Query(ctx context.Context, in *QueryRequest, opts ...client.CallOption) (*QueryResponse, error) {
|
||||
req := c.c.NewRequest(c.name, "Table.Query", in)
|
||||
out := new(QueryResponse)
|
||||
err := c.c.Call(ctx, req, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// Server API for Table service
|
||||
|
||||
type TableHandler interface {
|
||||
Create(context.Context, *Route, *CreateResponse) error
|
||||
Delete(context.Context, *Route, *DeleteResponse) error
|
||||
Update(context.Context, *Route, *UpdateResponse) error
|
||||
List(context.Context, *Request, *ListResponse) error
|
||||
Query(context.Context, *QueryRequest, *QueryResponse) error
|
||||
}
|
||||
|
||||
func RegisterTableHandler(s server.Server, hdlr TableHandler, opts ...server.HandlerOption) error {
|
||||
type table interface {
|
||||
Create(ctx context.Context, in *Route, out *CreateResponse) error
|
||||
Delete(ctx context.Context, in *Route, out *DeleteResponse) error
|
||||
Update(ctx context.Context, in *Route, out *UpdateResponse) error
|
||||
List(ctx context.Context, in *Request, out *ListResponse) error
|
||||
Query(ctx context.Context, in *QueryRequest, out *QueryResponse) error
|
||||
}
|
||||
type Table struct {
|
||||
table
|
||||
}
|
||||
h := &tableHandler{hdlr}
|
||||
return s.Handle(s.NewHandler(&Table{h}, opts...))
|
||||
}
|
||||
|
||||
type tableHandler struct {
|
||||
TableHandler
|
||||
}
|
||||
|
||||
func (h *tableHandler) Create(ctx context.Context, in *Route, out *CreateResponse) error {
|
||||
return h.TableHandler.Create(ctx, in, out)
|
||||
}
|
||||
|
||||
func (h *tableHandler) Delete(ctx context.Context, in *Route, out *DeleteResponse) error {
|
||||
return h.TableHandler.Delete(ctx, in, out)
|
||||
}
|
||||
|
||||
func (h *tableHandler) Update(ctx context.Context, in *Route, out *UpdateResponse) error {
|
||||
return h.TableHandler.Update(ctx, in, out)
|
||||
}
|
||||
|
||||
func (h *tableHandler) List(ctx context.Context, in *Request, out *ListResponse) error {
|
||||
return h.TableHandler.List(ctx, in, out)
|
||||
}
|
||||
|
||||
func (h *tableHandler) Query(ctx context.Context, in *QueryRequest, out *QueryResponse) error {
|
||||
return h.TableHandler.Query(ctx, in, out)
|
||||
}
|
1364
router/proto/router.pb.go
Normal file
1364
router/proto/router.pb.go
Normal file
File diff suppressed because it is too large
Load Diff
133
router/proto/router.proto
Normal file
133
router/proto/router.proto
Normal file
@@ -0,0 +1,133 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package go.micro.router;
|
||||
|
||||
// Router service is used by the proxy to lookup routes
|
||||
service Router {
|
||||
rpc Lookup(LookupRequest) returns (LookupResponse) {};
|
||||
rpc Watch(WatchRequest) returns (stream Event) {};
|
||||
rpc Advertise(Request) returns (stream Advert) {};
|
||||
rpc Process(Advert) returns (ProcessResponse) {};
|
||||
rpc Status(Request) returns (StatusResponse) {};
|
||||
}
|
||||
|
||||
service Table {
|
||||
rpc Create(Route) returns (CreateResponse) {};
|
||||
rpc Delete(Route) returns (DeleteResponse) {};
|
||||
rpc Update(Route) returns (UpdateResponse) {};
|
||||
rpc List(Request) returns (ListResponse) {};
|
||||
rpc Query(QueryRequest) returns (QueryResponse) {};
|
||||
}
|
||||
|
||||
// Empty request
|
||||
message Request {}
|
||||
|
||||
// ListResponse is returned by List
|
||||
message ListResponse {
|
||||
repeated Route routes = 1;
|
||||
}
|
||||
|
||||
// LookupRequest is made to Lookup
|
||||
message LookupRequest {
|
||||
Query query = 1;
|
||||
}
|
||||
|
||||
// LookupResponse is returned by Lookup
|
||||
message LookupResponse {
|
||||
repeated Route routes = 1;
|
||||
}
|
||||
|
||||
message QueryRequest{
|
||||
Query query = 1;
|
||||
}
|
||||
|
||||
message QueryResponse {
|
||||
repeated Route routes = 1;
|
||||
}
|
||||
|
||||
// WatchRequest is made to Watch Router
|
||||
message WatchRequest {}
|
||||
|
||||
// AdvertType defines the type of advert
|
||||
enum AdvertType {
|
||||
AdvertAnnounce = 0;
|
||||
AdvertUpdate = 1;
|
||||
}
|
||||
|
||||
// Advert is router advertsement streamed by Watch
|
||||
message Advert {
|
||||
// id of the advertising router
|
||||
string id = 1;
|
||||
// type of advertisement
|
||||
AdvertType type = 2;
|
||||
// unix timestamp of the advertisement
|
||||
int64 timestamp = 3;
|
||||
// TTL of the Advert
|
||||
int64 ttl = 4;
|
||||
// events is a list of advertised events
|
||||
repeated Event events = 5;
|
||||
}
|
||||
|
||||
// ProcessResponse is returned by Process
|
||||
message ProcessResponse {}
|
||||
|
||||
// CreateResponse is returned by Create
|
||||
message CreateResponse {}
|
||||
|
||||
// DeleteResponse is returned by Delete
|
||||
message DeleteResponse {}
|
||||
|
||||
// UpdateResponse is returned by Update
|
||||
message UpdateResponse {}
|
||||
|
||||
// EventType defines the type of event
|
||||
enum EventType {
|
||||
Create = 0;
|
||||
Delete = 1;
|
||||
Update = 2;
|
||||
}
|
||||
|
||||
// Event is routing table event
|
||||
message Event {
|
||||
// type of event
|
||||
EventType type = 1;
|
||||
// unix timestamp of event
|
||||
int64 timestamp = 2;
|
||||
// service route
|
||||
Route route = 3;
|
||||
}
|
||||
|
||||
// Query is passed in a LookupRequest
|
||||
message Query {
|
||||
// service to lookup
|
||||
string service = 1;
|
||||
// gateway to lookup
|
||||
string gateway = 2;
|
||||
// network to lookup
|
||||
string network = 3;
|
||||
}
|
||||
|
||||
// Route is a service route
|
||||
message Route {
|
||||
// service for the route
|
||||
string service = 1;
|
||||
// the address that advertise this route
|
||||
string address = 2;
|
||||
// gateway as the next hop
|
||||
string gateway = 3;
|
||||
// the network for this destination
|
||||
string network = 4;
|
||||
// the network link
|
||||
string link = 5;
|
||||
// the metric / score of this route
|
||||
int64 metric = 6;
|
||||
}
|
||||
|
||||
message Status {
|
||||
string code = 1;
|
||||
string error = 2;
|
||||
}
|
||||
|
||||
message StatusResponse {
|
||||
Status status = 1;
|
||||
}
|
74
router/query.go
Normal file
74
router/query.go
Normal file
@@ -0,0 +1,74 @@
|
||||
package router
|
||||
|
||||
// QueryOption sets routing table query options
|
||||
type QueryOption func(*QueryOptions)
|
||||
|
||||
// QueryOptions are routing table query options
|
||||
type QueryOptions struct {
|
||||
// Service is destination service name
|
||||
Service string
|
||||
// Gateway is route gateway
|
||||
Gateway string
|
||||
// Network is network address
|
||||
Network string
|
||||
}
|
||||
|
||||
// QueryService sets destination address
|
||||
func QueryService(s string) QueryOption {
|
||||
return func(o *QueryOptions) {
|
||||
o.Service = s
|
||||
}
|
||||
}
|
||||
|
||||
// QueryGateway sets route gateway
|
||||
func QueryGateway(g string) QueryOption {
|
||||
return func(o *QueryOptions) {
|
||||
o.Gateway = g
|
||||
}
|
||||
}
|
||||
|
||||
// QueryNetwork sets route network address
|
||||
func QueryNetwork(n string) QueryOption {
|
||||
return func(o *QueryOptions) {
|
||||
o.Network = n
|
||||
}
|
||||
}
|
||||
|
||||
// Query is routing table query
|
||||
type Query interface {
|
||||
// Options returns query options
|
||||
Options() QueryOptions
|
||||
}
|
||||
|
||||
// query is a basic implementation of Query
|
||||
type query struct {
|
||||
opts QueryOptions
|
||||
}
|
||||
|
||||
// NewQuery creates new query and returns it
|
||||
func NewQuery(opts ...QueryOption) Query {
|
||||
// default options
|
||||
qopts := QueryOptions{
|
||||
Service: "*",
|
||||
Gateway: "*",
|
||||
Network: "*",
|
||||
}
|
||||
|
||||
for _, o := range opts {
|
||||
o(&qopts)
|
||||
}
|
||||
|
||||
return &query{
|
||||
opts: qopts,
|
||||
}
|
||||
}
|
||||
|
||||
// Options returns query options
|
||||
func (q *query) Options() QueryOptions {
|
||||
return q.opts
|
||||
}
|
||||
|
||||
// String prints routing table query in human readable form
|
||||
func (q query) String() string {
|
||||
return "query"
|
||||
}
|
38
router/route.go
Normal file
38
router/route.go
Normal file
@@ -0,0 +1,38 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"hash/fnv"
|
||||
)
|
||||
|
||||
var (
|
||||
// DefaultLink is default network link
|
||||
DefaultLink = "local"
|
||||
// DefaultLocalMetric is default route cost metric for the local network
|
||||
DefaultLocalMetric = 1
|
||||
// DefaultNetworkMetric is default route cost metric for the micro network
|
||||
DefaultNetworkMetric = 10
|
||||
)
|
||||
|
||||
// Route is network route
|
||||
type Route struct {
|
||||
// Service is destination service name
|
||||
Service string
|
||||
// Address is service node address
|
||||
Address string
|
||||
// Gateway is route gateway
|
||||
Gateway string
|
||||
// Network is network address
|
||||
Network string
|
||||
// Link is network link
|
||||
Link string
|
||||
// Metric is the route cost metric
|
||||
Metric int
|
||||
}
|
||||
|
||||
// Hash returns route hash sum.
|
||||
func (r *Route) Hash() uint64 {
|
||||
h := fnv.New64()
|
||||
h.Reset()
|
||||
h.Write([]byte(r.Service + r.Address + r.Gateway + r.Network + r.Link))
|
||||
return h.Sum64()
|
||||
}
|
24
router/route_test.go
Normal file
24
router/route_test.go
Normal file
@@ -0,0 +1,24 @@
|
||||
package router
|
||||
|
||||
import "testing"
|
||||
|
||||
func TestHash(t *testing.T) {
|
||||
route1 := Route{
|
||||
Service: "dest.svc",
|
||||
Gateway: "dest.gw",
|
||||
Network: "dest.network",
|
||||
Link: "det.link",
|
||||
Metric: 10,
|
||||
}
|
||||
|
||||
// make a copy
|
||||
route2 := route1
|
||||
|
||||
route1Hash := route1.Hash()
|
||||
route2Hash := route2.Hash()
|
||||
|
||||
// we should get the same hash
|
||||
if route1Hash != route2Hash {
|
||||
t.Errorf("identical routes result in different hashes")
|
||||
}
|
||||
}
|
135
router/router.go
Normal file
135
router/router.go
Normal file
@@ -0,0 +1,135 @@
|
||||
// Package router provides a network routing control plane
|
||||
package router
|
||||
|
||||
import (
|
||||
"time"
|
||||
)
|
||||
|
||||
// Router is an interface for a routing control plane
|
||||
type Router interface {
|
||||
// Init initializes the router with options
|
||||
Init(...Option) error
|
||||
// Options returns the router options
|
||||
Options() Options
|
||||
// The routing table
|
||||
Table() Table
|
||||
// Advertise advertises routes to the network
|
||||
Advertise() (<-chan *Advert, error)
|
||||
// Process processes incoming adverts
|
||||
Process(*Advert) error
|
||||
// Lookup queries routes in the routing table
|
||||
Lookup(Query) ([]Route, error)
|
||||
// Watch returns a watcher which tracks updates to the routing table
|
||||
Watch(opts ...WatchOption) (Watcher, error)
|
||||
// Status returns router status
|
||||
Status() Status
|
||||
// Stop stops the router
|
||||
Stop() error
|
||||
// Returns the router implementation
|
||||
String() string
|
||||
}
|
||||
|
||||
type Table interface {
|
||||
// Create new route in the routing table
|
||||
Create(Route) error
|
||||
// Delete deletes existing route from the routing table
|
||||
Delete(Route) error
|
||||
// Update updates route in the routing table
|
||||
Update(Route) error
|
||||
// List returns the list of all routes in the table
|
||||
List() ([]Route, error)
|
||||
// Query queries routes in the routing table
|
||||
Query(Query) ([]Route, error)
|
||||
}
|
||||
|
||||
// Option used by the router
|
||||
type Option func(*Options)
|
||||
|
||||
// StatusCode defines router status
|
||||
type StatusCode int
|
||||
|
||||
const (
|
||||
// Running means the router is up and running
|
||||
Running StatusCode = iota
|
||||
// Advertising means the router is advertising
|
||||
Advertising
|
||||
// Stopped means the router has been stopped
|
||||
Stopped
|
||||
// Error means the router has encountered error
|
||||
Error
|
||||
)
|
||||
|
||||
func (s StatusCode) String() string {
|
||||
switch s {
|
||||
case Running:
|
||||
return "running"
|
||||
case Advertising:
|
||||
return "advertising"
|
||||
case Stopped:
|
||||
return "stopped"
|
||||
case Error:
|
||||
return "error"
|
||||
default:
|
||||
return "unknown"
|
||||
}
|
||||
}
|
||||
|
||||
// Status is router status
|
||||
type Status struct {
|
||||
// Error is router error
|
||||
Error error
|
||||
// Code defines router status
|
||||
Code StatusCode
|
||||
}
|
||||
|
||||
// AdvertType is route advertisement type
|
||||
type AdvertType int
|
||||
|
||||
const (
|
||||
// Announce is advertised when the router announces itself
|
||||
Announce AdvertType = iota
|
||||
// RouteUpdate advertises route updates
|
||||
RouteUpdate
|
||||
)
|
||||
|
||||
// String returns human readable advertisement type
|
||||
func (t AdvertType) String() string {
|
||||
switch t {
|
||||
case Announce:
|
||||
return "announce"
|
||||
case RouteUpdate:
|
||||
return "update"
|
||||
default:
|
||||
return "unknown"
|
||||
}
|
||||
}
|
||||
|
||||
// Advert contains a list of events advertised by the router to the network
|
||||
type Advert struct {
|
||||
// Id is the router Id
|
||||
Id string
|
||||
// Type is type of advert
|
||||
Type AdvertType
|
||||
// Timestamp marks the time when the update is sent
|
||||
Timestamp time.Time
|
||||
// TTL is Advert TTL
|
||||
TTL time.Duration
|
||||
// Events is a list of routing table events to advertise
|
||||
Events []*Event
|
||||
}
|
||||
|
||||
var (
|
||||
// DefaultAddress is default router address
|
||||
DefaultAddress = ":9093"
|
||||
// DefaultName is default router service name
|
||||
DefaultName = "go.micro.router"
|
||||
// DefaultNetwork is default micro network
|
||||
DefaultNetwork = "go.micro"
|
||||
// DefaultRouter is default network router
|
||||
DefaultRouter = NewRouter()
|
||||
)
|
||||
|
||||
// NewRouter creates new Router and returns it
|
||||
func NewRouter(opts ...Option) Router {
|
||||
return newRouter(opts...)
|
||||
}
|
316
router/service/service.go
Normal file
316
router/service/service.go
Normal file
@@ -0,0 +1,316 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/micro/go-micro/client"
|
||||
"github.com/micro/go-micro/router"
|
||||
pb "github.com/micro/go-micro/router/proto"
|
||||
)
|
||||
|
||||
type svc struct {
|
||||
sync.RWMutex
|
||||
opts router.Options
|
||||
callOpts []client.CallOption
|
||||
router pb.RouterService
|
||||
table *table
|
||||
status *router.Status
|
||||
exit chan bool
|
||||
errChan chan error
|
||||
advertChan chan *router.Advert
|
||||
}
|
||||
|
||||
// NewRouter creates new service router and returns it
|
||||
func NewRouter(opts ...router.Option) router.Router {
|
||||
// get default options
|
||||
options := router.DefaultOptions()
|
||||
|
||||
// apply requested options
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
// NOTE: might need some client opts here
|
||||
cli := client.DefaultClient
|
||||
|
||||
// set options client
|
||||
if options.Client != nil {
|
||||
cli = options.Client
|
||||
}
|
||||
|
||||
// NOTE: should we have Client/Service option in router.Options?
|
||||
s := &svc{
|
||||
opts: options,
|
||||
router: pb.NewRouterService(router.DefaultName, cli),
|
||||
}
|
||||
|
||||
// set the router address to call
|
||||
if len(options.Address) > 0 {
|
||||
s.callOpts = []client.CallOption{
|
||||
client.WithAddress(options.Address),
|
||||
}
|
||||
}
|
||||
// set the table
|
||||
s.table = &table{pb.NewTableService(router.DefaultName, cli), s.callOpts}
|
||||
|
||||
return s
|
||||
}
|
||||
|
||||
// Init initializes router with given options
|
||||
func (s *svc) Init(opts ...router.Option) error {
|
||||
for _, o := range opts {
|
||||
o(&s.opts)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Options returns router options
|
||||
func (s *svc) Options() router.Options {
|
||||
return s.opts
|
||||
}
|
||||
|
||||
func (s *svc) Table() router.Table {
|
||||
return s.table
|
||||
}
|
||||
|
||||
func (s *svc) advertiseEvents(advertChan chan *router.Advert, stream pb.Router_AdvertiseService) error {
|
||||
go func() {
|
||||
<-s.exit
|
||||
stream.Close()
|
||||
}()
|
||||
|
||||
var advErr error
|
||||
|
||||
for {
|
||||
resp, err := stream.Recv()
|
||||
if err != nil {
|
||||
if err != io.EOF {
|
||||
advErr = err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
events := make([]*router.Event, len(resp.Events))
|
||||
for i, event := range resp.Events {
|
||||
route := router.Route{
|
||||
Service: event.Route.Service,
|
||||
Address: event.Route.Address,
|
||||
Gateway: event.Route.Gateway,
|
||||
Network: event.Route.Network,
|
||||
Link: event.Route.Link,
|
||||
Metric: int(event.Route.Metric),
|
||||
}
|
||||
|
||||
events[i] = &router.Event{
|
||||
Type: router.EventType(event.Type),
|
||||
Timestamp: time.Unix(0, event.Timestamp),
|
||||
Route: route,
|
||||
}
|
||||
}
|
||||
|
||||
advert := &router.Advert{
|
||||
Id: resp.Id,
|
||||
Type: router.AdvertType(resp.Type),
|
||||
Timestamp: time.Unix(0, resp.Timestamp),
|
||||
TTL: time.Duration(resp.Ttl),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
select {
|
||||
case advertChan <- advert:
|
||||
case <-s.exit:
|
||||
close(advertChan)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// close the channel on exit
|
||||
close(advertChan)
|
||||
|
||||
return advErr
|
||||
}
|
||||
|
||||
// Advertise advertises routes to the network
|
||||
func (s *svc) Advertise() (<-chan *router.Advert, error) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
// get the status
|
||||
status := s.Status()
|
||||
|
||||
switch status.Code {
|
||||
case router.Running, router.Advertising:
|
||||
stream, err := s.router.Advertise(context.Background(), &pb.AdvertiseRequest{}, s.callOpts...)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed getting advert stream: %s", err)
|
||||
}
|
||||
// create advertise and event channels
|
||||
advertChan := make(chan *router.Advert)
|
||||
go s.advertiseEvents(advertChan, stream)
|
||||
return advertChan, nil
|
||||
case router.Stopped:
|
||||
// check if our router is stopped
|
||||
select {
|
||||
case <-s.exit:
|
||||
s.exit = make(chan bool)
|
||||
// call advertise again
|
||||
return s.Advertise()
|
||||
default:
|
||||
return nil, fmt.Errorf("not running")
|
||||
}
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("error: %s", s.status.Error)
|
||||
}
|
||||
|
||||
// Process processes incoming adverts
|
||||
func (s *svc) Process(advert *router.Advert) error {
|
||||
var events []*pb.Event
|
||||
for _, event := range advert.Events {
|
||||
route := &pb.Route{
|
||||
Service: event.Route.Service,
|
||||
Address: event.Route.Address,
|
||||
Gateway: event.Route.Gateway,
|
||||
Network: event.Route.Network,
|
||||
Link: event.Route.Link,
|
||||
Metric: int64(event.Route.Metric),
|
||||
}
|
||||
e := &pb.Event{
|
||||
Type: pb.EventType(event.Type),
|
||||
Timestamp: event.Timestamp.UnixNano(),
|
||||
Route: route,
|
||||
}
|
||||
events = append(events, e)
|
||||
}
|
||||
|
||||
advertReq := &pb.Advert{
|
||||
Id: s.Options().Id,
|
||||
Type: pb.AdvertType(advert.Type),
|
||||
Timestamp: advert.Timestamp.UnixNano(),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
if _, err := s.router.Process(context.Background(), advertReq, s.callOpts...); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Status returns router status
|
||||
func (s *svc) Status() router.Status {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
// check if its stopped
|
||||
select {
|
||||
case <-s.exit:
|
||||
return router.Status{
|
||||
Code: router.Stopped,
|
||||
Error: nil,
|
||||
}
|
||||
default:
|
||||
// don't block
|
||||
}
|
||||
|
||||
// check the remote router
|
||||
rsp, err := s.router.Status(context.Background(), &pb.Request{}, s.callOpts...)
|
||||
if err != nil {
|
||||
return router.Status{
|
||||
Code: router.Error,
|
||||
Error: err,
|
||||
}
|
||||
}
|
||||
|
||||
code := router.Running
|
||||
var serr error
|
||||
|
||||
switch rsp.Status.Code {
|
||||
case "running":
|
||||
code = router.Running
|
||||
case "advertising":
|
||||
code = router.Advertising
|
||||
case "stopped":
|
||||
code = router.Stopped
|
||||
case "error":
|
||||
code = router.Error
|
||||
}
|
||||
|
||||
if len(rsp.Status.Error) > 0 {
|
||||
serr = errors.New(rsp.Status.Error)
|
||||
}
|
||||
|
||||
return router.Status{
|
||||
Code: code,
|
||||
Error: serr,
|
||||
}
|
||||
}
|
||||
|
||||
// Remote router cannot be stopped
|
||||
func (s *svc) Stop() error {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
select {
|
||||
case <-s.exit:
|
||||
return nil
|
||||
default:
|
||||
close(s.exit)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Lookup looks up routes in the routing table and returns them
|
||||
func (s *svc) Lookup(q router.Query) ([]router.Route, error) {
|
||||
// call the router
|
||||
resp, err := s.router.Lookup(context.Background(), &pb.LookupRequest{
|
||||
Query: &pb.Query{
|
||||
Service: q.Options().Service,
|
||||
Gateway: q.Options().Gateway,
|
||||
Network: q.Options().Network,
|
||||
},
|
||||
}, s.callOpts...)
|
||||
|
||||
// errored out
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
routes := make([]router.Route, len(resp.Routes))
|
||||
for i, route := range resp.Routes {
|
||||
routes[i] = router.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int(route.Metric),
|
||||
}
|
||||
}
|
||||
|
||||
return routes, nil
|
||||
}
|
||||
|
||||
// Watch returns a watcher which allows to track updates to the routing table
|
||||
func (s *svc) Watch(opts ...router.WatchOption) (router.Watcher, error) {
|
||||
rsp, err := s.router.Watch(context.Background(), &pb.WatchRequest{}, s.callOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var options router.WatchOptions
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
return newWatcher(rsp, options)
|
||||
}
|
||||
|
||||
// Returns the router implementation
|
||||
func (s *svc) String() string {
|
||||
return "service"
|
||||
}
|
121
router/service/table.go
Normal file
121
router/service/table.go
Normal file
@@ -0,0 +1,121 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/micro/go-micro/client"
|
||||
"github.com/micro/go-micro/router"
|
||||
pb "github.com/micro/go-micro/router/proto"
|
||||
)
|
||||
|
||||
type table struct {
|
||||
table pb.TableService
|
||||
callOpts []client.CallOption
|
||||
}
|
||||
|
||||
// Create new route in the routing table
|
||||
func (t *table) Create(r router.Route) error {
|
||||
route := &pb.Route{
|
||||
Service: r.Service,
|
||||
Address: r.Address,
|
||||
Gateway: r.Gateway,
|
||||
Network: r.Network,
|
||||
Link: r.Link,
|
||||
Metric: int64(r.Metric),
|
||||
}
|
||||
|
||||
if _, err := t.table.Create(context.Background(), route, t.callOpts...); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Delete deletes existing route from the routing table
|
||||
func (t *table) Delete(r router.Route) error {
|
||||
route := &pb.Route{
|
||||
Service: r.Service,
|
||||
Address: r.Address,
|
||||
Gateway: r.Gateway,
|
||||
Network: r.Network,
|
||||
Link: r.Link,
|
||||
Metric: int64(r.Metric),
|
||||
}
|
||||
|
||||
if _, err := t.table.Delete(context.Background(), route, t.callOpts...); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Update updates route in the routing table
|
||||
func (t *table) Update(r router.Route) error {
|
||||
route := &pb.Route{
|
||||
Service: r.Service,
|
||||
Address: r.Address,
|
||||
Gateway: r.Gateway,
|
||||
Network: r.Network,
|
||||
Link: r.Link,
|
||||
Metric: int64(r.Metric),
|
||||
}
|
||||
|
||||
if _, err := t.table.Update(context.Background(), route, t.callOpts...); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// List returns the list of all routes in the table
|
||||
func (t *table) List() ([]router.Route, error) {
|
||||
resp, err := t.table.List(context.Background(), &pb.Request{}, t.callOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
routes := make([]router.Route, len(resp.Routes))
|
||||
for i, route := range resp.Routes {
|
||||
routes[i] = router.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int(route.Metric),
|
||||
}
|
||||
}
|
||||
|
||||
return routes, nil
|
||||
}
|
||||
|
||||
// Lookup looks up routes in the routing table and returns them
|
||||
func (t *table) Query(q router.Query) ([]router.Route, error) {
|
||||
// call the router
|
||||
resp, err := t.table.Query(context.Background(), &pb.QueryRequest{
|
||||
Query: &pb.Query{
|
||||
Service: q.Options().Service,
|
||||
Gateway: q.Options().Gateway,
|
||||
Network: q.Options().Network,
|
||||
},
|
||||
}, t.callOpts...)
|
||||
|
||||
// errored out
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
routes := make([]router.Route, len(resp.Routes))
|
||||
for i, route := range resp.Routes {
|
||||
routes[i] = router.Route{
|
||||
Service: route.Service,
|
||||
Address: route.Address,
|
||||
Gateway: route.Gateway,
|
||||
Network: route.Network,
|
||||
Link: route.Link,
|
||||
Metric: int(route.Metric),
|
||||
}
|
||||
}
|
||||
|
||||
return routes, nil
|
||||
}
|
117
router/service/watcher.go
Normal file
117
router/service/watcher.go
Normal file
@@ -0,0 +1,117 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/micro/go-micro/router"
|
||||
pb "github.com/micro/go-micro/router/proto"
|
||||
)
|
||||
|
||||
type watcher struct {
|
||||
sync.RWMutex
|
||||
opts router.WatchOptions
|
||||
resChan chan *router.Event
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
func newWatcher(rsp pb.Router_WatchService, opts router.WatchOptions) (*watcher, error) {
|
||||
w := &watcher{
|
||||
opts: opts,
|
||||
resChan: make(chan *router.Event),
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-w.done:
|
||||
return
|
||||
default:
|
||||
if err := w.watch(rsp); err != nil {
|
||||
w.Stop()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return w, nil
|
||||
}
|
||||
|
||||
// watchRouter watches router and send events to all registered watchers
|
||||
func (w *watcher) watch(stream pb.Router_WatchService) error {
|
||||
defer stream.Close()
|
||||
|
||||
var watchErr error
|
||||
|
||||
for {
|
||||
resp, err := stream.Recv()
|
||||
if err != nil {
|
||||
if err != io.EOF {
|
||||
watchErr = err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
route := router.Route{
|
||||
Service: resp.Route.Service,
|
||||
Address: resp.Route.Address,
|
||||
Gateway: resp.Route.Gateway,
|
||||
Network: resp.Route.Network,
|
||||
Link: resp.Route.Link,
|
||||
Metric: int(resp.Route.Metric),
|
||||
}
|
||||
|
||||
event := &router.Event{
|
||||
Type: router.EventType(resp.Type),
|
||||
Timestamp: time.Unix(0, resp.Timestamp),
|
||||
Route: route,
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case w.resChan <- event:
|
||||
case <-w.done:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return watchErr
|
||||
}
|
||||
|
||||
// Next is a blocking call that returns watch result
|
||||
func (w *watcher) Next() (*router.Event, error) {
|
||||
for {
|
||||
select {
|
||||
case res := <-w.resChan:
|
||||
switch w.opts.Service {
|
||||
case res.Route.Service, "*":
|
||||
return res, nil
|
||||
default:
|
||||
continue
|
||||
}
|
||||
case <-w.done:
|
||||
return nil, router.ErrWatcherStopped
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Chan returns event channel
|
||||
func (w *watcher) Chan() (<-chan *router.Event, error) {
|
||||
return w.resChan, nil
|
||||
}
|
||||
|
||||
// Stop stops watcher
|
||||
func (w *watcher) Stop() {
|
||||
w.Lock()
|
||||
defer w.Unlock()
|
||||
|
||||
select {
|
||||
case <-w.done:
|
||||
return
|
||||
default:
|
||||
close(w.done)
|
||||
}
|
||||
}
|
203
router/table.go
Normal file
203
router/table.go
Normal file
@@ -0,0 +1,203 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
// table is an in memory routing table
|
||||
type table struct {
|
||||
sync.RWMutex
|
||||
// routes stores service routes
|
||||
routes map[string]map[uint64]Route
|
||||
// watchers stores table watchers
|
||||
watchers map[string]*tableWatcher
|
||||
}
|
||||
|
||||
// newtable creates a new routing table and returns it
|
||||
func newTable(opts ...Option) *table {
|
||||
return &table{
|
||||
routes: make(map[string]map[uint64]Route),
|
||||
watchers: make(map[string]*tableWatcher),
|
||||
}
|
||||
}
|
||||
|
||||
// Create creates new route in the routing table
|
||||
func (t *table) Create(r Route) error {
|
||||
service := r.Service
|
||||
sum := r.Hash()
|
||||
|
||||
t.Lock()
|
||||
defer t.Unlock()
|
||||
|
||||
// check if there are any routes in the table for the route destination
|
||||
if _, ok := t.routes[service]; !ok {
|
||||
t.routes[service] = make(map[uint64]Route)
|
||||
t.routes[service][sum] = r
|
||||
go t.sendEvent(&Event{Type: Create, Timestamp: time.Now(), Route: r})
|
||||
return nil
|
||||
}
|
||||
|
||||
// add new route to the table for the route destination
|
||||
if _, ok := t.routes[service][sum]; !ok {
|
||||
t.routes[service][sum] = r
|
||||
go t.sendEvent(&Event{Type: Create, Timestamp: time.Now(), Route: r})
|
||||
return nil
|
||||
}
|
||||
|
||||
return ErrDuplicateRoute
|
||||
}
|
||||
|
||||
// Delete deletes the route from the routing table
|
||||
func (t *table) Delete(r Route) error {
|
||||
service := r.Service
|
||||
sum := r.Hash()
|
||||
|
||||
t.Lock()
|
||||
defer t.Unlock()
|
||||
|
||||
if _, ok := t.routes[service]; !ok {
|
||||
return ErrRouteNotFound
|
||||
}
|
||||
|
||||
delete(t.routes[service], sum)
|
||||
go t.sendEvent(&Event{Type: Delete, Timestamp: time.Now(), Route: r})
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Update updates routing table with the new route
|
||||
func (t *table) Update(r Route) error {
|
||||
service := r.Service
|
||||
sum := r.Hash()
|
||||
|
||||
t.Lock()
|
||||
defer t.Unlock()
|
||||
|
||||
// check if the route destination has any routes in the table
|
||||
if _, ok := t.routes[service]; !ok {
|
||||
t.routes[service] = make(map[uint64]Route)
|
||||
t.routes[service][sum] = r
|
||||
go t.sendEvent(&Event{Type: Create, Timestamp: time.Now(), Route: r})
|
||||
return nil
|
||||
}
|
||||
|
||||
t.routes[service][sum] = r
|
||||
go t.sendEvent(&Event{Type: Update, Timestamp: time.Now(), Route: r})
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// List returns a list of all routes in the table
|
||||
func (t *table) List() ([]Route, error) {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
|
||||
var routes []Route
|
||||
for _, rmap := range t.routes {
|
||||
for _, route := range rmap {
|
||||
routes = append(routes, route)
|
||||
}
|
||||
}
|
||||
|
||||
return routes, nil
|
||||
}
|
||||
|
||||
// isMatch checks if the route matches given network and router
|
||||
func isMatch(route Route, network, router string) bool {
|
||||
if network == "*" || network == route.Network {
|
||||
if router == "*" || router == route.Gateway {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// findRoutes finds all the routes for given network and router and returns them
|
||||
func findRoutes(routes map[uint64]Route, network, router string) []Route {
|
||||
var results []Route
|
||||
for _, route := range routes {
|
||||
if isMatch(route, network, router) {
|
||||
results = append(results, route)
|
||||
}
|
||||
}
|
||||
return results
|
||||
}
|
||||
|
||||
// Lookup queries routing table and returns all routes that match the lookup query
|
||||
func (t *table) Query(q Query) ([]Route, error) {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
|
||||
if q.Options().Service != "*" {
|
||||
if _, ok := t.routes[q.Options().Service]; !ok {
|
||||
return nil, ErrRouteNotFound
|
||||
}
|
||||
return findRoutes(t.routes[q.Options().Service], q.Options().Network, q.Options().Gateway), nil
|
||||
}
|
||||
|
||||
var results []Route
|
||||
// search through all destinations
|
||||
for _, routes := range t.routes {
|
||||
results = append(results, findRoutes(routes, q.Options().Network, q.Options().Gateway)...)
|
||||
}
|
||||
|
||||
return results, nil
|
||||
}
|
||||
|
||||
// Watch returns routing table entry watcher
|
||||
func (t *table) Watch(opts ...WatchOption) (Watcher, error) {
|
||||
// by default watch everything
|
||||
wopts := WatchOptions{
|
||||
Service: "*",
|
||||
}
|
||||
|
||||
for _, o := range opts {
|
||||
o(&wopts)
|
||||
}
|
||||
|
||||
w := &tableWatcher{
|
||||
id: uuid.New().String(),
|
||||
opts: wopts,
|
||||
resChan: make(chan *Event, 10),
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
|
||||
// when the watcher is stopped delete it
|
||||
go func() {
|
||||
<-w.done
|
||||
t.Lock()
|
||||
delete(t.watchers, w.id)
|
||||
t.Unlock()
|
||||
}()
|
||||
|
||||
// save the watcher
|
||||
t.Lock()
|
||||
t.watchers[w.id] = w
|
||||
t.Unlock()
|
||||
|
||||
return w, nil
|
||||
}
|
||||
|
||||
// sendEvent sends events to all subscribed watchers
|
||||
func (t *table) sendEvent(e *Event) {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
|
||||
for _, w := range t.watchers {
|
||||
select {
|
||||
case w.resChan <- e:
|
||||
case <-w.done:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrRouteNotFound is returned when no route was found in the routing table
|
||||
ErrRouteNotFound = errors.New("route not found")
|
||||
// ErrDuplicateRoute is returned when the route already exists
|
||||
ErrDuplicateRoute = errors.New("duplicate route")
|
||||
)
|
194
router/table_test.go
Normal file
194
router/table_test.go
Normal file
@@ -0,0 +1,194 @@
|
||||
package router
|
||||
|
||||
import "testing"
|
||||
|
||||
func testSetup() (*table, Route) {
|
||||
table := newTable()
|
||||
|
||||
route := Route{
|
||||
Service: "dest.svc",
|
||||
Gateway: "dest.gw",
|
||||
Network: "dest.network",
|
||||
Link: "det.link",
|
||||
Metric: 10,
|
||||
}
|
||||
|
||||
return table, route
|
||||
}
|
||||
|
||||
func TestCreate(t *testing.T) {
|
||||
table, route := testSetup()
|
||||
|
||||
if err := table.Create(route); err != nil {
|
||||
t.Errorf("error adding route: %s", err)
|
||||
}
|
||||
|
||||
// adds new route for the original destination
|
||||
route.Gateway = "dest.gw2"
|
||||
|
||||
if err := table.Create(route); err != nil {
|
||||
t.Errorf("error adding route: %s", err)
|
||||
}
|
||||
|
||||
// adding the same route under Insert policy must error
|
||||
if err := table.Create(route); err != ErrDuplicateRoute {
|
||||
t.Errorf("error adding route. Expected error: %s, found: %s", ErrDuplicateRoute, err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestDelete(t *testing.T) {
|
||||
table, route := testSetup()
|
||||
|
||||
if err := table.Create(route); err != nil {
|
||||
t.Errorf("error adding route: %s", err)
|
||||
}
|
||||
|
||||
// should fail to delete non-existant route
|
||||
prevSvc := route.Service
|
||||
route.Service = "randDest"
|
||||
|
||||
if err := table.Delete(route); err != ErrRouteNotFound {
|
||||
t.Errorf("error deleting route. Expected: %s, found: %s", ErrRouteNotFound, err)
|
||||
}
|
||||
|
||||
// we should be able to delete the existing route
|
||||
route.Service = prevSvc
|
||||
|
||||
if err := table.Delete(route); err != nil {
|
||||
t.Errorf("error deleting route: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestUpdate(t *testing.T) {
|
||||
table, route := testSetup()
|
||||
|
||||
if err := table.Create(route); err != nil {
|
||||
t.Errorf("error adding route: %s", err)
|
||||
}
|
||||
|
||||
// change the metric of the original route
|
||||
route.Metric = 200
|
||||
|
||||
if err := table.Update(route); err != nil {
|
||||
t.Errorf("error updating route: %s", err)
|
||||
}
|
||||
|
||||
// this should add a new route
|
||||
route.Service = "rand.dest"
|
||||
|
||||
if err := table.Update(route); err != nil {
|
||||
t.Errorf("error updating route: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestList(t *testing.T) {
|
||||
table, route := testSetup()
|
||||
|
||||
svc := []string{"one.svc", "two.svc", "three.svc"}
|
||||
|
||||
for i := 0; i < len(svc); i++ {
|
||||
route.Service = svc[i]
|
||||
if err := table.Create(route); err != nil {
|
||||
t.Errorf("error adding route: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
routes, err := table.List()
|
||||
if err != nil {
|
||||
t.Errorf("error listing routes: %s", err)
|
||||
}
|
||||
|
||||
if len(routes) != len(svc) {
|
||||
t.Errorf("incorrect number of routes listed. Expected: %d, found: %d", len(svc), len(routes))
|
||||
}
|
||||
}
|
||||
|
||||
func TestQuery(t *testing.T) {
|
||||
table, route := testSetup()
|
||||
|
||||
svc := []string{"svc1", "svc2", "svc3"}
|
||||
net := []string{"net1", "net2", "net1"}
|
||||
gw := []string{"gw1", "gw2", "gw3"}
|
||||
|
||||
for i := 0; i < len(svc); i++ {
|
||||
route.Service = svc[i]
|
||||
route.Network = net[i]
|
||||
route.Gateway = gw[i]
|
||||
if err := table.Create(route); err != nil {
|
||||
t.Errorf("error adding route: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// return all routes
|
||||
query := NewQuery()
|
||||
|
||||
routes, err := table.Query(query)
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
|
||||
// query particular net
|
||||
query = NewQuery(QueryNetwork("net1"))
|
||||
|
||||
routes, err = table.Query(query)
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
|
||||
if len(routes) != 2 {
|
||||
t.Errorf("incorrect number of routes returned. Expected: %d, found: %d", 2, len(routes))
|
||||
}
|
||||
|
||||
// query particular gateway
|
||||
gateway := "gw1"
|
||||
query = NewQuery(QueryGateway(gateway))
|
||||
|
||||
routes, err = table.Query(query)
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
|
||||
if len(routes) != 1 {
|
||||
t.Errorf("incorrect number of routes returned. Expected: %d, found: %d", 1, len(routes))
|
||||
}
|
||||
|
||||
if routes[0].Gateway != gateway {
|
||||
t.Errorf("incorrect route returned. Expected gateway: %s, found: %s", gateway, routes[0].Gateway)
|
||||
}
|
||||
|
||||
// query particular route
|
||||
network := "net1"
|
||||
query = NewQuery(
|
||||
QueryGateway(gateway),
|
||||
QueryNetwork(network),
|
||||
)
|
||||
|
||||
routes, err = table.Query(query)
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
|
||||
if len(routes) != 1 {
|
||||
t.Errorf("incorrect number of routes returned. Expected: %d, found: %d", 1, len(routes))
|
||||
}
|
||||
|
||||
if routes[0].Gateway != gateway {
|
||||
t.Errorf("incorrect route returned. Expected gateway: %s, found: %s", gateway, routes[0].Gateway)
|
||||
}
|
||||
|
||||
if routes[0].Network != network {
|
||||
t.Errorf("incorrect network returned. Expected network: %s, found: %s", network, routes[0].Network)
|
||||
}
|
||||
|
||||
// bullshit route query
|
||||
query = NewQuery(QueryService("foobar"))
|
||||
|
||||
routes, err = table.Query(query)
|
||||
if err != ErrRouteNotFound {
|
||||
t.Errorf("error looking up routes. Expected: %s, found: %s", ErrRouteNotFound, err)
|
||||
}
|
||||
|
||||
if len(routes) != 0 {
|
||||
t.Errorf("incorrect number of routes returned. Expected: %d, found: %d", 0, len(routes))
|
||||
}
|
||||
}
|
120
router/watcher.go
Normal file
120
router/watcher.go
Normal file
@@ -0,0 +1,120 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
// EventType defines routing table event
|
||||
type EventType int
|
||||
|
||||
const (
|
||||
// Create is emitted when a new route has been created
|
||||
Create EventType = iota
|
||||
// Delete is emitted when an existing route has been deleted
|
||||
Delete
|
||||
// Update is emitted when an existing route has been updated
|
||||
Update
|
||||
)
|
||||
|
||||
// String returns human readable event type
|
||||
func (t EventType) String() string {
|
||||
switch t {
|
||||
case Create:
|
||||
return "create"
|
||||
case Delete:
|
||||
return "delete"
|
||||
case Update:
|
||||
return "update"
|
||||
default:
|
||||
return "unknown"
|
||||
}
|
||||
}
|
||||
|
||||
// Event is returned by a call to Next on the watcher.
|
||||
type Event struct {
|
||||
// Type defines type of event
|
||||
Type EventType
|
||||
// Timestamp is event timestamp
|
||||
Timestamp time.Time
|
||||
// Route is table route
|
||||
Route Route
|
||||
}
|
||||
|
||||
// WatchOption is used to define what routes to watch in the table
|
||||
type WatchOption func(*WatchOptions)
|
||||
|
||||
// Watcher defines routing table watcher interface
|
||||
// Watcher returns updates to the routing table
|
||||
type Watcher interface {
|
||||
// Next is a blocking call that returns watch result
|
||||
Next() (*Event, error)
|
||||
// Chan returns event channel
|
||||
Chan() (<-chan *Event, error)
|
||||
// Stop stops watcher
|
||||
Stop()
|
||||
}
|
||||
|
||||
// WatchOptions are table watcher options
|
||||
type WatchOptions struct {
|
||||
// Service allows to watch specific service routes
|
||||
Service string
|
||||
}
|
||||
|
||||
// WatchService sets what service routes to watch
|
||||
// Service is the microservice name
|
||||
func WatchService(s string) WatchOption {
|
||||
return func(o *WatchOptions) {
|
||||
o.Service = s
|
||||
}
|
||||
}
|
||||
|
||||
type tableWatcher struct {
|
||||
sync.RWMutex
|
||||
id string
|
||||
opts WatchOptions
|
||||
resChan chan *Event
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
// Next returns the next noticed action taken on table
|
||||
// TODO: right now we only allow to watch particular service
|
||||
func (w *tableWatcher) Next() (*Event, error) {
|
||||
for {
|
||||
select {
|
||||
case res := <-w.resChan:
|
||||
switch w.opts.Service {
|
||||
case res.Route.Service, "*":
|
||||
return res, nil
|
||||
default:
|
||||
continue
|
||||
}
|
||||
case <-w.done:
|
||||
return nil, ErrWatcherStopped
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Chan returns watcher events channel
|
||||
func (w *tableWatcher) Chan() (<-chan *Event, error) {
|
||||
return w.resChan, nil
|
||||
}
|
||||
|
||||
// Stop stops routing table watcher
|
||||
func (w *tableWatcher) Stop() {
|
||||
w.Lock()
|
||||
defer w.Unlock()
|
||||
|
||||
select {
|
||||
case <-w.done:
|
||||
return
|
||||
default:
|
||||
close(w.done)
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrWatcherStopped is returned when routing table watcher has been stopped
|
||||
ErrWatcherStopped = errors.New("watcher stopped")
|
||||
)
|
Reference in New Issue
Block a user