commit
8e1ff80b9e
@ -12,7 +12,7 @@ but everything can be easily swapped out.
|
|||||||
|
|
||||||
Plugins are available at [github.com/micro/go-plugins](https://github.com/micro/go-plugins).
|
Plugins are available at [github.com/micro/go-plugins](https://github.com/micro/go-plugins).
|
||||||
|
|
||||||
Follow us on [Twitter](https://twitter.com/microhq) or join the [Slack](https://micro.mu/slack) community.
|
Follow us on [Twitter](https://twitter.com/microhq) or join the [Community](https://micro.mu/slack).
|
||||||
|
|
||||||
## Features
|
## Features
|
||||||
|
|
||||||
@ -32,11 +32,11 @@ and server handle this by default. This includes protobuf and json by default.
|
|||||||
|
|
||||||
- **Request/Response** - RPC based request/response with support for bidirectional streaming. We provide an abstraction for synchronous
|
- **Request/Response** - RPC based request/response with support for bidirectional streaming. We provide an abstraction for synchronous
|
||||||
communication. A request made to a service will be automatically resolved, load balanced, dialled and streamed. The default
|
communication. A request made to a service will be automatically resolved, load balanced, dialled and streamed. The default
|
||||||
transport is http/1.1 or http2 when tls is enabled.
|
transport is [gRPC](https://grpc.io/).
|
||||||
|
|
||||||
- **Async Messaging** - PubSub is built in as a first class citizen for asynchronous communication and event driven architectures.
|
- **Async Messaging** - PubSub is built in as a first class citizen for asynchronous communication and event driven architectures.
|
||||||
Event notifications are a core pattern in micro service development. The default messaging is point-to-point http/1.1 or http2 when tls
|
Event notifications are a core pattern in micro service development. The default messaging system is an embedded [NATS](https://nats.io/)
|
||||||
is enabled.
|
server.
|
||||||
|
|
||||||
- **Pluggable Interfaces** - Go Micro makes use of Go interfaces for each distributed system abstraction. Because of this these interfaces
|
- **Pluggable Interfaces** - Go Micro makes use of Go interfaces for each distributed system abstraction. Because of this these interfaces
|
||||||
are pluggable and allows Go Micro to be runtime agnostic. You can plugin any underlying technology. Find plugins in
|
are pluggable and allows Go Micro to be runtime agnostic. You can plugin any underlying technology. Find plugins in
|
||||||
|
@ -38,13 +38,9 @@ type Subscriber interface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
DefaultBroker Broker = newHttpBroker()
|
DefaultBroker Broker = NewBroker()
|
||||||
)
|
)
|
||||||
|
|
||||||
func NewBroker(opts ...Option) Broker {
|
|
||||||
return newHttpBroker(opts...)
|
|
||||||
}
|
|
||||||
|
|
||||||
func Init(opts ...Option) error {
|
func Init(opts ...Option) error {
|
||||||
return DefaultBroker.Init(opts...)
|
return DefaultBroker.Init(opts...)
|
||||||
}
|
}
|
||||||
|
@ -1,47 +0,0 @@
|
|||||||
package broker
|
|
||||||
|
|
||||||
import (
|
|
||||||
"github.com/micro/go-micro/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",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}
|
|
||||||
)
|
|
459
broker/default.go
Normal file
459
broker/default.go
Normal file
@ -0,0 +1,459 @@
|
|||||||
|
package broker
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"errors"
|
||||||
|
"net"
|
||||||
|
"net/url"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/micro/go-micro/codec/json"
|
||||||
|
"github.com/micro/go-micro/registry"
|
||||||
|
"github.com/micro/go-micro/util/addr"
|
||||||
|
"github.com/micro/go-micro/util/log"
|
||||||
|
"github.com/nats-io/nats-server/v2/server"
|
||||||
|
nats "github.com/nats-io/nats.go"
|
||||||
|
)
|
||||||
|
|
||||||
|
type natsBroker struct {
|
||||||
|
sync.Once
|
||||||
|
sync.RWMutex
|
||||||
|
|
||||||
|
// indicate if we're connected
|
||||||
|
connected bool
|
||||||
|
|
||||||
|
// address to bind routes to
|
||||||
|
addrs []string
|
||||||
|
// servers for the client
|
||||||
|
servers []string
|
||||||
|
|
||||||
|
// client connection and nats opts
|
||||||
|
conn *nats.Conn
|
||||||
|
opts Options
|
||||||
|
nopts nats.Options
|
||||||
|
|
||||||
|
// should we drain the connection
|
||||||
|
drain bool
|
||||||
|
closeCh chan (error)
|
||||||
|
|
||||||
|
// embedded server
|
||||||
|
server *server.Server
|
||||||
|
// configure to use local server
|
||||||
|
local bool
|
||||||
|
// server exit channel
|
||||||
|
exit chan bool
|
||||||
|
}
|
||||||
|
|
||||||
|
type subscriber struct {
|
||||||
|
s *nats.Subscription
|
||||||
|
opts SubscribeOptions
|
||||||
|
}
|
||||||
|
|
||||||
|
type publication struct {
|
||||||
|
t string
|
||||||
|
m *Message
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *publication) Topic() string {
|
||||||
|
return p.t
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *publication) Message() *Message {
|
||||||
|
return p.m
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *publication) Ack() error {
|
||||||
|
// nats does not support acking
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *subscriber) Options() SubscribeOptions {
|
||||||
|
return s.opts
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *subscriber) Topic() string {
|
||||||
|
return s.s.Subject
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *subscriber) Unsubscribe() error {
|
||||||
|
return s.s.Unsubscribe()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Address() string {
|
||||||
|
n.RLock()
|
||||||
|
defer n.RUnlock()
|
||||||
|
|
||||||
|
if n.server != nil {
|
||||||
|
return n.server.ClusterAddr().String()
|
||||||
|
}
|
||||||
|
|
||||||
|
if n.conn != nil && n.conn.IsConnected() {
|
||||||
|
return n.conn.ConnectedUrl()
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(n.addrs) > 0 {
|
||||||
|
return n.addrs[0]
|
||||||
|
}
|
||||||
|
|
||||||
|
return "127.0.0.1:-1"
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) setAddrs(addrs []string) []string {
|
||||||
|
//nolint:prealloc
|
||||||
|
var cAddrs []string
|
||||||
|
for _, addr := range addrs {
|
||||||
|
if len(addr) == 0 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if !strings.HasPrefix(addr, "nats://") {
|
||||||
|
addr = "nats://" + addr
|
||||||
|
}
|
||||||
|
cAddrs = append(cAddrs, addr)
|
||||||
|
}
|
||||||
|
// if there's no address and we weren't told to
|
||||||
|
// embed a local server then use the default url
|
||||||
|
if len(cAddrs) == 0 && !n.local {
|
||||||
|
cAddrs = []string{nats.DefaultURL}
|
||||||
|
}
|
||||||
|
return cAddrs
|
||||||
|
}
|
||||||
|
|
||||||
|
// serve stats a local nats server if needed
|
||||||
|
func (n *natsBroker) serve(exit chan bool) error {
|
||||||
|
// local server address
|
||||||
|
host := "127.0.0.1"
|
||||||
|
port := -1
|
||||||
|
|
||||||
|
// cluster address
|
||||||
|
caddr := "0.0.0.0"
|
||||||
|
cport := -1
|
||||||
|
|
||||||
|
// with no address we just default it
|
||||||
|
// this is a local client address
|
||||||
|
if len(n.addrs) > 0 {
|
||||||
|
address := n.addrs[0]
|
||||||
|
if strings.HasPrefix(address, "nats://") {
|
||||||
|
address = strings.TrimPrefix(address, "nats://")
|
||||||
|
}
|
||||||
|
|
||||||
|
// parse out the address
|
||||||
|
h, p, err := net.SplitHostPort(address)
|
||||||
|
if err == nil {
|
||||||
|
caddr = h
|
||||||
|
cport, _ = strconv.Atoi(p)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// 1. create new server
|
||||||
|
// 2. register the server
|
||||||
|
// 3. connect to other servers
|
||||||
|
|
||||||
|
// set cluster opts
|
||||||
|
cOpts := server.ClusterOpts{
|
||||||
|
Host: caddr,
|
||||||
|
Port: cport,
|
||||||
|
}
|
||||||
|
|
||||||
|
// get the routes for other nodes
|
||||||
|
var routes []*url.URL
|
||||||
|
|
||||||
|
// get existing nats servers to connect to
|
||||||
|
services, err := n.opts.Registry.GetService("go.micro.nats.broker")
|
||||||
|
if err == nil {
|
||||||
|
for _, service := range services {
|
||||||
|
for _, node := range service.Nodes {
|
||||||
|
u, err := url.Parse("nats://" + node.Address)
|
||||||
|
if err != nil {
|
||||||
|
log.Log(err)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
// append to the cluster routes
|
||||||
|
routes = append(routes, u)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// try get existing server
|
||||||
|
s := n.server
|
||||||
|
|
||||||
|
if s != nil {
|
||||||
|
// stop the existing server
|
||||||
|
s.Shutdown()
|
||||||
|
}
|
||||||
|
|
||||||
|
s, err = server.NewServer(&server.Options{
|
||||||
|
// Specify the host
|
||||||
|
Host: host,
|
||||||
|
// Use a random port
|
||||||
|
Port: port,
|
||||||
|
// Set the cluster ops
|
||||||
|
Cluster: cOpts,
|
||||||
|
// Set the routes
|
||||||
|
Routes: routes,
|
||||||
|
NoLog: true,
|
||||||
|
NoSigs: true,
|
||||||
|
MaxControlLine: 2048,
|
||||||
|
TLSConfig: n.opts.TLSConfig,
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// save the server
|
||||||
|
n.server = s
|
||||||
|
|
||||||
|
// start the server
|
||||||
|
go s.Start()
|
||||||
|
|
||||||
|
var ready bool
|
||||||
|
|
||||||
|
// wait till its ready for connections
|
||||||
|
for i := 0; i < 3; i++ {
|
||||||
|
if s.ReadyForConnections(time.Second) {
|
||||||
|
ready = true
|
||||||
|
break
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if !ready {
|
||||||
|
return errors.New("server not ready")
|
||||||
|
}
|
||||||
|
|
||||||
|
// set the client address
|
||||||
|
n.servers = []string{s.ClientURL()}
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
var advertise string
|
||||||
|
|
||||||
|
// parse out the address
|
||||||
|
_, port, err := net.SplitHostPort(s.ClusterAddr().String())
|
||||||
|
if err == nil {
|
||||||
|
addr, _ := addr.Extract("")
|
||||||
|
advertise = net.JoinHostPort(addr, port)
|
||||||
|
} else {
|
||||||
|
s.ClusterAddr().String()
|
||||||
|
}
|
||||||
|
|
||||||
|
// register the cluster address
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-exit:
|
||||||
|
// deregister on exit
|
||||||
|
n.opts.Registry.Deregister(®istry.Service{
|
||||||
|
Name: "go.micro.nats.broker",
|
||||||
|
Version: "v2",
|
||||||
|
Nodes: []*registry.Node{
|
||||||
|
{Id: s.ID(), Address: advertise},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
s.Shutdown()
|
||||||
|
return
|
||||||
|
default:
|
||||||
|
// register the broker
|
||||||
|
n.opts.Registry.Register(®istry.Service{
|
||||||
|
Name: "go.micro.nats.broker",
|
||||||
|
Version: "v2",
|
||||||
|
Nodes: []*registry.Node{
|
||||||
|
{Id: s.ID(), Address: advertise},
|
||||||
|
},
|
||||||
|
}, registry.RegisterTTL(time.Minute))
|
||||||
|
time.Sleep(time.Minute)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Connect() error {
|
||||||
|
n.Lock()
|
||||||
|
defer n.Unlock()
|
||||||
|
|
||||||
|
if !n.connected {
|
||||||
|
// create exit chan
|
||||||
|
n.exit = make(chan bool)
|
||||||
|
|
||||||
|
// start the local server
|
||||||
|
if err := n.serve(n.exit); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// set to connected
|
||||||
|
n.connected = true
|
||||||
|
}
|
||||||
|
|
||||||
|
status := nats.CLOSED
|
||||||
|
if n.conn != nil {
|
||||||
|
status = n.conn.Status()
|
||||||
|
}
|
||||||
|
|
||||||
|
switch status {
|
||||||
|
case nats.CONNECTED, nats.RECONNECTING, nats.CONNECTING:
|
||||||
|
return nil
|
||||||
|
default: // DISCONNECTED or CLOSED or DRAINING
|
||||||
|
opts := n.nopts
|
||||||
|
opts.Servers = n.servers
|
||||||
|
opts.Secure = n.opts.Secure
|
||||||
|
opts.TLSConfig = n.opts.TLSConfig
|
||||||
|
|
||||||
|
// secure might not be set
|
||||||
|
if n.opts.TLSConfig != nil {
|
||||||
|
opts.Secure = true
|
||||||
|
}
|
||||||
|
|
||||||
|
c, err := opts.Connect()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
n.conn = c
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Disconnect() error {
|
||||||
|
n.RLock()
|
||||||
|
defer n.RUnlock()
|
||||||
|
|
||||||
|
if !n.connected {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// drain the connection if specified
|
||||||
|
if n.drain {
|
||||||
|
n.conn.Drain()
|
||||||
|
return <-n.closeCh
|
||||||
|
}
|
||||||
|
|
||||||
|
// close the client connection
|
||||||
|
n.conn.Close()
|
||||||
|
|
||||||
|
// shutdown the local server
|
||||||
|
// and deregister
|
||||||
|
select {
|
||||||
|
case <-n.exit:
|
||||||
|
default:
|
||||||
|
close(n.exit)
|
||||||
|
}
|
||||||
|
|
||||||
|
// set not connected
|
||||||
|
n.connected = false
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Init(opts ...Option) error {
|
||||||
|
n.setOption(opts...)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Options() Options {
|
||||||
|
return n.opts
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Publish(topic string, msg *Message, opts ...PublishOption) error {
|
||||||
|
b, err := n.opts.Codec.Marshal(msg)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
n.RLock()
|
||||||
|
defer n.RUnlock()
|
||||||
|
return n.conn.Publish(topic, b)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) Subscribe(topic string, handler Handler, opts ...SubscribeOption) (Subscriber, error) {
|
||||||
|
if n.conn == nil {
|
||||||
|
return nil, errors.New("not connected")
|
||||||
|
}
|
||||||
|
|
||||||
|
opt := SubscribeOptions{
|
||||||
|
AutoAck: true,
|
||||||
|
Context: context.Background(),
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, o := range opts {
|
||||||
|
o(&opt)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn := func(msg *nats.Msg) {
|
||||||
|
var m Message
|
||||||
|
if err := n.opts.Codec.Unmarshal(msg.Data, &m); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
handler(&publication{m: &m, t: msg.Subject})
|
||||||
|
}
|
||||||
|
|
||||||
|
var sub *nats.Subscription
|
||||||
|
var err error
|
||||||
|
|
||||||
|
n.RLock()
|
||||||
|
if len(opt.Queue) > 0 {
|
||||||
|
sub, err = n.conn.QueueSubscribe(topic, opt.Queue, fn)
|
||||||
|
} else {
|
||||||
|
sub, err = n.conn.Subscribe(topic, fn)
|
||||||
|
}
|
||||||
|
n.RUnlock()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return &subscriber{s: sub, opts: opt}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) String() string {
|
||||||
|
return "nats-e"
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) setOption(opts ...Option) {
|
||||||
|
for _, o := range opts {
|
||||||
|
o(&n.opts)
|
||||||
|
}
|
||||||
|
|
||||||
|
n.Once.Do(func() {
|
||||||
|
n.nopts = nats.GetDefaultOptions()
|
||||||
|
})
|
||||||
|
|
||||||
|
// local embedded server
|
||||||
|
n.local = true
|
||||||
|
// set to drain
|
||||||
|
n.drain = true
|
||||||
|
|
||||||
|
if !n.opts.Secure {
|
||||||
|
n.opts.Secure = n.nopts.Secure
|
||||||
|
}
|
||||||
|
|
||||||
|
if n.opts.TLSConfig == nil {
|
||||||
|
n.opts.TLSConfig = n.nopts.TLSConfig
|
||||||
|
}
|
||||||
|
|
||||||
|
n.addrs = n.setAddrs(n.opts.Addrs)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) onClose(conn *nats.Conn) {
|
||||||
|
n.closeCh <- nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (n *natsBroker) onAsyncError(conn *nats.Conn, sub *nats.Subscription, err error) {
|
||||||
|
// There are kinds of different async error nats might callback, but we are interested
|
||||||
|
// in ErrDrainTimeout only here.
|
||||||
|
if err == nats.ErrDrainTimeout {
|
||||||
|
n.closeCh <- err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewBroker(opts ...Option) Broker {
|
||||||
|
options := Options{
|
||||||
|
// Default codec
|
||||||
|
Codec: json.Marshaler{},
|
||||||
|
Context: context.Background(),
|
||||||
|
Registry: registry.DefaultRegistry,
|
||||||
|
}
|
||||||
|
|
||||||
|
n := &natsBroker{
|
||||||
|
opts: options,
|
||||||
|
}
|
||||||
|
n.setOption(opts...)
|
||||||
|
|
||||||
|
return n
|
||||||
|
}
|
@ -1,11 +0,0 @@
|
|||||||
// Package http provides a http based message broker
|
|
||||||
package http
|
|
||||||
|
|
||||||
import (
|
|
||||||
"github.com/micro/go-micro/broker"
|
|
||||||
)
|
|
||||||
|
|
||||||
// NewBroker returns a new http broker
|
|
||||||
func NewBroker(opts ...broker.Option) broker.Broker {
|
|
||||||
return broker.NewBroker(opts...)
|
|
||||||
}
|
|
@ -1,23 +0,0 @@
|
|||||||
package http
|
|
||||||
|
|
||||||
import (
|
|
||||||
"context"
|
|
||||||
"net/http"
|
|
||||||
|
|
||||||
"github.com/micro/go-micro/broker"
|
|
||||||
)
|
|
||||||
|
|
||||||
// Handle registers the handler for the given pattern.
|
|
||||||
func Handle(pattern string, handler http.Handler) broker.Option {
|
|
||||||
return func(o *broker.Options) {
|
|
||||||
if o.Context == nil {
|
|
||||||
o.Context = context.Background()
|
|
||||||
}
|
|
||||||
handlers, ok := o.Context.Value("http_handlers").(map[string]http.Handler)
|
|
||||||
if !ok {
|
|
||||||
handlers = make(map[string]http.Handler)
|
|
||||||
}
|
|
||||||
handlers[pattern] = handler
|
|
||||||
o.Context = context.WithValue(o.Context, "http_handlers", handlers)
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,698 +0,0 @@
|
|||||||
package broker
|
|
||||||
|
|
||||||
import (
|
|
||||||
"bytes"
|
|
||||||
"context"
|
|
||||||
"crypto/tls"
|
|
||||||
"errors"
|
|
||||||
"fmt"
|
|
||||||
"io"
|
|
||||||
"io/ioutil"
|
|
||||||
"math/rand"
|
|
||||||
"net"
|
|
||||||
"net/http"
|
|
||||||
"net/url"
|
|
||||||
"runtime"
|
|
||||||
"sync"
|
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/google/uuid"
|
|
||||||
"github.com/micro/go-micro/codec/json"
|
|
||||||
merr "github.com/micro/go-micro/errors"
|
|
||||||
"github.com/micro/go-micro/registry"
|
|
||||||
"github.com/micro/go-micro/registry/cache"
|
|
||||||
maddr "github.com/micro/go-micro/util/addr"
|
|
||||||
mnet "github.com/micro/go-micro/util/net"
|
|
||||||
mls "github.com/micro/go-micro/util/tls"
|
|
||||||
"golang.org/x/net/http2"
|
|
||||||
)
|
|
||||||
|
|
||||||
// HTTP Broker is a point to point async broker
|
|
||||||
type httpBroker struct {
|
|
||||||
id string
|
|
||||||
address string
|
|
||||||
opts Options
|
|
||||||
|
|
||||||
mux *http.ServeMux
|
|
||||||
|
|
||||||
c *http.Client
|
|
||||||
r registry.Registry
|
|
||||||
|
|
||||||
sync.RWMutex
|
|
||||||
subscribers map[string][]*httpSubscriber
|
|
||||||
running bool
|
|
||||||
exit chan chan error
|
|
||||||
|
|
||||||
// offline message inbox
|
|
||||||
mtx sync.RWMutex
|
|
||||||
inbox map[string][][]byte
|
|
||||||
}
|
|
||||||
|
|
||||||
type httpSubscriber struct {
|
|
||||||
opts SubscribeOptions
|
|
||||||
id string
|
|
||||||
topic string
|
|
||||||
fn Handler
|
|
||||||
svc *registry.Service
|
|
||||||
hb *httpBroker
|
|
||||||
}
|
|
||||||
|
|
||||||
type httpEvent struct {
|
|
||||||
m *Message
|
|
||||||
t string
|
|
||||||
}
|
|
||||||
|
|
||||||
var (
|
|
||||||
DefaultSubPath = "/_sub"
|
|
||||||
serviceName = "go.micro.http.broker"
|
|
||||||
broadcastVersion = "ff.http.broadcast"
|
|
||||||
registerTTL = time.Minute
|
|
||||||
registerInterval = time.Second * 30
|
|
||||||
)
|
|
||||||
|
|
||||||
func init() {
|
|
||||||
rand.Seed(time.Now().Unix())
|
|
||||||
}
|
|
||||||
|
|
||||||
func newTransport(config *tls.Config) *http.Transport {
|
|
||||||
if config == nil {
|
|
||||||
config = &tls.Config{
|
|
||||||
InsecureSkipVerify: true,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
dialTLS := func(network string, addr string) (net.Conn, error) {
|
|
||||||
return tls.Dial(network, addr, config)
|
|
||||||
}
|
|
||||||
|
|
||||||
t := &http.Transport{
|
|
||||||
Proxy: http.ProxyFromEnvironment,
|
|
||||||
Dial: (&net.Dialer{
|
|
||||||
Timeout: 30 * time.Second,
|
|
||||||
KeepAlive: 30 * time.Second,
|
|
||||||
}).Dial,
|
|
||||||
TLSHandshakeTimeout: 10 * time.Second,
|
|
||||||
DialTLS: dialTLS,
|
|
||||||
}
|
|
||||||
runtime.SetFinalizer(&t, func(tr **http.Transport) {
|
|
||||||
(*tr).CloseIdleConnections()
|
|
||||||
})
|
|
||||||
|
|
||||||
// setup http2
|
|
||||||
http2.ConfigureTransport(t)
|
|
||||||
|
|
||||||
return t
|
|
||||||
}
|
|
||||||
|
|
||||||
func newHttpBroker(opts ...Option) Broker {
|
|
||||||
options := Options{
|
|
||||||
Codec: json.Marshaler{},
|
|
||||||
Context: context.TODO(),
|
|
||||||
Registry: registry.DefaultRegistry,
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, o := range opts {
|
|
||||||
o(&options)
|
|
||||||
}
|
|
||||||
|
|
||||||
// set address
|
|
||||||
addr := ":0"
|
|
||||||
if len(options.Addrs) > 0 && len(options.Addrs[0]) > 0 {
|
|
||||||
addr = options.Addrs[0]
|
|
||||||
}
|
|
||||||
|
|
||||||
h := &httpBroker{
|
|
||||||
id: uuid.New().String(),
|
|
||||||
address: addr,
|
|
||||||
opts: options,
|
|
||||||
r: options.Registry,
|
|
||||||
c: &http.Client{Transport: newTransport(options.TLSConfig)},
|
|
||||||
subscribers: make(map[string][]*httpSubscriber),
|
|
||||||
exit: make(chan chan error),
|
|
||||||
mux: http.NewServeMux(),
|
|
||||||
inbox: make(map[string][][]byte),
|
|
||||||
}
|
|
||||||
|
|
||||||
// specify the message handler
|
|
||||||
h.mux.Handle(DefaultSubPath, h)
|
|
||||||
|
|
||||||
// get optional handlers
|
|
||||||
if h.opts.Context != nil {
|
|
||||||
handlers, ok := h.opts.Context.Value("http_handlers").(map[string]http.Handler)
|
|
||||||
if ok {
|
|
||||||
for pattern, handler := range handlers {
|
|
||||||
h.mux.Handle(pattern, handler)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return h
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpEvent) Ack() error {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpEvent) Message() *Message {
|
|
||||||
return h.m
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpEvent) Topic() string {
|
|
||||||
return h.t
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpSubscriber) Options() SubscribeOptions {
|
|
||||||
return h.opts
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpSubscriber) Topic() string {
|
|
||||||
return h.topic
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpSubscriber) Unsubscribe() error {
|
|
||||||
return h.hb.unsubscribe(h)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) saveMessage(topic string, msg []byte) {
|
|
||||||
h.mtx.Lock()
|
|
||||||
defer h.mtx.Unlock()
|
|
||||||
|
|
||||||
// get messages
|
|
||||||
c := h.inbox[topic]
|
|
||||||
|
|
||||||
// save message
|
|
||||||
c = append(c, msg)
|
|
||||||
|
|
||||||
// max length 64
|
|
||||||
if len(c) > 64 {
|
|
||||||
c = c[:64]
|
|
||||||
}
|
|
||||||
|
|
||||||
// save inbox
|
|
||||||
h.inbox[topic] = c
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) getMessage(topic string, num int) [][]byte {
|
|
||||||
h.mtx.Lock()
|
|
||||||
defer h.mtx.Unlock()
|
|
||||||
|
|
||||||
// get messages
|
|
||||||
c, ok := h.inbox[topic]
|
|
||||||
if !ok {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// more message than requests
|
|
||||||
if len(c) >= num {
|
|
||||||
msg := c[:num]
|
|
||||||
h.inbox[topic] = c[num:]
|
|
||||||
return msg
|
|
||||||
}
|
|
||||||
|
|
||||||
// reset inbox
|
|
||||||
h.inbox[topic] = nil
|
|
||||||
|
|
||||||
// return all messages
|
|
||||||
return c
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) subscribe(s *httpSubscriber) error {
|
|
||||||
h.Lock()
|
|
||||||
defer h.Unlock()
|
|
||||||
|
|
||||||
if err := h.r.Register(s.svc, registry.RegisterTTL(registerTTL)); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
h.subscribers[s.topic] = append(h.subscribers[s.topic], s)
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) unsubscribe(s *httpSubscriber) error {
|
|
||||||
h.Lock()
|
|
||||||
defer h.Unlock()
|
|
||||||
|
|
||||||
//nolint:prealloc
|
|
||||||
var subscribers []*httpSubscriber
|
|
||||||
|
|
||||||
// look for subscriber
|
|
||||||
for _, sub := range h.subscribers[s.topic] {
|
|
||||||
// deregister and skip forward
|
|
||||||
if sub == s {
|
|
||||||
_ = h.r.Deregister(sub.svc)
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// keep subscriber
|
|
||||||
subscribers = append(subscribers, sub)
|
|
||||||
}
|
|
||||||
|
|
||||||
// set subscribers
|
|
||||||
h.subscribers[s.topic] = subscribers
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) run(l net.Listener) {
|
|
||||||
t := time.NewTicker(registerInterval)
|
|
||||||
defer t.Stop()
|
|
||||||
|
|
||||||
for {
|
|
||||||
select {
|
|
||||||
// heartbeat for each subscriber
|
|
||||||
case <-t.C:
|
|
||||||
h.RLock()
|
|
||||||
for _, subs := range h.subscribers {
|
|
||||||
for _, sub := range subs {
|
|
||||||
_ = h.r.Register(sub.svc, registry.RegisterTTL(registerTTL))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
// received exit signal
|
|
||||||
case ch := <-h.exit:
|
|
||||||
ch <- l.Close()
|
|
||||||
h.RLock()
|
|
||||||
for _, subs := range h.subscribers {
|
|
||||||
for _, sub := range subs {
|
|
||||||
_ = h.r.Deregister(sub.svc)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) ServeHTTP(w http.ResponseWriter, req *http.Request) {
|
|
||||||
if req.Method != "POST" {
|
|
||||||
err := merr.BadRequest("go.micro.broker", "Method not allowed")
|
|
||||||
http.Error(w, err.Error(), http.StatusMethodNotAllowed)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
defer req.Body.Close()
|
|
||||||
|
|
||||||
req.ParseForm()
|
|
||||||
|
|
||||||
b, err := ioutil.ReadAll(req.Body)
|
|
||||||
if err != nil {
|
|
||||||
errr := merr.InternalServerError("go.micro.broker", "Error reading request body: %v", err)
|
|
||||||
w.WriteHeader(500)
|
|
||||||
w.Write([]byte(errr.Error()))
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
var m *Message
|
|
||||||
if err = h.opts.Codec.Unmarshal(b, &m); err != nil {
|
|
||||||
errr := merr.InternalServerError("go.micro.broker", "Error parsing request body: %v", err)
|
|
||||||
w.WriteHeader(500)
|
|
||||||
w.Write([]byte(errr.Error()))
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
topic := m.Header[":topic"]
|
|
||||||
delete(m.Header, ":topic")
|
|
||||||
|
|
||||||
if len(topic) == 0 {
|
|
||||||
errr := merr.InternalServerError("go.micro.broker", "Topic not found")
|
|
||||||
w.WriteHeader(500)
|
|
||||||
w.Write([]byte(errr.Error()))
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
p := &httpEvent{m: m, t: topic}
|
|
||||||
id := req.Form.Get("id")
|
|
||||||
|
|
||||||
//nolint:prealloc
|
|
||||||
var subs []Handler
|
|
||||||
|
|
||||||
h.RLock()
|
|
||||||
for _, subscriber := range h.subscribers[topic] {
|
|
||||||
if id != subscriber.id {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
subs = append(subs, subscriber.fn)
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
|
|
||||||
// execute the handler
|
|
||||||
for _, fn := range subs {
|
|
||||||
fn(p)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Address() string {
|
|
||||||
h.RLock()
|
|
||||||
defer h.RUnlock()
|
|
||||||
return h.address
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Connect() error {
|
|
||||||
h.RLock()
|
|
||||||
if h.running {
|
|
||||||
h.RUnlock()
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
|
|
||||||
h.Lock()
|
|
||||||
defer h.Unlock()
|
|
||||||
|
|
||||||
var l net.Listener
|
|
||||||
var err error
|
|
||||||
|
|
||||||
if h.opts.Secure || h.opts.TLSConfig != nil {
|
|
||||||
config := h.opts.TLSConfig
|
|
||||||
|
|
||||||
fn := func(addr string) (net.Listener, error) {
|
|
||||||
if config == nil {
|
|
||||||
hosts := []string{addr}
|
|
||||||
|
|
||||||
// check if its a valid host:port
|
|
||||||
if host, _, err := net.SplitHostPort(addr); err == nil {
|
|
||||||
if len(host) == 0 {
|
|
||||||
hosts = maddr.IPs()
|
|
||||||
} else {
|
|
||||||
hosts = []string{host}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// generate a certificate
|
|
||||||
cert, err := mls.Certificate(hosts...)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
config = &tls.Config{Certificates: []tls.Certificate{cert}}
|
|
||||||
}
|
|
||||||
return tls.Listen("tcp", addr, config)
|
|
||||||
}
|
|
||||||
|
|
||||||
l, err = mnet.Listen(h.address, fn)
|
|
||||||
} else {
|
|
||||||
fn := func(addr string) (net.Listener, error) {
|
|
||||||
return net.Listen("tcp", addr)
|
|
||||||
}
|
|
||||||
|
|
||||||
l, err = mnet.Listen(h.address, fn)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
addr := h.address
|
|
||||||
h.address = l.Addr().String()
|
|
||||||
|
|
||||||
go http.Serve(l, h.mux)
|
|
||||||
go func() {
|
|
||||||
h.run(l)
|
|
||||||
h.Lock()
|
|
||||||
h.opts.Addrs = []string{addr}
|
|
||||||
h.address = addr
|
|
||||||
h.Unlock()
|
|
||||||
}()
|
|
||||||
|
|
||||||
// get registry
|
|
||||||
reg, ok := h.opts.Context.Value(registryKey).(registry.Registry)
|
|
||||||
if !ok {
|
|
||||||
reg = registry.DefaultRegistry
|
|
||||||
}
|
|
||||||
// set cache
|
|
||||||
h.r = cache.New(reg)
|
|
||||||
|
|
||||||
// set running
|
|
||||||
h.running = true
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Disconnect() error {
|
|
||||||
h.RLock()
|
|
||||||
if !h.running {
|
|
||||||
h.RUnlock()
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
|
|
||||||
h.Lock()
|
|
||||||
defer h.Unlock()
|
|
||||||
|
|
||||||
// stop cache
|
|
||||||
rc, ok := h.r.(cache.Cache)
|
|
||||||
if ok {
|
|
||||||
rc.Stop()
|
|
||||||
}
|
|
||||||
|
|
||||||
// exit and return err
|
|
||||||
ch := make(chan error)
|
|
||||||
h.exit <- ch
|
|
||||||
err := <-ch
|
|
||||||
|
|
||||||
// set not running
|
|
||||||
h.running = false
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Init(opts ...Option) error {
|
|
||||||
h.RLock()
|
|
||||||
if h.running {
|
|
||||||
h.RUnlock()
|
|
||||||
return errors.New("cannot init while connected")
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
|
|
||||||
h.Lock()
|
|
||||||
defer h.Unlock()
|
|
||||||
|
|
||||||
for _, o := range opts {
|
|
||||||
o(&h.opts)
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(h.opts.Addrs) > 0 && len(h.opts.Addrs[0]) > 0 {
|
|
||||||
h.address = h.opts.Addrs[0]
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(h.id) == 0 {
|
|
||||||
h.id = "go.micro.http.broker-" + uuid.New().String()
|
|
||||||
}
|
|
||||||
|
|
||||||
// get registry
|
|
||||||
reg, ok := h.opts.Context.Value(registryKey).(registry.Registry)
|
|
||||||
if !ok {
|
|
||||||
reg = registry.DefaultRegistry
|
|
||||||
}
|
|
||||||
|
|
||||||
// get cache
|
|
||||||
if rc, ok := h.r.(cache.Cache); ok {
|
|
||||||
rc.Stop()
|
|
||||||
}
|
|
||||||
|
|
||||||
// set registry
|
|
||||||
h.r = cache.New(reg)
|
|
||||||
|
|
||||||
// reconfigure tls config
|
|
||||||
if c := h.opts.TLSConfig; c != nil {
|
|
||||||
h.c = &http.Client{
|
|
||||||
Transport: newTransport(c),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Options() Options {
|
|
||||||
return h.opts
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Publish(topic string, msg *Message, opts ...PublishOption) error {
|
|
||||||
// create the message first
|
|
||||||
m := &Message{
|
|
||||||
Header: make(map[string]string),
|
|
||||||
Body: msg.Body,
|
|
||||||
}
|
|
||||||
|
|
||||||
for k, v := range msg.Header {
|
|
||||||
m.Header[k] = v
|
|
||||||
}
|
|
||||||
|
|
||||||
m.Header[":topic"] = topic
|
|
||||||
|
|
||||||
// encode the message
|
|
||||||
b, err := h.opts.Codec.Marshal(m)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// save the message
|
|
||||||
h.saveMessage(topic, b)
|
|
||||||
|
|
||||||
// now attempt to get the service
|
|
||||||
h.RLock()
|
|
||||||
s, err := h.r.GetService(serviceName)
|
|
||||||
if err != nil {
|
|
||||||
h.RUnlock()
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
h.RUnlock()
|
|
||||||
|
|
||||||
pub := func(node *registry.Node, t string, b []byte) error {
|
|
||||||
scheme := "http"
|
|
||||||
|
|
||||||
// check if secure is added in metadata
|
|
||||||
if node.Metadata["secure"] == "true" {
|
|
||||||
scheme = "https"
|
|
||||||
}
|
|
||||||
|
|
||||||
vals := url.Values{}
|
|
||||||
vals.Add("id", node.Id)
|
|
||||||
|
|
||||||
uri := fmt.Sprintf("%s://%s%s?%s", scheme, node.Address, DefaultSubPath, vals.Encode())
|
|
||||||
r, err := h.c.Post(uri, "application/json", bytes.NewReader(b))
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// discard response body
|
|
||||||
io.Copy(ioutil.Discard, r.Body)
|
|
||||||
r.Body.Close()
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
srv := func(s []*registry.Service, b []byte) {
|
|
||||||
for _, service := range s {
|
|
||||||
var nodes []*registry.Node
|
|
||||||
|
|
||||||
for _, node := range service.Nodes {
|
|
||||||
// only use nodes tagged with broker http
|
|
||||||
if node.Metadata["broker"] != "http" {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
// look for nodes for the topic
|
|
||||||
if node.Metadata["topic"] != topic {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
nodes = append(nodes, node)
|
|
||||||
}
|
|
||||||
|
|
||||||
// only process if we have nodes
|
|
||||||
if len(nodes) == 0 {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
switch service.Version {
|
|
||||||
// broadcast version means broadcast to all nodes
|
|
||||||
case broadcastVersion:
|
|
||||||
var success bool
|
|
||||||
|
|
||||||
// publish to all nodes
|
|
||||||
for _, node := range nodes {
|
|
||||||
// publish async
|
|
||||||
if err := pub(node, topic, b); err == nil {
|
|
||||||
success = true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// save if it failed to publish at least once
|
|
||||||
if !success {
|
|
||||||
h.saveMessage(topic, b)
|
|
||||||
}
|
|
||||||
default:
|
|
||||||
// select node to publish to
|
|
||||||
node := nodes[rand.Int()%len(nodes)]
|
|
||||||
|
|
||||||
// publish async to one node
|
|
||||||
if err := pub(node, topic, b); err != nil {
|
|
||||||
// if failed save it
|
|
||||||
h.saveMessage(topic, b)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// do the rest async
|
|
||||||
go func() {
|
|
||||||
// get a third of the backlog
|
|
||||||
messages := h.getMessage(topic, 8)
|
|
||||||
delay := (len(messages) > 1)
|
|
||||||
|
|
||||||
// publish all the messages
|
|
||||||
for _, msg := range messages {
|
|
||||||
// serialize here
|
|
||||||
srv(s, msg)
|
|
||||||
|
|
||||||
// sending a backlog of messages
|
|
||||||
if delay {
|
|
||||||
time.Sleep(time.Millisecond * 100)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}()
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) Subscribe(topic string, handler Handler, opts ...SubscribeOption) (Subscriber, error) {
|
|
||||||
var err error
|
|
||||||
var host, port string
|
|
||||||
options := NewSubscribeOptions(opts...)
|
|
||||||
|
|
||||||
// parse address for host, port
|
|
||||||
host, port, err = net.SplitHostPort(h.Address())
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
addr, err := maddr.Extract(host)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
var secure bool
|
|
||||||
|
|
||||||
if h.opts.Secure || h.opts.TLSConfig != nil {
|
|
||||||
secure = true
|
|
||||||
}
|
|
||||||
|
|
||||||
// register service
|
|
||||||
node := ®istry.Node{
|
|
||||||
Id: topic + "-" + h.id,
|
|
||||||
Address: mnet.HostPort(addr, port),
|
|
||||||
Metadata: map[string]string{
|
|
||||||
"secure": fmt.Sprintf("%t", secure),
|
|
||||||
"broker": "http",
|
|
||||||
"topic": topic,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
|
|
||||||
// check for queue group or broadcast queue
|
|
||||||
version := options.Queue
|
|
||||||
if len(version) == 0 {
|
|
||||||
version = broadcastVersion
|
|
||||||
}
|
|
||||||
|
|
||||||
service := ®istry.Service{
|
|
||||||
Name: serviceName,
|
|
||||||
Version: version,
|
|
||||||
Nodes: []*registry.Node{node},
|
|
||||||
}
|
|
||||||
|
|
||||||
// generate subscriber
|
|
||||||
subscriber := &httpSubscriber{
|
|
||||||
opts: options,
|
|
||||||
hb: h,
|
|
||||||
id: node.Id,
|
|
||||||
topic: topic,
|
|
||||||
fn: handler,
|
|
||||||
svc: service,
|
|
||||||
}
|
|
||||||
|
|
||||||
// subscribe now
|
|
||||||
if err := h.subscribe(subscriber); err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
// return the subscriber
|
|
||||||
return subscriber, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *httpBroker) String() string {
|
|
||||||
return "http"
|
|
||||||
}
|
|
@ -1,349 +0,0 @@
|
|||||||
package broker
|
|
||||||
|
|
||||||
import (
|
|
||||||
"sync"
|
|
||||||
"testing"
|
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/google/uuid"
|
|
||||||
"github.com/micro/go-micro/debug/log/noop"
|
|
||||||
"github.com/micro/go-micro/registry"
|
|
||||||
"github.com/micro/go-micro/registry/memory"
|
|
||||||
"github.com/micro/go-micro/util/log"
|
|
||||||
)
|
|
||||||
|
|
||||||
func newTestRegistry() registry.Registry {
|
|
||||||
return memory.NewRegistry(memory.Services(testData))
|
|
||||||
}
|
|
||||||
|
|
||||||
func sub(be *testing.B, c int) {
|
|
||||||
// set no op logger
|
|
||||||
log.SetLogger(noop.NewLog())
|
|
||||||
|
|
||||||
be.StopTimer()
|
|
||||||
m := newTestRegistry()
|
|
||||||
|
|
||||||
b := NewBroker(Registry(m))
|
|
||||||
topic := uuid.New().String()
|
|
||||||
|
|
||||||
if err := b.Init(); err != nil {
|
|
||||||
be.Fatalf("Unexpected init error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Connect(); err != nil {
|
|
||||||
be.Fatalf("Unexpected connect error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
msg := &Message{
|
|
||||||
Header: map[string]string{
|
|
||||||
"Content-Type": "application/json",
|
|
||||||
},
|
|
||||||
Body: []byte(`{"message": "Hello World"}`),
|
|
||||||
}
|
|
||||||
|
|
||||||
var subs []Subscriber
|
|
||||||
done := make(chan bool, c)
|
|
||||||
|
|
||||||
for i := 0; i < c; i++ {
|
|
||||||
sub, err := b.Subscribe(topic, func(p Event) error {
|
|
||||||
done <- true
|
|
||||||
m := p.Message()
|
|
||||||
|
|
||||||
if string(m.Body) != string(msg.Body) {
|
|
||||||
be.Fatalf("Unexpected msg %s, expected %s", string(m.Body), string(msg.Body))
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}, Queue("shared"))
|
|
||||||
if err != nil {
|
|
||||||
be.Fatalf("Unexpected subscribe error: %v", err)
|
|
||||||
}
|
|
||||||
subs = append(subs, sub)
|
|
||||||
}
|
|
||||||
|
|
||||||
for i := 0; i < be.N; i++ {
|
|
||||||
be.StartTimer()
|
|
||||||
if err := b.Publish(topic, msg); err != nil {
|
|
||||||
be.Fatalf("Unexpected publish error: %v", err)
|
|
||||||
}
|
|
||||||
<-done
|
|
||||||
be.StopTimer()
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, sub := range subs {
|
|
||||||
sub.Unsubscribe()
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Disconnect(); err != nil {
|
|
||||||
be.Fatalf("Unexpected disconnect error: %v", err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func pub(be *testing.B, c int) {
|
|
||||||
// set no op logger
|
|
||||||
log.SetLogger(noop.NewLog())
|
|
||||||
|
|
||||||
be.StopTimer()
|
|
||||||
m := newTestRegistry()
|
|
||||||
b := NewBroker(Registry(m))
|
|
||||||
topic := uuid.New().String()
|
|
||||||
|
|
||||||
if err := b.Init(); err != nil {
|
|
||||||
be.Fatalf("Unexpected init error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Connect(); err != nil {
|
|
||||||
be.Fatalf("Unexpected connect error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
msg := &Message{
|
|
||||||
Header: map[string]string{
|
|
||||||
"Content-Type": "application/json",
|
|
||||||
},
|
|
||||||
Body: []byte(`{"message": "Hello World"}`),
|
|
||||||
}
|
|
||||||
|
|
||||||
done := make(chan bool, c*4)
|
|
||||||
|
|
||||||
sub, err := b.Subscribe(topic, func(p Event) error {
|
|
||||||
done <- true
|
|
||||||
m := p.Message()
|
|
||||||
if string(m.Body) != string(msg.Body) {
|
|
||||||
be.Fatalf("Unexpected msg %s, expected %s", string(m.Body), string(msg.Body))
|
|
||||||
}
|
|
||||||
return nil
|
|
||||||
}, Queue("shared"))
|
|
||||||
if err != nil {
|
|
||||||
be.Fatalf("Unexpected subscribe error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
var wg sync.WaitGroup
|
|
||||||
ch := make(chan int, c*4)
|
|
||||||
be.StartTimer()
|
|
||||||
|
|
||||||
for i := 0; i < c; i++ {
|
|
||||||
go func() {
|
|
||||||
for range ch {
|
|
||||||
if err := b.Publish(topic, msg); err != nil {
|
|
||||||
be.Fatalf("Unexpected publish error: %v", err)
|
|
||||||
}
|
|
||||||
select {
|
|
||||||
case <-done:
|
|
||||||
case <-time.After(time.Second):
|
|
||||||
}
|
|
||||||
wg.Done()
|
|
||||||
}
|
|
||||||
}()
|
|
||||||
}
|
|
||||||
|
|
||||||
for i := 0; i < be.N; i++ {
|
|
||||||
wg.Add(1)
|
|
||||||
ch <- i
|
|
||||||
}
|
|
||||||
|
|
||||||
wg.Wait()
|
|
||||||
be.StopTimer()
|
|
||||||
sub.Unsubscribe()
|
|
||||||
close(ch)
|
|
||||||
close(done)
|
|
||||||
|
|
||||||
if err := b.Disconnect(); err != nil {
|
|
||||||
be.Fatalf("Unexpected disconnect error: %v", err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestBroker(t *testing.T) {
|
|
||||||
m := newTestRegistry()
|
|
||||||
b := NewBroker(Registry(m))
|
|
||||||
|
|
||||||
if err := b.Init(); err != nil {
|
|
||||||
t.Fatalf("Unexpected init error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Connect(); err != nil {
|
|
||||||
t.Fatalf("Unexpected connect error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
msg := &Message{
|
|
||||||
Header: map[string]string{
|
|
||||||
"Content-Type": "application/json",
|
|
||||||
},
|
|
||||||
Body: []byte(`{"message": "Hello World"}`),
|
|
||||||
}
|
|
||||||
|
|
||||||
done := make(chan bool)
|
|
||||||
|
|
||||||
sub, err := b.Subscribe("test", func(p Event) error {
|
|
||||||
m := p.Message()
|
|
||||||
|
|
||||||
if string(m.Body) != string(msg.Body) {
|
|
||||||
t.Fatalf("Unexpected msg %s, expected %s", string(m.Body), string(msg.Body))
|
|
||||||
}
|
|
||||||
|
|
||||||
close(done)
|
|
||||||
return nil
|
|
||||||
})
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("Unexpected subscribe error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Publish("test", msg); err != nil {
|
|
||||||
t.Fatalf("Unexpected publish error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
<-done
|
|
||||||
sub.Unsubscribe()
|
|
||||||
|
|
||||||
if err := b.Disconnect(); err != nil {
|
|
||||||
t.Fatalf("Unexpected disconnect error: %v", err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestConcurrentSubBroker(t *testing.T) {
|
|
||||||
m := newTestRegistry()
|
|
||||||
b := NewBroker(Registry(m))
|
|
||||||
|
|
||||||
if err := b.Init(); err != nil {
|
|
||||||
t.Fatalf("Unexpected init error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Connect(); err != nil {
|
|
||||||
t.Fatalf("Unexpected connect error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
msg := &Message{
|
|
||||||
Header: map[string]string{
|
|
||||||
"Content-Type": "application/json",
|
|
||||||
},
|
|
||||||
Body: []byte(`{"message": "Hello World"}`),
|
|
||||||
}
|
|
||||||
|
|
||||||
var subs []Subscriber
|
|
||||||
var wg sync.WaitGroup
|
|
||||||
|
|
||||||
for i := 0; i < 10; i++ {
|
|
||||||
sub, err := b.Subscribe("test", func(p Event) error {
|
|
||||||
defer wg.Done()
|
|
||||||
|
|
||||||
m := p.Message()
|
|
||||||
|
|
||||||
if string(m.Body) != string(msg.Body) {
|
|
||||||
t.Fatalf("Unexpected msg %s, expected %s", string(m.Body), string(msg.Body))
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
})
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("Unexpected subscribe error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
wg.Add(1)
|
|
||||||
subs = append(subs, sub)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Publish("test", msg); err != nil {
|
|
||||||
t.Fatalf("Unexpected publish error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
wg.Wait()
|
|
||||||
|
|
||||||
for _, sub := range subs {
|
|
||||||
sub.Unsubscribe()
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Disconnect(); err != nil {
|
|
||||||
t.Fatalf("Unexpected disconnect error: %v", err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestConcurrentPubBroker(t *testing.T) {
|
|
||||||
m := newTestRegistry()
|
|
||||||
b := NewBroker(Registry(m))
|
|
||||||
|
|
||||||
if err := b.Init(); err != nil {
|
|
||||||
t.Fatalf("Unexpected init error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := b.Connect(); err != nil {
|
|
||||||
t.Fatalf("Unexpected connect error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
msg := &Message{
|
|
||||||
Header: map[string]string{
|
|
||||||
"Content-Type": "application/json",
|
|
||||||
},
|
|
||||||
Body: []byte(`{"message": "Hello World"}`),
|
|
||||||
}
|
|
||||||
|
|
||||||
var wg sync.WaitGroup
|
|
||||||
|
|
||||||
sub, err := b.Subscribe("test", func(p Event) error {
|
|
||||||
defer wg.Done()
|
|
||||||
|
|
||||||
m := p.Message()
|
|
||||||
|
|
||||||
if string(m.Body) != string(msg.Body) {
|
|
||||||
t.Fatalf("Unexpected msg %s, expected %s", string(m.Body), string(msg.Body))
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
})
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("Unexpected subscribe error: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
for i := 0; i < 10; i++ {
|
|
||||||
wg.Add(1)
|
|
||||||
|
|
||||||
if err := b.Publish("test", msg); err != nil {
|
|
||||||
t.Fatalf("Unexpected publish error: %v", err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
wg.Wait()
|
|
||||||
|
|
||||||
sub.Unsubscribe()
|
|
||||||
|
|
||||||
if err := b.Disconnect(); err != nil {
|
|
||||||
t.Fatalf("Unexpected disconnect error: %v", err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkSub1(b *testing.B) {
|
|
||||||
sub(b, 1)
|
|
||||||
}
|
|
||||||
func BenchmarkSub8(b *testing.B) {
|
|
||||||
sub(b, 8)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkSub32(b *testing.B) {
|
|
||||||
sub(b, 32)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkSub64(b *testing.B) {
|
|
||||||
sub(b, 64)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkSub128(b *testing.B) {
|
|
||||||
sub(b, 128)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkPub1(b *testing.B) {
|
|
||||||
pub(b, 1)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkPub8(b *testing.B) {
|
|
||||||
pub(b, 8)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkPub32(b *testing.B) {
|
|
||||||
pub(b, 32)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkPub64(b *testing.B) {
|
|
||||||
pub(b, 64)
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkPub128(b *testing.B) {
|
|
||||||
pub(b, 128)
|
|
||||||
}
|
|
@ -119,17 +119,23 @@ func (n *natsBroker) serve(exit chan bool) error {
|
|||||||
|
|
||||||
// with no address we just default it
|
// with no address we just default it
|
||||||
// this is a local client address
|
// this is a local client address
|
||||||
if len(n.addrs) == 0 || n.local {
|
if len(n.addrs) == 0 {
|
||||||
|
// find an advertiseable ip
|
||||||
|
if h, err := addr.Extract(""); err != nil {
|
||||||
host = "127.0.0.1"
|
host = "127.0.0.1"
|
||||||
|
} else {
|
||||||
|
host = h
|
||||||
|
}
|
||||||
|
|
||||||
port = -1
|
port = -1
|
||||||
local = true
|
local = true
|
||||||
// with a local address we parse it
|
|
||||||
} else {
|
} else {
|
||||||
address := n.addrs[0]
|
address := n.addrs[0]
|
||||||
if strings.HasPrefix(address, "nats://") {
|
if strings.HasPrefix(address, "nats://") {
|
||||||
address = strings.TrimPrefix(address, "nats://")
|
address = strings.TrimPrefix(address, "nats://")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// check if its a local address and only then embed
|
||||||
if addr.IsLocal(address) {
|
if addr.IsLocal(address) {
|
||||||
h, p, err := net.SplitHostPort(address)
|
h, p, err := net.SplitHostPort(address)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
@ -266,10 +272,12 @@ func (n *natsBroker) Connect() error {
|
|||||||
// create exit chan
|
// create exit chan
|
||||||
n.exit = make(chan bool)
|
n.exit = make(chan bool)
|
||||||
|
|
||||||
// start the server if needed
|
// start embedded server if asked to
|
||||||
|
if n.local {
|
||||||
if err := n.serve(n.exit); err != nil {
|
if err := n.serve(n.exit); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// set to connected
|
// set to connected
|
||||||
n.connected = true
|
n.connected = true
|
||||||
|
@ -124,9 +124,11 @@ func NewBroker(opts ...broker.Option) broker.Broker {
|
|||||||
addrs = []string{"127.0.0.1:8001"}
|
addrs = []string{"127.0.0.1:8001"}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
cli := client.DefaultClient
|
||||||
|
|
||||||
return &serviceBroker{
|
return &serviceBroker{
|
||||||
Addrs: addrs,
|
Addrs: addrs,
|
||||||
Client: pb.NewBrokerService(DefaultName, client.DefaultClient),
|
Client: pb.NewBrokerService(DefaultName, cli),
|
||||||
options: options,
|
options: options,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -110,6 +110,9 @@ var (
|
|||||||
DefaultPoolSize = 100
|
DefaultPoolSize = 100
|
||||||
// DefaultPoolTTL sets the connection pool ttl
|
// DefaultPoolTTL sets the connection pool ttl
|
||||||
DefaultPoolTTL = time.Minute
|
DefaultPoolTTL = time.Minute
|
||||||
|
|
||||||
|
// NewClient returns a new client
|
||||||
|
NewClient func(...Option) Client = newRpcClient
|
||||||
)
|
)
|
||||||
|
|
||||||
// Makes a synchronous call to a service using the default client
|
// Makes a synchronous call to a service using the default client
|
||||||
@ -128,11 +131,6 @@ func NewMessage(topic string, payload interface{}, opts ...MessageOption) Messag
|
|||||||
return DefaultClient.NewMessage(topic, payload, opts...)
|
return DefaultClient.NewMessage(topic, payload, opts...)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Creates a new client with the options passed in
|
|
||||||
func NewClient(opt ...Option) Client {
|
|
||||||
return newRpcClient(opt...)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Creates a new request using the default client. Content Type will
|
// Creates a new request using the default client. Content Type will
|
||||||
// be set to the default within options and use the appropriate codec
|
// be set to the default within options and use the appropriate codec
|
||||||
func NewRequest(service, endpoint string, request interface{}, reqOpts ...RequestOption) Request {
|
func NewRequest(service, endpoint string, request interface{}, reqOpts ...RequestOption) Request {
|
||||||
|
@ -212,7 +212,10 @@ func (g *grpcClient) stream(ctx context.Context, node *registry.Node, req client
|
|||||||
grpcCallOptions = append(grpcCallOptions, opts...)
|
grpcCallOptions = append(grpcCallOptions, opts...)
|
||||||
}
|
}
|
||||||
|
|
||||||
st, err := cc.NewStream(ctx, desc, methodToGRPC(req.Service(), req.Endpoint()), grpcCallOptions...)
|
// create a new cancelling context
|
||||||
|
newCtx, cancel := context.WithCancel(ctx)
|
||||||
|
|
||||||
|
st, err := cc.NewStream(newCtx, desc, methodToGRPC(req.Service(), req.Endpoint()), grpcCallOptions...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.InternalServerError("go.micro.client", fmt.Sprintf("Error creating stream: %v", err))
|
return nil, errors.InternalServerError("go.micro.client", fmt.Sprintf("Error creating stream: %v", err))
|
||||||
}
|
}
|
||||||
@ -240,6 +243,7 @@ func (g *grpcClient) stream(ctx context.Context, node *registry.Node, req client
|
|||||||
response: rsp,
|
response: rsp,
|
||||||
stream: st,
|
stream: st,
|
||||||
conn: cc,
|
conn: cc,
|
||||||
|
cancel: cancel,
|
||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -19,6 +19,7 @@ type grpcStream struct {
|
|||||||
request client.Request
|
request client.Request
|
||||||
response client.Response
|
response client.Response
|
||||||
context context.Context
|
context context.Context
|
||||||
|
cancel func()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (g *grpcStream) Context() context.Context {
|
func (g *grpcStream) Context() context.Context {
|
||||||
@ -79,7 +80,8 @@ func (g *grpcStream) Close() error {
|
|||||||
if g.closed {
|
if g.closed {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
// cancel the context
|
||||||
|
g.cancel()
|
||||||
g.closed = true
|
g.closed = true
|
||||||
g.stream.CloseSend()
|
g.stream.CloseSend()
|
||||||
return g.conn.Close()
|
return g.conn.Close()
|
||||||
|
@ -1,16 +0,0 @@
|
|||||||
package mock
|
|
||||||
|
|
||||||
import (
|
|
||||||
"context"
|
|
||||||
)
|
|
||||||
|
|
||||||
type responseKey struct{}
|
|
||||||
|
|
||||||
func fromContext(ctx context.Context) (map[string][]MockResponse, bool) {
|
|
||||||
r, ok := ctx.Value(responseKey{}).(map[string][]MockResponse)
|
|
||||||
return r, ok
|
|
||||||
}
|
|
||||||
|
|
||||||
func newContext(ctx context.Context, r map[string][]MockResponse) context.Context {
|
|
||||||
return context.WithValue(ctx, responseKey{}, r)
|
|
||||||
}
|
|
@ -1,150 +0,0 @@
|
|||||||
// Package mock provides a mock client for testing
|
|
||||||
package mock
|
|
||||||
|
|
||||||
import (
|
|
||||||
"context"
|
|
||||||
"fmt"
|
|
||||||
"reflect"
|
|
||||||
"sync"
|
|
||||||
|
|
||||||
"github.com/micro/go-micro/client"
|
|
||||||
"github.com/micro/go-micro/errors"
|
|
||||||
)
|
|
||||||
|
|
||||||
var (
|
|
||||||
_ client.Client = NewClient()
|
|
||||||
)
|
|
||||||
|
|
||||||
type MockResponse struct {
|
|
||||||
Endpoint string
|
|
||||||
Response interface{}
|
|
||||||
Error error
|
|
||||||
}
|
|
||||||
|
|
||||||
type MockClient struct {
|
|
||||||
Client client.Client
|
|
||||||
Opts client.Options
|
|
||||||
|
|
||||||
sync.Mutex
|
|
||||||
Response map[string][]MockResponse
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) Init(opts ...client.Option) error {
|
|
||||||
m.Lock()
|
|
||||||
defer m.Unlock()
|
|
||||||
|
|
||||||
for _, opt := range opts {
|
|
||||||
opt(&m.Opts)
|
|
||||||
}
|
|
||||||
|
|
||||||
r, ok := fromContext(m.Opts.Context)
|
|
||||||
if !ok {
|
|
||||||
r = make(map[string][]MockResponse)
|
|
||||||
}
|
|
||||||
m.Response = r
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) Options() client.Options {
|
|
||||||
return m.Opts
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) NewMessage(topic string, msg interface{}, opts ...client.MessageOption) client.Message {
|
|
||||||
return m.Client.NewMessage(topic, msg, opts...)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) NewRequest(service, endpoint string, req interface{}, reqOpts ...client.RequestOption) client.Request {
|
|
||||||
return m.Client.NewRequest(service, endpoint, req, reqOpts...)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) Call(ctx context.Context, req client.Request, rsp interface{}, opts ...client.CallOption) error {
|
|
||||||
m.Lock()
|
|
||||||
defer m.Unlock()
|
|
||||||
|
|
||||||
response, ok := m.Response[req.Service()]
|
|
||||||
if !ok {
|
|
||||||
return errors.NotFound("go.micro.client.mock", "service not found")
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, r := range response {
|
|
||||||
if r.Endpoint != req.Endpoint() {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
if r.Error != nil {
|
|
||||||
return r.Error
|
|
||||||
}
|
|
||||||
|
|
||||||
v := reflect.ValueOf(rsp)
|
|
||||||
|
|
||||||
if t := reflect.TypeOf(rsp); t.Kind() == reflect.Ptr {
|
|
||||||
v = reflect.Indirect(v)
|
|
||||||
}
|
|
||||||
response := r.Response
|
|
||||||
if t := reflect.TypeOf(r.Response); t.Kind() == reflect.Func {
|
|
||||||
var request []reflect.Value
|
|
||||||
switch t.NumIn() {
|
|
||||||
case 1:
|
|
||||||
// one input params: (req)
|
|
||||||
request = append(request, reflect.ValueOf(req.Body()))
|
|
||||||
case 2:
|
|
||||||
// two input params: (ctx, req)
|
|
||||||
request = append(request, reflect.ValueOf(ctx), reflect.ValueOf(req.Body()))
|
|
||||||
}
|
|
||||||
|
|
||||||
responseValue := reflect.ValueOf(r.Response).Call(request)
|
|
||||||
response = responseValue[0].Interface()
|
|
||||||
if len(responseValue) == 2 {
|
|
||||||
// make it possible to return error in response function
|
|
||||||
respErr, ok := responseValue[1].Interface().(error)
|
|
||||||
if ok && respErr != nil {
|
|
||||||
return respErr
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
v.Set(reflect.ValueOf(response))
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
return fmt.Errorf("rpc: can't find service %s", req.Endpoint())
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) Stream(ctx context.Context, req client.Request, opts ...client.CallOption) (client.Stream, error) {
|
|
||||||
m.Lock()
|
|
||||||
defer m.Unlock()
|
|
||||||
|
|
||||||
// TODO: mock stream
|
|
||||||
return nil, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) Publish(ctx context.Context, p client.Message, opts ...client.PublishOption) error {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *MockClient) String() string {
|
|
||||||
return "mock"
|
|
||||||
}
|
|
||||||
|
|
||||||
func NewClient(opts ...client.Option) *MockClient {
|
|
||||||
options := client.Options{
|
|
||||||
Context: context.TODO(),
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, opt := range opts {
|
|
||||||
opt(&options)
|
|
||||||
}
|
|
||||||
|
|
||||||
r, ok := fromContext(options.Context)
|
|
||||||
if !ok {
|
|
||||||
r = make(map[string][]MockResponse)
|
|
||||||
}
|
|
||||||
|
|
||||||
return &MockClient{
|
|
||||||
Client: client.DefaultClient,
|
|
||||||
Opts: options,
|
|
||||||
Response: r,
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,67 +0,0 @@
|
|||||||
package mock
|
|
||||||
|
|
||||||
import (
|
|
||||||
"context"
|
|
||||||
"fmt"
|
|
||||||
"testing"
|
|
||||||
|
|
||||||
"github.com/micro/go-micro/errors"
|
|
||||||
)
|
|
||||||
|
|
||||||
func TestClient(t *testing.T) {
|
|
||||||
type TestResponse struct {
|
|
||||||
Param string
|
|
||||||
}
|
|
||||||
|
|
||||||
response := []MockResponse{
|
|
||||||
{Endpoint: "Foo.Bar", Response: map[string]interface{}{"foo": "bar"}},
|
|
||||||
{Endpoint: "Foo.Struct", Response: &TestResponse{Param: "aparam"}},
|
|
||||||
{Endpoint: "Foo.Fail", Error: errors.InternalServerError("go.mock", "failed")},
|
|
||||||
{Endpoint: "Foo.Func", Response: func() string { return "string" }},
|
|
||||||
{Endpoint: "Foo.FuncStruct", Response: func() *TestResponse { return &TestResponse{Param: "aparam"} }},
|
|
||||||
{Endpoint: "Foo.FuncWithReqBody", Response: func(req interface{}) string {
|
|
||||||
if req.(map[string]string)["foo"] == "bar" {
|
|
||||||
return "string"
|
|
||||||
}
|
|
||||||
return "wrong"
|
|
||||||
}},
|
|
||||||
{Endpoint: "Foo.FuncWithRequestContextAndResponse", Response: func(ctx context.Context, req interface{}) string {
|
|
||||||
return "something"
|
|
||||||
}},
|
|
||||||
{Endpoint: "Foo.FuncWithRequestContextAndResponseError", Response: func(ctx context.Context, req interface{}) (string, error) {
|
|
||||||
return "something", fmt.Errorf("mock error")
|
|
||||||
}},
|
|
||||||
}
|
|
||||||
|
|
||||||
c := NewClient(Response("go.mock", response))
|
|
||||||
|
|
||||||
for _, r := range response {
|
|
||||||
req := c.NewRequest("go.mock", r.Endpoint, map[string]string{"foo": "bar"})
|
|
||||||
var rsp interface{}
|
|
||||||
|
|
||||||
err := c.Call(context.TODO(), req, &rsp)
|
|
||||||
|
|
||||||
if err != r.Error {
|
|
||||||
if r.Endpoint != "Foo.FuncWithRequestContextAndResponseError" {
|
|
||||||
t.Fatalf("Expecter error %v got %v", r.Error, err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
t.Log(rsp)
|
|
||||||
if r.Endpoint == "Foo.FuncWithReqBody" {
|
|
||||||
req := c.NewRequest("go.mock", r.Endpoint, map[string]string{"foo": "wrong"})
|
|
||||||
var rsp interface{}
|
|
||||||
|
|
||||||
err := c.Call(context.TODO(), req, &rsp)
|
|
||||||
|
|
||||||
if err != r.Error {
|
|
||||||
t.Fatalf("Expecter error %v got %v", r.Error, err)
|
|
||||||
}
|
|
||||||
if rsp.(string) != "wrong" {
|
|
||||||
t.Fatalf("Expecter response 'wrong' got %v", rsp)
|
|
||||||
}
|
|
||||||
t.Log(rsp)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,17 +0,0 @@
|
|||||||
package mock
|
|
||||||
|
|
||||||
import (
|
|
||||||
"github.com/micro/go-micro/client"
|
|
||||||
)
|
|
||||||
|
|
||||||
// Response sets the response methods for a service
|
|
||||||
func Response(service string, response []MockResponse) client.Option {
|
|
||||||
return func(o *client.Options) {
|
|
||||||
r, ok := fromContext(o.Context)
|
|
||||||
if !ok {
|
|
||||||
r = make(map[string][]MockResponse)
|
|
||||||
}
|
|
||||||
r[service] = response
|
|
||||||
o.Context = newContext(o.Context, r)
|
|
||||||
}
|
|
||||||
}
|
|
@ -10,7 +10,7 @@ import (
|
|||||||
|
|
||||||
"github.com/google/uuid"
|
"github.com/google/uuid"
|
||||||
"github.com/micro/go-micro/broker"
|
"github.com/micro/go-micro/broker"
|
||||||
"github.com/micro/go-micro/client/pool"
|
"github.com/micro/go-micro/util/pool"
|
||||||
"github.com/micro/go-micro/client/selector"
|
"github.com/micro/go-micro/client/selector"
|
||||||
"github.com/micro/go-micro/codec"
|
"github.com/micro/go-micro/codec"
|
||||||
raw "github.com/micro/go-micro/codec/bytes"
|
raw "github.com/micro/go-micro/codec/bytes"
|
||||||
|
@ -8,23 +8,30 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/micro/cli"
|
"github.com/micro/cli"
|
||||||
|
"github.com/micro/go-micro/broker"
|
||||||
"github.com/micro/go-micro/client"
|
"github.com/micro/go-micro/client"
|
||||||
cgrpc "github.com/micro/go-micro/client/grpc"
|
"github.com/micro/go-micro/client/selector"
|
||||||
cmucp "github.com/micro/go-micro/client/mucp"
|
"github.com/micro/go-micro/registry"
|
||||||
|
"github.com/micro/go-micro/runtime"
|
||||||
"github.com/micro/go-micro/server"
|
"github.com/micro/go-micro/server"
|
||||||
sgrpc "github.com/micro/go-micro/server/grpc"
|
"github.com/micro/go-micro/store"
|
||||||
smucp "github.com/micro/go-micro/server/mucp"
|
"github.com/micro/go-micro/transport"
|
||||||
"github.com/micro/go-micro/util/log"
|
"github.com/micro/go-micro/util/log"
|
||||||
|
|
||||||
|
// clients
|
||||||
|
cgrpc "github.com/micro/go-micro/client/grpc"
|
||||||
|
cmucp "github.com/micro/go-micro/client/mucp"
|
||||||
|
|
||||||
|
// servers
|
||||||
|
sgrpc "github.com/micro/go-micro/server/grpc"
|
||||||
|
smucp "github.com/micro/go-micro/server/mucp"
|
||||||
|
|
||||||
// brokers
|
// brokers
|
||||||
"github.com/micro/go-micro/broker"
|
|
||||||
"github.com/micro/go-micro/broker/http"
|
|
||||||
"github.com/micro/go-micro/broker/memory"
|
"github.com/micro/go-micro/broker/memory"
|
||||||
"github.com/micro/go-micro/broker/nats"
|
"github.com/micro/go-micro/broker/nats"
|
||||||
brokerSrv "github.com/micro/go-micro/broker/service"
|
brokerSrv "github.com/micro/go-micro/broker/service"
|
||||||
|
|
||||||
// registries
|
// registries
|
||||||
"github.com/micro/go-micro/registry"
|
|
||||||
"github.com/micro/go-micro/registry/etcd"
|
"github.com/micro/go-micro/registry/etcd"
|
||||||
kreg "github.com/micro/go-micro/registry/kubernetes"
|
kreg "github.com/micro/go-micro/registry/kubernetes"
|
||||||
"github.com/micro/go-micro/registry/mdns"
|
"github.com/micro/go-micro/registry/mdns"
|
||||||
@ -32,27 +39,15 @@ import (
|
|||||||
regSrv "github.com/micro/go-micro/registry/service"
|
regSrv "github.com/micro/go-micro/registry/service"
|
||||||
|
|
||||||
// selectors
|
// selectors
|
||||||
"github.com/micro/go-micro/client/selector"
|
|
||||||
"github.com/micro/go-micro/client/selector/dns"
|
"github.com/micro/go-micro/client/selector/dns"
|
||||||
"github.com/micro/go-micro/client/selector/router"
|
"github.com/micro/go-micro/client/selector/router"
|
||||||
"github.com/micro/go-micro/client/selector/static"
|
"github.com/micro/go-micro/client/selector/static"
|
||||||
|
|
||||||
// transports
|
// transports
|
||||||
"github.com/micro/go-micro/transport"
|
|
||||||
tgrpc "github.com/micro/go-micro/transport/grpc"
|
|
||||||
thttp "github.com/micro/go-micro/transport/http"
|
thttp "github.com/micro/go-micro/transport/http"
|
||||||
tmem "github.com/micro/go-micro/transport/memory"
|
tmem "github.com/micro/go-micro/transport/memory"
|
||||||
"github.com/micro/go-micro/transport/quic"
|
|
||||||
|
|
||||||
// runtimes
|
|
||||||
"github.com/micro/go-micro/runtime"
|
|
||||||
"github.com/micro/go-micro/runtime/kubernetes"
|
|
||||||
|
|
||||||
// stores
|
// stores
|
||||||
"github.com/micro/go-micro/store"
|
|
||||||
cfStore "github.com/micro/go-micro/store/cloudflare"
|
|
||||||
ckStore "github.com/micro/go-micro/store/cockroach"
|
|
||||||
etcdStore "github.com/micro/go-micro/store/etcd"
|
|
||||||
memStore "github.com/micro/go-micro/store/memory"
|
memStore "github.com/micro/go-micro/store/memory"
|
||||||
svcStore "github.com/micro/go-micro/store/service"
|
svcStore "github.com/micro/go-micro/store/service"
|
||||||
)
|
)
|
||||||
@ -217,7 +212,6 @@ var (
|
|||||||
|
|
||||||
DefaultBrokers = map[string]func(...broker.Option) broker.Broker{
|
DefaultBrokers = map[string]func(...broker.Option) broker.Broker{
|
||||||
"service": brokerSrv.NewBroker,
|
"service": brokerSrv.NewBroker,
|
||||||
"http": http.NewBroker,
|
|
||||||
"memory": memory.NewBroker,
|
"memory": memory.NewBroker,
|
||||||
"nats": nats.NewBroker,
|
"nats": nats.NewBroker,
|
||||||
}
|
}
|
||||||
@ -236,9 +230,7 @@ var (
|
|||||||
}
|
}
|
||||||
|
|
||||||
DefaultSelectors = map[string]func(...selector.Option) selector.Selector{
|
DefaultSelectors = map[string]func(...selector.Option) selector.Selector{
|
||||||
"default": selector.NewSelector,
|
|
||||||
"dns": dns.NewSelector,
|
"dns": dns.NewSelector,
|
||||||
"cache": selector.NewSelector,
|
|
||||||
"router": router.NewSelector,
|
"router": router.NewSelector,
|
||||||
"static": static.NewSelector,
|
"static": static.NewSelector,
|
||||||
}
|
}
|
||||||
@ -251,27 +243,21 @@ var (
|
|||||||
DefaultTransports = map[string]func(...transport.Option) transport.Transport{
|
DefaultTransports = map[string]func(...transport.Option) transport.Transport{
|
||||||
"memory": tmem.NewTransport,
|
"memory": tmem.NewTransport,
|
||||||
"http": thttp.NewTransport,
|
"http": thttp.NewTransport,
|
||||||
"grpc": tgrpc.NewTransport,
|
|
||||||
"quic": quic.NewTransport,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
DefaultRuntimes = map[string]func(...runtime.Option) runtime.Runtime{
|
DefaultRuntimes = map[string]func(...runtime.Option) runtime.Runtime{
|
||||||
"local": runtime.NewRuntime,
|
"local": runtime.NewRuntime,
|
||||||
"kubernetes": kubernetes.NewRuntime,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
DefaultStores = map[string]func(...store.Option) store.Store{
|
DefaultStores = map[string]func(...store.Option) store.Store{
|
||||||
"memory": memStore.NewStore,
|
"memory": memStore.NewStore,
|
||||||
"cockroach": ckStore.NewStore,
|
|
||||||
"etcd": etcdStore.NewStore,
|
|
||||||
"cloudflare": cfStore.NewStore,
|
|
||||||
"service": svcStore.NewStore,
|
"service": svcStore.NewStore,
|
||||||
}
|
}
|
||||||
|
|
||||||
// used for default selection as the fall back
|
// used for default selection as the fall back
|
||||||
defaultClient = "grpc"
|
defaultClient = "grpc"
|
||||||
defaultServer = "grpc"
|
defaultServer = "grpc"
|
||||||
defaultBroker = "nats"
|
defaultBroker = "nats-e"
|
||||||
defaultRegistry = "mdns"
|
defaultRegistry = "mdns"
|
||||||
defaultSelector = "registry"
|
defaultSelector = "registry"
|
||||||
defaultTransport = "http"
|
defaultTransport = "http"
|
||||||
@ -558,8 +544,12 @@ func (c *cmd) Init(opts ...Option) error {
|
|||||||
for _, o := range opts {
|
for _, o := range opts {
|
||||||
o(&c.opts)
|
o(&c.opts)
|
||||||
}
|
}
|
||||||
|
if len(c.opts.Name) > 0 {
|
||||||
c.app.Name = c.opts.Name
|
c.app.Name = c.opts.Name
|
||||||
|
}
|
||||||
|
if len(c.opts.Version) > 0 {
|
||||||
c.app.Version = c.opts.Version
|
c.app.Version = c.opts.Version
|
||||||
|
}
|
||||||
c.app.HideVersion = len(c.opts.Version) == 0
|
c.app.HideVersion = len(c.opts.Version) == 0
|
||||||
c.app.Usage = c.opts.Description
|
c.app.Usage = c.opts.Description
|
||||||
c.app.RunAndExitOnError()
|
c.app.RunAndExitOnError()
|
||||||
|
71
debug/trace/default.go
Normal file
71
debug/trace/default.go
Normal file
@ -0,0 +1,71 @@
|
|||||||
|
package trace
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/google/uuid"
|
||||||
|
"github.com/micro/go-micro/util/ring"
|
||||||
|
)
|
||||||
|
|
||||||
|
type trace struct {
|
||||||
|
opts Options
|
||||||
|
|
||||||
|
// ring buffer of traces
|
||||||
|
buffer *ring.Buffer
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *trace) Read(opts ...ReadOption) ([]*Span, error) {
|
||||||
|
return []*Span{}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *trace) Start(ctx context.Context, name string) *Span {
|
||||||
|
span := &Span{
|
||||||
|
Name: name,
|
||||||
|
Trace: uuid.New().String(),
|
||||||
|
Id: uuid.New().String(),
|
||||||
|
Started: time.Now(),
|
||||||
|
Metadata: make(map[string]string),
|
||||||
|
}
|
||||||
|
|
||||||
|
// return span if no context
|
||||||
|
if ctx == nil {
|
||||||
|
return span
|
||||||
|
}
|
||||||
|
|
||||||
|
s, ok := FromContext(ctx)
|
||||||
|
if !ok {
|
||||||
|
return span
|
||||||
|
}
|
||||||
|
|
||||||
|
// set trace id
|
||||||
|
span.Trace = s.Trace
|
||||||
|
// set parent
|
||||||
|
span.Parent = s.Id
|
||||||
|
|
||||||
|
// return the sapn
|
||||||
|
return span
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *trace) Finish(s *Span) error {
|
||||||
|
// set finished time
|
||||||
|
s.Finished = time.Now()
|
||||||
|
|
||||||
|
// save the span
|
||||||
|
t.buffer.Put(s)
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewTrace(opts ...Option) Trace {
|
||||||
|
var options Options
|
||||||
|
for _, o := range opts {
|
||||||
|
o(&options)
|
||||||
|
}
|
||||||
|
|
||||||
|
return &trace{
|
||||||
|
opts: options,
|
||||||
|
// the last 64 requests
|
||||||
|
buffer: ring.New(64),
|
||||||
|
}
|
||||||
|
}
|
12
debug/trace/options.go
Normal file
12
debug/trace/options.go
Normal file
@ -0,0 +1,12 @@
|
|||||||
|
package trace
|
||||||
|
|
||||||
|
type Options struct{}
|
||||||
|
|
||||||
|
type Option func(o *Options)
|
||||||
|
|
||||||
|
type ReadOptions struct {
|
||||||
|
// Trace id
|
||||||
|
Trace string
|
||||||
|
}
|
||||||
|
|
||||||
|
type ReadOption func(o *ReadOptions)
|
48
debug/trace/trace.go
Normal file
48
debug/trace/trace.go
Normal file
@ -0,0 +1,48 @@
|
|||||||
|
// Package trace provides an interface for distributed tracing
|
||||||
|
package trace
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Trace is an interface for distributed tracing
|
||||||
|
type Trace interface {
|
||||||
|
// Start a trace
|
||||||
|
Start(ctx context.Context, name string) *Span
|
||||||
|
// Finish the trace
|
||||||
|
Finish(*Span) error
|
||||||
|
// Read the traces
|
||||||
|
Read(...ReadOption) ([]*Span, error)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Span is used to record an entry
|
||||||
|
type Span struct {
|
||||||
|
// Id of the trace
|
||||||
|
Trace string
|
||||||
|
// name of the span
|
||||||
|
Name string
|
||||||
|
// id of the span
|
||||||
|
Id string
|
||||||
|
// parent span id
|
||||||
|
Parent string
|
||||||
|
// Start time
|
||||||
|
Started time.Time
|
||||||
|
// Finish time
|
||||||
|
Finished time.Time
|
||||||
|
// associated data
|
||||||
|
Metadata map[string]string
|
||||||
|
}
|
||||||
|
|
||||||
|
type spanKey struct{}
|
||||||
|
|
||||||
|
// FromContext returns a span from context
|
||||||
|
func FromContext(ctx context.Context) (*Span, bool) {
|
||||||
|
s, ok := ctx.Value(spanKey{}).(*Span)
|
||||||
|
return s, ok
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewContext creates a new context with the span
|
||||||
|
func NewContext(ctx context.Context, s *Span) context.Context {
|
||||||
|
return context.WithValue(ctx, spanKey{}, s)
|
||||||
|
}
|
@ -1,24 +1,17 @@
|
|||||||
package micro
|
package micro
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/micro/go-micro/broker"
|
|
||||||
"github.com/micro/go-micro/client"
|
"github.com/micro/go-micro/client"
|
||||||
"github.com/micro/go-micro/server"
|
"github.com/micro/go-micro/server"
|
||||||
"github.com/micro/go-micro/store"
|
"github.com/micro/go-micro/store"
|
||||||
|
|
||||||
// set defaults
|
// set defaults
|
||||||
"github.com/micro/go-micro/broker/nats"
|
|
||||||
gcli "github.com/micro/go-micro/client/grpc"
|
gcli "github.com/micro/go-micro/client/grpc"
|
||||||
gsrv "github.com/micro/go-micro/server/grpc"
|
gsrv "github.com/micro/go-micro/server/grpc"
|
||||||
memStore "github.com/micro/go-micro/store/memory"
|
memStore "github.com/micro/go-micro/store/memory"
|
||||||
)
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
// default broker
|
|
||||||
broker.DefaultBroker = nats.NewBroker(
|
|
||||||
// embedded nats server
|
|
||||||
nats.LocalServer(),
|
|
||||||
)
|
|
||||||
// default client
|
// default client
|
||||||
client.DefaultClient = gcli.NewClient()
|
client.DefaultClient = gcli.NewClient()
|
||||||
// default server
|
// default server
|
||||||
|
28
go.mod
28
go.mod
@ -6,46 +6,58 @@ require (
|
|||||||
github.com/BurntSushi/toml v0.3.1
|
github.com/BurntSushi/toml v0.3.1
|
||||||
github.com/beevik/ntp v0.2.0
|
github.com/beevik/ntp v0.2.0
|
||||||
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/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/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect
|
github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect
|
||||||
github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c
|
github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c
|
||||||
github.com/fsnotify/fsnotify v1.4.7
|
github.com/fsnotify/fsnotify v1.4.7
|
||||||
github.com/fsouza/go-dockerclient v1.6.0
|
github.com/fsouza/go-dockerclient v1.6.0
|
||||||
github.com/ghodss/yaml v1.0.0
|
github.com/ghodss/yaml v1.0.0
|
||||||
github.com/go-acme/lego/v3 v3.2.0
|
github.com/go-acme/lego/v3 v3.3.0
|
||||||
github.com/go-playground/universal-translator v0.17.0 // indirect
|
github.com/go-playground/universal-translator v0.17.0 // indirect
|
||||||
|
github.com/go-telegram-bot-api/telegram-bot-api v4.6.4+incompatible // indirect
|
||||||
|
github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7 // indirect
|
||||||
github.com/golang/protobuf v1.3.2
|
github.com/golang/protobuf v1.3.2
|
||||||
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
|
github.com/gorilla/websocket v1.4.1
|
||||||
|
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect
|
||||||
|
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect
|
||||||
github.com/hashicorp/hcl v1.0.0
|
github.com/hashicorp/hcl v1.0.0
|
||||||
github.com/imdario/mergo v0.3.8
|
github.com/imdario/mergo v0.3.8
|
||||||
|
github.com/jonboulle/clockwork v0.1.0 // indirect
|
||||||
github.com/joncalhoun/qson v0.0.0-20170526102502-8a9cab3a62b1
|
github.com/joncalhoun/qson v0.0.0-20170526102502-8a9cab3a62b1
|
||||||
github.com/json-iterator/go v1.1.9
|
github.com/json-iterator/go v1.1.9
|
||||||
github.com/kr/pretty v0.1.0
|
github.com/kr/pretty v0.1.0
|
||||||
github.com/leodido/go-urn v1.2.0 // indirect
|
github.com/leodido/go-urn v1.2.0 // indirect
|
||||||
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/mholt/certmagic v0.9.0
|
github.com/mholt/certmagic v0.9.1
|
||||||
github.com/micro/cli v0.2.0
|
github.com/micro/cli v0.2.0
|
||||||
github.com/micro/mdns v0.3.0
|
github.com/micro/mdns v0.3.0
|
||||||
github.com/miekg/dns v1.1.26
|
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/gnatsd v1.4.1 // indirect
|
|
||||||
github.com/nats-io/nats-server v1.4.1
|
|
||||||
github.com/nats-io/nats-server/v2 v2.1.2
|
github.com/nats-io/nats-server/v2 v2.1.2
|
||||||
github.com/nats-io/nats.go v1.9.1
|
github.com/nats-io/nats.go v1.9.1
|
||||||
github.com/nlopes/slack v0.6.1-0.20191106133607-d06c2a2b3249
|
github.com/nlopes/slack v0.6.1-0.20191106133607-d06c2a2b3249
|
||||||
github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c
|
github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c
|
||||||
github.com/pkg/errors v0.8.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/technoweenie/multipartstreamer v1.0.1 // indirect
|
||||||
|
github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5 // indirect
|
||||||
|
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect
|
||||||
|
go.etcd.io/bbolt v1.3.3 // indirect
|
||||||
go.uber.org/zap v1.13.0
|
go.uber.org/zap v1.13.0
|
||||||
golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876
|
golang.org/x/crypto v0.0.0-20200117160349-530e935923ad
|
||||||
golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553
|
golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa
|
||||||
google.golang.org/grpc v1.26.0
|
google.golang.org/grpc v1.26.0
|
||||||
|
gopkg.in/go-playground/assert.v1 v1.2.1 // indirect
|
||||||
gopkg.in/go-playground/validator.v9 v9.31.0
|
gopkg.in/go-playground/validator.v9 v9.31.0
|
||||||
gopkg.in/src-d/go-git.v4 v4.13.1
|
gopkg.in/src-d/go-git.v4 v4.13.1
|
||||||
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
|
||||||
)
|
)
|
||||||
|
165
go.sum
165
go.sum
@ -1,8 +1,19 @@
|
|||||||
cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
||||||
cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
||||||
cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
|
cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
|
||||||
|
cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU=
|
||||||
|
cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY=
|
||||||
|
cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc=
|
||||||
|
cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0=
|
||||||
|
cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To=
|
||||||
|
cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o=
|
||||||
|
cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
|
||||||
|
cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I=
|
||||||
|
cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw=
|
||||||
contrib.go.opencensus.io/exporter/ocagent v0.4.12/go.mod h1:450APlNTSR6FrvC3CTRqYosuDstRB9un7SOx2k/9ckA=
|
contrib.go.opencensus.io/exporter/ocagent v0.4.12/go.mod h1:450APlNTSR6FrvC3CTRqYosuDstRB9un7SOx2k/9ckA=
|
||||||
|
dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
|
||||||
github.com/Azure/azure-sdk-for-go v32.4.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc=
|
github.com/Azure/azure-sdk-for-go v32.4.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc=
|
||||||
|
github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 h1:w+iIsaOQNcT7OZ575w+acHgRric5iCyQh+xv+KJ4HB8=
|
||||||
github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8=
|
github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8=
|
||||||
github.com/Azure/go-autorest/autorest v0.1.0/go.mod h1:AKyIcETwSUFxIcs/Wnq/C+kwCtlEYGUVd7FPNb2slmg=
|
github.com/Azure/go-autorest/autorest v0.1.0/go.mod h1:AKyIcETwSUFxIcs/Wnq/C+kwCtlEYGUVd7FPNb2slmg=
|
||||||
github.com/Azure/go-autorest/autorest v0.5.0/go.mod h1:9HLKlQjVBH6U3oDfsXOeVc56THsLPw1L03yban4xThw=
|
github.com/Azure/go-autorest/autorest v0.5.0/go.mod h1:9HLKlQjVBH6U3oDfsXOeVc56THsLPw1L03yban4xThw=
|
||||||
@ -18,20 +29,27 @@ github.com/Azure/go-autorest/logger v0.1.0/go.mod h1:oExouG+K6PryycPJfVSxi/koC6L
|
|||||||
github.com/Azure/go-autorest/tracing v0.1.0/go.mod h1:ROEEAFwXycQw7Sn3DXNtEedEvdeRAgDr0izn4z5Ij88=
|
github.com/Azure/go-autorest/tracing v0.1.0/go.mod h1:ROEEAFwXycQw7Sn3DXNtEedEvdeRAgDr0izn4z5Ij88=
|
||||||
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/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.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=
|
||||||
github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo=
|
github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo=
|
||||||
github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
|
github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
|
||||||
github.com/akamai/AkamaiOPEN-edgegrid-golang v0.9.0/go.mod h1:zpDJeKyp9ScW4NNrbdr+Eyxvry3ilGPewKoXw3XGN1k=
|
github.com/akamai/AkamaiOPEN-edgegrid-golang v0.9.0/go.mod h1:zpDJeKyp9ScW4NNrbdr+Eyxvry3ilGPewKoXw3XGN1k=
|
||||||
|
github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75 h1:3ILjVyslFbc4jl1w5TWuvvslFD/nDfR2H8tVaMVLrEY=
|
||||||
github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75/go.mod h1:uAXEEpARkRhCZfEvy/y0Jcc888f9tHCc1W7/UeEtreE=
|
github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75/go.mod h1:uAXEEpARkRhCZfEvy/y0Jcc888f9tHCc1W7/UeEtreE=
|
||||||
|
github.com/alcortesm/tgz v0.0.0-20161220082320-9c5fe88206d7 h1:uSoVVbwJiQipAclBbw+8quDsfcvFjOpI5iCf4p/cqCs=
|
||||||
github.com/alcortesm/tgz v0.0.0-20161220082320-9c5fe88206d7/go.mod h1:6zEj6s6u/ghQa61ZWa/C2Aw3RkjiTBOix7dkqa1VLIs=
|
github.com/alcortesm/tgz v0.0.0-20161220082320-9c5fe88206d7/go.mod h1:6zEj6s6u/ghQa61ZWa/C2Aw3RkjiTBOix7dkqa1VLIs=
|
||||||
github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
|
github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
|
||||||
github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
|
github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
|
||||||
github.com/aliyun/alibaba-cloud-sdk-go v0.0.0-20190808125512-07798873deee/go.mod h1:myCDvQSzCW+wB1WAlocEru4wMGJxy+vlxHdhegi1CDQ=
|
github.com/aliyun/alibaba-cloud-sdk-go v0.0.0-20190808125512-07798873deee/go.mod h1:myCDvQSzCW+wB1WAlocEru4wMGJxy+vlxHdhegi1CDQ=
|
||||||
github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5/go.mod h1:T/Aws4fEfogEE9v+HPhhw+CntffsBHJ8nXQCwKr0/g8=
|
github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5/go.mod h1:T/Aws4fEfogEE9v+HPhhw+CntffsBHJ8nXQCwKr0/g8=
|
||||||
|
github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239 h1:kFOfPq6dUM1hTo4JG6LR5AXSUEsOjtdm0kw0FtQtMJA=
|
||||||
github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c=
|
github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c=
|
||||||
github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
||||||
|
github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5 h1:0CwZNZbxp69SHPdPJAN/hZIm0C4OItdklCFmMRWYpio=
|
||||||
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=
|
||||||
@ -39,10 +57,13 @@ github.com/beevik/ntp v0.2.0 h1:sGsd+kAXzT0bfVfzJfce04g+dSRfrs+tbQW8lweuYgw=
|
|||||||
github.com/beevik/ntp v0.2.0/go.mod h1:hIHWr+l3+/clUnF44zdK+CWW7fO8dR5cIylAQ76NRpg=
|
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/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
|
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
|
||||||
github.com/bitly/go-simplejson v0.5.0 h1:6IH+V8/tVMab511d5bn4M7EwGXZf9Hj6i2xSwkNEM+Y=
|
github.com/bitly/go-simplejson v0.5.0 h1:6IH+V8/tVMab511d5bn4M7EwGXZf9Hj6i2xSwkNEM+Y=
|
||||||
github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA=
|
github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA=
|
||||||
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/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4=
|
||||||
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/cenkalti/backoff/v3 v3.0.0 h1:ske+9nBpD9qZsTBoF41nW5L+AIuFBKMeze18XQ3eG1c=
|
github.com/cenkalti/backoff/v3 v3.0.0 h1:ske+9nBpD9qZsTBoF41nW5L+AIuFBKMeze18XQ3eG1c=
|
||||||
@ -65,8 +86,12 @@ github.com/containerd/fifo v0.0.0-20190226154929-a9fb20d87448/go.mod h1:ODA38xgv
|
|||||||
github.com/containerd/go-runc v0.0.0-20180907222934-5a6d9f37cfa3/go.mod h1:IV7qH3hrUgRmyYrtgEeGWJfWbgcHL9CSRruz2Vqcph0=
|
github.com/containerd/go-runc v0.0.0-20180907222934-5a6d9f37cfa3/go.mod h1:IV7qH3hrUgRmyYrtgEeGWJfWbgcHL9CSRruz2Vqcph0=
|
||||||
github.com/containerd/ttrpc v0.0.0-20190828154514-0e0f228740de/go.mod h1:PvCDdDGpgqzQIzDW1TphrGLssLDZp2GuS+X5DkEJB8o=
|
github.com/containerd/ttrpc v0.0.0-20190828154514-0e0f228740de/go.mod h1:PvCDdDGpgqzQIzDW1TphrGLssLDZp2GuS+X5DkEJB8o=
|
||||||
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/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk=
|
||||||
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/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
|
||||||
github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e h1:Wf6HqHfScWJN9/ZjdUKyjop4mf3Qdd+1TvvltAvM3m8=
|
github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e h1:Wf6HqHfScWJN9/ZjdUKyjop4mf3Qdd+1TvvltAvM3m8=
|
||||||
github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
|
github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
|
||||||
github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg=
|
github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg=
|
||||||
@ -77,7 +102,7 @@ github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7Do
|
|||||||
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/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=
|
||||||
github.com/dnaeon/go-vcr v0.0.0-20180814043457-aafff18a5cc2/go.mod h1:aBB1+wY4s93YsC3HHjMBMrwTj2R9FHDzUr9KyGc8n1E=
|
github.com/dnaeon/go-vcr v0.0.0-20180814043457-aafff18a5cc2/go.mod h1:aBB1+wY4s93YsC3HHjMBMrwTj2R9FHDzUr9KyGc8n1E=
|
||||||
@ -99,6 +124,7 @@ github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.m
|
|||||||
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/exoscale/egoscale v0.18.1/go.mod h1:Z7OOdzzTOz1Q1PjQXumlz9Wn/CddH0zSYdCF3rnBKXE=
|
github.com/exoscale/egoscale v0.18.1/go.mod h1:Z7OOdzzTOz1Q1PjQXumlz9Wn/CddH0zSYdCF3rnBKXE=
|
||||||
github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M=
|
github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M=
|
||||||
|
github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568 h1:BHsljHzVlRcyQhjrss6TZTdY2VfCqZPbv5k3iBFa2ZQ=
|
||||||
github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc=
|
github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc=
|
||||||
github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c h1:pBgVXWDXju1m8W4lnEeIqTHPOzhTUO81a7yknM/xQR4=
|
github.com/forestgiant/sliceutil v0.0.0-20160425183142-94783f95db6c h1:pBgVXWDXju1m8W4lnEeIqTHPOzhTUO81a7yknM/xQR4=
|
||||||
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=
|
||||||
@ -108,13 +134,13 @@ github.com/fsouza/go-dockerclient v1.6.0 h1:f7j+AX94143JL1H3TiqSMkM4EcLDI0De1qD4
|
|||||||
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/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0=
|
github.com/gliderlabs/ssh v0.2.2/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0=
|
||||||
github.com/go-acme/lego v2.7.2+incompatible h1:ThhpPBgf6oa9X/vRd0kEmWOsX7+vmYdckmGZSb+FEp0=
|
github.com/go-acme/lego/v3 v3.3.0 h1:6BePZsOiYA4/w+M7QDytxQtMfCipMPGnWAHs9pWks98=
|
||||||
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.3.0/go.mod h1:iGSY2vQrvQs3WezicSB/oVbO2eCrD88dpWPwb1qLqu0=
|
||||||
github.com/go-acme/lego/v3 v3.2.0 h1:z0zvNlL1niv/1qA06V5X1BRC5PeLoGKAlVaWthXQz9c=
|
|
||||||
github.com/go-acme/lego/v3 v3.2.0/go.mod h1:074uqt+JS6plx+c9Xaiz6+L+GBb+7itGtzfcDM2AhEE=
|
|
||||||
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=
|
||||||
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
|
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
|
||||||
|
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-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
|
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
|
||||||
@ -124,6 +150,8 @@ github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTM
|
|||||||
github.com/go-playground/universal-translator v0.17.0 h1:icxd5fm+REJzpZx7ZfpaD876Lmtgy7VtROAbHHXk8no=
|
github.com/go-playground/universal-translator v0.17.0 h1:icxd5fm+REJzpZx7ZfpaD876Lmtgy7VtROAbHHXk8no=
|
||||||
github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA=
|
github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA=
|
||||||
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/go.mod h1:qf9acutJ8cwBUhm1bqgz6Bei9/C/c93FPDljKWwsOgM=
|
||||||
github.com/godbus/dbus v0.0.0-20190422162347-ade71ed3457e/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4=
|
github.com/godbus/dbus v0.0.0-20190422162347-ade71ed3457e/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4=
|
||||||
github.com/gofrs/uuid v3.2.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM=
|
github.com/gofrs/uuid v3.2.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM=
|
||||||
github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
|
github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
|
||||||
@ -131,9 +159,14 @@ github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7a
|
|||||||
github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE=
|
github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE=
|
||||||
github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
|
github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
|
||||||
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/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-20191227052852-215e87163ea7 h1:5ZkaAPbicIKTF2I64qf5Fh8Aa83Q/dnOafMYV0OMwjA=
|
||||||
|
github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/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=
|
||||||
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
||||||
|
github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s=
|
||||||
|
github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y=
|
||||||
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0=
|
github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0=
|
||||||
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
@ -141,13 +174,17 @@ github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs
|
|||||||
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
|
github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
|
||||||
github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
|
github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
|
||||||
|
github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo=
|
||||||
|
github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
|
||||||
github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
|
github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
|
||||||
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
||||||
|
github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg=
|
||||||
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
||||||
github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
|
github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
|
||||||
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
|
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
|
||||||
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
|
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
|
||||||
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
||||||
|
github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
||||||
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
|
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
|
||||||
github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=
|
github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=
|
||||||
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
||||||
@ -164,6 +201,11 @@ github.com/gorilla/websocket v1.2.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoA
|
|||||||
github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ=
|
github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ=
|
||||||
github.com/gorilla/websocket v1.4.1 h1:q7AeDBpnBk8AogcD4DSag/Ukw/KV+YhzLj2bP5HvKCM=
|
github.com/gorilla/websocket v1.4.1 h1:q7AeDBpnBk8AogcD4DSag/Ukw/KV+YhzLj2bP5HvKCM=
|
||||||
github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
|
github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
|
||||||
|
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 h1:THDBEeQ9xZ8JEaCLyLQqXMMdRqNr0QAUJTIkQAUtFjg=
|
||||||
|
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE=
|
||||||
|
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho=
|
||||||
|
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
|
||||||
|
github.com/grpc-ecosystem/grpc-gateway v1.8.5 h1:2+KSC78XiO6Qy0hIjfc1OD9H+hsaJdJlb8Kqsd41CTE=
|
||||||
github.com/grpc-ecosystem/grpc-gateway v1.8.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
|
github.com/grpc-ecosystem/grpc-gateway v1.8.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
|
||||||
github.com/h2non/parth v0.0.0-20190131123155-b4df798d6542/go.mod h1:Ow0tF8D4Kplbc8s8sSb3V2oUCygFHVp8gC3Dn6U4MNI=
|
github.com/h2non/parth v0.0.0-20190131123155-b4df798d6542/go.mod h1:Ow0tF8D4Kplbc8s8sSb3V2oUCygFHVp8gC3Dn6U4MNI=
|
||||||
github.com/hashicorp/errwrap v0.0.0-20141028054710-7554cd9344ce/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
|
github.com/hashicorp/errwrap v0.0.0-20141028054710-7554cd9344ce/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
|
||||||
@ -173,6 +215,7 @@ github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ
|
|||||||
github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
|
github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
|
||||||
github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4=
|
github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4=
|
||||||
github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ=
|
github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ=
|
||||||
|
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/imdario/mergo v0.3.8 h1:CGgOkSJeqMRmt0D9XLWExdT4m4F1vd3FV3VPt+0VxkQ=
|
github.com/imdario/mergo v0.3.8 h1:CGgOkSJeqMRmt0D9XLWExdT4m4F1vd3FV3VPt+0VxkQ=
|
||||||
@ -181,6 +224,8 @@ github.com/jbenet/go-context v0.0.0-20150711004518-d14ea06fba99 h1:BQSFePA1RWJOl
|
|||||||
github.com/jbenet/go-context v0.0.0-20150711004518-d14ea06fba99/go.mod h1:1lJo3i6rXxKeerYnT8Nvf0QmHCRC1n8sfWVwXF2Frvo=
|
github.com/jbenet/go-context v0.0.0-20150711004518-d14ea06fba99/go.mod h1:1lJo3i6rXxKeerYnT8Nvf0QmHCRC1n8sfWVwXF2Frvo=
|
||||||
github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI=
|
github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI=
|
||||||
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/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
|
||||||
github.com/joncalhoun/qson v0.0.0-20170526102502-8a9cab3a62b1 h1:lnrOS18wZBYrzdDmnUeg1OVk+kQ3rxG8mZWU89DpMIA=
|
github.com/joncalhoun/qson v0.0.0-20170526102502-8a9cab3a62b1 h1:lnrOS18wZBYrzdDmnUeg1OVk+kQ3rxG8mZWU89DpMIA=
|
||||||
github.com/joncalhoun/qson v0.0.0-20170526102502-8a9cab3a62b1/go.mod h1:DFXrEwSRX0p/aSvxE21319menCBFeQO0jXpRj7LEZUA=
|
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=
|
||||||
@ -199,6 +244,7 @@ github.com/klauspost/cpuid v1.2.0 h1:NMpwD2G9JSFOE1/TJjGSo5zG7Yb2bTe7eq1jH+irmeE
|
|||||||
github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
|
github.com/klauspost/cpuid v1.2.0/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=
|
||||||
github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
|
github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
|
||||||
|
github.com/konsorten/go-windows-terminal-sequences v1.0.2 h1:DB17ag19krx9CFsz4o3enTrPXyIXCl+2iCXH/aMAp9s=
|
||||||
github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
|
github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
|
||||||
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
|
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
|
||||||
github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
|
github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
|
||||||
@ -226,17 +272,18 @@ github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNx
|
|||||||
github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
|
github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
|
||||||
github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
|
github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
|
||||||
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/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
|
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
|
||||||
github.com/mholt/certmagic v0.9.0 h1:dYh9sZPDBTcIiPhYM/Qtv3V623/zFH34FmpbrQTpMAc=
|
github.com/mholt/certmagic v0.9.1 h1:wPzyouOyE+30NIQETJuhTB5ZQWz+0Hy038vaR5WWQDE=
|
||||||
github.com/mholt/certmagic v0.9.0/go.mod h1:91uJzK5K8IWtYQqTi5R2tsxV1pCde+wdGfaRaOZi6aQ=
|
github.com/mholt/certmagic v0.9.1/go.mod h1:nu8jbsbtwK4205EDH/ZUMTKsfYpJA1Q7MKXHfgTihNw=
|
||||||
github.com/micro/cli v0.2.0 h1:ut3rV5JWqZjsXIa2MvGF+qMUP8DAUTvHX9Br5gO4afA=
|
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 v0.2.0/go.mod h1:jRT9gmfVKWSS6pkKcXQ8YhUyj6bzwxK8Fp5b0Y7qNnk=
|
||||||
github.com/micro/mdns v0.3.0 h1:bYycYe+98AXR3s8Nq5qvt6C573uFTDPIYzJemWON0QE=
|
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/mdns v0.3.0/go.mod h1:KJ0dW7KmicXU2BV++qkLlmHYcVv7/hHnbtguSWt9Aoc=
|
||||||
github.com/miekg/dns v1.1.3/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
|
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.26 h1:gPxPSwALAeHJSjarOs00QjVdV9QoBvc1D2ujQUr5BzU=
|
github.com/miekg/dns v1.1.27 h1:aEH/kqUzUxGJ/UHcEKdJY+ugH6WEzsEBBSPa8zuy1aM=
|
||||||
github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
|
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=
|
||||||
github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
|
github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
|
||||||
github.com/mitchellh/go-vnc v0.0.0-20150629162542-723ed9867aed/go.mod h1:3rdaFaCv4AyBgu5ALFM0+tSuHrBh6v692nyQe3ikrq0=
|
github.com/mitchellh/go-vnc v0.0.0-20150629162542-723ed9867aed/go.mod h1:3rdaFaCv4AyBgu5ALFM0+tSuHrBh6v692nyQe3ikrq0=
|
||||||
@ -253,35 +300,30 @@ 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/gnatsd v1.4.1 h1:RconcfDeWpKCD6QIIwiVFcvForlXpWeJP7i5/lDLy44=
|
|
||||||
github.com/nats-io/gnatsd v1.4.1/go.mod h1:nqco77VO78hLCJpIcVfygDP2rPGfsEHkGTUk94uh5DQ=
|
|
||||||
github.com/nats-io/jwt v0.3.0 h1:xdnzwFETV++jNc4W1mw//qFyJGb2ABOombmZJQS4+Qo=
|
|
||||||
github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg=
|
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 v1.4.1 h1:Ul1oSOGNV/L8kjr4v6l2f9Yet6WY+LevH1/7cRZ/qyA=
|
|
||||||
github.com/nats-io/nats-server v1.4.1/go.mod h1:c8f/fHd2B6Hgms3LtCaI7y6pC4WD1f4SUxcCud5vhBc=
|
|
||||||
github.com/nats-io/nats-server/v2 v2.1.2 h1:i2Ly0B+1+rzNZHHWtD4ZwKi+OU5l+uQo1iDHZ2PmiIc=
|
github.com/nats-io/nats-server/v2 v2.1.2 h1:i2Ly0B+1+rzNZHHWtD4ZwKi+OU5l+uQo1iDHZ2PmiIc=
|
||||||
github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k=
|
github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k=
|
||||||
github.com/nats-io/nats.go v1.9.1 h1:ik3HbLhZ0YABLto7iX80pZLPw/6dx3T+++MZJwLnMrQ=
|
github.com/nats-io/nats.go v1.9.1 h1:ik3HbLhZ0YABLto7iX80pZLPw/6dx3T+++MZJwLnMrQ=
|
||||||
github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w=
|
github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w=
|
||||||
github.com/nats-io/nkeys v0.1.0 h1:qMd4+pRHgdr1nAClu+2h/2a5F2TmKcCzjCDazVgRoX4=
|
|
||||||
github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
|
github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
|
||||||
github.com/nats-io/nkeys v0.1.3 h1:6JrEfig+HzTH85yxzhSVbjHRJv9cn0p6n3IngIcM5/k=
|
github.com/nats-io/nkeys v0.1.3 h1:6JrEfig+HzTH85yxzhSVbjHRJv9cn0p6n3IngIcM5/k=
|
||||||
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/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw=
|
github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw=
|
||||||
github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c=
|
github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c=
|
||||||
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/nlopes/slack v0.6.0 h1:jt0jxVQGhssx1Ib7naAOZEZcGdtIhTzkP0nopK0AsRA=
|
|
||||||
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=
|
||||||
|
github.com/nrdcg/dnspod-go v0.3.0/go.mod h1:vZSoFSFeQVm2gWLMkyX61LZ8HI3BaqtHZWgPTGKr6KQ=
|
||||||
github.com/nrdcg/goinwx v0.6.1/go.mod h1:XPiut7enlbEdntAqalBIqcYcTEVhpv/dKWgDCX2SwKQ=
|
github.com/nrdcg/goinwx v0.6.1/go.mod h1:XPiut7enlbEdntAqalBIqcYcTEVhpv/dKWgDCX2SwKQ=
|
||||||
github.com/nrdcg/namesilo v0.2.1/go.mod h1:lwMvfQTyYq+BbjJd30ylEG4GPSS6PII0Tia4rRpRiyw=
|
github.com/nrdcg/namesilo v0.2.1/go.mod h1:lwMvfQTyYq+BbjJd30ylEG4GPSS6PII0Tia4rRpRiyw=
|
||||||
github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
|
github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
|
||||||
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
|
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
|
||||||
|
github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs=
|
||||||
github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
|
github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
|
||||||
|
github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU=
|
||||||
github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
|
github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
|
||||||
github.com/opencontainers/go-digest v0.0.0-20180430190053-c9281466c8b2/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s=
|
github.com/opencontainers/go-digest v0.0.0-20180430190053-c9281466c8b2/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s=
|
||||||
github.com/opencontainers/go-digest v1.0.0-rc1 h1:WzifXhOVOEOuFYOJAW6aQqW0TooG2iki3E3Ii+WN7gQ=
|
github.com/opencontainers/go-digest v1.0.0-rc1 h1:WzifXhOVOEOuFYOJAW6aQqW0TooG2iki3E3Ii+WN7gQ=
|
||||||
@ -293,6 +335,7 @@ github.com/opencontainers/runc v0.1.1 h1:GlxAyO6x8rfZYN9Tt0Kti5a/cP41iuiO2yYT0IJ
|
|||||||
github.com/opencontainers/runc v0.1.1/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U=
|
github.com/opencontainers/runc v0.1.1/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U=
|
||||||
github.com/opencontainers/runtime-spec v0.1.2-0.20190507144316-5b71a03e2700/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0=
|
github.com/opencontainers/runtime-spec v0.1.2-0.20190507144316-5b71a03e2700/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0=
|
||||||
github.com/opencontainers/runtime-tools v0.0.0-20181011054405-1d69bd0f9c39/go.mod h1:r3f7wjNzSs2extwzU3Y+6pKfobzPh+kKFJ3ofN+3nfs=
|
github.com/opencontainers/runtime-tools v0.0.0-20181011054405-1d69bd0f9c39/go.mod h1:r3f7wjNzSs2extwzU3Y+6pKfobzPh+kKFJ3ofN+3nfs=
|
||||||
|
github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
|
||||||
github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw=
|
github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw=
|
||||||
github.com/oracle/oci-go-sdk v7.0.0+incompatible/go.mod h1:VQb79nF8Z2cwLkLS35ukwStZIg5F66tcBccjip/j888=
|
github.com/oracle/oci-go-sdk v7.0.0+incompatible/go.mod h1:VQb79nF8Z2cwLkLS35ukwStZIg5F66tcBccjip/j888=
|
||||||
github.com/ovh/go-ovh v0.0.0-20181109152953-ba5adb4cf014/go.mod h1:joRatxRJaZBsY3JAOEMcoOp05CnZzsx4scTxi95DHyQ=
|
github.com/ovh/go-ovh v0.0.0-20181109152953-ba5adb4cf014/go.mod h1:joRatxRJaZBsY3JAOEMcoOp05CnZzsx4scTxi95DHyQ=
|
||||||
@ -301,25 +344,30 @@ github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c/go.mod h1:X07ZCGwU
|
|||||||
github.com/pelletier/go-buffruneio v0.2.0/go.mod h1:JkE26KsDizTr40EUHkXVtNPvgGtbSNq5BcowyYOWdKo=
|
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 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I=
|
|
||||||
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||||
|
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
|
||||||
|
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||||
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
|
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
|
||||||
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
||||||
github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
|
github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
|
||||||
github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs=
|
github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs=
|
||||||
github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
|
github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
|
||||||
|
github.com/prometheus/client_golang v1.1.0 h1:BQ53HtBmfOitExawJ6LokA4x8ov/z0SYYb0+HxJfRI8=
|
||||||
github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g=
|
github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g=
|
||||||
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
|
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
|
||||||
github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
|
github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
|
||||||
github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
|
github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
|
||||||
|
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 h1:gQz4mCbXsO+nc9n1hCxHcGA3Zx3Eo+UHZoInFGUIXNM=
|
||||||
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
|
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
|
||||||
github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
|
github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
|
||||||
github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
|
github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
|
||||||
|
github.com/prometheus/common v0.6.0 h1:kRhiuYSXR3+uv2IbVbZhUxK5zVD/2pp3Gd2PpvPkpEo=
|
||||||
github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc=
|
github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc=
|
||||||
github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
||||||
github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
||||||
github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
|
github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
|
||||||
github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ=
|
github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ=
|
||||||
|
github.com/prometheus/procfs v0.0.5 h1:3+auTFlqw+ZaQYJARz6ArODtkaIwtvBTx3N2NehQlL8=
|
||||||
github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ=
|
github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ=
|
||||||
github.com/rainycape/memcache v0.0.0-20150622160815-1031fa0ce2f2/go.mod h1:7tZKcyumwBO6qip7RNQ5r77yrssm9bfCowcLEBcU5IA=
|
github.com/rainycape/memcache v0.0.0-20150622160815-1031fa0ce2f2/go.mod h1:7tZKcyumwBO6qip7RNQ5r77yrssm9bfCowcLEBcU5IA=
|
||||||
github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
|
github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
|
||||||
@ -338,11 +386,12 @@ github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6Mwd
|
|||||||
github.com/skratchdot/open-golang v0.0.0-20160302144031-75fb7ed4208c/go.mod h1:sUM3LWHvSMaG192sy56D9F7CNvL7jUJVXoqM1QKLnog=
|
github.com/skratchdot/open-golang v0.0.0-20160302144031-75fb7ed4208c/go.mod h1:sUM3LWHvSMaG192sy56D9F7CNvL7jUJVXoqM1QKLnog=
|
||||||
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
|
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
|
||||||
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/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
|
||||||
github.com/src-d/gcfg v1.4.0 h1:xXbNR5AlLSA315x2UO+fTSSAXCDf+Ar38/6oyGbDKQ4=
|
github.com/src-d/gcfg v1.4.0 h1:xXbNR5AlLSA315x2UO+fTSSAXCDf+Ar38/6oyGbDKQ4=
|
||||||
github.com/src-d/gcfg v1.4.0/go.mod h1:p/UMsR43ujA89BJY9duynAwIpvqEujIH/jFlfL7jWoI=
|
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/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 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48=
|
|
||||||
github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE=
|
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=
|
||||||
@ -352,6 +401,8 @@ 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 h1:LnC5Kc/wtumK+WB441p7ynQJzVuNRJiqddSIE3IlSEQ=
|
||||||
|
github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/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=
|
||||||
github.com/uber-go/atomic v1.3.2/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g=
|
github.com/uber-go/atomic v1.3.2/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g=
|
||||||
github.com/urfave/cli v0.0.0-20171014202726-7bc6a0acffa5/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
|
github.com/urfave/cli v0.0.0-20171014202726-7bc6a0acffa5/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
|
||||||
@ -363,17 +414,25 @@ github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2
|
|||||||
github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ=
|
github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ=
|
||||||
github.com/xeipuuv/gojsonschema v0.0.0-20180618132009-1d523034197f/go.mod h1:5yf86TLmAcydyeJq5YvxkGPE2fm/u4myDekKRoLuqhs=
|
github.com/xeipuuv/gojsonschema v0.0.0-20180618132009-1d523034197f/go.mod h1:5yf86TLmAcydyeJq5YvxkGPE2fm/u4myDekKRoLuqhs=
|
||||||
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/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
|
||||||
|
go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk=
|
||||||
|
go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
|
||||||
go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
|
go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
|
||||||
go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
|
go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
|
||||||
go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
|
go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
|
||||||
go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
|
go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
|
||||||
go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
|
go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
|
||||||
|
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.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/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.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=
|
||||||
@ -385,21 +444,34 @@ golang.org/x/crypto v0.0.0-20190219172222-a4c6cb3142f2/go.mod h1:6SG95UA2DQfeDnf
|
|||||||
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-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-20191227163750-53104e6ec876 h1:sKJQZMuxjOAR/Uo2LBfU90onWEf1dF4C+0hPJCc9Mpc=
|
golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
|
||||||
golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
golang.org/x/crypto v0.0.0-20200117160349-530e935923ad h1:Jh8cai0fqIK+f6nG0UgPW5wFk8wmiMhM3AyciDBdtQg=
|
||||||
|
golang.org/x/crypto v0.0.0-20200117160349-530e935923ad/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
||||||
golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
||||||
|
golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
||||||
|
golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8=
|
||||||
|
golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek=
|
||||||
|
golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
|
||||||
|
golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
|
||||||
|
golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
|
||||||
golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
|
golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
|
||||||
golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
|
golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
|
||||||
golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
|
golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
|
||||||
golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||||
golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||||
|
golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||||
golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||||
|
golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f h1:J5lckAjkw6qYlOZNj90mLYNTEKDvWeuc1yieZ8qUzUE=
|
||||||
|
golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs=
|
||||||
|
golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE=
|
||||||
|
golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o=
|
||||||
golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc=
|
golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc=
|
||||||
|
golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
|
||||||
|
golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
|
||||||
golang.org/x/net v0.0.0-20180611182652-db08ff08e862/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
golang.org/x/net v0.0.0-20180611182652-db08ff08e862/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
@ -414,13 +486,15 @@ golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn
|
|||||||
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||||
golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||||
golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||||
|
golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
|
||||||
golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||||
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/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-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-20191209160850-c0dbc17a3553 h1:efeOvDhwQ29Dj3SdAV/MJf8oukgn+8D8WgaCaRMchF8=
|
golang.org/x/net v0.0.0-20191027093000-83d349e8ac1a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||||
golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa h1:F+8P+gmewFQYRk6JoLQLwjBCTu3mcIURZfNkVweuRKA=
|
||||||
|
golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||||
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
|
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
|
||||||
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
||||||
golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
||||||
@ -428,8 +502,9 @@ golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJ
|
|||||||
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU=
|
|
||||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY=
|
||||||
|
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
golang.org/x/sys v0.0.0-20180622082034-63fc586f45fe/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
golang.org/x/sys v0.0.0-20180622082034-63fc586f45fe/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
@ -441,17 +516,19 @@ golang.org/x/sys v0.0.0-20190129075346-302c3dd5f1cc/go.mod h1:STP8DvDyc/dI5b8T5h
|
|||||||
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-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=
|
||||||
golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
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-20190624142023-c5567b49c5d0/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/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||||
@ -468,43 +545,66 @@ golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGm
|
|||||||
golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
|
golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
|
||||||
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
|
golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
|
golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
||||||
golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
||||||
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
||||||
|
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-20190729092621-ff9f1409240a/go.mod h1:jcCCGcm9btYwXyDqrUWc6MKQKKGJCWEQ3AfLSRIbEuI=
|
golang.org/x/tools v0.0.0-20190729092621-ff9f1409240a/go.mod h1:jcCCGcm9btYwXyDqrUWc6MKQKKGJCWEQ3AfLSRIbEuI=
|
||||||
golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/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-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-20191029041327-9cc4af7d6b2c/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-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
|
golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
|
||||||
|
golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
|
||||||
|
golang.org/x/tools v0.0.0-20191216052735-49a3e744a425/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
|
||||||
|
golang.org/x/tools v0.0.0-20191216173652-a0e659d51361 h1:RIIXAeV6GvDBuADKumTODatUqANFZ+5BPMnzsy4hulY=
|
||||||
|
golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
|
||||||
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
||||||
|
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
||||||
google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk=
|
google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk=
|
||||||
google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE=
|
google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE=
|
||||||
|
google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M=
|
||||||
google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
|
google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
|
||||||
|
google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
|
||||||
|
google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI=
|
||||||
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
|
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
|
||||||
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
|
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
|
||||||
google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
|
google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
|
||||||
|
google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0=
|
||||||
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
||||||
google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
||||||
google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE=
|
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-20191216164720-4f79533eabd1 h1:aQktFqmDE2yjveXJlVIfslDFmFnUXSqG0i6KRcJAeMc=
|
||||||
|
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=
|
||||||
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
||||||
google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
||||||
google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
|
google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
|
||||||
|
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.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=
|
||||||
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
|
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
|
||||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||||
|
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY=
|
||||||
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||||
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/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
|
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
|
||||||
|
gopkg.in/go-playground/assert.v1 v1.2.1 h1:xoYuJVE7KT85PYWrN730RguIQO0ePzVRfFMXadIrXTM=
|
||||||
|
gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE=
|
||||||
gopkg.in/go-playground/validator.v9 v9.31.0 h1:bmXmP2RSNtFES+bn4uYuHT7iJFJv7Vj+an+ZQdDaD1M=
|
gopkg.in/go-playground/validator.v9 v9.31.0 h1:bmXmP2RSNtFES+bn4uYuHT7iJFJv7Vj+an+ZQdDaD1M=
|
||||||
gopkg.in/go-playground/validator.v9 v9.31.0/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ=
|
gopkg.in/go-playground/validator.v9 v9.31.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=
|
||||||
@ -517,11 +617,13 @@ 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 h1:0SQA1pRztfTFx2miS8sA97XvooFeNOmvUenF4o0EcVg=
|
gopkg.in/src-d/go-billy.v4 v4.3.2 h1:0SQA1pRztfTFx2miS8sA97XvooFeNOmvUenF4o0EcVg=
|
||||||
gopkg.in/src-d/go-billy.v4 v4.3.2/go.mod h1:nDjArDMp+XMs1aFAESLRjfGSgfvoYN0hDfzEk0GjC98=
|
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 h1:ivZFOIltbce2Mo8IjzUHAFoq/IylO9WHhNOAJK+LsJg=
|
||||||
gopkg.in/src-d/go-git-fixtures.v3 v3.5.0/go.mod h1:dLBcvytrw/TYZsNTWCnkNF2DSIlzWYqTe3rJR56Ac7g=
|
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 h1:SRtFyV8Kxc0UP7aCHcijOMQGPxHSmMOPrzulQWolkYE=
|
gopkg.in/src-d/go-git.v4 v4.13.1 h1:SRtFyV8Kxc0UP7aCHcijOMQGPxHSmMOPrzulQWolkYE=
|
||||||
gopkg.in/src-d/go-git.v4 v4.13.1/go.mod h1:nx5NYcxdKxq5fpltdHnPa2Exj4Sx0EclMWZQbYDu2z8=
|
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/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
|
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
|
||||||
gopkg.in/warnings.v0 v0.1.2 h1:wFXVbFY8DY5/xOe1ECiWdKCzZlxgshcYVNkBHstARME=
|
gopkg.in/warnings.v0 v0.1.2 h1:wFXVbFY8DY5/xOe1ECiWdKCzZlxgshcYVNkBHstARME=
|
||||||
gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI=
|
gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI=
|
||||||
@ -529,11 +631,16 @@ gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bl
|
|||||||
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
|
gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
|
||||||
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
|
gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo=
|
||||||
gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw=
|
gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw=
|
||||||
honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
|
honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM=
|
||||||
honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg=
|
honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg=
|
||||||
k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk=
|
k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk=
|
||||||
|
rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8=
|
||||||
|
sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs=
|
||||||
|
sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o=
|
||||||
|
@ -13,6 +13,7 @@ import (
|
|||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/micro/go-micro/client"
|
"github.com/micro/go-micro/client"
|
||||||
|
cmucp "github.com/micro/go-micro/client/mucp"
|
||||||
rtr "github.com/micro/go-micro/client/selector/router"
|
rtr "github.com/micro/go-micro/client/selector/router"
|
||||||
"github.com/micro/go-micro/network/resolver/dns"
|
"github.com/micro/go-micro/network/resolver/dns"
|
||||||
pbNet "github.com/micro/go-micro/network/service/proto"
|
pbNet "github.com/micro/go-micro/network/service/proto"
|
||||||
@ -20,6 +21,7 @@ import (
|
|||||||
"github.com/micro/go-micro/router"
|
"github.com/micro/go-micro/router"
|
||||||
pbRtr "github.com/micro/go-micro/router/service/proto"
|
pbRtr "github.com/micro/go-micro/router/service/proto"
|
||||||
"github.com/micro/go-micro/server"
|
"github.com/micro/go-micro/server"
|
||||||
|
smucp "github.com/micro/go-micro/server/mucp"
|
||||||
"github.com/micro/go-micro/transport"
|
"github.com/micro/go-micro/transport"
|
||||||
"github.com/micro/go-micro/tunnel"
|
"github.com/micro/go-micro/tunnel"
|
||||||
bun "github.com/micro/go-micro/tunnel/broker"
|
bun "github.com/micro/go-micro/tunnel/broker"
|
||||||
@ -80,8 +82,6 @@ type network struct {
|
|||||||
closed chan bool
|
closed chan bool
|
||||||
// whether we've discovered by the network
|
// whether we've discovered by the network
|
||||||
discovered chan bool
|
discovered chan bool
|
||||||
// solicted checks whether routes were solicited by one node
|
|
||||||
solicited chan *node
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// message is network message
|
// message is network message
|
||||||
@ -140,7 +140,7 @@ func newNetwork(opts ...Option) Network {
|
|||||||
)
|
)
|
||||||
|
|
||||||
// server is network server
|
// server is network server
|
||||||
server := server.NewServer(
|
server := smucp.NewServer(
|
||||||
server.Id(options.Id),
|
server.Id(options.Id),
|
||||||
server.Address(peerAddress),
|
server.Address(peerAddress),
|
||||||
server.Advertise(advertise),
|
server.Advertise(advertise),
|
||||||
@ -150,7 +150,7 @@ func newNetwork(opts ...Option) Network {
|
|||||||
)
|
)
|
||||||
|
|
||||||
// client is network client
|
// client is network client
|
||||||
client := client.NewClient(
|
client := cmucp.NewClient(
|
||||||
client.Broker(tunBroker),
|
client.Broker(tunBroker),
|
||||||
client.Transport(tunTransport),
|
client.Transport(tunTransport),
|
||||||
client.Selector(
|
client.Selector(
|
||||||
@ -176,7 +176,6 @@ func newNetwork(opts ...Option) Network {
|
|||||||
tunClient: make(map[string]tunnel.Session),
|
tunClient: make(map[string]tunnel.Session),
|
||||||
peerLinks: make(map[string]tunnel.Link),
|
peerLinks: make(map[string]tunnel.Link),
|
||||||
discovered: make(chan bool, 1),
|
discovered: make(chan bool, 1),
|
||||||
solicited: make(chan *node, 32),
|
|
||||||
}
|
}
|
||||||
|
|
||||||
network.node.network = network
|
network.node.network = network
|
||||||
@ -346,12 +345,10 @@ func (n *network) advertise(advertChan <-chan *router.Advert) {
|
|||||||
Events: events,
|
Events: events,
|
||||||
}
|
}
|
||||||
|
|
||||||
// send the advert to a select number of random peers
|
|
||||||
if advert.Type != router.Solicitation {
|
|
||||||
// get a list of node peers
|
// get a list of node peers
|
||||||
peers := n.Peers()
|
peers := n.Peers()
|
||||||
|
|
||||||
// there is no one to send to
|
// continue if there is no one to send to
|
||||||
if len(peers) == 0 {
|
if len(peers) == 0 {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
@ -369,41 +366,6 @@ func (n *network) advertise(advertChan <-chan *router.Advert) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
// it's a solication, someone asked for it
|
|
||||||
// so we're going to pick off the node and send it
|
|
||||||
select {
|
|
||||||
case peer := <-n.solicited:
|
|
||||||
// someone requested the route
|
|
||||||
n.sendTo("advert", ControlChannel, peer, msg)
|
|
||||||
default:
|
|
||||||
// get a list of node peers
|
|
||||||
peers := n.Peers()
|
|
||||||
|
|
||||||
// only proceed if we have a peer
|
|
||||||
if len(peers) == 0 {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
// pick a random peer from the list of peers
|
|
||||||
peer := n.node.GetPeerNode(peers[rnd.Intn(len(peers))].Id())
|
|
||||||
// only proceed with a peer
|
|
||||||
if peer == nil {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
// attempt to send the advert to the peer
|
|
||||||
if err := n.sendTo("advert", ControlChannel, peer, msg); err != nil {
|
|
||||||
log.Debugf("Network failed to advertise routes to %s: %v, sending multicast", peer.Id(), err)
|
|
||||||
// send a multicast message if we fail to send Unicast message
|
|
||||||
if err := n.sendMsg("advert", ControlChannel, msg); err != nil {
|
|
||||||
log.Debugf("Network failed to advertise routes to %s: %v", peer.Id(), err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
case <-n.closed:
|
case <-n.closed:
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -740,38 +702,6 @@ func (n *network) processCtrlChan(listener tunnel.Listener) {
|
|||||||
if err := n.router.Process(advert); err != nil {
|
if err := n.router.Process(advert); err != nil {
|
||||||
log.Debugf("Network failed to process advert %s: %v", advert.Id, err)
|
log.Debugf("Network failed to process advert %s: %v", advert.Id, err)
|
||||||
}
|
}
|
||||||
case "solicit":
|
|
||||||
pbRtrSolicit := new(pbRtr.Solicit)
|
|
||||||
if err := proto.Unmarshal(m.msg.Body, pbRtrSolicit); err != nil {
|
|
||||||
log.Debugf("Network fail to unmarshal solicit message: %v", err)
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
log.Debugf("Network received solicit message from: %s", pbRtrSolicit.Id)
|
|
||||||
|
|
||||||
// ignore solicitation when requested by you
|
|
||||||
if pbRtrSolicit.Id == n.options.Id {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
log.Tracef("Network router flushing routes for: %s", pbRtrSolicit.Id)
|
|
||||||
|
|
||||||
peer := &node{
|
|
||||||
id: pbRtrSolicit.Id,
|
|
||||||
link: m.msg.Header["Micro-Link"],
|
|
||||||
}
|
|
||||||
|
|
||||||
// specify that someone solicited the route
|
|
||||||
select {
|
|
||||||
case n.solicited <- peer:
|
|
||||||
default:
|
|
||||||
// don't block
|
|
||||||
}
|
|
||||||
|
|
||||||
// advertise all the routes when a new node has connected
|
|
||||||
if err := n.router.Solicit(); err != nil {
|
|
||||||
log.Debugf("Network failed to solicit routes: %s", err)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
case <-n.closed:
|
case <-n.closed:
|
||||||
return
|
return
|
||||||
@ -850,52 +780,18 @@ func (n *network) processNetChan(listener tunnel.Listener) {
|
|||||||
Peer: node,
|
Peer: node,
|
||||||
}
|
}
|
||||||
|
|
||||||
// get a list of all of our routes
|
// get a list of the best routes for each service in our routing table
|
||||||
routes, err := n.options.Router.Table().List()
|
routes, err := n.getProtoRoutes()
|
||||||
switch err {
|
if err != nil {
|
||||||
case nil:
|
|
||||||
// encode the routes to protobuf
|
|
||||||
pbRoutes := make([]*pbRtr.Route, 0, len(routes))
|
|
||||||
for _, route := range routes {
|
|
||||||
// generate new route proto
|
|
||||||
pbRoute := pbUtil.RouteToProto(route)
|
|
||||||
// mask the route before outbounding
|
|
||||||
n.maskRoute(pbRoute)
|
|
||||||
// add to list of routes
|
|
||||||
pbRoutes = append(pbRoutes, pbRoute)
|
|
||||||
}
|
|
||||||
// pack the routes into the sync message
|
|
||||||
msg.Routes = pbRoutes
|
|
||||||
default:
|
|
||||||
// we can't list the routes
|
|
||||||
log.Debugf("Network node %s failed listing routes: %v", n.id, err)
|
log.Debugf("Network node %s failed listing routes: %v", n.id, err)
|
||||||
}
|
}
|
||||||
|
// attached the routes to the message
|
||||||
|
msg.Routes = routes
|
||||||
|
|
||||||
// send sync message to the newly connected peer
|
// send sync message to the newly connected peer
|
||||||
if err := n.sendTo("sync", NetworkChannel, peer, msg); err != nil {
|
if err := n.sendTo("sync", NetworkChannel, peer, msg); err != nil {
|
||||||
log.Debugf("Network failed to send sync message: %v", err)
|
log.Debugf("Network failed to send sync message: %v", err)
|
||||||
}
|
}
|
||||||
// wait for a short period of time before sending a solicit message
|
|
||||||
<-time.After(time.Millisecond * 100)
|
|
||||||
|
|
||||||
// send a solicit message when discovering new peer
|
|
||||||
// this triggers the node to flush its routing table to the network
|
|
||||||
// and leads to faster convergence of the network
|
|
||||||
solicit := &pbRtr.Solicit{
|
|
||||||
Id: n.options.Id,
|
|
||||||
}
|
|
||||||
|
|
||||||
// ask for the new nodes routes
|
|
||||||
if err := n.sendTo("solicit", ControlChannel, peer, solicit); err != nil {
|
|
||||||
log.Debugf("Network failed to send solicit message: %s", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
// now advertise our own routes
|
|
||||||
select {
|
|
||||||
case n.solicited <- peer:
|
|
||||||
default:
|
|
||||||
// don't block
|
|
||||||
}
|
|
||||||
}()
|
}()
|
||||||
case "peer":
|
case "peer":
|
||||||
// mark the time the message has been received
|
// mark the time the message has been received
|
||||||
@ -931,38 +827,27 @@ func (n *network) processNetChan(listener tunnel.Listener) {
|
|||||||
log.Debugf("Network failed updating peer links: %s", err)
|
log.Debugf("Network failed updating peer links: %s", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// if it's a new peer i.e. we do not have it in our graph, we solicit its routes
|
// if it's a new peer i.e. we do not have it in our graph, we request full sync
|
||||||
if err := n.node.AddPeer(peer); err == nil {
|
if err := n.node.AddPeer(peer); err == nil {
|
||||||
go func() {
|
go func() {
|
||||||
msg := PeersToProto(n.node, MaxDepth)
|
// marshal node graph into protobuf
|
||||||
|
node := PeersToProto(n.node, MaxDepth)
|
||||||
|
|
||||||
// advertise yourself to the peer
|
msg := &pbNet.Sync{
|
||||||
if err := n.sendTo("peer", NetworkChannel, peer, msg); err != nil {
|
Peer: node,
|
||||||
log.Debugf("Network failed to advertise peers: %v", err)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
<-time.After(time.Millisecond * 100)
|
// get a list of the best routes for each service in our routing table
|
||||||
|
routes, err := n.getProtoRoutes()
|
||||||
// send a solicit message when discovering new peer
|
if err != nil {
|
||||||
solicit := &pbRtr.Solicit{
|
log.Debugf("Network node %s failed listing routes: %v", n.id, err)
|
||||||
Id: n.options.Id,
|
|
||||||
}
|
}
|
||||||
|
// attached the routes to the message
|
||||||
|
msg.Routes = routes
|
||||||
|
|
||||||
// then solicit this peer
|
// send sync message to the newly connected peer
|
||||||
if err := n.sendTo("solicit", ControlChannel, peer, solicit); err != nil {
|
if err := n.sendTo("sync", NetworkChannel, peer, msg); err != nil {
|
||||||
log.Debugf("Network failed to send solicit message: %s", err)
|
log.Debugf("Network failed to send sync message: %v", err)
|
||||||
}
|
|
||||||
|
|
||||||
// now advertise our own routes
|
|
||||||
select {
|
|
||||||
case n.solicited <- peer:
|
|
||||||
default:
|
|
||||||
// don't block
|
|
||||||
}
|
|
||||||
|
|
||||||
// advertise all the routes when a new node has connected
|
|
||||||
if err := n.router.Solicit(); err != nil {
|
|
||||||
log.Debugf("Network failed to solicit routes: %s", err)
|
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
@ -1045,7 +930,68 @@ func (n *network) processNetChan(listener tunnel.Listener) {
|
|||||||
|
|
||||||
// add all the routes we have received in the sync message
|
// add all the routes we have received in the sync message
|
||||||
for _, pbRoute := range pbNetSync.Routes {
|
for _, pbRoute := range pbNetSync.Routes {
|
||||||
|
// unmarshal the routes received from remote peer
|
||||||
route := pbUtil.ProtoToRoute(pbRoute)
|
route := pbUtil.ProtoToRoute(pbRoute)
|
||||||
|
// continue if we are the originator of the route
|
||||||
|
if route.Router == n.router.Options().Id {
|
||||||
|
log.Debugf("Network node %s skipping route addition: route already present", n.id)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
metric := n.getRouteMetric(route.Router, route.Gateway, route.Link)
|
||||||
|
// check we don't overflow max int 64
|
||||||
|
if d := route.Metric + metric; d <= 0 {
|
||||||
|
// set to max int64 if we overflow
|
||||||
|
route.Metric = math.MaxInt64
|
||||||
|
} else {
|
||||||
|
// set the combined value of metrics otherwise
|
||||||
|
route.Metric = d
|
||||||
|
}
|
||||||
|
|
||||||
|
/////////////////////////////////////////////////////////////////////
|
||||||
|
// maybe we should not be this clever ¯\_(ツ)_/¯ //
|
||||||
|
/////////////////////////////////////////////////////////////////////
|
||||||
|
// lookup best routes for the services in the just received route
|
||||||
|
q := []router.QueryOption{
|
||||||
|
router.QueryService(route.Service),
|
||||||
|
router.QueryStrategy(n.router.Options().Advertise),
|
||||||
|
}
|
||||||
|
|
||||||
|
routes, err := n.router.Table().Query(q...)
|
||||||
|
if err != nil && err != router.ErrRouteNotFound {
|
||||||
|
log.Debugf("Network node %s failed listing best routes for %s: %v", n.id, route.Service, err)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
// we found no routes for the given service
|
||||||
|
// create the new route we have just received
|
||||||
|
if len(routes) == 0 {
|
||||||
|
if err := n.router.Table().Create(route); err != nil && err != router.ErrDuplicateRoute {
|
||||||
|
log.Debugf("Network node %s failed to add route: %v", n.id, err)
|
||||||
|
}
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
// find the best route for the given service
|
||||||
|
// from the routes that we would advertise
|
||||||
|
bestRoute := routes[0]
|
||||||
|
for _, r := range routes[0:] {
|
||||||
|
if bestRoute.Metric > r.Metric {
|
||||||
|
bestRoute = r
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Take the best route to given service and:
|
||||||
|
// only add new routes if the metric is better
|
||||||
|
// than the metric of our best route
|
||||||
|
|
||||||
|
if bestRoute.Metric <= route.Metric {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
///////////////////////////////////////////////////////////////////////
|
||||||
|
///////////////////////////////////////////////////////////////////////
|
||||||
|
|
||||||
|
// add route to the routing table
|
||||||
if err := n.router.Table().Create(route); err != nil && err != router.ErrDuplicateRoute {
|
if err := n.router.Table().Create(route); err != nil && err != router.ErrDuplicateRoute {
|
||||||
log.Debugf("Network node %s failed to add route: %v", n.id, err)
|
log.Debugf("Network node %s failed to add route: %v", n.id, err)
|
||||||
}
|
}
|
||||||
@ -1262,6 +1208,7 @@ func (n *network) manage() {
|
|||||||
links[peer.link] = time.Now()
|
links[peer.link] = time.Now()
|
||||||
}
|
}
|
||||||
case <-prune.C:
|
case <-prune.C:
|
||||||
|
log.Debugf("Network node %s pruning stale peers", n.id)
|
||||||
pruned := n.PruneStalePeers(PruneTime)
|
pruned := n.PruneStalePeers(PruneTime)
|
||||||
|
|
||||||
for id, peer := range pruned {
|
for id, peer := range pruned {
|
||||||
@ -1328,26 +1275,13 @@ func (n *network) manage() {
|
|||||||
Peer: node,
|
Peer: node,
|
||||||
}
|
}
|
||||||
|
|
||||||
// get a list of all of our routes
|
// get a list of the best routes for each service in our routing table
|
||||||
routes, err := n.options.Router.Table().List()
|
routes, err := n.getProtoRoutes()
|
||||||
switch err {
|
if err != nil {
|
||||||
case nil:
|
|
||||||
// encode the routes to protobuf
|
|
||||||
pbRoutes := make([]*pbRtr.Route, 0, len(routes))
|
|
||||||
for _, route := range routes {
|
|
||||||
// generate new route proto
|
|
||||||
pbRoute := pbUtil.RouteToProto(route)
|
|
||||||
// mask the route before outbounding
|
|
||||||
n.maskRoute(pbRoute)
|
|
||||||
// add to list of routes
|
|
||||||
pbRoutes = append(pbRoutes, pbRoute)
|
|
||||||
}
|
|
||||||
// pack the routes into the sync message
|
|
||||||
msg.Routes = pbRoutes
|
|
||||||
default:
|
|
||||||
// we can't list the routes
|
|
||||||
log.Debugf("Network node %s failed listing routes: %v", n.id, err)
|
log.Debugf("Network node %s failed listing routes: %v", n.id, err)
|
||||||
}
|
}
|
||||||
|
// attached the routes to the message
|
||||||
|
msg.Routes = routes
|
||||||
|
|
||||||
// send sync message to the newly connected peer
|
// send sync message to the newly connected peer
|
||||||
if err := n.sendTo("sync", NetworkChannel, peer, msg); err != nil {
|
if err := n.sendTo("sync", NetworkChannel, peer, msg); err != nil {
|
||||||
@ -1360,6 +1294,34 @@ func (n *network) manage() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// getAdvertProtoRoutes returns a list of routes to advertise to remote peer
|
||||||
|
// based on the advertisement strategy encoded in protobuf
|
||||||
|
// It returns error if the routes failed to be retrieved from the routing table
|
||||||
|
func (n *network) getProtoRoutes() ([]*pbRtr.Route, error) {
|
||||||
|
// get a list of the best routes for each service in our routing table
|
||||||
|
q := []router.QueryOption{
|
||||||
|
router.QueryStrategy(n.router.Options().Advertise),
|
||||||
|
}
|
||||||
|
|
||||||
|
routes, err := n.router.Table().Query(q...)
|
||||||
|
if err != nil && err != router.ErrRouteNotFound {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// encode the routes to protobuf
|
||||||
|
pbRoutes := make([]*pbRtr.Route, 0, len(routes))
|
||||||
|
for _, route := range routes {
|
||||||
|
// generate new route proto
|
||||||
|
pbRoute := pbUtil.RouteToProto(route)
|
||||||
|
// mask the route before outbounding
|
||||||
|
n.maskRoute(pbRoute)
|
||||||
|
// add to list of routes
|
||||||
|
pbRoutes = append(pbRoutes, pbRoute)
|
||||||
|
}
|
||||||
|
|
||||||
|
return pbRoutes, nil
|
||||||
|
}
|
||||||
|
|
||||||
func (n *network) sendConnect() {
|
func (n *network) sendConnect() {
|
||||||
// send connect message to NetworkChannel
|
// send connect message to NetworkChannel
|
||||||
// NOTE: in theory we could do this as soon as
|
// NOTE: in theory we could do this as soon as
|
||||||
@ -1390,12 +1352,11 @@ func (n *network) sendTo(method, channel string, peer *node, msg proto.Message)
|
|||||||
c, err := n.tunnel.Dial(channel, tunnel.DialWait(false), tunnel.DialLink(peer.link))
|
c, err := n.tunnel.Dial(channel, tunnel.DialWait(false), tunnel.DialLink(peer.link))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if peerNode := n.GetPeerNode(peer.id); peerNode != nil {
|
if peerNode := n.GetPeerNode(peer.id); peerNode != nil {
|
||||||
log.Debugf("Network found peer %s: %v", peer.id, peerNode)
|
|
||||||
// update node status when error happens
|
// update node status when error happens
|
||||||
peerNode.status.err.Update(err)
|
peerNode.status.err.Update(err)
|
||||||
log.Debugf("Network increment node peer %p %v count to: %d", peerNode, peerNode, peerNode.status.Error().Count())
|
log.Debugf("Network increment peer %v error count to: %d", peerNode, peerNode, peerNode.status.Error().Count())
|
||||||
if count := peerNode.status.Error().Count(); count == MaxPeerErrors {
|
if count := peerNode.status.Error().Count(); count == MaxPeerErrors {
|
||||||
log.Debugf("Network node peer %v count exceeded %d: %d", peerNode, MaxPeerErrors, peerNode.status.Error().Count())
|
log.Debugf("Network peer %v error count exceeded %d. Prunning.", peerNode, MaxPeerErrors)
|
||||||
n.PrunePeer(peerNode.id)
|
n.PrunePeer(peerNode.id)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1771,7 +1732,6 @@ func (n *network) Close() error {
|
|||||||
n.Unlock()
|
n.Unlock()
|
||||||
return nil
|
return nil
|
||||||
default:
|
default:
|
||||||
// TODO: send close message to the network channel
|
|
||||||
close(n.closed)
|
close(n.closed)
|
||||||
|
|
||||||
// set connected to false
|
// set connected to false
|
||||||
|
@ -281,6 +281,7 @@ func (p *Proxy) watchRoutes() {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
defer w.Stop()
|
||||||
|
|
||||||
for {
|
for {
|
||||||
event, err := w.Next()
|
event, err := w.Next()
|
||||||
|
@ -6,13 +6,14 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/micro/go-micro/client"
|
"github.com/micro/go-micro/client"
|
||||||
|
"github.com/micro/go-micro/client/grpc"
|
||||||
"github.com/micro/go-micro/registry"
|
"github.com/micro/go-micro/registry"
|
||||||
pb "github.com/micro/go-micro/registry/service/proto"
|
pb "github.com/micro/go-micro/registry/service/proto"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
// The default service name
|
// The default service name
|
||||||
DefaultService = "go.micro.service"
|
DefaultService = "go.micro.registry"
|
||||||
)
|
)
|
||||||
|
|
||||||
type serviceRegistry struct {
|
type serviceRegistry struct {
|
||||||
@ -128,7 +129,7 @@ func (s *serviceRegistry) Watch(opts ...registry.WatchOption) (registry.Watcher,
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (s *serviceRegistry) String() string {
|
func (s *serviceRegistry) String() string {
|
||||||
return s.name
|
return "service"
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewRegistry returns a new registry service client
|
// NewRegistry returns a new registry service client
|
||||||
@ -149,7 +150,7 @@ func NewRegistry(opts ...registry.Option) registry.Registry {
|
|||||||
mReg := registry.NewRegistry()
|
mReg := registry.NewRegistry()
|
||||||
|
|
||||||
// create new client with mdns
|
// create new client with mdns
|
||||||
cli := client.NewClient(
|
cli := grpc.NewClient(
|
||||||
client.Registry(mReg),
|
client.Registry(mReg),
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -82,6 +82,7 @@ func ToProto(s *registry.Service) *pb.Service {
|
|||||||
Metadata: s.Metadata,
|
Metadata: s.Metadata,
|
||||||
Endpoints: endpoints,
|
Endpoints: endpoints,
|
||||||
Nodes: nodes,
|
Nodes: nodes,
|
||||||
|
Options: new(pb.Options),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -88,9 +88,10 @@ func (r *router) Init(opts ...Option) error {
|
|||||||
|
|
||||||
// Options returns router options
|
// Options returns router options
|
||||||
func (r *router) Options() Options {
|
func (r *router) Options() Options {
|
||||||
r.Lock()
|
r.RLock()
|
||||||
|
defer r.RUnlock()
|
||||||
|
|
||||||
options := r.options
|
options := r.options
|
||||||
r.Unlock()
|
|
||||||
|
|
||||||
return options
|
return options
|
||||||
}
|
}
|
||||||
@ -115,9 +116,6 @@ func (r *router) manageRoute(route Route, action string) error {
|
|||||||
if err := r.table.Update(route); err != nil {
|
if err := r.table.Update(route); err != nil {
|
||||||
return fmt.Errorf("failed updating route for service %s: %s", route.Service, err)
|
return fmt.Errorf("failed updating route for service %s: %s", route.Service, err)
|
||||||
}
|
}
|
||||||
case "solicit":
|
|
||||||
// nothing to do here
|
|
||||||
return nil
|
|
||||||
default:
|
default:
|
||||||
return fmt.Errorf("failed to manage route for service %s: unknown action %s", route.Service, action)
|
return fmt.Errorf("failed to manage route for service %s: unknown action %s", route.Service, action)
|
||||||
}
|
}
|
||||||
@ -736,74 +734,24 @@ func (r *router) Process(a *Advert) error {
|
|||||||
|
|
||||||
// flushRouteEvents returns a slice of events, one per each route in the routing table
|
// flushRouteEvents returns a slice of events, one per each route in the routing table
|
||||||
func (r *router) flushRouteEvents(evType EventType) ([]*Event, error) {
|
func (r *router) flushRouteEvents(evType EventType) ([]*Event, error) {
|
||||||
// Do not advertise anything
|
// get a list of routes for each service in our routing table
|
||||||
if r.options.Advertise == AdvertiseNone {
|
// for the configured advertising strategy
|
||||||
return []*Event{}, nil
|
q := []QueryOption{
|
||||||
|
QueryStrategy(r.options.Advertise),
|
||||||
}
|
}
|
||||||
|
|
||||||
// list all routes
|
routes, err := r.Table().Query(q...)
|
||||||
routes, err := r.table.List()
|
if err != nil && err != ErrRouteNotFound {
|
||||||
if err != nil {
|
return nil, err
|
||||||
return nil, fmt.Errorf("failed listing routes: %s", err)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Return all the routes
|
log.Debugf("Router advertising %d routes with strategy %s", len(routes), r.options.Advertise)
|
||||||
if r.options.Advertise == AdvertiseAll {
|
|
||||||
// build a list of events to advertise
|
// build a list of events to advertise
|
||||||
events := make([]*Event, len(routes))
|
events := make([]*Event, len(routes))
|
||||||
for i, route := range routes {
|
|
||||||
event := &Event{
|
|
||||||
Type: evType,
|
|
||||||
Timestamp: time.Now(),
|
|
||||||
Route: route,
|
|
||||||
}
|
|
||||||
events[i] = event
|
|
||||||
}
|
|
||||||
return events, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// routeMap stores the routes we're going to advertise
|
|
||||||
bestRoutes := make(map[string]Route)
|
|
||||||
|
|
||||||
// set whether we're advertising only local
|
|
||||||
advertiseLocal := r.options.Advertise == AdvertiseLocal
|
|
||||||
|
|
||||||
// go through all routes found in the routing table and collapse them to optimal routes
|
|
||||||
for _, route := range routes {
|
|
||||||
// if we're only advertising local routes
|
|
||||||
if advertiseLocal && route.Link != "local" {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
// now we're going to find the best routes
|
|
||||||
|
|
||||||
routeKey := route.Service + "@" + route.Network
|
|
||||||
current, ok := bestRoutes[routeKey]
|
|
||||||
if !ok {
|
|
||||||
bestRoutes[routeKey] = route
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// if the current optimal route metric is higher than routing table route, replace it
|
|
||||||
if current.Metric > route.Metric {
|
|
||||||
bestRoutes[routeKey] = route
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// if the metrics are the same, prefer advertising your own route
|
|
||||||
if current.Metric == route.Metric {
|
|
||||||
if route.Router == r.options.Id {
|
|
||||||
bestRoutes[routeKey] = route
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
log.Debugf("Router advertising %d %s routes out of %d", len(bestRoutes), r.options.Advertise, len(routes))
|
|
||||||
|
|
||||||
// build a list of events to advertise
|
|
||||||
events := make([]*Event, len(bestRoutes))
|
|
||||||
var i int
|
var i int
|
||||||
|
|
||||||
for _, route := range bestRoutes {
|
for _, route := range routes {
|
||||||
event := &Event{
|
event := &Event{
|
||||||
Type: evType,
|
Type: evType,
|
||||||
Timestamp: time.Now(),
|
Timestamp: time.Now(),
|
||||||
@ -816,25 +764,6 @@ func (r *router) flushRouteEvents(evType EventType) ([]*Event, error) {
|
|||||||
return events, nil
|
return events, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Solicit advertises all of its routes to the network
|
|
||||||
// It returns error if the router fails to list the routes
|
|
||||||
func (r *router) Solicit() error {
|
|
||||||
events, err := r.flushRouteEvents(Update)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed solicit routes: %s", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
// advertise the routes
|
|
||||||
r.advertWg.Add(1)
|
|
||||||
|
|
||||||
go func() {
|
|
||||||
r.publishAdvert(Solicitation, events)
|
|
||||||
r.advertWg.Done()
|
|
||||||
}()
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// Lookup routes in the routing table
|
// Lookup routes in the routing table
|
||||||
func (r *router) Lookup(q ...QueryOption) ([]Route, error) {
|
func (r *router) Lookup(q ...QueryOption) ([]Route, error) {
|
||||||
return r.table.Query(q...)
|
return r.table.Query(q...)
|
||||||
|
@ -18,7 +18,6 @@ func routerTestSetup() Router {
|
|||||||
func TestRouterStartStop(t *testing.T) {
|
func TestRouterStartStop(t *testing.T) {
|
||||||
r := routerTestSetup()
|
r := routerTestSetup()
|
||||||
|
|
||||||
log.Debugf("TestRouterStartStop STARTING")
|
|
||||||
if err := r.Start(); err != nil {
|
if err := r.Start(); err != nil {
|
||||||
t.Errorf("failed to start router: %v", err)
|
t.Errorf("failed to start router: %v", err)
|
||||||
}
|
}
|
||||||
|
@ -28,8 +28,6 @@ type Router interface {
|
|||||||
Advertise() (<-chan *Advert, error)
|
Advertise() (<-chan *Advert, error)
|
||||||
// Process processes incoming adverts
|
// Process processes incoming adverts
|
||||||
Process(*Advert) error
|
Process(*Advert) error
|
||||||
// Solicit advertises the whole routing table
|
|
||||||
Solicit() error
|
|
||||||
// Lookup queries routes in the routing table
|
// Lookup queries routes in the routing table
|
||||||
Lookup(...QueryOption) ([]Route, error)
|
Lookup(...QueryOption) ([]Route, error)
|
||||||
// Watch returns a watcher which tracks updates to the routing table
|
// Watch returns a watcher which tracks updates to the routing table
|
||||||
@ -111,8 +109,6 @@ const (
|
|||||||
Announce AdvertType = iota
|
Announce AdvertType = iota
|
||||||
// RouteUpdate advertises route updates
|
// RouteUpdate advertises route updates
|
||||||
RouteUpdate
|
RouteUpdate
|
||||||
// Solicitation indicates routes were solicited
|
|
||||||
Solicitation
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// String returns human readable advertisement type
|
// String returns human readable advertisement type
|
||||||
@ -122,8 +118,6 @@ func (t AdvertType) String() string {
|
|||||||
return "announce"
|
return "announce"
|
||||||
case RouteUpdate:
|
case RouteUpdate:
|
||||||
return "update"
|
return "update"
|
||||||
case Solicitation:
|
|
||||||
return "solicitation"
|
|
||||||
default:
|
default:
|
||||||
return "unknown"
|
return "unknown"
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// source: github.com/micro/go-micro/router/proto/router.proto
|
// source: router.proto
|
||||||
|
|
||||||
package go_micro_router
|
package go_micro_router
|
||||||
|
|
||||||
@ -43,7 +43,7 @@ func (x AdvertType) String() string {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (AdvertType) EnumDescriptor() ([]byte, []int) {
|
func (AdvertType) EnumDescriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{0}
|
return fileDescriptor_367072455c71aedc, []int{0}
|
||||||
}
|
}
|
||||||
|
|
||||||
// EventType defines the type of event
|
// EventType defines the type of event
|
||||||
@ -72,7 +72,7 @@ func (x EventType) String() string {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (EventType) EnumDescriptor() ([]byte, []int) {
|
func (EventType) EnumDescriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{1}
|
return fileDescriptor_367072455c71aedc, []int{1}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Empty request
|
// Empty request
|
||||||
@ -86,7 +86,7 @@ func (m *Request) Reset() { *m = Request{} }
|
|||||||
func (m *Request) String() string { return proto.CompactTextString(m) }
|
func (m *Request) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Request) ProtoMessage() {}
|
func (*Request) ProtoMessage() {}
|
||||||
func (*Request) Descriptor() ([]byte, []int) {
|
func (*Request) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{0}
|
return fileDescriptor_367072455c71aedc, []int{0}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Request) XXX_Unmarshal(b []byte) error {
|
func (m *Request) XXX_Unmarshal(b []byte) error {
|
||||||
@ -118,7 +118,7 @@ func (m *Response) Reset() { *m = Response{} }
|
|||||||
func (m *Response) String() string { return proto.CompactTextString(m) }
|
func (m *Response) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Response) ProtoMessage() {}
|
func (*Response) ProtoMessage() {}
|
||||||
func (*Response) Descriptor() ([]byte, []int) {
|
func (*Response) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{1}
|
return fileDescriptor_367072455c71aedc, []int{1}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Response) XXX_Unmarshal(b []byte) error {
|
func (m *Response) XXX_Unmarshal(b []byte) error {
|
||||||
@ -151,7 +151,7 @@ func (m *ListResponse) Reset() { *m = ListResponse{} }
|
|||||||
func (m *ListResponse) String() string { return proto.CompactTextString(m) }
|
func (m *ListResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*ListResponse) ProtoMessage() {}
|
func (*ListResponse) ProtoMessage() {}
|
||||||
func (*ListResponse) Descriptor() ([]byte, []int) {
|
func (*ListResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{2}
|
return fileDescriptor_367072455c71aedc, []int{2}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *ListResponse) XXX_Unmarshal(b []byte) error {
|
func (m *ListResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -191,7 +191,7 @@ func (m *LookupRequest) Reset() { *m = LookupRequest{} }
|
|||||||
func (m *LookupRequest) String() string { return proto.CompactTextString(m) }
|
func (m *LookupRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*LookupRequest) ProtoMessage() {}
|
func (*LookupRequest) ProtoMessage() {}
|
||||||
func (*LookupRequest) Descriptor() ([]byte, []int) {
|
func (*LookupRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{3}
|
return fileDescriptor_367072455c71aedc, []int{3}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *LookupRequest) XXX_Unmarshal(b []byte) error {
|
func (m *LookupRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -231,7 +231,7 @@ func (m *LookupResponse) Reset() { *m = LookupResponse{} }
|
|||||||
func (m *LookupResponse) String() string { return proto.CompactTextString(m) }
|
func (m *LookupResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*LookupResponse) ProtoMessage() {}
|
func (*LookupResponse) ProtoMessage() {}
|
||||||
func (*LookupResponse) Descriptor() ([]byte, []int) {
|
func (*LookupResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{4}
|
return fileDescriptor_367072455c71aedc, []int{4}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *LookupResponse) XXX_Unmarshal(b []byte) error {
|
func (m *LookupResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -271,7 +271,7 @@ func (m *QueryRequest) Reset() { *m = QueryRequest{} }
|
|||||||
func (m *QueryRequest) String() string { return proto.CompactTextString(m) }
|
func (m *QueryRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*QueryRequest) ProtoMessage() {}
|
func (*QueryRequest) ProtoMessage() {}
|
||||||
func (*QueryRequest) Descriptor() ([]byte, []int) {
|
func (*QueryRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{5}
|
return fileDescriptor_367072455c71aedc, []int{5}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *QueryRequest) XXX_Unmarshal(b []byte) error {
|
func (m *QueryRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -311,7 +311,7 @@ func (m *QueryResponse) Reset() { *m = QueryResponse{} }
|
|||||||
func (m *QueryResponse) String() string { return proto.CompactTextString(m) }
|
func (m *QueryResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*QueryResponse) ProtoMessage() {}
|
func (*QueryResponse) ProtoMessage() {}
|
||||||
func (*QueryResponse) Descriptor() ([]byte, []int) {
|
func (*QueryResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{6}
|
return fileDescriptor_367072455c71aedc, []int{6}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *QueryResponse) XXX_Unmarshal(b []byte) error {
|
func (m *QueryResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -350,7 +350,7 @@ func (m *WatchRequest) Reset() { *m = WatchRequest{} }
|
|||||||
func (m *WatchRequest) String() string { return proto.CompactTextString(m) }
|
func (m *WatchRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*WatchRequest) ProtoMessage() {}
|
func (*WatchRequest) ProtoMessage() {}
|
||||||
func (*WatchRequest) Descriptor() ([]byte, []int) {
|
func (*WatchRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{7}
|
return fileDescriptor_367072455c71aedc, []int{7}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *WatchRequest) XXX_Unmarshal(b []byte) error {
|
func (m *WatchRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -392,7 +392,7 @@ func (m *Advert) Reset() { *m = Advert{} }
|
|||||||
func (m *Advert) String() string { return proto.CompactTextString(m) }
|
func (m *Advert) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Advert) ProtoMessage() {}
|
func (*Advert) ProtoMessage() {}
|
||||||
func (*Advert) Descriptor() ([]byte, []int) {
|
func (*Advert) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{8}
|
return fileDescriptor_367072455c71aedc, []int{8}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Advert) XXX_Unmarshal(b []byte) error {
|
func (m *Advert) XXX_Unmarshal(b []byte) error {
|
||||||
@ -448,47 +448,6 @@ func (m *Advert) GetEvents() []*Event {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Solicit solicits routes
|
|
||||||
type Solicit struct {
|
|
||||||
// id of the soliciting router
|
|
||||||
Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
|
|
||||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
|
||||||
XXX_unrecognized []byte `json:"-"`
|
|
||||||
XXX_sizecache int32 `json:"-"`
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *Solicit) Reset() { *m = Solicit{} }
|
|
||||||
func (m *Solicit) String() string { return proto.CompactTextString(m) }
|
|
||||||
func (*Solicit) ProtoMessage() {}
|
|
||||||
func (*Solicit) Descriptor() ([]byte, []int) {
|
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{9}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *Solicit) XXX_Unmarshal(b []byte) error {
|
|
||||||
return xxx_messageInfo_Solicit.Unmarshal(m, b)
|
|
||||||
}
|
|
||||||
func (m *Solicit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
|
||||||
return xxx_messageInfo_Solicit.Marshal(b, m, deterministic)
|
|
||||||
}
|
|
||||||
func (m *Solicit) XXX_Merge(src proto.Message) {
|
|
||||||
xxx_messageInfo_Solicit.Merge(m, src)
|
|
||||||
}
|
|
||||||
func (m *Solicit) XXX_Size() int {
|
|
||||||
return xxx_messageInfo_Solicit.Size(m)
|
|
||||||
}
|
|
||||||
func (m *Solicit) XXX_DiscardUnknown() {
|
|
||||||
xxx_messageInfo_Solicit.DiscardUnknown(m)
|
|
||||||
}
|
|
||||||
|
|
||||||
var xxx_messageInfo_Solicit proto.InternalMessageInfo
|
|
||||||
|
|
||||||
func (m *Solicit) GetId() string {
|
|
||||||
if m != nil {
|
|
||||||
return m.Id
|
|
||||||
}
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
|
|
||||||
// ProcessResponse is returned by Process
|
// ProcessResponse is returned by Process
|
||||||
type ProcessResponse struct {
|
type ProcessResponse struct {
|
||||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||||
@ -500,7 +459,7 @@ func (m *ProcessResponse) Reset() { *m = ProcessResponse{} }
|
|||||||
func (m *ProcessResponse) String() string { return proto.CompactTextString(m) }
|
func (m *ProcessResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*ProcessResponse) ProtoMessage() {}
|
func (*ProcessResponse) ProtoMessage() {}
|
||||||
func (*ProcessResponse) Descriptor() ([]byte, []int) {
|
func (*ProcessResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{10}
|
return fileDescriptor_367072455c71aedc, []int{9}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *ProcessResponse) XXX_Unmarshal(b []byte) error {
|
func (m *ProcessResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -532,7 +491,7 @@ func (m *CreateResponse) Reset() { *m = CreateResponse{} }
|
|||||||
func (m *CreateResponse) String() string { return proto.CompactTextString(m) }
|
func (m *CreateResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*CreateResponse) ProtoMessage() {}
|
func (*CreateResponse) ProtoMessage() {}
|
||||||
func (*CreateResponse) Descriptor() ([]byte, []int) {
|
func (*CreateResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{11}
|
return fileDescriptor_367072455c71aedc, []int{10}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *CreateResponse) XXX_Unmarshal(b []byte) error {
|
func (m *CreateResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -564,7 +523,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} }
|
|||||||
func (m *DeleteResponse) String() string { return proto.CompactTextString(m) }
|
func (m *DeleteResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*DeleteResponse) ProtoMessage() {}
|
func (*DeleteResponse) ProtoMessage() {}
|
||||||
func (*DeleteResponse) Descriptor() ([]byte, []int) {
|
func (*DeleteResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{12}
|
return fileDescriptor_367072455c71aedc, []int{11}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *DeleteResponse) XXX_Unmarshal(b []byte) error {
|
func (m *DeleteResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -596,7 +555,7 @@ func (m *UpdateResponse) Reset() { *m = UpdateResponse{} }
|
|||||||
func (m *UpdateResponse) String() string { return proto.CompactTextString(m) }
|
func (m *UpdateResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*UpdateResponse) ProtoMessage() {}
|
func (*UpdateResponse) ProtoMessage() {}
|
||||||
func (*UpdateResponse) Descriptor() ([]byte, []int) {
|
func (*UpdateResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{13}
|
return fileDescriptor_367072455c71aedc, []int{12}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *UpdateResponse) XXX_Unmarshal(b []byte) error {
|
func (m *UpdateResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -634,7 +593,7 @@ func (m *Event) Reset() { *m = Event{} }
|
|||||||
func (m *Event) String() string { return proto.CompactTextString(m) }
|
func (m *Event) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Event) ProtoMessage() {}
|
func (*Event) ProtoMessage() {}
|
||||||
func (*Event) Descriptor() ([]byte, []int) {
|
func (*Event) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{14}
|
return fileDescriptor_367072455c71aedc, []int{13}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Event) XXX_Unmarshal(b []byte) error {
|
func (m *Event) XXX_Unmarshal(b []byte) error {
|
||||||
@ -693,7 +652,7 @@ func (m *Query) Reset() { *m = Query{} }
|
|||||||
func (m *Query) String() string { return proto.CompactTextString(m) }
|
func (m *Query) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Query) ProtoMessage() {}
|
func (*Query) ProtoMessage() {}
|
||||||
func (*Query) Descriptor() ([]byte, []int) {
|
func (*Query) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{15}
|
return fileDescriptor_367072455c71aedc, []int{14}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Query) XXX_Unmarshal(b []byte) error {
|
func (m *Query) XXX_Unmarshal(b []byte) error {
|
||||||
@ -760,7 +719,7 @@ func (m *Route) Reset() { *m = Route{} }
|
|||||||
func (m *Route) String() string { return proto.CompactTextString(m) }
|
func (m *Route) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Route) ProtoMessage() {}
|
func (*Route) ProtoMessage() {}
|
||||||
func (*Route) Descriptor() ([]byte, []int) {
|
func (*Route) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{16}
|
return fileDescriptor_367072455c71aedc, []int{15}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Route) XXX_Unmarshal(b []byte) error {
|
func (m *Route) XXX_Unmarshal(b []byte) error {
|
||||||
@ -842,7 +801,7 @@ func (m *Status) Reset() { *m = Status{} }
|
|||||||
func (m *Status) String() string { return proto.CompactTextString(m) }
|
func (m *Status) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Status) ProtoMessage() {}
|
func (*Status) ProtoMessage() {}
|
||||||
func (*Status) Descriptor() ([]byte, []int) {
|
func (*Status) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{17}
|
return fileDescriptor_367072455c71aedc, []int{16}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Status) XXX_Unmarshal(b []byte) error {
|
func (m *Status) XXX_Unmarshal(b []byte) error {
|
||||||
@ -888,7 +847,7 @@ func (m *StatusResponse) Reset() { *m = StatusResponse{} }
|
|||||||
func (m *StatusResponse) String() string { return proto.CompactTextString(m) }
|
func (m *StatusResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*StatusResponse) ProtoMessage() {}
|
func (*StatusResponse) ProtoMessage() {}
|
||||||
func (*StatusResponse) Descriptor() ([]byte, []int) {
|
func (*StatusResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_2dd64c6ec344e37e, []int{18}
|
return fileDescriptor_367072455c71aedc, []int{17}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *StatusResponse) XXX_Unmarshal(b []byte) error {
|
func (m *StatusResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -928,7 +887,6 @@ func init() {
|
|||||||
proto.RegisterType((*QueryResponse)(nil), "go.micro.router.QueryResponse")
|
proto.RegisterType((*QueryResponse)(nil), "go.micro.router.QueryResponse")
|
||||||
proto.RegisterType((*WatchRequest)(nil), "go.micro.router.WatchRequest")
|
proto.RegisterType((*WatchRequest)(nil), "go.micro.router.WatchRequest")
|
||||||
proto.RegisterType((*Advert)(nil), "go.micro.router.Advert")
|
proto.RegisterType((*Advert)(nil), "go.micro.router.Advert")
|
||||||
proto.RegisterType((*Solicit)(nil), "go.micro.router.Solicit")
|
|
||||||
proto.RegisterType((*ProcessResponse)(nil), "go.micro.router.ProcessResponse")
|
proto.RegisterType((*ProcessResponse)(nil), "go.micro.router.ProcessResponse")
|
||||||
proto.RegisterType((*CreateResponse)(nil), "go.micro.router.CreateResponse")
|
proto.RegisterType((*CreateResponse)(nil), "go.micro.router.CreateResponse")
|
||||||
proto.RegisterType((*DeleteResponse)(nil), "go.micro.router.DeleteResponse")
|
proto.RegisterType((*DeleteResponse)(nil), "go.micro.router.DeleteResponse")
|
||||||
@ -940,56 +898,52 @@ func init() {
|
|||||||
proto.RegisterType((*StatusResponse)(nil), "go.micro.router.StatusResponse")
|
proto.RegisterType((*StatusResponse)(nil), "go.micro.router.StatusResponse")
|
||||||
}
|
}
|
||||||
|
|
||||||
func init() {
|
func init() { proto.RegisterFile("router.proto", fileDescriptor_367072455c71aedc) }
|
||||||
proto.RegisterFile("github.com/micro/go-micro/router/proto/router.proto", fileDescriptor_2dd64c6ec344e37e)
|
|
||||||
}
|
|
||||||
|
|
||||||
var fileDescriptor_2dd64c6ec344e37e = []byte{
|
var fileDescriptor_367072455c71aedc = []byte{
|
||||||
// 736 bytes of a gzipped FileDescriptorProto
|
// 693 bytes of a gzipped FileDescriptorProto
|
||||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0x4f, 0x4f, 0xdb, 0x4a,
|
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0x4f, 0x4f, 0xdb, 0x4a,
|
||||||
0x10, 0xb7, 0x93, 0xd8, 0x79, 0x99, 0x17, 0x42, 0xde, 0xe8, 0x09, 0x4c, 0xde, 0x03, 0x22, 0x9f,
|
0x10, 0xb7, 0x93, 0xd8, 0x79, 0x99, 0x17, 0x8c, 0xdf, 0xe8, 0x09, 0xac, 0xb4, 0x40, 0xe4, 0x13,
|
||||||
0x10, 0xa2, 0x4e, 0x15, 0xae, 0xfd, 0x43, 0xa0, 0x54, 0x95, 0xca, 0xa1, 0x35, 0xa0, 0x9e, 0x8d,
|
0x42, 0xc8, 0x54, 0xe9, 0xb5, 0xff, 0x02, 0xa5, 0xaa, 0x54, 0x0e, 0xad, 0x0b, 0xea, 0xd9, 0xd8,
|
||||||
0xb3, 0x0a, 0x16, 0x49, 0xd6, 0xec, 0xae, 0x41, 0x39, 0xf7, 0xd3, 0xf4, 0xd2, 0x4b, 0x3f, 0x52,
|
0x23, 0x6a, 0x91, 0xd8, 0x66, 0x77, 0x03, 0xca, 0xb9, 0x9f, 0xa6, 0xe7, 0x7e, 0xa4, 0x5e, 0xfb,
|
||||||
0xbf, 0x48, 0xe5, 0xdd, 0x75, 0x08, 0x71, 0x16, 0x09, 0x4e, 0xd9, 0xf9, 0xf7, 0x9b, 0x99, 0xdd,
|
0x21, 0x2a, 0xef, 0xae, 0x43, 0x88, 0x31, 0x12, 0x9c, 0xbc, 0xf3, 0xef, 0x37, 0xb3, 0x3b, 0xbf,
|
||||||
0xdf, 0x8c, 0x03, 0x87, 0xa3, 0x44, 0x5c, 0x67, 0x57, 0x41, 0x4c, 0x27, 0xbd, 0x49, 0x12, 0x33,
|
0x19, 0x43, 0x9f, 0xe5, 0x33, 0x41, 0x2c, 0x28, 0x58, 0x2e, 0x72, 0x5c, 0xbf, 0xc8, 0x83, 0x69,
|
||||||
0xda, 0x1b, 0xd1, 0x57, 0xea, 0xc0, 0x68, 0x26, 0x08, 0xeb, 0xa5, 0x8c, 0x8a, 0x42, 0x08, 0xa4,
|
0x1a, 0xb3, 0x3c, 0x50, 0x6a, 0xbf, 0x07, 0xdd, 0x90, 0xae, 0x66, 0xc4, 0x85, 0x0f, 0xf0, 0x4f,
|
||||||
0x80, 0xeb, 0x23, 0x1a, 0x48, 0x9f, 0x40, 0xa9, 0xfd, 0x06, 0xd4, 0x43, 0x72, 0x9b, 0x11, 0x2e,
|
0x48, 0xbc, 0xc8, 0x33, 0x4e, 0xfe, 0x1b, 0xe8, 0x9f, 0xa4, 0x5c, 0x54, 0x32, 0x06, 0x60, 0xcb,
|
||||||
0x7c, 0x80, 0xbf, 0x42, 0xc2, 0x53, 0x3a, 0xe5, 0xc4, 0x7f, 0x07, 0xcd, 0xb3, 0x84, 0x8b, 0x42,
|
0x00, 0xee, 0x99, 0xc3, 0xf6, 0xee, 0xbf, 0xa3, 0x8d, 0x60, 0x05, 0x28, 0x08, 0xcb, 0x4f, 0xa8,
|
||||||
0xc6, 0x00, 0x5c, 0x19, 0xc0, 0x3d, 0xbb, 0x5b, 0xdd, 0xfb, 0xbb, 0xbf, 0x11, 0x2c, 0x01, 0x05,
|
0xbd, 0xfc, 0xd7, 0xb0, 0x76, 0x92, 0xe7, 0x97, 0xb3, 0x42, 0x83, 0xe3, 0x3e, 0x58, 0x57, 0x33,
|
||||||
0x61, 0xfe, 0x13, 0x6a, 0x2f, 0xff, 0x2d, 0xac, 0x9d, 0x51, 0x7a, 0x93, 0xa5, 0x1a, 0x1c, 0x0f,
|
0x62, 0x73, 0xcf, 0x1c, 0x9a, 0xf7, 0xc6, 0x7f, 0x29, 0xad, 0xa1, 0x72, 0xf2, 0xdf, 0x81, 0x53,
|
||||||
0xc0, 0xb9, 0xcd, 0x08, 0x9b, 0x79, 0x76, 0xd7, 0x5e, 0x19, 0xff, 0x35, 0xb7, 0x86, 0xca, 0xc9,
|
0x85, 0x3f, 0xb1, 0x80, 0x57, 0xd0, 0x57, 0x88, 0x4f, 0xca, 0xff, 0x16, 0xd6, 0x74, 0xf4, 0x13,
|
||||||
0x3f, 0x82, 0x56, 0x11, 0xfe, 0xc2, 0x02, 0xde, 0x40, 0x53, 0x21, 0xbe, 0x28, 0xff, 0x7b, 0x58,
|
0xd3, 0x3b, 0xd0, 0xff, 0x16, 0x89, 0xf8, 0x7b, 0xf5, 0xb6, 0x3f, 0x4d, 0xb0, 0xc7, 0xc9, 0x35,
|
||||||
0xd3, 0xd1, 0x2f, 0x4c, 0xdf, 0x82, 0xe6, 0xb7, 0x48, 0xc4, 0xd7, 0xc5, 0xdd, 0xfe, 0xb0, 0xc1,
|
0x31, 0x81, 0x0e, 0xb4, 0xd2, 0x44, 0x96, 0xd1, 0x0b, 0x5b, 0x69, 0x82, 0x07, 0xd0, 0x11, 0xf3,
|
||||||
0x1d, 0x0c, 0xef, 0x08, 0x13, 0xd8, 0x82, 0x4a, 0x32, 0x94, 0x65, 0x34, 0xc2, 0x4a, 0x32, 0xc4,
|
0x82, 0xbc, 0xd6, 0xd0, 0xdc, 0x75, 0x46, 0xcf, 0x6a, 0xc0, 0x2a, 0xec, 0x74, 0x5e, 0x50, 0x28,
|
||||||
0x1e, 0xd4, 0xc4, 0x2c, 0x25, 0x5e, 0xa5, 0x6b, 0xef, 0xb5, 0xfa, 0xff, 0x95, 0x80, 0x55, 0xd8,
|
0x1d, 0xf1, 0x39, 0xf4, 0x44, 0x3a, 0x25, 0x2e, 0xa2, 0x69, 0xe1, 0xb5, 0x87, 0xe6, 0x6e, 0x3b,
|
||||||
0xc5, 0x2c, 0x25, 0xa1, 0x74, 0xc4, 0xff, 0xa1, 0x21, 0x92, 0x09, 0xe1, 0x22, 0x9a, 0xa4, 0x5e,
|
0xbc, 0x55, 0xa0, 0x0b, 0x6d, 0x21, 0x26, 0x5e, 0x47, 0xea, 0xcb, 0x63, 0x59, 0x3b, 0x5d, 0x53,
|
||||||
0xb5, 0x6b, 0xef, 0x55, 0xc3, 0x07, 0x05, 0xb6, 0xa1, 0x2a, 0xc4, 0xd8, 0xab, 0x49, 0x7d, 0x7e,
|
0x26, 0xb8, 0x67, 0x35, 0xd4, 0x7e, 0x5c, 0x9a, 0x43, 0xed, 0xe5, 0xff, 0x07, 0xeb, 0x9f, 0x59,
|
||||||
0xcc, 0x6b, 0x27, 0x77, 0x64, 0x2a, 0xb8, 0xe7, 0x18, 0x6a, 0x3f, 0xcd, 0xcd, 0xa1, 0xf6, 0xf2,
|
0x1e, 0x13, 0xe7, 0x0b, 0x3a, 0xb8, 0xe0, 0x1c, 0x31, 0x8a, 0x04, 0x2d, 0x6b, 0xde, 0xd3, 0x84,
|
||||||
0xb7, 0xa0, 0x7e, 0x4e, 0xc7, 0x49, 0x9c, 0x94, 0x6a, 0xf5, 0xff, 0x81, 0xf5, 0x2f, 0x8c, 0xc6,
|
0xee, 0x6a, 0xce, 0x8a, 0x64, 0xd9, 0xe7, 0x87, 0x09, 0x96, 0x84, 0xc6, 0x40, 0xdf, 0xd1, 0x94,
|
||||||
0x84, 0xf3, 0x39, 0x53, 0xda, 0xd0, 0x3a, 0x61, 0x24, 0x12, 0x64, 0x51, 0xf3, 0x81, 0x8c, 0xc9,
|
0x77, 0x1c, 0xdc, 0x5f, 0x40, 0xd3, 0x15, 0x5b, 0xab, 0x57, 0xdc, 0x07, 0x4b, 0xc6, 0xc9, 0xcb,
|
||||||
0x63, 0xcd, 0x65, 0x3a, 0x5c, 0xf4, 0xf9, 0x6e, 0x83, 0x23, 0xb3, 0x62, 0xa0, 0xdb, 0xb7, 0x65,
|
0x37, 0xf7, 0x42, 0x39, 0xf9, 0x67, 0x60, 0xc9, 0x5e, 0xa2, 0x07, 0x5d, 0x4e, 0xec, 0x3a, 0x8d,
|
||||||
0xfb, 0x9d, 0xd5, 0xb5, 0x99, 0xba, 0xaf, 0x2c, 0x77, 0x7f, 0x00, 0x8e, 0x8c, 0x93, 0xf7, 0x62,
|
0x49, 0xbf, 0x7e, 0x25, 0x96, 0x96, 0x8b, 0x48, 0xd0, 0x4d, 0x34, 0x97, 0xc9, 0x7a, 0x61, 0x25,
|
||||||
0x7e, 0x26, 0xe5, 0xe4, 0x5f, 0x82, 0x23, 0x9f, 0x19, 0x3d, 0xa8, 0x73, 0xc2, 0xee, 0x92, 0x98,
|
0x96, 0x96, 0x8c, 0xc4, 0x4d, 0xce, 0x2e, 0x65, 0xb2, 0x5e, 0x58, 0x89, 0xfe, 0x2f, 0x13, 0x2c,
|
||||||
0xe8, 0x66, 0x0b, 0x31, 0xb7, 0x8c, 0x22, 0x41, 0xee, 0xa3, 0x99, 0x4c, 0xd6, 0x08, 0x0b, 0x31,
|
0x99, 0xe7, 0x61, 0xdc, 0x28, 0x49, 0x18, 0x71, 0x5e, 0xe1, 0x6a, 0x71, 0x39, 0x63, 0xbb, 0x31,
|
||||||
0xb7, 0x4c, 0x89, 0xb8, 0xa7, 0xec, 0x46, 0x26, 0x6b, 0x84, 0x85, 0xe8, 0xff, 0xb2, 0xc1, 0x91,
|
0x63, 0xe7, 0x4e, 0x46, 0xdc, 0xd0, 0x1c, 0x64, 0x9e, 0x25, 0x0d, 0x5a, 0x42, 0x84, 0xce, 0x24,
|
||||||
0x79, 0x9e, 0xc6, 0x8d, 0x86, 0x43, 0x46, 0x38, 0x2f, 0x70, 0xb5, 0xb8, 0x98, 0xb1, 0x6a, 0xcc,
|
0xcd, 0x2e, 0x3d, 0x5b, 0x6a, 0xe5, 0xb9, 0xf4, 0x9d, 0x92, 0x60, 0x69, 0xec, 0x75, 0xe5, 0xeb,
|
||||||
0x58, 0x7b, 0x94, 0x11, 0x37, 0x34, 0x3d, 0x99, 0xe7, 0x48, 0x83, 0x96, 0x10, 0xa1, 0x36, 0x4e,
|
0x69, 0xc9, 0x1f, 0x81, 0xfd, 0x55, 0x44, 0x62, 0xc6, 0xcb, 0xa8, 0x38, 0x4f, 0xaa, 0x92, 0xe5,
|
||||||
0xa6, 0x37, 0x9e, 0x2b, 0xb5, 0xf2, 0x9c, 0xfb, 0x4e, 0x88, 0x60, 0x49, 0xec, 0xd5, 0xe5, 0xed,
|
0x19, 0xff, 0x07, 0x8b, 0x18, 0xcb, 0x99, 0xae, 0x56, 0x09, 0xfe, 0x18, 0x1c, 0x15, 0xb3, 0x98,
|
||||||
0x69, 0xc9, 0xef, 0x83, 0x7b, 0x2e, 0x22, 0x91, 0xf1, 0x3c, 0x2a, 0xa6, 0xc3, 0xa2, 0x64, 0x79,
|
0x86, 0x03, 0xb0, 0xb9, 0xd4, 0xe8, 0x69, 0xda, 0xac, 0x75, 0x40, 0x07, 0x68, 0xb7, 0xbd, 0x11,
|
||||||
0xc6, 0x7f, 0xc1, 0x21, 0x8c, 0x51, 0xa6, 0xab, 0x55, 0x82, 0x3f, 0x80, 0x96, 0x8a, 0x99, 0x0f,
|
0xc0, 0x2d, 0x8d, 0x11, 0xc1, 0x51, 0xd2, 0x38, 0xcb, 0xf2, 0x59, 0x16, 0x93, 0x6b, 0xa0, 0x0b,
|
||||||
0x4a, 0x0f, 0x5c, 0x2e, 0x35, 0x7a, 0xd0, 0x36, 0x4b, 0x2f, 0xa0, 0x03, 0xb4, 0xdb, 0x7e, 0x1f,
|
0x7d, 0xa5, 0x53, 0x1c, 0x72, 0xcd, 0xbd, 0x03, 0xe8, 0x2d, 0x68, 0x81, 0x00, 0xb6, 0x22, 0xa0,
|
||||||
0xe0, 0x81, 0xe1, 0x88, 0xd0, 0x52, 0xd2, 0x60, 0x3a, 0xa5, 0xd9, 0x34, 0x26, 0x6d, 0x0b, 0xdb,
|
0x6b, 0x94, 0x67, 0x45, 0x3d, 0xd7, 0x2c, 0xcf, 0x3a, 0xa0, 0x35, 0xfa, 0xd3, 0x02, 0x3b, 0x54,
|
||||||
0xd0, 0x54, 0x3a, 0xc5, 0xa1, 0xb6, 0xbd, 0xdf, 0x83, 0xc6, 0x9c, 0x16, 0x08, 0xe0, 0x2a, 0x02,
|
0x4f, 0xf2, 0x09, 0x6c, 0xb5, 0x3f, 0x70, 0xbb, 0x56, 0xda, 0x9d, 0xbd, 0x34, 0xd8, 0x69, 0xb4,
|
||||||
0xb6, 0xad, 0xfc, 0xac, 0xa8, 0xd7, 0xb6, 0xf3, 0xb3, 0x0e, 0xa8, 0xf4, 0x7f, 0x56, 0xc1, 0x0d,
|
0x6b, 0x12, 0x1b, 0x78, 0x08, 0x96, 0x9c, 0x65, 0xdc, 0xaa, 0xf9, 0x2e, 0xcf, 0xf8, 0xa0, 0x61,
|
||||||
0xd5, 0x95, 0x7c, 0x06, 0x57, 0xad, 0x16, 0xdc, 0x29, 0x95, 0xf6, 0x68, 0x65, 0x75, 0x76, 0x8d,
|
0xae, 0x7c, 0xe3, 0x85, 0x89, 0x87, 0xd0, 0x53, 0xd7, 0x4b, 0x39, 0xa1, 0x57, 0x27, 0xac, 0x86,
|
||||||
0x76, 0x4d, 0x62, 0x0b, 0x8f, 0xc1, 0x91, 0x63, 0x8e, 0xdb, 0x25, 0xdf, 0xc5, 0xf1, 0xef, 0x18,
|
0xd8, 0x6c, 0x98, 0x7e, 0x89, 0xf1, 0x01, 0xba, 0x7a, 0x2e, 0xb1, 0xc9, 0x6f, 0x30, 0xac, 0x19,
|
||||||
0x46, 0xce, 0xb7, 0x5e, 0xdb, 0x78, 0x0c, 0x0d, 0xd5, 0x5e, 0xc2, 0x09, 0x7a, 0x65, 0xc2, 0x6a,
|
0x56, 0x47, 0xd9, 0xc0, 0xe3, 0x05, 0x07, 0x9a, 0x0b, 0xd9, 0x69, 0xea, 0xe8, 0x02, 0x66, 0xf4,
|
||||||
0x88, 0x4d, 0xc3, 0x62, 0x90, 0x18, 0x47, 0x0f, 0x23, 0x6b, 0x46, 0xd8, 0x5a, 0x61, 0x99, 0x77,
|
0xbb, 0x05, 0xd6, 0x69, 0x74, 0x3e, 0x21, 0x3c, 0xaa, 0x9a, 0x83, 0x0d, 0xa3, 0x78, 0x0f, 0xdc,
|
||||||
0xf2, 0x11, 0xea, 0x7a, 0xb2, 0xd1, 0x94, 0xa9, 0xd3, 0x2d, 0x19, 0x96, 0x97, 0x81, 0x85, 0xa7,
|
0xca, 0x3a, 0x31, 0x4a, 0x10, 0xd5, 0xd5, 0x47, 0x80, 0xac, 0x6c, 0x20, 0x09, 0xa2, 0xe8, 0xf0,
|
||||||
0x73, 0x16, 0x99, 0x0b, 0xd9, 0x35, 0x71, 0x62, 0x0e, 0xd3, 0xff, 0x5d, 0x01, 0xe7, 0x22, 0xba,
|
0x08, 0x90, 0x95, 0xa5, 0x65, 0xe0, 0x18, 0x3a, 0xe5, 0xbf, 0xef, 0x81, 0xd7, 0xa9, 0x13, 0x61,
|
||||||
0x1a, 0x13, 0x3c, 0x29, 0x9e, 0x17, 0x0d, 0xc3, 0xbc, 0x02, 0x6e, 0x69, 0x21, 0x59, 0x39, 0x88,
|
0xf9, 0x67, 0xe9, 0x1b, 0xf8, 0xb1, 0xda, 0x39, 0x5b, 0x0d, 0xff, 0x19, 0x0d, 0xb4, 0xdd, 0x64,
|
||||||
0xe2, 0xc5, 0x33, 0x40, 0x96, 0x76, 0x98, 0x04, 0x51, 0x84, 0x7a, 0x06, 0xc8, 0xd2, 0xda, 0xb3,
|
0xae, 0x90, 0xce, 0x6d, 0xf9, 0xdf, 0x7e, 0xf9, 0x37, 0x00, 0x00, 0xff, 0xff, 0x86, 0x75, 0x28,
|
||||||
0x70, 0x00, 0xb5, 0xfc, 0xc3, 0xfa, 0xc4, 0xed, 0x94, 0xa9, 0xb4, 0xf8, 0x25, 0xf6, 0x2d, 0xfc,
|
0x0b, 0xc7, 0x07, 0x00, 0x00,
|
||||||
0x54, 0x6c, 0xad, 0x6d, 0xc3, 0x47, 0x4c, 0x03, 0xed, 0x98, 0xcc, 0x05, 0xd2, 0x95, 0x2b, 0xff,
|
|
||||||
0x14, 0x1c, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x47, 0x98, 0xd8, 0x20, 0x4b, 0x08, 0x00, 0x00,
|
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
// Code generated by protoc-gen-micro. DO NOT EDIT.
|
// Code generated by protoc-gen-micro. DO NOT EDIT.
|
||||||
// source: github.com/micro/go-micro/router/proto/router.proto
|
// source: router.proto
|
||||||
|
|
||||||
package go_micro_router
|
package go_micro_router
|
||||||
|
|
||||||
@ -37,7 +37,6 @@ type RouterService interface {
|
|||||||
Lookup(ctx context.Context, in *LookupRequest, opts ...client.CallOption) (*LookupResponse, error)
|
Lookup(ctx context.Context, in *LookupRequest, opts ...client.CallOption) (*LookupResponse, error)
|
||||||
Watch(ctx context.Context, in *WatchRequest, opts ...client.CallOption) (Router_WatchService, error)
|
Watch(ctx context.Context, in *WatchRequest, opts ...client.CallOption) (Router_WatchService, error)
|
||||||
Advertise(ctx context.Context, in *Request, opts ...client.CallOption) (Router_AdvertiseService, error)
|
Advertise(ctx context.Context, in *Request, opts ...client.CallOption) (Router_AdvertiseService, error)
|
||||||
Solicit(ctx context.Context, in *Request, opts ...client.CallOption) (*Response, error)
|
|
||||||
Process(ctx context.Context, in *Advert, opts ...client.CallOption) (*ProcessResponse, error)
|
Process(ctx context.Context, in *Advert, opts ...client.CallOption) (*ProcessResponse, error)
|
||||||
Status(ctx context.Context, in *Request, opts ...client.CallOption) (*StatusResponse, error)
|
Status(ctx context.Context, in *Request, opts ...client.CallOption) (*StatusResponse, error)
|
||||||
}
|
}
|
||||||
@ -158,16 +157,6 @@ func (x *routerServiceAdvertise) Recv() (*Advert, error) {
|
|||||||
return m, nil
|
return m, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *routerService) Solicit(ctx context.Context, in *Request, opts ...client.CallOption) (*Response, error) {
|
|
||||||
req := c.c.NewRequest(c.name, "Router.Solicit", in)
|
|
||||||
out := new(Response)
|
|
||||||
err := c.c.Call(ctx, req, out, opts...)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
return out, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (c *routerService) Process(ctx context.Context, in *Advert, opts ...client.CallOption) (*ProcessResponse, error) {
|
func (c *routerService) Process(ctx context.Context, in *Advert, opts ...client.CallOption) (*ProcessResponse, error) {
|
||||||
req := c.c.NewRequest(c.name, "Router.Process", in)
|
req := c.c.NewRequest(c.name, "Router.Process", in)
|
||||||
out := new(ProcessResponse)
|
out := new(ProcessResponse)
|
||||||
@ -194,7 +183,6 @@ type RouterHandler interface {
|
|||||||
Lookup(context.Context, *LookupRequest, *LookupResponse) error
|
Lookup(context.Context, *LookupRequest, *LookupResponse) error
|
||||||
Watch(context.Context, *WatchRequest, Router_WatchStream) error
|
Watch(context.Context, *WatchRequest, Router_WatchStream) error
|
||||||
Advertise(context.Context, *Request, Router_AdvertiseStream) error
|
Advertise(context.Context, *Request, Router_AdvertiseStream) error
|
||||||
Solicit(context.Context, *Request, *Response) error
|
|
||||||
Process(context.Context, *Advert, *ProcessResponse) error
|
Process(context.Context, *Advert, *ProcessResponse) error
|
||||||
Status(context.Context, *Request, *StatusResponse) error
|
Status(context.Context, *Request, *StatusResponse) error
|
||||||
}
|
}
|
||||||
@ -204,7 +192,6 @@ func RegisterRouterHandler(s server.Server, hdlr RouterHandler, opts ...server.H
|
|||||||
Lookup(ctx context.Context, in *LookupRequest, out *LookupResponse) error
|
Lookup(ctx context.Context, in *LookupRequest, out *LookupResponse) error
|
||||||
Watch(ctx context.Context, stream server.Stream) error
|
Watch(ctx context.Context, stream server.Stream) error
|
||||||
Advertise(ctx context.Context, stream server.Stream) error
|
Advertise(ctx context.Context, stream server.Stream) error
|
||||||
Solicit(ctx context.Context, in *Request, out *Response) error
|
|
||||||
Process(ctx context.Context, in *Advert, out *ProcessResponse) error
|
Process(ctx context.Context, in *Advert, out *ProcessResponse) error
|
||||||
Status(ctx context.Context, in *Request, out *StatusResponse) error
|
Status(ctx context.Context, in *Request, out *StatusResponse) error
|
||||||
}
|
}
|
||||||
@ -293,10 +280,6 @@ func (x *routerAdvertiseStream) Send(m *Advert) error {
|
|||||||
return x.stream.Send(m)
|
return x.stream.Send(m)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *routerHandler) Solicit(ctx context.Context, in *Request, out *Response) error {
|
|
||||||
return h.RouterHandler.Solicit(ctx, in, out)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *routerHandler) Process(ctx context.Context, in *Advert, out *ProcessResponse) error {
|
func (h *routerHandler) Process(ctx context.Context, in *Advert, out *ProcessResponse) error {
|
||||||
return h.RouterHandler.Process(ctx, in, out)
|
return h.RouterHandler.Process(ctx, in, out)
|
||||||
}
|
}
|
||||||
|
@ -7,7 +7,6 @@ service Router {
|
|||||||
rpc Lookup(LookupRequest) returns (LookupResponse) {};
|
rpc Lookup(LookupRequest) returns (LookupResponse) {};
|
||||||
rpc Watch(WatchRequest) returns (stream Event) {};
|
rpc Watch(WatchRequest) returns (stream Event) {};
|
||||||
rpc Advertise(Request) returns (stream Advert) {};
|
rpc Advertise(Request) returns (stream Advert) {};
|
||||||
rpc Solicit(Request) returns (Response) {};
|
|
||||||
rpc Process(Advert) returns (ProcessResponse) {};
|
rpc Process(Advert) returns (ProcessResponse) {};
|
||||||
rpc Status(Request) returns (StatusResponse) {};
|
rpc Status(Request) returns (StatusResponse) {};
|
||||||
}
|
}
|
||||||
@ -74,12 +73,6 @@ message Advert {
|
|||||||
repeated Event events = 5;
|
repeated Event events = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Solicit solicits routes
|
|
||||||
message Solicit {
|
|
||||||
// id of the soliciting router
|
|
||||||
string id = 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
// ProcessResponse is returned by Process
|
// ProcessResponse is returned by Process
|
||||||
message ProcessResponse {}
|
message ProcessResponse {}
|
||||||
|
|
||||||
|
@ -220,42 +220,6 @@ func (s *svc) Process(advert *router.Advert) error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Solicit advertise all routes
|
|
||||||
func (s *svc) Solicit() error {
|
|
||||||
// list all the routes
|
|
||||||
routes, err := s.table.List()
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// build events to advertise
|
|
||||||
events := make([]*router.Event, len(routes))
|
|
||||||
for i := range events {
|
|
||||||
events[i] = &router.Event{
|
|
||||||
Type: router.Update,
|
|
||||||
Timestamp: time.Now(),
|
|
||||||
Route: routes[i],
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
advert := &router.Advert{
|
|
||||||
Id: s.opts.Id,
|
|
||||||
Type: router.RouteUpdate,
|
|
||||||
Timestamp: time.Now(),
|
|
||||||
TTL: time.Duration(router.DefaultAdvertTTL),
|
|
||||||
Events: events,
|
|
||||||
}
|
|
||||||
|
|
||||||
select {
|
|
||||||
case s.advertChan <- advert:
|
|
||||||
case <-s.exit:
|
|
||||||
close(s.advertChan)
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// Status returns router status
|
// Status returns router status
|
||||||
func (s *svc) Status() router.Status {
|
func (s *svc) Status() router.Status {
|
||||||
s.Lock()
|
s.Lock()
|
||||||
|
@ -14,6 +14,7 @@ type watcher struct {
|
|||||||
opts router.WatchOptions
|
opts router.WatchOptions
|
||||||
resChan chan *router.Event
|
resChan chan *router.Event
|
||||||
done chan struct{}
|
done chan struct{}
|
||||||
|
stream pb.Router_WatchService
|
||||||
}
|
}
|
||||||
|
|
||||||
func newWatcher(rsp pb.Router_WatchService, opts router.WatchOptions) (*watcher, error) {
|
func newWatcher(rsp pb.Router_WatchService, opts router.WatchOptions) (*watcher, error) {
|
||||||
@ -21,6 +22,7 @@ func newWatcher(rsp pb.Router_WatchService, opts router.WatchOptions) (*watcher,
|
|||||||
opts: opts,
|
opts: opts,
|
||||||
resChan: make(chan *router.Event),
|
resChan: make(chan *router.Event),
|
||||||
done: make(chan struct{}),
|
done: make(chan struct{}),
|
||||||
|
stream: rsp,
|
||||||
}
|
}
|
||||||
|
|
||||||
go func() {
|
go func() {
|
||||||
@ -42,8 +44,6 @@ func newWatcher(rsp pb.Router_WatchService, opts router.WatchOptions) (*watcher,
|
|||||||
|
|
||||||
// watchRouter watches router and send events to all registered watchers
|
// watchRouter watches router and send events to all registered watchers
|
||||||
func (w *watcher) watch(stream pb.Router_WatchService) error {
|
func (w *watcher) watch(stream pb.Router_WatchService) error {
|
||||||
defer stream.Close()
|
|
||||||
|
|
||||||
var watchErr error
|
var watchErr error
|
||||||
|
|
||||||
for {
|
for {
|
||||||
@ -110,6 +110,7 @@ func (w *watcher) Stop() {
|
|||||||
case <-w.done:
|
case <-w.done:
|
||||||
return
|
return
|
||||||
default:
|
default:
|
||||||
|
w.stream.Close()
|
||||||
close(w.done)
|
close(w.done)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -42,6 +42,8 @@ func (t *table) sendEvent(e *Event) {
|
|||||||
select {
|
select {
|
||||||
case w.resChan <- e:
|
case w.resChan <- e:
|
||||||
case <-w.done:
|
case <-w.done:
|
||||||
|
// don't block forever
|
||||||
|
case <-time.After(time.Second):
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -155,7 +155,7 @@ func (r *runtime) Create(s *Service, opts ...CreateOption) error {
|
|||||||
defer r.Unlock()
|
defer r.Unlock()
|
||||||
|
|
||||||
if _, ok := r.services[s.Name]; ok {
|
if _, ok := r.services[s.Name]; ok {
|
||||||
return errors.New("service already registered")
|
return errors.New("service already running")
|
||||||
}
|
}
|
||||||
|
|
||||||
var options CreateOptions
|
var options CreateOptions
|
||||||
|
@ -266,6 +266,11 @@ func (k *kubernetes) Create(s *runtime.Service, opts ...runtime.CreateOption) er
|
|||||||
o(&options)
|
o(&options)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// hackish
|
||||||
|
if len(options.Type) == 0 {
|
||||||
|
options.Type = k.options.Type
|
||||||
|
}
|
||||||
|
|
||||||
// quickly prevalidate the name and version
|
// quickly prevalidate the name and version
|
||||||
name := s.Name
|
name := s.Name
|
||||||
if len(s.Version) > 0 {
|
if len(s.Version) > 0 {
|
||||||
|
@ -1,10 +1,12 @@
|
|||||||
package kubernetes
|
package kubernetes
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"encoding/json"
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime"
|
"github.com/micro/go-micro/runtime"
|
||||||
|
"github.com/micro/go-micro/util/kubernetes/api"
|
||||||
"github.com/micro/go-micro/util/kubernetes/client"
|
"github.com/micro/go-micro/util/kubernetes/client"
|
||||||
"github.com/micro/go-micro/util/log"
|
"github.com/micro/go-micro/util/log"
|
||||||
)
|
)
|
||||||
@ -18,6 +20,12 @@ type service struct {
|
|||||||
kdeploy *client.Deployment
|
kdeploy *client.Deployment
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func parseError(err error) *api.Status {
|
||||||
|
status := new(api.Status)
|
||||||
|
json.Unmarshal([]byte(err.Error()), &status)
|
||||||
|
return status
|
||||||
|
}
|
||||||
|
|
||||||
func newService(s *runtime.Service, c runtime.CreateOptions) *service {
|
func newService(s *runtime.Service, c runtime.CreateOptions) *service {
|
||||||
// use pre-formatted name/version
|
// use pre-formatted name/version
|
||||||
name := client.Format(s.Name)
|
name := client.Format(s.Name)
|
||||||
@ -90,12 +98,20 @@ func (s *service) Start(k client.Client) error {
|
|||||||
if err := k.Create(deploymentResource(s.kdeploy)); err != nil {
|
if err := k.Create(deploymentResource(s.kdeploy)); err != nil {
|
||||||
log.Debugf("Runtime failed to create deployment: %v", err)
|
log.Debugf("Runtime failed to create deployment: %v", err)
|
||||||
s.Status("error", err)
|
s.Status("error", err)
|
||||||
|
v := parseError(err)
|
||||||
|
if v.Reason == "AlreadyExists" {
|
||||||
|
return runtime.ErrAlreadyExists
|
||||||
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
// create service now that the deployment has been created
|
// create service now that the deployment has been created
|
||||||
if err := k.Create(serviceResource(s.kservice)); err != nil {
|
if err := k.Create(serviceResource(s.kservice)); err != nil {
|
||||||
log.Debugf("Runtime failed to create service: %v", err)
|
log.Debugf("Runtime failed to create service: %v", err)
|
||||||
s.Status("error", err)
|
s.Status("error", err)
|
||||||
|
v := parseError(err)
|
||||||
|
if v.Reason == "AlreadyExists" {
|
||||||
|
return runtime.ErrAlreadyExists
|
||||||
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
package build
|
package build
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/micro/go-micro/runtime/source"
|
"github.com/micro/go-micro/runtime/local/source"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Builder builds binaries
|
// Builder builds binaries
|
@ -9,7 +9,7 @@ import (
|
|||||||
"path/filepath"
|
"path/filepath"
|
||||||
|
|
||||||
docker "github.com/fsouza/go-dockerclient"
|
docker "github.com/fsouza/go-dockerclient"
|
||||||
"github.com/micro/go-micro/runtime/build"
|
"github.com/micro/go-micro/runtime/local/build"
|
||||||
"github.com/micro/go-micro/util/log"
|
"github.com/micro/go-micro/util/log"
|
||||||
)
|
)
|
||||||
|
|
@ -6,7 +6,7 @@ import (
|
|||||||
"os/exec"
|
"os/exec"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime/build"
|
"github.com/micro/go-micro/runtime/local/build"
|
||||||
)
|
)
|
||||||
|
|
||||||
type Builder struct {
|
type Builder struct {
|
@ -10,7 +10,7 @@ import (
|
|||||||
"strconv"
|
"strconv"
|
||||||
"syscall"
|
"syscall"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime/process"
|
"github.com/micro/go-micro/runtime/local/process"
|
||||||
)
|
)
|
||||||
|
|
||||||
func (p *Process) Exec(exe *process.Executable) error {
|
func (p *Process) Exec(exe *process.Executable) error {
|
@ -2,7 +2,7 @@
|
|||||||
package os
|
package os
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/micro/go-micro/runtime/process"
|
"github.com/micro/go-micro/runtime/local/process"
|
||||||
)
|
)
|
||||||
|
|
||||||
type Process struct{}
|
type Process struct{}
|
@ -4,7 +4,7 @@ package process
|
|||||||
import (
|
import (
|
||||||
"io"
|
"io"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime/build"
|
"github.com/micro/go-micro/runtime/local/build"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Process manages a running process
|
// Process manages a running process
|
@ -6,7 +6,7 @@ import (
|
|||||||
"path/filepath"
|
"path/filepath"
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime/source"
|
"github.com/micro/go-micro/runtime/local/source"
|
||||||
git "gopkg.in/src-d/go-git.v4"
|
git "gopkg.in/src-d/go-git.v4"
|
||||||
)
|
)
|
||||||
|
|
@ -7,7 +7,7 @@ import (
|
|||||||
"path/filepath"
|
"path/filepath"
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime/source"
|
"github.com/micro/go-micro/runtime/local/source"
|
||||||
)
|
)
|
||||||
|
|
||||||
type Source struct {
|
type Source struct {
|
@ -54,6 +54,13 @@ type ReadOptions struct {
|
|||||||
Type string
|
Type string
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// CreateType sets the type of service to create
|
||||||
|
func CreateType(t string) CreateOption {
|
||||||
|
return func(o *CreateOptions) {
|
||||||
|
o.Type = t
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// WithCommand specifies the command to execute
|
// WithCommand specifies the command to execute
|
||||||
func WithCommand(args ...string) CreateOption {
|
func WithCommand(args ...string) CreateOption {
|
||||||
return func(o *CreateOptions) {
|
return func(o *CreateOptions) {
|
||||||
|
@ -1,13 +1,18 @@
|
|||||||
// Package runtime is a service runtime manager
|
// Package runtime is a service runtime manager
|
||||||
package runtime
|
package runtime
|
||||||
|
|
||||||
import "time"
|
import (
|
||||||
|
"errors"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
// DefaultRuntime is default micro runtime
|
// DefaultRuntime is default micro runtime
|
||||||
DefaultRuntime Runtime = NewRuntime()
|
DefaultRuntime Runtime = NewRuntime()
|
||||||
// DefaultName is default runtime service name
|
// DefaultName is default runtime service name
|
||||||
DefaultName = "go.micro.runtime"
|
DefaultName = "go.micro.runtime"
|
||||||
|
|
||||||
|
ErrAlreadyExists = errors.New("already exists")
|
||||||
)
|
)
|
||||||
|
|
||||||
// Runtime is a service runtime manager
|
// Runtime is a service runtime manager
|
||||||
|
@ -5,10 +5,9 @@ import (
|
|||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/micro/go-micro/runtime/build"
|
"github.com/micro/go-micro/runtime/local/build"
|
||||||
|
"github.com/micro/go-micro/runtime/local/process"
|
||||||
"github.com/micro/go-micro/runtime/process"
|
proc "github.com/micro/go-micro/runtime/local/process/os"
|
||||||
proc "github.com/micro/go-micro/runtime/process/os"
|
|
||||||
"github.com/micro/go-micro/util/log"
|
"github.com/micro/go-micro/util/log"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -15,25 +15,6 @@ type svc struct {
|
|||||||
runtime pb.RuntimeService
|
runtime pb.RuntimeService
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewRuntime creates new service runtime and returns it
|
|
||||||
func NewRuntime(opts ...runtime.Option) runtime.Runtime {
|
|
||||||
// get default options
|
|
||||||
options := runtime.Options{}
|
|
||||||
|
|
||||||
// apply requested options
|
|
||||||
for _, o := range opts {
|
|
||||||
o(&options)
|
|
||||||
}
|
|
||||||
|
|
||||||
// create default client
|
|
||||||
cli := client.DefaultClient
|
|
||||||
|
|
||||||
return &svc{
|
|
||||||
options: options,
|
|
||||||
runtime: pb.NewRuntimeService(runtime.DefaultName, cli),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Init initializes runtime with given options
|
// Init initializes runtime with given options
|
||||||
func (s *svc) Init(opts ...runtime.Option) error {
|
func (s *svc) Init(opts ...runtime.Option) error {
|
||||||
s.Lock()
|
s.Lock()
|
||||||
@ -183,3 +164,22 @@ func (s *svc) Stop() error {
|
|||||||
func (s *svc) String() string {
|
func (s *svc) String() string {
|
||||||
return "service"
|
return "service"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// NewRuntime creates new service runtime and returns it
|
||||||
|
func NewRuntime(opts ...runtime.Option) runtime.Runtime {
|
||||||
|
// get default options
|
||||||
|
options := runtime.Options{}
|
||||||
|
|
||||||
|
// apply requested options
|
||||||
|
for _, o := range opts {
|
||||||
|
o(&options)
|
||||||
|
}
|
||||||
|
|
||||||
|
// create default client
|
||||||
|
cli := client.DefaultClient
|
||||||
|
|
||||||
|
return &svc{
|
||||||
|
options: options,
|
||||||
|
runtime: pb.NewRuntimeService(runtime.DefaultName, cli),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -653,6 +653,7 @@ func (g *grpcServer) Register() error {
|
|||||||
opts = append(opts, broker.DisableAutoAck())
|
opts = append(opts, broker.DisableAutoAck())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
log.Logf("Subscribing to topic: %s", sb.Topic())
|
||||||
sub, err := config.Broker.Subscribe(sb.Topic(), handler, opts...)
|
sub, err := config.Broker.Subscribe(sb.Topic(), handler, opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
@ -751,6 +752,8 @@ func (g *grpcServer) Start() error {
|
|||||||
g.opts.Address = ts.Addr().String()
|
g.opts.Address = ts.Addr().String()
|
||||||
g.Unlock()
|
g.Unlock()
|
||||||
|
|
||||||
|
// only connect if we're subscribed
|
||||||
|
if len(g.subscribers) > 0 {
|
||||||
// connect to the broker
|
// connect to the broker
|
||||||
if err := config.Broker.Connect(); err != nil {
|
if err := config.Broker.Connect(); err != nil {
|
||||||
return err
|
return err
|
||||||
@ -760,6 +763,7 @@ func (g *grpcServer) Start() error {
|
|||||||
bname := config.Broker.String()
|
bname := config.Broker.String()
|
||||||
|
|
||||||
log.Logf("Broker [%s] Connected to %s", bname, baddr)
|
log.Logf("Broker [%s] Connected to %s", bname, baddr)
|
||||||
|
}
|
||||||
|
|
||||||
// announce self to the world
|
// announce self to the world
|
||||||
if err := g.Register(); err != nil {
|
if err := g.Register(); err != nil {
|
||||||
@ -810,7 +814,18 @@ func (g *grpcServer) Start() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// stop the grpc server
|
// stop the grpc server
|
||||||
|
exit := make(chan bool)
|
||||||
|
|
||||||
|
go func() {
|
||||||
g.srv.GracefulStop()
|
g.srv.GracefulStop()
|
||||||
|
close(exit)
|
||||||
|
}()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-exit:
|
||||||
|
case <-time.After(time.Second):
|
||||||
|
g.srv.Stop()
|
||||||
|
}
|
||||||
|
|
||||||
// close transport
|
// close transport
|
||||||
ch <- nil
|
ch <- nil
|
||||||
|
@ -654,12 +654,12 @@ func (s *rpcServer) Register() error {
|
|||||||
opts = append(opts, broker.DisableAutoAck())
|
opts = append(opts, broker.DisableAutoAck())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
log.Logf("Subscribing to topic: %s", sub.Topic())
|
||||||
sub, err := config.Broker.Subscribe(sb.Topic(), s.HandleEvent, opts...)
|
sub, err := config.Broker.Subscribe(sb.Topic(), s.HandleEvent, opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
log.Logf("Subscribing %s to topic: %s", node.Id, sub.Topic())
|
|
||||||
s.subscribers[sb] = []broker.Subscriber{sub}
|
s.subscribers[sb] = []broker.Subscriber{sub}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -136,6 +136,9 @@ var (
|
|||||||
DefaultRegisterCheck = func(context.Context) error { return nil }
|
DefaultRegisterCheck = func(context.Context) error { return nil }
|
||||||
DefaultRegisterInterval = time.Second * 30
|
DefaultRegisterInterval = time.Second * 30
|
||||||
DefaultRegisterTTL = time.Minute
|
DefaultRegisterTTL = time.Minute
|
||||||
|
|
||||||
|
// NewServer creates a new server
|
||||||
|
NewServer func(...Option) Server = newRpcServer
|
||||||
)
|
)
|
||||||
|
|
||||||
// DefaultOptions returns config options for the default service
|
// DefaultOptions returns config options for the default service
|
||||||
@ -151,11 +154,6 @@ func Init(opt ...Option) {
|
|||||||
DefaultServer.Init(opt...)
|
DefaultServer.Init(opt...)
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewServer returns a new server with options passed in
|
|
||||||
func NewServer(opt ...Option) Server {
|
|
||||||
return newRpcServer(opt...)
|
|
||||||
}
|
|
||||||
|
|
||||||
// NewRouter returns a new router
|
// NewRouter returns a new router
|
||||||
func NewRouter() *router {
|
func NewRouter() *router {
|
||||||
return newRpcRouter()
|
return newRpcRouter()
|
||||||
|
@ -76,6 +76,11 @@ func (s *service) Init(opts ...Option) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// set cmd name
|
||||||
|
if len(s.opts.Cmd.App().Name) == 0 {
|
||||||
|
s.opts.Cmd.App().Name = s.Server().Options().Name
|
||||||
|
}
|
||||||
|
|
||||||
// Initialise the command flags, overriding new service
|
// Initialise the command flags, overriding new service
|
||||||
_ = s.opts.Cmd.Init(
|
_ = s.opts.Cmd.Init(
|
||||||
cmd.Broker(&s.opts.Broker),
|
cmd.Broker(&s.opts.Broker),
|
||||||
|
@ -16,13 +16,13 @@ var (
|
|||||||
|
|
||||||
// Status is an object that is returned when a request
|
// Status is an object that is returned when a request
|
||||||
// failed or delete succeeded.
|
// failed or delete succeeded.
|
||||||
// type Status struct {
|
type Status struct {
|
||||||
// Kind string `json:"kind"`
|
Kind string `json:"kind"`
|
||||||
// Status string `json:"status"`
|
Status string `json:"status"`
|
||||||
// Message string `json:"message"`
|
Message string `json:"message"`
|
||||||
// Reason string `json:"reason"`
|
Reason string `json:"reason"`
|
||||||
// Code int `json:"code"`
|
Code int `json:"code"`
|
||||||
// }
|
}
|
||||||
|
|
||||||
// Response ...
|
// Response ...
|
||||||
type Response struct {
|
type Response struct {
|
||||||
|
@ -5,6 +5,8 @@ var templates = map[string]string{
|
|||||||
"service": serviceTmpl,
|
"service": serviceTmpl,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// stripped image pull policy always
|
||||||
|
// imagePullPolicy: Always
|
||||||
var deploymentTmpl = `
|
var deploymentTmpl = `
|
||||||
apiVersion: apps/v1
|
apiVersion: apps/v1
|
||||||
kind: Deployment
|
kind: Deployment
|
||||||
@ -63,7 +65,6 @@ spec:
|
|||||||
- {{.}}
|
- {{.}}
|
||||||
{{- end }}
|
{{- end }}
|
||||||
image: {{ .Image }}
|
image: {{ .Image }}
|
||||||
imagePullPolicy: Always
|
|
||||||
ports:
|
ports:
|
||||||
{{- with .Ports }}
|
{{- with .Ports }}
|
||||||
{{- range . }}
|
{{- range . }}
|
||||||
|
Loading…
Reference in New Issue
Block a user