v3 refactor (#1868)

* Move to v3

Co-authored-by: Ben Toogood <bentoogood@gmail.com>
This commit is contained in:
Asim Aslam
2020-07-27 13:22:00 +01:00
committed by GitHub
parent 9dfeb98111
commit 563768b58a
424 changed files with 6383 additions and 22490 deletions

View File

@@ -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"
}

View File

@@ -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

View File

@@ -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(),
}

View File

@@ -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"
}

View File

@@ -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",

View File

@@ -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{}),
}

View File

@@ -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")
}

View 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)
}
}

View File

@@ -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...)
}

View File

@@ -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

View File

@@ -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)
}

View File

@@ -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;
}

View File

@@ -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"
}

View File

@@ -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
}

View File

@@ -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)
}
}

View File

@@ -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

View File

@@ -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)
}
}