diff --git a/.gitignore b/.gitignore index f7c0506..5998959 100644 --- a/.gitignore +++ b/.gitignore @@ -28,6 +28,7 @@ _testmain.go *.test .proc *.conf +consul-esm # IDE files *.iml @@ -55,4 +56,4 @@ Icon .AppleDesktop Network Trash Folder Temporary Items -.apdisk \ No newline at end of file +.apdisk diff --git a/agent.go b/agent.go index fdbf1a9..4aec0c2 100644 --- a/agent.go +++ b/agent.go @@ -8,7 +8,9 @@ import ( "sync" "time" + "github.com/armon/go-metrics" "github.com/hashicorp/consul/api" + "github.com/hashicorp/consul/lib" "github.com/hashicorp/go-uuid" ) @@ -28,8 +30,24 @@ var ( // deregisterTime is the time the TTL check must be in a failed state for // the ESM service in consul to be deregistered. deregisterTime = 30 * time.Minute + + // Specifies a minimum interval that check's can run on + minimumInterval = 1 * time.Second + + // Specifies the maximum transaction size for kv store ops + maximumTransactionSize = 64 ) +type lastKnownStatus struct { + status string + time time.Time +} + +func (s lastKnownStatus) isExpired(ttl time.Duration) bool { + statusAge := time.Now().Sub(s.time) + return statusAge >= ttl +} + type Agent struct { config *Config client *api.Client @@ -44,7 +62,11 @@ type Agent struct { inflightLock sync.Mutex // Custom func to hook into for testing. - watchedNodeFunc func(map[string]bool, []*api.Node) + watchedNodeFunc func(map[string]bool, []*api.Node) + knownNodeStatuses map[string]lastKnownStatus + knwonNodeStatusesLock sync.Mutex + + memSink *metrics.InmemSink } func NewAgent(config *Config, logger *log.Logger) (*Agent, error) { @@ -64,13 +86,20 @@ func NewAgent(config *Config, logger *log.Logger) (*Agent, error) { } } + memSink, err := lib.InitTelemetry(config.Telemetry) + if err != nil { + return nil, err + } + agent := Agent{ - config: config, - client: client, - id: id, - logger: logger, - shutdownCh: make(chan struct{}), - inflightPings: make(map[string]struct{}), + config: config, + client: client, + id: id, + logger: logger, + shutdownCh: make(chan struct{}), + inflightPings: make(map[string]struct{}), + knownNodeStatuses: make(map[string]lastKnownStatus), + memSink: memSink, } logger.Printf("[INFO] Connecting to Consul on %s...", clientConf.Address) @@ -308,6 +337,8 @@ func (a *Agent) watchNodeList() { continue } + a.logger.Printf("[INFO] Fetched %d nodes from catalog", len(nodes)) + var pingList []*api.Node for _, node := range nodes { if pingNodes[node.Node] { @@ -344,7 +375,7 @@ func (a *Agent) watchHealthChecks(nodeListCh chan map[string]bool) { // Start a check runner to track and run the health checks we're responsible for and call // UpdateChecks when we get an update from watchHealthChecks. - a.checkRunner = NewCheckRunner(a.logger, a.client, a.config.CheckUpdateInterval) + a.checkRunner = NewCheckRunner(a.logger, a.client, a.config.CheckUpdateInterval, minimumInterval) go a.checkRunner.reapServices(a.shutdownCh) defer a.checkRunner.Stop() @@ -387,3 +418,23 @@ func (a *Agent) watchHealthChecks(nodeListCh chan map[string]bool) { } } } + +// Check last visible node status. +// Returns true, if status is changed since last update and false otherwise. +func (a *Agent) shouldUpdateNodeStatus(node string, newStatus string) bool { + a.knwonNodeStatusesLock.Lock() + defer a.knwonNodeStatusesLock.Unlock() + ttl := a.config.NodeHealthRefreshInterval + lastStatus, exists := a.knownNodeStatuses[node] + if !exists || lastStatus.isExpired(ttl) { + return true + } + return newStatus != lastStatus.status +} + +// Update last visible node status. +func (a *Agent) updateLastKnownNodeStatus(node string, newStatus string) { + a.knwonNodeStatusesLock.Lock() + defer a.knwonNodeStatusesLock.Unlock() + a.knownNodeStatuses[node] = lastKnownStatus{newStatus, time.Now()} +} diff --git a/check.go b/check.go index 890578a..eb751e6 100644 --- a/check.go +++ b/check.go @@ -10,6 +10,7 @@ import ( "sync" "time" + "github.com/armon/go-metrics" consulchecks "github.com/hashicorp/consul/agent/checks" "github.com/hashicorp/consul/api" "github.com/hashicorp/consul/lib" @@ -24,6 +25,8 @@ var ( defaultInterval = 30 * time.Second ) +type CheckIdSet map[types.CheckID]bool + type CheckRunner struct { sync.RWMutex @@ -39,9 +42,10 @@ type CheckRunner struct { deferCheck map[types.CheckID]*time.Timer CheckUpdateInterval time.Duration + MinimumInterval time.Duration } -func NewCheckRunner(logger *log.Logger, client *api.Client, updateInterval time.Duration) *CheckRunner { +func NewCheckRunner(logger *log.Logger, client *api.Client, updateInterval time.Duration, minimumInterval time.Duration) *CheckRunner { return &CheckRunner{ logger: logger, client: client, @@ -51,6 +55,7 @@ func NewCheckRunner(logger *log.Logger, client *api.Client, updateInterval time. checksCritical: make(map[types.CheckID]time.Time), deferCheck: make(map[types.CheckID]*time.Timer), CheckUpdateInterval: updateInterval, + MinimumInterval: minimumInterval, } } @@ -67,13 +72,118 @@ func (c *CheckRunner) Stop() { } } +// Update an HTTP check +func (c *CheckRunner) updateCheckHttp(latestCheck *api.HealthCheck, checkHash types.CheckID, definition *api.HealthCheckDefinition, updated CheckIdSet, added CheckIdSet) bool { + http := &consulchecks.CheckHTTP{ + Notify: c, + CheckID: checkHash, + HTTP: definition.HTTP, + Header: definition.Header, + Method: definition.Method, + Interval: definition.IntervalDuration, + Timeout: definition.TimeoutDuration, + Logger: c.logger, + TLSClientConfig: &tls.Config{ + InsecureSkipVerify: definition.TLSSkipVerify}, + } + + if check, checkExists := c.checks[checkHash]; checkExists { + httpCheck, httpCheckExists := c.checksHTTP[checkHash] + if httpCheckExists && + check.Status == latestCheck.Status && + httpCheck.HTTP == http.HTTP && + reflect.DeepEqual(httpCheck.Header, http.Header) && + httpCheck.Method == http.Method && + httpCheck.TLSClientConfig.InsecureSkipVerify == http.TLSClientConfig.InsecureSkipVerify && + httpCheck.Interval == http.Interval && + httpCheck.Timeout == http.Timeout && + check.Definition.DeregisterCriticalServiceAfter == definition.DeregisterCriticalServiceAfter { + return false + } + + c.logger.Printf("[INFO] Updating HTTP check %q", checkHash) + + if httpCheckExists { + httpCheck.Stop() + } else { + tcpCheck, tcpCheckExists := c.checksTCP[checkHash] + if !tcpCheckExists { + c.logger.Printf("[WARN] Inconsistency check %q - is not TCP and HTTP", checkHash) + return false + } + tcpCheck.Stop() + delete(c.checksTCP, checkHash) + } + + updated[checkHash] = true + } else { + added[checkHash] = true + } + + http.Start() + c.checksHTTP[checkHash] = http + + return true +} + +func (c *CheckRunner) updateCheckTcp(latestCheck *api.HealthCheck, checkHash types.CheckID, definition *api.HealthCheckDefinition, updated CheckIdSet, added CheckIdSet) bool { + tcp := &consulchecks.CheckTCP{ + Notify: c, + CheckID: checkHash, + TCP: definition.TCP, + Interval: definition.IntervalDuration, + Timeout: definition.TimeoutDuration, + Logger: c.logger, + } + + if check, checkExists := c.checks[checkHash]; checkExists { + tcpCheck, tcpCheckExists := c.checksTCP[checkHash] + if tcpCheckExists && + check.Status == latestCheck.Status && + tcpCheck.TCP == tcp.TCP && + tcpCheck.Interval == tcp.Interval && + tcpCheck.Timeout == tcp.Timeout && + check.Definition.DeregisterCriticalServiceAfter == definition.DeregisterCriticalServiceAfter { + return false + } + + c.logger.Printf("[INFO] Updating TCP check %q", checkHash) + + if tcpCheckExists { + tcpCheck.Stop() + } else { + httpCheck, httpCheckExists := c.checksHTTP[checkHash] + if !httpCheckExists { + c.logger.Printf("[WARN] Inconsistency check %q - is not TCP and HTTP", checkHash) + return false + } + httpCheck.Stop() + delete(c.checksHTTP, checkHash) + } + + updated[checkHash] = true + } else { + added[checkHash] = true + } + + tcp.Start() + c.checksTCP[checkHash] = tcp + + return true +} + // UpdateChecks takes a list of checks from the catalog and updates // our list of running checks to match. func (c *CheckRunner) UpdateChecks(checks api.HealthChecks) { + defer metrics.MeasureSince([]string{"checks", "update"}, time.Now()) c.Lock() defer c.Unlock() - found := make(map[types.CheckID]struct{}) + found := make(CheckIdSet) + + added := make(CheckIdSet) + updated := make(CheckIdSet) + removed := make(CheckIdSet) for _, check := range checks { // Skip the ping-based node check since we're managing that separately @@ -82,95 +192,33 @@ func (c *CheckRunner) UpdateChecks(checks api.HealthChecks) { } checkHash := checkHash(check) - found[checkHash] = struct{}{} - definition := check.Definition if definition.IntervalDuration == 0 { definition.IntervalDuration = defaultInterval } - if definition.HTTP != "" { - http := &consulchecks.CheckHTTP{ - Notify: c, - CheckID: checkHash, - HTTP: definition.HTTP, - Header: definition.Header, - Method: definition.Method, - Interval: definition.IntervalDuration, - Timeout: definition.TimeoutDuration, - Logger: c.logger, - TLSClientConfig: &tls.Config{ - InsecureSkipVerify: definition.TLSSkipVerify}, - } - if _, ok := c.checks[checkHash]; ok { - if data, ok := c.checksHTTP[checkHash]; ok && - c.checks[checkHash].Status == check.Status && - data.HTTP == http.HTTP && - reflect.DeepEqual(data.Header, http.Header) && - data.Method == http.Method && - data.TLSClientConfig.InsecureSkipVerify == - http.TLSClientConfig.InsecureSkipVerify && - data.Interval == http.Interval && - data.Timeout == http.Timeout && - c.checks[checkHash].Definition.DeregisterCriticalServiceAfter == definition.DeregisterCriticalServiceAfter { - continue - } - c.logger.Printf("[INFO] Updating HTTP check %q", checkHash) - if c.checks[checkHash].Definition.HTTP != "" { - httpCheck := c.checksHTTP[checkHash] - httpCheck.Stop() - } else { - if _, ok := c.checksTCP[checkHash]; !ok { - c.logger.Printf("[WARN] Inconsistency check %q - is not TCP and HTTP", checkHash) - continue - } - tcpCheck := c.checksTCP[checkHash] - tcpCheck.Stop() - delete(c.checksTCP, checkHash) - } - } - http.Start() - c.checksHTTP[checkHash] = http - } else if definition.TCP != "" { - tcp := &consulchecks.CheckTCP{ - Notify: c, - CheckID: checkHash, - TCP: definition.TCP, - Interval: definition.IntervalDuration, - Timeout: definition.TimeoutDuration, - Logger: c.logger, - } + // here we verify that the interval is not less then the minimum + if definition.IntervalDuration < c.MinimumInterval { + definition.IntervalDuration = c.MinimumInterval + } - if _, ok := c.checks[checkHash]; ok { - if data, ok := c.checksTCP[checkHash]; ok && - c.checks[checkHash].Status == check.Status && - data.TCP == tcp.TCP && - data.Interval == tcp.Interval && - data.Timeout == tcp.Timeout && - c.checks[checkHash].Definition.DeregisterCriticalServiceAfter == definition.DeregisterCriticalServiceAfter { - continue - } - c.logger.Printf("[INFO] Updating TCP check %q", checkHash) - if c.checks[checkHash].Definition.TCP != "" { - tcpCheck := c.checksTCP[checkHash] - tcpCheck.Stop() - } else { - if _, ok := c.checksHTTP[checkHash]; !ok { - c.logger.Printf("[WARN] Inconsistency check %q - is not TCP and HTTP", checkHash) - continue - } - httpCheck := c.checksHTTP[checkHash] - httpCheck.Stop() - delete(c.checksHTTP, checkHash) - } - } - tcp.Start() - c.checksTCP[checkHash] = tcp + anyUpdates := false + + if definition.HTTP != "" { + anyUpdates = c.updateCheckHttp(check, checkHash, &definition, updated, added) + } else if definition.TCP != "" { + anyUpdates = c.updateCheckTcp(check, checkHash, &definition, updated, added) } else { c.logger.Printf("[WARN] check %q is not a valid HTTP or TCP check", checkHash) continue } + // if we had to fix the interval and we had to update the service, put some trace out + if anyUpdates && check.Definition.IntervalDuration < c.MinimumInterval { + c.logger.Printf("[WARN] Check interval too low at %v for check %s", check.Definition.Interval, check.Name) + } + + found[checkHash] = true c.checks[checkHash] = check } @@ -180,17 +228,24 @@ func (c *CheckRunner) UpdateChecks(checks api.HealthChecks) { if _, ok := found[checkHash]; !ok { delete(c.checks, checkHash) delete(c.checksCritical, checkHash) - if check.Definition.HTTP != "" { - httpCheck := c.checksHTTP[checkHash] + + if httpCheck, httpCheckExists := c.checksHTTP[checkHash]; httpCheckExists { httpCheck.Stop() delete(c.checksHTTP, checkHash) - } else { - tcpCheck := c.checksTCP[checkHash] + } + if tcpCheck, tcpCheckExists := c.checksTCP[checkHash]; tcpCheckExists { tcpCheck.Stop() delete(c.checksTCP, checkHash) } + + removed[checkHash] = true } } + + if len(added) > 0 || len(updated) > 0 || len(removed) > 0 { + c.logger.Printf("[INFO] Updated %d checks, found %d, added %d, updated %d, removed %d", + len(checks), len(found), len(added), len(updated), len(removed)) + } } // UpdateCheck handles the output of an HTTP/TCP check and decides whether or not @@ -251,9 +306,9 @@ func (c *CheckRunner) handleCheckUpdate(check *api.HealthCheck, status, output s } var existing *api.HealthCheck checkID := strings.TrimPrefix(string(check.CheckID), check.Node+"/") - for _, c := range checks { - if c.CheckID == checkID { - existing = c + for _, check := range checks { + if check.CheckID == checkID { + existing = check break } } @@ -263,6 +318,9 @@ func (c *CheckRunner) handleCheckUpdate(check *api.HealthCheck, status, output s existing.Status = status existing.Output = output + + c.logger.Printf("[INFO] Updating output and status for %q", existing.CheckID) + ops := api.TxnOps{ &api.TxnOp{ Check: &api.CheckTxnOp{ @@ -271,6 +329,7 @@ func (c *CheckRunner) handleCheckUpdate(check *api.HealthCheck, status, output s }, }, } + metrics.IncrCounter([]string{"check", "txn"}, 1) ok, resp, _, err := c.client.Txn().Txn(ops, nil) if err != nil { c.logger.Printf("[WARN] Error updating check status in Consul: %v", err) diff --git a/check_test.go b/check_test.go index 06bbdbe..3e40776 100644 --- a/check_test.go +++ b/check_test.go @@ -23,7 +23,7 @@ func TestCheck_HTTP(t *testing.T) { } logger := log.New(LOGOUT, "", 0) - runner := NewCheckRunner(logger, client, 0) + runner := NewCheckRunner(logger, client, 0, 0) defer runner.Stop() // Register an external node with an initially critical http check. @@ -125,7 +125,7 @@ func TestCheck_TCP(t *testing.T) { } logger := log.New(LOGOUT, "", 0) - runner := NewCheckRunner(logger, client, 0) + runner := NewCheckRunner(logger, client, 0, 0) defer runner.Stop() // Register an external node with an initially critical http check diff --git a/config.go b/config.go index 0454724..685f1aa 100644 --- a/config.go +++ b/config.go @@ -10,6 +10,7 @@ import ( "github.com/hashicorp/consul/api" "github.com/hashicorp/consul/command/flags" + "github.com/hashicorp/consul/lib" "github.com/hashicorp/hcl" "github.com/hashicorp/hcl/hcl/ast" "github.com/mitchellh/mapstructure" @@ -29,12 +30,13 @@ type Config struct { Tag string KVPath string - NodeMeta map[string]string - Interval time.Duration - DeregisterAfter time.Duration - CheckUpdateInterval time.Duration - CoordinateUpdateInterval time.Duration - NodeReconnectTimeout time.Duration + NodeMeta map[string]string + Interval time.Duration + DeregisterAfter time.Duration + CheckUpdateInterval time.Duration + CoordinateUpdateInterval time.Duration + NodeHealthRefreshInterval time.Duration + NodeReconnectTimeout time.Duration HTTPAddr string Token string @@ -47,6 +49,11 @@ type Config struct { PingType string + DisableRedundantStatusUpdates bool + DisableCoordinateUpdates bool + + Telemetry lib.TelemetryConfig + // Test-only fields. id string } @@ -90,15 +97,43 @@ func DefaultConfig() *Config { NodeMeta: map[string]string{ "external-node": "true", }, - Interval: 10 * time.Second, - DeregisterAfter: 72 * time.Hour, - CheckUpdateInterval: 5 * time.Minute, - CoordinateUpdateInterval: 10 * time.Second, - NodeReconnectTimeout: 72 * time.Hour, - PingType: PingTypeUDP, + Interval: 10 * time.Second, + DeregisterAfter: 72 * time.Hour, + CheckUpdateInterval: 5 * time.Minute, + CoordinateUpdateInterval: 10 * time.Second, + NodeHealthRefreshInterval: 1 * time.Hour, + NodeReconnectTimeout: 72 * time.Hour, + PingType: PingTypeUDP, + DisableRedundantStatusUpdates: false, + DisableCoordinateUpdates: false, } } +type Telemetry struct { + CirconusAPIApp *string `mapstructure:"circonus_api_app"` + CirconusAPIToken *string `mapstructure:"circonus_api_token"` + CirconusAPIURL *string `mapstructure:"circonus_api_url"` + CirconusBrokerID *string `mapstructure:"circonus_broker_id"` + CirconusBrokerSelectTag *string `mapstructure:"circonus_broker_select_tag"` + CirconusCheckDisplayName *string `mapstructure:"circonus_check_display_name"` + CirconusCheckForceMetricActivation *string `mapstructure:"circonus_check_force_metric_activation"` + CirconusCheckID *string `mapstructure:"circonus_check_id"` + CirconusCheckInstanceID *string `mapstructure:"circonus_check_instance_id"` + CirconusCheckSearchTag *string `mapstructure:"circonus_check_search_tag"` + CirconusCheckTags *string `mapstructure:"circonus_check_tags"` + CirconusSubmissionInterval *string `mapstructure:"circonus_submission_interval"` + CirconusSubmissionURL *string `mapstructure:"circonus_submission_url"` + DisableHostname *bool `mapstructure:"disable_hostname"` + DogstatsdAddr *string `mapstructure:"dogstatsd_addr"` + DogstatsdTags []string `mapstructure:"dogstatsd_tags"` + FilterDefault *bool `mapstructure:"filter_default"` + PrefixFilter []string `mapstructure:"prefix_filter"` + MetricsPrefix *string `mapstructure:"metrics_prefix"` + PrometheusRetentionTime *string `mapstructure:"prometheus_retention_time"` + StatsdAddr *string `mapstructure:"statsd_address"` + StatsiteAddr *string `mapstructure:"statsite_address"` +} + type HumanConfig struct { LogLevel flags.StringValue `mapstructure:"log_level"` EnableSyslog flags.BoolValue `mapstructure:"enable_syslog"` @@ -122,6 +157,11 @@ type HumanConfig struct { TLSServerName flags.StringValue `mapstructure:"tls_server_name"` PingType flags.StringValue `mapstructure:"ping_type"` + + DisableRedundantStatusUpdates flags.BoolValue `mapstructure:"disable_redundant_status_updates"` + DisableCoordinateUpdates flags.BoolValue `mapstructure:"disable_cooridinate_updates"` + + Telemetry []Telemetry `mapstructure:"telemetry"` } func DecodeConfig(r io.Reader) (*HumanConfig, error) { @@ -148,6 +188,11 @@ func DecodeConfig(r io.Reader) (*HumanConfig, error) { return nil, fmt.Errorf("only one node_meta block allowed") } + telemetry := list.Filter("telemetry") + if len(telemetry.Elem().Items) > 1 { + return nil, fmt.Errorf("only one telemetry block allowed") + } + // Decode the full thing into a map[string]interface for ease of use var config HumanConfig var m map[string]interface{} @@ -232,9 +277,8 @@ func MergeConfigPaths(dst *Config, paths []string) error { if err != nil { return err } - MergeConfig(dst, src) - return nil + return MergeConfig(dst, src) } for _, path := range paths { @@ -246,7 +290,22 @@ func MergeConfigPaths(dst *Config, paths []string) error { return nil } -func MergeConfig(dst *Config, src *HumanConfig) { +func stringVal(v *string) string { + if v == nil { + return "" + } + return *v +} + +func boolVal(v *bool) bool { + if v == nil { + return false + } + + return *v +} + +func MergeConfig(dst *Config, src *HumanConfig) error { src.LogLevel.Merge(&dst.LogLevel) src.Service.Merge(&dst.Service) src.Tag.Merge(&dst.Tag) @@ -265,4 +324,65 @@ func MergeConfig(dst *Config, src *HumanConfig) { src.KeyFile.Merge(&dst.KeyFile) src.TLSServerName.Merge(&dst.TLSServerName) src.PingType.Merge(&dst.PingType) + src.DisableRedundantStatusUpdates.Merge(&dst.DisableRedundantStatusUpdates) + src.DisableCoordinateUpdates.Merge(&dst.DisableCoordinateUpdates) + + // We check on parse time that there is at most one + if len(src.Telemetry) != 0 { + telemetry := src.Telemetry[0] + // Parse the metric filters + var telemetryAllowedPrefixes, telemetryBlockedPrefixes []string + for _, rule := range telemetry.PrefixFilter { + if rule == "" { + fmt.Println("[WARN] Cannot have empty filter rule in prefix_filter") + continue + } + switch rule[0] { + case '+': + telemetryAllowedPrefixes = append(telemetryAllowedPrefixes, rule[1:]) + case '-': + telemetryBlockedPrefixes = append(telemetryBlockedPrefixes, rule[1:]) + default: + fmt.Printf("[WARN] Filter rule must begin with either '+' or '-': %q\n", rule) + } + } + + var prometheusRetentionTime time.Duration + if telemetry.PrometheusRetentionTime != nil { + d, err := time.ParseDuration(*telemetry.PrometheusRetentionTime) + if err != nil { + return fmt.Errorf("prometheus_retention_time: invalid duration: %q: %s", *telemetry.PrometheusRetentionTime, err) + } + + prometheusRetentionTime = d + } + + dst.Telemetry = lib.TelemetryConfig{ + CirconusAPIApp: stringVal(telemetry.CirconusAPIApp), + CirconusAPIToken: stringVal(telemetry.CirconusAPIToken), + CirconusAPIURL: stringVal(telemetry.CirconusAPIURL), + CirconusBrokerID: stringVal(telemetry.CirconusBrokerID), + CirconusBrokerSelectTag: stringVal(telemetry.CirconusBrokerSelectTag), + CirconusCheckDisplayName: stringVal(telemetry.CirconusCheckDisplayName), + CirconusCheckForceMetricActivation: stringVal(telemetry.CirconusCheckForceMetricActivation), + CirconusCheckID: stringVal(telemetry.CirconusCheckID), + CirconusCheckInstanceID: stringVal(telemetry.CirconusCheckInstanceID), + CirconusCheckSearchTag: stringVal(telemetry.CirconusCheckSearchTag), + CirconusCheckTags: stringVal(telemetry.CirconusCheckTags), + CirconusSubmissionInterval: stringVal(telemetry.CirconusSubmissionInterval), + CirconusSubmissionURL: stringVal(telemetry.CirconusSubmissionURL), + DisableHostname: boolVal(telemetry.DisableHostname), + DogstatsdAddr: stringVal(telemetry.DogstatsdAddr), + DogstatsdTags: telemetry.DogstatsdTags, + PrometheusRetentionTime: prometheusRetentionTime, + FilterDefault: boolVal(telemetry.FilterDefault), + AllowedPrefixes: telemetryAllowedPrefixes, + BlockedPrefixes: telemetryBlockedPrefixes, + MetricsPrefix: stringVal(telemetry.MetricsPrefix), + StatsdAddr: stringVal(telemetry.StatsdAddr), + StatsiteAddr: stringVal(telemetry.StatsiteAddr), + } + } + + return nil } diff --git a/config_test.go b/config_test.go index 68ad79a..c1a2f76 100644 --- a/config_test.go +++ b/config_test.go @@ -6,6 +6,8 @@ import ( "strings" "testing" "time" + + "github.com/hashicorp/consul/lib" ) func TestDecodeMergeConfig(t *testing.T) { @@ -29,6 +31,10 @@ cert_file = "cert.pem" key_file = "key.pem" tls_server_name = "example.io" ping_type = "socket" +telemetry { + statsd_address = "example.io:8888" + prefix_filter = ["+good", "-bad", "+better", "-worse", "wrong", ""] +} `) expected := &Config{ @@ -51,6 +57,11 @@ ping_type = "socket" KeyFile: "key.pem", TLSServerName: "example.io", PingType: PingTypeSocket, + Telemetry: lib.TelemetryConfig{ + StatsdAddr: "example.io:8888", + AllowedPrefixes: []string{"good", "better"}, + BlockedPrefixes: []string{"bad", "worse"}, + }, } result := &Config{} diff --git a/coordinate.go b/coordinate.go index 7bd9629..2d29e2e 100644 --- a/coordinate.go +++ b/coordinate.go @@ -7,6 +7,7 @@ import ( "strings" "time" + "github.com/armon/go-metrics" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/api" multierror "github.com/hashicorp/go-multierror" @@ -136,11 +137,30 @@ func (a *Agent) nodeTicker(numNodes int) *time.Ticker { return time.NewTicker(waitTime) } -// updateHealthyNode updates the node's health check and clears any kv -// critical tracking associated with it. +// updateHealthyNode updates the node's health check, additionally it debounces repeated updates func (a *Agent) updateHealthyNode(node *api.Node, kvClient *api.KV, key string, kvPair *api.KVPair) error { status := api.HealthPassing + isChanged := a.shouldUpdateNodeStatus(node.Node, status) + if !isChanged { + a.logger.Printf("[TRACE] Debounce: skipping healthy node status update for node %s", node.Node) + return nil + } else { + a.logger.Printf("[TRACE] Debounce: healthy node status update. Node %s, status %s", node.Node, status) + } + + err := a.updateHealthyNodeTxn(node, kvClient, key, kvPair) + if err == nil { + // only if the transaction succeed, record a node status update otherwise we should retry + a.updateLastKnownNodeStatus(node.Node, status) + } + + return err +} + +// updateHealthyNode updates the node's health check and clears any kv +// critical tracking associated with it. +func (a *Agent) updateHealthyNodeTxn(node *api.Node, kvClient *api.KV, key string, kvPair *api.KVPair) error { // If a critical node went back to passing, delete the KV entry for it. var ops api.TxnOps if kvPair != nil { @@ -155,14 +175,33 @@ func (a *Agent) updateHealthyNode(node *api.Node, kvClient *api.KV, key string, } // Batch the possible KV deletion operation with the external health check update. - return a.updateNodeCheck(node, ops, status, NodeAliveStatus) + return a.updateNodeCheck(node, ops, api.HealthPassing, NodeAliveStatus) } -// updateFailedNode sets the node's health check to critical and checks whether -// the node has exceeded its timeout an needs to be reaped. +// updateFailedNode sets the node's health check to critical, additionally it debounces repeated updates func (a *Agent) updateFailedNode(node *api.Node, kvClient *api.KV, key string, kvPair *api.KVPair) error { status := api.HealthCritical + isChanged := a.shouldUpdateNodeStatus(node.Node, status) + if !isChanged { + a.logger.Printf("[TRACE] Debounce: skipping failed node status update for node %s", node.Node) + return nil + } else { + a.logger.Printf("[TRACE] Debounce: failed node status update. Node %s, status %s", node.Node, status) + } + + err := a.updateFailedNodeTxn(node, kvClient, key, kvPair) + if err == nil { + // only if the transaction succeed, record a node status update otherwise we should retry + a.updateLastKnownNodeStatus(node.Node, status) + } + + return err +} + +// updateFailedNodeTxn sets the node's health check to critical and checks whether +// the node has exceeded its timeout an needs to be reaped. +func (a *Agent) updateFailedNodeTxn(node *api.Node, kvClient *api.KV, key string, kvPair *api.KVPair) error { // If there's no existing key tracking how long the node has been critical, create one. var ops api.TxnOps if kvPair == nil { @@ -221,11 +260,12 @@ func (a *Agent) updateFailedNode(node *api.Node, kvClient *api.KV, key string, k } // Batch our KV update tracking the critical time with the external health check update. - return a.updateNodeCheck(node, ops, status, NodeCriticalStatus) + return a.updateNodeCheck(node, ops, api.HealthCritical, NodeCriticalStatus) } // updateNodeCheck updates the node's externalNodeHealth check with the given status/output. func (a *Agent) updateNodeCheck(node *api.Node, ops api.TxnOps, status, output string) error { + metrics.IncrCounter([]string{"coord", "txn"}, 1) // Update the external health check status. ops = append(ops, &api.TxnOp{ Check: &api.CheckTxnOp{ @@ -269,6 +309,11 @@ func (a *Agent) runClientTxn(ops api.TxnOps) error { // updateNodeCoordinate updates the node's coordinate entry based on the // given RTT from a ping func (a *Agent) updateNodeCoordinate(node *api.Node, rtt time.Duration) error { + if a.config.DisableCoordinateUpdates { + a.logger.Printf("[TRACE] Debounce: skipping coordinate update for node %s", node.Node) + return nil + } + // Get coordinate info for the node. coords, _, err := a.client.Coordinate().Node(node.Node, nil) if err != nil && !strings.Contains(err.Error(), "Unexpected response code: 404") { @@ -315,17 +360,23 @@ func (a *Agent) updateNodeCoordinate(node *api.Node, rtt time.Duration) error { return fmt.Errorf("error updating coordinate for node %q: %v", node.Node, err) } - // Update the coordinate in the catalog. - _, err = a.client.Coordinate().Update(&api.CoordinateEntry{ - Node: coord.Node, - Segment: coord.Segment, - Coord: newCoord, - }, nil) + // Update the coordinate in the catalog + // - if there is a significant change + // - if there isn't a coordinate already there + if len(coords) == 0 || coord.Coord.DistanceTo(newCoord) > time.Millisecond { + _, err = a.client.Coordinate().Update(&api.CoordinateEntry{ + Node: coord.Node, + Segment: coord.Segment, + Coord: newCoord, + }, nil) - if err != nil { - return fmt.Errorf("error applying coordinate update for node %q: %v", node.Node, err) + if err != nil { + return fmt.Errorf("error applying coordinate update for node %q: %v", node.Node, err) + } + a.logger.Printf("[INFO] Updated coordinates for node %q with distance %q from previous", node.Node, coord.Coord.DistanceTo(newCoord)) + } else { + a.logger.Printf("[TRACE] Skipped update for coordinates, node %q change %q not significant", node.Node, coord.Coord.DistanceTo(newCoord)) } - a.logger.Printf("[TRACE] Updated coordinates for node %q", node.Node) return nil } diff --git a/coordinate_test.go b/coordinate_test.go index 10d1f4e..3145576 100644 --- a/coordinate_test.go +++ b/coordinate_test.go @@ -34,7 +34,12 @@ func TestCoordinate_updateNodeCoordinate(t *testing.T) { t.Fatal(err) } - agent := &Agent{client: client, logger: log.New(LOGOUT, "", log.LstdFlags)} + agent := &Agent{ + client: client, + config: DefaultConfig(), + logger: log.New(LOGOUT, "", log.LstdFlags), + knownNodeStatuses: make(map[string]lastKnownStatus), + } agent.updateNodeCoordinate(&api.Node{Node: "external"}, 1*time.Second) var coords []*api.CoordinateEntry @@ -73,7 +78,12 @@ func TestCoordinate_updateNodeCheck(t *testing.T) { t.Fatal(err) } - agent := &Agent{client: client, logger: log.New(LOGOUT, "", log.LstdFlags)} + agent := &Agent{ + client: client, + config: DefaultConfig(), + logger: log.New(LOGOUT, "", log.LstdFlags), + knownNodeStatuses: make(map[string]lastKnownStatus), + } if err := agent.updateFailedNode(&api.Node{Node: "external"}, client.KV(), "testkey", nil); err != nil { t.Fatal(err) } @@ -167,6 +177,7 @@ func TestCoordinate_reapFailedNode(t *testing.T) { client: client, config: DefaultConfig(), logger: log.New(LOGOUT, "", log.LstdFlags), + knownNodeStatuses: make(map[string]lastKnownStatus), } agent.config.NodeReconnectTimeout = 200 * time.Millisecond @@ -203,8 +214,8 @@ func TestCoordinate_reapFailedNode(t *testing.T) { t.Fatal(err) } - // Call updateFailedNode again to reap the node - if err := agent.updateFailedNode(&api.Node{Node: "external"}, client.KV(), "testkey", kvPair); err != nil { + // Call updateFailedNode again to reap the node (use the Txn version to skip debounce checks) + if err := agent.updateFailedNodeTxn(&api.Node{Node: "external"}, client.KV(), "testkey", kvPair); err != nil { t.Fatal(err) } diff --git a/go.mod b/go.mod index d0afbc5..ecf03f3 100644 --- a/go.mod +++ b/go.mod @@ -1,16 +1,31 @@ module github.com/hashicorp/consul-esm -go 1.12 +go 1.13 require ( + github.com/DataDog/datadog-go v3.2.0+incompatible // indirect + github.com/Microsoft/go-winio v0.4.5 // indirect + github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 github.com/hashicorp/consul v1.6.1 github.com/hashicorp/consul/api v1.2.0 - github.com/hashicorp/consul/sdk v0.2.0 + github.com/hashicorp/consul/sdk v0.4.0 + github.com/hashicorp/go-immutable-radix v1.1.0 // indirect github.com/hashicorp/go-multierror v1.0.0 + github.com/hashicorp/go-retryablehttp v0.6.3 // indirect + github.com/hashicorp/go-rootcerts v1.0.1 // indirect + github.com/hashicorp/go-sockaddr v1.0.2 // indirect github.com/hashicorp/go-uuid v1.0.1 + github.com/hashicorp/go-version v1.1.0 // indirect + github.com/hashicorp/golang-lru v0.5.1 // indirect github.com/hashicorp/hcl v1.0.0 github.com/hashicorp/serf v0.8.4 + github.com/miekg/dns v1.1.22 // indirect github.com/mitchellh/cli v1.0.0 + github.com/mitchellh/hashstructure v1.0.0 // indirect github.com/mitchellh/mapstructure v1.1.2 + github.com/prometheus/client_golang v1.2.1 // indirect github.com/sparrc/go-ping v0.0.0-20190613174326-4e5b6552494c + golang.org/x/crypto v0.0.0-20191002192127-34f69633bfdc // indirect + golang.org/x/net v0.0.0-20191003171128-d98b1b443823 // indirect + google.golang.org/grpc v1.25.0 // indirect ) diff --git a/go.sum b/go.sum index a3466ee..ad5bf08 100644 --- a/go.sum +++ b/go.sum @@ -4,35 +4,47 @@ github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX github.com/Azure/go-autorest v10.7.0+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= github.com/Azure/go-autorest v10.15.3+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/DataDog/datadog-go v2.2.0+incompatible h1:V5BKkxACZLjzHjSgBbr2gvLA2Ae49yhc6CSY7MLy5k4= github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= +github.com/DataDog/datadog-go v3.2.0+incompatible h1:qSG2N4FghB1He/r2mFrWKCaL7dXCilEuNEeAn20fdD4= +github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/Jeffail/gabs v1.1.0/go.mod h1:6xMvQMK4k33lb7GUUpaAPh6nKMmemQeg5d4gn7/bOXc= -github.com/Microsoft/go-winio v0.4.3 h1:M3NHMuPgMSUPdE5epwNUHlRPSVzHs8HpRTrVXhR0myo= github.com/Microsoft/go-winio v0.4.3/go.mod h1:VhR8bwka0BXejwEJY73c50VrPtXAaKcyvVC4A4RozmA= +github.com/Microsoft/go-winio v0.4.5 h1:U2XsGR5dBg1yzwSEJoP2dE2/aAXpmad+CNG2hE9Pd5k= +github.com/Microsoft/go-winio v0.4.5/go.mod h1:VhR8bwka0BXejwEJY73c50VrPtXAaKcyvVC4A4RozmA= github.com/NYTimes/gziphandler v1.0.1/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk= github.com/SAP/go-hdb v0.12.0/go.mod h1:etBT+FAi1t5k3K3tf5vQTnosgYmhDkRi8jEnQqCnxF0= github.com/SermoDigital/jose v0.0.0-20180104203859-803625baeddc/go.mod h1:ARgCUhI1MHQH+ONky/PAtmVHQrP5JlGY0F3poXOp/fA= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/abdullin/seq v0.0.0-20160510034733-d5467c17e7af/go.mod h1:5Jv4cbFiHJMsVxt52+i0Ha45fjshj6wxYr1r19tB9bw= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/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-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e h1:QEF07wC0T1rKkctt1RINW/+RMTVmiwxETico2l3gxJA= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 h1:EFSB7Zo9Eg91v7MJPVsifUysc/wPdN+NOnVe6bWbdBM= github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310 h1:BUAU3CGlLvorLI26FmByPp2eC2qla6E1Tw+scpcg/to= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/armon/go-radix v1.0.0 h1:F4z6KzEeeQIMeLFa97iZU6vupzoecKdU5TX24SNppXI= github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/asaskevich/govalidator v0.0.0-20180319081651-7d2e70ef918f/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY= github.com/aws/aws-sdk-go v1.15.24/go.mod h1:mFuSZ37Z9YOHbQEwBWztmVzqXrEkub65tZoCYDt7FT0= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= 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.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0 h1:ByYyxL9InA1OWqxJqqp2A5pYHUrCiAL6K3J+LKSsQkY= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932/go.mod h1:NOuUCSz6Q9T7+igc/hlvDOUdtWKryOrtFyIVABv/p7k= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/cenkalti/backoff v2.1.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.0 h1:yTUvW7Vhb89inJ+8irsUqiWjh8iT6sQPZiQzI6ReGkA= +github.com/cespare/xxhash/v2 v2.1.0/go.mod h1:dgIUBU3pDso/gPgZ1osOZ0iQf77oPR28Tjxl5dIMyVM= github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible h1:C29Ae4G5GtYyYMm1aztcyj/J5ckgJm2zwdDajFbx1NY= github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= github.com/circonus-labs/circonusllhist v0.1.3 h1:TJH+oke8D16535+jHExHj4nQvzlZrj7ug5D7I/orNUA= @@ -54,36 +66,52 @@ github.com/docker/go-units v0.3.3/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDD github.com/duosecurity/duo_api_golang v0.0.0-20190308151101-6c680f768e74/go.mod h1:UqXY1lYT/ERa4OEAywUqdok1T4RCRdArkhic1Opuavo= github.com/elazarl/go-bindata-assetfs v0.0.0-20160803192304-e1a2a7ec64b0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= github.com/envoyproxy/go-control-plane v0.8.0/go.mod h1:GSSbY9P1neVhdY7G4wu+IK1rk/dqhiCC/4ExuWJZVuk= +github.com/envoyproxy/go-control-plane v0.9.0 h1:67WMNTvGrl7V1dWdKCeTwxDr7nio9clKoTlLhwIPnT4= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/protoc-gen-validate v0.0.14/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/envoyproxy/protoc-gen-validate v0.1.0 h1:EQciDnbrYxy13PgWoY8AqoxGiPrpgBZ1R8UNe3ddc+A= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= +github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/fatih/structs v0.0.0-20180123065059-ebf56d35bba7/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-ini/ini v1.25.4/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-ldap/ldap v3.0.2+incompatible/go.mod h1:qfd9rJvER9Q0/D/Sqn1DfHRoBp40uXYvFoEVrNEPqRc= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= github.com/go-sql-driver/mysql v0.0.0-20180618115901-749ddf1598b4/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-test/deep v1.0.2/go.mod h1:wGDj63lr65AM2AQyKZd/NYHGb0R+1RLqB8NKt3aSFNA= github.com/gocql/gocql v0.0.0-20180617115710-e06f8c1bcd78/go.mod h1:4Fw1eo5iaEhDUs8XyuhSVCVy52Jq3L+/3GJgYkwc+/0= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= +github.com/gogo/protobuf v1.1.1 h1:72R+M5VuhED/KujmZVcIquuo8mBgX4oVda//DQb3PXo= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= 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/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1 h1:YF8+flBXS5eO826T4nzqPrxfhQThhXl0YzfuUPu4SBg= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +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/snappy v0.0.0-20170215233205-553a64147049/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 h1:964Od4U6p2jUkFxvCydnIczKteheJEzHRToSGK3Bnlw= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0 h1:crn/baboCvb5fXaQ0IJ1SGTsTVrWpDsCWC8EGETZijY= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= github.com/google/go-querystring v0.0.0-20170111101155-53e6ce116135/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= -github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf h1:+RRA9JqSOZFfKrOeqr2z77+8R2RKyh8PG66dcu1V0ck= github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI= +github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/googleapis/gnostic v0.2.0/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY= github.com/gophercloud/gophercloud v0.0.0-20180828235145-f29afc2cceca/go.mod h1:3WdhXV3rUYy9p6AUW8d94kr+HS62Y4VL9mBnFxsD8q4= github.com/gopherjs/gopherjs v0.0.0-20180825215210-0210a2f0f73c/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -94,8 +122,9 @@ github.com/hashicorp/consul v1.6.1 h1:ISPgwOO8/vPYrCXQNyx63eJAYjPGRnmFsXK7aj2XIC github.com/hashicorp/consul v1.6.1/go.mod h1:Veio4vVqbH2SuyGK7tpblfu8d4d/OnPs6smO4nnXk6w= github.com/hashicorp/consul/api v1.2.0 h1:oPsuzLp2uk7I7rojPKuncWbZ+m5TMoD4Ivs+2Rkeh4Y= github.com/hashicorp/consul/api v1.2.0/go.mod h1:1SIkFYi2ZTXUE5Kgt179+4hH33djo11+0Eo2XgTAtkw= -github.com/hashicorp/consul/sdk v0.2.0 h1:GWFYFmry/k4b1hEoy7kSkmU8e30GAyI4VZHk0fRxeL4= github.com/hashicorp/consul/sdk v0.2.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/consul/sdk v0.4.0 h1:zBtCfKJZcJDBvSCkQJch4ulp59m1rATFLKwNo/LYY30= +github.com/hashicorp/consul/sdk v0.4.0/go.mod h1:fY08Y9z5SvJqevyZNy6WWPXiG3KwBPAvlcdx16zZ0fM= github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-bexpr v0.1.2 h1:ijMXI4qERbzxbCnkxmfUtwMyjrrk3y+Vt0MxojNCbBs= @@ -105,10 +134,14 @@ github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtng github.com/hashicorp/go-cleanhttp v0.5.1 h1:dH3aiDG9Jvb5r5+bYHsikaOUIpcM0xvgMXVoDkXMzJM= github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-discover v0.0.0-20190403160810-22221edb15cd/go.mod h1:ueUgD9BeIocT7QNuvxSyJyPAM9dfifBcaWmeybb67OY= -github.com/hashicorp/go-hclog v0.9.1 h1:9PZfAcVEvez4yhLH2TBU64/h/z4xlFI80cWXRrxuKuM= github.com/hashicorp/go-hclog v0.9.1/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ= -github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= +github.com/hashicorp/go-hclog v0.9.2 h1:CG6TE5H9/JXsFWJCfoIVpKFIkFe6ysEuHirp4DxCsHI= +github.com/hashicorp/go-hclog v0.9.2/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ= +github.com/hashicorp/go-hclog v0.12.0 h1:d4QkX8FRTYaKaCZBoXYY8zJX2BXjWxurN/GA2tkrmZM= +github.com/hashicorp/go-hclog v0.12.0/go.mod h1:whpDNt7SSdeAju8AWKIWsul05p54N/39EeqMAyrmvFQ= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-immutable-radix v1.1.0 h1:vN9wG1D6KG6YHRTWr8512cxGOVgTMEfgEdSj/hr8MPc= +github.com/hashicorp/go-immutable-radix v1.1.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-memdb v0.0.0-20180223233045-1289e7fffe71/go.mod h1:kbfItVoBJwCfKXDXN4YoAXjxcFVZ7MRrJzyTX6H4giE= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= @@ -117,22 +150,27 @@ github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uP github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= github.com/hashicorp/go-plugin v0.0.0-20180331002553-e8d22c780116/go.mod h1:JSqWYsict+jzcj0+xElxyrBQRPNoiWQuddnxArJ7XHQ= github.com/hashicorp/go-raftchunking v0.6.1/go.mod h1:cGlg3JtDy7qy6c/3Bu660Mic1JF+7lWqIwCFSb08fX0= -github.com/hashicorp/go-retryablehttp v0.5.3 h1:QlWt0KvWT0lq8MFppF9tsJGF+ynG7ztc2KIPhzRGk7s= github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= -github.com/hashicorp/go-rootcerts v1.0.0 h1:Rqb66Oo1X/eSV1x66xbDccZjhJigjg0+e82kpwzSwCI= +github.com/hashicorp/go-retryablehttp v0.6.3 h1:tuulM+WnToeqa05z83YLmKabZxrySOmJAd4mJ+s2Nfg= +github.com/hashicorp/go-retryablehttp v0.6.3/go.mod h1:vAew36LZh98gCBJNLH42IQ1ER/9wtLZZ8meHqQvEYWY= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-sockaddr v1.0.0 h1:GeH6tui99pF4NJgfnhp+L6+FfobzVW3Ah46sLo0ICXs= +github.com/hashicorp/go-rootcerts v1.0.1 h1:DMo4fmknnz0E0evoNYnV48RjWndOsmd6OW+09R3cEP8= +github.com/hashicorp/go-rootcerts v1.0.1/go.mod h1:pqUvnprVnM5bf7AOirdbb01K4ccR319Vf4pU3K5EGc8= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-sockaddr v1.0.2 h1:ztczhD1jLxIRjVejw8gFomI1BQZOe2WoVOu0SyteCQc= +github.com/hashicorp/go-sockaddr v1.0.2/go.mod h1:rB4wwRAUzs07qva3c5SdrY/NEtAUjGlgmH/UkBUC97A= github.com/hashicorp/go-syslog v1.0.0 h1:KaodqZuhUoZereWVIYmpUgZysurB1kBLX2j0MwMrUAE= github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE= github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-version v0.0.0-20170202080759-03c5bf6be031 h1:c3Xdf5fTpk+hqhxqCO+ymqjfUXV9+GZqNgTtlnVzDos= github.com/hashicorp/go-version v0.0.0-20170202080759-03c5bf6be031/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go-version v1.1.0 h1:bPIoEKD27tNdebFGGxxYwcL4nepeY4j1QP23PFRGzg0= +github.com/hashicorp/go-version v1.1.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= -github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1 h1:0hERBMJE1eitiLkihrMvRVBYAkpHzc/J3QdDN+dAcgU= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/hcl v0.0.0-20180906183839-65a6292f0157/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= 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= @@ -163,34 +201,52 @@ github.com/jefferai/jsonx v0.0.0-20160721235117-9cc31c3135ee/go.mod h1:N0t2vlmpe github.com/jmespath/go-jmespath v0.0.0-20160202185014-0b12d6b521d8/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/joyent/triton-go v0.0.0-20180628001255-830d2b111e62/go.mod h1:U+RSyWxWd04xTqnuOQxnai7XGS2PrPY2cfGoDKtMHjA= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.7 h1:KfgG9LzI+pYjr4xvmz/5H4FXjokeP+rlHLhv3iH62Fo= +github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/keybase/go-crypto v0.0.0-20180614160407-5114a9a81e1b/go.mod h1:ghbZscTyKdM07+Fw3KSi0hcJm+AlEUWj8QLlPtijN/M= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/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/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/lib/pq v0.0.0-20180523175426-90697d60dd84/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-isatty v0.0.3 h1:ns/ykhmWi7G9O+8a448SecJU3nSMBXJfqQkl0upE1jI= +github.com/mattn/go-colorable v0.1.4 h1:snbPLB8fVfU9iwbbo30TPtbLRzwWu6aJS6Xh4eaaviA= +github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= +github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= +github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= 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/miekg/dns v1.0.14 h1:9jZdLNd/P4+SfEJ0TNyxYpsK8N4GtfylBLqtbYN1sbA= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/miekg/dns v1.1.22 h1:Jm64b3bO9kP43ddLjL2EY3Io6bmy1qGb9Xxz6TqS6rc= +github.com/miekg/dns v1.1.22/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso= github.com/mitchellh/cli v1.0.0 h1:iGBIsUe3+HZ/AD/Vd7DErOt5sU9fa8Uj7A2s1aggv1Y= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= github.com/mitchellh/copystructure v1.0.0 h1:Laisrj+bAB6b/yJwB5Bt3ITZhGJdqmxquMKeZ+mmkFQ= github.com/mitchellh/copystructure v1.0.0/go.mod h1:SNtv71yrdKgLRyLFxmLdkAbkKEFWgYaq1OVrnRcwhnw= -github.com/mitchellh/go-homedir v1.0.0 h1:vKb8ShqSby24Yrqr/yDYkuFz8d0WUjys40rvnGC8aR0= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +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-testing-interface v1.0.0 h1:fzU/JVNcaqHQEcVFAKeR41fkiLdIPrefOvVG1VZ96U0= github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= +github.com/mitchellh/go-wordwrap v1.0.0/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo= github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/hashstructure v0.0.0-20170609045927-2bca23e0e452 h1:hOY53G+kBFhbYFpRVxHl5eS7laP6B1+Cq+Z9Dry1iMU= github.com/mitchellh/hashstructure v0.0.0-20170609045927-2bca23e0e452/go.mod h1:QjSHrPWS+BGUVBYkbTZWEnOh3G1DutKwClXU/ABz6AQ= +github.com/mitchellh/hashstructure v1.0.0 h1:ZkRJX1CyOoTkar7p/mLS5TZU4nJ1Rn/F8u9dGS02Q3Y= +github.com/mitchellh/hashstructure v1.0.0/go.mod h1:QjSHrPWS+BGUVBYkbTZWEnOh3G1DutKwClXU/ABz6AQ= github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= @@ -198,8 +254,13 @@ github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= github.com/mitchellh/reflectwalk v1.0.1 h1:FVzMWA5RllMAKIdUSC8mdWo3XtwoecrH79BY70sEEpE= github.com/mitchellh/reflectwalk v1.0.1/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nicolai86/scaleway-sdk v1.10.2-0.20180628010248-798f60e20bb2/go.mod h1:TLb2Sg7HQcgGdloNxkrmtgDNR9uVYF3lfdFIN4Ro6Sk= github.com/oklog/run v0.0.0-20180308005104-6934b124db28/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -222,22 +283,37 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/posener/complete v1.1.1 h1:ccV59UEOTzVDnDUEFdT95ZzHVZ+5+158q8+SJb2QV5w= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= -github.com/prometheus/client_golang v0.9.2 h1:awm861/B8OKDd2I/6o1dy3ra4BamzKhYOiGItCeZ740= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 h1:idejC8f05m9MGOsuEi1ATq9shN03HrxNkD/luQvxCv8= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.2.1 h1:JnMpQc6ppsNgw9QPAGF6Dod479itz7lvlsMzzNayLOI= +github.com/prometheus/client_golang v1.2.1/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/common v0.0.0-20181126121408-4724e9255275 h1:PnBWHBf+6L0jOqq0gIVUe6Yk0/QMZ640k6NvkxcBf+8= +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/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a h1:9a8MnZMP0X2nLJdBg+pBmGgkJlSaKC2KaQmTCk1XDtE= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.7.0 h1:L+1lyG48J1zAQXA3RBX/nG/B3gjlHq0zTt2tlbJLyCY= +github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= +github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +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/renier/xmlrpc v0.0.0-20170708154548-ce4a1a486c03/go.mod h1:gRAiPF5C5Nd0eyyRdqIu9qTiFSoZzpTq727b5B8fkkU= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +github.com/ryanuber/columnize v2.1.0+incompatible h1:j1Wcmh8OrK4Q7GXY+V7SVSY8nUWQxHW5TkBe7YUl+2s= +github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/go-glob v0.0.0-20170128012129-256dc444b735/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/shirou/gopsutil v0.0.0-20181107111621-48177ef5f880/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/sirupsen/logrus v1.0.6/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/smartystreets/assertions v0.0.0-20180820201707-7c9eb446e3cf/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20180222194500-ef6db91d284a/go.mod h1:XDJAKZRPZ1CvBcN2aX5YOUTYGHki24fSF0Iv48Ibg0s= github.com/softlayer/softlayer-go v0.0.0-20180806151055-260589d94c7d/go.mod h1:Cw4GTlQccdRGSEf6KiMju767x0NEHE0YIVPJSaXjlsw= @@ -249,26 +325,41 @@ github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+ github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/tent/http-link-go v0.0.0-20130702225549-ac974c61c2f9/go.mod h1:RHkNRtSLfOK7qBTHaeSX1D6BNpI3qw7NTxsmNr4RvN8= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926 h1:G3dpKMzFDjgEh2q1Z7zUUtKa8ViPtH+ocF0bE0g00O8= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= github.com/vmware/govmomi v0.18.0/go.mod h1:URlwyTFZX72RmxtxuaFL2Uj3fD1JTvZdx59bHWk6aFU= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4 h1:HuIa8hRrWRSrqYzx1qI49NNxhdi2PrY7gxVSq1JjLDc= golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/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-20191002192127-34f69633bfdc h1:c0o/qxkaO2LF5t6fQrT4b5hzyggAkLLlCUjqfRxd8Q4= +golang.org/x/crypto v0.0.0-20191002192127-34f69633bfdc/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= 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-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +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-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c h1:uOCk1iQW6Vc18bnC13MfzScl+wdKBmM9Y9kU7Z83/lw= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 h1:dfGZHvZk057jK2MCeWus/TowKpJ8y4AmooUzdBSR9GU= +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-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191003171128-d98b1b443823 h1:Ypyv6BNJh07T1pUSrehkLemqPKXhus2MkfktJ91kRh4= +golang.org/x/net v0.0.0-20191003171128-d98b1b443823/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/oauth2 v0.0.0-20170807180024-9a379c6b3e95/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -278,14 +369,27 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEha golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/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-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 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-20190508220229-2d0786266e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190523142557-0e01d883c5c5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a h1:aYOabOQFp6Vj6W1F80affTUvO9UxmJRx8K0gsfABByQ= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/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/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191008105621-543471e840be/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191010194322-b09406accb47 h1:/XfQ9z7ib8eEJX2hdgFTZJ/ntt0swNk5oYBziWeTCvY= +golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9 h1:1/DFK4b7JH8DmkqhUk48onnSfrPzImPoVxuomtbT2nk= +golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= @@ -293,18 +397,28 @@ golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/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-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/api v0.0.0-20180829000535-087779f1d2c9/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8 h1:Nw54tB0rB7hY/N0NQvRW8DG4Yk3Q6T9cu9RcFQDu1tc= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +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.23.0 h1:AzbTB6ux+okLTzP8Ru1Xs41C303zdcfEht7MQnYJt5A= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.0 h1:ItERT+UbGdX+s4u+nQNlVM/Q7cbmf7icKfvzbWqVtq0= +google.golang.org/grpc v1.25.0/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d/go.mod h1:cuepJuh7vyXfUyUwEgHQXw849cJrilpS5NeIjOWESAw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/gemnasium/logrus-airbrake-hook.v2 v2.1.2/go.mod h1:Xk6kEKp8OKb+X14hQBKWaSkCsqBpgog8nAV2xsGOxlo= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= @@ -313,6 +427,10 @@ gopkg.in/ory-am/dockertest.v3 v3.3.4/go.mod h1:s9mmoLkaGeAh97qygnNj4xWkiN7e1SKek gopkg.in/square/go-jose.v2 v2.3.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= 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/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/leader.go b/leader.go index 9424eef..6d69fa1 100644 --- a/leader.go +++ b/leader.go @@ -103,6 +103,16 @@ func nodeLists(nodes []*api.Node, insts []*api.ServiceEntry, return healthNodes, pingNodes } +func (a *Agent) commitOps(ops api.KVTxnOps) bool { + success, results, _, err := a.client.KV().Txn(ops, nil) + if err != nil || !success { + a.logger.Printf("[ERR] Error writing state to KV store: %v, %v", err, results) + // Try again after the wait because we got an error. + return false + } + return true +} + // computeWatchedNodes watches both the list of registered ESM instances and // the list of external nodes registered in Consul and decides which nodes each // ESM instance should be in charge of, writing the output to the KV store. @@ -119,17 +129,26 @@ func (a *Agent) computeWatchedNodes(stopCh <-chan struct{}) { var prevHealthNodes map[string][]string var prevPingNodes map[string][]string - firstRun := true + // Avoid blocking on first pass + retryTimer := time.After(0) + for { - if !firstRun { - select { - case <-stopCh: - return - case externalNodes = <-nodeCh: - case healthyInstances = <-instanceCh: - } + select { + case <-stopCh: + return + case externalNodes = <-nodeCh: + case healthyInstances = <-instanceCh: + case <-retryTimer: + } + + // Next time through block until either nodes or instances are updated + retryTimer = nil + + // Wait for some instances to become available, if there are none, then there isn't anything we can do + if len(healthyInstances) == 0 { + retryTimer = time.After(retryTime) + continue } - firstRun = false healthNodes, pingNodes := nodeLists(externalNodes, healthyInstances) @@ -151,14 +170,20 @@ func (a *Agent) computeWatchedNodes(stopCh <-chan struct{}) { Value: bytes, } ops = append(ops, op) + + // Flush any ops if we're nearing a transaction limit + if len(ops) >= maximumTransactionSize { + if !a.commitOps(ops) { + retryTimer = time.After(retryTime) + continue + } + ops = api.KVTxnOps{} + } } - success, results, _, err := a.client.KV().Txn(ops, nil) - if err != nil || !success { - a.logger.Printf("[ERR] Error writing state to KV store: %v, %v", - err, results) - // Try again after the wait because we got an error. - firstRun = true - time.Sleep(retryTime) + + // Final flush for ops + if !a.commitOps(ops) { + retryTimer = time.After(retryTime) continue } @@ -208,6 +233,8 @@ func (a *Agent) watchExternalNodes(nodeCh chan []*api.Node, stopCh <-chan struct opts.WaitIndex = meta.LastIndex + a.logger.Printf("[INFO] Updating external node list, set to %d items", len(externalNodes)) + nodeCh <- externalNodes } } diff --git a/main.go b/main.go index 370da19..8952691 100644 --- a/main.go +++ b/main.go @@ -90,7 +90,10 @@ func main() { ui.Info(fmt.Sprintf(" Service Tag: %q", config.Tag)) ui.Info(fmt.Sprintf(" Service ID: %q", agent.serviceID())) ui.Info(fmt.Sprintf("Node Reconnect Timeout: %q", config.NodeReconnectTimeout.String())) - + ui.Info(fmt.Sprintf(" No Redundant updates: %t", config.DisableRedundantStatusUpdates)) + ui.Info(fmt.Sprintf(" Disable coordinates: %t", config.DisableCoordinateUpdates)) + ui.Info(fmt.Sprintf(" Statsd address: %q", config.Telemetry.StatsdAddr)) + ui.Info(fmt.Sprintf(" Metrix prefix: %q", config.Telemetry.MetricsPrefix)) ui.Info("") ui.Output("Log data will now stream in as it occurs:\n")