Deprecate client/selector (#1767)

* client/{grpc,rpc}: depricate selector (wip)

* {client,cmd}: remove client/selector

* deprecate client/selector

* router/static: fix lookup

* config/cmd: add support for legacy static selector flag

* config/cmd: add support for legacy dns selector flag
This commit is contained in:
ben-toogood 2020-07-01 17:06:59 +01:00 committed by GitHub
parent a63480a81a
commit 174e44b846
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 428 additions and 1572 deletions

View File

@ -6,9 +6,9 @@ import (
goapi "github.com/micro/go-micro/v2/api" goapi "github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/handler" "github.com/micro/go-micro/v2/api/handler"
"github.com/micro/go-micro/v2/api/handler/util"
api "github.com/micro/go-micro/v2/api/proto" api "github.com/micro/go-micro/v2/api/proto"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/util/ctx" "github.com/micro/go-micro/v2/util/ctx"
) )
@ -71,10 +71,8 @@ func (a *apiHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// create the context from headers // create the context from headers
cx := ctx.FromRequest(r) cx := ctx.FromRequest(r)
// create strategy
so := selector.WithStrategy(strategy(service.Services))
if err := c.Call(cx, req, rsp, client.WithSelectOption(so)); err != nil { if err := c.Call(cx, req, rsp, client.WithRouter(util.Router(service.Services))); err != nil {
w.Header().Set("Content-Type", "application/json") w.Header().Set("Content-Type", "application/json")
ce := errors.Parse(err.Error()) ce := errors.Parse(err.Error())
switch ce.Code { switch ce.Code {

View File

@ -8,8 +8,6 @@ import (
"strings" "strings"
api "github.com/micro/go-micro/v2/api/proto" api "github.com/micro/go-micro/v2/api/proto"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/registry"
"github.com/oxtoacart/bpool" "github.com/oxtoacart/bpool"
) )
@ -109,11 +107,3 @@ func requestToProto(r *http.Request) (*api.Request, error) {
return req, nil return req, nil
} }
// strategy is a hack for selection
func strategy(services []*registry.Service) selector.Strategy {
return func(_ []*registry.Service) selector.Next {
// ignore input to this function, use services above
return selector.Random(services)
}
}

View File

@ -4,13 +4,14 @@ package http
import ( import (
"errors" "errors"
"fmt" "fmt"
"math/rand"
"net/http" "net/http"
"net/http/httputil" "net/http/httputil"
"net/url" "net/url"
"github.com/micro/go-micro/v2/api" "github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/handler" "github.com/micro/go-micro/v2/api/handler"
"github.com/micro/go-micro/v2/client/selector" "github.com/micro/go-micro/v2/registry"
) )
const ( const (
@ -64,16 +65,19 @@ func (h *httpHandler) getService(r *http.Request) (string, error) {
return "", errors.New("no route found") return "", errors.New("no route found")
} }
// create a random selector // get the nodes for this service
next := selector.Random(service.Services) var nodes []*registry.Node
for _, srv := range service.Services {
// get the next node nodes = append(nodes, srv.Nodes...)
s, err := next()
if err != nil {
return "", nil
} }
return fmt.Sprintf("http://%s", s.Address), nil // select a random node
if len(nodes) == 0 {
return "", errors.New("no route found")
}
node := nodes[rand.Int()%len(nodes)]
return fmt.Sprintf("http://%s", node.Address), nil
} }
func (h *httpHandler) String() string { func (h *httpHandler) String() string {

View File

@ -12,16 +12,15 @@ import (
jsonpatch "github.com/evanphx/json-patch/v5" jsonpatch "github.com/evanphx/json-patch/v5"
"github.com/micro/go-micro/v2/api" "github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/handler" "github.com/micro/go-micro/v2/api/handler"
"github.com/micro/go-micro/v2/api/handler/util"
"github.com/micro/go-micro/v2/api/internal/proto" "github.com/micro/go-micro/v2/api/internal/proto"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/codec" "github.com/micro/go-micro/v2/codec"
"github.com/micro/go-micro/v2/codec/jsonrpc" "github.com/micro/go-micro/v2/codec/jsonrpc"
"github.com/micro/go-micro/v2/codec/protorpc" "github.com/micro/go-micro/v2/codec/protorpc"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/logger" "github.com/micro/go-micro/v2/logger"
"github.com/micro/go-micro/v2/metadata" "github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/util/ctx" "github.com/micro/go-micro/v2/util/ctx"
"github.com/micro/go-micro/v2/util/qson" "github.com/micro/go-micro/v2/util/qson"
"github.com/oxtoacart/bpool" "github.com/oxtoacart/bpool"
@ -65,14 +64,6 @@ func (b *buffer) Write(_ []byte) (int, error) {
return 0, nil return 0, nil
} }
// strategy is a hack for selection
func strategy(services []*registry.Service) selector.Strategy {
return func(_ []*registry.Service) selector.Next {
// ignore input to this function, use services above
return selector.Random(services)
}
}
func (h *rpcHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { func (h *rpcHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
bsize := handler.DefaultMaxRecvSize bsize := handler.DefaultMaxRecvSize
if h.opts.MaxRecvSize > 0 { if h.opts.MaxRecvSize > 0 {
@ -141,8 +132,8 @@ func (h *rpcHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
return return
} }
// create strategy // create custom router
so := selector.WithStrategy(strategy(service.Services)) callOpt := client.WithRouter(util.Router(service.Services))
// walk the standard call path // walk the standard call path
// get payload // get payload
@ -174,7 +165,7 @@ func (h *rpcHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
) )
// make the call // make the call
if err := c.Call(cx, req, response, client.WithSelectOption(so)); err != nil { if err := c.Call(cx, req, response, callOpt); err != nil {
writeError(w, r, err) writeError(w, r, err)
return return
} }
@ -209,7 +200,7 @@ func (h *rpcHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
client.WithContentType(ct), client.WithContentType(ct),
) )
// make the call // make the call
if err := c.Call(cx, req, &response, client.WithSelectOption(so)); err != nil { if err := c.Call(cx, req, &response, callOpt); err != nil {
writeError(w, r, err) writeError(w, r, err)
return return
} }

View File

@ -13,8 +13,8 @@ import (
"github.com/gobwas/ws" "github.com/gobwas/ws"
"github.com/gobwas/ws/wsutil" "github.com/gobwas/ws/wsutil"
"github.com/micro/go-micro/v2/api" "github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/handler/util"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
raw "github.com/micro/go-micro/v2/codec/bytes" raw "github.com/micro/go-micro/v2/codec/bytes"
"github.com/micro/go-micro/v2/logger" "github.com/micro/go-micro/v2/logger"
) )
@ -110,9 +110,11 @@ func serveWebsocket(ctx context.Context, w http.ResponseWriter, r *http.Request,
client.StreamingRequest(), client.StreamingRequest(),
) )
so := selector.WithStrategy(strategy(service.Services)) // create custom router
callOpt := client.WithRouter(util.Router(service.Services))
// create a new stream // create a new stream
stream, err := c.Stream(ctx, req, client.WithSelectOption(so)) stream, err := c.Stream(ctx, req, callOpt)
if err != nil { if err != nil {
if logger.V(logger.ErrorLevel, logger.DefaultLogger) { if logger.V(logger.ErrorLevel, logger.DefaultLogger) {
logger.Error(err) logger.Error(err)

View File

@ -0,0 +1,32 @@
package util
import (
"github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/router"
)
// Router is a hack for API routing
func Router(srvs []*registry.Service) router.Router {
var routes []router.Route
for _, srv := range srvs {
for _, n := range srv.Nodes {
routes = append(routes, router.Route{Address: n.Address, Metadata: n.Metadata})
}
}
return &apiRouter{routes: routes}
}
func (r *apiRouter) Lookup(...router.QueryOption) ([]router.Route, error) {
return r.routes, nil
}
type apiRouter struct {
routes []router.Route
router.Router
}
func (r *apiRouter) String() string {
return "api"
}

View File

@ -5,6 +5,7 @@ import (
"errors" "errors"
"fmt" "fmt"
"io" "io"
"math/rand"
"net" "net"
"net/http" "net/http"
"net/http/httputil" "net/http/httputil"
@ -13,7 +14,7 @@ import (
"github.com/micro/go-micro/v2/api" "github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/handler" "github.com/micro/go-micro/v2/api/handler"
"github.com/micro/go-micro/v2/client/selector" "github.com/micro/go-micro/v2/registry"
) )
const ( const (
@ -70,16 +71,19 @@ func (wh *webHandler) getService(r *http.Request) (string, error) {
return "", errors.New("no route found") return "", errors.New("no route found")
} }
// create a random selector // get the nodes
next := selector.Random(service.Services) var nodes []*registry.Node
for _, srv := range service.Services {
// get the next node nodes = append(nodes, srv.Nodes...)
s, err := next() }
if err != nil { if len(nodes) == 0 {
return "", nil return "", errors.New("no route found")
} }
return fmt.Sprintf("http://%s", s.Address), nil // select a random node
node := nodes[rand.Int()%len(nodes)]
return fmt.Sprintf("http://%s", node.Address), nil
} }
// serveWebSocket used to serve a web socket proxied connection // serveWebSocket used to serve a web socket proxied connection

View File

@ -22,11 +22,6 @@ type Client interface {
String() string String() string
} }
// Router manages request routing
type Router interface {
SendRequest(context.Context, Request) (Response, error)
}
// Message is the interface for publishing asynchronously // Message is the interface for publishing asynchronously
type Message interface { type Message interface {
Topic() string Topic() string

View File

@ -5,6 +5,7 @@ import (
"context" "context"
"crypto/tls" "crypto/tls"
"fmt" "fmt"
"math/rand"
"net" "net"
"reflect" "reflect"
"strings" "strings"
@ -13,11 +14,11 @@ import (
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
raw "github.com/micro/go-micro/v2/codec/bytes" raw "github.com/micro/go-micro/v2/codec/bytes"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/metadata" "github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/selector"
pnet "github.com/micro/go-micro/v2/util/net" pnet "github.com/micro/go-micro/v2/util/net"
"google.golang.org/grpc" "google.golang.org/grpc"
@ -72,41 +73,59 @@ func (g *grpcClient) secure(addr string) grpc.DialOption {
return grpc.WithInsecure() return grpc.WithInsecure()
} }
func (g *grpcClient) next(request client.Request, opts client.CallOptions) (selector.Next, error) { // lookupRoute for a request using the router and then choose one using the selector
service, address, _ := pnet.Proxy(request.Service(), opts.Address) func (g *grpcClient) lookupRoute(req client.Request, opts client.CallOptions) (*router.Route, error) {
// check to see if the proxy has been set, if it has we don't need to lookup the routes; net.Proxy
// return remote address // returns a slice of addresses, so we'll use a random one. Eventually we should to use the
if len(address) > 0 { // selector for this.
return func() (*registry.Node, error) { service, addresses, _ := pnet.Proxy(req.Service(), opts.Address)
return &registry.Node{ if len(addresses) > 0 {
Address: address[0], return &router.Route{
}, nil Service: service,
Address: addresses[rand.Int()%len(addresses)],
}, nil }, nil
} }
// if the network was set, pass it to the selector // construct the router query
sopts := opts.SelectOptions query := []router.QueryOption{router.QueryService(req.Service())}
// if a custom network was requested, pass this to the router. By default the router will use it's
// own network, which is set during initialisation.
if len(opts.Network) > 0 { if len(opts.Network) > 0 {
sopts = append(sopts, selector.WithDomain(opts.Network)) query = append(query, router.QueryNetwork(opts.Network))
} }
// get next nodes from the selector // use the router passed as a call option, or fallback to the grpc clients router
next, err := g.opts.Selector.Select(service, sopts...) if opts.Router == nil {
if err != nil { opts.Router = g.opts.Router
if err == selector.ErrNotFound {
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
}
return nil, errors.InternalServerError("go.micro.client", "error selecting %s node: %s", service, err.Error())
} }
return next, nil // lookup the routes which can be used to execute the request
routes, err := opts.Router.Lookup(query...)
if err == router.ErrRouteNotFound {
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", req.Service(), err.Error())
} else if err != nil {
return nil, errors.InternalServerError("go.micro.client", "error getting next %s node: %s", req.Service(), err.Error())
}
// use the selector passed as a call option, or fallback to the grpc clients selector
if opts.Selector == nil {
opts.Selector = g.opts.Selector
}
// select the route to use for the request
if route, err := opts.Selector.Select(routes); err == selector.ErrNoneAvailable {
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", req.Service(), err.Error())
} else if err != nil {
return nil, errors.InternalServerError("go.micro.client", "error getting next %s node: %s", req.Service(), err.Error())
} else {
return route, nil
}
} }
func (g *grpcClient) call(ctx context.Context, node *registry.Node, req client.Request, rsp interface{}, opts client.CallOptions) error { func (g *grpcClient) call(ctx context.Context, route *router.Route, req client.Request, rsp interface{}, opts client.CallOptions) error {
var header map[string]string var header map[string]string
address := node.Address
header = make(map[string]string) header = make(map[string]string)
if md, ok := metadata.FromContext(ctx); ok { if md, ok := metadata.FromContext(ctx); ok {
header = make(map[string]string, len(md)) header = make(map[string]string, len(md))
@ -137,7 +156,7 @@ func (g *grpcClient) call(ctx context.Context, node *registry.Node, req client.R
grpcDialOptions := []grpc.DialOption{ grpcDialOptions := []grpc.DialOption{
grpc.WithTimeout(opts.DialTimeout), grpc.WithTimeout(opts.DialTimeout),
g.secure(address), g.secure(route.Address),
grpc.WithDefaultCallOptions( grpc.WithDefaultCallOptions(
grpc.MaxCallRecvMsgSize(maxRecvMsgSize), grpc.MaxCallRecvMsgSize(maxRecvMsgSize),
grpc.MaxCallSendMsgSize(maxSendMsgSize), grpc.MaxCallSendMsgSize(maxSendMsgSize),
@ -148,13 +167,13 @@ func (g *grpcClient) call(ctx context.Context, node *registry.Node, req client.R
grpcDialOptions = append(grpcDialOptions, opts...) grpcDialOptions = append(grpcDialOptions, opts...)
} }
cc, err := g.pool.getConn(address, grpcDialOptions...) cc, err := g.pool.getConn(route.Address, grpcDialOptions...)
if err != nil { if err != nil {
return errors.InternalServerError("go.micro.client", fmt.Sprintf("Error sending request: %v", err)) return errors.InternalServerError("go.micro.client", fmt.Sprintf("Error sending request: %v", err))
} }
defer func() { defer func() {
// defer execution of release // defer execution of release
g.pool.release(address, cc, grr) g.pool.release(route.Address, cc, grr)
}() }()
ch := make(chan error, 1) ch := make(chan error, 1)
@ -180,11 +199,9 @@ func (g *grpcClient) call(ctx context.Context, node *registry.Node, req client.R
return grr return grr
} }
func (g *grpcClient) stream(ctx context.Context, node *registry.Node, req client.Request, rsp interface{}, opts client.CallOptions) error { func (g *grpcClient) stream(ctx context.Context, route *router.Route, req client.Request, rsp interface{}, opts client.CallOptions) error {
var header map[string]string var header map[string]string
address := node.Address
if md, ok := metadata.FromContext(ctx); ok { if md, ok := metadata.FromContext(ctx); ok {
header = make(map[string]string, len(md)) header = make(map[string]string, len(md))
for k, v := range md { for k, v := range md {
@ -222,14 +239,14 @@ func (g *grpcClient) stream(ctx context.Context, node *registry.Node, req client
grpcDialOptions := []grpc.DialOption{ grpcDialOptions := []grpc.DialOption{
grpc.WithTimeout(opts.DialTimeout), grpc.WithTimeout(opts.DialTimeout),
g.secure(address), g.secure(route.Address),
} }
if opts := g.getGrpcDialOptions(); opts != nil { if opts := g.getGrpcDialOptions(); opts != nil {
grpcDialOptions = append(grpcDialOptions, opts...) grpcDialOptions = append(grpcDialOptions, opts...)
} }
cc, err := grpc.DialContext(dialCtx, address, grpcDialOptions...) cc, err := grpc.DialContext(dialCtx, route.Address, grpcDialOptions...)
if err != nil { if err != nil {
return errors.InternalServerError("go.micro.client", fmt.Sprintf("Error sending request: %v", err)) return errors.InternalServerError("go.micro.client", fmt.Sprintf("Error sending request: %v", err))
} }
@ -396,11 +413,6 @@ func (g *grpcClient) Call(ctx context.Context, req client.Request, rsp interface
opt(&callOpts) opt(&callOpts)
} }
next, err := g.next(req, callOpts)
if err != nil {
return err
}
// check if we already have a deadline // check if we already have a deadline
d, ok := ctx.Deadline() d, ok := ctx.Deadline()
if !ok { if !ok {
@ -443,19 +455,19 @@ func (g *grpcClient) Call(ctx context.Context, req client.Request, rsp interface
time.Sleep(t) time.Sleep(t)
} }
// select next node // lookup the route to send the reques to
node, err := next() route, err := g.lookupRoute(req, callOpts)
service := req.Service()
if err != nil { if err != nil {
if err == selector.ErrNotFound { return err
return errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
}
return errors.InternalServerError("go.micro.client", "error selecting %s node: %s", service, err.Error())
} }
// make the call // make the call
err = gcall(ctx, node, req, rsp, callOpts) err = gcall(ctx, route, req, rsp, callOpts)
g.opts.Selector.Mark(service, node, err)
// record the result of the call to inform future routing decisions
g.opts.Selector.Record(*route, err)
// try and transform the error to a go-micro error
if verr, ok := err.(*errors.Error); ok { if verr, ok := err.(*errors.Error); ok {
return verr return verr
} }
@ -503,11 +515,6 @@ func (g *grpcClient) Stream(ctx context.Context, req client.Request, opts ...cli
opt(&callOpts) opt(&callOpts)
} }
next, err := g.next(req, callOpts)
if err != nil {
return nil, err
}
// #200 - streams shouldn't have a request timeout set on the context // #200 - streams shouldn't have a request timeout set on the context
// should we noop right here? // should we noop right here?
@ -537,20 +544,25 @@ func (g *grpcClient) Stream(ctx context.Context, req client.Request, opts ...cli
time.Sleep(t) time.Sleep(t)
} }
node, err := next() // lookup the route to send the reques to
service := req.Service() route, err := g.lookupRoute(req, callOpts)
if err != nil { if err != nil {
if err == selector.ErrNotFound { return nil, err
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
}
return nil, errors.InternalServerError("go.micro.client", "error selecting %s node: %s", service, err.Error())
} }
// make the call // make the call
stream := &grpcStream{} stream := &grpcStream{}
err = g.stream(ctx, node, req, stream, callOpts) err = g.stream(ctx, route, req, stream, callOpts)
g.opts.Selector.Mark(service, node, err) // record the result of the call to inform future routing decisions
g.opts.Selector.Record(*route, err)
// try and transform the error to a go-micro error
if verr, ok := err.(*errors.Error); ok {
return nil, verr
}
g.opts.Selector.Record(*route, err)
return stream, err return stream, err
} }
@ -577,7 +589,7 @@ func (g *grpcClient) Stream(ctx context.Context, req client.Request, opts ...cli
return rsp.stream, nil return rsp.stream, nil
} }
retry, rerr := callOpts.Retry(ctx, req, i, err) retry, rerr := callOpts.Retry(ctx, req, i, grr)
if rerr != nil { if rerr != nil {
return nil, rerr return nil, rerr
} }

View File

@ -6,10 +6,10 @@ import (
"testing" "testing"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/registry/memory" "github.com/micro/go-micro/v2/registry/memory"
"github.com/micro/go-micro/v2/router"
pgrpc "google.golang.org/grpc" pgrpc "google.golang.org/grpc"
pb "google.golang.org/grpc/examples/helloworld/helloworld" pb "google.golang.org/grpc/examples/helloworld/helloworld"
) )
@ -56,16 +56,11 @@ func TestGRPCClient(t *testing.T) {
}, },
}) })
// create selector // create router
se := selector.NewSelector( rtr := router.NewRouter(router.Registry(r))
selector.Registry(r),
)
// create client // create client
c := NewClient( c := NewClient(client.Router(rtr))
client.Registry(r),
client.Selector(se),
)
testMethods := []string{ testMethods := []string{
"/helloworld.Greeter/SayHello", "/helloworld.Greeter/SayHello",

View File

@ -5,9 +5,10 @@ import (
"time" "time"
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/codec" "github.com/micro/go-micro/v2/codec"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/selector"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"
) )
@ -18,13 +19,10 @@ type Options struct {
// Plugged interfaces // Plugged interfaces
Broker broker.Broker Broker broker.Broker
Codecs map[string]codec.NewCodec Codecs map[string]codec.NewCodec
Registry registry.Registry Router router.Router
Selector selector.Selector Selector selector.Selector
Transport transport.Transport Transport transport.Transport
// Router sets the router
Router Router
// Connection Pool // Connection Pool
PoolSize int PoolSize int
PoolTTL time.Duration PoolTTL time.Duration
@ -44,26 +42,28 @@ type Options struct {
} }
type CallOptions struct { type CallOptions struct {
SelectOptions []selector.SelectOption
// Address of remote hosts // Address of remote hosts
Address []string Address []string
// Backoff func // Backoff func
Backoff BackoffFunc Backoff BackoffFunc
// Check if retriable func // Duration to cache the response for
Retry RetryFunc CacheExpiry time.Duration
// Transport Dial Timeout // Transport Dial Timeout
DialTimeout time.Duration DialTimeout time.Duration
// Number of Call attempts // Number of Call attempts
Retries int Retries int
// Check if retriable func
Retry RetryFunc
// Request/Response timeout // Request/Response timeout
RequestTimeout time.Duration RequestTimeout time.Duration
// Router to use for this call
Router router.Router
// Selector to use for the call
Selector selector.Selector
// Stream timeout for the stream // Stream timeout for the stream
StreamTimeout time.Duration StreamTimeout time.Duration
// Use the services own auth token // Use the services own auth token
ServiceToken bool ServiceToken bool
// Duration to cache the response for
CacheExpiry time.Duration
// Network to lookup the route within // Network to lookup the route within
Network string Network string
@ -112,8 +112,8 @@ func NewOptions(options ...Option) Options {
PoolSize: DefaultPoolSize, PoolSize: DefaultPoolSize,
PoolTTL: DefaultPoolTTL, PoolTTL: DefaultPoolTTL,
Broker: broker.DefaultBroker, Broker: broker.DefaultBroker,
Router: router.DefaultRouter,
Selector: selector.DefaultSelector, Selector: selector.DefaultSelector,
Registry: registry.DefaultRegistry,
Transport: transport.DefaultTransport, Transport: transport.DefaultTransport,
} }
@ -159,15 +159,6 @@ func PoolTTL(d time.Duration) Option {
} }
} }
// Registry to find nodes for a given service
func Registry(r registry.Registry) Option {
return func(o *Options) {
o.Registry = r
// set in the selector
o.Selector.Init(selector.Registry(r))
}
}
// Transport to use for communication e.g http, rabbitmq, etc // Transport to use for communication e.g http, rabbitmq, etc
func Transport(t transport.Transport) Option { func Transport(t transport.Transport) Option {
return func(o *Options) { return func(o *Options) {
@ -175,7 +166,14 @@ func Transport(t transport.Transport) Option {
} }
} }
// Select is used to select a node to route a request to // Router is used to lookup routes for a service
func Router(r router.Router) Option {
return func(o *Options) {
o.Router = r
}
}
// Selector is used to select a route
func Selector(s selector.Selector) Option { func Selector(s selector.Selector) Option {
return func(o *Options) { return func(o *Options) {
o.Selector = s o.Selector = s
@ -219,6 +217,13 @@ func Retry(fn RetryFunc) Option {
} }
} }
// Registry sets the routers registry
func Registry(r registry.Registry) Option {
return func(o *Options) {
o.Router.Init(router.Registry(r))
}
}
// The request timeout. // The request timeout.
// Should this be a Call Option? // Should this be a Call Option?
func RequestTimeout(d time.Duration) Option { func RequestTimeout(d time.Duration) Option {
@ -264,12 +269,6 @@ func WithAddress(a ...string) CallOption {
} }
} }
func WithSelectOption(so ...selector.SelectOption) CallOption {
return func(o *CallOptions) {
o.SelectOptions = append(o.SelectOptions, so...)
}
}
// WithCallWrapper is a CallOption which adds to the existing CallFunc wrappers // WithCallWrapper is a CallOption which adds to the existing CallFunc wrappers
func WithCallWrapper(cw ...CallWrapper) CallOption { func WithCallWrapper(cw ...CallWrapper) CallOption {
return func(o *CallOptions) { return func(o *CallOptions) {
@ -347,6 +346,20 @@ func WithNetwork(n string) CallOption {
} }
} }
// WithRouter sets the router to use for this call
func WithRouter(r router.Router) CallOption {
return func(o *CallOptions) {
o.Router = r
}
}
// WithSelector sets the selector to use for this call
func WithSelector(s selector.Selector) CallOption {
return func(o *CallOptions) {
o.Selector = s
}
}
func WithMessageContentType(ct string) MessageOption { func WithMessageContentType(ct string) MessageOption {
return func(o *MessageOptions) { return func(o *MessageOptions) {
o.ContentType = ct o.ContentType = ct
@ -366,10 +379,3 @@ func StreamingRequest() RequestOption {
o.Stream = true o.Stream = true
} }
} }
// WithRouter sets the client router
func WithRouter(r Router) Option {
return func(o *Options) {
o.Router = r
}
}

View File

@ -3,17 +3,18 @@ package client
import ( import (
"context" "context"
"fmt" "fmt"
"math/rand"
"sync/atomic" "sync/atomic"
"time" "time"
"github.com/google/uuid" "github.com/google/uuid"
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/codec" "github.com/micro/go-micro/v2/codec"
raw "github.com/micro/go-micro/v2/codec/bytes" raw "github.com/micro/go-micro/v2/codec/bytes"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/metadata" "github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/selector"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"
"github.com/micro/go-micro/v2/util/buf" "github.com/micro/go-micro/v2/util/buf"
"github.com/micro/go-micro/v2/util/net" "github.com/micro/go-micro/v2/util/net"
@ -63,9 +64,56 @@ func (r *rpcClient) newCodec(contentType string) (codec.NewCodec, error) {
return nil, fmt.Errorf("Unsupported Content-Type: %s", contentType) return nil, fmt.Errorf("Unsupported Content-Type: %s", contentType)
} }
func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request, resp interface{}, opts CallOptions) error { func (r *rpcClient) lookupRoute(req Request, opts CallOptions) (*router.Route, error) {
address := node.Address // check to see if the proxy has been set, if it has we don't need to lookup the routes; net.Proxy
// returns a slice of addresses, so we'll use a random one. Eventually we should to use the
// selector for this.
service, addresses, _ := net.Proxy(req.Service(), opts.Address)
if len(addresses) > 0 {
return &router.Route{
Service: service,
Address: addresses[rand.Int()%len(addresses)],
}, nil
}
// construct the router query
query := []router.QueryOption{router.QueryService(req.Service())}
// if a custom network was requested, pass this to the router. By default the router will use it's
// own network, which is set during initialisation.
if len(opts.Network) > 0 {
query = append(query, router.QueryNetwork(opts.Network))
}
// use the router passed as a call option, or fallback to the rpc clients router
if opts.Router == nil {
opts.Router = r.opts.Router
}
// lookup the routes which can be used to execute the request
routes, err := opts.Router.Lookup(query...)
if err == router.ErrRouteNotFound {
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", req.Service(), err.Error())
} else if err != nil {
return nil, errors.InternalServerError("go.micro.client", "error getting next %s node: %s", req.Service(), err.Error())
}
// use the selector passed as a call option, or fallback to the rpc clients selector
if opts.Selector == nil {
opts.Selector = r.opts.Selector
}
// select the route to use for the request
if route, err := opts.Selector.Select(routes); err == selector.ErrNoneAvailable {
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", req.Service(), err.Error())
} else if err != nil {
return nil, errors.InternalServerError("go.micro.client", "error getting next %s node: %s", req.Service(), err.Error())
} else {
return route, nil
}
}
func (r *rpcClient) call(ctx context.Context, route *router.Route, req Request, resp interface{}, opts CallOptions) error {
msg := &transport.Message{ msg := &transport.Message{
Header: make(map[string]string), Header: make(map[string]string),
} }
@ -90,7 +138,7 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
msg.Header["Accept"] = req.ContentType() msg.Header["Accept"] = req.ContentType()
// setup old protocol // setup old protocol
cf := setupProtocol(msg, node) cf := setupProtocol(msg, route)
// no codec specified // no codec specified
if cf == nil { if cf == nil {
@ -109,7 +157,7 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
dOpts = append(dOpts, transport.WithTimeout(opts.DialTimeout)) dOpts = append(dOpts, transport.WithTimeout(opts.DialTimeout))
} }
c, err := r.pool.Get(address, dOpts...) c, err := r.pool.Get(route.Address, dOpts...)
if err != nil { if err != nil {
return errors.InternalServerError("go.micro.client", "connection error: %v", err) return errors.InternalServerError("go.micro.client", "connection error: %v", err)
} }
@ -182,9 +230,7 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
return nil return nil
} }
func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request, opts CallOptions) (Stream, error) { func (r *rpcClient) stream(ctx context.Context, route *router.Route, req Request, opts CallOptions) (Stream, error) {
address := node.Address
msg := &transport.Message{ msg := &transport.Message{
Header: make(map[string]string), Header: make(map[string]string),
} }
@ -206,7 +252,7 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
msg.Header["Accept"] = req.ContentType() msg.Header["Accept"] = req.ContentType()
// set old codecs // set old codecs
cf := setupProtocol(msg, node) cf := setupProtocol(msg, route)
// no codec specified // no codec specified
if cf == nil { if cf == nil {
@ -225,7 +271,7 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
dOpts = append(dOpts, transport.WithTimeout(opts.DialTimeout)) dOpts = append(dOpts, transport.WithTimeout(opts.DialTimeout))
} }
c, err := r.opts.Transport.Dial(address, dOpts...) c, err := r.opts.Transport.Dial(route.Address, dOpts...)
if err != nil { if err != nil {
return nil, errors.InternalServerError("go.micro.client", "connection error: %v", err) return nil, errors.InternalServerError("go.micro.client", "connection error: %v", err)
} }
@ -320,43 +366,6 @@ func (r *rpcClient) Options() Options {
return r.opts return r.opts
} }
// next returns an iterator for the next nodes to call
func (r *rpcClient) next(request Request, opts CallOptions) (selector.Next, error) {
// try get the proxy
service, address, _ := net.Proxy(request.Service(), opts.Address)
// return remote address
if len(address) > 0 {
nodes := make([]*registry.Node, len(address))
for i, addr := range address {
nodes[i] = &registry.Node{
Address: addr,
// Set the protocol
Metadata: map[string]string{
"protocol": "mucp",
},
}
}
// crude return method
return func() (*registry.Node, error) {
return nodes[time.Now().Unix()%int64(len(nodes))], nil
}, nil
}
// get next nodes from the selector
next, err := r.opts.Selector.Select(service, opts.SelectOptions...)
if err != nil {
if err == selector.ErrNotFound {
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
}
return nil, errors.InternalServerError("go.micro.client", "error selecting %s node: %s", service, err.Error())
}
return next, nil
}
func (r *rpcClient) Call(ctx context.Context, request Request, response interface{}, opts ...CallOption) error { func (r *rpcClient) Call(ctx context.Context, request Request, response interface{}, opts ...CallOption) error {
// make a copy of call opts // make a copy of call opts
callOpts := r.opts.CallOptions callOpts := r.opts.CallOptions
@ -364,14 +373,8 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
opt(&callOpts) opt(&callOpts)
} }
next, err := r.next(request, callOpts)
if err != nil {
return err
}
// check if we already have a deadline // check if we already have a deadline
d, ok := ctx.Deadline() if d, ok := ctx.Deadline(); !ok {
if !ok {
// no deadline so we create a new one // no deadline so we create a new one
var cancel context.CancelFunc var cancel context.CancelFunc
ctx, cancel = context.WithTimeout(ctx, callOpts.RequestTimeout) ctx, cancel = context.WithTimeout(ctx, callOpts.RequestTimeout)
@ -379,8 +382,8 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
} else { } else {
// got a deadline so no need to setup context // got a deadline so no need to setup context
// but we need to set the timeout we pass along // but we need to set the timeout we pass along
opt := WithRequestTimeout(d.Sub(time.Now())) remaining := d.Sub(time.Now())
opt(&callOpts) WithRequestTimeout(remaining)(&callOpts)
} }
// should we noop right here? // should we noop right here?
@ -411,19 +414,18 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
time.Sleep(t) time.Sleep(t)
} }
// select next node // lookup the route to send the request via
node, err := next() route, err := r.lookupRoute(request, callOpts)
service := request.Service()
if err != nil { if err != nil {
if err == selector.ErrNotFound { return err
return errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
}
return errors.InternalServerError("go.micro.client", "error getting next %s node: %s", service, err.Error())
} }
// make the call // make the call
err = rcall(ctx, node, request, response, callOpts) err = rcall(ctx, route, request, response, callOpts)
r.opts.Selector.Mark(service, node, err)
// record the result of the call to inform future routing decisions
r.opts.Selector.Record(*route, err)
return err return err
} }
@ -475,11 +477,6 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
opt(&callOpts) opt(&callOpts)
} }
next, err := r.next(request, callOpts)
if err != nil {
return nil, err
}
// should we noop right here? // should we noop right here?
select { select {
case <-ctx.Done(): case <-ctx.Done():
@ -499,17 +496,18 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
time.Sleep(t) time.Sleep(t)
} }
node, err := next() // lookup the route to send the request via
service := request.Service() route, err := r.lookupRoute(request, callOpts)
if err != nil { if err != nil {
if err == selector.ErrNotFound { return nil, err
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
}
return nil, errors.InternalServerError("go.micro.client", "error getting next %s node: %s", service, err.Error())
} }
stream, err := r.stream(ctx, node, request, callOpts) // perform the call
r.opts.Selector.Mark(service, node, err) stream, err := r.stream(ctx, route, request, callOpts)
// record the result of the call to inform future routing decisions
r.opts.Selector.Record(*route, err)
return stream, err return stream, err
} }

View File

@ -5,10 +5,10 @@ import (
"fmt" "fmt"
"testing" "testing"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/registry/memory" "github.com/micro/go-micro/v2/registry/memory"
"github.com/micro/go-micro/v2/router"
) )
func newTestRegistry() registry.Registry { func newTestRegistry() registry.Registry {
@ -22,7 +22,7 @@ func TestCallAddress(t *testing.T) {
address := "10.1.10.1:8080" address := "10.1.10.1:8080"
wrap := func(cf CallFunc) CallFunc { wrap := func(cf CallFunc) CallFunc {
return func(ctx context.Context, node *registry.Node, req Request, rsp interface{}, opts CallOptions) error { return func(ctx context.Context, route *router.Route, req Request, rsp interface{}, opts CallOptions) error {
called = true called = true
if req.Service() != service { if req.Service() != service {
@ -33,8 +33,8 @@ func TestCallAddress(t *testing.T) {
return fmt.Errorf("expected service: %s got %s", endpoint, req.Endpoint()) return fmt.Errorf("expected service: %s got %s", endpoint, req.Endpoint())
} }
if node.Address != address { if route.Address != address {
return fmt.Errorf("expected address: %s got %s", address, node.Address) return fmt.Errorf("expected address: %s got %s", address, route.Address)
} }
// don't do the call // don't do the call
@ -47,7 +47,6 @@ func TestCallAddress(t *testing.T) {
Registry(r), Registry(r),
WrapCall(wrap), WrapCall(wrap),
) )
c.Options().Selector.Init(selector.Registry(r))
req := c.NewRequest(service, endpoint, nil) req := c.NewRequest(service, endpoint, nil)
@ -70,7 +69,7 @@ func TestCallRetry(t *testing.T) {
var called int var called int
wrap := func(cf CallFunc) CallFunc { wrap := func(cf CallFunc) CallFunc {
return func(ctx context.Context, node *registry.Node, req Request, rsp interface{}, opts CallOptions) error { return func(ctx context.Context, route *router.Route, req Request, rsp interface{}, opts CallOptions) error {
called++ called++
if called == 1 { if called == 1 {
return errors.InternalServerError("test.error", "retry request") return errors.InternalServerError("test.error", "retry request")
@ -86,7 +85,6 @@ func TestCallRetry(t *testing.T) {
Registry(r), Registry(r),
WrapCall(wrap), WrapCall(wrap),
) )
c.Options().Selector.Init(selector.Registry(r))
req := c.NewRequest(service, endpoint, nil) req := c.NewRequest(service, endpoint, nil)
@ -109,7 +107,7 @@ func TestCallWrapper(t *testing.T) {
address := "10.1.10.1:8080" address := "10.1.10.1:8080"
wrap := func(cf CallFunc) CallFunc { wrap := func(cf CallFunc) CallFunc {
return func(ctx context.Context, node *registry.Node, req Request, rsp interface{}, opts CallOptions) error { return func(ctx context.Context, route *router.Route, req Request, rsp interface{}, opts CallOptions) error {
called = true called = true
if req.Service() != service { if req.Service() != service {
@ -120,8 +118,8 @@ func TestCallWrapper(t *testing.T) {
return fmt.Errorf("expected service: %s got %s", endpoint, req.Endpoint()) return fmt.Errorf("expected service: %s got %s", endpoint, req.Endpoint())
} }
if node.Address != address { if route.Address != address {
return fmt.Errorf("expected address: %s got %s", address, node.Address) return fmt.Errorf("expected address: %s got %s", address, route.Address)
} }
// don't do the call // don't do the call
@ -134,7 +132,6 @@ func TestCallWrapper(t *testing.T) {
Registry(r), Registry(r),
WrapCall(wrap), WrapCall(wrap),
) )
c.Options().Selector.Init(selector.Registry(r))
r.Register(&registry.Service{ r.Register(&registry.Service{
Name: service, Name: service,

View File

@ -12,7 +12,7 @@ import (
"github.com/micro/go-micro/v2/codec/proto" "github.com/micro/go-micro/v2/codec/proto"
"github.com/micro/go-micro/v2/codec/protorpc" "github.com/micro/go-micro/v2/codec/protorpc"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"
) )
@ -128,11 +128,9 @@ func setHeaders(m *codec.Message, stream string) {
} }
// setupProtocol sets up the old protocol // setupProtocol sets up the old protocol
func setupProtocol(msg *transport.Message, node *registry.Node) codec.NewCodec { func setupProtocol(msg *transport.Message, route *router.Route) codec.NewCodec {
protocol := node.Metadata["protocol"] // get the protocol from route metadata
if protocol := route.Metadata["protocol"]; len(protocol) > 0 {
// got protocol
if len(protocol) > 0 {
return nil return nil
} }

View File

@ -1,47 +0,0 @@
package selector
import (
"github.com/micro/go-micro/v2/registry"
)
var (
// mock data
testData = map[string][]*registry.Service{
"foo": {
{
Name: "foo",
Version: "1.0.0",
Nodes: []*registry.Node{
{
Id: "foo-1.0.0-123",
Address: "localhost:9999",
},
{
Id: "foo-1.0.0-321",
Address: "localhost:9999",
},
},
},
{
Name: "foo",
Version: "1.0.1",
Nodes: []*registry.Node{
{
Id: "foo-1.0.1-321",
Address: "localhost:6666",
},
},
},
{
Name: "foo",
Version: "1.0.3",
Nodes: []*registry.Node{
{
Id: "foo-1.0.3-345",
Address: "localhost:8888",
},
},
},
},
}
)

View File

@ -1,132 +0,0 @@
package selector
import (
"time"
"github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/registry/cache"
)
type registrySelector struct {
so Options
rc cache.Cache
}
func (c *registrySelector) newCache() cache.Cache {
ropts := []cache.Option{}
if c.so.Context != nil {
if t, ok := c.so.Context.Value("selector_ttl").(time.Duration); ok {
ropts = append(ropts, cache.WithTTL(t))
}
}
return cache.New(c.so.Registry, ropts...)
}
func (c *registrySelector) Init(opts ...Option) error {
for _, o := range opts {
o(&c.so)
}
c.rc.Stop()
c.rc = c.newCache()
return nil
}
func (c *registrySelector) Options() Options {
return c.so
}
func (c *registrySelector) Select(service string, opts ...SelectOption) (Next, error) {
sopts := SelectOptions{Strategy: c.so.Strategy}
for _, opt := range opts {
opt(&sopts)
}
// a specific domain was requested, only lookup the services in that domain
if len(sopts.Domain) > 0 {
services, err := c.rc.GetService(service, registry.GetDomain(sopts.Domain))
if err != nil && err != registry.ErrNotFound {
return nil, err
}
for _, filter := range sopts.Filters {
services = filter(services)
}
if len(services) == 0 {
return nil, ErrNoneAvailable
}
return sopts.Strategy(services), nil
}
// get the service. Because the service could be running in the current or the default domain,
// we call both. For example, go.micro.service.foo could be running in the services current domain,
// however the runtime (go.micro.runtime) will always be run in the default domain.
services, err := c.rc.GetService(service, registry.GetDomain(c.so.Domain))
if err != nil && err != registry.ErrNotFound {
return nil, err
}
if c.so.Domain != registry.DefaultDomain {
srvs, err := c.rc.GetService(service, registry.GetDomain(registry.DefaultDomain))
if err != nil && err != registry.ErrNotFound {
return nil, err
}
if err == nil {
services = append(services, srvs...)
}
}
if services == nil {
return nil, ErrNoneAvailable
}
// apply the filters
for _, filter := range sopts.Filters {
services = filter(services)
}
// if there's nothing left, return
if len(services) == 0 {
return nil, ErrNoneAvailable
}
return sopts.Strategy(services), nil
}
func (c *registrySelector) Mark(service string, node *registry.Node, err error) {
}
func (c *registrySelector) Reset(service string) {
}
// Close stops the watcher and destroys the cache
func (c *registrySelector) Close() error {
c.rc.Stop()
return nil
}
func (c *registrySelector) String() string {
return "registry"
}
func NewSelector(opts ...Option) Selector {
sopts := Options{
Strategy: Random,
}
for _, opt := range opts {
opt(&sopts)
}
if sopts.Registry == nil {
sopts.Registry = registry.DefaultRegistry
}
s := &registrySelector{
so: sopts,
}
s.rc = s.newCache()
return s
}

View File

@ -1,32 +0,0 @@
package selector
import (
"os"
"testing"
"github.com/micro/go-micro/v2/registry/memory"
)
func TestRegistrySelector(t *testing.T) {
counts := map[string]int{}
r := memory.NewRegistry(memory.Services(testData))
cache := NewSelector(Registry(r))
next, err := cache.Select("foo")
if err != nil {
t.Errorf("Unexpected error calling cache select: %v", err)
}
for i := 0; i < 100; i++ {
node, err := next()
if err != nil {
t.Errorf("Expected node err, got err: %v", err)
}
counts[node.Id]++
}
if len(os.Getenv("IN_TRAVIS_CI")) == 0 {
t.Logf("Selector Counts %v", counts)
}
}

View File

@ -1,124 +0,0 @@
// Package dns provides a dns SRV selector
package dns
import (
"fmt"
"net"
"strconv"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/registry"
)
type dnsSelector struct {
options selector.Options
domain string
}
var (
DefaultDomain = "local"
)
func (d *dnsSelector) Init(opts ...selector.Option) error {
for _, o := range opts {
o(&d.options)
}
return nil
}
func (d *dnsSelector) Options() selector.Options {
return d.options
}
func (d *dnsSelector) Select(service string, opts ...selector.SelectOption) (selector.Next, error) {
var srv []*net.SRV
// check if its host:port
host, port, err := net.SplitHostPort(service)
// not host:port
if err != nil {
// lookup the SRV record
_, srvs, err := net.LookupSRV(service, "tcp", d.domain)
if err != nil {
return nil, err
}
// set SRV records
srv = srvs
// got host:port
} else {
p, _ := strconv.Atoi(port)
// lookup the A record
ips, err := net.LookupHost(host)
if err != nil {
return nil, err
}
// create SRV records
for _, ip := range ips {
srv = append(srv, &net.SRV{
Target: ip,
Port: uint16(p),
})
}
}
nodes := make([]*registry.Node, 0, len(srv))
for _, node := range srv {
nodes = append(nodes, &registry.Node{
Id: node.Target,
Address: fmt.Sprintf("%s:%d", node.Target, node.Port),
})
}
services := []*registry.Service{
{
Name: service,
Nodes: nodes,
},
}
sopts := selector.SelectOptions{
Strategy: d.options.Strategy,
}
for _, opt := range opts {
opt(&sopts)
}
// apply the filters
for _, filter := range sopts.Filters {
services = filter(services)
}
// if there's nothing left, return
if len(services) == 0 {
return nil, selector.ErrNoneAvailable
}
return sopts.Strategy(services), nil
}
func (d *dnsSelector) Mark(service string, node *registry.Node, err error) {}
func (d *dnsSelector) Reset(service string) {}
func (d *dnsSelector) Close() error {
return nil
}
func (d *dnsSelector) String() string {
return "dns"
}
func NewSelector(opts ...selector.Option) selector.Selector {
options := selector.Options{
Strategy: selector.Random,
}
for _, o := range opts {
o(&options)
}
return &dnsSelector{options: options, domain: DefaultDomain}
}

View File

@ -1,73 +0,0 @@
package selector
import (
"github.com/micro/go-micro/v2/registry"
)
// FilterEndpoint is an endpoint based Select Filter which will
// only return services with the endpoint specified.
func FilterEndpoint(name string) Filter {
return func(old []*registry.Service) []*registry.Service {
var services []*registry.Service
for _, service := range old {
for _, ep := range service.Endpoints {
if ep.Name == name {
services = append(services, service)
break
}
}
}
return services
}
}
// FilterLabel is a label based Select Filter which will
// only return services with the label specified.
func FilterLabel(key, val string) Filter {
return func(old []*registry.Service) []*registry.Service {
var services []*registry.Service
for _, service := range old {
serv := new(registry.Service)
var nodes []*registry.Node
for _, node := range service.Nodes {
if node.Metadata == nil {
continue
}
if node.Metadata[key] == val {
nodes = append(nodes, node)
}
}
// only add service if there's some nodes
if len(nodes) > 0 {
// copy
*serv = *service
serv.Nodes = nodes
services = append(services, serv)
}
}
return services
}
}
// FilterVersion is a version based Select Filter which will
// only return services with the version specified.
func FilterVersion(version string) Filter {
return func(old []*registry.Service) []*registry.Service {
var services []*registry.Service
for _, service := range old {
if service.Version == version {
services = append(services, service)
}
}
return services
}
}

View File

@ -1,239 +0,0 @@
package selector
import (
"testing"
"github.com/micro/go-micro/v2/registry"
)
func TestFilterEndpoint(t *testing.T) {
testData := []struct {
services []*registry.Service
endpoint string
count int
}{
{
services: []*registry.Service{
{
Name: "test",
Version: "1.0.0",
Endpoints: []*registry.Endpoint{
{
Name: "Foo.Bar",
},
},
},
{
Name: "test",
Version: "1.1.0",
Endpoints: []*registry.Endpoint{
{
Name: "Baz.Bar",
},
},
},
},
endpoint: "Foo.Bar",
count: 1,
},
{
services: []*registry.Service{
{
Name: "test",
Version: "1.0.0",
Endpoints: []*registry.Endpoint{
{
Name: "Foo.Bar",
},
},
},
{
Name: "test",
Version: "1.1.0",
Endpoints: []*registry.Endpoint{
{
Name: "Foo.Bar",
},
},
},
},
endpoint: "Bar.Baz",
count: 0,
},
}
for _, data := range testData {
filter := FilterEndpoint(data.endpoint)
services := filter(data.services)
if len(services) != data.count {
t.Fatalf("Expected %d services, got %d", data.count, len(services))
}
for _, service := range services {
var seen bool
for _, ep := range service.Endpoints {
if ep.Name == data.endpoint {
seen = true
break
}
}
if !seen && data.count > 0 {
t.Fatalf("Expected %d services but seen is %t; result %+v", data.count, seen, services)
}
}
}
}
func TestFilterLabel(t *testing.T) {
testData := []struct {
services []*registry.Service
label [2]string
count int
}{
{
services: []*registry.Service{
{
Name: "test",
Version: "1.0.0",
Nodes: []*registry.Node{
{
Id: "test-1",
Address: "localhost",
Metadata: map[string]string{
"foo": "bar",
},
},
},
},
{
Name: "test",
Version: "1.1.0",
Nodes: []*registry.Node{
{
Id: "test-2",
Address: "localhost",
Metadata: map[string]string{
"foo": "baz",
},
},
},
},
},
label: [2]string{"foo", "bar"},
count: 1,
},
{
services: []*registry.Service{
{
Name: "test",
Version: "1.0.0",
Nodes: []*registry.Node{
{
Id: "test-1",
Address: "localhost",
},
},
},
{
Name: "test",
Version: "1.1.0",
Nodes: []*registry.Node{
{
Id: "test-2",
Address: "localhost",
},
},
},
},
label: [2]string{"foo", "bar"},
count: 0,
},
}
for _, data := range testData {
filter := FilterLabel(data.label[0], data.label[1])
services := filter(data.services)
if len(services) != data.count {
t.Fatalf("Expected %d services, got %d", data.count, len(services))
}
for _, service := range services {
var seen bool
for _, node := range service.Nodes {
if node.Metadata[data.label[0]] != data.label[1] {
t.Fatalf("Expected %s=%s but got %s=%s for service %+v node %+v",
data.label[0], data.label[1], data.label[0], node.Metadata[data.label[0]], service, node)
}
seen = true
}
if !seen {
t.Fatalf("Expected node for %s=%s but saw none; results %+v", data.label[0], data.label[1], service)
}
}
}
}
func TestFilterVersion(t *testing.T) {
testData := []struct {
services []*registry.Service
version string
count int
}{
{
services: []*registry.Service{
{
Name: "test",
Version: "1.0.0",
},
{
Name: "test",
Version: "1.1.0",
},
},
version: "1.0.0",
count: 1,
},
{
services: []*registry.Service{
{
Name: "test",
Version: "1.0.0",
},
{
Name: "test",
Version: "1.1.0",
},
},
version: "2.0.0",
count: 0,
},
}
for _, data := range testData {
filter := FilterVersion(data.version)
services := filter(data.services)
if len(services) != data.count {
t.Fatalf("Expected %d services, got %d", data.count, len(services))
}
var seen bool
for _, service := range services {
if service.Version != data.version {
t.Fatalf("Expected version %s, got %s", data.version, service.Version)
}
seen = true
}
if !seen && data.count > 0 {
t.Fatalf("Expected %d services but seen is %t; result %+v", data.count, seen, services)
}
}
}

View File

@ -1,78 +0,0 @@
package selector
import (
"context"
"github.com/micro/go-micro/v2/registry"
)
type Options struct {
Registry registry.Registry
Strategy Strategy
// Domain to lookup services from within the registry
Domain string
// Other options for implementations of the interface
// can be stored in a context
Context context.Context
}
type SelectOptions struct {
Filters []Filter
Strategy Strategy
Domain string
// Other options for implementations of the interface
// can be stored in a context
Context context.Context
}
// Option used to initialise the selector
type Option func(*Options)
// SelectOption used when making a select call
type SelectOption func(*SelectOptions)
// Registry sets the registry used by the selector
func Registry(r registry.Registry) Option {
return func(o *Options) {
o.Registry = r
}
}
// Domain sets the domain used by the selector
func Domain(d string) Option {
return func(o *Options) {
o.Domain = d
}
}
// SetStrategy sets the default strategy for the selector
func SetStrategy(fn Strategy) Option {
return func(o *Options) {
o.Strategy = fn
}
}
// WithFilter adds a filter function to the list of filters
// used during the Select call.
func WithFilter(fn ...Filter) SelectOption {
return func(o *SelectOptions) {
o.Filters = append(o.Filters, fn...)
}
}
// Strategy sets the selector strategy
func WithStrategy(fn Strategy) SelectOption {
return func(o *SelectOptions) {
o.Strategy = fn
}
}
// WithDomain sets the registry domain to use for the selection
func WithDomain(d string) SelectOption {
return func(o *SelectOptions) {
o.Domain = d
}
}

View File

@ -1,18 +0,0 @@
package registry
import (
"context"
"time"
"github.com/micro/go-micro/v2/client/selector"
)
// Set the registry cache ttl
func TTL(t time.Duration) selector.Option {
return func(o *selector.Options) {
if o.Context == nil {
o.Context = context.Background()
}
o.Context = context.WithValue(o.Context, "selector_ttl", t)
}
}

View File

@ -1,11 +0,0 @@
// Package registry uses the go-micro registry for selection
package registry
import (
"github.com/micro/go-micro/v2/client/selector"
)
// NewSelector returns a new registry selector
func NewSelector(opts ...selector.Option) selector.Selector {
return selector.NewSelector(opts...)
}

View File

@ -1,270 +0,0 @@
// Package router is a network/router selector
package router
import (
"context"
"os"
"sort"
"sync"
"github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/router"
pb "github.com/micro/go-micro/v2/router/service/proto"
)
type routerSelector struct {
opts selector.Options
// the router
r router.Router
// the client we have
c client.Client
// the client for the remote router
rs pb.RouterService
// name of the router
name string
// address of the remote router
addr string
// whether to use the remote router
remote bool
}
type clientKey struct{}
type routerKey struct{}
// getRoutes returns the routes whether they are remote or local
func (r *routerSelector) getRoutes(service string) ([]router.Route, error) {
if !r.remote {
// lookup router for routes for the service
return r.r.Lookup(
router.QueryService(service),
)
}
// lookup the remote router
var addrs []string
// set the remote address if specified
if len(r.addr) > 0 {
addrs = append(addrs, r.addr)
} else {
// we have a name so we need to check the registry
services, err := r.c.Options().Registry.GetService(r.name)
if err != nil {
return nil, err
}
for _, service := range services {
for _, node := range service.Nodes {
addrs = append(addrs, node.Address)
}
}
}
// no router addresses available
if len(addrs) == 0 {
return nil, selector.ErrNoneAvailable
}
var pbRoutes *pb.LookupResponse
var err error
// TODO: implement backoff and retries
for _, addr := range addrs {
// call the router
pbRoutes, err = r.rs.Lookup(context.Background(), &pb.LookupRequest{
Query: &pb.Query{
Service: service,
},
}, client.WithAddress(addr))
if err != nil {
continue
}
break
}
// errored out
if err != nil {
return nil, err
}
// no routes
if pbRoutes == nil {
return nil, selector.ErrNoneAvailable
}
routes := make([]router.Route, 0, len(pbRoutes.Routes))
// convert from pb to []*router.Route
for _, r := range pbRoutes.Routes {
routes = append(routes, router.Route{
Service: r.Service,
Address: r.Address,
Gateway: r.Gateway,
Network: r.Network,
Link: r.Link,
Metric: r.Metric,
})
}
return routes, nil
}
func (r *routerSelector) Init(opts ...selector.Option) error {
// no op
return nil
}
func (r *routerSelector) Options() selector.Options {
return r.opts
}
func (r *routerSelector) Select(service string, opts ...selector.SelectOption) (selector.Next, error) {
// TODO: pull routes asynchronously and cache
routes, err := r.getRoutes(service)
if err != nil {
return nil, err
}
// no routes return not found error
if len(routes) == 0 {
return nil, selector.ErrNotFound
}
// TODO: apply filters by pseudo constructing service
// sort the routes based on metric
sort.Slice(routes, func(i, j int) bool {
return routes[i].Metric < routes[j].Metric
})
// roundrobin assuming routes are in metric preference order
var i int
var mtx sync.Mutex
return func() (*registry.Node, error) {
// get index and increment counter with every call to next
mtx.Lock()
idx := i
i++
mtx.Unlock()
// get route based on idx
route := routes[idx%len(routes)]
// defaults to gateway and no port
address := route.Address
if len(route.Gateway) > 0 {
address = route.Gateway
}
// return as a node
return &registry.Node{
// TODO: add id and metadata if we can
Address: address,
}, nil
}, nil
}
func (r *routerSelector) Mark(service string, node *registry.Node, err error) {
// TODO: pass back metrics or information to the router
}
func (r *routerSelector) Reset(service string) {
// TODO: reset the metrics or information at the router
}
func (r *routerSelector) Close() error {
// stop the router advertisements
return r.r.Close()
}
func (r *routerSelector) String() string {
return "router"
}
// NewSelector returns a new router based selector
func NewSelector(opts ...selector.Option) selector.Selector {
options := selector.Options{
Context: context.Background(),
}
for _, o := range opts {
o(&options)
}
// set default registry if not set
if options.Registry == nil {
options.Registry = registry.DefaultRegistry
}
// try get router from the context
r, ok := options.Context.Value(routerKey{}).(router.Router)
if !ok {
// TODO: Use router.DefaultRouter?
r = router.NewRouter(
router.Registry(options.Registry),
)
}
// try get client from the context
c, ok := options.Context.Value(clientKey{}).(client.Client)
if !ok {
c = client.DefaultClient
}
// get the router from env vars if its a remote service
remote := true
routerName := os.Getenv("MICRO_ROUTER")
routerAddress := os.Getenv("MICRO_ROUTER_ADDRESS")
// start the router advertisements if we're running it locally
if len(routerName) == 0 && len(routerAddress) == 0 {
go r.Advertise()
remote = false
}
return &routerSelector{
opts: options,
// set the internal router
r: r,
// set the client
c: c,
// set the router client
rs: pb.NewRouterService(routerName, c),
// name of the router
name: routerName,
// address of router
addr: routerAddress,
// let ourselves know to use the remote router
remote: remote,
}
}
// WithClient sets the client for the request
func WithClient(c client.Client) selector.Option {
return func(o *selector.Options) {
if o.Context == nil {
o.Context = context.Background()
}
o.Context = context.WithValue(o.Context, clientKey{}, c)
}
}
// WithRouter sets the router as an option
func WithRouter(r router.Router) selector.Option {
return func(o *selector.Options) {
if o.Context == nil {
o.Context = context.Background()
}
o.Context = context.WithValue(o.Context, routerKey{}, r)
}
}

View File

@ -1,43 +0,0 @@
// Package selector is a way to pick a list of service nodes
package selector
import (
"errors"
"github.com/micro/go-micro/v2/registry"
)
// Selector builds on the registry as a mechanism to pick nodes
// and mark their status. This allows host pools and other things
// to be built using various algorithms.
type Selector interface {
Init(opts ...Option) error
Options() Options
// Select returns a function which should return the next node
Select(service string, opts ...SelectOption) (Next, error)
// Mark sets the success/error against a node
Mark(service string, node *registry.Node, err error)
// Reset returns state back to zero for a service
Reset(service string)
// Close renders the selector unusable
Close() error
// Name of the selector
String() string
}
// Next is a function that returns the next node
// based on the selector's strategy
type Next func() (*registry.Node, error)
// Filter is used to filter a service during the selection process
type Filter func([]*registry.Service) []*registry.Service
// Strategy is a selection strategy e.g random, round robin
type Strategy func([]*registry.Service) Next
var (
DefaultSelector = NewSelector()
ErrNotFound = errors.New("not found")
ErrNoneAvailable = errors.New("none available")
)

View File

@ -1,58 +0,0 @@
// Package static provides a static resolver which returns the name/ip passed in without any change
package static
import (
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/registry"
)
// staticSelector is a static selector
type staticSelector struct {
opts selector.Options
}
func (s *staticSelector) Init(opts ...selector.Option) error {
for _, o := range opts {
o(&s.opts)
}
return nil
}
func (s *staticSelector) Options() selector.Options {
return s.opts
}
func (s *staticSelector) Select(service string, opts ...selector.SelectOption) (selector.Next, error) {
return func() (*registry.Node, error) {
return &registry.Node{
Id: service,
Address: service,
}, nil
}, nil
}
func (s *staticSelector) Mark(service string, node *registry.Node, err error) {
return
}
func (s *staticSelector) Reset(service string) {
return
}
func (s *staticSelector) Close() error {
return nil
}
func (s *staticSelector) String() string {
return "static"
}
func NewSelector(opts ...selector.Option) selector.Selector {
var options selector.Options
for _, o := range opts {
o(&options)
}
return &staticSelector{
opts: options,
}
}

View File

@ -1,56 +0,0 @@
package selector
import (
"math/rand"
"sync"
"time"
"github.com/micro/go-micro/v2/registry"
)
func init() {
rand.Seed(time.Now().UnixNano())
}
// Random is a random strategy algorithm for node selection
func Random(services []*registry.Service) Next {
nodes := make([]*registry.Node, 0, len(services))
for _, service := range services {
nodes = append(nodes, service.Nodes...)
}
return func() (*registry.Node, error) {
if len(nodes) == 0 {
return nil, ErrNoneAvailable
}
i := rand.Int() % len(nodes)
return nodes[i], nil
}
}
// RoundRobin is a roundrobin strategy algorithm for node selection
func RoundRobin(services []*registry.Service) Next {
nodes := make([]*registry.Node, 0, len(services))
for _, service := range services {
nodes = append(nodes, service.Nodes...)
}
var i = rand.Int()
var mtx sync.Mutex
return func() (*registry.Node, error) {
if len(nodes) == 0 {
return nil, ErrNoneAvailable
}
mtx.Lock()
node := nodes[i%len(nodes)]
i++
mtx.Unlock()
return node, nil
}
}

View File

@ -1,58 +0,0 @@
package selector
import (
"os"
"testing"
"github.com/micro/go-micro/v2/registry"
)
func TestStrategies(t *testing.T) {
testData := []*registry.Service{
{
Name: "test1",
Version: "latest",
Nodes: []*registry.Node{
{
Id: "test1-1",
Address: "10.0.0.1:1001",
},
{
Id: "test1-2",
Address: "10.0.0.2:1002",
},
},
},
{
Name: "test1",
Version: "default",
Nodes: []*registry.Node{
{
Id: "test1-3",
Address: "10.0.0.3:1003",
},
{
Id: "test1-4",
Address: "10.0.0.4:1004",
},
},
},
}
for name, strategy := range map[string]Strategy{"random": Random, "roundrobin": RoundRobin} {
next := strategy(testData)
counts := make(map[string]int)
for i := 0; i < 100; i++ {
node, err := next()
if err != nil {
t.Fatal(err)
}
counts[node.Id]++
}
if len(os.Getenv("IN_TRAVIS_CI")) == 0 {
t.Logf("%s: %+v\n", name, counts)
}
}
}

View File

@ -3,11 +3,11 @@ package client
import ( import (
"context" "context"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/router"
) )
// CallFunc represents the individual call func // CallFunc represents the individual call func
type CallFunc func(ctx context.Context, node *registry.Node, req Request, rsp interface{}, opts CallOptions) error type CallFunc func(ctx context.Context, route *router.Route, req Request, rsp interface{}, opts CallOptions) error
// CallWrapper is a low level wrapper for the CallFunc // CallWrapper is a low level wrapper for the CallFunc
type CallWrapper func(CallFunc) CallFunc type CallWrapper func(CallFunc) CallFunc

View File

@ -12,7 +12,6 @@ import (
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/grpc" "github.com/micro/go-micro/v2/client/grpc"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/config" "github.com/micro/go-micro/v2/config"
configSrc "github.com/micro/go-micro/v2/config/source" configSrc "github.com/micro/go-micro/v2/config/source"
configSrv "github.com/micro/go-micro/v2/config/source/service" configSrv "github.com/micro/go-micro/v2/config/source/service"
@ -25,6 +24,7 @@ import (
registrySrv "github.com/micro/go-micro/v2/registry/service" registrySrv "github.com/micro/go-micro/v2/registry/service"
"github.com/micro/go-micro/v2/router" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/runtime" "github.com/micro/go-micro/v2/runtime"
"github.com/micro/go-micro/v2/selector"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
"github.com/micro/go-micro/v2/store" "github.com/micro/go-micro/v2/store"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"
@ -65,9 +65,8 @@ import (
srvRuntime "github.com/micro/go-micro/v2/runtime/service" srvRuntime "github.com/micro/go-micro/v2/runtime/service"
// selectors // selectors
"github.com/micro/go-micro/v2/client/selector/dns" randSelector "github.com/micro/go-micro/v2/selector/random"
sRouter "github.com/micro/go-micro/v2/client/selector/router" roundSelector "github.com/micro/go-micro/v2/selector/roundrobin"
"github.com/micro/go-micro/v2/client/selector/static"
// transports // transports
thttp "github.com/micro/go-micro/v2/transport/http" thttp "github.com/micro/go-micro/v2/transport/http"
@ -377,9 +376,8 @@ var (
} }
DefaultSelectors = map[string]func(...selector.Option) selector.Selector{ DefaultSelectors = map[string]func(...selector.Option) selector.Selector{
"dns": dns.NewSelector, "random": randSelector.NewSelector,
"router": sRouter.NewSelector, "roundrobin": roundSelector.NewSelector,
"static": static.NewSelector,
} }
DefaultServers = map[string]func(...server.Option) server.Server{ DefaultServers = map[string]func(...server.Option) server.Server{
@ -644,9 +642,6 @@ func (c *cmd) Before(ctx *cli.Context) error {
return err return err
} }
// Setup selector options
selectorOpts := []selector.Option{selector.Registry(*c.opts.Registry)}
// Setup broker options. // Setup broker options.
brokerOpts := []broker.Option{} brokerOpts := []broker.Option{}
if len(ctx.String("broker_address")) > 0 { if len(ctx.String("broker_address")) > 0 {
@ -669,15 +664,25 @@ func (c *cmd) Before(ctx *cli.Context) error {
*c.opts.Registry = r(registryOpts...) *c.opts.Registry = r(registryOpts...)
serverOpts = append(serverOpts, server.Registry(*c.opts.Registry)) serverOpts = append(serverOpts, server.Registry(*c.opts.Registry))
clientOpts = append(clientOpts, client.Registry(*c.opts.Registry))
brokerOpts = append(brokerOpts, broker.Registry(*c.opts.Registry)) brokerOpts = append(brokerOpts, broker.Registry(*c.opts.Registry))
selectorOpts = append(selectorOpts, selector.Registry(*c.opts.Registry))
} else if len(registryOpts) > 0 { } else if len(registryOpts) > 0 {
if err := (*c.opts.Registry).Init(registryOpts...); err != nil { if err := (*c.opts.Registry).Init(registryOpts...); err != nil {
logger.Fatalf("Error configuring registry: %v", err) logger.Fatalf("Error configuring registry: %v", err)
} }
} }
// Add support for legacy selectors until v3.
if ctx.String("selector") == "static" {
ctx.Set("router", "static")
ctx.Set("selector", "")
logger.Warnf("DEPRECATION WARNING: router/static now provides static routing, use '--router=static'. Support for the static selector flag will be removed in v3.")
}
if ctx.String("selector") == "dns" {
ctx.Set("router", "dns")
ctx.Set("selector", "")
logger.Warnf("DEPRECATION WARNING: router/dns now provides dns routing, use '--router=dns'. Support for the dns selector flag will be removed in v3.")
}
// Set the selector // Set the selector
if name := ctx.String("selector"); len(name) > 0 && (*c.opts.Selector).String() != name { if name := ctx.String("selector"); len(name) > 0 && (*c.opts.Selector).String() != name {
s, ok := c.opts.Selectors[name] s, ok := c.opts.Selectors[name]
@ -685,12 +690,8 @@ func (c *cmd) Before(ctx *cli.Context) error {
logger.Fatalf("Selector %s not found", name) logger.Fatalf("Selector %s not found", name)
} }
*c.opts.Selector = s(selectorOpts...) *c.opts.Selector = s()
clientOpts = append(clientOpts, client.Selector(*c.opts.Selector)) clientOpts = append(clientOpts, client.Selector(*c.opts.Selector))
} else if len(selectorOpts) > 0 {
if err := (*c.opts.Selector).Init(selectorOpts...); err != nil {
logger.Fatalf("Error configuring selctor: %v", err)
}
} }
// Set the router, this must happen before the rest of the server as it'll route server requests // Set the router, this must happen before the rest of the server as it'll route server requests
@ -716,8 +717,7 @@ func (c *cmd) Before(ctx *cli.Context) error {
} }
*c.opts.Router = r(routerOpts...) *c.opts.Router = r(routerOpts...)
// todo: set the router in the client clientOpts = append(clientOpts, client.Router(*c.opts.Router))
// clientOpts = append(clientOpts, client.Router(*c.opts.Router))
} else if len(routerOpts) > 0 { } else if len(routerOpts) > 0 {
if err := (*c.opts.Router).Init(routerOpts...); err != nil { if err := (*c.opts.Router).Init(routerOpts...); err != nil {
logger.Fatalf("Error configuring router: %v", err) logger.Fatalf("Error configuring router: %v", err)

View File

@ -6,13 +6,13 @@ import (
"github.com/micro/go-micro/v2/auth" "github.com/micro/go-micro/v2/auth"
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/config" "github.com/micro/go-micro/v2/config"
"github.com/micro/go-micro/v2/debug/profile" "github.com/micro/go-micro/v2/debug/profile"
"github.com/micro/go-micro/v2/debug/trace" "github.com/micro/go-micro/v2/debug/trace"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/router" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/runtime" "github.com/micro/go-micro/v2/runtime"
"github.com/micro/go-micro/v2/selector"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
"github.com/micro/go-micro/v2/store" "github.com/micro/go-micro/v2/store"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"

15
go.mod
View File

@ -7,14 +7,9 @@ replace github.com/imdario/mergo => github.com/imdario/mergo v0.3.8
require ( require (
github.com/BurntSushi/toml v0.3.1 github.com/BurntSushi/toml v0.3.1
github.com/bitly/go-simplejson v0.5.0 github.com/bitly/go-simplejson v0.5.0
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 // indirect
github.com/bwmarrin/discordgo v0.20.2 github.com/bwmarrin/discordgo v0.20.2
github.com/caddyserver/certmagic v0.10.6 github.com/caddyserver/certmagic v0.10.6
github.com/coreos/bbolt v1.3.3 // indirect
github.com/coreos/etcd v3.3.18+incompatible github.com/coreos/etcd v3.3.18+incompatible
github.com/coreos/go-semver v0.3.0 // indirect
github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect
github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect
github.com/cpuguy83/go-md2man/v2 v2.0.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.0 // indirect
github.com/davecgh/go-spew v1.1.1 github.com/davecgh/go-spew v1.1.1
github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/dgrijalva/jwt-go v3.2.0+incompatible
@ -26,7 +21,6 @@ require (
github.com/ghodss/yaml v1.0.0 github.com/ghodss/yaml v1.0.0
github.com/go-acme/lego/v3 v3.4.0 github.com/go-acme/lego/v3 v3.4.0
github.com/go-git/go-git/v5 v5.1.0 github.com/go-git/go-git/v5 v5.1.0
github.com/go-telegram-bot-api/telegram-bot-api v4.6.4+incompatible // indirect
github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee
github.com/gobwas/pool v0.2.0 // indirect github.com/gobwas/pool v0.2.0 // indirect
github.com/gobwas/ws v1.0.3 github.com/gobwas/ws v1.0.3
@ -34,19 +28,16 @@ require (
github.com/golang/protobuf v1.4.0 github.com/golang/protobuf v1.4.0
github.com/google/uuid v1.1.1 github.com/google/uuid v1.1.1
github.com/gorilla/handlers v1.4.2 github.com/gorilla/handlers v1.4.2
github.com/gorilla/websocket v1.4.1 // indirect
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect
github.com/grpc-ecosystem/grpc-gateway v1.9.5 // indirect github.com/grpc-ecosystem/grpc-gateway v1.9.5 // indirect
github.com/hashicorp/hcl v1.0.0 github.com/hashicorp/hcl v1.0.0
github.com/hpcloud/tail v1.0.0 github.com/hpcloud/tail v1.0.0
github.com/imdario/mergo v0.3.9 github.com/imdario/mergo v0.3.9
github.com/jonboulle/clockwork v0.1.0 // indirect
github.com/json-iterator/go v1.1.9 // indirect github.com/json-iterator/go v1.1.9 // indirect
github.com/kr/pretty v0.1.0 github.com/kr/pretty v0.1.0
github.com/lib/pq v1.3.0 github.com/lib/pq v1.3.0
github.com/lucas-clemente/quic-go v0.14.1 github.com/lucas-clemente/quic-go v0.14.1
github.com/micro/cli/v2 v2.1.2 github.com/micro/cli/v2 v2.1.2
github.com/micro/go-micro v1.18.0
github.com/miekg/dns v1.1.27 github.com/miekg/dns v1.1.27
github.com/mitchellh/hashstructure v1.0.0 github.com/mitchellh/hashstructure v1.0.0
github.com/nats-io/nats-server/v2 v2.1.6 // indirect github.com/nats-io/nats-server/v2 v2.1.6 // indirect
@ -55,11 +46,8 @@ require (
github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c
github.com/patrickmn/go-cache v2.1.0+incompatible github.com/patrickmn/go-cache v2.1.0+incompatible
github.com/pkg/errors v0.9.1 github.com/pkg/errors v0.9.1
github.com/soheilhy/cmux v0.1.4 // indirect
github.com/stretchr/testify v1.4.0 github.com/stretchr/testify v1.4.0
github.com/technoweenie/multipartstreamer v1.0.1 // indirect
github.com/tmc/grpc-websocket-proxy v0.0.0-20200122045848-3419fae592fc // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20200122045848-3419fae592fc // indirect
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect
go.etcd.io/bbolt v1.3.4 go.etcd.io/bbolt v1.3.4
go.uber.org/zap v1.13.0 go.uber.org/zap v1.13.0
golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37 golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37
@ -70,5 +58,4 @@ require (
google.golang.org/grpc v1.26.0 google.golang.org/grpc v1.26.0
google.golang.org/protobuf v1.22.0 // indirect google.golang.org/protobuf v1.22.0 // indirect
gopkg.in/telegram-bot-api.v4 v4.6.4 gopkg.in/telegram-bot-api.v4 v4.6.4
sigs.k8s.io/yaml v1.1.0 // indirect
) )

81
go.sum
View File

@ -30,8 +30,10 @@ github.com/Azure/go-autorest/tracing v0.1.0/go.mod h1:ROEEAFwXycQw7Sn3DXNtEedEvd
github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
github.com/Microsoft/go-winio v0.4.14/go.mod h1:qXqCSQ3Xa7+6tgxaGTIe4Kpcdsi+P8jBhyzoq1bpyYA=
github.com/Microsoft/go-winio v0.4.15-0.20190919025122-fc70bd9a86b5 h1:ygIc8M6trr62pF5DucadTWGdEB4mEyvzi0e2nbcmcyA= github.com/Microsoft/go-winio v0.4.15-0.20190919025122-fc70bd9a86b5 h1:ygIc8M6trr62pF5DucadTWGdEB4mEyvzi0e2nbcmcyA=
github.com/Microsoft/go-winio v0.4.15-0.20190919025122-fc70bd9a86b5/go.mod h1:tTuCMEN+UleMWgg9dVx4Hu52b1bJo+59jBh3ajtinzw= github.com/Microsoft/go-winio v0.4.15-0.20190919025122-fc70bd9a86b5/go.mod h1:tTuCMEN+UleMWgg9dVx4Hu52b1bJo+59jBh3ajtinzw=
github.com/Microsoft/hcsshim v0.8.6/go.mod h1:Op3hHsoHPAvb6lceZHDtd9OkTew38wNoXnJs8iY7rUg=
github.com/Microsoft/hcsshim v0.8.7-0.20191101173118-65519b62243c h1:YMP6olTU903X3gxQJckdmiP8/zkSMq4kN3uipsU9XjU= github.com/Microsoft/hcsshim v0.8.7-0.20191101173118-65519b62243c h1:YMP6olTU903X3gxQJckdmiP8/zkSMq4kN3uipsU9XjU=
github.com/Microsoft/hcsshim v0.8.7-0.20191101173118-65519b62243c/go.mod h1:7xhjOwRV2+0HXGmM0jxaEu+ZiXJFoVZOTfL/dmqbrD8= github.com/Microsoft/hcsshim v0.8.7-0.20191101173118-65519b62243c/go.mod h1:7xhjOwRV2+0HXGmM0jxaEu+ZiXJFoVZOTfL/dmqbrD8=
github.com/OpenDNS/vegadns2client v0.0.0-20180418235048-a3fa4a771d87/go.mod h1:iGLljf5n9GjT6kc0HBvyI1nOKnGQbNB66VzSNbK5iks= github.com/OpenDNS/vegadns2client v0.0.0-20180418235048-a3fa4a771d87/go.mod h1:iGLljf5n9GjT6kc0HBvyI1nOKnGQbNB66VzSNbK5iks=
@ -53,6 +55,7 @@ github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5 h1:0CwZNZbxp69SHPd
github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5/go.mod h1:wHh0iHkYZB8zMSxRWpUBQtwG5a7fFgvEO+odwuTv2gs= github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5/go.mod h1:wHh0iHkYZB8zMSxRWpUBQtwG5a7fFgvEO+odwuTv2gs=
github.com/aws/aws-sdk-go v1.23.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.23.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
github.com/baiyubin/aliyun-sts-go-sdk v0.0.0-20180326062324-cfa1a18b161f/go.mod h1:AuiFmCCPBSrqvVMvuqFuk0qogytodnVFVSN5CeJB8Gc= github.com/baiyubin/aliyun-sts-go-sdk v0.0.0-20180326062324-cfa1a18b161f/go.mod h1:AuiFmCCPBSrqvVMvuqFuk0qogytodnVFVSN5CeJB8Gc=
github.com/beevik/ntp v0.2.0/go.mod h1:hIHWr+l3+/clUnF44zdK+CWW7fO8dR5cIylAQ76NRpg=
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
@ -62,10 +65,13 @@ github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngE
github.com/blang/semver v3.1.0+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnwebNt5EWlYSAyrTnjyyk= github.com/blang/semver v3.1.0+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnwebNt5EWlYSAyrTnjyyk=
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY=
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4=
github.com/bwmarrin/discordgo v0.19.0/go.mod h1:O9S4p+ofTFwB02em7jkpkV8M3R0/PUVOwN61zSZ0r4Q=
github.com/bwmarrin/discordgo v0.20.1/go.mod h1:O9S4p+ofTFwB02em7jkpkV8M3R0/PUVOwN61zSZ0r4Q=
github.com/bwmarrin/discordgo v0.20.2 h1:nA7jiTtqUA9lT93WL2jPjUp8ZTEInRujBdx1C9gkr20= github.com/bwmarrin/discordgo v0.20.2 h1:nA7jiTtqUA9lT93WL2jPjUp8ZTEInRujBdx1C9gkr20=
github.com/bwmarrin/discordgo v0.20.2/go.mod h1:O9S4p+ofTFwB02em7jkpkV8M3R0/PUVOwN61zSZ0r4Q= github.com/bwmarrin/discordgo v0.20.2/go.mod h1:O9S4p+ofTFwB02em7jkpkV8M3R0/PUVOwN61zSZ0r4Q=
github.com/caddyserver/certmagic v0.10.6 h1:sCya6FmfaN74oZE46kqfaFOVoROD/mF36rTQfjN7TZc= github.com/caddyserver/certmagic v0.10.6 h1:sCya6FmfaN74oZE46kqfaFOVoROD/mF36rTQfjN7TZc=
github.com/caddyserver/certmagic v0.10.6/go.mod h1:Y8jcUBctgk/IhpAzlHKfimZNyXCkfGgRTC0orl8gROQ= github.com/caddyserver/certmagic v0.10.6/go.mod h1:Y8jcUBctgk/IhpAzlHKfimZNyXCkfGgRTC0orl8gROQ=
github.com/cenkalti/backoff/v3 v3.0.0/go.mod h1:cIeZDE3IrqwwJl6VUwCN6trj1oXrTS4rc0ij+ULvLYs=
github.com/cenkalti/backoff/v4 v4.0.0 h1:6VeaLF9aI+MAUQ95106HwWzYZgJJpZ4stumjj6RFYAU= github.com/cenkalti/backoff/v4 v4.0.0 h1:6VeaLF9aI+MAUQ95106HwWzYZgJJpZ4stumjj6RFYAU=
github.com/cenkalti/backoff/v4 v4.0.0/go.mod h1:eEew/i+1Q6OrCDZh3WiXYv3+nJwBASZ8Bog/87DQnVg= github.com/cenkalti/backoff/v4 v4.0.0/go.mod h1:eEew/i+1Q6OrCDZh3WiXYv3+nJwBASZ8Bog/87DQnVg=
github.com/census-instrumentation/opencensus-proto v0.2.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/census-instrumentation/opencensus-proto v0.2.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
@ -80,6 +86,7 @@ github.com/containerd/console v0.0.0-20180822173158-c12b1e7919c1/go.mod h1:Tj/on
github.com/containerd/containerd v1.3.0-beta.2.0.20190828155532-0293cbd26c69/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= github.com/containerd/containerd v1.3.0-beta.2.0.20190828155532-0293cbd26c69/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA=
github.com/containerd/containerd v1.3.0 h1:xjvXQWABwS2uiv3TWgQt5Uth60Gu86LTGZXMJkjc7rY= github.com/containerd/containerd v1.3.0 h1:xjvXQWABwS2uiv3TWgQt5Uth60Gu86LTGZXMJkjc7rY=
github.com/containerd/containerd v1.3.0/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= github.com/containerd/containerd v1.3.0/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA=
github.com/containerd/continuity v0.0.0-20181203112020-004b46473808/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y=
github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc h1:TP+534wVlf61smEIq1nwLLAjQVEK2EADoW3CX9AuT+8= github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc h1:TP+534wVlf61smEIq1nwLLAjQVEK2EADoW3CX9AuT+8=
github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y=
github.com/containerd/fifo v0.0.0-20190226154929-a9fb20d87448/go.mod h1:ODA38xgv3Kuk8dQz2ZQXpnv/UZZUHUCL7pnLehbXgQI= github.com/containerd/fifo v0.0.0-20190226154929-a9fb20d87448/go.mod h1:ODA38xgv3Kuk8dQz2ZQXpnv/UZZUHUCL7pnLehbXgQI=
@ -88,6 +95,7 @@ github.com/containerd/ttrpc v0.0.0-20190828154514-0e0f228740de/go.mod h1:PvCDdDG
github.com/containerd/typeurl v0.0.0-20180627222232-a93fcdb778cd/go.mod h1:Cm3kwCdlkCfMSHURc+r6fwoGH6/F1hH3S4sg0rLFWPc= github.com/containerd/typeurl v0.0.0-20180627222232-a93fcdb778cd/go.mod h1:Cm3kwCdlkCfMSHURc+r6fwoGH6/F1hH3S4sg0rLFWPc=
github.com/coreos/bbolt v1.3.3 h1:n6AiVyVRKQFNb6mJlwESEvvLoDyiTzXX7ORAUlkeBdY= github.com/coreos/bbolt v1.3.3 h1:n6AiVyVRKQFNb6mJlwESEvvLoDyiTzXX7ORAUlkeBdY=
github.com/coreos/bbolt v1.3.3/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/bbolt v1.3.3/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk=
github.com/coreos/etcd v3.3.17+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
github.com/coreos/etcd v3.3.18+incompatible h1:Zz1aXgDrFFi1nadh58tA9ktt06cmPTwNNP3dXwIq1lE= github.com/coreos/etcd v3.3.18+incompatible h1:Zz1aXgDrFFi1nadh58tA9ktt06cmPTwNNP3dXwIq1lE=
github.com/coreos/etcd v3.3.18+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/etcd v3.3.18+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM=
@ -102,10 +110,12 @@ github.com/cpu/goacmedns v0.0.1/go.mod h1:sesf/pNnCYwUevQEQfEwY0Y3DydlQWSGZbaMEl
github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU=
github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM= github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM=
github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU=
github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY=
github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/decker502/dnspod-go v0.2.0/go.mod h1:qsurYu1FgxcDwfSwXJdLt4kRsBLZeosEb9uq4Sy+08g=
github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM=
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
github.com/dimchansky/utfbom v1.1.0/go.mod h1:rO41eb7gLfo8SF1jd9F8HplJm1Fewwi4mQvIirEdv+8= github.com/dimchansky/utfbom v1.1.0/go.mod h1:rO41eb7gLfo8SF1jd9F8HplJm1Fewwi4mQvIirEdv+8=
@ -113,6 +123,7 @@ github.com/dnaeon/go-vcr v0.0.0-20180814043457-aafff18a5cc2/go.mod h1:aBB1+wY4s9
github.com/dnsimple/dnsimple-go v0.30.0/go.mod h1:O5TJ0/U6r7AfT8niYNlmohpLbCSG+c71tQlGr9SeGrg= github.com/dnsimple/dnsimple-go v0.30.0/go.mod h1:O5TJ0/U6r7AfT8niYNlmohpLbCSG+c71tQlGr9SeGrg=
github.com/docker/distribution v2.7.1+incompatible h1:a5mlkVzth6W5A4fOsS3D2EO5BUmsJpcB+cRlLU7cSug= github.com/docker/distribution v2.7.1+incompatible h1:a5mlkVzth6W5A4fOsS3D2EO5BUmsJpcB+cRlLU7cSug=
github.com/docker/distribution v2.7.1+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= github.com/docker/distribution v2.7.1+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w=
github.com/docker/docker v1.4.2-0.20190710153559-aa8249ae1b8b/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk=
github.com/docker/docker v1.4.2-0.20191101170500-ac7306503d23 h1:oqgGT9O61YAYvI41EBsLePOr+LE6roB0xY4gpkZuFSE= github.com/docker/docker v1.4.2-0.20191101170500-ac7306503d23 h1:oqgGT9O61YAYvI41EBsLePOr+LE6roB0xY4gpkZuFSE=
github.com/docker/docker v1.4.2-0.20191101170500-ac7306503d23/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/docker v1.4.2-0.20191101170500-ac7306503d23/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk=
github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ=
@ -126,6 +137,7 @@ github.com/ef-ds/deque v1.0.4-0.20190904040645-54cb57c252a1 h1:jFGzikHboUMRXmMBt
github.com/ef-ds/deque v1.0.4-0.20190904040645-54cb57c252a1/go.mod h1:HvODWzv6Y6kBf3Ah2WzN1bHjDUezGLaAhwuWVwfpEJs= github.com/ef-ds/deque v1.0.4-0.20190904040645-54cb57c252a1/go.mod h1:HvODWzv6Y6kBf3Ah2WzN1bHjDUezGLaAhwuWVwfpEJs=
github.com/emirpasic/gods v1.12.0 h1:QAUIPSaCu4G+POclxeqb3F+WPpdKqFGlw36+yOzGlrg= github.com/emirpasic/gods v1.12.0 h1:QAUIPSaCu4G+POclxeqb3F+WPpdKqFGlw36+yOzGlrg=
github.com/emirpasic/gods v1.12.0/go.mod h1:YfzfFFoVP/catgzJb4IKIqXjX78Ha8FMSDh3ymbK86o= github.com/emirpasic/gods v1.12.0/go.mod h1:YfzfFFoVP/catgzJb4IKIqXjX78Ha8FMSDh3ymbK86o=
github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
github.com/evanphx/json-patch/v5 v5.0.0 h1:dKTrUeykyQwKb/kx7Z+4ukDs6l+4L41HqG1XHnhX7WE= github.com/evanphx/json-patch/v5 v5.0.0 h1:dKTrUeykyQwKb/kx7Z+4ukDs6l+4L41HqG1XHnhX7WE=
@ -138,12 +150,14 @@ github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c h1:pBgVXWDXj
github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c/go.mod h1:pFdJbAhRf7rh6YYMUdIQGyzne6zYL1tCUW8QV2B3UfY= github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c/go.mod h1:pFdJbAhRf7rh6YYMUdIQGyzne6zYL1tCUW8QV2B3UfY=
github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I=
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
github.com/fsouza/go-dockerclient v1.4.4/go.mod h1:PrwszSL5fbmsESocROrOGq/NULMXRw+bajY0ltzD6MA=
github.com/fsouza/go-dockerclient v1.6.0 h1:f7j+AX94143JL1H3TiqSMkM4EcLDI0De1qD4GGn3Hig= github.com/fsouza/go-dockerclient v1.6.0 h1:f7j+AX94143JL1H3TiqSMkM4EcLDI0De1qD4GGn3Hig=
github.com/fsouza/go-dockerclient v1.6.0/go.mod h1:YWwtNPuL4XTX1SKJQk86cWPmmqwx+4np9qfPbb+znGc= github.com/fsouza/go-dockerclient v1.6.0/go.mod h1:YWwtNPuL4XTX1SKJQk86cWPmmqwx+4np9qfPbb+znGc=
github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk=
github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
github.com/gliderlabs/ssh v0.2.2 h1:6zsha5zo/TWhRhwqCD3+EarCAgZ2yN28ipRnGPnwkI0= github.com/gliderlabs/ssh v0.2.2 h1:6zsha5zo/TWhRhwqCD3+EarCAgZ2yN28ipRnGPnwkI0=
github.com/gliderlabs/ssh v0.2.2/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/gliderlabs/ssh v0.2.2/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0=
github.com/go-acme/lego/v3 v3.1.0/go.mod h1:074uqt+JS6plx+c9Xaiz6+L+GBb+7itGtzfcDM2AhEE=
github.com/go-acme/lego/v3 v3.4.0 h1:deB9NkelA+TfjGHVw8J7iKl/rMtffcGMWSMmptvMv0A= github.com/go-acme/lego/v3 v3.4.0 h1:deB9NkelA+TfjGHVw8J7iKl/rMtffcGMWSMmptvMv0A=
github.com/go-acme/lego/v3 v3.4.0/go.mod h1:xYbLDuxq3Hy4bMUT1t9JIuz6GWIWb3m5X+TeTHYaT7M= github.com/go-acme/lego/v3 v3.4.0/go.mod h1:xYbLDuxq3Hy4bMUT1t9JIuz6GWIWb3m5X+TeTHYaT7M=
github.com/go-cmd/cmd v1.0.5/go.mod h1:y8q8qlK5wQibcw63djSl/ntiHUHXHGdCkPk0j4QeW4s= github.com/go-cmd/cmd v1.0.5/go.mod h1:y8q8qlK5wQibcw63djSl/ntiHUHXHGdCkPk0j4QeW4s=
@ -159,8 +173,13 @@ github.com/go-git/go-git/v5 v5.1.0/go.mod h1:ZKfuPUoY1ZqIG4QG9BDBh3G4gLM5zvPuSJA
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
github.com/go-ini/ini v1.44.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= github.com/go-ini/ini v1.44.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8=
github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
github.com/go-log/log v0.1.0 h1:wudGTNsiGzrD5ZjgIkVZ517ugi2XRe9Q/xRCzwEO4/U=
github.com/go-log/log v0.1.0/go.mod h1:4mBwpdRMFLiuXZDCwU2lKQFsoSCo72j3HqBK9d81N2M=
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM=
github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8=
github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY=
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
github.com/go-telegram-bot-api/telegram-bot-api v4.6.4+incompatible h1:2cauKuaELYAEARXRkq2LrJ0yDDv1rW7+wrTEdVL3uaU= github.com/go-telegram-bot-api/telegram-bot-api v4.6.4+incompatible h1:2cauKuaELYAEARXRkq2LrJ0yDDv1rW7+wrTEdVL3uaU=
github.com/go-telegram-bot-api/telegram-bot-api v4.6.4+incompatible/go.mod h1:qf9acutJ8cwBUhm1bqgz6Bei9/C/c93FPDljKWwsOgM= github.com/go-telegram-bot-api/telegram-bot-api v4.6.4+incompatible/go.mod h1:qf9acutJ8cwBUhm1bqgz6Bei9/C/c93FPDljKWwsOgM=
@ -179,6 +198,7 @@ github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zV
github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
github.com/golang/groupcache v0.0.0-20191002201903-404acd9df4cc/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY=
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
@ -247,6 +267,7 @@ github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T
github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
github.com/iij/doapi v0.0.0-20190504054126-0bbf12d6d7df/go.mod h1:QMZY7/J/KSQEhKWFeDesPjMj+wCHReeknARU3wqlyN4= github.com/iij/doapi v0.0.0-20190504054126-0bbf12d6d7df/go.mod h1:QMZY7/J/KSQEhKWFeDesPjMj+wCHReeknARU3wqlyN4=
github.com/ijc/Gotty v0.0.0-20170406111628-a8b993ba6abd/go.mod h1:3LVOLeyx9XVvwPgrt2be44XgSqndprz1G18rSk8KD84=
github.com/imdario/mergo v0.3.8 h1:CGgOkSJeqMRmt0D9XLWExdT4m4F1vd3FV3VPt+0VxkQ= github.com/imdario/mergo v0.3.8 h1:CGgOkSJeqMRmt0D9XLWExdT4m4F1vd3FV3VPt+0VxkQ=
github.com/imdario/mergo v0.3.8/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= github.com/imdario/mergo v0.3.8/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA=
github.com/jbenet/go-context v0.0.0-20150711004518-d14ea06fba99 h1:BQSFePA1RWJOlocH6Fxy8MmwDt+yVQYULKfN0RoTN8A= github.com/jbenet/go-context v0.0.0-20150711004518-d14ea06fba99 h1:BQSFePA1RWJOlocH6Fxy8MmwDt+yVQYULKfN0RoTN8A=
@ -255,9 +276,11 @@ github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJS
github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo=
github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
github.com/joncalhoun/qson v0.0.0-20170526102502-8a9cab3a62b1/go.mod h1:DFXrEwSRX0p/aSvxE21319menCBFeQO0jXpRj7LEZUA=
github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns= github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns=
github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
@ -267,6 +290,7 @@ github.com/kevinburke/ssh_config v0.0.0-20190725054713-01f96b0aa0cd h1:Coekwdh0v
github.com/kevinburke/ssh_config v0.0.0-20190725054713-01f96b0aa0cd/go.mod h1:CT57kijsi8u/K/BOFA39wgDQJ9CxiF4nAY/ojJ6r6mM= github.com/kevinburke/ssh_config v0.0.0-20190725054713-01f96b0aa0cd/go.mod h1:CT57kijsi8u/K/BOFA39wgDQJ9CxiF4nAY/ojJ6r6mM=
github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/klauspost/cpuid v1.2.3 h1:CCtW0xUnWGVINKvE/WWOYKdsPV6mawAtvQuSl8guwQs= github.com/klauspost/cpuid v1.2.3 h1:CCtW0xUnWGVINKvE/WWOYKdsPV6mawAtvQuSl8guwQs=
github.com/klauspost/cpuid v1.2.3/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.2.3/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/kolo/xmlrpc v0.0.0-20190717152603-07c4ee3fd181/go.mod h1:o03bZfuBwAXHetKXuInt4S7omeXUu62/A845kiycsSQ= github.com/kolo/xmlrpc v0.0.0-20190717152603-07c4ee3fd181/go.mod h1:o03bZfuBwAXHetKXuInt4S7omeXUu62/A845kiycsSQ=
@ -277,21 +301,28 @@ github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFB
github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
github.com/kr/pty v1.1.8/go.mod h1:O1sed60cT9XZ5uDucP5qwvh+TE3NnUj51EiZO/lmSfw=
github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/labbsr0x/bindman-dns-webhook v1.0.2/go.mod h1:p6b+VCXIR8NYKpDr8/dg1HKfQoRHCdcsROXKvmoehKA= github.com/labbsr0x/bindman-dns-webhook v1.0.2/go.mod h1:p6b+VCXIR8NYKpDr8/dg1HKfQoRHCdcsROXKvmoehKA=
github.com/labbsr0x/goh v1.0.1/go.mod h1:8K2UhVoaWXcCU7Lxoa2omWnC8gyW8px7/lmO61c027w= github.com/labbsr0x/goh v1.0.1/go.mod h1:8K2UhVoaWXcCU7Lxoa2omWnC8gyW8px7/lmO61c027w=
github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw=
github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII=
github.com/lib/pq v1.2.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.3.0 h1:/qkRGz8zljWiDcFvgpwUpwIAPu3r07TDvs3Rws+o/pU= github.com/lib/pq v1.3.0 h1:/qkRGz8zljWiDcFvgpwUpwIAPu3r07TDvs3Rws+o/pU=
github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/linode/linodego v0.10.0/go.mod h1:cziNP7pbvE3mXIPneHj0oRY8L1WtGEIKlZ8LANE4eXA= github.com/linode/linodego v0.10.0/go.mod h1:cziNP7pbvE3mXIPneHj0oRY8L1WtGEIKlZ8LANE4eXA=
github.com/liquidweb/liquidweb-go v1.6.0/go.mod h1:UDcVnAMDkZxpw4Y7NOHkqoeiGacVLEIG/i5J9cyixzQ= github.com/liquidweb/liquidweb-go v1.6.0/go.mod h1:UDcVnAMDkZxpw4Y7NOHkqoeiGacVLEIG/i5J9cyixzQ=
github.com/lucas-clemente/quic-go v0.12.1/go.mod h1:UXJJPE4RfFef/xPO5wQm0tITK8gNfqwTxjbE7s3Vb8s=
github.com/lucas-clemente/quic-go v0.13.1/go.mod h1:Vn3/Fb0/77b02SGhQk36KzOUmXgVpFfizUfW5WMaqyU=
github.com/lucas-clemente/quic-go v0.14.1 h1:c1aKoBZKOPA+49q96B1wGkibyPP0AxYh45WuAoq+87E= github.com/lucas-clemente/quic-go v0.14.1 h1:c1aKoBZKOPA+49q96B1wGkibyPP0AxYh45WuAoq+87E=
github.com/lucas-clemente/quic-go v0.14.1/go.mod h1:Vn3/Fb0/77b02SGhQk36KzOUmXgVpFfizUfW5WMaqyU= github.com/lucas-clemente/quic-go v0.14.1/go.mod h1:Vn3/Fb0/77b02SGhQk36KzOUmXgVpFfizUfW5WMaqyU=
github.com/marten-seemann/chacha20 v0.2.0 h1:f40vqzzx+3GdOmzQoItkLX5WLvHgPgyYqFFIO5Gh4hQ= github.com/marten-seemann/chacha20 v0.2.0 h1:f40vqzzx+3GdOmzQoItkLX5WLvHgPgyYqFFIO5Gh4hQ=
github.com/marten-seemann/chacha20 v0.2.0/go.mod h1:HSdjFau7GzYRj+ahFNwsO3ouVJr1HFkWoEwNDb4TMtE= github.com/marten-seemann/chacha20 v0.2.0/go.mod h1:HSdjFau7GzYRj+ahFNwsO3ouVJr1HFkWoEwNDb4TMtE=
github.com/marten-seemann/qpack v0.1.0/go.mod h1:LFt1NU/Ptjip0C2CPkhimBz5CGE3WGDAUWqna+CNTrI= github.com/marten-seemann/qpack v0.1.0/go.mod h1:LFt1NU/Ptjip0C2CPkhimBz5CGE3WGDAUWqna+CNTrI=
github.com/marten-seemann/qtls v0.3.2/go.mod h1:xzjG7avBwGGbdZ8dTGxlBnLArsVKLvwmjgmPuiQEcYk=
github.com/marten-seemann/qtls v0.4.1 h1:YlT8QP3WCCvvok7MGEZkMldXbyqgr8oFg5/n8Gtbkks= github.com/marten-seemann/qtls v0.4.1 h1:YlT8QP3WCCvvok7MGEZkMldXbyqgr8oFg5/n8Gtbkks=
github.com/marten-seemann/qtls v0.4.1/go.mod h1:pxVXcHHw1pNIt8Qo0pwSYQEoZ8yYOOPXTCZLQQunvRc= github.com/marten-seemann/qtls v0.4.1/go.mod h1:pxVXcHHw1pNIt8Qo0pwSYQEoZ8yYOOPXTCZLQQunvRc=
github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
@ -300,9 +331,21 @@ github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp
github.com/mattn/go-tty v0.0.0-20180219170247-931426f7535a/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= github.com/mattn/go-tty v0.0.0-20180219170247-931426f7535a/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE=
github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
github.com/mholt/certmagic v0.7.5/go.mod h1:91uJzK5K8IWtYQqTi5R2tsxV1pCde+wdGfaRaOZi6aQ=
github.com/mholt/certmagic v0.8.3/go.mod h1:91uJzK5K8IWtYQqTi5R2tsxV1pCde+wdGfaRaOZi6aQ=
github.com/micro/cli v0.2.0 h1:ut3rV5JWqZjsXIa2MvGF+qMUP8DAUTvHX9Br5gO4afA=
github.com/micro/cli v0.2.0/go.mod h1:jRT9gmfVKWSS6pkKcXQ8YhUyj6bzwxK8Fp5b0Y7qNnk=
github.com/micro/cli/v2 v2.1.2 h1:43J1lChg/rZCC1rvdqZNFSQDrGT7qfMrtp6/ztpIkEM= github.com/micro/cli/v2 v2.1.2 h1:43J1lChg/rZCC1rvdqZNFSQDrGT7qfMrtp6/ztpIkEM=
github.com/micro/cli/v2 v2.1.2/go.mod h1:EguNh6DAoWKm9nmk+k/Rg0H3lQnDxqzu5x5srOtGtYg= github.com/micro/cli/v2 v2.1.2/go.mod h1:EguNh6DAoWKm9nmk+k/Rg0H3lQnDxqzu5x5srOtGtYg=
github.com/micro/go-micro v1.16.0/go.mod h1:A0F58bHLh2m0LAI9QyhvmbN8c1cxhAZo3cM6s+iDsrM=
github.com/micro/go-micro v1.18.0 h1:gP70EZVHpJuUIT0YWth192JmlIci+qMOEByHm83XE9E=
github.com/micro/go-micro v1.18.0/go.mod h1:klwUJL1gkdY1MHFyz+fFJXn52dKcty4hoe95Mp571AA=
github.com/micro/mdns v0.3.0 h1:bYycYe+98AXR3s8Nq5qvt6C573uFTDPIYzJemWON0QE=
github.com/micro/mdns v0.3.0/go.mod h1:KJ0dW7KmicXU2BV++qkLlmHYcVv7/hHnbtguSWt9Aoc=
github.com/micro/protoc-gen-micro v1.0.0/go.mod h1:C8ij4DJhapBmypcT00AXdb0cZ675/3PqUO02buWWqbE=
github.com/miekg/dns v1.1.3/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
github.com/miekg/dns v1.1.15/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= github.com/miekg/dns v1.1.15/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
github.com/miekg/dns v1.1.22/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
github.com/miekg/dns v1.1.27 h1:aEH/kqUzUxGJ/UHcEKdJY+ugH6WEzsEBBSPa8zuy1aM= github.com/miekg/dns v1.1.27 h1:aEH/kqUzUxGJ/UHcEKdJY+ugH6WEzsEBBSPa8zuy1aM=
github.com/miekg/dns v1.1.27/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.27/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM=
github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y=
@ -321,12 +364,18 @@ github.com/morikuni/aec v0.0.0-20170113033406-39771216ff4c h1:nXxl5PrvVm2L/wCy8d
github.com/morikuni/aec v0.0.0-20170113033406-39771216ff4c/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/morikuni/aec v0.0.0-20170113033406-39771216ff4c/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc=
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
github.com/namedotcom/go v0.0.0-20180403034216-08470befbe04/go.mod h1:5sN+Lt1CaY4wsPvgQH/jsuJi4XO2ssZbdsIizr4CVC8= github.com/namedotcom/go v0.0.0-20180403034216-08470befbe04/go.mod h1:5sN+Lt1CaY4wsPvgQH/jsuJi4XO2ssZbdsIizr4CVC8=
github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg=
github.com/nats-io/jwt v0.3.2 h1:+RB5hMpXUUA2dfxuhBTEkMOrYmM+gKIZYS1KjSostMI= github.com/nats-io/jwt v0.3.2 h1:+RB5hMpXUUA2dfxuhBTEkMOrYmM+gKIZYS1KjSostMI=
github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU= github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU=
github.com/nats-io/nats-server/v2 v2.1.0/go.mod h1:r5y0WgCag0dTj/qiHkHrXAcKQ/f5GMOZaEGdoxxnJ4I=
github.com/nats-io/nats-server/v2 v2.1.6 h1:qAaHZaS8pRRNQLFaiBA1rq5WynyEGp9DFgmMfoaiXGY= github.com/nats-io/nats-server/v2 v2.1.6 h1:qAaHZaS8pRRNQLFaiBA1rq5WynyEGp9DFgmMfoaiXGY=
github.com/nats-io/nats-server/v2 v2.1.6/go.mod h1:BL1NOtaBQ5/y97djERRVWNouMW7GT3gxnmbE/eC8u8A= github.com/nats-io/nats-server/v2 v2.1.6/go.mod h1:BL1NOtaBQ5/y97djERRVWNouMW7GT3gxnmbE/eC8u8A=
github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM=
github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w=
github.com/nats-io/nats.go v1.9.2 h1:oDeERm3NcZVrPpdR/JpGdWHMv3oJ8yY30YwxKq+DU2s= github.com/nats-io/nats.go v1.9.2 h1:oDeERm3NcZVrPpdR/JpGdWHMv3oJ8yY30YwxKq+DU2s=
github.com/nats-io/nats.go v1.9.2/go.mod h1:AjGArbfyR50+afOUotNX2Xs5SYHf+CoOa5HH1eEl2HE= github.com/nats-io/nats.go v1.9.2/go.mod h1:AjGArbfyR50+afOUotNX2Xs5SYHf+CoOa5HH1eEl2HE=
github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4=
github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
github.com/nats-io/nkeys v0.1.4 h1:aEsHIssIk6ETN5m2/MD8Y4B2X7FfXrBAUdkyRvbVYzA= github.com/nats-io/nkeys v0.1.4 h1:aEsHIssIk6ETN5m2/MD8Y4B2X7FfXrBAUdkyRvbVYzA=
github.com/nats-io/nkeys v0.1.4/go.mod h1:XdZpAbhgyyODYqjTawOnIOI7VlbKSarI9Gfy1tqEu/s= github.com/nats-io/nkeys v0.1.4/go.mod h1:XdZpAbhgyyODYqjTawOnIOI7VlbKSarI9Gfy1tqEu/s=
@ -335,6 +384,7 @@ github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OS
github.com/nbio/st v0.0.0-20140626010706-e9e8d9816f32/go.mod h1:9wM+0iRr9ahx58uYLpLIr5fm8diHn0JbqRycJi6w0Ms= github.com/nbio/st v0.0.0-20140626010706-e9e8d9816f32/go.mod h1:9wM+0iRr9ahx58uYLpLIr5fm8diHn0JbqRycJi6w0Ms=
github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs=
github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno=
github.com/nlopes/slack v0.6.0/go.mod h1:JzQ9m3PMAqcpeCam7UaHSuBuupz7CmpjehYMayT6YOk=
github.com/nlopes/slack v0.6.1-0.20191106133607-d06c2a2b3249 h1:Pr5gZa2VcmktVwq0lyC39MsN5tz356vC/pQHKvq+QBo= github.com/nlopes/slack v0.6.1-0.20191106133607-d06c2a2b3249 h1:Pr5gZa2VcmktVwq0lyC39MsN5tz356vC/pQHKvq+QBo=
github.com/nlopes/slack v0.6.1-0.20191106133607-d06c2a2b3249/go.mod h1:JzQ9m3PMAqcpeCam7UaHSuBuupz7CmpjehYMayT6YOk= github.com/nlopes/slack v0.6.1-0.20191106133607-d06c2a2b3249/go.mod h1:JzQ9m3PMAqcpeCam7UaHSuBuupz7CmpjehYMayT6YOk=
github.com/nrdcg/auroradns v1.0.0/go.mod h1:6JPXKzIRzZzMqtTDgueIhTi6rFf1QvYE/HzqidhOhjw= github.com/nrdcg/auroradns v1.0.0/go.mod h1:6JPXKzIRzZzMqtTDgueIhTi6rFf1QvYE/HzqidhOhjw=
@ -365,6 +415,7 @@ github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c h1:rp5dCmg/yLR3mgF
github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c/go.mod h1:X07ZCGwUbLaax7L0S3Tw4hpejzu63ZrrQiUe6W0hcy0= github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c/go.mod h1:X07ZCGwUbLaax7L0S3Tw4hpejzu63ZrrQiUe6W0hcy0=
github.com/patrickmn/go-cache v2.1.0+incompatible h1:HRMgzkcYKYpi3C8ajMPV8OFXaaRUnok+kx1WdO15EQc= github.com/patrickmn/go-cache v2.1.0+incompatible h1:HRMgzkcYKYpi3C8ajMPV8OFXaaRUnok+kx1WdO15EQc=
github.com/patrickmn/go-cache v2.1.0+incompatible/go.mod h1:3Qf8kWWT7OJRJbdiICTKqZju1ZixQ/KpMGzzAfe6+WQ= github.com/patrickmn/go-cache v2.1.0+incompatible/go.mod h1:3Qf8kWWT7OJRJbdiICTKqZju1ZixQ/KpMGzzAfe6+WQ=
github.com/pelletier/go-buffruneio v0.2.0/go.mod h1:JkE26KsDizTr40EUHkXVtNPvgGtbSNq5BcowyYOWdKo=
github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
@ -400,6 +451,7 @@ github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0
github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
github.com/sacloud/libsacloud v1.26.1/go.mod h1:79ZwATmHLIFZIMd7sxA3LwzVy/B77uj3LDoToVTxDoQ= github.com/sacloud/libsacloud v1.26.1/go.mod h1:79ZwATmHLIFZIMd7sxA3LwzVy/B77uj3LDoToVTxDoQ=
github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0=
github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM=
github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo=
@ -413,9 +465,11 @@ github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1
github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E=
github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
github.com/src-d/gcfg v1.4.0/go.mod h1:p/UMsR43ujA89BJY9duynAwIpvqEujIH/jFlfL7jWoI=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A=
github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE=
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk=
@ -424,6 +478,7 @@ github.com/syndtr/gocapability v0.0.0-20170704070218-db04d3cc01c8/go.mod h1:hkRG
github.com/technoweenie/multipartstreamer v1.0.1 h1:XRztA5MXiR1TIRHxH2uNxXxaIkKQDeX7m2XsSOlQEnM= github.com/technoweenie/multipartstreamer v1.0.1 h1:XRztA5MXiR1TIRHxH2uNxXxaIkKQDeX7m2XsSOlQEnM=
github.com/technoweenie/multipartstreamer v1.0.1/go.mod h1:jNVxdtShOxzAsukZwTSw6MDx5eUJoiEBsSvzDU9uzog= github.com/technoweenie/multipartstreamer v1.0.1/go.mod h1:jNVxdtShOxzAsukZwTSw6MDx5eUJoiEBsSvzDU9uzog=
github.com/timewasted/linode v0.0.0-20160829202747-37e84520dcf7/go.mod h1:imsgLplxEC/etjIhdr3dNzV3JeT27LbVu5pYWm0JCBY= github.com/timewasted/linode v0.0.0-20160829202747-37e84520dcf7/go.mod h1:imsgLplxEC/etjIhdr3dNzV3JeT27LbVu5pYWm0JCBY=
github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
github.com/tmc/grpc-websocket-proxy v0.0.0-20200122045848-3419fae592fc h1:yUaosFVTJwnltaHbSNC3i82I92quFs+OFPRl8kNMVwo= github.com/tmc/grpc-websocket-proxy v0.0.0-20200122045848-3419fae592fc h1:yUaosFVTJwnltaHbSNC3i82I92quFs+OFPRl8kNMVwo=
github.com/tmc/grpc-websocket-proxy v0.0.0-20200122045848-3419fae592fc/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200122045848-3419fae592fc/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
github.com/transip/gotransip v0.0.0-20190812104329-6d8d9179b66f/go.mod h1:i0f4R4o2HM0m3DZYQWsj6/MEowD57VzoH0v3d7igeFY= github.com/transip/gotransip v0.0.0-20190812104329-6d8d9179b66f/go.mod h1:i0f4R4o2HM0m3DZYQWsj6/MEowD57VzoH0v3d7igeFY=
@ -439,6 +494,7 @@ github.com/xeipuuv/gojsonschema v0.0.0-20180618132009-1d523034197f/go.mod h1:5yf
github.com/xeipuuv/gojsonschema v1.1.0/go.mod h1:5yf86TLmAcydyeJq5YvxkGPE2fm/u4myDekKRoLuqhs= github.com/xeipuuv/gojsonschema v1.1.0/go.mod h1:5yf86TLmAcydyeJq5YvxkGPE2fm/u4myDekKRoLuqhs=
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
go.etcd.io/bbolt v1.3.4 h1:hi1bXHMVrlQh6WwxAy+qZCV/SYIlqo+Ushwdpa4tAKg= go.etcd.io/bbolt v1.3.4 h1:hi1bXHMVrlQh6WwxAy+qZCV/SYIlqo+Ushwdpa4tAKg=
go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ=
go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
@ -450,27 +506,35 @@ go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY=
go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
go.uber.org/multierr v1.2.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
go.uber.org/multierr v1.3.0 h1:sFPn2GLc3poCkfrpIXGhBD2X0CMIo4Q/zSULXrj/+uc= go.uber.org/multierr v1.3.0 h1:sFPn2GLc3poCkfrpIXGhBD2X0CMIo4Q/zSULXrj/+uc=
go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
go.uber.org/ratelimit v0.0.0-20180316092928-c15da0234277/go.mod h1:2X8KaoNd1J0lZV+PxJk/5+DGbO/tpwLR1m++a7FnB/Y= go.uber.org/ratelimit v0.0.0-20180316092928-c15da0234277/go.mod h1:2X8KaoNd1J0lZV+PxJk/5+DGbO/tpwLR1m++a7FnB/Y=
go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4=
go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA=
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU= go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU=
go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
golang.org/x/crypto v0.0.0-20180621125126-a49355c7e3f8/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180621125126-a49355c7e3f8/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20181030102418-4d3f4d9ffa16/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181030102418-4d3f4d9ffa16/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190130090550-b01c7a725664/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190219172222-a4c6cb3142f2/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190219172222-a4c6cb3142f2/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190228161510-8dd112bcdc25/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190418165655-df01cb2cc480/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE= golang.org/x/crypto v0.0.0-20190418165655-df01cb2cc480/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE=
golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190829043050-9756ffdc2472/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190829043050-9756ffdc2472/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190911031432-227b76d455e7/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392/go.mod h1:/lpIB1dKB+9EgE3H3cr1v9wB50oz8l4C4h62xy7jSTY=
golang.org/x/crypto v0.0.0-20190927123631-a832865fa7ad/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190927123631-a832865fa7ad/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20191108234033-bd318be0434a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59 h1:3zb4D3T4G8jdExgVU/95+vQXfpEPiMdCaZgmGVxjNHM= golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59 h1:3zb4D3T4G8jdExgVU/95+vQXfpEPiMdCaZgmGVxjNHM=
golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
@ -517,7 +581,9 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL
golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190930134127-c5a3c61f89f3/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190930134127-c5a3c61f89f3/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20191011234655-491137f69257/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20191027093000-83d349e8ac1a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20191027093000-83d349e8ac1a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20191109021931-daa7c04131f5/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200301022130-244492dfa37a h1:GuSPYbZzB5/dcLNCwLQLsg3obCJtX9IJhpXkvY7kzk0= golang.org/x/net v0.0.0-20200301022130-244492dfa37a h1:GuSPYbZzB5/dcLNCwLQLsg3obCJtX9IJhpXkvY7kzk0=
golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2 h1:eDrdRpKgkcCqKZQwyZRyeFZgfqt37SL7Kv3tok06cKE= golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2 h1:eDrdRpKgkcCqKZQwyZRyeFZgfqt37SL7Kv3tok06cKE=
@ -539,9 +605,11 @@ golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5h
golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190129075346-302c3dd5f1cc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190209173611-3b5209105503/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190209173611-3b5209105503/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190221075227-b4e8571b14e0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190221075227-b4e8571b14e0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190228124157-a34e9553db1e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@ -551,12 +619,15 @@ golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20190514135907-3a4b5fb9f71f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190514135907-3a4b5fb9f71f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190710143415-6ec70d6a5542/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe h1:6fAMxZRR6sl1Uq8U61gxU+kPTs2tR8uOySCbBP7BN/M= golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe h1:6fAMxZRR6sl1Uq8U61gxU+kPTs2tR8uOySCbBP7BN/M=
golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191110163157-d32e6e3b99c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5 h1:LfCXLvNmTYH9kEmVgqbnsWfruoXZIrh4YBgqVHtDvw0= golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5 h1:LfCXLvNmTYH9kEmVgqbnsWfruoXZIrh4YBgqVHtDvw0=
golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527 h1:uYVVQ9WP/Ds2ROhcaGPeIdVq0RIXVLwsHlnvJ+cT1So= golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527 h1:uYVVQ9WP/Ds2ROhcaGPeIdVq0RIXVLwsHlnvJ+cT1So=
@ -589,7 +660,9 @@ golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBn
golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
golang.org/x/tools v0.0.0-20190729092621-ff9f1409240a/go.mod h1:jcCCGcm9btYwXyDqrUWc6MKQKKGJCWEQ3AfLSRIbEuI=
golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
@ -621,6 +694,7 @@ google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRn
google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8=
google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1 h1:aQktFqmDE2yjveXJlVIfslDFmFnUXSqG0i6KRcJAeMc= google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1 h1:aQktFqmDE2yjveXJlVIfslDFmFnUXSqG0i6KRcJAeMc=
google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
@ -630,6 +704,8 @@ google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiq
google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
google.golang.org/grpc v1.22.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.22.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA=
google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg=
google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
@ -649,6 +725,8 @@ gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8
gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE=
gopkg.in/go-playground/validator.v9 v9.30.0/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ=
gopkg.in/h2non/gock.v1 v1.0.15/go.mod h1:sX4zAkdYX1TRGJ2JY156cFspQn4yRWn6p9EMdODlynE= gopkg.in/h2non/gock.v1 v1.0.15/go.mod h1:sX4zAkdYX1TRGJ2JY156cFspQn4yRWn6p9EMdODlynE=
gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/ini.v1 v1.44.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.44.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
@ -657,6 +735,9 @@ gopkg.in/resty.v1 v1.9.1/go.mod h1:vo52Hzryw9PnPHcJfPsBiFW62XhNx5OczbV9y+IMpgc=
gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo=
gopkg.in/square/go-jose.v2 v2.3.1 h1:SK5KegNXmKmqE342YYN2qPHEnUYeoMiXXl1poUlI+o4= gopkg.in/square/go-jose.v2 v2.3.1 h1:SK5KegNXmKmqE342YYN2qPHEnUYeoMiXXl1poUlI+o4=
gopkg.in/square/go-jose.v2 v2.3.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/square/go-jose.v2 v2.3.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI=
gopkg.in/src-d/go-billy.v4 v4.3.2/go.mod h1:nDjArDMp+XMs1aFAESLRjfGSgfvoYN0hDfzEk0GjC98=
gopkg.in/src-d/go-git-fixtures.v3 v3.5.0/go.mod h1:dLBcvytrw/TYZsNTWCnkNF2DSIlzWYqTe3rJR56Ac7g=
gopkg.in/src-d/go-git.v4 v4.13.1/go.mod h1:nx5NYcxdKxq5fpltdHnPa2Exj4Sx0EclMWZQbYDu2z8=
gopkg.in/telegram-bot-api.v4 v4.6.4 h1:hpHWhzn4jTCsAJZZ2loNKfy2QWyPDRJVl3aTFXeMW8g= gopkg.in/telegram-bot-api.v4 v4.6.4 h1:hpHWhzn4jTCsAJZZ2loNKfy2QWyPDRJVl3aTFXeMW8g=
gopkg.in/telegram-bot-api.v4 v4.6.4/go.mod h1:5DpGO5dbumb40px+dXcwCpcjmeHNYLpk0bp3XRNvWDM= gopkg.in/telegram-bot-api.v4 v4.6.4/go.mod h1:5DpGO5dbumb40px+dXcwCpcjmeHNYLpk0bp3XRNvWDM=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=

View File

@ -14,7 +14,6 @@ import (
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
cmucp "github.com/micro/go-micro/v2/client/mucp" cmucp "github.com/micro/go-micro/v2/client/mucp"
rtr "github.com/micro/go-micro/v2/client/selector/router"
"github.com/micro/go-micro/v2/logger" "github.com/micro/go-micro/v2/logger"
"github.com/micro/go-micro/v2/network/resolver/dns" "github.com/micro/go-micro/v2/network/resolver/dns"
pbNet "github.com/micro/go-micro/v2/network/service/proto" pbNet "github.com/micro/go-micro/v2/network/service/proto"
@ -153,11 +152,7 @@ func newNetwork(opts ...Option) Network {
client := cmucp.NewClient( client := cmucp.NewClient(
client.Broker(tunBroker), client.Broker(tunBroker),
client.Transport(tunTransport), client.Transport(tunTransport),
client.Selector( client.Router(options.Router),
rtr.NewSelector(
rtr.WithRouter(options.Router),
),
),
) )
network := &network{ network := &network{

View File

@ -8,7 +8,6 @@ import (
"github.com/micro/go-micro/v2/auth" "github.com/micro/go-micro/v2/auth"
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/config" "github.com/micro/go-micro/v2/config"
"github.com/micro/go-micro/v2/config/cmd" "github.com/micro/go-micro/v2/config/cmd"
"github.com/micro/go-micro/v2/debug/profile" "github.com/micro/go-micro/v2/debug/profile"
@ -16,6 +15,7 @@ import (
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/router" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/runtime" "github.com/micro/go-micro/v2/runtime"
"github.com/micro/go-micro/v2/selector"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
"github.com/micro/go-micro/v2/store" "github.com/micro/go-micro/v2/store"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"
@ -139,8 +139,9 @@ func Store(s store.Store) Option {
func Registry(r registry.Registry) Option { func Registry(r registry.Registry) Option {
return func(o *Options) { return func(o *Options) {
o.Registry = r o.Registry = r
// Update Client and Server // Update router
o.Client.Init(client.Registry(r)) o.Router.Init(router.Registry(r))
// Update server
o.Server.Init(server.Registry(r)) o.Server.Init(server.Registry(r))
// Update Broker // Update Broker
o.Broker.Init(broker.Registry(r)) o.Broker.Init(broker.Registry(r))

View File

@ -13,9 +13,9 @@ import (
"github.com/micro/go-micro/v2/broker" "github.com/micro/go-micro/v2/broker"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/config/cmd" "github.com/micro/go-micro/v2/config/cmd"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/selector"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
"github.com/micro/go-micro/v2/transport" "github.com/micro/go-micro/v2/transport"
) )

View File

@ -11,7 +11,6 @@ import (
"time" "time"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/codec" "github.com/micro/go-micro/v2/codec"
"github.com/micro/go-micro/v2/codec/bytes" "github.com/micro/go-micro/v2/codec/bytes"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
@ -19,6 +18,7 @@ import (
"github.com/micro/go-micro/v2/metadata" "github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/proxy" "github.com/micro/go-micro/v2/proxy"
"github.com/micro/go-micro/v2/router" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/selector/roundrobin"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
) )
@ -394,7 +394,7 @@ func (p *Proxy) ServeRequest(ctx context.Context, req server.Request, rsp server
//nolint:prealloc //nolint:prealloc
opts := []client.CallOption{ opts := []client.CallOption{
// set strategy to round robin // set strategy to round robin
client.WithSelectOption(selector.WithStrategy(selector.RoundRobin)), client.WithSelector(roundrobin.NewSelector()),
} }
// if the address is already set just serve it // if the address is already set just serve it

View File

@ -1,6 +1,8 @@
package static package static
import "github.com/micro/go-micro/v2/router" import (
"github.com/micro/go-micro/v2/router"
)
// NewRouter returns an initialized static router // NewRouter returns an initialized static router
func NewRouter(opts ...router.Option) router.Router { func NewRouter(opts ...router.Option) router.Router {
@ -39,8 +41,8 @@ func (s *static) Process(*router.Advert) error {
return nil return nil
} }
func (s *static) Lookup(...router.QueryOption) ([]router.Route, error) { func (s *static) Lookup(opts ...router.QueryOption) ([]router.Route, error) {
return nil, nil return s.table.Query(opts...)
} }
func (s *static) Watch(opts ...router.WatchOption) (router.Watcher, error) { func (s *static) Watch(opts ...router.WatchOption) (router.Watcher, error) {

View File

@ -11,6 +11,7 @@ import (
gcli "github.com/micro/go-micro/v2/client/grpc" gcli "github.com/micro/go-micro/v2/client/grpc"
"github.com/micro/go-micro/v2/errors" "github.com/micro/go-micro/v2/errors"
rmemory "github.com/micro/go-micro/v2/registry/memory" rmemory "github.com/micro/go-micro/v2/registry/memory"
"github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
gsrv "github.com/micro/go-micro/v2/server/grpc" gsrv "github.com/micro/go-micro/v2/server/grpc"
tgrpc "github.com/micro/go-micro/v2/transport/grpc" tgrpc "github.com/micro/go-micro/v2/transport/grpc"
@ -110,14 +111,17 @@ func TestGRPCServer(t *testing.T) {
r := rmemory.NewRegistry() r := rmemory.NewRegistry()
b := bmemory.NewBroker() b := bmemory.NewBroker()
tr := tgrpc.NewTransport() tr := tgrpc.NewTransport()
rtr := router.NewRouter(router.Registry(r))
s := gsrv.NewServer( s := gsrv.NewServer(
server.Broker(b), server.Broker(b),
server.Name("foo"), server.Name("foo"),
server.Registry(r), server.Registry(r),
server.Transport(tr), server.Transport(tr),
) )
c := gcli.NewClient( c := gcli.NewClient(
client.Registry(r), client.Router(rtr),
client.Broker(b), client.Broker(b),
client.Transport(tr), client.Transport(tr),
) )

View File

@ -83,11 +83,12 @@ func Server(s server.Server) Option {
func Registry(r registry.Registry) Option { func Registry(r registry.Registry) Option {
return func(o *Options) { return func(o *Options) {
o.Registry = r o.Registry = r
// Update Client and Server // Update server
o.Client.Init(client.Registry(r))
o.Server.Init(server.Registry(r)) o.Server.Init(server.Registry(r))
// Update Broker // Update Broker
o.Broker.Init(broker.Registry(r)) o.Broker.Init(broker.Registry(r))
// Update router
o.Client.Init(client.Registry(r))
} }
} }

View File

@ -8,9 +8,9 @@ import (
"net/http" "net/http"
"strings" "strings"
"github.com/micro/go-micro/v2/client/selector"
"github.com/micro/go-micro/v2/metadata" "github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/selector/random"
) )
// Write sets the status and body on a http ResponseWriter // Write sets the status and body on a http ResponseWriter
@ -47,7 +47,7 @@ func WriteInternalServerError(w http.ResponseWriter, err error) {
func NewRoundTripper(opts ...Option) http.RoundTripper { func NewRoundTripper(opts ...Option) http.RoundTripper {
options := Options{ options := Options{
Registry: registry.DefaultRegistry, Router: router.DefaultRouter,
} }
for _, o := range opts { for _, o := range opts {
o(&options) o(&options)
@ -55,7 +55,7 @@ func NewRoundTripper(opts ...Option) http.RoundTripper {
return &roundTripper{ return &roundTripper{
rt: http.DefaultTransport, rt: http.DefaultTransport,
st: selector.Random, st: random.NewSelector(),
opts: options, opts: options,
} }
} }

View File

@ -8,14 +8,15 @@ import (
"github.com/micro/go-micro/v2/registry" "github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/registry/memory" "github.com/micro/go-micro/v2/registry/memory"
"github.com/micro/go-micro/v2/router"
regRouter "github.com/micro/go-micro/v2/router/registry"
) )
func TestRoundTripper(t *testing.T) { func TestRoundTripper(t *testing.T) {
m := memory.NewRegistry() m := memory.NewRegistry()
r := regRouter.NewRouter(router.Registry(m))
rt := NewRoundTripper( rt := NewRoundTripper(WithRouter(r))
WithRegistry(m),
)
http.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { http.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) {
w.Write([]byte(`hello world`)) w.Write([]byte(`hello world`))

View File

@ -1,17 +1,15 @@
package http package http
import ( import "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/registry"
)
type Options struct { type Options struct {
Registry registry.Registry Router router.Router
} }
type Option func(*Options) type Option func(*Options)
func WithRegistry(r registry.Registry) Option { func WithRouter(r router.Router) Option {
return func(o *Options) { return func(o *Options) {
o.Registry = r o.Router = r
} }
} }

View File

@ -4,34 +4,35 @@ import (
"errors" "errors"
"net/http" "net/http"
"github.com/micro/go-micro/v2/client/selector" "github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/selector"
) )
type roundTripper struct { type roundTripper struct {
rt http.RoundTripper rt http.RoundTripper
st selector.Strategy st selector.Selector
opts Options opts Options
} }
func (r *roundTripper) RoundTrip(req *http.Request) (*http.Response, error) { func (r *roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
s, err := r.opts.Registry.GetService(req.URL.Host) routes, err := r.opts.Router.Lookup(router.QueryService(req.URL.Host))
if err != nil { if err != nil {
return nil, err return nil, err
} }
next := r.st(s)
// rudimentary retry 3 times // rudimentary retry 3 times
for i := 0; i < 3; i++ { for i := 0; i < 3; i++ {
n, err := next() route, err := r.st.Select(routes)
if err != nil { if err != nil {
continue continue
} }
req.URL.Host = n.Address
req.URL.Host = route.Address
w, err := r.rt.RoundTrip(req) w, err := r.rt.RoundTrip(req)
if err != nil { if err != nil {
continue continue
} }
return w, nil return w, nil
} }

View File

@ -8,6 +8,7 @@ import (
bmemory "github.com/micro/go-micro/v2/broker/memory" bmemory "github.com/micro/go-micro/v2/broker/memory"
"github.com/micro/go-micro/v2/client" "github.com/micro/go-micro/v2/client"
rmemory "github.com/micro/go-micro/v2/registry/memory" rmemory "github.com/micro/go-micro/v2/registry/memory"
"github.com/micro/go-micro/v2/router"
"github.com/micro/go-micro/v2/server" "github.com/micro/go-micro/v2/server"
tmemory "github.com/micro/go-micro/v2/transport/memory" tmemory "github.com/micro/go-micro/v2/transport/memory"
wrapper "github.com/micro/go-micro/v2/util/wrapper" wrapper "github.com/micro/go-micro/v2/util/wrapper"
@ -36,6 +37,7 @@ func TestStaticClientWrapper(t *testing.T) {
reg := rmemory.NewRegistry() reg := rmemory.NewRegistry()
brk := bmemory.NewBroker(broker.Registry(reg)) brk := bmemory.NewBroker(broker.Registry(reg))
tr := tmemory.NewTransport() tr := tmemory.NewTransport()
rtr := router.NewRouter(router.Registry(reg))
srv := server.NewServer( srv := server.NewServer(
server.Broker(brk), server.Broker(brk),
@ -53,7 +55,7 @@ func TestStaticClientWrapper(t *testing.T) {
} }
cli := client.NewClient( cli := client.NewClient(
client.Registry(reg), client.Router(rtr),
client.Broker(brk), client.Broker(brk),
client.Transport(tr), client.Transport(tr),
) )

View File

@ -120,7 +120,7 @@ func (s *service) register() error {
return nil return nil
} }
// default to service registry // default to service registry
r := s.opts.Service.Client().Options().Registry r := s.opts.Service.Options().Registry
// switch to option if specified // switch to option if specified
if s.opts.Registry != nil { if s.opts.Registry != nil {
r = s.opts.Registry r = s.opts.Registry
@ -173,7 +173,7 @@ func (s *service) deregister() error {
return nil return nil
} }
// default to service registry // default to service registry
r := s.opts.Service.Client().Options().Registry r := s.opts.Service.Options().Registry
// switch to option if specified // switch to option if specified
if s.opts.Registry != nil { if s.opts.Registry != nil {
r = s.opts.Registry r = s.opts.Registry
@ -302,7 +302,7 @@ func (s *service) stop() error {
func (s *service) Client() *http.Client { func (s *service) Client() *http.Client {
rt := mhttp.NewRoundTripper( rt := mhttp.NewRoundTripper(
mhttp.WithRegistry(s.opts.Registry), mhttp.WithRouter(s.opts.Service.Options().Router),
) )
return &http.Client{ return &http.Client{
Transport: rt, Transport: rt,