v3 refactor (#1868)
* Move to v3 Co-authored-by: Ben Toogood <bentoogood@gmail.com>
This commit is contained in:
@@ -1,722 +0,0 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/micro/go-micro/v2/logger"
|
||||
"github.com/micro/go-micro/v2/registry"
|
||||
)
|
||||
|
||||
var (
|
||||
// AdvertiseEventsTick is time interval in which the router advertises route updates
|
||||
AdvertiseEventsTick = 10 * time.Second
|
||||
// DefaultAdvertTTL is default advertisement TTL
|
||||
DefaultAdvertTTL = 2 * time.Minute
|
||||
)
|
||||
|
||||
// router implements default router
|
||||
type router struct {
|
||||
sync.RWMutex
|
||||
|
||||
running bool
|
||||
table *table
|
||||
options Options
|
||||
exit chan bool
|
||||
eventChan chan *Event
|
||||
|
||||
// advert subscribers
|
||||
sub sync.RWMutex
|
||||
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)
|
||||
}
|
||||
|
||||
// construct the router
|
||||
r := &router{
|
||||
options: options,
|
||||
subscribers: make(map[string]chan *Advert),
|
||||
}
|
||||
|
||||
// create the new table, passing the fetchRoute method in as a fallback if
|
||||
// the table doesn't contain the result for a query.
|
||||
r.table = newTable(r.fetchRoutes)
|
||||
|
||||
// start the router and return
|
||||
r.start()
|
||||
return r
|
||||
}
|
||||
|
||||
// Init initializes router with given options
|
||||
func (r *router) Init(opts ...Option) error {
|
||||
// stop the router before we initialize
|
||||
if err := r.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
for _, o := range opts {
|
||||
o(&r.options)
|
||||
}
|
||||
|
||||
// restart the router
|
||||
if err := r.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Options returns router options
|
||||
func (r *router) Options() Options {
|
||||
r.RLock()
|
||||
defer r.RUnlock()
|
||||
|
||||
options := r.options
|
||||
|
||||
return options
|
||||
}
|
||||
|
||||
// Table returns routing table
|
||||
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 "delete":
|
||||
if err := r.table.Delete(route); err != nil && err != ErrRouteNotFound {
|
||||
return fmt.Errorf("failed deleting route for service %s: %s", route.Service, err)
|
||||
}
|
||||
case "update":
|
||||
if err := r.table.Update(route); err != nil {
|
||||
return fmt.Errorf("failed updating 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) manageRoutes(service *registry.Service, action, network 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: network,
|
||||
Router: r.options.Id,
|
||||
Link: DefaultLink,
|
||||
Metric: DefaultLocalMetric,
|
||||
Metadata: node.Metadata,
|
||||
}
|
||||
|
||||
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(registry.ListDomain(registry.WildcardDomain))
|
||||
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 services domain from metadata. Fallback to wildcard.
|
||||
var domain string
|
||||
if service.Metadata != nil && len(service.Metadata["domain"]) > 0 {
|
||||
domain = service.Metadata["domain"]
|
||||
} else {
|
||||
domain = registry.WildcardDomain
|
||||
}
|
||||
|
||||
// get the service to retrieve all its info
|
||||
srvs, err := reg.GetService(service.Name, registry.GetDomain(domain))
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
// manage the routes for all returned services
|
||||
for _, srv := range srvs {
|
||||
if err := r.manageRoutes(srv, action, domain); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// fetchRoutes retrieves all the routes for a given service and creates them in the routing table
|
||||
func (r *router) fetchRoutes(service string) error {
|
||||
services, err := r.options.Registry.GetService(service, registry.GetDomain(registry.WildcardDomain))
|
||||
if err == registry.ErrNotFound {
|
||||
return nil
|
||||
} else if err != nil {
|
||||
return fmt.Errorf("failed getting services: %v", err)
|
||||
}
|
||||
|
||||
for _, srv := range services {
|
||||
var domain string
|
||||
if srv.Metadata != nil && len(srv.Metadata["domain"]) > 0 {
|
||||
domain = srv.Metadata["domain"]
|
||||
} else {
|
||||
domain = registry.WildcardDomain
|
||||
}
|
||||
|
||||
if err := r.manageRoutes(srv, "create", domain); 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 {
|
||||
exit := make(chan bool)
|
||||
|
||||
defer func() {
|
||||
close(exit)
|
||||
}()
|
||||
|
||||
go func() {
|
||||
defer w.Stop()
|
||||
|
||||
select {
|
||||
case <-exit:
|
||||
return
|
||||
case <-r.exit:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
res, err := w.Next()
|
||||
if err != nil {
|
||||
if err != registry.ErrWatcherStopped {
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
if res.Service == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// get the services domain from metadata. Fallback to wildcard.
|
||||
var domain string
|
||||
if res.Service.Metadata != nil && len(res.Service.Metadata["domain"]) > 0 {
|
||||
domain = res.Service.Metadata["domain"]
|
||||
} else {
|
||||
domain = registry.WildcardDomain
|
||||
}
|
||||
|
||||
if err := r.manageRoutes(res.Service, res.Action, domain); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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 {
|
||||
exit := make(chan bool)
|
||||
|
||||
defer func() {
|
||||
close(exit)
|
||||
}()
|
||||
|
||||
// wait in the background for the router to stop
|
||||
// when the router stops, stop the watcher and exit
|
||||
go func() {
|
||||
defer w.Stop()
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
case <-exit:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
event, err := w.Next()
|
||||
if err != nil {
|
||||
if err != ErrWatcherStopped {
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return nil
|
||||
case r.eventChan <- event:
|
||||
// process event
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// publishAdvert publishes router advert to advert channel
|
||||
func (r *router) publishAdvert(advType AdvertType, events []*Event) {
|
||||
a := &Advert{
|
||||
Id: r.options.Id,
|
||||
Type: advType,
|
||||
TTL: DefaultAdvertTTL,
|
||||
Timestamp: time.Now(),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
r.sub.RLock()
|
||||
for _, sub := range r.subscribers {
|
||||
// now send the message
|
||||
select {
|
||||
case sub <- a:
|
||||
case <-r.exit:
|
||||
r.sub.RUnlock()
|
||||
return
|
||||
}
|
||||
}
|
||||
r.sub.RUnlock()
|
||||
}
|
||||
|
||||
// adverts maintains a map of router adverts
|
||||
type adverts map[uint64]*Event
|
||||
|
||||
// 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()
|
||||
|
||||
// adverts is a map of advert events
|
||||
adverts := make(adverts)
|
||||
|
||||
// routing table watcher
|
||||
w, err := r.Watch()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer w.Stop()
|
||||
|
||||
go func() {
|
||||
var err error
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
default:
|
||||
if w == nil {
|
||||
// routing table watcher
|
||||
w, err = r.Watch()
|
||||
if err != nil {
|
||||
if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
|
||||
logger.Errorf("Error creating watcher: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
if err := r.watchTable(w); err != nil {
|
||||
if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
|
||||
logger.Errorf("Error watching table: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
|
||||
if w != nil {
|
||||
// reset
|
||||
w.Stop()
|
||||
w = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
// If we're not advertising any events then sip processing them entirely
|
||||
if r.options.Advertise == AdvertiseNone {
|
||||
continue
|
||||
}
|
||||
|
||||
var events []*Event
|
||||
|
||||
// collect all events which are not flapping
|
||||
for key, event := range adverts {
|
||||
// if we only advertise local routes skip processing anything not link local
|
||||
if r.options.Advertise == AdvertiseLocal && event.Route.Link != "local" {
|
||||
continue
|
||||
}
|
||||
|
||||
// copy the event and append
|
||||
e := new(Event)
|
||||
// this is ok, because router.Event only contains builtin types
|
||||
// and no references so this creates a deep copy of struct Event
|
||||
*e = *event
|
||||
events = append(events, e)
|
||||
// delete the advert from adverts
|
||||
delete(adverts, key)
|
||||
}
|
||||
|
||||
// advertise events to subscribers
|
||||
if len(events) > 0 {
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router publishing %d events", len(events))
|
||||
}
|
||||
go r.publishAdvert(RouteUpdate, events)
|
||||
}
|
||||
case e := <-r.eventChan:
|
||||
// if event is nil, continue
|
||||
if e == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// If we're not advertising any events then skip processing them entirely
|
||||
if r.options.Advertise == AdvertiseNone {
|
||||
continue
|
||||
}
|
||||
|
||||
// if we only advertise local routes skip processing anything not link local
|
||||
if r.options.Advertise == AdvertiseLocal && e.Route.Link != "local" {
|
||||
continue
|
||||
}
|
||||
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router processing table event %s for service %s %s", e.Type, e.Route.Service, e.Route.Address)
|
||||
}
|
||||
|
||||
// check if we have already registered the route
|
||||
hash := e.Route.Hash()
|
||||
ev, ok := adverts[hash]
|
||||
if !ok {
|
||||
ev = e
|
||||
adverts[hash] = e
|
||||
continue
|
||||
}
|
||||
|
||||
// override the route event only if the previous event was different
|
||||
if ev.Type != e.Type {
|
||||
ev = e
|
||||
}
|
||||
case <-r.exit:
|
||||
if w != nil {
|
||||
w.Stop()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// drain all the events, only called on Stop
|
||||
func (r *router) drain() {
|
||||
for {
|
||||
select {
|
||||
case <-r.eventChan:
|
||||
default:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// start the router. Should be called under lock.
|
||||
func (r *router) start() error {
|
||||
if r.running {
|
||||
return nil
|
||||
}
|
||||
|
||||
if r.options.Prewarm {
|
||||
// add all local service routes into the routing table
|
||||
if err := r.manageRegistryRoutes(r.options.Registry, "create"); err != nil {
|
||||
return fmt.Errorf("failed adding registry routes: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// add default gateway into routing table
|
||||
if r.options.Gateway != "" {
|
||||
// note, the only non-default value is the gateway
|
||||
route := Route{
|
||||
Service: "*",
|
||||
Address: "*",
|
||||
Gateway: r.options.Gateway,
|
||||
Network: "*",
|
||||
Router: r.options.Id,
|
||||
Link: DefaultLink,
|
||||
Metric: DefaultLocalMetric,
|
||||
}
|
||||
if err := r.table.Create(route); err != nil {
|
||||
return fmt.Errorf("failed adding default gateway route: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// create error and exit channels
|
||||
r.exit = make(chan bool)
|
||||
|
||||
// registry watcher
|
||||
w, err := r.options.Registry.Watch(registry.WatchDomain(registry.WildcardDomain))
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed creating registry watcher: %v", err)
|
||||
}
|
||||
|
||||
go func() {
|
||||
var err error
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-r.exit:
|
||||
if w != nil {
|
||||
w.Stop()
|
||||
}
|
||||
return
|
||||
default:
|
||||
if w == nil {
|
||||
w, err = r.options.Registry.Watch()
|
||||
if err != nil {
|
||||
if logger.V(logger.WarnLevel, logger.DefaultLogger) {
|
||||
logger.Warnf("failed creating registry watcher: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
if err := r.watchRegistry(w); err != nil {
|
||||
if logger.V(logger.WarnLevel, logger.DefaultLogger) {
|
||||
logger.Warnf("Error watching the registry: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
|
||||
if w != nil {
|
||||
w.Stop()
|
||||
w = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
r.running = true
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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()
|
||||
|
||||
if !r.running {
|
||||
return nil, errors.New("not running")
|
||||
}
|
||||
|
||||
// we're mutating the subscribers so they need to be locked also
|
||||
r.sub.Lock()
|
||||
defer r.sub.Unlock()
|
||||
|
||||
// already advertising
|
||||
if r.eventChan != nil {
|
||||
advertChan := make(chan *Advert, 128)
|
||||
r.subscribers[uuid.New().String()] = advertChan
|
||||
return advertChan, nil
|
||||
}
|
||||
|
||||
// list all the routes and pack them into even slice to advertise
|
||||
events, err := r.flushRouteEvents(Create)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to flush routes: %s", err)
|
||||
}
|
||||
|
||||
// create event channels
|
||||
r.eventChan = make(chan *Event)
|
||||
|
||||
// create advert channel
|
||||
advertChan := make(chan *Advert, 128)
|
||||
r.subscribers[uuid.New().String()] = advertChan
|
||||
|
||||
// advertise your presence
|
||||
go r.publishAdvert(Announce, events)
|
||||
|
||||
go func() {
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
default:
|
||||
if err := r.advertiseEvents(); err != nil {
|
||||
if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
|
||||
logger.Errorf("Error adveritising events: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return advertChan, nil
|
||||
|
||||
}
|
||||
|
||||
// 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)
|
||||
})
|
||||
|
||||
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
|
||||
logger.Tracef("Router %s processing advert from: %s", r.options.Id, a.Id)
|
||||
}
|
||||
|
||||
for _, event := range events {
|
||||
// skip if the router is the origin of this route
|
||||
if event.Route.Router == r.options.Id {
|
||||
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
|
||||
logger.Tracef("Router skipping processing its own route: %s", r.options.Id)
|
||||
}
|
||||
continue
|
||||
}
|
||||
// create a copy of the route
|
||||
route := event.Route
|
||||
action := event.Type
|
||||
|
||||
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
|
||||
logger.Tracef("Router %s applying %s from router %s for service %s %s", r.options.Id, action, route.Router, route.Service, route.Address)
|
||||
}
|
||||
|
||||
if err := r.manageRoute(route, action.String()); err != nil {
|
||||
return fmt.Errorf("failed applying action %s to routing table: %s", action, err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// flushRouteEvents returns a slice of events, one per each route in the routing table
|
||||
func (r *router) flushRouteEvents(evType EventType) ([]*Event, error) {
|
||||
// get a list of routes for each service in our routing table
|
||||
// for the configured advertising strategy
|
||||
q := []QueryOption{
|
||||
QueryStrategy(r.options.Advertise),
|
||||
}
|
||||
|
||||
routes, err := r.Table().Query(q...)
|
||||
if err != nil && err != ErrRouteNotFound {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router advertising %d routes with strategy %s", len(routes), r.options.Advertise)
|
||||
}
|
||||
|
||||
// build a list of events to advertise
|
||||
events := make([]*Event, len(routes))
|
||||
var i int
|
||||
|
||||
for _, route := range routes {
|
||||
event := &Event{
|
||||
Type: evType,
|
||||
Timestamp: time.Now(),
|
||||
Route: route,
|
||||
}
|
||||
events[i] = event
|
||||
i++
|
||||
}
|
||||
|
||||
return events, nil
|
||||
}
|
||||
|
||||
// Lookup routes in the routing table
|
||||
func (r *router) Lookup(q ...QueryOption) ([]Route, error) {
|
||||
return r.table.Query(q...)
|
||||
}
|
||||
|
||||
// Watch routes
|
||||
func (r *router) Watch(opts ...WatchOption) (Watcher, error) {
|
||||
return r.table.Watch(opts...)
|
||||
}
|
||||
|
||||
// Close the router
|
||||
func (r *router) Close() error {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return nil
|
||||
default:
|
||||
close(r.exit)
|
||||
|
||||
// extract the events
|
||||
r.drain()
|
||||
|
||||
r.sub.Lock()
|
||||
// close advert subscribers
|
||||
for id, sub := range r.subscribers {
|
||||
// close the channel
|
||||
close(sub)
|
||||
// delete the subscriber
|
||||
delete(r.subscribers, id)
|
||||
}
|
||||
r.sub.Unlock()
|
||||
}
|
||||
|
||||
// close and remove event chan
|
||||
if r.eventChan != nil {
|
||||
close(r.eventChan)
|
||||
r.eventChan = nil
|
||||
}
|
||||
|
||||
r.running = false
|
||||
return nil
|
||||
}
|
||||
|
||||
// String prints debugging information about router
|
||||
func (r *router) String() string {
|
||||
return "registry"
|
||||
}
|
@@ -5,7 +5,7 @@ import (
|
||||
"net"
|
||||
"strconv"
|
||||
|
||||
"github.com/micro/go-micro/v2/router"
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
// NewRouter returns an initialized dns router
|
||||
|
@@ -4,7 +4,8 @@ import (
|
||||
"context"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/micro/go-micro/v2/registry"
|
||||
"github.com/micro/go-micro/v3/registry"
|
||||
"github.com/micro/go-micro/v3/registry/mdns"
|
||||
)
|
||||
|
||||
// Options are router options
|
||||
@@ -23,8 +24,8 @@ type Options struct {
|
||||
Advertise Strategy
|
||||
// Context for additional options
|
||||
Context context.Context
|
||||
// Prewarm the route table on router startup
|
||||
Prewarm bool
|
||||
// Precache the route table on router startup
|
||||
Precache bool
|
||||
}
|
||||
|
||||
// Id sets Router Id
|
||||
@@ -69,10 +70,10 @@ func Advertise(a Strategy) Option {
|
||||
}
|
||||
}
|
||||
|
||||
// Prewarm sets whether to prewarm the route table
|
||||
func Prewarm(b bool) Option {
|
||||
// Precache sets whether to precache the route table
|
||||
func Precache(b bool) Option {
|
||||
return func(o *Options) {
|
||||
o.Prewarm = b
|
||||
o.Precache = b
|
||||
}
|
||||
}
|
||||
|
||||
@@ -82,7 +83,7 @@ func DefaultOptions() Options {
|
||||
Id: uuid.New().String(),
|
||||
Address: DefaultAddress,
|
||||
Network: DefaultNetwork,
|
||||
Registry: registry.DefaultRegistry,
|
||||
Registry: mdns.NewRegistry(),
|
||||
Advertise: AdvertiseLocal,
|
||||
Context: context.Background(),
|
||||
}
|
||||
|
@@ -1,8 +1,723 @@
|
||||
package registry
|
||||
|
||||
import "github.com/micro/go-micro/v2/router"
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
// NewRouter returns an initialised registry router
|
||||
func NewRouter(opts ...router.Option) router.Router {
|
||||
return router.NewRouter(opts...)
|
||||
"github.com/google/uuid"
|
||||
"github.com/micro/go-micro/v3/logger"
|
||||
"github.com/micro/go-micro/v3/registry"
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
var (
|
||||
// AdvertiseEventsTick is time interval in which the router advertises route updates
|
||||
AdvertiseEventsTick = 10 * time.Second
|
||||
// DefaultAdvertTTL is default advertisement TTL
|
||||
DefaultAdvertTTL = 2 * time.Minute
|
||||
)
|
||||
|
||||
// rtr implements router interface
|
||||
type rtr struct {
|
||||
sync.RWMutex
|
||||
|
||||
running bool
|
||||
table *table
|
||||
options router.Options
|
||||
exit chan bool
|
||||
eventChan chan *router.Event
|
||||
|
||||
// advert subscribers
|
||||
sub sync.RWMutex
|
||||
subscribers map[string]chan *router.Advert
|
||||
}
|
||||
|
||||
// NewRouter creates new 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)
|
||||
}
|
||||
|
||||
// construct the router
|
||||
r := &rtr{
|
||||
options: options,
|
||||
subscribers: make(map[string]chan *router.Advert),
|
||||
}
|
||||
|
||||
// create the new table, passing the fetchRoute method in as a fallback if
|
||||
// the table doesn't contain the result for a query.
|
||||
r.table = newTable(r.fetchRoutes)
|
||||
return r
|
||||
}
|
||||
|
||||
// Init initializes router with given options
|
||||
func (r *rtr) Init(opts ...router.Option) error {
|
||||
// stop the router before we initialize
|
||||
if err := r.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
for _, o := range opts {
|
||||
o(&r.options)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Options returns router options
|
||||
func (r *rtr) Options() router.Options {
|
||||
r.RLock()
|
||||
defer r.RUnlock()
|
||||
|
||||
options := r.options
|
||||
|
||||
return options
|
||||
}
|
||||
|
||||
// Table returns routing table
|
||||
func (r *rtr) Table() router.Table {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
return r.table
|
||||
}
|
||||
|
||||
// manageRoute applies action on a given route
|
||||
func (r *rtr) manageRoute(route router.Route, action string) error {
|
||||
switch action {
|
||||
case "create":
|
||||
if err := r.table.Create(route); err != nil && err != router.ErrDuplicateRoute {
|
||||
return fmt.Errorf("failed adding route for service %s: %s", route.Service, err)
|
||||
}
|
||||
case "delete":
|
||||
if err := r.table.Delete(route); err != nil && err != router.ErrRouteNotFound {
|
||||
return fmt.Errorf("failed deleting route for service %s: %s", route.Service, err)
|
||||
}
|
||||
case "update":
|
||||
if err := r.table.Update(route); err != nil {
|
||||
return fmt.Errorf("failed updating 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 *rtr) manageRoutes(service *registry.Service, action, network 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 := router.Route{
|
||||
Service: service.Name,
|
||||
Address: node.Address,
|
||||
Gateway: "",
|
||||
Network: network,
|
||||
Router: r.options.Id,
|
||||
Link: router.DefaultLink,
|
||||
Metric: router.DefaultLocalMetric,
|
||||
Metadata: node.Metadata,
|
||||
}
|
||||
|
||||
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 *rtr) manageRegistryRoutes(reg registry.Registry, action string) error {
|
||||
services, err := reg.ListServices(registry.ListDomain(registry.WildcardDomain))
|
||||
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 services domain from metadata. Fallback to wildcard.
|
||||
var domain string
|
||||
if service.Metadata != nil && len(service.Metadata["domain"]) > 0 {
|
||||
domain = service.Metadata["domain"]
|
||||
} else {
|
||||
domain = registry.WildcardDomain
|
||||
}
|
||||
|
||||
// get the service to retrieve all its info
|
||||
srvs, err := reg.GetService(service.Name, registry.GetDomain(domain))
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
// manage the routes for all returned services
|
||||
for _, srv := range srvs {
|
||||
if err := r.manageRoutes(srv, action, domain); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// fetchRoutes retrieves all the routes for a given service and creates them in the routing table
|
||||
func (r *rtr) fetchRoutes(service string) error {
|
||||
services, err := r.options.Registry.GetService(service, registry.GetDomain(registry.WildcardDomain))
|
||||
if err == registry.ErrNotFound {
|
||||
return nil
|
||||
} else if err != nil {
|
||||
return fmt.Errorf("failed getting services: %v", err)
|
||||
}
|
||||
|
||||
for _, srv := range services {
|
||||
var domain string
|
||||
if srv.Metadata != nil && len(srv.Metadata["domain"]) > 0 {
|
||||
domain = srv.Metadata["domain"]
|
||||
} else {
|
||||
domain = registry.WildcardDomain
|
||||
}
|
||||
|
||||
if err := r.manageRoutes(srv, "create", domain); 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 *rtr) watchRegistry(w registry.Watcher) error {
|
||||
exit := make(chan bool)
|
||||
|
||||
defer func() {
|
||||
close(exit)
|
||||
}()
|
||||
|
||||
go func() {
|
||||
defer w.Stop()
|
||||
|
||||
select {
|
||||
case <-exit:
|
||||
return
|
||||
case <-r.exit:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
res, err := w.Next()
|
||||
if err != nil {
|
||||
if err != registry.ErrWatcherStopped {
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
if res.Service == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// get the services domain from metadata. Fallback to wildcard.
|
||||
var domain string
|
||||
if res.Service.Metadata != nil && len(res.Service.Metadata["domain"]) > 0 {
|
||||
domain = res.Service.Metadata["domain"]
|
||||
} else {
|
||||
domain = registry.WildcardDomain
|
||||
}
|
||||
|
||||
if err := r.manageRoutes(res.Service, res.Action, domain); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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 *rtr) watchTable(w router.Watcher) error {
|
||||
exit := make(chan bool)
|
||||
|
||||
defer func() {
|
||||
close(exit)
|
||||
}()
|
||||
|
||||
// wait in the background for the router to stop
|
||||
// when the router stops, stop the watcher and exit
|
||||
go func() {
|
||||
defer w.Stop()
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
case <-exit:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
event, err := w.Next()
|
||||
if err != nil {
|
||||
if err != router.ErrWatcherStopped {
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return nil
|
||||
case r.eventChan <- event:
|
||||
// process event
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// publishAdvert publishes router advert to advert channel
|
||||
func (r *rtr) publishAdvert(advType router.AdvertType, events []*router.Event) {
|
||||
a := &router.Advert{
|
||||
Id: r.options.Id,
|
||||
Type: advType,
|
||||
TTL: DefaultAdvertTTL,
|
||||
Timestamp: time.Now(),
|
||||
Events: events,
|
||||
}
|
||||
|
||||
r.sub.RLock()
|
||||
for _, sub := range r.subscribers {
|
||||
// now send the message
|
||||
select {
|
||||
case sub <- a:
|
||||
case <-r.exit:
|
||||
r.sub.RUnlock()
|
||||
return
|
||||
}
|
||||
}
|
||||
r.sub.RUnlock()
|
||||
}
|
||||
|
||||
// adverts maintains a map of router adverts
|
||||
type adverts map[uint64]*router.Event
|
||||
|
||||
// advertiseEvents advertises routing table events
|
||||
// It suppresses unhealthy flapping events and advertises healthy events upstream.
|
||||
func (r *rtr) advertiseEvents() error {
|
||||
// ticker to periodically scan event for advertising
|
||||
ticker := time.NewTicker(AdvertiseEventsTick)
|
||||
defer ticker.Stop()
|
||||
|
||||
// adverts is a map of advert events
|
||||
adverts := make(adverts)
|
||||
|
||||
// routing table watcher
|
||||
w, err := r.Watch()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer w.Stop()
|
||||
|
||||
go func() {
|
||||
var err error
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
default:
|
||||
if w == nil {
|
||||
// routing table watcher
|
||||
w, err = r.Watch()
|
||||
if err != nil {
|
||||
if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
|
||||
logger.Errorf("Error creating watcher: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
if err := r.watchTable(w); err != nil {
|
||||
if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
|
||||
logger.Errorf("Error watching table: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
|
||||
if w != nil {
|
||||
// reset
|
||||
w.Stop()
|
||||
w = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
// If we're not advertising any events then sip processing them entirely
|
||||
if r.options.Advertise == router.AdvertiseNone {
|
||||
continue
|
||||
}
|
||||
|
||||
var events []*router.Event
|
||||
|
||||
// collect all events which are not flapping
|
||||
for key, event := range adverts {
|
||||
// if we only advertise local routes skip processing anything not link local
|
||||
if r.options.Advertise == router.AdvertiseLocal && event.Route.Link != "local" {
|
||||
continue
|
||||
}
|
||||
|
||||
// copy the event and append
|
||||
e := new(router.Event)
|
||||
// this is ok, because router.Event only contains builtin types
|
||||
// and no references so this creates a deep copy of struct Event
|
||||
*e = *event
|
||||
events = append(events, e)
|
||||
// delete the advert from adverts
|
||||
delete(adverts, key)
|
||||
}
|
||||
|
||||
// advertise events to subscribers
|
||||
if len(events) > 0 {
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router publishing %d events", len(events))
|
||||
}
|
||||
go r.publishAdvert(router.RouteUpdate, events)
|
||||
}
|
||||
case e := <-r.eventChan:
|
||||
// if event is nil, continue
|
||||
if e == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// If we're not advertising any events then skip processing them entirely
|
||||
if r.options.Advertise == router.AdvertiseNone {
|
||||
continue
|
||||
}
|
||||
|
||||
// if we only advertise local routes skip processing anything not link local
|
||||
if r.options.Advertise == router.AdvertiseLocal && e.Route.Link != "local" {
|
||||
continue
|
||||
}
|
||||
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router processing table event %s for service %s %s", e.Type, e.Route.Service, e.Route.Address)
|
||||
}
|
||||
|
||||
// check if we have already registered the route
|
||||
hash := e.Route.Hash()
|
||||
ev, ok := adverts[hash]
|
||||
if !ok {
|
||||
ev = e
|
||||
adverts[hash] = e
|
||||
continue
|
||||
}
|
||||
|
||||
// override the route event only if the previous event was different
|
||||
if ev.Type != e.Type {
|
||||
ev = e
|
||||
}
|
||||
case <-r.exit:
|
||||
if w != nil {
|
||||
w.Stop()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// drain all the events, only called on Stop
|
||||
func (r *rtr) drain() {
|
||||
for {
|
||||
select {
|
||||
case <-r.eventChan:
|
||||
default:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// start the router. Should be called under lock.
|
||||
func (r *rtr) start() error {
|
||||
if r.running {
|
||||
return nil
|
||||
}
|
||||
|
||||
if r.options.Precache {
|
||||
// add all local service routes into the routing table
|
||||
if err := r.manageRegistryRoutes(r.options.Registry, "create"); err != nil {
|
||||
return fmt.Errorf("failed adding registry routes: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// add default gateway into routing table
|
||||
if r.options.Gateway != "" {
|
||||
// note, the only non-default value is the gateway
|
||||
route := router.Route{
|
||||
Service: "*",
|
||||
Address: "*",
|
||||
Gateway: r.options.Gateway,
|
||||
Network: "*",
|
||||
Router: r.options.Id,
|
||||
Link: router.DefaultLink,
|
||||
Metric: router.DefaultLocalMetric,
|
||||
}
|
||||
if err := r.table.Create(route); err != nil {
|
||||
return fmt.Errorf("failed adding default gateway route: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// create error and exit channels
|
||||
r.exit = make(chan bool)
|
||||
|
||||
// registry watcher
|
||||
w, err := r.options.Registry.Watch(registry.WatchDomain(registry.WildcardDomain))
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed creating registry watcher: %v", err)
|
||||
}
|
||||
|
||||
go func() {
|
||||
var err error
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-r.exit:
|
||||
if w != nil {
|
||||
w.Stop()
|
||||
}
|
||||
return
|
||||
default:
|
||||
if w == nil {
|
||||
w, err = r.options.Registry.Watch()
|
||||
if err != nil {
|
||||
if logger.V(logger.WarnLevel, logger.DefaultLogger) {
|
||||
logger.Warnf("failed creating registry watcher: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
if err := r.watchRegistry(w); err != nil {
|
||||
if logger.V(logger.WarnLevel, logger.DefaultLogger) {
|
||||
logger.Warnf("Error watching the registry: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
|
||||
if w != nil {
|
||||
w.Stop()
|
||||
w = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
r.running = true
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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 *rtr) Advertise() (<-chan *router.Advert, error) {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
if r.running {
|
||||
return nil, errors.New("cannot re-advertise, already running")
|
||||
}
|
||||
|
||||
// start the router
|
||||
r.start()
|
||||
|
||||
// we're mutating the subscribers so they need to be locked also
|
||||
r.sub.Lock()
|
||||
defer r.sub.Unlock()
|
||||
|
||||
// already advertising
|
||||
if r.eventChan != nil {
|
||||
advertChan := make(chan *router.Advert, 128)
|
||||
r.subscribers[uuid.New().String()] = advertChan
|
||||
return advertChan, nil
|
||||
}
|
||||
|
||||
// list all the routes and pack them into even slice to advertise
|
||||
events, err := r.flushRouteEvents(router.Create)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to flush routes: %s", err)
|
||||
}
|
||||
|
||||
// create event channels
|
||||
r.eventChan = make(chan *router.Event)
|
||||
|
||||
// create advert channel
|
||||
advertChan := make(chan *router.Advert, 128)
|
||||
r.subscribers[uuid.New().String()] = advertChan
|
||||
|
||||
// advertise your presence
|
||||
go r.publishAdvert(router.Announce, events)
|
||||
|
||||
go func() {
|
||||
select {
|
||||
case <-r.exit:
|
||||
return
|
||||
default:
|
||||
if err := r.advertiseEvents(); err != nil {
|
||||
if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
|
||||
logger.Errorf("Error adveritising events: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return advertChan, nil
|
||||
|
||||
}
|
||||
|
||||
// Process updates the routing table using the advertised values
|
||||
func (r *rtr) Process(a *router.Advert) error {
|
||||
// NOTE: event sorting might not be necessary
|
||||
// copy update events intp new slices
|
||||
events := make([]*router.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)
|
||||
})
|
||||
|
||||
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
|
||||
logger.Tracef("Router %s processing advert from: %s", r.options.Id, a.Id)
|
||||
}
|
||||
|
||||
for _, event := range events {
|
||||
// skip if the router is the origin of this route
|
||||
if event.Route.Router == r.options.Id {
|
||||
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
|
||||
logger.Tracef("Router skipping processing its own route: %s", r.options.Id)
|
||||
}
|
||||
continue
|
||||
}
|
||||
// create a copy of the route
|
||||
route := event.Route
|
||||
action := event.Type
|
||||
|
||||
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
|
||||
logger.Tracef("Router %s applying %s from router %s for service %s %s", r.options.Id, action, route.Router, route.Service, route.Address)
|
||||
}
|
||||
|
||||
if err := r.manageRoute(route, action.String()); err != nil {
|
||||
return fmt.Errorf("failed applying action %s to routing table: %s", action, err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// flushRouteEvents returns a slice of events, one per each route in the routing table
|
||||
func (r *rtr) flushRouteEvents(evType router.EventType) ([]*router.Event, error) {
|
||||
// get a list of routes for each service in our routing table
|
||||
// for the configured advertising strategy
|
||||
q := []router.QueryOption{
|
||||
router.QueryStrategy(r.options.Advertise),
|
||||
}
|
||||
|
||||
routes, err := r.table.Query(q...)
|
||||
if err != nil && err != router.ErrRouteNotFound {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router advertising %d routes with strategy %s", len(routes), r.options.Advertise)
|
||||
}
|
||||
|
||||
// build a list of events to advertise
|
||||
events := make([]*router.Event, len(routes))
|
||||
var i int
|
||||
|
||||
for _, route := range routes {
|
||||
event := &router.Event{
|
||||
Type: evType,
|
||||
Timestamp: time.Now(),
|
||||
Route: route,
|
||||
}
|
||||
events[i] = event
|
||||
i++
|
||||
}
|
||||
|
||||
return events, nil
|
||||
}
|
||||
|
||||
// Lookup routes in the routing table
|
||||
func (r *rtr) Lookup(q ...router.QueryOption) ([]router.Route, error) {
|
||||
return r.Table().Query(q...)
|
||||
}
|
||||
|
||||
// Watch routes
|
||||
func (r *rtr) Watch(opts ...router.WatchOption) (router.Watcher, error) {
|
||||
return r.table.Watch(opts...)
|
||||
}
|
||||
|
||||
// Close the router
|
||||
func (r *rtr) Close() error {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
|
||||
select {
|
||||
case <-r.exit:
|
||||
return nil
|
||||
default:
|
||||
if !r.running {
|
||||
return nil
|
||||
}
|
||||
close(r.exit)
|
||||
|
||||
// extract the events
|
||||
r.drain()
|
||||
|
||||
r.sub.Lock()
|
||||
// close advert subscribers
|
||||
for id, sub := range r.subscribers {
|
||||
// close the channel
|
||||
close(sub)
|
||||
// delete the subscriber
|
||||
delete(r.subscribers, id)
|
||||
}
|
||||
r.sub.Unlock()
|
||||
}
|
||||
|
||||
// close and remove event chan
|
||||
if r.eventChan != nil {
|
||||
close(r.eventChan)
|
||||
r.eventChan = nil
|
||||
}
|
||||
|
||||
r.running = false
|
||||
return nil
|
||||
}
|
||||
|
||||
// String prints debugging information about router
|
||||
func (r *rtr) String() string {
|
||||
return "registry"
|
||||
}
|
||||
|
@@ -1,4 +1,4 @@
|
||||
package router
|
||||
package registry
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
@@ -7,12 +7,13 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/micro/go-micro/v2/registry/memory"
|
||||
"github.com/micro/go-micro/v3/registry/memory"
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
func routerTestSetup() Router {
|
||||
func routerTestSetup() router.Router {
|
||||
r := memory.NewRegistry()
|
||||
return newRouter(Registry(r))
|
||||
return NewRouter(router.Registry(r))
|
||||
}
|
||||
|
||||
func TestRouterClose(t *testing.T) {
|
||||
@@ -50,8 +51,8 @@ func TestRouterAdvertise(t *testing.T) {
|
||||
|
||||
// Generate random unique routes
|
||||
nrRoutes := 5
|
||||
routes := make([]Route, nrRoutes)
|
||||
route := Route{
|
||||
routes := make([]router.Route, nrRoutes)
|
||||
route := router.Route{
|
||||
Service: "dest.svc",
|
||||
Address: "dest.addr",
|
||||
Gateway: "dest.gw",
|
@@ -1,19 +1,12 @@
|
||||
package router
|
||||
package registry
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/micro/go-micro/v2/logger"
|
||||
)
|
||||
|
||||
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")
|
||||
"github.com/micro/go-micro/v3/logger"
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
// table is an in-memory routing table
|
||||
@@ -22,22 +15,22 @@ type table struct {
|
||||
// fetchRoutes for a service
|
||||
fetchRoutes func(string) error
|
||||
// routes stores service routes
|
||||
routes map[string]map[uint64]Route
|
||||
routes map[string]map[uint64]router.Route
|
||||
// watchers stores table watchers
|
||||
watchers map[string]*tableWatcher
|
||||
}
|
||||
|
||||
// newtable creates a new routing table and returns it
|
||||
func newTable(fetchRoutes func(string) error, opts ...Option) *table {
|
||||
func newTable(fetchRoutes func(string) error, opts ...router.Option) *table {
|
||||
return &table{
|
||||
fetchRoutes: fetchRoutes,
|
||||
routes: make(map[string]map[uint64]Route),
|
||||
routes: make(map[string]map[uint64]router.Route),
|
||||
watchers: make(map[string]*tableWatcher),
|
||||
}
|
||||
}
|
||||
|
||||
// sendEvent sends events to all subscribed watchers
|
||||
func (t *table) sendEvent(e *Event) {
|
||||
func (t *table) sendEvent(e *router.Event) {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
|
||||
@@ -56,7 +49,7 @@ func (t *table) sendEvent(e *Event) {
|
||||
}
|
||||
|
||||
// Create creates new route in the routing table
|
||||
func (t *table) Create(r Route) error {
|
||||
func (t *table) Create(r router.Route) error {
|
||||
service := r.Service
|
||||
sum := r.Hash()
|
||||
|
||||
@@ -65,24 +58,24 @@ func (t *table) Create(r Route) error {
|
||||
|
||||
// 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] = make(map[uint64]router.Route)
|
||||
}
|
||||
|
||||
// add new route to the table for the route destination
|
||||
if _, ok := t.routes[service][sum]; !ok {
|
||||
t.routes[service][sum] = r
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router emitting %s for route: %s", Create, r.Address)
|
||||
logger.Debugf("Router emitting %s for route: %s", router.Create, r.Address)
|
||||
}
|
||||
go t.sendEvent(&Event{Type: Create, Timestamp: time.Now(), Route: r})
|
||||
go t.sendEvent(&router.Event{Type: router.Create, Timestamp: time.Now(), Route: r})
|
||||
return nil
|
||||
}
|
||||
|
||||
return ErrDuplicateRoute
|
||||
return router.ErrDuplicateRoute
|
||||
}
|
||||
|
||||
// Delete deletes the route from the routing table
|
||||
func (t *table) Delete(r Route) error {
|
||||
func (t *table) Delete(r router.Route) error {
|
||||
service := r.Service
|
||||
sum := r.Hash()
|
||||
|
||||
@@ -90,11 +83,11 @@ func (t *table) Delete(r Route) error {
|
||||
defer t.Unlock()
|
||||
|
||||
if _, ok := t.routes[service]; !ok {
|
||||
return ErrRouteNotFound
|
||||
return router.ErrRouteNotFound
|
||||
}
|
||||
|
||||
if _, ok := t.routes[service][sum]; !ok {
|
||||
return ErrRouteNotFound
|
||||
return router.ErrRouteNotFound
|
||||
}
|
||||
|
||||
delete(t.routes[service], sum)
|
||||
@@ -102,15 +95,15 @@ func (t *table) Delete(r Route) error {
|
||||
delete(t.routes, service)
|
||||
}
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router emitting %s for route: %s", Delete, r.Address)
|
||||
logger.Debugf("Router emitting %s for route: %s", router.Delete, r.Address)
|
||||
}
|
||||
go t.sendEvent(&Event{Type: Delete, Timestamp: time.Now(), Route: r})
|
||||
go t.sendEvent(&router.Event{Type: router.Delete, Timestamp: time.Now(), Route: r})
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Update updates routing table with the new route
|
||||
func (t *table) Update(r Route) error {
|
||||
func (t *table) Update(r router.Route) error {
|
||||
service := r.Service
|
||||
sum := r.Hash()
|
||||
|
||||
@@ -119,15 +112,15 @@ func (t *table) Update(r Route) error {
|
||||
|
||||
// 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] = make(map[uint64]router.Route)
|
||||
}
|
||||
|
||||
if _, ok := t.routes[service][sum]; !ok {
|
||||
t.routes[service][sum] = r
|
||||
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
|
||||
logger.Debugf("Router emitting %s for route: %s", Update, r.Address)
|
||||
logger.Debugf("Router emitting %s for route: %s", router.Update, r.Address)
|
||||
}
|
||||
go t.sendEvent(&Event{Type: Update, Timestamp: time.Now(), Route: r})
|
||||
go t.sendEvent(&router.Event{Type: router.Update, Timestamp: time.Now(), Route: r})
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -138,11 +131,11 @@ func (t *table) Update(r Route) error {
|
||||
}
|
||||
|
||||
// List returns a list of all routes in the table
|
||||
func (t *table) List() ([]Route, error) {
|
||||
func (t *table) List() ([]router.Route, error) {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
|
||||
var routes []Route
|
||||
var routes []router.Route
|
||||
for _, rmap := range t.routes {
|
||||
for _, route := range rmap {
|
||||
routes = append(routes, route)
|
||||
@@ -153,7 +146,7 @@ func (t *table) List() ([]Route, error) {
|
||||
}
|
||||
|
||||
// isMatch checks if the route matches given query options
|
||||
func isMatch(route Route, address, gateway, network, router string, strategy Strategy) bool {
|
||||
func isMatch(route router.Route, address, gateway, network, rtr string, strategy router.Strategy) bool {
|
||||
// matches the values provided
|
||||
match := func(a, b string) bool {
|
||||
if a == "*" || b == "*" || a == b {
|
||||
@@ -171,7 +164,7 @@ func isMatch(route Route, address, gateway, network, router string, strategy Str
|
||||
// by default assume we are querying all routes
|
||||
link := "*"
|
||||
// if AdvertiseLocal change the link query accordingly
|
||||
if strategy == AdvertiseLocal {
|
||||
if strategy == router.AdvertiseLocal {
|
||||
link = "local"
|
||||
}
|
||||
|
||||
@@ -179,7 +172,7 @@ func isMatch(route Route, address, gateway, network, router string, strategy Str
|
||||
values := []compare{
|
||||
{gateway, route.Gateway},
|
||||
{network, route.Network},
|
||||
{router, route.Router},
|
||||
{rtr, route.Router},
|
||||
{address, route.Address},
|
||||
{link, route.Link},
|
||||
}
|
||||
@@ -195,12 +188,12 @@ func isMatch(route Route, address, gateway, network, router string, strategy Str
|
||||
}
|
||||
|
||||
// findRoutes finds all the routes for given network and router and returns them
|
||||
func findRoutes(routes map[uint64]Route, address, gateway, network, router string, strategy Strategy) []Route {
|
||||
func findRoutes(routes map[uint64]router.Route, address, gateway, network, rtr string, strategy router.Strategy) []router.Route {
|
||||
// routeMap stores the routes we're going to advertise
|
||||
routeMap := make(map[string][]Route)
|
||||
routeMap := make(map[string][]router.Route)
|
||||
|
||||
for _, route := range routes {
|
||||
if isMatch(route, address, gateway, network, router, strategy) {
|
||||
if isMatch(route, address, gateway, network, rtr, strategy) {
|
||||
// add matchihg route to the routeMap
|
||||
routeKey := route.Service + "@" + route.Network
|
||||
// append the first found route to routeMap
|
||||
@@ -211,13 +204,13 @@ func findRoutes(routes map[uint64]Route, address, gateway, network, router strin
|
||||
}
|
||||
|
||||
// if AdvertiseAll, keep appending
|
||||
if strategy == AdvertiseAll || strategy == AdvertiseLocal {
|
||||
if strategy == router.AdvertiseAll || strategy == router.AdvertiseLocal {
|
||||
routeMap[routeKey] = append(routeMap[routeKey], route)
|
||||
continue
|
||||
}
|
||||
|
||||
// now we're going to find the best routes
|
||||
if strategy == AdvertiseBest {
|
||||
if strategy == router.AdvertiseBest {
|
||||
// if the current optimal route metric is higher than routing table route, replace it
|
||||
if len(routeMap[routeKey]) > 0 {
|
||||
// NOTE: we know that when AdvertiseBest is set, we only ever have one item in current
|
||||
@@ -230,7 +223,7 @@ func findRoutes(routes map[uint64]Route, address, gateway, network, router strin
|
||||
}
|
||||
}
|
||||
|
||||
var results []Route
|
||||
var results []router.Route
|
||||
for _, route := range routeMap {
|
||||
results = append(results, route...)
|
||||
}
|
||||
@@ -239,20 +232,20 @@ func findRoutes(routes map[uint64]Route, address, gateway, network, router strin
|
||||
}
|
||||
|
||||
// Lookup queries routing table and returns all routes that match the lookup query
|
||||
func (t *table) Query(q ...QueryOption) ([]Route, error) {
|
||||
func (t *table) Query(q ...router.QueryOption) ([]router.Route, error) {
|
||||
// create new query options
|
||||
opts := NewQuery(q...)
|
||||
opts := router.NewQuery(q...)
|
||||
|
||||
// create a cwslicelist of query results
|
||||
results := make([]Route, 0, len(t.routes))
|
||||
results := make([]router.Route, 0, len(t.routes))
|
||||
|
||||
// if No routes are queried, return early
|
||||
if opts.Strategy == AdvertiseNone {
|
||||
if opts.Strategy == router.AdvertiseNone {
|
||||
return results, nil
|
||||
}
|
||||
|
||||
// readAndFilter routes for this service under read lock.
|
||||
readAndFilter := func() ([]Route, bool) {
|
||||
readAndFilter := func() ([]router.Route, bool) {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
|
||||
@@ -280,7 +273,7 @@ func (t *table) Query(q ...QueryOption) ([]Route, error) {
|
||||
return routes, nil
|
||||
}
|
||||
|
||||
return nil, ErrRouteNotFound
|
||||
return nil, router.ErrRouteNotFound
|
||||
}
|
||||
|
||||
// search through all destinations
|
||||
@@ -294,9 +287,9 @@ func (t *table) Query(q ...QueryOption) ([]Route, error) {
|
||||
}
|
||||
|
||||
// Watch returns routing table entry watcher
|
||||
func (t *table) Watch(opts ...WatchOption) (Watcher, error) {
|
||||
func (t *table) Watch(opts ...router.WatchOption) (router.Watcher, error) {
|
||||
// by default watch everything
|
||||
wopts := WatchOptions{
|
||||
wopts := router.WatchOptions{
|
||||
Service: "*",
|
||||
}
|
||||
|
||||
@@ -307,7 +300,7 @@ func (t *table) Watch(opts ...WatchOption) (Watcher, error) {
|
||||
w := &tableWatcher{
|
||||
id: uuid.New().String(),
|
||||
opts: wopts,
|
||||
resChan: make(chan *Event, 10),
|
||||
resChan: make(chan *router.Event, 10),
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
|
@@ -1,15 +1,17 @@
|
||||
package router
|
||||
package registry
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
func testSetup() (*table, Route) {
|
||||
router := newRouter().(*router)
|
||||
table := router.table
|
||||
func testSetup() (*table, router.Route) {
|
||||
routr := NewRouter().(*rtr)
|
||||
table := newTable(routr.fetchRoutes)
|
||||
|
||||
route := Route{
|
||||
route := router.Route{
|
||||
Service: "dest.svc",
|
||||
Address: "dest.addr",
|
||||
Gateway: "dest.gw",
|
||||
@@ -37,8 +39,8 @@ func TestCreate(t *testing.T) {
|
||||
}
|
||||
|
||||
// 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)
|
||||
if err := table.Create(route); err != router.ErrDuplicateRoute {
|
||||
t.Errorf("error adding route. Expected error: %s, found: %s", router.ErrDuplicateRoute, err)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -53,8 +55,8 @@ func TestDelete(t *testing.T) {
|
||||
prevSvc := route.Service
|
||||
route.Service = "randDest"
|
||||
|
||||
if err := table.Delete(route); err != ErrRouteNotFound {
|
||||
t.Errorf("error deleting route. Expected: %s, found: %s", ErrRouteNotFound, err)
|
||||
if err := table.Delete(route); err != router.ErrRouteNotFound {
|
||||
t.Errorf("error deleting route. Expected: %s, found: %s", router.ErrRouteNotFound, err)
|
||||
}
|
||||
|
||||
// we should be able to delete the existing route
|
||||
@@ -138,7 +140,7 @@ func TestQuery(t *testing.T) {
|
||||
// query routes particular network
|
||||
network := "net1"
|
||||
|
||||
routes, err = table.Query(QueryNetwork(network))
|
||||
routes, err = table.Query(router.QueryNetwork(network))
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
@@ -156,7 +158,7 @@ func TestQuery(t *testing.T) {
|
||||
// query routes for particular gateway
|
||||
gateway := "gw1"
|
||||
|
||||
routes, err = table.Query(QueryGateway(gateway))
|
||||
routes, err = table.Query(router.QueryGateway(gateway))
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
@@ -170,9 +172,9 @@ func TestQuery(t *testing.T) {
|
||||
}
|
||||
|
||||
// query routes for particular router
|
||||
router := "rtr1"
|
||||
rt := "rtr1"
|
||||
|
||||
routes, err = table.Query(QueryRouter(router))
|
||||
routes, err = table.Query(router.QueryRouter(rt))
|
||||
if err != nil {
|
||||
t.Errorf("error looking up routes: %s", err)
|
||||
}
|
||||
@@ -181,15 +183,15 @@ func TestQuery(t *testing.T) {
|
||||
t.Errorf("incorrect number of routes returned. Expected: %d, found: %d", 1, len(routes))
|
||||
}
|
||||
|
||||
if routes[0].Router != router {
|
||||
t.Errorf("incorrect route returned. Expected router: %s, found: %s", router, routes[0].Router)
|
||||
if routes[0].Router != rt {
|
||||
t.Errorf("incorrect route returned. Expected router: %s, found: %s", rt, routes[0].Router)
|
||||
}
|
||||
|
||||
// query particular gateway and network
|
||||
query := []QueryOption{
|
||||
QueryGateway(gateway),
|
||||
QueryNetwork(network),
|
||||
QueryRouter(router),
|
||||
query := []router.QueryOption{
|
||||
router.QueryGateway(gateway),
|
||||
router.QueryNetwork(network),
|
||||
router.QueryRouter(rt),
|
||||
}
|
||||
|
||||
routes, err = table.Query(query...)
|
||||
@@ -209,14 +211,14 @@ func TestQuery(t *testing.T) {
|
||||
t.Errorf("incorrect network returned. Expected network: %s, found: %s", network, routes[0].Network)
|
||||
}
|
||||
|
||||
if routes[0].Router != router {
|
||||
t.Errorf("incorrect route returned. Expected router: %s, found: %s", router, routes[0].Router)
|
||||
if routes[0].Router != rt {
|
||||
t.Errorf("incorrect route returned. Expected router: %s, found: %s", rt, routes[0].Router)
|
||||
}
|
||||
|
||||
// non-existen route query
|
||||
routes, err = table.Query(QueryService("foobar"))
|
||||
if err != ErrRouteNotFound {
|
||||
t.Errorf("error looking up routes. Expected: %s, found: %s", ErrRouteNotFound, err)
|
||||
routes, err = table.Query(router.QueryService("foobar"))
|
||||
if err != router.ErrRouteNotFound {
|
||||
t.Errorf("error looking up routes. Expected: %s, found: %s", router.ErrRouteNotFound, err)
|
||||
}
|
||||
|
||||
if len(routes) != 0 {
|
||||
@@ -224,10 +226,10 @@ func TestQuery(t *testing.T) {
|
||||
}
|
||||
|
||||
// query NO routes
|
||||
query = []QueryOption{
|
||||
QueryGateway(gateway),
|
||||
QueryNetwork(network),
|
||||
QueryStrategy(AdvertiseNone),
|
||||
query = []router.QueryOption{
|
||||
router.QueryGateway(gateway),
|
||||
router.QueryNetwork(network),
|
||||
router.QueryStrategy(router.AdvertiseNone),
|
||||
}
|
||||
|
||||
routes, err = table.Query(query...)
|
||||
@@ -249,10 +251,10 @@ func TestQuery(t *testing.T) {
|
||||
}
|
||||
|
||||
// query local routes
|
||||
query = []QueryOption{
|
||||
QueryGateway("*"),
|
||||
QueryNetwork("*"),
|
||||
QueryStrategy(AdvertiseLocal),
|
||||
query = []router.QueryOption{
|
||||
router.QueryGateway("*"),
|
||||
router.QueryNetwork("*"),
|
||||
router.QueryStrategy(router.AdvertiseLocal),
|
||||
}
|
||||
|
||||
routes, err = table.Query(query...)
|
||||
@@ -275,9 +277,9 @@ func TestQuery(t *testing.T) {
|
||||
}
|
||||
|
||||
// query best routes for svcX
|
||||
query = []QueryOption{
|
||||
QueryService("svcX"),
|
||||
QueryStrategy(AdvertiseBest),
|
||||
query = []router.QueryOption{
|
||||
router.QueryService("svcX"),
|
||||
router.QueryStrategy(router.AdvertiseBest),
|
||||
}
|
||||
|
||||
routes, err = table.Query(query...)
|
||||
@@ -292,15 +294,15 @@ func TestQuery(t *testing.T) {
|
||||
|
||||
func TestFallback(t *testing.T) {
|
||||
|
||||
r := &router{
|
||||
subscribers: make(map[string]chan *Advert),
|
||||
options: DefaultOptions(),
|
||||
r := &rtr{
|
||||
subscribers: make(map[string]chan *router.Advert),
|
||||
options: router.DefaultOptions(),
|
||||
}
|
||||
route := Route{
|
||||
route := router.Route{
|
||||
Service: "go.micro.service.foo",
|
||||
Router: r.options.Id,
|
||||
Link: DefaultLink,
|
||||
Metric: DefaultLocalMetric,
|
||||
Link: router.DefaultLink,
|
||||
Metric: router.DefaultLocalMetric,
|
||||
}
|
||||
r.table = newTable(func(s string) error {
|
||||
r.table.Create(route)
|
||||
@@ -308,7 +310,7 @@ func TestFallback(t *testing.T) {
|
||||
})
|
||||
r.start()
|
||||
|
||||
rts, err := r.Lookup(QueryService("go.micro.service.foo"))
|
||||
rts, err := r.Lookup(router.QueryService("go.micro.service.foo"))
|
||||
if err != nil {
|
||||
t.Errorf("error looking up service %s", err)
|
||||
}
|
||||
@@ -321,7 +323,7 @@ func TestFallback(t *testing.T) {
|
||||
t.Errorf("error deleting route %s", err)
|
||||
}
|
||||
|
||||
rts, err = r.Lookup(QueryService("go.micro.service.foo"))
|
||||
rts, err = r.Lookup(router.QueryService("go.micro.service.foo"))
|
||||
if err != nil {
|
||||
t.Errorf("error looking up service %s", err)
|
||||
}
|
||||
@@ -332,15 +334,15 @@ func TestFallback(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestFallbackError(t *testing.T) {
|
||||
r := &router{
|
||||
subscribers: make(map[string]chan *Advert),
|
||||
options: DefaultOptions(),
|
||||
r := &rtr{
|
||||
subscribers: make(map[string]chan *router.Advert),
|
||||
options: router.DefaultOptions(),
|
||||
}
|
||||
r.table = newTable(func(s string) error {
|
||||
return fmt.Errorf("ERROR")
|
||||
})
|
||||
r.start()
|
||||
_, err := r.Lookup(QueryService("go.micro.service.foo"))
|
||||
_, err := r.Lookup(router.QueryService("go.micro.service.foo"))
|
||||
if err == nil {
|
||||
t.Errorf("expected error looking up service but none returned")
|
||||
}
|
52
router/registry/watcher.go
Normal file
52
router/registry/watcher.go
Normal file
@@ -0,0 +1,52 @@
|
||||
package registry
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
// tableWatcher implements routing table Watcher
|
||||
type tableWatcher struct {
|
||||
sync.RWMutex
|
||||
id string
|
||||
opts router.WatchOptions
|
||||
resChan chan *router.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() (*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 watcher events channel
|
||||
func (w *tableWatcher) Chan() (<-chan *router.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)
|
||||
}
|
||||
}
|
@@ -2,6 +2,7 @@
|
||||
package router
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"time"
|
||||
)
|
||||
|
||||
@@ -12,8 +13,10 @@ var (
|
||||
DefaultName = "go.micro.router"
|
||||
// DefaultNetwork is default micro network
|
||||
DefaultNetwork = "micro"
|
||||
// DefaultRouter is default network router
|
||||
DefaultRouter = NewRouter()
|
||||
// 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")
|
||||
)
|
||||
|
||||
// Router is an interface for a routing control plane
|
||||
@@ -135,8 +138,3 @@ func (s Strategy) String() string {
|
||||
return "unknown"
|
||||
}
|
||||
}
|
||||
|
||||
// NewRouter creates new Router and returns it
|
||||
func NewRouter(opts ...Option) Router {
|
||||
return newRouter(opts...)
|
||||
}
|
||||
|
@@ -1,22 +0,0 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/micro/go-micro/v2/client"
|
||||
"github.com/micro/go-micro/v2/router"
|
||||
)
|
||||
|
||||
type clientKey struct{}
|
||||
|
||||
// Client to call router service
|
||||
func Client(c client.Client) router.Option {
|
||||
return func(o *router.Options) {
|
||||
if o.Context == nil {
|
||||
o.Context = context.WithValue(context.Background(), clientKey{}, c)
|
||||
return
|
||||
}
|
||||
|
||||
o.Context = context.WithValue(o.Context, clientKey{}, c)
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@@ -1,421 +0,0 @@
|
||||
// Code generated by protoc-gen-micro. DO NOT EDIT.
|
||||
// source: router/service/proto/router.proto
|
||||
|
||||
package router
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
math "math"
|
||||
)
|
||||
|
||||
import (
|
||||
context "context"
|
||||
api "github.com/micro/go-micro/v2/api"
|
||||
client "github.com/micro/go-micro/v2/client"
|
||||
server "github.com/micro/go-micro/v2/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 _ api.Endpoint
|
||||
var _ context.Context
|
||||
var _ client.Option
|
||||
var _ server.Option
|
||||
|
||||
// Api Endpoints for Router service
|
||||
|
||||
func NewRouterEndpoints() []*api.Endpoint {
|
||||
return []*api.Endpoint{}
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
type routerService struct {
|
||||
c client.Client
|
||||
name string
|
||||
}
|
||||
|
||||
func NewRouterService(name string, c client.Client) RouterService {
|
||||
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 {
|
||||
Context() context.Context
|
||||
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) Context() context.Context {
|
||||
return x.stream.Context()
|
||||
}
|
||||
|
||||
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 {
|
||||
Context() context.Context
|
||||
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) Context() context.Context {
|
||||
return x.stream.Context()
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
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 {
|
||||
Context() context.Context
|
||||
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) Context() context.Context {
|
||||
return x.stream.Context()
|
||||
}
|
||||
|
||||
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 {
|
||||
Context() context.Context
|
||||
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) Context() context.Context {
|
||||
return x.stream.Context()
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
// Api Endpoints for Table service
|
||||
|
||||
func NewTableEndpoints() []*api.Endpoint {
|
||||
return []*api.Endpoint{}
|
||||
}
|
||||
|
||||
// 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 {
|
||||
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)
|
||||
}
|
@@ -1,135 +0,0 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package go.micro.router;
|
||||
|
||||
option go_package = "github.com/micro/go-micro/v2/router/service/proto;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) {};
|
||||
}
|
||||
|
||||
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 {}
|
||||
|
||||
// Empty response
|
||||
message Response {}
|
||||
|
||||
// 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;
|
||||
}
|
||||
|
||||
// QueryRequest queries Table for Routes
|
||||
message QueryRequest{
|
||||
Query query = 1;
|
||||
}
|
||||
|
||||
// QueryResponse is returned by Query
|
||||
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 {
|
||||
// the unique event id
|
||||
string id = 1;
|
||||
// type of event
|
||||
EventType type = 2;
|
||||
// unix timestamp of event
|
||||
int64 timestamp = 3;
|
||||
// service route
|
||||
Route route = 4;
|
||||
}
|
||||
|
||||
// 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;
|
||||
// router if the router id
|
||||
string router = 5;
|
||||
// the network link
|
||||
string link = 6;
|
||||
// the metric / score of this route
|
||||
int64 metric = 7;
|
||||
// metadata for the route
|
||||
map<string,string> metadata = 8;
|
||||
}
|
@@ -1,271 +0,0 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/micro/go-micro/v2/client"
|
||||
"github.com/micro/go-micro/v2/errors"
|
||||
"github.com/micro/go-micro/v2/router"
|
||||
pb "github.com/micro/go-micro/v2/router/service/proto"
|
||||
)
|
||||
|
||||
type svc struct {
|
||||
sync.RWMutex
|
||||
opts router.Options
|
||||
callOpts []client.CallOption
|
||||
router pb.RouterService
|
||||
table *table
|
||||
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
|
||||
|
||||
// get options client from the context. We set this in the context to prevent an import loop, as
|
||||
// the client depends on the router
|
||||
if c, ok := options.Context.Value(clientKey{}).(client.Client); ok {
|
||||
cli = c
|
||||
}
|
||||
|
||||
// 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 {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
for _, o := range opts {
|
||||
o(&s.opts)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Options returns router options
|
||||
func (s *svc) Options() router.Options {
|
||||
s.Lock()
|
||||
opts := s.opts
|
||||
s.Unlock()
|
||||
|
||||
return opts
|
||||
}
|
||||
|
||||
// Table returns routing table
|
||||
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: event.Route.Metric,
|
||||
Metadata: event.Route.Metadata,
|
||||
}
|
||||
|
||||
events[i] = &router.Event{
|
||||
Id: event.Id,
|
||||
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()
|
||||
|
||||
stream, err := s.router.Advertise(context.Background(), &pb.Request{}, 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
|
||||
}
|
||||
|
||||
// Process processes incoming adverts
|
||||
func (s *svc) Process(advert *router.Advert) error {
|
||||
events := make([]*pb.Event, 0, len(advert.Events))
|
||||
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: event.Route.Metric,
|
||||
Metadata: event.Route.Metadata,
|
||||
}
|
||||
e := &pb.Event{
|
||||
Id: event.Id,
|
||||
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
|
||||
}
|
||||
|
||||
// Remote router cannot be closed
|
||||
func (s *svc) Close() 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.QueryOption) ([]router.Route, error) {
|
||||
// call the router
|
||||
query := router.NewQuery(q...)
|
||||
|
||||
resp, err := s.router.Lookup(context.Background(), &pb.LookupRequest{
|
||||
Query: &pb.Query{
|
||||
Service: query.Service,
|
||||
Gateway: query.Gateway,
|
||||
Network: query.Network,
|
||||
},
|
||||
}, s.callOpts...)
|
||||
|
||||
if verr, ok := err.(*errors.Error); ok && verr.Code == http.StatusNotFound {
|
||||
return nil, router.ErrRouteNotFound
|
||||
} else 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: route.Metric,
|
||||
Metadata: route.Metadata,
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
options := router.WatchOptions{
|
||||
Service: "*",
|
||||
}
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
return newWatcher(rsp, options)
|
||||
}
|
||||
|
||||
// Returns the router implementation
|
||||
func (s *svc) String() string {
|
||||
return "service"
|
||||
}
|
@@ -1,123 +0,0 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/micro/go-micro/v2/client"
|
||||
"github.com/micro/go-micro/v2/router"
|
||||
pb "github.com/micro/go-micro/v2/router/service/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: 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: 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: 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: route.Metric,
|
||||
}
|
||||
}
|
||||
|
||||
return routes, nil
|
||||
}
|
||||
|
||||
// Lookup looks up routes in the routing table and returns them
|
||||
func (t *table) Query(q ...router.QueryOption) ([]router.Route, error) {
|
||||
query := router.NewQuery(q...)
|
||||
|
||||
// call the router
|
||||
resp, err := t.table.Query(context.Background(), &pb.QueryRequest{
|
||||
Query: &pb.Query{
|
||||
Service: query.Service,
|
||||
Gateway: query.Gateway,
|
||||
Network: query.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: route.Metric,
|
||||
}
|
||||
}
|
||||
|
||||
return routes, nil
|
||||
}
|
@@ -1,118 +0,0 @@
|
||||
package service
|
||||
|
||||
import (
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/micro/go-micro/v2/router"
|
||||
pb "github.com/micro/go-micro/v2/router/service/proto"
|
||||
)
|
||||
|
||||
type watcher struct {
|
||||
sync.RWMutex
|
||||
opts router.WatchOptions
|
||||
resChan chan *router.Event
|
||||
done chan struct{}
|
||||
stream pb.Router_WatchService
|
||||
}
|
||||
|
||||
func newWatcher(rsp pb.Router_WatchService, opts router.WatchOptions) (*watcher, error) {
|
||||
w := &watcher{
|
||||
opts: opts,
|
||||
resChan: make(chan *router.Event),
|
||||
done: make(chan struct{}),
|
||||
stream: rsp,
|
||||
}
|
||||
|
||||
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 {
|
||||
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: resp.Route.Metric,
|
||||
Metadata: resp.Route.Metadata,
|
||||
}
|
||||
|
||||
event := &router.Event{
|
||||
Id: resp.Id,
|
||||
Type: router.EventType(resp.Type),
|
||||
Timestamp: time.Unix(0, resp.Timestamp),
|
||||
Route: route,
|
||||
}
|
||||
|
||||
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:
|
||||
w.stream.Close()
|
||||
close(w.done)
|
||||
}
|
||||
}
|
@@ -1,7 +1,7 @@
|
||||
package static
|
||||
|
||||
import (
|
||||
"github.com/micro/go-micro/v2/router"
|
||||
"github.com/micro/go-micro/v3/router"
|
||||
)
|
||||
|
||||
// NewRouter returns an initialized static router
|
||||
|
@@ -2,7 +2,6 @@ package router
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
@@ -77,48 +76,3 @@ func WatchService(s string) WatchOption {
|
||||
o.Service = s
|
||||
}
|
||||
}
|
||||
|
||||
// tableWatcher implements routing table Watcher
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user