New config parser, HCL support, multiple bind addrs (#3480)

* new config parser for agent

This patch implements a new config parser for the consul agent which
makes the following changes to the previous implementation:

 * add HCL support
 * all configuration fragments in tests and for default config are
   expressed as HCL fragments
 * HCL fragments can be provided on the command line so that they
   can eventually replace the command line flags.
 * HCL/JSON fragments are parsed into a temporary Config structure
   which can be merged using reflection (all values are pointers).
   The existing merge logic of overwrite for values and append
   for slices has been preserved.
 * A single builder process generates a typed runtime configuration
   for the agent.

The new implementation is more strict and fails in the builder process
if no valid runtime configuration can be generated. Therefore,
additional validations in other parts of the code should be removed.

The builder also pre-computes all required network addresses so that no
address/port magic should be required where the configuration is used
and should therefore be removed.

* Upgrade github.com/hashicorp/hcl to support int64

* improve error messages

* fix directory permission test

* Fix rtt test

* Fix ForceLeave test

* Skip performance test for now until we know what to do

* Update github.com/hashicorp/memberlist to update log prefix

* Make memberlist use the default logger

* improve config error handling

* do not fail on non-existing data-dir

* experiment with non-uniform timeouts to get a handle on stalled leader elections

* Run tests for packages separately to eliminate the spurious port conflicts

* refactor private address detection and unify approach for ipv4 and ipv6.

Fixes #2825

* do not allow unix sockets for DNS

* improve bind and advertise addr error handling

* go through builder using test coverage

* minimal update to the docs

* more coverage tests fixed

* more tests

* fix makefile

* cleanup

* fix port conflicts with external port server 'porter'

* stop test server on error

* do not run api test that change global ENV concurrently with the other tests

* Run remaining api tests concurrently

* no need for retry with the port number service

* monkey patch race condition in go-sockaddr until we understand why that fails

* monkey patch hcl decoder race condidtion until we understand why that fails

* monkey patch spurious errors in strings.EqualFold from here

* add test for hcl decoder race condition. Run with go test -parallel 128

* Increase timeout again

* cleanup

* don't log port allocations by default

* use base command arg parsing to format help output properly

* handle -dc deprecation case in Build

* switch autopilot.max_trailing_logs to int

* remove duplicate test case

* remove unused methods

* remove comments about flag/config value inconsistencies

* switch got and want around since the error message was misleading.

* Removes a stray debug log.

* Removes a stray newline in imports.

* Fixes TestACL_Version8.

* Runs go fmt.

* Adds a default case for unknown address types.

* Reoders and reformats some imports.

* Adds some comments and fixes typos.

* Reorders imports.

* add unix socket support for dns later

* drop all deprecated flags and arguments

* fix wrong field name

* remove stray node-id file

* drop unnecessary patch section in test

* drop duplicate test

* add test for LeaveOnTerm and SkipLeaveOnInt in client mode

* drop "bla" and add clarifying comment for the test

* split up tests to support enterprise/non-enterprise tests

* drop raft multiplier and derive values during build phase

* sanitize runtime config reflectively and add test

* detect invalid config fields

* fix tests with invalid config fields

* use different values for wan sanitiziation test

* drop recursor in favor of recursors

* allow dns_config.udp_answer_limit to be zero

* make sure tests run on machines with multiple ips

* Fix failing tests in a few more places by providing a bind address in the test

* Gets rid of skipped TestAgent_CheckPerformanceSettings and adds case for builder.

* Add porter to server_test.go to make tests there less flaky

* go fmt
This commit is contained in:
Frank Schröder 2017-09-25 20:40:42 +02:00 committed by James Phillips
parent 78ce956385
commit 69a088ca85
117 changed files with 8849 additions and 6546 deletions

View File

@ -34,12 +34,14 @@ bin: tools
dev: vendorfmt dev-build
dev-build:
@echo "--> Building consul"
mkdir -p pkg/$(GOOS)_$(GOARCH)/ bin/
go install -ldflags '$(GOLDFLAGS)' -tags '$(GOTAGS)'
cp $(GOPATH)/bin/consul bin/
cp $(GOPATH)/bin/consul pkg/$(GOOS)_$(GOARCH)
vendorfmt:
@echo "--> Formatting vendor/vendor.json"
test -x $(GOPATH)/bin/vendorfmt || go get -u github.com/magiconair/vendorfmt/cmd/vendorfmt
vendorfmt
@ -56,22 +58,34 @@ cov:
gocov test $(GOFILES) | gocov-html > /tmp/coverage.html
open /tmp/coverage.html
test: dev-build vet
test: other-consul porter dev-build vet
@echo "--> Running go test"
@rm -f test.log exit-code
go test -tags '$(GOTAGS)' -i ./...
go test $(GOTEST_FLAGS) -tags '$(GOTAGS)' -timeout 7m -v ./... 2>&1 >test.log ; echo $$? > exit-code
@echo "Exit code: `cat exit-code`" >> test.log
@echo "----"
porter go test $(GOTEST_FLAGS) -tags '$(GOTAGS)' -timeout 5m -v ./... &>test.log ; echo $$? > exit-code
@echo "Exit code: $$(cat exit-code)" >> test.log
@grep -A5 'DATA RACE' test.log || true
@grep -A10 'panic: test timed out' test.log || true
@grep '^PASS' test.log | uniq || true
@grep -A1 -- '--- SKIP:' test.log || true
@grep -A1 -- '--- FAIL:' test.log || true
@grep '^FAIL' test.log || true
@test "$$TRAVIS" == "true" && cat test.log || true
@exit $$(cat exit-code)
@if [ "$$(cat exit-code)" == "0" ] ; then echo "PASS" ; exit 0 ; else exit 1 ; fi
test-race:
$(MAKE) GOTEST_FLAGS=-race
other-consul:
@echo "--> Checking for other consul instances"
@if ps -ef | grep 'consul agent' | grep -v grep ; then \
echo "Found other running consul agents. This may affect your tests." ; \
exit 1 ; \
fi
porter:
@echo "--> Building port number service..."
go install github.com/hashicorp/consul/test/porter/cmd/porter
cover:
go test $(GOFILES) --cover

View File

@ -7,6 +7,7 @@ import (
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/golang-lru"
@ -67,7 +68,7 @@ type aclManager struct {
}
// newACLManager returns an ACL manager based on the given config.
func newACLManager(config *Config) (*aclManager, error) {
func newACLManager(config *config.RuntimeConfig) (*aclManager, error) {
// Set up the cache from ID to ACL (we don't cache policies like the
// servers; only one level).
acls, err := lru.New2Q(aclCacheSize)
@ -218,7 +219,7 @@ func (m *aclManager) lookupACL(a *Agent, id string) (acl.ACL, error) {
// and some is informative (e.g. catalog and health).
func (a *Agent) resolveToken(id string) (acl.ACL, error) {
// Disable ACLs if version 8 enforcement isn't enabled.
if !(*a.config.ACLEnforceVersion8) {
if !a.config.ACLEnforceVersion8 {
return nil, nil
}

View File

@ -33,9 +33,9 @@ func makeTestACL(t *testing.T, srv *HTTPServer) string {
func TestACL_Bootstrap(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLMasterToken = ""
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_master_token = ""
`)
defer a.Shutdown()
tests := []struct {
@ -278,7 +278,7 @@ func TestACL_List(t *testing.T) {
func TestACLReplicationStatus(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), TestACLConfig())
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/acl/replication", nil)

View File

@ -2,11 +2,13 @@ package agent
import (
"fmt"
"os"
"strings"
"testing"
"time"
rawacl "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/testutil"
"github.com/hashicorp/consul/types"
@ -15,9 +17,18 @@ import (
func TestACL_Bad_Config(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDownPolicy = "nope"
cfg.DataDir = testutil.TempDir(t, "agent")
dataDir := testutil.TempDir(t, "agent")
defer os.Remove(dataDir)
cfg := TestConfig(config.Source{
Name: "acl",
Format: "hcl",
Data: `
acl_down_policy = "nope"
data_dir = "` + dataDir + `"
`,
})
// do not use TestAgent here since we want
// the agent to fail during startup.
@ -40,33 +51,61 @@ func (m *MockServer) GetPolicy(args *structs.ACLPolicyRequest, reply *structs.AC
func TestACL_Version8(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLEnforceVersion8 = Bool(false)
a := NewTestAgent(t.Name(), cfg)
defer a.Shutdown()
m := MockServer{
// With version 8 enforcement off, this should not get called.
getPolicyFn: func(*structs.ACLPolicyRequest, *structs.ACLPolicy) error {
t.Fatalf("should not have called to server")
return nil
},
}
if err := a.registerEndpoint("ACL", &m); err != nil {
t.Fatalf("err: %v", err)
}
t.Run("version 8 disabled", func(t *testing.T) {
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = false
`)
defer a.Shutdown()
if token, err := a.resolveToken("nope"); token != nil || err != nil {
t.Fatalf("bad: %v err: %v", token, err)
}
m := MockServer{
getPolicyFn: func(*structs.ACLPolicyRequest, *structs.ACLPolicy) error {
t.Fatalf("should not have called to server")
return nil
},
}
if err := a.registerEndpoint("ACL", &m); err != nil {
t.Fatalf("err: %v", err)
}
if token, err := a.resolveToken("nope"); token != nil || err != nil {
t.Fatalf("bad: %v err: %v", token, err)
}
})
t.Run("version 8 enabled", func(t *testing.T) {
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
var called bool
m := MockServer{
getPolicyFn: func(*structs.ACLPolicyRequest, *structs.ACLPolicy) error {
called = true
return fmt.Errorf("token not found")
},
}
if err := a.registerEndpoint("ACL", &m); err != nil {
t.Fatalf("err: %v", err)
}
if _, err := a.resolveToken("nope"); err != nil {
t.Fatalf("err: %v", err)
}
if !called {
t.Fatalf("bad")
}
})
}
func TestACL_Disabled(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLDisabledTTL = 10 * time.Millisecond
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_disabled_ttl = "10ms"
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{
@ -103,7 +142,7 @@ func TestACL_Disabled(t *testing.T) {
// Wait the waiting period and make sure it checks again. Do a few tries
// to make sure we don't think it's disabled.
time.Sleep(2 * cfg.ACLDisabledTTL)
time.Sleep(2 * 10 * time.Millisecond)
for i := 0; i < 10; i++ {
_, err := a.resolveToken("nope")
if !rawacl.IsErrNotFound(err) {
@ -117,10 +156,10 @@ func TestACL_Disabled(t *testing.T) {
func TestACL_Special_IDs(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
cfg.ACLAgentMasterToken = "towel"
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
acl_agent_master_token = "towel"
`)
defer a.Shutdown()
m := MockServer{
@ -159,10 +198,10 @@ func TestACL_Special_IDs(t *testing.T) {
if acl == nil {
t.Fatalf("should not be nil")
}
if !acl.AgentRead(cfg.NodeName) {
if !acl.AgentRead(a.config.NodeName) {
t.Fatalf("should be able to read agent")
}
if !acl.AgentWrite(cfg.NodeName) {
if !acl.AgentWrite(a.config.NodeName) {
t.Fatalf("should be able to write agent")
}
if !acl.NodeRead("hello") {
@ -175,11 +214,10 @@ func TestACL_Special_IDs(t *testing.T) {
func TestACL_Down_Deny(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLDownPolicy = "deny"
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_down_policy = "deny"
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{
@ -199,18 +237,17 @@ func TestACL_Down_Deny(t *testing.T) {
if acl == nil {
t.Fatalf("should not be nil")
}
if acl.AgentRead(cfg.NodeName) {
if acl.AgentRead(a.config.NodeName) {
t.Fatalf("should deny")
}
}
func TestACL_Down_Allow(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLDownPolicy = "allow"
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_down_policy = "allow"
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{
@ -230,18 +267,17 @@ func TestACL_Down_Allow(t *testing.T) {
if acl == nil {
t.Fatalf("should not be nil")
}
if !acl.AgentRead(cfg.NodeName) {
if !acl.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
}
func TestACL_Down_Extend(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLDownPolicy = "extend-cache"
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_down_policy = "extend-cache"
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{
@ -252,7 +288,7 @@ func TestACL_Down_Extend(t *testing.T) {
Policy: &rawacl.Policy{
Agents: []*rawacl.AgentPolicy{
&rawacl.AgentPolicy{
Node: cfg.NodeName,
Node: a.config.NodeName,
Policy: "read",
},
},
@ -272,10 +308,10 @@ func TestACL_Down_Extend(t *testing.T) {
if acl == nil {
t.Fatalf("should not be nil")
}
if !acl.AgentRead(cfg.NodeName) {
if !acl.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
if acl.AgentWrite(cfg.NodeName) {
if acl.AgentWrite(a.config.NodeName) {
t.Fatalf("should deny")
}
@ -290,10 +326,10 @@ func TestACL_Down_Extend(t *testing.T) {
if acl == nil {
t.Fatalf("should not be nil")
}
if acl.AgentRead(cfg.NodeName) {
if acl.AgentRead(a.config.NodeName) {
t.Fatalf("should deny")
}
if acl.AgentWrite(cfg.NodeName) {
if acl.AgentWrite(a.config.NodeName) {
t.Fatalf("should deny")
}
@ -306,20 +342,19 @@ func TestACL_Down_Extend(t *testing.T) {
if acl == nil {
t.Fatalf("should not be nil")
}
if !acl.AgentRead(cfg.NodeName) {
if !acl.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
if acl.AgentWrite(cfg.NodeName) {
if acl.AgentWrite(a.config.NodeName) {
t.Fatalf("should deny")
}
}
func TestACL_Cache(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{
@ -331,7 +366,7 @@ func TestACL_Cache(t *testing.T) {
Policy: &rawacl.Policy{
Agents: []*rawacl.AgentPolicy{
&rawacl.AgentPolicy{
Node: cfg.NodeName,
Node: a.config.NodeName,
Policy: "read",
},
},
@ -352,10 +387,10 @@ func TestACL_Cache(t *testing.T) {
if rule == nil {
t.Fatalf("should not be nil")
}
if !rule.AgentRead(cfg.NodeName) {
if !rule.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
if rule.AgentWrite(cfg.NodeName) {
if rule.AgentWrite(a.config.NodeName) {
t.Fatalf("should deny")
}
if rule.NodeRead("nope") {
@ -374,10 +409,10 @@ func TestACL_Cache(t *testing.T) {
if rule == nil {
t.Fatalf("should not be nil")
}
if !rule.AgentRead(cfg.NodeName) {
if !rule.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
if rule.AgentWrite(cfg.NodeName) {
if rule.AgentWrite(a.config.NodeName) {
t.Fatalf("should deny")
}
if rule.NodeRead("nope") {
@ -403,7 +438,7 @@ func TestACL_Cache(t *testing.T) {
Policy: &rawacl.Policy{
Agents: []*rawacl.AgentPolicy{
&rawacl.AgentPolicy{
Node: cfg.NodeName,
Node: a.config.NodeName,
Policy: "write",
},
},
@ -419,10 +454,10 @@ func TestACL_Cache(t *testing.T) {
if rule == nil {
t.Fatalf("should not be nil")
}
if !rule.AgentRead(cfg.NodeName) {
if !rule.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
if !rule.AgentWrite(cfg.NodeName) {
if !rule.AgentWrite(a.config.NodeName) {
t.Fatalf("should allow")
}
if rule.NodeRead("nope") {
@ -449,10 +484,10 @@ func TestACL_Cache(t *testing.T) {
if rule == nil {
t.Fatalf("should not be nil")
}
if !rule.AgentRead(cfg.NodeName) {
if !rule.AgentRead(a.config.NodeName) {
t.Fatalf("should allow")
}
if !rule.AgentWrite(cfg.NodeName) {
if !rule.AgentWrite(a.config.NodeName) {
t.Fatalf("should allow")
}
if rule.NodeRead("nope") {
@ -499,10 +534,9 @@ func catalogPolicy(req *structs.ACLPolicyRequest, reply *structs.ACLPolicy) erro
func TestACL_vetServiceRegister(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}
@ -545,10 +579,9 @@ func TestACL_vetServiceRegister(t *testing.T) {
func TestACL_vetServiceUpdate(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}
@ -581,10 +614,9 @@ func TestACL_vetServiceUpdate(t *testing.T) {
func TestACL_vetCheckRegister(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}
@ -664,10 +696,9 @@ func TestACL_vetCheckRegister(t *testing.T) {
func TestACL_vetCheckUpdate(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}
@ -720,10 +751,9 @@ func TestACL_vetCheckUpdate(t *testing.T) {
func TestACL_filterMembers(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}
@ -756,10 +786,9 @@ func TestACL_filterMembers(t *testing.T) {
func TestACL_filterServices(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}
@ -787,10 +816,9 @@ func TestACL_filterServices(t *testing.T) {
func TestACL_filterChecks(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLEnforceVersion8 = Bool(true)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_enforce_version_8 = true
`)
defer a.Shutdown()
m := MockServer{catalogPolicy}

View File

@ -20,6 +20,7 @@ import (
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/consul"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/systemd"
@ -83,7 +84,7 @@ type notifier interface {
// requests to other Consul servers.
type Agent struct {
// config is the agent configuration.
config *Config
config *config.RuntimeConfig
// Used for writing our logs
logger *log.Logger
@ -164,15 +165,9 @@ type Agent struct {
endpoints map[string]string
endpointsLock sync.RWMutex
// dnsAddr is the address the DNS server binds to
dnsAddrs []ProtoAddr
// dnsServer provides the DNS API
dnsServers []*DNSServer
// httpAddrs are the addresses per protocol the HTTP server binds to
httpAddrs []ProtoAddr
// httpServers provides the HTTP API on various endpoints
httpServers []*HTTPServer
@ -189,21 +184,13 @@ type Agent struct {
tokens *token.Store
}
func New(c *Config) (*Agent, error) {
func New(c *config.RuntimeConfig) (*Agent, error) {
if c.Datacenter == "" {
return nil, fmt.Errorf("Must configure a Datacenter")
}
if c.DataDir == "" && !c.DevMode {
return nil, fmt.Errorf("Must configure a DataDir")
}
dnsAddrs, err := c.DNSAddrs()
if err != nil {
return nil, fmt.Errorf("Invalid DNS bind address: %s", err)
}
httpAddrs, err := c.HTTPAddrs()
if err != nil {
return nil, fmt.Errorf("Invalid HTTP bind address: %s", err)
}
acls, err := newACLManager(c)
if err != nil {
return nil, err
@ -225,8 +212,6 @@ func New(c *Config) (*Agent, error) {
retryJoinCh: make(chan error),
shutdownCh: make(chan struct{}),
endpoints: make(map[string]string),
dnsAddrs: dnsAddrs,
httpAddrs: httpAddrs,
tokens: new(token.Store),
}
@ -269,7 +254,7 @@ func (a *Agent) Start() error {
consulCfg.ServerUp = a.state.ConsulServerUp
// Setup either the client or the server.
if c.Server {
if c.ServerMode {
server, err := consul.NewServerLogger(consulCfg, a.logger, a.tokens)
if err != nil {
return fmt.Errorf("Failed to start Consul server: %v", err)
@ -322,14 +307,14 @@ func (a *Agent) Start() error {
// create listeners and unstarted servers
// see comment on listenHTTP why we are doing this
httpln, err := a.listenHTTP(a.httpAddrs)
httpln, err := a.listenHTTP()
if err != nil {
return err
}
// start HTTP servers
// start HTTP and HTTPS servers
for _, l := range httpln {
srv := NewHTTPServer(l.Addr().String(), a)
srv := NewHTTPServer(l.Addr(), a)
if err := a.serveHTTP(l, srv); err != nil {
return err
}
@ -349,10 +334,8 @@ func (a *Agent) Start() error {
}
func (a *Agent) listenAndServeDNS() error {
notif := make(chan ProtoAddr, len(a.dnsAddrs))
for _, p := range a.dnsAddrs {
p := p // capture loop var
notif := make(chan net.Addr, len(a.config.DNSAddrs))
for _, addr := range a.config.DNSAddrs {
// create server
s, err := NewDNSServer(a)
if err != nil {
@ -362,22 +345,21 @@ func (a *Agent) listenAndServeDNS() error {
// start server
a.wgServers.Add(1)
go func() {
go func(addr net.Addr) {
defer a.wgServers.Done()
err := s.ListenAndServe(p.Net, p.Addr, func() { notif <- p })
err := s.ListenAndServe(addr.Network(), addr.String(), func() { notif <- addr })
if err != nil && !strings.Contains(err.Error(), "accept") {
a.logger.Printf("[ERR] agent: Error starting DNS server %s (%s): %v", p.Addr, p.Net, err)
a.logger.Printf("[ERR] agent: Error starting DNS server %s (%s): %v", addr.String(), addr.Network(), err)
}
}()
}(addr)
}
// wait for servers to be up
timeout := time.After(time.Second)
for range a.dnsAddrs {
for range a.config.DNSAddrs {
select {
case p := <-notif:
a.logger.Printf("[INFO] agent: Started DNS server %s (%s)", p.Addr, p.Net)
case addr := <-notif:
a.logger.Printf("[INFO] agent: Started DNS server %s (%s)", addr.String(), addr.Network())
continue
case <-timeout:
return fmt.Errorf("agent: timeout starting DNS servers")
@ -401,43 +383,56 @@ func (a *Agent) listenAndServeDNS() error {
//
// This approach should ultimately be refactored to the point where we just
// start the server and any error should trigger a proper shutdown of the agent.
func (a *Agent) listenHTTP(addrs []ProtoAddr) ([]net.Listener, error) {
func (a *Agent) listenHTTP() ([]net.Listener, error) {
var ln []net.Listener
for _, p := range addrs {
var l net.Listener
var err error
switch {
case p.Net == "unix":
l, err = a.listenSocket(p.Addr, a.config.UnixSockets)
start := func(proto string, addrs []net.Addr) error {
for _, addr := range addrs {
var l net.Listener
var err error
case p.Net == "tcp" && p.Proto == "http":
l, err = net.Listen("tcp", p.Addr)
switch x := addr.(type) {
case *net.UnixAddr:
l, err = a.listenSocket(x.Name)
if err != nil {
return err
}
case p.Net == "tcp" && p.Proto == "https":
var tlscfg *tls.Config
tlscfg, err = a.config.IncomingHTTPSConfig()
if err != nil {
break
case *net.TCPAddr:
l, err = net.Listen("tcp", x.String())
if err != nil {
return err
}
l = &tcpKeepAliveListener{l.(*net.TCPListener)}
if proto == "https" {
tlscfg, err := a.config.IncomingHTTPSConfig()
if err != nil {
return err
}
l = tls.NewListener(l, tlscfg)
}
default:
return fmt.Errorf("unsupported address type %T", addr)
}
l, err = tls.Listen("tcp", p.Addr, tlscfg)
default:
return nil, fmt.Errorf("%s:%s listener not supported", p.Net, p.Proto)
ln = append(ln, l)
}
return nil
}
if err != nil {
for _, l := range ln {
l.Close()
}
return nil, err
if err := start("http", a.config.HTTPAddrs); err != nil {
for _, l := range ln {
l.Close()
}
if tcpl, ok := l.(*net.TCPListener); ok {
l = &tcpKeepAliveListener{tcpl}
return nil, err
}
if err := start("https", a.config.HTTPSAddrs); err != nil {
for _, l := range ln {
l.Close()
}
ln = append(ln, l)
return nil, err
}
return ln, nil
}
@ -459,7 +454,7 @@ func (ln tcpKeepAliveListener) Accept() (c net.Conn, err error) {
return tc, nil
}
func (a *Agent) listenSocket(path string, perm FilePermissions) (net.Listener, error) {
func (a *Agent) listenSocket(path string) (net.Listener, error) {
if _, err := os.Stat(path); !os.IsNotExist(err) {
a.logger.Printf("[WARN] agent: Replacing socket %q", path)
}
@ -470,8 +465,9 @@ func (a *Agent) listenSocket(path string, perm FilePermissions) (net.Listener, e
if err != nil {
return nil, err
}
if err := setFilePermissions(path, perm); err != nil {
return nil, fmt.Errorf("Failed setting up HTTP socket: %s", err)
user, group, mode := a.config.UnixSocketUser, a.config.UnixSocketGroup, a.config.UnixSocketMode
if err := setFilePermissions(path, user, group, mode); err != nil {
return nil, fmt.Errorf("Failed setting up socket: %s", err)
}
return l, nil
}
@ -492,11 +488,11 @@ func (a *Agent) serveHTTP(l net.Listener, srv *HTTPServer) error {
if strings.Contains("*tls.listener", fmt.Sprintf("%T", l)) {
srv.proto = "https"
}
notif := make(chan string)
notif := make(chan net.Addr)
a.wgServers.Add(1)
go func() {
defer a.wgServers.Done()
notif <- srv.Addr
notif <- l.Addr()
err := srv.Serve(l)
if err != nil && err != http.ErrServerClosed {
a.logger.Print(err)
@ -506,9 +502,9 @@ func (a *Agent) serveHTTP(l net.Listener, srv *HTTPServer) error {
select {
case addr := <-notif:
if srv.proto == "https" {
a.logger.Printf("[INFO] agent: Started HTTPS server on %s", addr)
a.logger.Printf("[INFO] agent: Started HTTPS server on %s (%s)", addr.String(), addr.Network())
} else {
a.logger.Printf("[INFO] agent: Started HTTP server on %s", addr)
a.logger.Printf("[INFO] agent: Started HTTP server on %s (%s)", addr.String(), addr.Network())
}
return nil
case <-time.After(time.Second):
@ -518,32 +514,49 @@ func (a *Agent) serveHTTP(l net.Listener, srv *HTTPServer) error {
// reloadWatches stops any existing watch plans and attempts to load the given
// set of watches.
func (a *Agent) reloadWatches(cfg *Config) error {
func (a *Agent) reloadWatches(cfg *config.RuntimeConfig) error {
// Watches use the API to talk to this agent, so that must be enabled.
addrs, err := cfg.HTTPAddrs()
if err != nil {
return err
}
if len(addrs) == 0 {
if len(cfg.HTTPAddrs) == 0 {
return fmt.Errorf("watch plans require an HTTP or HTTPS endpoint")
}
// Compile the watches
var watchPlans []*watch.Plan
for _, params := range cfg.Watches {
// Parse the watches, excluding the handler
wp, err := watch.ParseExempt(params, []string{"handler"})
if err != nil {
return fmt.Errorf("Failed to parse watch (%#v): %v", params, err)
}
// Get the handler
h := wp.Exempt["handler"]
if _, ok := h.(string); h == nil || !ok {
return fmt.Errorf("Watch handler must be a string")
}
// Store the watch plan
watchPlans = append(watchPlans, wp)
}
// Stop the current watches.
for _, wp := range a.watchPlans {
wp.Stop()
}
a.watchPlans = nil
// deterine the primary http endpoint
addr := cfg.HTTPAddrs[0].String()
if cfg.HTTPAddrs[0].Network() == "unix" {
addr = "unix://" + addr
}
// Fire off a goroutine for each new watch plan.
for _, wp := range cfg.WatchPlans {
for _, wp := range watchPlans {
a.watchPlans = append(a.watchPlans, wp)
go func(wp *watch.Plan) {
wp.Handler = makeWatchHandler(a.LogOutput, wp.Exempt["handler"])
wp.LogOutput = a.LogOutput
addr := addrs[0].String()
if addrs[0].Net == "unix" {
addr = "unix://" + addr
}
if err := wp.Run(addr); err != nil {
a.logger.Printf("[ERR] Failed to run watch: %v", err)
}
@ -557,99 +570,60 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
// Start with the provided config or default config
base := consul.DefaultConfig()
// a.config.ConsulConfig, if set, is a partial configuration for the
// consul server or client. Therefore, clone and augment it but
// don't use it as base directly.
if a.config.ConsulConfig != nil {
base = new(consul.Config)
*base = *a.config.ConsulConfig
}
// This is set when the agent starts up
base.NodeID = a.config.NodeID
// Apply dev mode
base.DevMode = a.config.DevMode
// Apply performance factors
if a.config.Performance.RaftMultiplier > 0 {
base.ScaleRaft(a.config.Performance.RaftMultiplier)
}
// Override with our config
if a.config.Datacenter != "" {
base.Datacenter = a.config.Datacenter
}
if a.config.DataDir != "" {
base.DataDir = a.config.DataDir
}
if a.config.NodeName != "" {
base.NodeName = a.config.NodeName
}
if a.config.Ports.SerfLan != 0 {
base.SerfLANConfig.MemberlistConfig.BindPort = a.config.Ports.SerfLan
base.SerfLANConfig.MemberlistConfig.AdvertisePort = a.config.Ports.SerfLan
}
if a.config.Ports.SerfWan != 0 {
base.SerfWANConfig.MemberlistConfig.BindPort = a.config.Ports.SerfWan
base.SerfWANConfig.MemberlistConfig.AdvertisePort = a.config.Ports.SerfWan
}
if a.config.BindAddr != "" {
bindAddr := &net.TCPAddr{
IP: net.ParseIP(a.config.BindAddr),
Port: a.config.Ports.Server,
}
base.RPCAddr = bindAddr
// todo(fs): these are now always set in the runtime config so we can simplify this
// todo(fs): or is there a reason to keep it like that?
base.Datacenter = a.config.Datacenter
base.DataDir = a.config.DataDir
base.NodeName = a.config.NodeName
// Set the Serf configs using the old default behavior, we may
// override these in the code right below.
base.SerfLANConfig.MemberlistConfig.BindAddr = a.config.BindAddr
base.SerfWANConfig.MemberlistConfig.BindAddr = a.config.BindAddr
base.CoordinateUpdateBatchSize = a.config.ConsulCoordinateUpdateBatchSize
base.CoordinateUpdateMaxBatches = a.config.ConsulCoordinateUpdateMaxBatches
base.CoordinateUpdatePeriod = a.config.ConsulCoordinateUpdatePeriod
base.RaftConfig.HeartbeatTimeout = a.config.ConsulRaftHeartbeatTimeout
base.RaftConfig.LeaderLeaseTimeout = a.config.ConsulRaftLeaderLeaseTimeout
base.RaftConfig.ElectionTimeout = a.config.ConsulRaftElectionTimeout
base.SerfLANConfig.MemberlistConfig.BindAddr = a.config.SerfBindAddrLAN.IP.String()
base.SerfLANConfig.MemberlistConfig.BindPort = a.config.SerfPortLAN
base.SerfLANConfig.MemberlistConfig.AdvertiseAddr = a.config.SerfAdvertiseAddrLAN.IP.String()
base.SerfLANConfig.MemberlistConfig.AdvertisePort = a.config.SerfPortLAN
base.SerfLANConfig.MemberlistConfig.GossipVerifyIncoming = a.config.EncryptVerifyIncoming
base.SerfLANConfig.MemberlistConfig.GossipVerifyOutgoing = a.config.EncryptVerifyOutgoing
base.SerfLANConfig.MemberlistConfig.GossipInterval = a.config.ConsulSerfLANGossipInterval
base.SerfLANConfig.MemberlistConfig.ProbeInterval = a.config.ConsulSerfLANProbeInterval
base.SerfLANConfig.MemberlistConfig.ProbeTimeout = a.config.ConsulSerfLANProbeTimeout
base.SerfLANConfig.MemberlistConfig.SuspicionMult = a.config.ConsulSerfLANSuspicionMult
base.SerfWANConfig.MemberlistConfig.BindAddr = a.config.SerfBindAddrWAN.IP.String()
base.SerfWANConfig.MemberlistConfig.BindPort = a.config.SerfPortWAN
base.SerfWANConfig.MemberlistConfig.AdvertiseAddr = a.config.SerfAdvertiseAddrWAN.IP.String()
base.SerfWANConfig.MemberlistConfig.AdvertisePort = a.config.SerfPortWAN
base.SerfWANConfig.MemberlistConfig.GossipVerifyIncoming = a.config.EncryptVerifyIncoming
base.SerfWANConfig.MemberlistConfig.GossipVerifyOutgoing = a.config.EncryptVerifyOutgoing
base.SerfWANConfig.MemberlistConfig.GossipInterval = a.config.ConsulSerfWANGossipInterval
base.SerfWANConfig.MemberlistConfig.ProbeInterval = a.config.ConsulSerfWANProbeInterval
base.SerfWANConfig.MemberlistConfig.ProbeTimeout = a.config.ConsulSerfWANProbeTimeout
base.SerfWANConfig.MemberlistConfig.SuspicionMult = a.config.ConsulSerfWANSuspicionMult
base.RPCAddr = a.config.RPCBindAddr
base.RPCAdvertise = a.config.RPCAdvertiseAddr
if a.config.ReconnectTimeoutLAN != 0 {
base.SerfLANConfig.ReconnectTimeout = a.config.ReconnectTimeoutLAN
}
if a.config.SerfLanBindAddr != "" {
base.SerfLANConfig.MemberlistConfig.BindAddr = a.config.SerfLanBindAddr
}
if a.config.SerfWanBindAddr != "" {
base.SerfWANConfig.MemberlistConfig.BindAddr = a.config.SerfWanBindAddr
if a.config.ReconnectTimeoutWAN != 0 {
base.SerfWANConfig.ReconnectTimeout = a.config.ReconnectTimeoutWAN
}
if a.config.AdvertiseAddr != "" {
base.SerfLANConfig.MemberlistConfig.AdvertiseAddr = a.config.AdvertiseAddr
base.SerfWANConfig.MemberlistConfig.AdvertiseAddr = a.config.AdvertiseAddr
if a.config.AdvertiseAddrWan != "" {
base.SerfWANConfig.MemberlistConfig.AdvertiseAddr = a.config.AdvertiseAddrWan
}
base.RPCAdvertise = &net.TCPAddr{
IP: net.ParseIP(a.config.AdvertiseAddr),
Port: a.config.Ports.Server,
}
}
if a.config.AdvertiseAddrs.SerfLan != nil {
base.SerfLANConfig.MemberlistConfig.AdvertiseAddr = a.config.AdvertiseAddrs.SerfLan.IP.String()
base.SerfLANConfig.MemberlistConfig.AdvertisePort = a.config.AdvertiseAddrs.SerfLan.Port
}
if a.config.AdvertiseAddrs.SerfWan != nil {
base.SerfWANConfig.MemberlistConfig.AdvertiseAddr = a.config.AdvertiseAddrs.SerfWan.IP.String()
base.SerfWANConfig.MemberlistConfig.AdvertisePort = a.config.AdvertiseAddrs.SerfWan.Port
}
if a.config.ReconnectTimeoutLan != 0 {
base.SerfLANConfig.ReconnectTimeout = a.config.ReconnectTimeoutLan
}
if a.config.ReconnectTimeoutWan != 0 {
base.SerfWANConfig.ReconnectTimeout = a.config.ReconnectTimeoutWan
}
if a.config.EncryptVerifyIncoming != nil {
base.SerfWANConfig.MemberlistConfig.GossipVerifyIncoming = *a.config.EncryptVerifyIncoming
base.SerfLANConfig.MemberlistConfig.GossipVerifyIncoming = *a.config.EncryptVerifyIncoming
}
if a.config.EncryptVerifyOutgoing != nil {
base.SerfWANConfig.MemberlistConfig.GossipVerifyOutgoing = *a.config.EncryptVerifyOutgoing
base.SerfLANConfig.MemberlistConfig.GossipVerifyOutgoing = *a.config.EncryptVerifyOutgoing
}
if a.config.AdvertiseAddrs.RPC != nil {
base.RPCAdvertise = a.config.AdvertiseAddrs.RPC
}
base.Segment = a.config.Segment
base.Segment = a.config.SegmentName
if len(a.config.Segments) > 0 {
segments, err := a.segmentConfig()
if err != nil {
@ -666,8 +640,8 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
if a.config.BootstrapExpect != 0 {
base.BootstrapExpect = a.config.BootstrapExpect
}
if a.config.Protocol > 0 {
base.ProtocolVersion = uint8(a.config.Protocol)
if a.config.RPCProtocol > 0 {
base.ProtocolVersion = uint8(a.config.RPCProtocol)
}
if a.config.RaftProtocol != 0 {
base.RaftConfig.ProtocolVersion = raft.ProtocolVersion(a.config.RaftProtocol)
@ -678,7 +652,7 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
if a.config.ACLDatacenter != "" {
base.ACLDatacenter = a.config.ACLDatacenter
}
if a.config.ACLTTLRaw != "" {
if a.config.ACLTTL != 0 {
base.ACLTTL = a.config.ACLTTL
}
if a.config.ACLDefaultPolicy != "" {
@ -688,35 +662,35 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
base.ACLDownPolicy = a.config.ACLDownPolicy
}
base.EnableACLReplication = a.config.EnableACLReplication
if a.config.ACLEnforceVersion8 != nil {
base.ACLEnforceVersion8 = *a.config.ACLEnforceVersion8
if a.config.ACLEnforceVersion8 {
base.ACLEnforceVersion8 = a.config.ACLEnforceVersion8
}
if a.config.SessionTTLMinRaw != "" {
if a.config.SessionTTLMin != 0 {
base.SessionTTLMin = a.config.SessionTTLMin
}
if a.config.Autopilot.CleanupDeadServers != nil {
base.AutopilotConfig.CleanupDeadServers = *a.config.Autopilot.CleanupDeadServers
if a.config.AutopilotCleanupDeadServers {
base.AutopilotConfig.CleanupDeadServers = a.config.AutopilotCleanupDeadServers
}
if a.config.Autopilot.LastContactThreshold != nil {
base.AutopilotConfig.LastContactThreshold = *a.config.Autopilot.LastContactThreshold
if a.config.AutopilotLastContactThreshold != 0 {
base.AutopilotConfig.LastContactThreshold = a.config.AutopilotLastContactThreshold
}
if a.config.Autopilot.MaxTrailingLogs != nil {
base.AutopilotConfig.MaxTrailingLogs = *a.config.Autopilot.MaxTrailingLogs
if a.config.AutopilotMaxTrailingLogs != 0 {
base.AutopilotConfig.MaxTrailingLogs = uint64(a.config.AutopilotMaxTrailingLogs)
}
if a.config.Autopilot.ServerStabilizationTime != nil {
base.AutopilotConfig.ServerStabilizationTime = *a.config.Autopilot.ServerStabilizationTime
if a.config.AutopilotServerStabilizationTime != 0 {
base.AutopilotConfig.ServerStabilizationTime = a.config.AutopilotServerStabilizationTime
}
if a.config.NonVotingServer {
base.NonVoter = a.config.NonVotingServer
}
if a.config.Autopilot.RedundancyZoneTag != "" {
base.AutopilotConfig.RedundancyZoneTag = a.config.Autopilot.RedundancyZoneTag
if a.config.AutopilotRedundancyZoneTag != "" {
base.AutopilotConfig.RedundancyZoneTag = a.config.AutopilotRedundancyZoneTag
}
if a.config.Autopilot.DisableUpgradeMigration != nil {
base.AutopilotConfig.DisableUpgradeMigration = *a.config.Autopilot.DisableUpgradeMigration
if a.config.AutopilotDisableUpgradeMigration {
base.AutopilotConfig.DisableUpgradeMigration = a.config.AutopilotDisableUpgradeMigration
}
if a.config.Autopilot.UpgradeVersionTag != "" {
base.AutopilotConfig.UpgradeVersionTag = a.config.Autopilot.UpgradeVersionTag
if a.config.AutopilotUpgradeVersionTag != "" {
base.AutopilotConfig.UpgradeVersionTag = a.config.AutopilotUpgradeVersionTag
}
// make sure the advertise address is always set
@ -725,11 +699,11 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
}
// Rate limiting for RPC calls.
if a.config.Limits.RPCRate > 0 {
base.RPCRate = a.config.Limits.RPCRate
if a.config.RPCRateLimit > 0 {
base.RPCRate = a.config.RPCRateLimit
}
if a.config.Limits.RPCMaxBurst > 0 {
base.RPCMaxBurst = a.config.Limits.RPCMaxBurst
if a.config.RPCMaxBurst > 0 {
base.RPCMaxBurst = a.config.RPCMaxBurst
}
// set the src address for outgoing rpc connections
@ -757,7 +731,7 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
base.CertFile = a.config.CertFile
base.KeyFile = a.config.KeyFile
base.ServerName = a.config.ServerName
base.Domain = a.config.Domain
base.Domain = a.config.DNSDomain
base.TLSMinVersion = a.config.TLSMinVersion
base.TLSCipherSuites = a.config.TLSCipherSuites
base.TLSPreferServerCipherSuites = a.config.TLSPreferServerCipherSuites
@ -787,45 +761,37 @@ func (a *Agent) segmentConfig() ([]consul.NetworkSegment, error) {
var segments []consul.NetworkSegment
config := a.config
for _, segment := range config.Segments {
for _, s := range config.Segments {
serfConf := consul.DefaultConfig().SerfLANConfig
if segment.Advertise != "" {
serfConf.MemberlistConfig.AdvertiseAddr = segment.Advertise
} else {
serfConf.MemberlistConfig.AdvertiseAddr = a.config.AdvertiseAddr
}
if segment.Bind != "" {
serfConf.MemberlistConfig.BindAddr = segment.Bind
} else {
serfConf.MemberlistConfig.BindAddr = a.config.BindAddr
}
serfConf.MemberlistConfig.AdvertisePort = segment.Port
serfConf.MemberlistConfig.BindPort = segment.Port
serfConf.MemberlistConfig.BindAddr = s.Bind.IP.String()
serfConf.MemberlistConfig.BindPort = s.Bind.Port
serfConf.MemberlistConfig.AdvertiseAddr = s.Advertise.IP.String()
serfConf.MemberlistConfig.AdvertisePort = s.Advertise.Port
if config.ReconnectTimeoutLan != 0 {
serfConf.ReconnectTimeout = config.ReconnectTimeoutLan
if config.ReconnectTimeoutLAN != 0 {
serfConf.ReconnectTimeout = config.ReconnectTimeoutLAN
}
if config.EncryptVerifyIncoming != nil {
serfConf.MemberlistConfig.GossipVerifyIncoming = *config.EncryptVerifyIncoming
if config.EncryptVerifyIncoming {
serfConf.MemberlistConfig.GossipVerifyIncoming = config.EncryptVerifyIncoming
}
if config.EncryptVerifyOutgoing != nil {
serfConf.MemberlistConfig.GossipVerifyOutgoing = *config.EncryptVerifyOutgoing
if config.EncryptVerifyOutgoing {
serfConf.MemberlistConfig.GossipVerifyOutgoing = config.EncryptVerifyOutgoing
}
var rpcAddr *net.TCPAddr
if segment.RPCListener {
if s.RPCListener {
rpcAddr = &net.TCPAddr{
IP: net.ParseIP(segment.Bind),
Port: a.config.Ports.Server,
IP: s.Bind.IP,
Port: a.config.ServerPort,
}
}
segments = append(segments, consul.NetworkSegment{
Name: segment.Name,
Name: s.Name,
Bind: serfConf.MemberlistConfig.BindAddr,
Port: segment.Port,
Advertise: serfConf.MemberlistConfig.AdvertiseAddr,
Port: s.Bind.Port,
RPCAddr: rpcAddr,
SerfConfig: serfConf,
})
@ -852,7 +818,7 @@ func (a *Agent) makeRandomID() (string, error) {
// gopsutil change implementations without affecting in-place upgrades of nodes.
func (a *Agent) makeNodeID() (string, error) {
// If they've disabled host-based IDs then just make a random one.
if *a.config.DisableHostNodeID {
if a.config.DisableHostNodeID {
return a.makeRandomID()
}
@ -889,7 +855,7 @@ func (a *Agent) makeNodeID() (string, error) {
// setupNodeID will pull the persisted node ID, if any, or create a random one
// and persist it.
func (a *Agent) setupNodeID(config *Config) error {
func (a *Agent) setupNodeID(config *config.RuntimeConfig) error {
// If they've configured a node ID manually then just use that, as
// long as it's valid.
if config.NodeID != "" {
@ -961,7 +927,7 @@ func (a *Agent) setupBaseKeyrings(config *consul.Config) error {
if err := loadKeyring(config.SerfLANConfig, keys); err != nil {
return err
}
if a.config.Server {
if a.config.ServerMode {
if err := loadKeyring(config.SerfWANConfig, keys); err != nil {
return err
}
@ -981,7 +947,7 @@ func (a *Agent) setupBaseKeyrings(config *consul.Config) error {
return err
}
}
if a.config.Server {
if a.config.ServerMode {
if _, err := os.Stat(fileWAN); err != nil {
if err := initKeyring(fileWAN, a.config.EncryptKey); err != nil {
return err
@ -996,7 +962,7 @@ LOAD:
if err := loadKeyringFile(config.SerfLANConfig); err != nil {
return err
}
if a.config.Server {
if a.config.ServerMode {
if _, err := os.Stat(fileWAN); err == nil {
config.SerfWANConfig.KeyringFile = fileWAN
}
@ -1150,12 +1116,12 @@ func (a *Agent) ShutdownEndpoints() {
a.dnsServers = nil
for _, srv := range a.httpServers {
a.logger.Printf("[INFO] agent: Stopping %s server %s", strings.ToUpper(srv.proto), srv.Addr)
a.logger.Printf("[INFO] agent: Stopping %s server %s (%s)", strings.ToUpper(srv.proto), srv.addr.String(), srv.addr.Network())
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
defer cancel()
srv.Shutdown(ctx)
if ctx.Err() == context.DeadlineExceeded {
a.logger.Printf("[WARN] agent: Timeout stopping %s server %s", strings.ToUpper(srv.proto), srv.Addr)
a.logger.Printf("[WARN] agent: Timeout stopping %s server %s (%s)", strings.ToUpper(srv.proto), srv.addr.String(), srv.addr.Network())
}
}
a.httpServers = nil
@ -2007,7 +1973,7 @@ func (a *Agent) deletePid() error {
// loadServices will load service definitions from configuration and persisted
// definitions on disk, and load them into the local agent.
func (a *Agent) loadServices(conf *Config) error {
func (a *Agent) loadServices(conf *config.RuntimeConfig) error {
// Register the services from config
for _, service := range conf.Services {
ns := service.NodeService()
@ -2096,7 +2062,7 @@ func (a *Agent) unloadServices() error {
// loadChecks loads check definitions and/or persisted check definitions from
// disk and re-registers them with the local agent.
func (a *Agent) loadChecks(conf *Config) error {
func (a *Agent) loadChecks(conf *config.RuntimeConfig) error {
// Register the checks from config
for _, check := range conf.Checks {
health := check.HealthCheck(conf.NodeName)
@ -2200,15 +2166,15 @@ func (a *Agent) restoreCheckState(snap map[types.CheckID]*structs.HealthCheck) {
// loadMetadata loads node metadata fields from the agent config and
// updates them on the local agent.
func (a *Agent) loadMetadata(conf *Config) error {
func (a *Agent) loadMetadata(conf *config.RuntimeConfig) error {
a.state.Lock()
defer a.state.Unlock()
for key, value := range conf.Meta {
for key, value := range conf.NodeMeta {
a.state.metadata[key] = value
}
a.state.metadata[structs.MetaSegmentKey] = conf.Segment
a.state.metadata[structs.MetaSegmentKey] = conf.SegmentName
a.state.changeMade()
@ -2316,7 +2282,7 @@ func (a *Agent) DisableNodeMaintenance() {
a.logger.Printf("[INFO] agent: Node left maintenance mode")
}
func (a *Agent) ReloadConfig(newCfg *Config) error {
func (a *Agent) ReloadConfig(newCfg *config.RuntimeConfig) error {
// Bulk update the services and checks
a.PauseSync()
defer a.ResumeSync()
@ -2351,7 +2317,7 @@ func (a *Agent) ReloadConfig(newCfg *Config) error {
}
// Update filtered metrics
metrics.UpdateFilter(newCfg.Telemetry.AllowedPrefixes, newCfg.Telemetry.BlockedPrefixes)
metrics.UpdateFilter(newCfg.TelemetryAllowedPrefixes, newCfg.TelemetryBlockedPrefixes)
return nil
}

View File

@ -8,6 +8,7 @@ import (
"strings"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/ipaddr"
@ -20,7 +21,7 @@ import (
)
type Self struct {
Config *Config
Config config.RuntimeConfig
Coord *coordinate.Coordinate
Member serf.Member
Stats map[string]map[string]string
@ -48,8 +49,8 @@ func (s *HTTPServer) AgentSelf(resp http.ResponseWriter, req *http.Request) (int
}
return Self{
Config: s.agent.config,
Coord: cs[s.agent.config.Segment],
Config: s.agent.config.Sanitized(),
Coord: cs[s.agent.config.SegmentName],
Member: s.agent.LocalMember(),
Stats: s.agent.Stats(),
Meta: s.agent.state.Metadata(),

View File

@ -14,12 +14,12 @@ import (
"time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/logger"
"github.com/hashicorp/consul/testutil/retry"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/consul/watch"
"github.com/hashicorp/serf/serf"
)
@ -41,7 +41,7 @@ func makeReadOnlyAgentACL(t *testing.T, srv *HTTPServer) string {
func TestAgent_Services(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
srv1 := &structs.NodeService{
@ -106,7 +106,7 @@ func TestAgent_Services_ACLFilter(t *testing.T) {
func TestAgent_Checks(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
chk1 := &structs.HealthCheck{
@ -171,9 +171,11 @@ func TestAgent_Checks_ACLFilter(t *testing.T) {
func TestAgent_Self(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Meta = map[string]string{"somekey": "somevalue"}
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
node_meta {
somekey = "somevalue"
}
`)
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/agent/self", nil)
@ -183,11 +185,11 @@ func TestAgent_Self(t *testing.T) {
}
val := obj.(Self)
if int(val.Member.Port) != a.Config.Ports.SerfLan {
if int(val.Member.Port) != a.Config.SerfPortLAN {
t.Fatalf("incorrect port: %v", obj)
}
if int(val.Config.Ports.SerfLan) != a.Config.Ports.SerfLan {
if int(val.Config.SerfPortLAN) != a.Config.SerfPortLAN {
t.Fatalf("incorrect port: %v", obj)
}
@ -195,21 +197,12 @@ func TestAgent_Self(t *testing.T) {
if err != nil {
t.Fatalf("err: %v", err)
}
if c := cs[cfg.Segment]; !reflect.DeepEqual(c, val.Coord) {
if c := cs[a.config.SegmentName]; !reflect.DeepEqual(c, val.Coord) {
t.Fatalf("coordinates are not equal: %v != %v", c, val.Coord)
}
delete(val.Meta, structs.MetaSegmentKey) // Added later, not in config.
if !reflect.DeepEqual(cfg.Meta, val.Meta) {
t.Fatalf("meta fields are not equal: %v != %v", cfg.Meta, val.Meta)
}
// Make sure there's nothing called "token" that's leaked.
raw, err := a.srv.marshalJSON(req, obj)
if err != nil {
t.Fatalf("err: %v", err)
}
if bytes.Contains(bytes.ToLower(raw), []byte("token")) {
t.Fatalf("bad: %s", raw)
if !reflect.DeepEqual(a.config.NodeMeta, val.Meta) {
t.Fatalf("meta fields are not equal: %v != %v", a.config.NodeMeta, val.Meta)
}
}
@ -271,36 +264,44 @@ func TestAgent_Metrics_ACLDeny(t *testing.T) {
func TestAgent_Reload(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLEnforceVersion8 = Bool(false)
cfg.Services = []*structs.ServiceDefinition{
&structs.ServiceDefinition{Name: "redis"},
}
params := map[string]interface{}{
"datacenter": "dc1",
"type": "key",
"key": "test",
"handler": "true",
}
wp, err := watch.ParseExempt(params, []string{"handler"})
if err != nil {
t.Fatalf("Expected watch.Parse to succeed %v", err)
}
cfg.WatchPlans = append(cfg.WatchPlans, wp)
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
acl_enforce_version_8 = false
services = [
{
name = "redis"
}
]
watches = [
{
datacenter = "dc1"
type = "key"
key = "test"
handler = "true"
}
]
`)
defer a.Shutdown()
if _, ok := a.state.services["redis"]; !ok {
t.Fatalf("missing redis service")
}
cfg2 := TestConfig()
cfg2.ACLEnforceVersion8 = Bool(false)
cfg2.Services = []*structs.ServiceDefinition{
&structs.ServiceDefinition{Name: "redis-reloaded"},
}
cfg2 := TestConfig(config.Source{
Name: "reload",
Format: "hcl",
Data: `
data_dir = "` + a.Config.DataDir + `"
node_id = "` + string(a.Config.NodeID) + `"
node_name = "` + a.Config.NodeName + `"
acl_enforce_version_8 = false
services = [
{
name = "redis-reloaded"
}
]
`,
})
if err := a.ReloadConfig(cfg2); err != nil {
t.Fatalf("got error %v want nil", err)
@ -309,7 +310,7 @@ func TestAgent_Reload(t *testing.T) {
t.Fatalf("missing redis-reloaded service")
}
for _, wp := range cfg.WatchPlans {
for _, wp := range a.watchPlans {
if !wp.IsStopped() {
t.Fatalf("Reloading configs should stop watch plans of the previous configuration")
}
@ -344,7 +345,7 @@ func TestAgent_Reload_ACLDeny(t *testing.T) {
func TestAgent_Members(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/agent/members", nil)
@ -357,14 +358,14 @@ func TestAgent_Members(t *testing.T) {
t.Fatalf("bad members: %v", obj)
}
if int(val[0].Port) != a.Config.Ports.SerfLan {
if int(val[0].Port) != a.Config.SerfPortLAN {
t.Fatalf("not lan: %v", obj)
}
}
func TestAgent_Members_WAN(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/agent/members?wan=true", nil)
@ -377,7 +378,7 @@ func TestAgent_Members_WAN(t *testing.T) {
t.Fatalf("bad members: %v", obj)
}
if int(val[0].Port) != a.Config.Ports.SerfWan {
if int(val[0].Port) != a.Config.SerfPortWAN {
t.Fatalf("not wan: %v", obj)
}
}
@ -414,12 +415,12 @@ func TestAgent_Members_ACLFilter(t *testing.T) {
func TestAgent_Join(t *testing.T) {
t.Parallel()
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
a2 := NewTestAgent(t.Name(), nil)
a2 := NewTestAgent(t.Name(), "")
defer a2.Shutdown()
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan)
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.SerfPortLAN)
req, _ := http.NewRequest("GET", fmt.Sprintf("/v1/agent/join/%s", addr), nil)
obj, err := a1.srv.AgentJoin(nil, req)
if err != nil {
@ -442,12 +443,12 @@ func TestAgent_Join(t *testing.T) {
func TestAgent_Join_WAN(t *testing.T) {
t.Parallel()
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
a2 := NewTestAgent(t.Name(), nil)
a2 := NewTestAgent(t.Name(), "")
defer a2.Shutdown()
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfWan)
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.SerfPortWAN)
req, _ := http.NewRequest("GET", fmt.Sprintf("/v1/agent/join/%s?wan=true", addr), nil)
obj, err := a1.srv.AgentJoin(nil, req)
if err != nil {
@ -472,10 +473,10 @@ func TestAgent_Join_ACLDeny(t *testing.T) {
t.Parallel()
a1 := NewTestAgent(t.Name(), TestACLConfig())
defer a1.Shutdown()
a2 := NewTestAgent(t.Name(), nil)
a2 := NewTestAgent(t.Name(), "")
defer a2.Shutdown()
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan)
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.SerfPortLAN)
t.Run("no token", func(t *testing.T) {
req, _ := http.NewRequest("GET", fmt.Sprintf("/v1/agent/join/%s", addr), nil)
@ -510,19 +511,19 @@ func (n *mockNotifier) Notify(state string) error {
func TestAgent_JoinLANNotify(t *testing.T) {
t.Parallel()
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
cfg2 := TestConfig()
cfg2.Server = false
cfg2.Bootstrap = false
a2 := NewTestAgent(t.Name(), cfg2)
a2 := NewTestAgent(t.Name(), `
server = false
bootstrap = false
`)
defer a2.Shutdown()
notif := &mockNotifier{}
a1.joinLANNotifier = notif
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan)
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.SerfPortLAN)
_, err := a1.JoinLAN([]string{addr})
if err != nil {
t.Fatalf("err: %v", err)
@ -535,17 +536,17 @@ func TestAgent_JoinLANNotify(t *testing.T) {
func TestAgent_Leave(t *testing.T) {
t.Parallel()
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
cfg2 := TestConfig()
cfg2.Server = false
cfg2.Bootstrap = false
a2 := NewTestAgent(t.Name(), cfg2)
a2 := NewTestAgent(t.Name(), `
server = false
bootstrap = false
`)
defer a2.Shutdown()
// Join first
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan)
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.SerfPortLAN)
_, err := a1.JoinLAN([]string{addr})
if err != nil {
t.Fatalf("err: %v", err)
@ -600,18 +601,18 @@ func TestAgent_Leave_ACLDeny(t *testing.T) {
func TestAgent_ForceLeave(t *testing.T) {
t.Parallel()
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
a2 := NewTestAgent(t.Name(), nil)
a2 := NewTestAgent(t.Name(), "")
// Join first
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan)
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.SerfPortLAN)
_, err := a1.JoinLAN([]string{addr})
if err != nil {
t.Fatalf("err: %v", err)
}
// todo(fs): this test probably needs work
// this test probably needs work
a2.Shutdown()
// Force leave now
@ -662,7 +663,7 @@ func TestAgent_ForceLeave_ACLDeny(t *testing.T) {
func TestAgent_RegisterCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -703,7 +704,7 @@ func TestAgent_RegisterCheck(t *testing.T) {
func TestAgent_RegisterCheck_Passing(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -739,7 +740,7 @@ func TestAgent_RegisterCheck_Passing(t *testing.T) {
func TestAgent_RegisterCheck_BadStatus(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -785,7 +786,7 @@ func TestAgent_RegisterCheck_ACLDeny(t *testing.T) {
func TestAgent_DeregisterCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
chk := &structs.HealthCheck{Name: "test", CheckID: "test"}
@ -836,7 +837,7 @@ func TestAgent_DeregisterCheckACLDeny(t *testing.T) {
func TestAgent_PassCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
chk := &structs.HealthCheck{Name: "test", CheckID: "test"}
@ -889,7 +890,7 @@ func TestAgent_PassCheck_ACLDeny(t *testing.T) {
func TestAgent_WarnCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
chk := &structs.HealthCheck{Name: "test", CheckID: "test"}
@ -942,7 +943,7 @@ func TestAgent_WarnCheck_ACLDeny(t *testing.T) {
func TestAgent_FailCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
chk := &structs.HealthCheck{Name: "test", CheckID: "test"}
@ -995,7 +996,7 @@ func TestAgent_FailCheck_ACLDeny(t *testing.T) {
func TestAgent_UpdateCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
chk := &structs.HealthCheck{Name: "test", CheckID: "test"}
@ -1122,7 +1123,7 @@ func TestAgent_UpdateCheck_ACLDeny(t *testing.T) {
func TestAgent_RegisterService(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
args := &structs.ServiceDefinition{
@ -1211,7 +1212,7 @@ func TestAgent_RegisterService_ACLDeny(t *testing.T) {
func TestAgent_RegisterService_InvalidAddress(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
for _, addr := range []string{"0.0.0.0", "::", "[::]"} {
@ -1239,7 +1240,7 @@ func TestAgent_RegisterService_InvalidAddress(t *testing.T) {
func TestAgent_DeregisterService(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
service := &structs.NodeService{
@ -1299,7 +1300,7 @@ func TestAgent_DeregisterService_ACLDeny(t *testing.T) {
func TestAgent_ServiceMaintenance_BadRequest(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
t.Run("not PUT", func(t *testing.T) {
@ -1349,7 +1350,7 @@ func TestAgent_ServiceMaintenance_BadRequest(t *testing.T) {
func TestAgent_ServiceMaintenance_Enable(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register the service
@ -1391,7 +1392,7 @@ func TestAgent_ServiceMaintenance_Enable(t *testing.T) {
func TestAgent_ServiceMaintenance_Disable(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register the service
@ -1456,7 +1457,7 @@ func TestAgent_ServiceMaintenance_ACLDeny(t *testing.T) {
func TestAgent_NodeMaintenance_BadRequest(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Fails on non-PUT
@ -1482,7 +1483,7 @@ func TestAgent_NodeMaintenance_BadRequest(t *testing.T) {
func TestAgent_NodeMaintenance_Enable(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Force the node into maintenance mode
@ -1514,7 +1515,7 @@ func TestAgent_NodeMaintenance_Enable(t *testing.T) {
func TestAgent_NodeMaintenance_Disable(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Force the node into maintenance mode
@ -1558,7 +1559,7 @@ func TestAgent_NodeMaintenance_ACLDeny(t *testing.T) {
func TestAgent_RegisterCheck_Service(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
args := &structs.ServiceDefinition{
@ -1687,11 +1688,11 @@ func TestAgent_Monitor_ACLDeny(t *testing.T) {
func TestAgent_Token(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLToken = ""
cfg.ACLAgentToken = ""
cfg.ACLAgentMasterToken = ""
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_token = ""
acl_agent_token = ""
acl_agent_master_token = ""
`)
defer a.Shutdown()
type tokens struct {

View File

@ -9,7 +9,6 @@ import (
"os"
"path/filepath"
"reflect"
"runtime"
"strings"
"testing"
"time"
@ -19,7 +18,7 @@ import (
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testutil"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-uuid"
uuid "github.com/hashicorp/go-uuid"
"github.com/hashicorp/raft"
"github.com/pascaldekloe/goe/verify"
)
@ -40,10 +39,10 @@ func externalIP() (string, error) {
}
func TestAgent_MultiStartStop(t *testing.T) {
for i := 0; i < 100; i++ {
for i := 0; i < 10; i++ {
t.Run("", func(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
time.Sleep(250 * time.Millisecond)
a.Shutdown()
})
@ -52,7 +51,7 @@ func TestAgent_MultiStartStop(t *testing.T) {
func TestAgent_StartStop(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
// defer a.Shutdown()
if err := a.Leave(); err != nil {
@ -71,7 +70,7 @@ func TestAgent_StartStop(t *testing.T) {
func TestAgent_RPCPing(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
var out struct{}
@ -80,79 +79,14 @@ func TestAgent_RPCPing(t *testing.T) {
}
}
func TestAgent_CheckSerfBindAddrsSettings(t *testing.T) {
t.Parallel()
if runtime.GOOS == "darwin" {
t.Skip("skip test on macOS to avoid firewall warning dialog")
}
cfg := TestConfig()
ip, err := externalIP()
if err != nil {
t.Fatalf("Unable to get a non-loopback IP: %v", err)
}
cfg.SerfLanBindAddr = ip
cfg.SerfWanBindAddr = ip
a := NewTestAgent(t.Name(), cfg)
defer a.Shutdown()
serfWanBind := a.consulConfig().SerfWANConfig.MemberlistConfig.BindAddr
if serfWanBind != ip {
t.Fatalf("SerfWanBindAddr is should be a non-loopback IP not %s", serfWanBind)
}
serfLanBind := a.consulConfig().SerfLANConfig.MemberlistConfig.BindAddr
if serfLanBind != ip {
t.Fatalf("SerfLanBindAddr is should be a non-loopback IP not %s", serfWanBind)
}
}
func TestAgent_CheckAdvertiseAddrsSettings(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.AdvertiseAddrs.SerfLan, _ = net.ResolveTCPAddr("tcp", "127.0.0.42:1233")
cfg.AdvertiseAddrs.SerfWan, _ = net.ResolveTCPAddr("tcp", "127.0.0.43:1234")
cfg.AdvertiseAddrs.RPC, _ = net.ResolveTCPAddr("tcp", "127.0.0.44:1235")
cfg.SetupTaggedAndAdvertiseAddrs()
a := NewTestAgent(t.Name(), cfg)
defer a.Shutdown()
serfLanAddr := a.consulConfig().SerfLANConfig.MemberlistConfig.AdvertiseAddr
if serfLanAddr != "127.0.0.42" {
t.Fatalf("SerfLan is not properly set to '127.0.0.42': %s", serfLanAddr)
}
serfLanPort := a.consulConfig().SerfLANConfig.MemberlistConfig.AdvertisePort
if serfLanPort != 1233 {
t.Fatalf("SerfLan is not properly set to '1233': %d", serfLanPort)
}
serfWanAddr := a.consulConfig().SerfWANConfig.MemberlistConfig.AdvertiseAddr
if serfWanAddr != "127.0.0.43" {
t.Fatalf("SerfWan is not properly set to '127.0.0.43': %s", serfWanAddr)
}
serfWanPort := a.consulConfig().SerfWANConfig.MemberlistConfig.AdvertisePort
if serfWanPort != 1234 {
t.Fatalf("SerfWan is not properly set to '1234': %d", serfWanPort)
}
rpc := a.consulConfig().RPCAdvertise
if rpc != cfg.AdvertiseAddrs.RPC {
t.Fatalf("RPC is not properly set to %v: %s", cfg.AdvertiseAddrs.RPC, rpc)
}
expected := map[string]string{
"lan": a.Config.AdvertiseAddr,
"wan": a.Config.AdvertiseAddrWan,
}
if !reflect.DeepEqual(a.Config.TaggedAddresses, expected) {
t.Fatalf("Tagged addresses not set up properly: %v", a.Config.TaggedAddresses)
}
}
func TestAgent_TokenStore(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLToken = "user"
cfg.ACLAgentToken = "agent"
cfg.ACLAgentMasterToken = "master"
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
acl_token = "user"
acl_agent_token = "agent"
acl_agent_master_token = "master"`,
)
defer a.Shutdown()
if got, want := a.tokens.UserToken(), "user"; got != want {
@ -166,49 +100,10 @@ func TestAgent_TokenStore(t *testing.T) {
}
}
func TestAgent_CheckPerformanceSettings(t *testing.T) {
t.Parallel()
// Try a default config.
{
cfg := TestConfig()
cfg.Bootstrap = false
cfg.ConsulConfig = nil
a := NewTestAgent(t.Name(), cfg)
defer a.Shutdown()
raftMult := time.Duration(consul.DefaultRaftMultiplier)
r := a.consulConfig().RaftConfig
def := raft.DefaultConfig()
if r.HeartbeatTimeout != raftMult*def.HeartbeatTimeout ||
r.ElectionTimeout != raftMult*def.ElectionTimeout ||
r.LeaderLeaseTimeout != raftMult*def.LeaderLeaseTimeout {
t.Fatalf("bad: %#v", *r)
}
}
// Try a multiplier.
{
cfg := TestConfig()
cfg.Bootstrap = false
cfg.Performance.RaftMultiplier = 99
a := NewTestAgent(t.Name(), cfg)
defer a.Shutdown()
const raftMult time.Duration = 99
r := a.consulConfig().RaftConfig
def := raft.DefaultConfig()
if r.HeartbeatTimeout != raftMult*def.HeartbeatTimeout ||
r.ElectionTimeout != raftMult*def.ElectionTimeout ||
r.LeaderLeaseTimeout != raftMult*def.LeaderLeaseTimeout {
t.Fatalf("bad: %#v", *r)
}
}
}
func TestAgent_ReconnectConfigSettings(t *testing.T) {
t.Parallel()
func() {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
lan := a.consulConfig().SerfLANConfig.ReconnectTimeout
@ -223,10 +118,10 @@ func TestAgent_ReconnectConfigSettings(t *testing.T) {
}()
func() {
cfg := TestConfig()
cfg.ReconnectTimeoutLan = 24 * time.Hour
cfg.ReconnectTimeoutWan = 36 * time.Hour
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
reconnect_timeout = "24h"
reconnect_timeout_wan = "36h"
`)
defer a.Shutdown()
lan := a.consulConfig().SerfLANConfig.ReconnectTimeout
@ -243,11 +138,13 @@ func TestAgent_ReconnectConfigSettings(t *testing.T) {
func TestAgent_setupNodeID(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.NodeID = ""
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
node_id = ""
`)
defer a.Shutdown()
cfg := a.config
// The auto-assigned ID should be valid.
id := a.consulConfig().NodeID
if _, err := uuid.ParseUUID(string(id)); err != nil {
@ -309,9 +206,9 @@ func TestAgent_setupNodeID(t *testing.T) {
func TestAgent_makeNodeID(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.NodeID = ""
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
node_id = ""
`)
defer a.Shutdown()
// We should get a valid host-based ID initially.
@ -334,7 +231,7 @@ func TestAgent_makeNodeID(t *testing.T) {
// Turn on host-based IDs and try again. We should get the same ID
// each time (and a different one from the random one above).
a.Config.DisableHostNodeID = Bool(false)
a.Config.DisableHostNodeID = false
id, err = a.makeNodeID()
if err != nil {
t.Fatalf("err: %v", err)
@ -355,9 +252,9 @@ func TestAgent_makeNodeID(t *testing.T) {
func TestAgent_AddService(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.NodeName = "node1"
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
node_name = "node1"
`)
defer a.Shutdown()
tests := []struct {
@ -501,7 +398,7 @@ func TestAgent_AddService(t *testing.T) {
func TestAgent_RemoveService(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Remove a service that doesn't exist
@ -595,9 +492,9 @@ func TestAgent_RemoveService(t *testing.T) {
func TestAgent_RemoveServiceRemovesAllChecks(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.NodeName = "node1"
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
node_name = "node1"
`)
defer a.Shutdown()
svc := &structs.NodeService{ID: "redis", Service: "redis", Port: 8000}
@ -645,9 +542,9 @@ func TestAgent_RemoveServiceRemovesAllChecks(t *testing.T) {
func TestAgent_AddCheck(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
enable_script_checks = true
`)
defer a.Shutdown()
health := &structs.HealthCheck{
@ -684,9 +581,9 @@ func TestAgent_AddCheck(t *testing.T) {
func TestAgent_AddCheck_StartPassing(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
enable_script_checks = true
`)
defer a.Shutdown()
health := &structs.HealthCheck{
@ -723,9 +620,9 @@ func TestAgent_AddCheck_StartPassing(t *testing.T) {
func TestAgent_AddCheck_MinInterval(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
enable_script_checks = true
`)
defer a.Shutdown()
health := &structs.HealthCheck{
@ -758,9 +655,9 @@ func TestAgent_AddCheck_MinInterval(t *testing.T) {
func TestAgent_AddCheck_MissingService(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
enable_script_checks = true
`)
defer a.Shutdown()
health := &structs.HealthCheck{
@ -781,7 +678,7 @@ func TestAgent_AddCheck_MissingService(t *testing.T) {
func TestAgent_AddCheck_RestoreState(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create some state and persist it
@ -825,7 +722,7 @@ func TestAgent_AddCheck_RestoreState(t *testing.T) {
func TestAgent_AddCheck_ExecDisable(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
health := &structs.HealthCheck{
@ -851,9 +748,9 @@ func TestAgent_AddCheck_ExecDisable(t *testing.T) {
func TestAgent_RemoveCheck(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
enable_script_checks = true
`)
defer a.Shutdown()
// Remove check that doesn't exist
@ -899,7 +796,7 @@ func TestAgent_RemoveCheck(t *testing.T) {
func TestAgent_updateTTLCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
health := &structs.HealthCheck{
@ -933,11 +830,15 @@ func TestAgent_updateTTLCheck(t *testing.T) {
func TestAgent_PersistService(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Server = false
cfg.DataDir = testutil.TempDir(t, "agent") // we manage the data dir
a := NewTestAgent(t.Name(), cfg)
defer os.RemoveAll(cfg.DataDir)
dataDir := testutil.TempDir(t, "agent") // we manage the data dir
cfg := `
server = false
bootstrap = false
data_dir = "` + dataDir + `"
`
a := &TestAgent{Name: t.Name(), HCL: cfg, DataDir: dataDir}
a.Start()
defer os.RemoveAll(dataDir)
defer a.Shutdown()
svc := &structs.NodeService{
@ -1001,7 +902,8 @@ func TestAgent_PersistService(t *testing.T) {
a.Shutdown()
// Should load it back during later start
a2 := NewTestAgent(t.Name()+"-a2", cfg)
a2 := &TestAgent{Name: t.Name(), HCL: cfg, DataDir: dataDir}
a2.Start()
defer a2.Shutdown()
restored, ok := a2.state.services[svc.ID]
@ -1019,7 +921,7 @@ func TestAgent_PersistService(t *testing.T) {
func TestAgent_persistedService_compat(t *testing.T) {
t.Parallel()
// Tests backwards compatibility of persisted services from pre-0.5.1
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
svc := &structs.NodeService{
@ -1063,7 +965,7 @@ func TestAgent_persistedService_compat(t *testing.T) {
func TestAgent_PurgeService(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
svc := &structs.NodeService{
@ -1102,10 +1004,16 @@ func TestAgent_PurgeService(t *testing.T) {
func TestAgent_PurgeServiceOnDuplicate(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Server = false
a := NewTestAgent(t.Name(), cfg)
dataDir := testutil.TempDir(t, "agent") // we manage the data dir
cfg := `
data_dir = "` + dataDir + `"
server = false
bootstrap = false
`
a := &TestAgent{Name: t.Name(), HCL: cfg, DataDir: dataDir}
a.Start()
defer a.Shutdown()
defer os.RemoveAll(dataDir)
svc1 := &structs.NodeService{
ID: "redis",
@ -1122,42 +1030,46 @@ func TestAgent_PurgeServiceOnDuplicate(t *testing.T) {
// Try bringing the agent back up with the service already
// existing in the config
svc2 := &structs.ServiceDefinition{
ID: "redis",
Name: "redis",
Tags: []string{"bar"},
Port: 9000,
}
cfg.Services = []*structs.ServiceDefinition{svc2}
a2 := NewTestAgent(t.Name()+"-a2", cfg)
a2 := &TestAgent{Name: t.Name() + "-a2", HCL: cfg + `
service = {
id = "redis"
name = "redis"
tags = ["bar"]
port = 9000
}
`, DataDir: dataDir}
a2.Start()
defer a2.Shutdown()
file := filepath.Join(a.Config.DataDir, servicesDir, stringHash(svc1.ID))
if _, err := os.Stat(file); err == nil {
t.Fatalf("should have removed persisted service")
}
result, ok := a2.state.services[svc2.ID]
result, ok := a2.state.services["redis"]
if !ok {
t.Fatalf("missing service registration")
}
if !reflect.DeepEqual(result.Tags, svc2.Tags) || result.Port != svc2.Port {
if !reflect.DeepEqual(result.Tags, []string{"bar"}) || result.Port != 9000 {
t.Fatalf("bad: %#v", result)
}
}
func TestAgent_PersistCheck(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Server = false
cfg.DataDir = testutil.TempDir(t, "agent") // we manage the data dir
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
defer os.RemoveAll(cfg.DataDir)
dataDir := testutil.TempDir(t, "agent") // we manage the data dir
cfg := `
data_dir = "` + dataDir + `"
server = false
bootstrap = false
enable_script_checks = true
`
a := &TestAgent{Name: t.Name(), HCL: cfg, DataDir: dataDir}
a.Start()
defer os.RemoveAll(dataDir)
defer a.Shutdown()
check := &structs.HealthCheck{
Node: cfg.NodeName,
Node: a.config.NodeName,
CheckID: "mem",
Name: "memory check",
Status: api.HealthPassing,
@ -1223,7 +1135,8 @@ func TestAgent_PersistCheck(t *testing.T) {
a.Shutdown()
// Should load it back during later start
a2 := NewTestAgent(t.Name()+"-a2", cfg)
a2 := &TestAgent{Name: t.Name() + "-a2", HCL: cfg, DataDir: dataDir}
a2.Start()
defer a2.Shutdown()
result, ok := a2.state.checks[check.CheckID]
@ -1248,7 +1161,7 @@ func TestAgent_PersistCheck(t *testing.T) {
func TestAgent_PurgeCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
check := &structs.HealthCheck{
@ -1282,16 +1195,21 @@ func TestAgent_PurgeCheck(t *testing.T) {
func TestAgent_PurgeCheckOnDuplicate(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Server = false
cfg.DataDir = testutil.TempDir(t, "agent") // we manage the data dir
cfg.EnableScriptChecks = true
a := NewTestAgent(t.Name(), cfg)
defer os.RemoveAll(cfg.DataDir)
nodeID := NodeID()
dataDir := testutil.TempDir(t, "agent")
a := NewTestAgent(t.Name(), `
node_id = "`+nodeID+`"
node_name = "Node `+nodeID+`"
data_dir = "`+dataDir+`"
server = false
bootstrap = false
enable_script_checks = true
`)
defer os.RemoveAll(dataDir)
defer a.Shutdown()
check1 := &structs.HealthCheck{
Node: cfg.NodeName,
Node: a.Config.NodeName,
CheckID: "mem",
Name: "memory check",
Status: api.HealthPassing,
@ -1304,42 +1222,53 @@ func TestAgent_PurgeCheckOnDuplicate(t *testing.T) {
a.Shutdown()
// Start again with the check registered in config
check2 := &structs.CheckDefinition{
ID: "mem",
Name: "memory check",
Notes: "my cool notes",
Script: "/bin/check-redis.py",
Interval: 30 * time.Second,
}
cfg.Checks = []*structs.CheckDefinition{check2}
a2 := NewTestAgent(t.Name()+"-a2", cfg)
a2 := NewTestAgent(t.Name()+"-a2", `
node_id = "`+nodeID+`"
node_name = "Node `+nodeID+`"
data_dir = "`+dataDir+`"
server = false
bootstrap = false
enable_script_checks = true
check = {
id = "mem"
name = "memory check"
notes = "my cool notes"
script = "/bin/check-redis.py"
interval = "30s"
}
`)
defer a2.Shutdown()
file := filepath.Join(a.Config.DataDir, checksDir, checkIDHash(check1.CheckID))
file := filepath.Join(dataDir, checksDir, checkIDHash(check1.CheckID))
if _, err := os.Stat(file); err == nil {
t.Fatalf("should have removed persisted check")
}
result, ok := a2.state.checks[check2.ID]
result, ok := a2.state.checks["mem"]
if !ok {
t.Fatalf("missing check registration")
}
expected := check2.HealthCheck(cfg.NodeName)
if !reflect.DeepEqual(expected, result) {
t.Fatalf("bad: %#v", result)
expected := &structs.HealthCheck{
Node: a2.Config.NodeName,
CheckID: "mem",
Name: "memory check",
Status: api.HealthCritical,
Notes: "my cool notes",
}
if got, want := result, expected; !verify.Values(t, "", got, want) {
t.FailNow()
}
}
func TestAgent_loadChecks_token(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Checks = append(cfg.Checks, &structs.CheckDefinition{
ID: "rabbitmq",
Name: "rabbitmq",
Token: "abc123",
TTL: 10 * time.Second,
})
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
check = {
id = "rabbitmq"
name = "rabbitmq"
token = "abc123"
ttl = "10s"
}
`)
defer a.Shutdown()
checks := a.state.Checks()
@ -1353,7 +1282,7 @@ func TestAgent_loadChecks_token(t *testing.T) {
func TestAgent_unloadChecks(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// First register a service
@ -1405,14 +1334,14 @@ func TestAgent_unloadChecks(t *testing.T) {
func TestAgent_loadServices_token(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.Services = append(cfg.Services, &structs.ServiceDefinition{
ID: "rabbitmq",
Name: "rabbitmq",
Port: 5672,
Token: "abc123",
})
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
service = {
id = "rabbitmq"
name = "rabbitmq"
port = 5672
token = "abc123"
}
`)
defer a.Shutdown()
services := a.state.Services()
@ -1426,7 +1355,7 @@ func TestAgent_loadServices_token(t *testing.T) {
func TestAgent_unloadServices(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
svc := &structs.NodeService{
@ -1462,7 +1391,7 @@ func TestAgent_unloadServices(t *testing.T) {
func TestAgent_Service_MaintenanceMode(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
svc := &structs.NodeService{
@ -1526,10 +1455,10 @@ func TestAgent_Service_MaintenanceMode(t *testing.T) {
func TestAgent_Service_Reap(t *testing.T) {
// t.Parallel() // timing test. no parallel
cfg := TestConfig()
cfg.CheckReapInterval = 50 * time.Millisecond
cfg.CheckDeregisterIntervalMin = 0
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
check_reap_interval = "50ms"
check_deregister_interval_min = "0s"
`)
defer a.Shutdown()
svc := &structs.NodeService{
@ -1600,10 +1529,10 @@ func TestAgent_Service_Reap(t *testing.T) {
func TestAgent_Service_NoReap(t *testing.T) {
// t.Parallel() // timing test. no parallel
cfg := TestConfig()
cfg.CheckReapInterval = 50 * time.Millisecond
cfg.CheckDeregisterIntervalMin = 0
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
check_reap_interval = "50ms"
check_deregister_interval_min = "0s"
`)
defer a.Shutdown()
svc := &structs.NodeService{
@ -1653,7 +1582,7 @@ func TestAgent_Service_NoReap(t *testing.T) {
func TestAgent_addCheck_restoresSnapshot(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// First register a service
@ -1696,7 +1625,7 @@ func TestAgent_addCheck_restoresSnapshot(t *testing.T) {
func TestAgent_NodeMaintenanceMode(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Enter maintenance mode for the node
@ -1741,7 +1670,7 @@ func TestAgent_NodeMaintenanceMode(t *testing.T) {
func TestAgent_checkStateSnapshot(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// First register a service
@ -1798,7 +1727,7 @@ func TestAgent_checkStateSnapshot(t *testing.T) {
func TestAgent_loadChecks_checkFails(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Persist a health check with an invalid service ID
@ -1833,7 +1762,7 @@ func TestAgent_loadChecks_checkFails(t *testing.T) {
func TestAgent_persistCheckState(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create the TTL check to persist
@ -1880,7 +1809,7 @@ func TestAgent_persistCheckState(t *testing.T) {
func TestAgent_loadCheckState(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create a check whose state will expire immediately
@ -1941,7 +1870,7 @@ func TestAgent_loadCheckState(t *testing.T) {
func TestAgent_purgeCheckState(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// No error if the state does not exist
@ -1974,9 +1903,9 @@ func TestAgent_purgeCheckState(t *testing.T) {
func TestAgent_GetCoordinate(t *testing.T) {
t.Parallel()
check := func(server bool) {
cfg := TestConfig()
cfg.Server = server
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
server = true
`)
defer a.Shutdown()
// This doesn't verify the returned coordinate, but it makes

View File

@ -14,7 +14,7 @@ import (
func TestCatalogRegister(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -33,7 +33,7 @@ func TestCatalogRegister(t *testing.T) {
t.Fatalf("bad: %v", res)
}
// todo(fs): data race
// data race
func() {
a.state.Lock()
defer a.state.Unlock()
@ -53,7 +53,7 @@ func TestCatalogRegister(t *testing.T) {
func TestCatalogRegister_Service_InvalidAddress(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
for _, addr := range []string{"0.0.0.0", "::", "[::]"} {
@ -78,7 +78,7 @@ func TestCatalogRegister_Service_InvalidAddress(t *testing.T) {
func TestCatalogDeregister(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -97,7 +97,7 @@ func TestCatalogDeregister(t *testing.T) {
func TestCatalogDatacenters(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
retry.Run(t, func(r *retry.R) {
@ -115,7 +115,7 @@ func TestCatalogDatacenters(t *testing.T) {
func TestCatalogNodes(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -148,7 +148,7 @@ func TestCatalogNodes(t *testing.T) {
func TestCatalogNodes_MetaFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register a node with a meta field
@ -188,22 +188,22 @@ func TestCatalogNodes_MetaFilter(t *testing.T) {
func TestCatalogNodes_WanTranslation(t *testing.T) {
t.Parallel()
cfg1 := TestConfig()
cfg1.Datacenter = "dc1"
cfg1.TranslateWanAddrs = true
cfg1.ACLDatacenter = ""
a1 := NewTestAgent(t.Name(), cfg1)
a1 := NewTestAgent(t.Name(), `
datacenter = "dc1"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a1.Shutdown()
cfg2 := TestConfig()
cfg2.Datacenter = "dc2"
cfg2.TranslateWanAddrs = true
cfg2.ACLDatacenter = ""
a2 := NewTestAgent(t.Name(), cfg2)
a2 := NewTestAgent(t.Name(), `
datacenter = "dc2"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a2.Shutdown()
// Wait for the WAN join.
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.Ports.SerfWan)
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.SerfPortWAN)
if _, err := a2.JoinWAN([]string{addr}); err != nil {
t.Fatalf("err: %v", err)
}
@ -282,7 +282,7 @@ func TestCatalogNodes_WanTranslation(t *testing.T) {
func TestCatalogNodes_Blocking(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -350,7 +350,7 @@ func TestCatalogNodes_Blocking(t *testing.T) {
func TestCatalogNodes_DistanceSort(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register nodes.
@ -434,7 +434,7 @@ func TestCatalogNodes_DistanceSort(t *testing.T) {
func TestCatalogServices(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -469,7 +469,7 @@ func TestCatalogServices(t *testing.T) {
func TestCatalogServices_NodeMetaFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -510,7 +510,7 @@ func TestCatalogServices_NodeMetaFilter(t *testing.T) {
func TestCatalogServiceNodes(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Make sure an empty list is returned, not a nil
@ -563,7 +563,7 @@ func TestCatalogServiceNodes(t *testing.T) {
func TestCatalogServiceNodes_NodeMetaFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Make sure an empty list is returned, not a nil
@ -618,22 +618,22 @@ func TestCatalogServiceNodes_NodeMetaFilter(t *testing.T) {
func TestCatalogServiceNodes_WanTranslation(t *testing.T) {
t.Parallel()
cfg1 := TestConfig()
cfg1.Datacenter = "dc1"
cfg1.TranslateWanAddrs = true
cfg1.ACLDatacenter = ""
a1 := NewTestAgent(t.Name(), cfg1)
a1 := NewTestAgent(t.Name(), `
datacenter = "dc1"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a1.Shutdown()
cfg2 := TestConfig()
cfg2.Datacenter = "dc2"
cfg2.TranslateWanAddrs = true
cfg2.ACLDatacenter = ""
a2 := NewTestAgent(t.Name(), cfg2)
a2 := NewTestAgent(t.Name(), `
datacenter = "dc2"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a2.Shutdown()
// Wait for the WAN join.
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.Ports.SerfWan)
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.SerfPortWAN)
if _, err := a2.srv.agent.JoinWAN([]string{addr}); err != nil {
t.Fatalf("err: %v", err)
}
@ -703,7 +703,7 @@ func TestCatalogServiceNodes_WanTranslation(t *testing.T) {
func TestCatalogServiceNodes_DistanceSort(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register nodes.
@ -790,7 +790,7 @@ func TestCatalogServiceNodes_DistanceSort(t *testing.T) {
func TestCatalogNodeServices(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register node
@ -825,22 +825,22 @@ func TestCatalogNodeServices(t *testing.T) {
func TestCatalogNodeServices_WanTranslation(t *testing.T) {
t.Parallel()
cfg1 := TestConfig()
cfg1.Datacenter = "dc1"
cfg1.TranslateWanAddrs = true
cfg1.ACLDatacenter = ""
a1 := NewTestAgent(t.Name(), cfg1)
a1 := NewTestAgent(t.Name(), `
datacenter = "dc1"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a1.Shutdown()
cfg2 := TestConfig()
cfg2.Datacenter = "dc2"
cfg2.TranslateWanAddrs = true
cfg2.ACLDatacenter = ""
a2 := NewTestAgent(t.Name(), cfg2)
a2 := NewTestAgent(t.Name(), `
datacenter = "dc2"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a2.Shutdown()
// Wait for the WAN join.
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.Ports.SerfWan)
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.SerfPortWAN)
if _, err := a2.srv.agent.JoinWAN([]string{addr}); err != nil {
t.Fatalf("err: %v", err)
}

File diff suppressed because it is too large Load Diff

1174
agent/config/builder.go Normal file

File diff suppressed because it is too large Load Diff

423
agent/config/config.go Normal file
View File

@ -0,0 +1,423 @@
package config
import (
"encoding/json"
"fmt"
"strings"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/hcl"
"github.com/mitchellh/mapstructure"
)
const (
SerfLANKeyring = "serf/local.keyring"
SerfWANKeyring = "serf/remote.keyring"
)
type Source struct {
Name string
Format string
Data string
}
func NewSource(name, data string) Source {
return Source{Name: name, Format: FormatFrom(name), Data: data}
}
func FormatFrom(name string) string {
if strings.HasSuffix(name, ".hcl") {
return "hcl"
}
return "json"
}
// Parse parses a config fragment in either JSON or HCL format.
func Parse(data string, format string) (c Config, err error) {
var raw map[string]interface{}
switch format {
case "json":
err = json.Unmarshal([]byte(data), &raw)
case "hcl":
err = hcl.Decode(&raw, data)
default:
err = fmt.Errorf("invalid format: %s", format)
}
if err != nil {
return Config{}, err
}
// We want to be able to report fields which we cannot map as an
// error so that users find typos in their configuration quickly. To
// achieve this we use the mapstructure library which maps a a raw
// map[string]interface{} to a nested structure and reports unused
// fields. The input for a mapstructure.Decode expects a
// map[string]interface{} as produced by encoding/json.
//
// The HCL language allows to repeat map keys which forces it to
// store nested structs as []map[string]interface{} instead of
// map[string]interface{}. This is an ambiguity which makes the
// generated structures incompatible with a corresponding JSON
// struct. It also does not work well with the mapstructure library.
//
// In order to still use the mapstructure library to find unused
// fields we patch instances of []map[string]interface{} to a
// map[string]interface{} before we decode that into a Config
// struct.
//
// However, Config has some fields which are either
// []map[string]interface{} or are arrays of structs which
// encoding/json will decode to []map[string]interface{}. Therefore,
// we need to be able to specify exceptions for this mapping. The
// patchSliceOfMaps() implements that mapping. All fields of type
// []map[string]interface{} are mapped to map[string]interface{} if
// it contains at most one value. If there is more than one value it
// panics. To define exceptions one can specify the nested field
// names in dot notation.
//
// todo(fs): There might be an easier way to achieve the same thing
// todo(fs): but this approach works for now.
m := patchSliceOfMaps(raw, []string{
"checks",
"segments",
"service.checks",
"services",
"services.checks",
"watches",
})
// toJSON := func(v interface{}) string {
// b, err := json.MarshalIndent(v, "", " ")
// if err != nil {
// panic(err)
// }
// return string(b)
// }
// fmt.Println("raw:", toJSON(raw))
// fmt.Println("patched:", toJSON(m))
var md mapstructure.Metadata
d, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{
Metadata: &md,
Result: &c,
})
if err != nil {
return Config{}, err
}
if err := d.Decode(m); err != nil {
return Config{}, err
}
for _, k := range md.Unused {
err = multierror.Append(err, fmt.Errorf("invalid config key %s", k))
}
return
}
// Config defines the format of a configuration file in either JSON or
// HCL format.
//
// It must contain only pointer values, slices and maps to support
// standardized merging of multiple Config structs into one.
//
// Since this is the format which users use to specify their
// configuration it should be treated as an external API which cannot be
// changed and refactored at will since this will break existing setups.
type Config struct {
ACLAgentMasterToken *string `json:"acl_agent_master_token,omitempty" hcl:"acl_agent_master_token" mapstructure:"acl_agent_master_token"`
ACLAgentToken *string `json:"acl_agent_token,omitempty" hcl:"acl_agent_token" mapstructure:"acl_agent_token"`
ACLDatacenter *string `json:"acl_datacenter,omitempty" hcl:"acl_datacenter" mapstructure:"acl_datacenter"`
ACLDefaultPolicy *string `json:"acl_default_policy,omitempty" hcl:"acl_default_policy" mapstructure:"acl_default_policy"`
ACLDownPolicy *string `json:"acl_down_policy,omitempty" hcl:"acl_down_policy" mapstructure:"acl_down_policy"`
ACLEnforceVersion8 *bool `json:"acl_enforce_version_8,omitempty" hcl:"acl_enforce_version_8" mapstructure:"acl_enforce_version_8"`
ACLMasterToken *string `json:"acl_master_token,omitempty" hcl:"acl_master_token" mapstructure:"acl_master_token"`
ACLReplicationToken *string `json:"acl_replication_token,omitempty" hcl:"acl_replication_token" mapstructure:"acl_replication_token"`
ACLTTL *string `json:"acl_ttl,omitempty" hcl:"acl_ttl" mapstructure:"acl_ttl"`
ACLToken *string `json:"acl_token,omitempty" hcl:"acl_token" mapstructure:"acl_token"`
Addresses Addresses `json:"addresses,omitempty" hcl:"addresses" mapstructure:"addresses"`
AdvertiseAddrLAN *string `json:"advertise_addr,omitempty" hcl:"advertise_addr" mapstructure:"advertise_addr"`
AdvertiseAddrWAN *string `json:"advertise_addr_wan,omitempty" hcl:"advertise_addr_wan" mapstructure:"advertise_addr_wan"`
AdvertiseAddrs AdvertiseAddrsConfig `json:"advertise_addrs,omitempty" hcl:"advertise_addrs" mapstructure:"advertise_addrs"`
Autopilot Autopilot `json:"autopilot,omitempty" hcl:"autopilot" mapstructure:"autopilot"`
BindAddr *string `json:"bind_addr,omitempty" hcl:"bind_addr" mapstructure:"bind_addr"`
Bootstrap *bool `json:"bootstrap,omitempty" hcl:"bootstrap" mapstructure:"bootstrap"`
BootstrapExpect *int `json:"bootstrap_expect,omitempty" hcl:"bootstrap_expect" mapstructure:"bootstrap_expect"`
CAFile *string `json:"ca_file,omitempty" hcl:"ca_file" mapstructure:"ca_file"`
CAPath *string `json:"ca_path,omitempty" hcl:"ca_path" mapstructure:"ca_path"`
CertFile *string `json:"cert_file,omitempty" hcl:"cert_file" mapstructure:"cert_file"`
Check *CheckDefinition `json:"check,omitempty" hcl:"check" mapstructure:"check"` // needs to be a pointer to avoid partial merges
CheckUpdateInterval *string `json:"check_update_interval,omitempty" hcl:"check_update_interval" mapstructure:"check_update_interval"`
Checks []CheckDefinition `json:"checks,omitempty" hcl:"checks" mapstructure:"checks"`
ClientAddr *string `json:"client_addr,omitempty" hcl:"client_addr" mapstructure:"client_addr"`
DNS DNS `json:"dns_config,omitempty" hcl:"dns_config" mapstructure:"dns_config"`
DNSDomain *string `json:"domain,omitempty" hcl:"domain" mapstructure:"domain"`
DNSRecursors []string `json:"recursors,omitempty" hcl:"recursors" mapstructure:"recursors"`
DataDir *string `json:"data_dir,omitempty" hcl:"data_dir" mapstructure:"data_dir"`
Datacenter *string `json:"datacenter,omitempty" hcl:"datacenter" mapstructure:"datacenter"`
DisableAnonymousSignature *bool `json:"disable_anonymous_signature,omitempty" hcl:"disable_anonymous_signature" mapstructure:"disable_anonymous_signature"`
DisableCoordinates *bool `json:"disable_coordinates,omitempty" hcl:"disable_coordinates" mapstructure:"disable_coordinates"`
DisableHostNodeID *bool `json:"disable_host_node_id,omitempty" hcl:"disable_host_node_id" mapstructure:"disable_host_node_id"`
DisableKeyringFile *bool `json:"disable_keyring_file,omitempty" hcl:"disable_keyring_file" mapstructure:"disable_keyring_file"`
DisableRemoteExec *bool `json:"disable_remote_exec,omitempty" hcl:"disable_remote_exec" mapstructure:"disable_remote_exec"`
DisableUpdateCheck *bool `json:"disable_update_check,omitempty" hcl:"disable_update_check" mapstructure:"disable_update_check"`
EnableACLReplication *bool `json:"enable_acl_replication,omitempty" hcl:"enable_acl_replication" mapstructure:"enable_acl_replication"`
EnableDebug *bool `json:"enable_debug,omitempty" hcl:"enable_debug" mapstructure:"enable_debug"`
EnableScriptChecks *bool `json:"enable_script_checks,omitempty" hcl:"enable_script_checks" mapstructure:"enable_script_checks"`
EnableSyslog *bool `json:"enable_syslog,omitempty" hcl:"enable_syslog" mapstructure:"enable_syslog"`
EnableUI *bool `json:"enable_ui,omitempty" hcl:"enable_ui" mapstructure:"enable_ui"`
EncryptKey *string `json:"encrypt,omitempty" hcl:"encrypt" mapstructure:"encrypt"`
EncryptVerifyIncoming *bool `json:"encrypt_verify_incoming,omitempty" hcl:"encrypt_verify_incoming" mapstructure:"encrypt_verify_incoming"`
EncryptVerifyOutgoing *bool `json:"encrypt_verify_outgoing,omitempty" hcl:"encrypt_verify_outgoing" mapstructure:"encrypt_verify_outgoing"`
HTTPConfig HTTPConfig `json:"http_config,omitempty" hcl:"http_config" mapstructure:"http_config"`
KeyFile *string `json:"key_file,omitempty" hcl:"key_file" mapstructure:"key_file"`
LeaveOnTerm *bool `json:"leave_on_terminate,omitempty" hcl:"leave_on_terminate" mapstructure:"leave_on_terminate"`
Limits Limits `json:"limits,omitempty" hcl:"limits" mapstructure:"limits"`
LogLevel *string `json:"log_level,omitempty" hcl:"log_level" mapstructure:"log_level"`
NodeID *string `json:"node_id,omitempty" hcl:"node_id" mapstructure:"node_id"`
NodeMeta map[string]string `json:"node_meta,omitempty" hcl:"node_meta" mapstructure:"node_meta"`
NodeName *string `json:"node_name,omitempty" hcl:"node_name" mapstructure:"node_name"`
NonVotingServer *bool `json:"non_voting_server,omitempty" hcl:"non_voting_server" mapstructure:"non_voting_server"`
Performance Performance `json:"performance,omitempty" hcl:"performance" mapstructure:"performance"`
PidFile *string `json:"pid_file,omitempty" hcl:"pid_file" mapstructure:"pid_file"`
Ports Ports `json:"ports,omitempty" hcl:"ports" mapstructure:"ports"`
RPCProtocol *int `json:"protocol,omitempty" hcl:"protocol" mapstructure:"protocol"`
RaftProtocol *int `json:"raft_protocol,omitempty" hcl:"raft_protocol" mapstructure:"raft_protocol"`
ReconnectTimeoutLAN *string `json:"reconnect_timeout,omitempty" hcl:"reconnect_timeout" mapstructure:"reconnect_timeout"`
ReconnectTimeoutWAN *string `json:"reconnect_timeout_wan,omitempty" hcl:"reconnect_timeout_wan" mapstructure:"reconnect_timeout_wan"`
RejoinAfterLeave *bool `json:"rejoin_after_leave,omitempty" hcl:"rejoin_after_leave" mapstructure:"rejoin_after_leave"`
RetryJoinIntervalLAN *string `json:"retry_interval,omitempty" hcl:"retry_interval" mapstructure:"retry_interval"`
RetryJoinIntervalWAN *string `json:"retry_interval_wan,omitempty" hcl:"retry_interval_wan" mapstructure:"retry_interval_wan"`
RetryJoinLAN []string `json:"retry_join,omitempty" hcl:"retry_join" mapstructure:"retry_join"`
RetryJoinMaxAttemptsLAN *int `json:"retry_max,omitempty" hcl:"retry_max" mapstructure:"retry_max"`
RetryJoinMaxAttemptsWAN *int `json:"retry_max_wan,omitempty" hcl:"retry_max_wan" mapstructure:"retry_max_wan"`
RetryJoinWAN []string `json:"retry_join_wan,omitempty" hcl:"retry_join_wan" mapstructure:"retry_join_wan"`
SegmentName *string `json:"segment,omitempty" hcl:"segment" mapstructure:"segment"`
Segments []Segment `json:"segments,omitempty" hcl:"segments" mapstructure:"segments"`
SerfBindAddrLAN *string `json:"serf_lan,omitempty" hcl:"serf_lan" mapstructure:"serf_lan"`
SerfBindAddrWAN *string `json:"serf_wan,omitempty" hcl:"serf_wan" mapstructure:"serf_wan"`
ServerMode *bool `json:"server,omitempty" hcl:"server" mapstructure:"server"`
ServerName *string `json:"server_name,omitempty" hcl:"server_name" mapstructure:"server_name"`
Service *ServiceDefinition `json:"service,omitempty" hcl:"service" mapstructure:"service"`
Services []ServiceDefinition `json:"services,omitempty" hcl:"services" mapstructure:"services"`
SessionTTLMin *string `json:"session_ttl_min,omitempty" hcl:"session_ttl_min" mapstructure:"session_ttl_min"`
SkipLeaveOnInt *bool `json:"skip_leave_on_interrupt,omitempty" hcl:"skip_leave_on_interrupt" mapstructure:"skip_leave_on_interrupt"`
StartJoinAddrsLAN []string `json:"start_join,omitempty" hcl:"start_join" mapstructure:"start_join"`
StartJoinAddrsWAN []string `json:"start_join_wan,omitempty" hcl:"start_join_wan" mapstructure:"start_join_wan"`
SyslogFacility *string `json:"syslog_facility,omitempty" hcl:"syslog_facility" mapstructure:"syslog_facility"`
TLSCipherSuites *string `json:"tls_cipher_suites,omitempty" hcl:"tls_cipher_suites" mapstructure:"tls_cipher_suites"`
TLSMinVersion *string `json:"tls_min_version,omitempty" hcl:"tls_min_version" mapstructure:"tls_min_version"`
TLSPreferServerCipherSuites *bool `json:"tls_prefer_server_cipher_suites,omitempty" hcl:"tls_prefer_server_cipher_suites" mapstructure:"tls_prefer_server_cipher_suites"`
TaggedAddresses map[string]string `json:"tagged_addresses,omitempty" hcl:"tagged_addresses" mapstructure:"tagged_addresses"`
Telemetry Telemetry `json:"telemetry,omitempty" hcl:"telemetry" mapstructure:"telemetry"`
TranslateWANAddrs *bool `json:"translate_wan_addrs,omitempty" hcl:"translate_wan_addrs" mapstructure:"translate_wan_addrs"`
UIDir *string `json:"ui_dir,omitempty" hcl:"ui_dir" mapstructure:"ui_dir"`
UnixSocket UnixSocket `json:"unix_sockets,omitempty" hcl:"unix_sockets" mapstructure:"unix_sockets"`
VerifyIncoming *bool `json:"verify_incoming,omitempty" hcl:"verify_incoming" mapstructure:"verify_incoming"`
VerifyIncomingHTTPS *bool `json:"verify_incoming_https,omitempty" hcl:"verify_incoming_https" mapstructure:"verify_incoming_https"`
VerifyIncomingRPC *bool `json:"verify_incoming_rpc,omitempty" hcl:"verify_incoming_rpc" mapstructure:"verify_incoming_rpc"`
VerifyOutgoing *bool `json:"verify_outgoing,omitempty" hcl:"verify_outgoing" mapstructure:"verify_outgoing"`
VerifyServerHostname *bool `json:"verify_server_hostname,omitempty" hcl:"verify_server_hostname" mapstructure:"verify_server_hostname"`
Watches []map[string]interface{} `json:"watches,omitempty" hcl:"watches" mapstructure:"watches"`
// non-user configurable values
ACLDisabledTTL *string `json:"acl_disabled_ttl,omitempty" hcl:"acl_disabled_ttl" mapstructure:"acl_disabled_ttl"`
AEInterval *string `json:"ae_interval,omitempty" hcl:"ae_interval" mapstructure:"ae_interval"`
CheckDeregisterIntervalMin *string `json:"check_deregister_interval_min,omitempty" hcl:"check_deregister_interval_min" mapstructure:"check_deregister_interval_min"`
CheckReapInterval *string `json:"check_reap_interval,omitempty" hcl:"check_reap_interval" mapstructure:"check_reap_interval"`
Consul Consul `json:"consul,omitempty" hcl:"consul" mapstructure:"consul"`
Revision *string `json:"revision,omitempty" hcl:"revision" mapstructure:"revision"`
SegmentLimit *int `json:"segment_limit,omitempty" hcl:"segment_limit" mapstructure:"segment_limit"`
SegmentNameLimit *int `json:"segment_name_limit,omitempty" hcl:"segment_name_limit" mapstructure:"segment_name_limit"`
SyncCoordinateIntervalMin *string `json:"sync_coordinate_interval_min,omitempty" hcl:"sync_coordinate_interval_min" mapstructure:"sync_coordinate_interval_min"`
SyncCoordinateRateTarget *float64 `json:"sync_coordinate_rate_target,omitempty" hcl:"sync_coordinate_rate_target" mapstructure:"sync_coordinate_rate_target"`
Version *string `json:"version,omitempty" hcl:"version" mapstructure:"version"`
VersionPrerelease *string `json:"version_prerelease,omitempty" hcl:"version_prerelease" mapstructure:"version_prerelease"`
}
type Consul struct {
Coordinate struct {
UpdateBatchSize *int `json:"update_batch_size,omitempty" hcl:"update_batch_size" mapstructure:"update_batch_size"`
UpdateMaxBatches *int `json:"update_max_batches,omitempty" hcl:"update_max_batches" mapstructure:"update_max_batches"`
UpdatePeriod *string `json:"update_period,omitempty" hcl:"update_period" mapstructure:"update_period"`
} `json:"coordinate,omitempty" hcl:"coordinate" mapstructure:"coordinate"`
Raft struct {
ElectionTimeout *string `json:"election_timeout,omitempty" hcl:"election_timeout" mapstructure:"election_timeout"`
HeartbeatTimeout *string `json:"heartbeat_timeout,omitempty" hcl:"heartbeat_timeout" mapstructure:"heartbeat_timeout"`
LeaderLeaseTimeout *string `json:"leader_lease_timeout,omitempty" hcl:"leader_lease_timeout" mapstructure:"leader_lease_timeout"`
} `json:"raft,omitempty" hcl:"raft" mapstructure:"raft"`
SerfLAN struct {
Memberlist struct {
GossipInterval *string `json:"gossip_interval,omitempty" hcl:"gossip_interval" mapstructure:"gossip_interval"`
ProbeInterval *string `json:"probe_interval,omitempty" hcl:"probe_interval" mapstructure:"probe_interval"`
ProbeTimeout *string `json:"probe_timeout,omitempty" hcl:"probe_timeout" mapstructure:"probe_timeout"`
SuspicionMult *int `json:"suspicion_mult,omitempty" hcl:"suspicion_mult" mapstructure:"suspicion_mult"`
} `json:"memberlist,omitempty" hcl:"memberlist" mapstructure:"memberlist"`
} `json:"serf_lan,omitempty" hcl:"serf_lan" mapstructure:"serf_lan"`
SerfWAN struct {
Memberlist struct {
GossipInterval *string `json:"gossip_interval,omitempty" hcl:"gossip_interval" mapstructure:"gossip_interval"`
ProbeInterval *string `json:"probe_interval,omitempty" hcl:"probe_interval" mapstructure:"probe_interval"`
ProbeTimeout *string `json:"probe_timeout,omitempty" hcl:"probe_timeout" mapstructure:"probe_timeout"`
SuspicionMult *int `json:"suspicion_mult,omitempty" hcl:"suspicion_mult" mapstructure:"suspicion_mult"`
} `json:"memberlist,omitempty" hcl:"memberlist" mapstructure:"memberlist"`
} `json:"serf_wan,omitempty" hcl:"serf_wan" mapstructure:"serf_wan"`
Server struct {
HealthInterval *string `json:"health_interval,omitempty" hcl:"health_interval" mapstructure:"health_interval"`
} `json:"server,omitempty" hcl:"server" mapstructure:"server"`
}
type Addresses struct {
DNS *string `json:"dns,omitempty" hcl:"dns" mapstructure:"dns"`
HTTP *string `json:"http,omitempty" hcl:"http" mapstructure:"http"`
HTTPS *string `json:"https,omitempty" hcl:"https" mapstructure:"https"`
}
type AdvertiseAddrsConfig struct {
RPC *string `json:"rpc,omitempty" hcl:"rpc" mapstructure:"rpc"`
SerfLAN *string `json:"serf_lan,omitempty" hcl:"serf_lan" mapstructure:"serf_lan"`
SerfWAN *string `json:"serf_wan,omitempty" hcl:"serf_wan" mapstructure:"serf_wan"`
}
type Autopilot struct {
CleanupDeadServers *bool `json:"cleanup_dead_servers,omitempty" hcl:"cleanup_dead_servers" mapstructure:"cleanup_dead_servers"`
DisableUpgradeMigration *bool `json:"disable_upgrade_migration,omitempty" hcl:"disable_upgrade_migration" mapstructure:"disable_upgrade_migration"`
LastContactThreshold *string `json:"last_contact_threshold,omitempty" hcl:"last_contact_threshold" mapstructure:"last_contact_threshold"`
MaxTrailingLogs *int `json:"max_trailing_logs,omitempty" hcl:"max_trailing_logs" mapstructure:"max_trailing_logs"`
RedundancyZoneTag *string `json:"redundancy_zone_tag,omitempty" hcl:"redundancy_zone_tag" mapstructure:"redundancy_zone_tag"`
ServerStabilizationTime *string `json:"server_stabilization_time,omitempty" hcl:"server_stabilization_time" mapstructure:"server_stabilization_time"`
UpgradeVersionTag *string `json:"upgrade_version_tag,omitempty" hcl:"upgrade_version_tag" mapstructure:"upgrade_version_tag"`
}
type ServiceDefinition struct {
ID *string `json:"id,omitempty" hcl:"id" mapstructure:"id"`
Name *string `json:"name,omitempty" hcl:"name" mapstructure:"name"`
Tags []string `json:"tags,omitempty" hcl:"tags" mapstructure:"tags"`
Address *string `json:"address,omitempty" hcl:"address" mapstructure:"address"`
Port *int `json:"port,omitempty" hcl:"port" mapstructure:"port"`
Check *CheckDefinition `json:"check,omitempty" hcl:"check" mapstructure:"check"`
Checks []CheckDefinition `json:"checks,omitempty" hcl:"checks" mapstructure:"checks"`
Token *string `json:"token,omitempty" hcl:"token" mapstructure:"token"`
EnableTagOverride *bool `json:"enable_tag_override,omitempty" hcl:"enable_tag_override" mapstructure:"enable_tag_override"`
}
type CheckDefinition struct {
ID *string `json:"id,omitempty" hcl:"id" mapstructure:"id"`
CheckID *string `json:"check_id,omitempty" hcl:"check_id" mapstructure:"check_id"`
Name *string `json:"name,omitempty" hcl:"name" mapstructure:"name"`
Notes *string `json:"notes,omitempty" hcl:"notes" mapstructure:"notes"`
ServiceID *string `json:"service_id,omitempty" hcl:"service_id" mapstructure:"service_id"`
Token *string `json:"token,omitempty" hcl:"token" mapstructure:"token"`
Status *string `json:"status,omitempty" hcl:"status" mapstructure:"status"`
Script *string `json:"script,omitempty" hcl:"script" mapstructure:"script"`
HTTP *string `json:"http,omitempty" hcl:"http" mapstructure:"http"`
Header map[string][]string `json:"header,omitempty" hcl:"header" mapstructure:"header"`
Method *string `json:"method,omitempty" hcl:"method" mapstructure:"method"`
TCP *string `json:"tcp,omitempty" hcl:"tcp" mapstructure:"tcp"`
Interval *string `json:"interval,omitempty" hcl:"interval" mapstructure:"interval"`
DockerContainerID *string `json:"docker_container_id,omitempty" hcl:"docker_container_id" mapstructure:"docker_container_id"`
Shell *string `json:"shell,omitempty" hcl:"shell" mapstructure:"shell"`
TLSSkipVerify *bool `json:"tls_skip_verify,omitempty" hcl:"tls_skip_verify" mapstructure:"tls_skip_verify"`
Timeout *string `json:"timeout,omitempty" hcl:"timeout" mapstructure:"timeout"`
TTL *string `json:"ttl,omitempty" hcl:"ttl" mapstructure:"ttl"`
DeregisterCriticalServiceAfter *string `json:"deregister_critical_service_after,omitempty" hcl:"deregister_critical_service_after" mapstructure:"deregister_critical_service_after"`
}
type DNS struct {
AllowStale *bool `json:"allow_stale,omitempty" hcl:"allow_stale" mapstructure:"allow_stale"`
DisableCompression *bool `json:"disable_compression,omitempty" hcl:"disable_compression" mapstructure:"disable_compression"`
EnableTruncate *bool `json:"enable_truncate,omitempty" hcl:"enable_truncate" mapstructure:"enable_truncate"`
MaxStale *string `json:"max_stale,omitempty" hcl:"max_stale" mapstructure:"max_stale"`
NodeTTL *string `json:"node_ttl,omitempty" hcl:"node_ttl" mapstructure:"node_ttl"`
OnlyPassing *bool `json:"only_passing,omitempty" hcl:"only_passing" mapstructure:"only_passing"`
RecursorTimeout *string `json:"recursor_timeout,omitempty" hcl:"recursor_timeout" mapstructure:"recursor_timeout"`
ServiceTTL map[string]string `json:"service_ttl,omitempty" hcl:"service_ttl" mapstructure:"service_ttl"`
UDPAnswerLimit *int `json:"udp_answer_limit,omitempty" hcl:"udp_answer_limit" mapstructure:"udp_answer_limit"`
}
type HTTPConfig struct {
BlockEndpoints []string `json:"block_endpoints,omitempty" hcl:"block_endpoints" mapstructure:"block_endpoints"`
ResponseHeaders map[string]string `json:"response_headers,omitempty" hcl:"response_headers" mapstructure:"response_headers"`
}
type Performance struct {
RaftMultiplier *int `json:"raft_multiplier,omitempty" hcl:"raft_multiplier" mapstructure:"raft_multiplier"` // todo(fs): validate as uint
}
type Telemetry struct {
CirconusAPIApp *string `json:"circonus_api_app,omitempty" hcl:"circonus_api_app" mapstructure:"circonus_api_app"`
CirconusAPIToken *string `json:"circonus_api_token,omitempty" json:"-" hcl:"circonus_api_token" mapstructure:"circonus_api_token" json:"-"`
CirconusAPIURL *string `json:"circonus_api_url,omitempty" hcl:"circonus_api_url" mapstructure:"circonus_api_url"`
CirconusBrokerID *string `json:"circonus_broker_id,omitempty" hcl:"circonus_broker_id" mapstructure:"circonus_broker_id"`
CirconusBrokerSelectTag *string `json:"circonus_broker_select_tag,omitempty" hcl:"circonus_broker_select_tag" mapstructure:"circonus_broker_select_tag"`
CirconusCheckDisplayName *string `json:"circonus_check_display_name,omitempty" hcl:"circonus_check_display_name" mapstructure:"circonus_check_display_name"`
CirconusCheckForceMetricActivation *string `json:"circonus_check_force_metric_activation,omitempty" hcl:"circonus_check_force_metric_activation" mapstructure:"circonus_check_force_metric_activation"`
CirconusCheckID *string `json:"circonus_check_id,omitempty" hcl:"circonus_check_id" mapstructure:"circonus_check_id"`
CirconusCheckInstanceID *string `json:"circonus_check_instance_id,omitempty" hcl:"circonus_check_instance_id" mapstructure:"circonus_check_instance_id"`
CirconusCheckSearchTag *string `json:"circonus_check_search_tag,omitempty" hcl:"circonus_check_search_tag" mapstructure:"circonus_check_search_tag"`
CirconusCheckTags *string `json:"circonus_check_tags,omitempty" hcl:"circonus_check_tags" mapstructure:"circonus_check_tags"`
CirconusSubmissionInterval *string `json:"circonus_submission_interval,omitempty" hcl:"circonus_submission_interval" mapstructure:"circonus_submission_interval"`
CirconusSubmissionURL *string `json:"circonus_submission_url,omitempty" hcl:"circonus_submission_url" mapstructure:"circonus_submission_url"`
DisableHostname *bool `json:"disable_hostname,omitempty" hcl:"disable_hostname" mapstructure:"disable_hostname"`
DogstatsdAddr *string `json:"dogstatsd_addr,omitempty" hcl:"dogstatsd_addr" mapstructure:"dogstatsd_addr"`
DogstatsdTags []string `json:"dogstatsd_tags,omitempty" hcl:"dogstatsd_tags" mapstructure:"dogstatsd_tags"`
FilterDefault *bool `json:"filter_default,omitempty" hcl:"filter_default" mapstructure:"filter_default"`
PrefixFilter []string `json:"prefix_filter,omitempty" hcl:"prefix_filter" mapstructure:"prefix_filter"`
StatsdAddr *string `json:"statsd_address,omitempty" hcl:"statsd_address" mapstructure:"statsd_address"`
StatsiteAddr *string `json:"statsite_address,omitempty" hcl:"statsite_address" mapstructure:"statsite_address"`
StatsitePrefix *string `json:"statsite_prefix,omitempty" hcl:"statsite_prefix" mapstructure:"statsite_prefix"`
}
type Ports struct {
DNS *int `json:"dns,omitempty" hcl:"dns" mapstructure:"dns"`
HTTP *int `json:"http,omitempty" hcl:"http" mapstructure:"http"`
HTTPS *int `json:"https,omitempty" hcl:"https" mapstructure:"https"`
SerfLAN *int `json:"serf_lan,omitempty" hcl:"serf_lan" mapstructure:"serf_lan"`
SerfWAN *int `json:"serf_wan,omitempty" hcl:"serf_wan" mapstructure:"serf_wan"`
Server *int `json:"server,omitempty" hcl:"server" mapstructure:"server"`
}
type RetryJoinAzure struct {
ClientID *string `json:"client_id,omitempty" hcl:"client_id" mapstructure:"client_id"`
SecretAccessKey *string `json:"secret_access_key,omitempty" hcl:"secret_access_key" mapstructure:"secret_access_key"`
SubscriptionID *string `json:"subscription_id,omitempty" hcl:"subscription_id" mapstructure:"subscription_id"`
TagName *string `json:"tag_name,omitempty" hcl:"tag_name" mapstructure:"tag_name"`
TagValue *string `json:"tag_value,omitempty" hcl:"tag_value" mapstructure:"tag_value"`
TenantID *string `json:"tenant_id,omitempty" hcl:"tenant_id" mapstructure:"tenant_id"`
}
type RetryJoinEC2 struct {
AccessKeyID *string `json:"access_key_id,omitempty" hcl:"access_key_id" mapstructure:"access_key_id"`
Region *string `json:"region,omitempty" hcl:"region" mapstructure:"region"`
SecretAccessKey *string `json:"secret_access_key,omitempty" hcl:"secret_access_key" mapstructure:"secret_access_key"`
TagKey *string `json:"tag_key,omitempty" hcl:"tag_key" mapstructure:"tag_key"`
TagValue *string `json:"tag_value,omitempty" hcl:"tag_value" mapstructure:"tag_value"`
}
type RetryJoinGCE struct {
CredentialsFile *string `json:"credentials_file,omitempty" hcl:"credentials_file" mapstructure:"credentials_file"`
ProjectName *string `json:"project_name,omitempty" hcl:"project_name" mapstructure:"project_name"`
TagValue *string `json:"tag_value,omitempty" hcl:"tag_value" mapstructure:"tag_value"`
ZonePattern *string `json:"zone_pattern,omitempty" hcl:"zone_pattern" mapstructure:"zone_pattern"`
}
type UnixSocket struct {
Group *string `json:"group,omitempty" hcl:"group" mapstructure:"group"`
Mode *string `json:"mode,omitempty" hcl:"mode" mapstructure:"mode"`
User *string `json:"user,omitempty" hcl:"user" mapstructure:"user"`
}
type Limits struct {
RPCMaxBurst *int `json:"rpc_max_burst,omitempty" hcl:"rpc_max_burst" mapstructure:"rpc_max_burst"`
RPCRate *float64 `json:"rpc_rate,omitempty" hcl:"rpc_rate" mapstructure:"rpc_rate"`
}
type Segment struct {
Advertise *string `json:"advertise,omitempty" hcl:"advertise" mapstructure:"advertise"`
Bind *string `json:"bind,omitempty" hcl:"bind" mapstructure:"bind"`
Name *string `json:"name,omitempty" hcl:"name" mapstructure:"name"`
Port *int `json:"port,omitempty" hcl:"port" mapstructure:"port"`
RPCListener *bool `json:"rpc_listener,omitempty" hcl:"rpc_listener" mapstructure:"rpc_listener"`
}

243
agent/config/default.go Normal file
View File

@ -0,0 +1,243 @@
package config
import (
"fmt"
"strconv"
"github.com/hashicorp/consul/agent/consul"
"github.com/hashicorp/consul/version"
)
func DefaultRPCProtocol() (int, error) {
src := DefaultSource()
c, err := Parse(src.Data, src.Format)
if err != nil {
return 0, fmt.Errorf("Error parsing default config: %s", err)
}
if c.RPCProtocol == nil {
return 0, fmt.Errorf("No default RPC protocol set")
}
return *c.RPCProtocol, nil
}
// DefaultSource is the default agent configuration.
// This needs to be merged first in the head.
// todo(fs): The values are sourced from multiple sources.
// todo(fs): IMO, this should be the definitive default for all configurable values
// todo(fs): and whatever is in here should clobber every default value. Hence, no sourcing.
func DefaultSource() Source {
return Source{
Name: "default",
Format: "hcl",
Data: `
acl_default_policy = "allow"
acl_down_policy = "extend-cache"
acl_enforce_version_8 = true
acl_ttl = "30s"
bind_addr = "0.0.0.0"
bootstrap = false
bootstrap_expect = 0
check_update_interval = "5m"
client_addr = "127.0.0.1"
datacenter = "` + consul.DefaultDC + `"
disable_coordinates = false
disable_host_node_id = true
disable_remote_exec = true
domain = "consul."
encrypt_verify_incoming = true
encrypt_verify_outgoing = true
log_level = "INFO"
protocol = 2
retry_interval = "30s"
retry_interval_wan = "30s"
server = false
syslog_facility = "LOCAL0"
tls_min_version = "tls10"
dns_config = {
allow_stale = true
udp_answer_limit = 3
max_stale = "87600h"
recursor_timeout = "2s"
}
limits = {
rpc_rate = -1
rpc_max_burst = 1000
}
performance = {
raft_multiplier = ` + strconv.Itoa(int(consul.DefaultRaftMultiplier)) + `
}
ports = {
dns = 8600
http = 8500
https = -1
serf_lan = ` + strconv.Itoa(consul.DefaultLANSerfPort) + `
serf_wan = ` + strconv.Itoa(consul.DefaultWANSerfPort) + `
server = ` + strconv.Itoa(consul.DefaultRPCPort) + `
}
telemetry = {
statsite_prefix = "consul"
filter_default = true
}
`,
}
}
// DevSource is the additional default configuration for dev mode.
// This should be merged in the head after the default configuration.
func DevSource() Source {
return Source{
Name: "dev",
Format: "hcl",
Data: `
bind_addr = "127.0.0.1"
disable_anonymous_signature = true
disable_keyring_file = true
enable_debug = true
enable_ui = true
log_level = "DEBUG"
server = true
performance = {
raft_multiplier = 1
}
`,
}
}
// NonUserSource contains the values the user cannot configure.
// This needs to be merged in the tail.
func NonUserSource() Source {
return Source{
Name: "non-user",
Format: "hcl",
Data: `
acl_disabled_ttl = "120s"
check_deregister_interval_min = "1m"
check_reap_interval = "30s"
ae_interval = "1m"
sync_coordinate_rate_target = 64
sync_coordinate_interval_min = "15s"
# segment_limit is the maximum number of network segments that may be declared.
segment_limit = 64
# SegmentNameLimit is the maximum segment name length.
segment_name_limit = 64
`,
}
}
// VersionSource creates a config source for the version parameters.
// This should be merged in the tail since these values are not
// user configurable.
func VersionSource(rev, ver, verPre string) Source {
return Source{
Name: "version",
Format: "hcl",
Data: fmt.Sprintf(`revision = %q version = %q version_prerelease = %q`, rev, ver, verPre),
}
}
// DefaultVersionSource returns the version config source for the embedded
// version numbers.
func DefaultVersionSource() Source {
return VersionSource(version.GitCommit, version.Version, version.VersionPrerelease)
}
// DefaultConsulSource returns the default configuration for the consul agent.
// This should be merged in the tail since these values are not user configurable.
func DefaultConsulSource() Source {
cfg := consul.DefaultConfig()
raft := cfg.RaftConfig
serfLAN := cfg.SerfLANConfig.MemberlistConfig
serfWAN := cfg.SerfWANConfig.MemberlistConfig
return Source{
Name: "consul",
Format: "hcl",
Data: `
consul = {
coordinate = {
update_batch_size = ` + strconv.Itoa(cfg.CoordinateUpdateBatchSize) + `
update_max_batches = ` + strconv.Itoa(cfg.CoordinateUpdateMaxBatches) + `
update_period = "` + cfg.CoordinateUpdatePeriod.String() + `"
}
raft = {
election_timeout = "` + raft.ElectionTimeout.String() + `"
heartbeat_timeout = "` + raft.HeartbeatTimeout.String() + `"
leader_lease_timeout = "` + raft.LeaderLeaseTimeout.String() + `"
}
serf_lan = {
memberlist = {
gossip_interval = "` + serfLAN.GossipInterval.String() + `"
probe_interval = "` + serfLAN.ProbeInterval.String() + `"
probe_timeout = "` + serfLAN.ProbeTimeout.String() + `"
suspicion_mult = ` + strconv.Itoa(serfLAN.SuspicionMult) + `
}
}
serf_wan = {
memberlist = {
gossip_interval = "` + serfWAN.GossipInterval.String() + `"
probe_interval = "` + serfWAN.ProbeInterval.String() + `"
probe_timeout = "` + serfWAN.ProbeTimeout.String() + `"
suspicion_mult = ` + strconv.Itoa(serfWAN.SuspicionMult) + `
}
}
server = {
health_interval = "` + cfg.ServerHealthInterval.String() + `"
}
}
`,
}
}
// DevConsulSource returns the consul agent configuration for the dev mode.
// This should be merged in the tail after the DefaultConsulSource.
func DevConsulSource() Source {
return Source{
Name: "consul-dev",
Format: "hcl",
Data: `
consul = {
coordinate = {
update_period = "100ms"
}
raft = {
election_timeout = "52ms"
heartbeat_timeout = "35ms"
leader_lease_timeout = "20ms"
}
serf_lan = {
memberlist = {
gossip_interval = "100ms"
probe_interval = "100ms"
probe_timeout = "100ms"
suspicion_mult = 3
}
}
serf_wan = {
memberlist = {
gossip_interval = "100ms"
probe_interval = "100ms"
probe_timeout = "100ms"
suspicion_mult = 3
}
}
server = {
health_interval = "10ms"
}
}
`,
}
}
func DefaultRuntimeConfig(hcl string) *RuntimeConfig {
b, err := NewBuilder(Flags{HCL: []string{hcl}})
if err != nil {
panic(err)
}
rt, err := b.BuildAndValidate()
if err != nil {
panic(err)
}
return &rt
}

81
agent/config/doc.go Normal file
View File

@ -0,0 +1,81 @@
// Package config contains the command line and config file code for the
// consul agent.
//
// The consul agent configuration is generated from multiple sources:
//
// * config files
// * environment variables (which?)
// * cmd line args
//
// Each of these argument sets needs to be parsed, validated and then
// merged with the other sources to build the final configuration.
//
// This patch introduces a distinction between the user and the runtime
// configuration. The user configuration defines the external interface for
// the user, i.e. the command line flags, the environment variables and the
// config file format which cannot be changed without breaking the users'
// setup.
//
// The runtime configuration is the merged, validated and mangled
// configuration structure suitable for the consul agent. Both structures
// are similar but different and the runtime configuration can be
// refactored at will without affecting the user configuration format.
//
// For this, the user configuration consists of several structures for
// config files and command line arguments. Again, the config file and
// command line structs are similar but not identical for historical
// reasons and to allow evolving them differently.
//
// All of the user configuration structs have pointer values to
// unambiguously merge values from several sources into the final value.
//
// The runtime configuration has no pointer values and should be passed by
// value to avoid accidental or malicious runtime configuration changes.
// Runtime updates need to be handled through a new configuration
// instances.
// # Removed command line flags
//
// * "-atlas" is deprecated and is no longer used. Please remove it from your configuration.
// * "-atlas-token" is deprecated and is no longer used. Please remove it from your configuration.
// * "-atlas-join" is deprecated and is no longer used. Please remove it from your configuration.
// * "-atlas-endpoint" is deprecated and is no longer used. Please remove it from your configuration.
// * "-dc" is deprecated. Please use "-datacenter" instead
// * "-retry-join-azure-tag-name" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-azure-tag-value" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-ec2-region" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-ec2-tag-key" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-ec2-tag-value" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-gce-credentials-file" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-gce-project-name" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-gce-tag-name" is deprecated. Please use "-retry-join" instead.
// * "-retry-join-gce-zone-pattern" is deprecated. Please use "-retry-join" instead.
//
// # Removed configuration fields
//
// * "addresses.rpc" is deprecated and is no longer used. Please remove it from your configuration.
// * "ports.rpc" is deprecated and is no longer used. Please remove it from your configuration.
// * "atlas_infrastructure" is deprecated and is no longer used. Please remove it from your configuration.
// * "atlas_token" is deprecated and is no longer used. Please remove it from your configuration.
// * "atlas_acl_token" is deprecated and is no longer used. Please remove it from your configuration.
// * "atlas_join" is deprecated and is no longer used. Please remove it from your configuration.
// * "atlas_endpoint" is deprecated and is no longer used. Please remove it from your configuration.
// * "http_api_response_headers" is deprecated. Please use "http_config.response_headers" instead.
// * "dogstatsd_addr" is deprecated. Please use "telemetry.dogstatsd_addr" instead.
// * "dogstatsd_tags" is deprecated. Please use "telemetry.dogstatsd_tags" instead.
// * "recursor" is deprecated. Please use "recursors" instead.
// * "statsd_addr" is deprecated. Please use "telemetry.statsd_addr" instead.
// * "statsite_addr" is deprecated. Please use "telemetry.statsite_addr" instead.
// * "statsite_prefix" is deprecated. Please use "telemetry.statsite_prefix" instead.
// * "retry_join_azure" is deprecated. Please use "retry_join" instead.
// * "retry_join_ec2" is deprecated. Please use "retry_join" instead.
// * "retry_join_gce" is deprecated. Please use "retry_join" instead.
//
// # Removed service config alias fields
//
// * "serviceid" is deprecated in service definitions. Please use "service_id" instead.
// * "dockercontainerid" is deprecated in service definitions. Please use "docker_container_id" instead.
// * "tlsskipverify" is deprecated in service definitions. Please use "tls_skip_verify" instead.
// * "deregistercriticalserviceafter" is deprecated in service definitions. Please use "deregister_critical_service_after" instead.
package config

104
agent/config/flags.go Normal file
View File

@ -0,0 +1,104 @@
package config
import (
"flag"
"fmt"
"time"
)
// Flags defines the command line flags.
type Flags struct {
// Config contains the command line arguments that can also be set
// in a config file.
Config Config
// ConfigFiles contains the list of config files and directories
// that should be read.
ConfigFiles []string
// DevMode indicates whether the agent should be started in development
// mode. This cannot be configured in a config file.
DevMode *bool
// HCL contains an arbitrary config in hcl format.
HCL []string
}
// ParseFlag parses the arguments into a Flags struct.
func ParseFlags(args []string) (Flags, error) {
var f Flags
fs := flag.NewFlagSet("agent", flag.ContinueOnError)
AddFlags(fs, &f)
if err := fs.Parse(args); err != nil {
return Flags{}, err
}
return f, nil
}
// AddFlags adds the command line flags for the agent.
func AddFlags(fs *flag.FlagSet, f *Flags) {
add := func(p interface{}, name, help string) {
switch x := p.(type) {
case **bool:
fs.Var(newBoolPtrValue(x), name, help)
case **time.Duration:
fs.Var(newDurationPtrValue(x), name, help)
case **int:
fs.Var(newIntPtrValue(x), name, help)
case **string:
fs.Var(newStringPtrValue(x), name, help)
case *[]string:
fs.Var(newStringSliceValue(x), name, help)
case *map[string]string:
fs.Var(newStringMapValue(x), name, help)
default:
panic(fmt.Sprintf("invalid type: %T", p))
}
}
// command line flags ordered by flag name
add(&f.Config.AdvertiseAddrLAN, "advertise", "Sets the advertise address to use.")
add(&f.Config.AdvertiseAddrWAN, "advertise-wan", "Sets address to advertise on WAN instead of -advertise address.")
add(&f.Config.BindAddr, "bind", "Sets the bind address for cluster communication.")
add(&f.Config.Bootstrap, "bootstrap", "Sets server to bootstrap mode.")
add(&f.Config.BootstrapExpect, "bootstrap-expect", "Sets server to expect bootstrap mode.")
add(&f.Config.ClientAddr, "client", "Sets the address to bind for client access. This includes RPC, DNS, HTTP and HTTPS (if configured).")
add(&f.ConfigFiles, "config-dir", "Path to a directory to read configuration files from. This will read every file ending in '.json' as configuration in this directory in alphabetical order. Can be specified multiple times.")
add(&f.ConfigFiles, "config-file", "Path to a JSON file to read configuration from. Can be specified multiple times.")
add(&f.Config.DataDir, "data-dir", "Path to a data directory to store agent state.")
add(&f.Config.Datacenter, "datacenter", "Datacenter of the agent.")
add(&f.DevMode, "dev", "Starts the agent in development mode.")
add(&f.Config.DisableHostNodeID, "disable-host-node-id", "Setting this to true will prevent Consul from using information from the host to generate a node ID, and will cause Consul to generate a random node ID instead.")
add(&f.Config.DisableKeyringFile, "disable-keyring-file", "Disables the backing up of the keyring to a file.")
add(&f.Config.Ports.DNS, "dns-port", "DNS port to use.")
add(&f.Config.DNSDomain, "domain", "Domain to use for DNS interface.")
add(&f.Config.EnableScriptChecks, "enable-script-checks", "Enables health check scripts.")
add(&f.Config.EncryptKey, "encrypt", "Provides the gossip encryption key.")
add(&f.Config.Ports.HTTP, "http-port", "Sets the HTTP API port to listen on.")
add(&f.Config.StartJoinAddrsLAN, "join", "Address of an agent to join at start time. Can be specified multiple times.")
add(&f.Config.StartJoinAddrsWAN, "join-wan", "Address of an agent to join -wan at start time. Can be specified multiple times.")
add(&f.Config.LogLevel, "log-level", "Log level of the agent.")
add(&f.Config.NodeName, "node", "Name of this node. Must be unique in the cluster.")
add(&f.Config.NodeID, "node-id", "A unique ID for this node across space and time. Defaults to a randomly-generated ID that persists in the data-dir.")
add(&f.Config.NodeMeta, "node-meta", "An arbitrary metadata key/value pair for this node, of the format `key:value`. Can be specified multiple times.")
add(&f.Config.NonVotingServer, "non-voting-server", "(Enterprise-only) This flag is used to make the server not participate in the Raft quorum, and have it only receive the data replication stream. This can be used to add read scalability to a cluster in cases where a high volume of reads to servers are needed.")
add(&f.Config.PidFile, "pid-file", "Path to file to store agent PID.")
add(&f.Config.RPCProtocol, "protocol", "Sets the protocol version. Defaults to latest.")
add(&f.Config.RaftProtocol, "raft-protocol", "Sets the Raft protocol version. Defaults to latest.")
add(&f.Config.DNSRecursors, "recursor", "Address of an upstream DNS server. Can be specified multiple times.")
add(&f.Config.RejoinAfterLeave, "rejoin", "Ignores a previous leave and attempts to rejoin the cluster.")
add(&f.Config.RetryJoinIntervalLAN, "retry-interval", "Time to wait between join attempts.")
add(&f.Config.RetryJoinIntervalWAN, "retry-interval-wan", "Time to wait between join -wan attempts.")
add(&f.Config.RetryJoinLAN, "retry-join", "Address of an agent to join at start time with retries enabled. Can be specified multiple times.")
add(&f.Config.RetryJoinWAN, "retry-join-wan", "Address of an agent to join -wan at start time with retries enabled. Can be specified multiple times.")
add(&f.Config.RetryJoinMaxAttemptsLAN, "retry-max", "Maximum number of join attempts. Defaults to 0, which will retry indefinitely.")
add(&f.Config.RetryJoinMaxAttemptsWAN, "retry-max-wan", "Maximum number of join -wan attempts. Defaults to 0, which will retry indefinitely.")
add(&f.Config.SerfBindAddrLAN, "serf-lan-bind", "Address to bind Serf LAN listeners to.")
add(&f.Config.SegmentName, "segment", "(Enterprise-only) Sets the network segment to join.")
add(&f.Config.SerfBindAddrWAN, "serf-wan-bind", "Address to bind Serf WAN listeners to.")
add(&f.Config.ServerMode, "server", "Switches agent to server mode.")
add(&f.Config.EnableSyslog, "syslog", "Enables logging to syslog.")
add(&f.Config.EnableUI, "ui", "Enables the built-in static web UI server.")
add(&f.Config.UIDir, "ui-dir", "Path to directory containing the web UI resources.")
add(&f.HCL, "hcl", "hcl config fragment. Can be specified multiple times.")
}

View File

@ -0,0 +1,75 @@
package config
import (
"reflect"
"strings"
"testing"
"github.com/pascaldekloe/goe/verify"
)
// TestParseFlags tests whether command line flags are properly parsed
// into the Flags/File structure. It contains an example for every type
// that is parsed. It does not test the conversion into the final
// runtime configuration. See TestConfig for that.
func TestParseFlags(t *testing.T) {
tests := []struct {
args []string
flags Flags
err error
}{
{},
{
args: []string{`-bind`, `a`},
flags: Flags{Config: Config{BindAddr: pString("a")}},
},
{
args: []string{`-bootstrap`},
flags: Flags{Config: Config{Bootstrap: pBool(true)}},
},
{
args: []string{`-bootstrap=true`},
flags: Flags{Config: Config{Bootstrap: pBool(true)}},
},
{
args: []string{`-bootstrap=false`},
flags: Flags{Config: Config{Bootstrap: pBool(false)}},
},
{
args: []string{`-bootstrap`, `true`},
flags: Flags{Config: Config{Bootstrap: pBool(true)}},
},
{
args: []string{`-config-file`, `a`, `-config-dir`, `b`, `-config-file`, `c`, `-config-dir`, `d`},
flags: Flags{ConfigFiles: []string{"a", "b", "c", "d"}},
},
{
args: []string{`-datacenter`, `a`},
flags: Flags{Config: Config{Datacenter: pString("a")}},
},
{
args: []string{`-dns-port`, `1`},
flags: Flags{Config: Config{Ports: Ports{DNS: pInt(1)}}},
},
{
args: []string{`-join`, `a`, `-join`, `b`},
flags: Flags{Config: Config{StartJoinAddrsLAN: []string{"a", "b"}}},
},
{
args: []string{`-node-meta`, `a:b`, `-node-meta`, `c:d`},
flags: Flags{Config: Config{NodeMeta: map[string]string{"a": "b", "c": "d"}}},
},
}
for _, tt := range tests {
t.Run(strings.Join(tt.args, " "), func(t *testing.T) {
flags, err := ParseFlags(tt.args)
if got, want := err, tt.err; !reflect.DeepEqual(got, want) {
t.Fatalf("got error %v want %v", got, want)
}
if !verify.Values(t, "flag", flags, tt.flags) {
t.FailNow()
}
})
}
}

200
agent/config/flagset.go Normal file
View File

@ -0,0 +1,200 @@
package config
import (
"strconv"
"strings"
"time"
)
// boolPtrValue is a flag.Value which stores the value in a *bool if it
// can be parsed with strconv.ParseBool. If the value was not set the
// pointer is nil.
type boolPtrValue struct {
v **bool
b bool
}
func newBoolPtrValue(p **bool) *boolPtrValue {
return &boolPtrValue{p, false}
}
func (s *boolPtrValue) IsBoolFlag() bool { return true }
func (s *boolPtrValue) Set(val string) error {
b, err := strconv.ParseBool(val)
if err != nil {
return err
}
*s.v, s.b = &b, true
return nil
}
func (s *boolPtrValue) Get() interface{} {
if s.b {
return *s.v
}
return (*bool)(nil)
}
func (s *boolPtrValue) String() string {
if s.b {
return strconv.FormatBool(**s.v)
}
return ""
}
// durationPtrValue is a flag.Value which stores the value in a
// *time.Duration if it can be parsed with time.ParseDuration. If the
// value was not set the pointer is nil.
type durationPtrValue struct {
v **time.Duration
b bool
}
func newDurationPtrValue(p **time.Duration) *durationPtrValue {
return &durationPtrValue{p, false}
}
func (s *durationPtrValue) Set(val string) error {
d, err := time.ParseDuration(val)
if err != nil {
return err
}
*s.v, s.b = &d, true
return nil
}
func (s *durationPtrValue) Get() interface{} {
if s.b {
return *s.v
}
return (*time.Duration)(nil)
}
func (s *durationPtrValue) String() string {
if s.b {
return (*(*s).v).String()
}
return ""
}
// intPtrValue is a flag.Value which stores the value in a *int if it
// can be parsed with strconv.Atoi. If the value was not set the pointer
// is nil.
type intPtrValue struct {
v **int
b bool
}
func newIntPtrValue(p **int) *intPtrValue {
return &intPtrValue{p, false}
}
func (s *intPtrValue) Set(val string) error {
n, err := strconv.Atoi(val)
if err != nil {
return err
}
*s.v, s.b = &n, true
return nil
}
func (s *intPtrValue) Get() interface{} {
if s.b {
return *s.v
}
return (*int)(nil)
}
func (s *intPtrValue) String() string {
if s.b {
return strconv.Itoa(**s.v)
}
return ""
}
// stringMapValue is a flag.Value which stores the value in a map[string]string if the
// value is in "key:value" format. This can be specified multiple times.
type stringMapValue map[string]string
func newStringMapValue(p *map[string]string) *stringMapValue {
*p = map[string]string{}
return (*stringMapValue)(p)
}
func (s *stringMapValue) Set(val string) error {
p := strings.SplitN(val, ":", 2)
k, v := p[0], ""
if len(p) == 2 {
v = p[1]
}
(*s)[k] = v
return nil
}
func (s *stringMapValue) Get() interface{} {
return s
}
func (s *stringMapValue) String() string {
var x []string
for k, v := range *s {
if v == "" {
x = append(x, k)
} else {
x = append(x, k+":"+v)
}
}
return strings.Join(x, " ")
}
// stringPtrValue is a flag.Value which stores the value in a *string.
// If the value was not set the pointer is nil.
type stringPtrValue struct {
v **string
b bool
}
func newStringPtrValue(p **string) *stringPtrValue {
return &stringPtrValue{p, false}
}
func (s *stringPtrValue) Set(val string) error {
*s.v, s.b = &val, true
return nil
}
func (s *stringPtrValue) Get() interface{} {
if s.b {
return *s.v
}
return (*string)(nil)
}
func (s *stringPtrValue) String() string {
if s.b {
return **s.v
}
return ""
}
// stringSliceValue is a flag.Value which appends the value to a []string.
// This can be specified multiple times.
type stringSliceValue []string
func newStringSliceValue(p *[]string) *stringSliceValue {
return (*stringSliceValue)(p)
}
func (s *stringSliceValue) Set(val string) error {
*s = append(*s, val)
return nil
}
func (s *stringSliceValue) Get() interface{} {
return s
}
func (s *stringSliceValue) String() string {
return strings.Join(*s, " ")
}

59
agent/config/merge.go Normal file
View File

@ -0,0 +1,59 @@
package config
import (
"fmt"
"reflect"
)
// Merge recursively combines a set of config file structures into a single structure
// according to the following rules:
//
// * only values of type struct, slice, map and pointer to simple types are allowed. Other types panic.
// * when merging two structs the result is the recursive merge of all fields according to the rules below
// * when merging two slices the result is the second slice appended to the first
// * when merging two maps the result is the second map if it is not empty, otherwise the first
// * when merging two pointer values the result is the second value if it is not nil, otherwise the first
func Merge(files ...Config) Config {
var a Config
for _, b := range files {
a = merge(a, b).(Config)
}
return a
}
func merge(a, b interface{}) interface{} {
return mergeValue(reflect.ValueOf(a), reflect.ValueOf(b)).Interface()
}
func mergeValue(a, b reflect.Value) reflect.Value {
switch a.Kind() {
case reflect.Map:
if b.Len() > 0 {
return b
}
return a
case reflect.Ptr:
if !b.IsNil() {
return b
}
return a
case reflect.Slice:
if !a.IsValid() {
a = reflect.Zero(a.Type())
}
return reflect.AppendSlice(a, b)
case reflect.Struct:
r := reflect.New(a.Type()) // &struct{}
for i := 0; i < a.NumField(); i++ {
v := mergeValue(a.Field(i), b.Field(i))
r.Elem().Field(i).Set(v)
}
return r.Elem() // *struct
default:
panic(fmt.Sprintf("unsupported element type: %v", a.Type()))
}
}

View File

@ -0,0 +1,57 @@
package config
import (
"testing"
"time"
"github.com/pascaldekloe/goe/verify"
)
func TestMerge(t *testing.T) {
tests := []struct {
desc string
cfgs []Config
want Config
}{
{
"top level fields",
[]Config{
{AdvertiseAddrLAN: pString("a")},
{AdvertiseAddrLAN: pString("b")},
{RaftProtocol: pInt(1)},
{RaftProtocol: pInt(2)},
{ServerMode: pBool(false)},
{ServerMode: pBool(true)},
{StartJoinAddrsLAN: []string{"a"}},
{StartJoinAddrsLAN: []string{"b"}},
{NodeMeta: map[string]string{"a": "b"}},
{NodeMeta: map[string]string{"c": "d"}},
{Ports: Ports{DNS: pInt(1)}},
{Ports: Ports{DNS: pInt(2), HTTP: pInt(3)}},
},
Config{
AdvertiseAddrLAN: pString("b"),
RaftProtocol: pInt(2),
ServerMode: pBool(true),
StartJoinAddrsLAN: []string{"a", "b"},
NodeMeta: map[string]string{"c": "d"},
Ports: Ports{DNS: pInt(2), HTTP: pInt(3)},
},
},
}
for _, tt := range tests {
t.Run(tt.desc, func(t *testing.T) {
got, want := Merge(tt.cfgs...), tt.want
if !verify.Values(t, "", got, want) {
t.FailNow()
}
})
}
}
func pBool(v bool) *bool { return &v }
func pInt(v int) *int { return &v }
func pString(v string) *string { return &v }
func pDuration(v time.Duration) *string { s := v.String(); return &s }
func pFloat64(v float64) *float64 { return &v }

73
agent/config/patch_hcl.go Normal file
View File

@ -0,0 +1,73 @@
package config
import (
"fmt"
)
func patchSliceOfMaps(m map[string]interface{}, skip []string) map[string]interface{} {
return patchValue("", m, skip).(map[string]interface{})
}
func patchValue(name string, v interface{}, skip []string) interface{} {
// fmt.Printf("%q: %T\n", name, v)
switch x := v.(type) {
case map[string]interface{}:
if len(x) == 0 {
return x
}
mm := make(map[string]interface{})
for k, v := range x {
key := k
if name != "" {
key = name + "." + k
}
mm[k] = patchValue(key, v, skip)
}
return mm
case []interface{}:
if len(x) == 0 {
return nil
}
if strSliceContains(name, skip) {
for i, y := range x {
x[i] = patchValue(name, y, skip)
}
return x
}
if _, ok := x[0].(map[string]interface{}); !ok {
return x
}
if len(x) > 1 {
panic(fmt.Sprintf("%s: []map[string]interface{} with more than one element not supported: %s", name, v))
}
return patchValue(name, x[0], skip)
case []map[string]interface{}:
if len(x) == 0 {
return nil
}
if strSliceContains(name, skip) {
for i, y := range x {
x[i] = patchValue(name, y, skip).(map[string]interface{})
}
return x
}
if len(x) > 1 {
panic(fmt.Sprintf("%s: []map[string]interface{} with more than one element not supported: %s", name, v))
}
return patchValue(name, x[0], skip)
default:
return v
}
}
func strSliceContains(s string, v []string) bool {
for _, vv := range v {
if s == vv {
return true
}
}
return false
}

View File

@ -0,0 +1,78 @@
package config
import (
"encoding/json"
"fmt"
"reflect"
"testing"
)
func parse(s string) map[string]interface{} {
var m map[string]interface{}
if err := json.Unmarshal([]byte(s), &m); err != nil {
panic(s + ":" + err.Error())
}
return m
}
func TestPatchSliceOfMaps(t *testing.T) {
tests := []struct {
in, out string
skip []string
}{
{
in: `{"a":{"b":"c"}}`,
out: `{"a":{"b":"c"}}`,
},
{
in: `{"a":[{"b":"c"}]}`,
out: `{"a":{"b":"c"}}`,
},
{
in: `{"a":[{"b":[{"c":"d"}]}]}`,
out: `{"a":{"b":{"c":"d"}}}`,
},
{
in: `{"a":[{"b":"c"}]}`,
out: `{"a":[{"b":"c"}]}`,
skip: []string{"a"},
},
{
in: `{
"services": [
{
"checks": [
{
"header": [
{"a":"b"}
]
}
]
}
]
}`,
out: `{
"services": [
{
"checks": [
{
"header": {"a":"b"}
}
]
}
]
}`,
skip: []string{"services", "services.checks"},
},
}
for i, tt := range tests {
desc := fmt.Sprintf("%02d: %s -> %s skip: %v", i, tt.in, tt.out, tt.skip)
t.Run(desc, func(t *testing.T) {
out := patchSliceOfMaps(parse(tt.in), tt.skip)
if got, want := out, parse(tt.out); !reflect.DeepEqual(got, want) {
t.Fatalf("\ngot %#v\nwant %#v", got, want)
}
})
}
}

258
agent/config/runtime.go Normal file
View File

@ -0,0 +1,258 @@
package config
import (
"crypto/tls"
"net"
"reflect"
"strings"
"time"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/consul/types"
"golang.org/x/time/rate"
)
// RuntimeConfig specifies the configuration the consul agent actually
// uses. Is is derived from one or more Config structures which can come
// from files, flags and/or environment variables.
type RuntimeConfig struct {
// non-user configurable values
AEInterval time.Duration
ACLDisabledTTL time.Duration
CheckDeregisterIntervalMin time.Duration
CheckReapInterval time.Duration
SegmentLimit int
SegmentNameLimit int
SyncCoordinateRateTarget float64
SyncCoordinateIntervalMin time.Duration
Revision string
Version string
VersionPrerelease string
// consul config
ConsulCoordinateUpdateMaxBatches int
ConsulCoordinateUpdateBatchSize int
ConsulCoordinateUpdatePeriod time.Duration
ConsulRaftElectionTimeout time.Duration
ConsulRaftHeartbeatTimeout time.Duration
ConsulRaftLeaderLeaseTimeout time.Duration
ConsulSerfLANGossipInterval time.Duration
ConsulSerfLANProbeInterval time.Duration
ConsulSerfLANProbeTimeout time.Duration
ConsulSerfLANSuspicionMult int
ConsulSerfWANGossipInterval time.Duration
ConsulSerfWANProbeInterval time.Duration
ConsulSerfWANProbeTimeout time.Duration
ConsulSerfWANSuspicionMult int
ConsulServerHealthInterval time.Duration
ACLAgentMasterToken string
ACLAgentToken string
ACLDatacenter string
ACLDefaultPolicy string
ACLDownPolicy string
ACLEnforceVersion8 bool
ACLMasterToken string
ACLReplicationToken string
ACLTTL time.Duration
ACLToken string
AutopilotCleanupDeadServers bool
AutopilotDisableUpgradeMigration bool
AutopilotLastContactThreshold time.Duration
AutopilotMaxTrailingLogs int
AutopilotRedundancyZoneTag string
AutopilotServerStabilizationTime time.Duration
AutopilotUpgradeVersionTag string
DNSAllowStale bool
DNSDisableCompression bool
DNSDomain string
DNSEnableTruncate bool
DNSMaxStale time.Duration
DNSNodeTTL time.Duration
DNSOnlyPassing bool
DNSRecursorTimeout time.Duration
DNSServiceTTL map[string]time.Duration
DNSUDPAnswerLimit int
DNSRecursors []string
HTTPBlockEndpoints []string
HTTPResponseHeaders map[string]string
TelemetryCirconusAPIApp string
TelemetryCirconusAPIToken string
TelemetryCirconusAPIURL string
TelemetryCirconusBrokerID string
TelemetryCirconusBrokerSelectTag string
TelemetryCirconusCheckDisplayName string
TelemetryCirconusCheckForceMetricActivation string
TelemetryCirconusCheckID string
TelemetryCirconusCheckInstanceID string
TelemetryCirconusCheckSearchTag string
TelemetryCirconusCheckTags string
TelemetryCirconusSubmissionInterval string
TelemetryCirconusSubmissionURL string
TelemetryDisableHostname bool
TelemetryDogstatsdAddr string
TelemetryDogstatsdTags []string
TelemetryFilterDefault bool
TelemetryAllowedPrefixes []string
TelemetryBlockedPrefixes []string
TelemetryStatsdAddr string
TelemetryStatsiteAddr string
TelemetryStatsitePrefix string
AdvertiseAddrLAN *net.IPAddr
AdvertiseAddrWAN *net.IPAddr
BindAddr *net.IPAddr
Bootstrap bool
BootstrapExpect int
CAFile string
CAPath string
CertFile string
CheckUpdateInterval time.Duration
Checks []*structs.CheckDefinition
ClientAddrs []*net.IPAddr
DNSAddrs []net.Addr
DNSPort int
DataDir string
Datacenter string
DevMode bool
DisableAnonymousSignature bool
DisableCoordinates bool
DisableHostNodeID bool
DisableKeyringFile bool
DisableRemoteExec bool
DisableUpdateCheck bool
EnableACLReplication bool
EnableDebug bool
EnableScriptChecks bool
EnableSyslog bool
EnableUI bool
EncryptKey string
EncryptVerifyIncoming bool
EncryptVerifyOutgoing bool
HTTPAddrs []net.Addr
HTTPPort int
HTTPSAddrs []net.Addr
HTTPSPort int
KeyFile string
LeaveOnTerm bool
LogLevel string
NodeID types.NodeID
NodeMeta map[string]string
NodeName string
NonVotingServer bool
PidFile string
RPCAdvertiseAddr *net.TCPAddr
RPCBindAddr *net.TCPAddr
RPCMaxBurst int
RPCProtocol int
RPCRateLimit rate.Limit
RaftProtocol int
ReconnectTimeoutLAN time.Duration
ReconnectTimeoutWAN time.Duration
RejoinAfterLeave bool
RetryJoinIntervalLAN time.Duration
RetryJoinIntervalWAN time.Duration
RetryJoinLAN []string
RetryJoinMaxAttemptsLAN int
RetryJoinMaxAttemptsWAN int
RetryJoinWAN []string
SegmentName string
Segments []structs.NetworkSegment
SerfAdvertiseAddrLAN *net.TCPAddr
SerfAdvertiseAddrWAN *net.TCPAddr
SerfBindAddrLAN *net.TCPAddr
SerfBindAddrWAN *net.TCPAddr
SerfPortLAN int
SerfPortWAN int
ServerMode bool
ServerName string
ServerPort int
Services []*structs.ServiceDefinition
SessionTTLMin time.Duration
SkipLeaveOnInt bool
StartJoinAddrsLAN []string
StartJoinAddrsWAN []string
SyslogFacility string
TLSCipherSuites []uint16
TLSMinVersion string
TLSPreferServerCipherSuites bool
TaggedAddresses map[string]string
TranslateWANAddrs bool
UIDir string
UnixSocketGroup string
UnixSocketMode string
UnixSocketUser string
VerifyIncoming bool
VerifyIncomingHTTPS bool
VerifyIncomingRPC bool
VerifyOutgoing bool
VerifyServerHostname bool
Watches []map[string]interface{}
}
// IncomingHTTPSConfig returns the TLS configuration for HTTPS
// connections to consul.
func (c *RuntimeConfig) IncomingHTTPSConfig() (*tls.Config, error) {
tc := &tlsutil.Config{
VerifyIncoming: c.VerifyIncoming || c.VerifyIncomingHTTPS,
VerifyOutgoing: c.VerifyOutgoing,
CAFile: c.CAFile,
CAPath: c.CAPath,
CertFile: c.CertFile,
KeyFile: c.KeyFile,
NodeName: c.NodeName,
ServerName: c.ServerName,
TLSMinVersion: c.TLSMinVersion,
CipherSuites: c.TLSCipherSuites,
PreferServerCipherSuites: c.TLSPreferServerCipherSuites,
}
return tc.IncomingTLSConfig()
}
func (c *RuntimeConfig) Sanitized() RuntimeConfig {
isSecret := func(name string) bool {
name = strings.ToLower(name)
return strings.Contains(name, "key") || strings.Contains(name, "token") || strings.Contains(name, "secret")
}
cleanRetryJoin := func(a []string) (b []string) {
for _, line := range a {
var fields []string
for _, f := range strings.Fields(line) {
if isSecret(f) {
kv := strings.SplitN(f, "=", 2)
fields = append(fields, kv[0]+"=hidden")
} else {
fields = append(fields, f)
}
}
b = append(b, strings.Join(fields, " "))
}
return b
}
// sanitize all fields with secrets
typ := reflect.TypeOf(RuntimeConfig{})
rawval := reflect.ValueOf(*c)
sanval := reflect.New(typ) // *RuntimeConfig
for i := 0; i < typ.NumField(); i++ {
f := typ.Field(i)
if f.Type.Kind() == reflect.String && isSecret(f.Name) {
sanval.Elem().Field(i).Set(reflect.ValueOf("hidden"))
} else {
sanval.Elem().Field(i).Set(rawval.Field(i))
}
}
san := sanval.Elem().Interface().(RuntimeConfig)
// sanitize retry-join config strings
san.RetryJoinLAN = cleanRetryJoin(san.RetryJoinLAN)
san.RetryJoinWAN = cleanRetryJoin(san.RetryJoinWAN)
return san
}

3560
agent/config/runtime_test.go Normal file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,17 @@
// +build !ent
package config
import (
"github.com/hashicorp/consul/agent/structs"
)
func (b *Builder) validateSegments(rt RuntimeConfig) error {
if rt.SegmentName != "" {
return structs.ErrSegmentsNotSupported
}
if len(rt.Segments) > 0 {
return structs.ErrSegmentsNotSupported
}
return nil
}

View File

@ -0,0 +1,45 @@
// +build !ent
package config
import (
"net"
"os"
"testing"
"github.com/hashicorp/consul/testutil"
)
func TestSegments(t *testing.T) {
dataDir := testutil.TempDir(t, "consul")
defer os.RemoveAll(dataDir)
tests := []configTest{
{
desc: "segment name not in OSS",
flags: []string{
`-data-dir=` + dataDir,
},
json: []string{`{ "server": true, "segment": "a" }`},
hcl: []string{` server = true segment = "a" `},
err: `Network segments are not supported in this version of Consul`,
privatev4: func() ([]*net.IPAddr, error) {
return []*net.IPAddr{ipAddr("10.0.0.1")}, nil
},
},
{
desc: "segments not in OSS",
flags: []string{
`-data-dir=` + dataDir,
},
json: []string{`{ "segments":[{ "name":"x", "advertise": "unix:///foo" }] }`},
hcl: []string{`segments = [{ name = "x" advertise = "unix:///foo" }]`},
err: `Network segments are not supported in this version of Consul`,
privatev4: func() ([]*net.IPAddr, error) {
return []*net.IPAddr{ipAddr("10.0.0.1")}, nil
},
},
}
testConfig(t, tests, dataDir)
}

File diff suppressed because it is too large Load Diff

View File

@ -23,8 +23,11 @@ func (c *Client) setupSerf(conf *serf.Config, ch chan serf.Event, path string) (
conf.Tags["vsn_min"] = fmt.Sprintf("%d", ProtocolVersionMin)
conf.Tags["vsn_max"] = fmt.Sprintf("%d", ProtocolVersionMax)
conf.Tags["build"] = c.config.Build
conf.MemberlistConfig.LogOutput = c.config.LogOutput
conf.LogOutput = c.config.LogOutput
if c.logger == nil {
conf.MemberlistConfig.LogOutput = c.config.LogOutput
conf.LogOutput = c.config.LogOutput
}
conf.MemberlistConfig.Logger = c.logger
conf.Logger = c.logger
conf.EventCh = ch
conf.ProtocolVersion = protocolVersionMap[c.config.ProtocolVersion]

View File

@ -438,7 +438,6 @@ func DefaultConfig() *Config {
// Use a transitional version of the raft protocol to interoperate with
// versions 1 and 3
conf.RaftConfig.ProtocolVersion = 2
conf.ScaleRaft(DefaultRaftMultiplier)
// Disable shutdown on removal
conf.RaftConfig.ShutdownOnRemove = false
@ -449,19 +448,6 @@ func DefaultConfig() *Config {
return conf
}
// ScaleRaft sets the config to have Raft timing parameters scaled by the given
// performance multiplier. This is done in an idempotent way so it's not tricky
// to call this when composing configurations and potentially calling this
// multiple times on the same structure.
func (c *Config) ScaleRaft(raftMultRaw uint) {
raftMult := time.Duration(raftMultRaw)
def := raft.DefaultConfig()
c.RaftConfig.HeartbeatTimeout = raftMult * def.HeartbeatTimeout
c.RaftConfig.ElectionTimeout = raftMult * def.ElectionTimeout
c.RaftConfig.LeaderLeaseTimeout = raftMult * def.LeaderLeaseTimeout
}
// tlsConfig maps this config into a tlsutil config.
func (c *Config) tlsConfig() *tlsutil.Config {
tlsConf := &tlsutil.Config{

View File

@ -67,8 +67,11 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string, w
if s.config.UseTLS {
conf.Tags["use_tls"] = "1"
}
conf.MemberlistConfig.LogOutput = s.config.LogOutput
conf.LogOutput = s.config.LogOutput
if s.logger == nil {
conf.MemberlistConfig.LogOutput = s.config.LogOutput
conf.LogOutput = s.config.LogOutput
}
conf.MemberlistConfig.Logger = s.logger
conf.Logger = s.logger
conf.EventCh = ch
conf.ProtocolVersion = protocolVersionMap[s.config.ProtocolVersion]

View File

@ -13,6 +13,7 @@ import (
"github.com/hashicorp/consul/agent/metadata"
"github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/test/porter"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/testutil"
"github.com/hashicorp/consul/testutil/retry"
@ -40,6 +41,10 @@ func testServerConfig(t *testing.T) (string, *Config) {
dir := testutil.TempDir(t, "consul")
config := DefaultConfig()
ports, err := porter.RandomPorts(3)
if err != nil {
t.Fatal("RandomPorts:", err)
}
config.NodeName = uniqueNodeName(t.Name())
config.Bootstrap = true
config.Datacenter = "dc1"
@ -48,7 +53,7 @@ func testServerConfig(t *testing.T) (string, *Config) {
// bind the rpc server to a random port. config.RPCAdvertise will be
// set to the listen address unless it was set in the configuration.
// In that case get the address from srv.Listener.Addr().
config.RPCAddr = &net.TCPAddr{IP: []byte{127, 0, 0, 1}}
config.RPCAddr = &net.TCPAddr{IP: []byte{127, 0, 0, 1}, Port: ports[0]}
nodeID, err := uuid.GenerateUUID()
if err != nil {
@ -60,14 +65,16 @@ func testServerConfig(t *testing.T) (string, *Config) {
// memberlist will update the value of BindPort after bind
// to the actual value.
config.SerfLANConfig.MemberlistConfig.BindAddr = "127.0.0.1"
config.SerfLANConfig.MemberlistConfig.BindPort = 0
config.SerfLANConfig.MemberlistConfig.BindPort = ports[1]
config.SerfLANConfig.MemberlistConfig.AdvertisePort = ports[1]
config.SerfLANConfig.MemberlistConfig.SuspicionMult = 2
config.SerfLANConfig.MemberlistConfig.ProbeTimeout = 50 * time.Millisecond
config.SerfLANConfig.MemberlistConfig.ProbeInterval = 100 * time.Millisecond
config.SerfLANConfig.MemberlistConfig.GossipInterval = 100 * time.Millisecond
config.SerfWANConfig.MemberlistConfig.BindAddr = "127.0.0.1"
config.SerfWANConfig.MemberlistConfig.BindPort = 0
config.SerfWANConfig.MemberlistConfig.BindPort = ports[2]
config.SerfWANConfig.MemberlistConfig.AdvertisePort = ports[2]
config.SerfWANConfig.MemberlistConfig.SuspicionMult = 2
config.SerfWANConfig.MemberlistConfig.ProbeTimeout = 50 * time.Millisecond
config.SerfWANConfig.MemberlistConfig.ProbeInterval = 100 * time.Millisecond

View File

@ -12,7 +12,7 @@ import (
func TestCoordinate_Datacenters(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/coordinate/datacenters", nil)
@ -33,7 +33,7 @@ func TestCoordinate_Datacenters(t *testing.T) {
func TestCoordinate_Nodes(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Make sure an empty list is non-nil.

View File

@ -12,6 +12,7 @@ import (
"regexp"
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/consul"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib"
@ -34,12 +35,26 @@ const (
var InvalidDnsRe = regexp.MustCompile(`[^A-Za-z0-9\\-]+`)
type dnsConfig struct {
AllowStale bool
Datacenter string
EnableTruncate bool
MaxStale time.Duration
NodeName string
NodeTTL time.Duration
OnlyPassing bool
RecursorTimeout time.Duration
SegmentName string
ServiceTTL map[string]time.Duration
UDPAnswerLimit int
}
// DNSServer is used to wrap an Agent and expose various
// service discovery endpoints using a DNS interface.
type DNSServer struct {
*dns.Server
agent *Agent
config *DNSConfig
config *dnsConfig
domain string
recursors []string
logger *log.Logger
@ -61,20 +76,37 @@ func NewDNSServer(a *Agent) (*DNSServer, error) {
}
// Make sure domain is FQDN, make it case insensitive for ServeMux
domain := dns.Fqdn(strings.ToLower(a.config.Domain))
domain := dns.Fqdn(strings.ToLower(a.config.DNSDomain))
dnscfg := GetDNSConfig(a.config)
srv := &DNSServer{
agent: a,
config: &a.config.DNSConfig,
config: dnscfg,
domain: domain,
logger: a.logger,
recursors: recursors,
}
srv.disableCompression.Store(a.config.DNSConfig.DisableCompression)
srv.disableCompression.Store(a.config.DNSDisableCompression)
return srv, nil
}
func GetDNSConfig(conf *config.RuntimeConfig) *dnsConfig {
return &dnsConfig{
AllowStale: conf.DNSAllowStale,
Datacenter: conf.Datacenter,
EnableTruncate: conf.DNSEnableTruncate,
MaxStale: conf.DNSMaxStale,
NodeName: conf.NodeName,
NodeTTL: conf.DNSNodeTTL,
OnlyPassing: conf.DNSOnlyPassing,
RecursorTimeout: conf.DNSRecursorTimeout,
SegmentName: conf.SegmentName,
ServiceTTL: conf.DNSServiceTTL,
UDPAnswerLimit: conf.DNSUDPAnswerLimit,
}
}
func (s *DNSServer) ListenAndServe(network, addr string, notif func()) error {
mux := dns.NewServeMux()
mux.HandleFunc("arpa.", s.handlePtr)
@ -150,7 +182,7 @@ func (d *DNSServer) handlePtr(resp dns.ResponseWriter, req *dns.Msg) {
Datacenter: datacenter,
QueryOptions: structs.QueryOptions{
Token: d.agent.tokens.UserToken(),
AllowStale: *d.config.AllowStale,
AllowStale: d.config.AllowStale,
},
}
var out structs.IndexedNodes
@ -309,7 +341,6 @@ func (d *DNSServer) nameservers(edns bool) (ns []dns.RR, extra []dns.RR) {
}
ns = append(ns, nsrr)
// A or AAAA glue record
glue := d.formatNodeRecord(addr, fqdn, dns.TypeANY, d.config.NodeTTL, edns)
extra = append(extra, glue...)
@ -466,7 +497,7 @@ func (d *DNSServer) nodeLookup(network, datacenter, node string, req, resp *dns.
Node: node,
QueryOptions: structs.QueryOptions{
Token: d.agent.tokens.UserToken(),
AllowStale: *d.config.AllowStale,
AllowStale: d.config.AllowStale,
},
}
var out structs.IndexedNodeServices
@ -621,7 +652,7 @@ func syncExtra(index map[string]dns.RR, resp *dns.Msg) {
// 1035. Enforce an arbitrary limit that can be further ratcheted down by
// config, and then make sure the response doesn't exceed 512 bytes. Any extra
// records will be trimmed along with answers.
func trimUDPResponse(config *DNSConfig, req, resp *dns.Msg) (trimmed bool) {
func trimUDPResponse(req, resp *dns.Msg, udpAnswerLimit int) (trimmed bool) {
numAnswers := len(resp.Answer)
hasExtra := len(resp.Extra) > 0
maxSize := defaultMaxUDPSize
@ -642,7 +673,7 @@ func trimUDPResponse(config *DNSConfig, req, resp *dns.Msg) (trimmed bool) {
}
// This cuts UDP responses to a useful but limited number of responses.
maxAnswers := lib.MinInt(maxUDPAnswerLimit, config.UDPAnswerLimit)
maxAnswers := lib.MinInt(maxUDPAnswerLimit, udpAnswerLimit)
if maxSize == defaultMaxUDPSize && numAnswers > maxAnswers {
resp.Answer = resp.Answer[:maxAnswers]
if hasExtra {
@ -678,7 +709,7 @@ func (d *DNSServer) lookupServiceNodes(datacenter, service, tag string) (structs
TagFilter: tag != "",
QueryOptions: structs.QueryOptions{
Token: d.agent.tokens.UserToken(),
AllowStale: *d.config.AllowStale,
AllowStale: d.config.AllowStale,
},
}
@ -703,7 +734,6 @@ func (d *DNSServer) lookupServiceNodes(datacenter, service, tag string) (structs
// Filter out any service nodes due to health checks
out.Nodes = out.Nodes.Filter(d.config.OnlyPassing)
return out, nil
}
@ -746,7 +776,7 @@ func (d *DNSServer) serviceLookup(network, datacenter, service, tag string, req,
// If the network is not TCP, restrict the number of responses
if network != "tcp" {
wasTrimmed := trimUDPResponse(d.config, req, resp)
wasTrimmed := trimUDPResponse(req, resp, d.config.UDPAnswerLimit)
// Flag that there are more records to return in the UDP response
if wasTrimmed && d.config.EnableTruncate {
@ -769,7 +799,7 @@ func (d *DNSServer) preparedQueryLookup(network, datacenter, query string, req,
QueryIDOrName: query,
QueryOptions: structs.QueryOptions{
Token: d.agent.tokens.UserToken(),
AllowStale: *d.config.AllowStale,
AllowStale: d.config.AllowStale,
},
// Always pass the local agent through. In the DNS interface, there
@ -778,7 +808,7 @@ func (d *DNSServer) preparedQueryLookup(network, datacenter, query string, req,
// relative to ourself on the server side.
Agent: structs.QuerySource{
Datacenter: d.agent.config.Datacenter,
Segment: d.agent.config.Segment,
Segment: d.agent.config.SegmentName,
Node: d.agent.config.NodeName,
},
}
@ -855,7 +885,7 @@ RPC:
// If the network is not TCP, restrict the number of responses.
if network != "tcp" {
wasTrimmed := trimUDPResponse(d.config, req, resp)
wasTrimmed := trimUDPResponse(req, resp, d.config.UDPAnswerLimit)
// Flag that there are more records to return in the UDP response
if wasTrimmed && d.config.EnableTruncate {

File diff suppressed because it is too large Load Diff

View File

@ -16,7 +16,7 @@ import (
func TestEventFire(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer([]byte("test"))
@ -55,9 +55,9 @@ func TestEventFire(t *testing.T) {
func TestEventFire_token(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLDefaultPolicy = "deny"
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_default_policy = "deny"
`)
defer a.Shutdown()
// Create an ACL token
@ -116,7 +116,7 @@ func TestEventFire_token(t *testing.T) {
func TestEventList(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
p := &UserEvent{Name: "test"}
@ -148,7 +148,7 @@ func TestEventList(t *testing.T) {
func TestEventList_Filter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
p := &UserEvent{Name: "test"}
@ -235,7 +235,7 @@ func TestEventList_ACLFilter(t *testing.T) {
func TestEventList_Blocking(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
p := &UserEvent{Name: "test"}
@ -286,7 +286,7 @@ func TestEventList_Blocking(t *testing.T) {
func TestEventList_EventBufOrder(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Fire some events in a non-sequential order

View File

@ -18,7 +18,7 @@ import (
func TestHealthChecksInState(t *testing.T) {
t.Parallel()
t.Run("warning", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/state/warning?dc=dc1", nil)
@ -41,7 +41,7 @@ func TestHealthChecksInState(t *testing.T) {
})
t.Run("passing", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/state/passing?dc=dc1", nil)
@ -66,7 +66,7 @@ func TestHealthChecksInState(t *testing.T) {
func TestHealthChecksInState_NodeMetaFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
args := &structs.RegisterRequest{
@ -106,7 +106,7 @@ func TestHealthChecksInState_NodeMetaFilter(t *testing.T) {
func TestHealthChecksInState_DistanceSort(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
args := &structs.RegisterRequest{
@ -180,7 +180,7 @@ func TestHealthChecksInState_DistanceSort(t *testing.T) {
func TestHealthNodeChecks(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/node/nope?dc=dc1", nil)
@ -214,7 +214,7 @@ func TestHealthNodeChecks(t *testing.T) {
func TestHealthServiceChecks(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/checks/consul?dc=dc1", nil)
@ -265,7 +265,7 @@ func TestHealthServiceChecks(t *testing.T) {
func TestHealthServiceChecks_NodeMetaFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/checks/consul?dc=dc1&node-meta=somekey:somevalue", nil)
@ -317,7 +317,7 @@ func TestHealthServiceChecks_NodeMetaFilter(t *testing.T) {
func TestHealthServiceChecks_DistanceSort(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create a service check
@ -396,7 +396,7 @@ func TestHealthServiceChecks_DistanceSort(t *testing.T) {
func TestHealthServiceNodes(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/service/consul?dc=dc1", nil)
@ -462,7 +462,7 @@ func TestHealthServiceNodes(t *testing.T) {
func TestHealthServiceNodes_NodeMetaFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/health/service/consul?dc=dc1&node-meta=somekey:somevalue", nil)
@ -514,7 +514,7 @@ func TestHealthServiceNodes_NodeMetaFilter(t *testing.T) {
func TestHealthServiceNodes_DistanceSort(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create a service check
@ -593,7 +593,7 @@ func TestHealthServiceNodes_DistanceSort(t *testing.T) {
func TestHealthServiceNodes_PassingFilter(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create a failing service check
@ -687,22 +687,22 @@ func TestHealthServiceNodes_PassingFilter(t *testing.T) {
func TestHealthServiceNodes_WanTranslation(t *testing.T) {
t.Parallel()
cfg1 := TestConfig()
cfg1.Datacenter = "dc1"
cfg1.TranslateWanAddrs = true
cfg1.ACLDatacenter = ""
a1 := NewTestAgent(t.Name(), cfg1)
a1 := NewTestAgent(t.Name(), `
datacenter = "dc1"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a1.Shutdown()
cfg2 := TestConfig()
cfg2.Datacenter = "dc2"
cfg2.TranslateWanAddrs = true
cfg2.ACLDatacenter = ""
a2 := NewTestAgent(t.Name(), cfg2)
a2 := NewTestAgent(t.Name(), `
datacenter = "dc2"
translate_wan_addrs = true
acl_datacenter = ""
`)
defer a2.Shutdown()
// Wait for the WAN join.
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.Ports.SerfWan)
addr := fmt.Sprintf("127.0.0.1:%d", a1.Config.SerfPortWAN)
if _, err := a2.JoinWAN([]string{addr}); err != nil {
t.Fatalf("err: %v", err)
}

View File

@ -3,6 +3,7 @@ package agent
import (
"encoding/json"
"fmt"
"net"
"net/http"
"net/http/pprof"
"net/url"
@ -25,14 +26,17 @@ type HTTPServer struct {
// proto is filled by the agent to "http" or "https".
proto string
addr net.Addr
}
func NewHTTPServer(addr string, a *Agent) *HTTPServer {
func NewHTTPServer(addr net.Addr, a *Agent) *HTTPServer {
s := &HTTPServer{
Server: &http.Server{Addr: addr},
Server: &http.Server{Addr: addr.String()},
agent: a,
blacklist: NewBlacklist(a.config.HTTPConfig.BlockEndpoints),
blacklist: NewBlacklist(a.config.HTTPBlockEndpoints),
addr: addr,
}
s.Server.Handler = s.handler(a.config.EnableDebug)
return s
}
@ -200,8 +204,8 @@ var (
// wrap is used to wrap functions to make them more convenient
func (s *HTTPServer) wrap(handler func(resp http.ResponseWriter, req *http.Request) (interface{}, error)) http.HandlerFunc {
return func(resp http.ResponseWriter, req *http.Request) {
setHeaders(resp, s.agent.config.HTTPConfig.ResponseHeaders)
setTranslateAddr(resp, s.agent.config.TranslateWanAddrs)
setHeaders(resp, s.agent.config.HTTPResponseHeaders)
setTranslateAddr(resp, s.agent.config.TranslateWANAddrs)
// Obfuscate any tokens from appearing in the logs
formVals, err := url.ParseQuery(req.URL.RawQuery)

View File

@ -33,14 +33,16 @@ func TestHTTPServer_UnixSocket(t *testing.T) {
defer os.RemoveAll(tempDir)
socket := filepath.Join(tempDir, "test.sock")
cfg := TestConfig()
cfg.Addresses.HTTP = "unix://" + socket
// Only testing mode, since uid/gid might not be settable
// from test environment.
cfg.UnixSockets = UnixSocketConfig{}
cfg.UnixSockets.Perms = "0777"
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
addresses {
http = "unix://`+socket+`"
}
unix_sockets {
mode = "0777"
}
`)
defer a.Shutdown()
// Ensure the socket was created
@ -58,10 +60,9 @@ func TestHTTPServer_UnixSocket(t *testing.T) {
}
// Ensure we can get a response from the socket.
path := socketPath(a.Config.Addresses.HTTP)
trans := cleanhttp.DefaultTransport()
trans.DialContext = func(_ context.Context, _, _ string) (net.Conn, error) {
return net.Dial("unix", path)
return net.Dial("unix", socket)
}
client := &http.Client{
Transport: trans,
@ -103,9 +104,11 @@ func TestHTTPServer_UnixSocket_FileExists(t *testing.T) {
t.Fatalf("not a regular file: %s", socket)
}
cfg := TestConfig()
cfg.Addresses.HTTP = "unix://" + socket
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
addresses {
http = "unix://`+socket+`"
}
`)
defer a.Shutdown()
// Ensure the file was replaced by the socket
@ -198,12 +201,11 @@ func TestSetMeta(t *testing.T) {
func TestHTTPAPI_BlockEndpoints(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.HTTPConfig.BlockEndpoints = []string{
"/v1/agent/self",
}
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
http_config {
block_endpoints = ["/v1/agent/self"]
}
`)
defer a.Shutdown()
handler := func(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
@ -235,7 +237,7 @@ func TestHTTPAPI_TranslateAddrHeader(t *testing.T) {
t.Parallel()
// Header should not be present if address translation is off.
{
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
resp := httptest.NewRecorder()
@ -254,9 +256,9 @@ func TestHTTPAPI_TranslateAddrHeader(t *testing.T) {
// Header should be set to true if it's turned on.
{
cfg := TestConfig()
cfg.TranslateWanAddrs = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
translate_wan_addrs = true
`)
defer a.Shutdown()
resp := httptest.NewRecorder()
@ -276,12 +278,14 @@ func TestHTTPAPI_TranslateAddrHeader(t *testing.T) {
func TestHTTPAPIResponseHeaders(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.HTTPConfig.ResponseHeaders = map[string]string{
"Access-Control-Allow-Origin": "*",
"X-XSS-Protection": "1; mode=block",
}
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
http_config {
response_headers = {
"Access-Control-Allow-Origin" = "*"
"X-XSS-Protection" = "1; mode=block"
}
}
`)
defer a.Shutdown()
resp := httptest.NewRecorder()
@ -305,7 +309,7 @@ func TestHTTPAPIResponseHeaders(t *testing.T) {
func TestContentTypeIsJSON(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
resp := httptest.NewRecorder()
@ -389,7 +393,7 @@ func TestPrettyPrintBare(t *testing.T) {
}
func testPrettyPrint(pretty string, t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
r := &structs.DirEntry{Key: "key"}
@ -417,7 +421,7 @@ func testPrettyPrint(pretty string, t *testing.T) {
func TestParseSource(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Default is agent's DC and no node (since the user didn't care, then
@ -565,7 +569,7 @@ func TestACLResolution(t *testing.T) {
reqBothTokens, _ := http.NewRequest("GET", "/v1/catalog/nodes?token=baz", nil)
reqBothTokens.Header.Add("X-Consul-Token", "zap")
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Check when no token is set
@ -603,9 +607,9 @@ func TestACLResolution(t *testing.T) {
func TestEnableWebUI(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.EnableUI = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
enable_ui = true
`)
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/ui/", nil)

View File

@ -33,10 +33,10 @@ func TestAgent_LoadKeyrings(t *testing.T) {
// Should be no configured keyring file by default
t.Run("no keys", func(t *testing.T) {
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
c1 := a1.Config.ConsulConfig
c1 := a1.consulConfig()
if c1.SerfLANConfig.KeyringFile != "" {
t.Fatalf("bad: %#v", c1.SerfLANConfig.KeyringFile)
}
@ -57,7 +57,7 @@ func TestAgent_LoadKeyrings(t *testing.T) {
a2.Start()
defer a2.Shutdown()
c2 := a2.Config.ConsulConfig
c2 := a2.consulConfig()
if c2.SerfLANConfig.KeyringFile == "" {
t.Fatalf("should have keyring file")
}
@ -80,13 +80,14 @@ func TestAgent_LoadKeyrings(t *testing.T) {
// Client should auto-load only the LAN keyring file
t.Run("client with keys", func(t *testing.T) {
cfg3 := TestConfig()
cfg3.Server = false
a3 := &TestAgent{Name: t.Name(), Config: cfg3, Key: key}
a3 := &TestAgent{Name: t.Name(), HCL: `
server = false
bootstrap = false
`, Key: key}
a3.Start()
defer a3.Shutdown()
c3 := a3.Config.ConsulConfig
c3 := a3.consulConfig()
if c3.SerfLANConfig.KeyringFile == "" {
t.Fatalf("should have keyring file")
}
@ -111,10 +112,10 @@ func TestAgent_InmemKeyrings(t *testing.T) {
// Should be no configured keyring file by default
t.Run("no keys", func(t *testing.T) {
a1 := NewTestAgent(t.Name(), nil)
a1 := NewTestAgent(t.Name(), "")
defer a1.Shutdown()
c1 := a1.Config.ConsulConfig
c1 := a1.consulConfig()
if c1.SerfLANConfig.KeyringFile != "" {
t.Fatalf("bad: %#v", c1.SerfLANConfig.KeyringFile)
}
@ -131,15 +132,14 @@ func TestAgent_InmemKeyrings(t *testing.T) {
// Server should auto-load LAN and WAN keyring
t.Run("server with keys", func(t *testing.T) {
cfg2 := TestConfig()
cfg2.EncryptKey = key
cfg2.DisableKeyringFile = true
a2 := &TestAgent{Name: t.Name(), Config: cfg2}
a2 := &TestAgent{Name: t.Name(), HCL: `
encrypt = "` + key + `"
disable_keyring_file = true
`}
a2.Start()
defer a2.Shutdown()
c2 := a2.Config.ConsulConfig
c2 := a2.consulConfig()
if c2.SerfLANConfig.KeyringFile != "" {
t.Fatalf("should not have keyring file")
}
@ -162,15 +162,16 @@ func TestAgent_InmemKeyrings(t *testing.T) {
// Client should auto-load only the LAN keyring
t.Run("client with keys", func(t *testing.T) {
cfg3 := TestConfig()
cfg3.EncryptKey = key
cfg3.DisableKeyringFile = true
cfg3.Server = false
a3 := &TestAgent{Name: t.Name(), Config: cfg3}
a3 := &TestAgent{Name: t.Name(), HCL: `
encrypt = "` + key + `"
server = false
bootstrap = false
disable_keyring_file = true
`}
a3.Start()
defer a3.Shutdown()
c3 := a3.Config.ConsulConfig
c3 := a3.consulConfig()
if c3.SerfLANConfig.KeyringFile != "" {
t.Fatalf("should not have keyring file")
}
@ -201,16 +202,15 @@ func TestAgent_InmemKeyrings(t *testing.T) {
t.Fatalf("err: %v", err)
}
cfg4 := TestConfig()
cfg4.EncryptKey = key
cfg4.DisableKeyringFile = true
cfg4.DataDir = dir
a4 := &TestAgent{Name: t.Name(), Config: cfg4}
a4 := &TestAgent{Name: t.Name(), HCL: `
encrypt = "` + key + `"
disable_keyring_file = true
data_dir = "` + dir + `"
`}
a4.Start()
defer a4.Shutdown()
c4 := a4.Config.ConsulConfig
c4 := a4.consulConfig()
if c4.SerfLANConfig.KeyringFile != "" {
t.Fatalf("should not have keyring file")
}
@ -276,11 +276,11 @@ func TestAgentKeyring_ACL(t *testing.T) {
key1 := "tbLJg26ZJyJ9pK3qhc9jig=="
key2 := "4leC33rgtXKIVUr9Nr0snQ=="
cfg := TestACLConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLDefaultPolicy = "deny"
a := &TestAgent{Name: t.Name(), Config: cfg, Key: key1}
a := &TestAgent{Name: t.Name(), HCL: TestACLConfig() + `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_default_policy = "deny"
`, Key: key1}
a.Start()
defer a.Shutdown()

View File

@ -13,7 +13,7 @@ import (
func TestKVSEndpoint_PUT_GET_DELETE(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
keys := []string{
@ -72,7 +72,7 @@ func TestKVSEndpoint_PUT_GET_DELETE(t *testing.T) {
func TestKVSEndpoint_Recurse(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
keys := []string{
@ -148,7 +148,7 @@ func TestKVSEndpoint_Recurse(t *testing.T) {
func TestKVSEndpoint_DELETE_CAS(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
{
@ -214,7 +214,7 @@ func TestKVSEndpoint_DELETE_CAS(t *testing.T) {
func TestKVSEndpoint_CAS(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
{
@ -290,7 +290,7 @@ func TestKVSEndpoint_CAS(t *testing.T) {
func TestKVSEndpoint_ListKeys(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
keys := []string{
@ -339,7 +339,7 @@ func TestKVSEndpoint_ListKeys(t *testing.T) {
func TestKVSEndpoint_AcquireRelease(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Acquire the lock
@ -396,7 +396,7 @@ func TestKVSEndpoint_AcquireRelease(t *testing.T) {
func TestKVSEndpoint_GET_Raw(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
buf := bytes.NewBuffer([]byte("test"))
@ -426,7 +426,7 @@ func TestKVSEndpoint_GET_Raw(t *testing.T) {
func TestKVSEndpoint_PUT_ConflictingFlags(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("PUT", "/v1/kv/test?cas=0&acquire=xxx", nil)
@ -445,7 +445,7 @@ func TestKVSEndpoint_PUT_ConflictingFlags(t *testing.T) {
func TestKVSEndpoint_DELETE_ConflictingFlags(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("DELETE", "/v1/kv/test?recurse&cas=0", nil)

View File

@ -10,6 +10,7 @@ import (
"time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/api"
@ -90,10 +91,10 @@ type localState struct {
}
// NewLocalState creates a is used to initialize the local state
func NewLocalState(c *Config, lg *log.Logger, tokens *token.Store) *localState {
func NewLocalState(c *config.RuntimeConfig, lg *log.Logger, tokens *token.Store) *localState {
lc := localStateConfig{
AEInterval: c.AEInterval,
AdvertiseAddr: c.AdvertiseAddr,
AdvertiseAddr: c.AdvertiseAddrLAN.String(),
CheckUpdateInterval: c.CheckUpdateInterval,
Datacenter: c.Datacenter,
NodeID: c.NodeID,

View File

@ -5,11 +5,13 @@ import (
"testing"
"time"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testutil/retry"
"github.com/hashicorp/consul/types"
"github.com/pascaldekloe/goe/verify"
)
func TestAgentAntiEntropy_Services(t *testing.T) {
@ -128,11 +130,9 @@ func TestAgentAntiEntropy_Services(t *testing.T) {
addrs := services.NodeServices.Node.TaggedAddresses
meta := services.NodeServices.Node.Meta
delete(meta, structs.MetaSegmentKey) // Added later, not in config.
if id != a.Config.NodeID ||
!reflect.DeepEqual(addrs, a.Config.TaggedAddresses) ||
!reflect.DeepEqual(meta, a.Config.Meta) {
r.Fatalf("bad: %v", services.NodeServices.Node)
}
verify.Values(r, "node id", id, a.config.NodeID)
verify.Values(r, "tagged addrs", addrs, a.config.TaggedAddresses)
verify.Values(r, "node meta", meta, a.config.NodeMeta)
// We should have 6 services (consul included)
if len(services.NodeServices.Services) != 6 {
@ -356,7 +356,7 @@ func TestAgentAntiEntropy_EnableTagOverride(t *testing.T) {
func TestAgentAntiEntropy_Services_WithChecks(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
{
@ -484,27 +484,27 @@ func TestAgentAntiEntropy_Services_WithChecks(t *testing.T) {
}
var testRegisterRules = `
node "" {
policy = "write"
}
node "" {
policy = "write"
}
service "api" {
policy = "write"
}
service "api" {
policy = "write"
}
service "consul" {
policy = "write"
}
`
service "consul" {
policy = "write"
}
`
func TestAgentAntiEntropy_Services_ACLDeny(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLDefaultPolicy = "deny"
cfg.ACLEnforceVersion8 = Bool(true)
a := &TestAgent{Name: t.Name(), Config: cfg, NoInitialSync: true}
a := &TestAgent{Name: t.Name(), HCL: `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_default_policy = "deny"
acl_enforce_version_8 = true
`, NoInitialSync: true}
a.Start()
defer a.Shutdown()
@ -830,11 +830,9 @@ func TestAgentAntiEntropy_Checks(t *testing.T) {
addrs := services.NodeServices.Node.TaggedAddresses
meta := services.NodeServices.Node.Meta
delete(meta, structs.MetaSegmentKey) // Added later, not in config.
if id != a.Config.NodeID ||
!reflect.DeepEqual(addrs, a.Config.TaggedAddresses) ||
!reflect.DeepEqual(meta, a.Config.Meta) {
t.Fatalf("bad: %v", services.NodeServices.Node)
}
verify.Values(t, "node id", id, a.config.NodeID)
verify.Values(t, "tagged addrs", addrs, a.config.TaggedAddresses)
verify.Values(t, "node meta", meta, a.config.NodeMeta)
}
// Remove one of the checks
@ -900,12 +898,12 @@ func TestAgentAntiEntropy_Checks(t *testing.T) {
func TestAgentAntiEntropy_Checks_ACLDeny(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLDefaultPolicy = "deny"
cfg.ACLEnforceVersion8 = Bool(true)
a := &TestAgent{Name: t.Name(), Config: cfg, NoInitialSync: true}
a := &TestAgent{Name: t.Name(), HCL: `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_default_policy = "deny"
acl_enforce_version_8 = true
`, NoInitialSync: true}
a.Start()
defer a.Shutdown()
@ -1155,9 +1153,9 @@ func TestAgentAntiEntropy_Checks_ACLDeny(t *testing.T) {
func TestAgentAntiEntropy_Check_DeferSync(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.CheckUpdateInterval = 500 * time.Millisecond
a := &TestAgent{Name: t.Name(), Config: cfg, NoInitialSync: true}
a := &TestAgent{Name: t.Name(), HCL: `
check_update_interval = "500ms"
`, NoInitialSync: true}
a.Start()
defer a.Shutdown()
@ -1230,7 +1228,7 @@ func TestAgentAntiEntropy_Check_DeferSync(t *testing.T) {
reg := structs.RegisterRequest{
Datacenter: a.Config.Datacenter,
Node: a.Config.NodeName,
Address: a.Config.AdvertiseAddr,
Address: a.Config.AdvertiseAddrLAN.IP.String(),
TaggedAddresses: a.Config.TaggedAddresses,
Check: eCopy,
WriteRequest: structs.WriteRequest{},
@ -1329,10 +1327,16 @@ func TestAgentAntiEntropy_Check_DeferSync(t *testing.T) {
func TestAgentAntiEntropy_NodeInfo(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.NodeID = types.NodeID("40e4a748-2192-161a-0510-9bf59fe950b5")
cfg.Meta["somekey"] = "somevalue"
a := &TestAgent{Name: t.Name(), Config: cfg, NoInitialSync: true}
nodeID := types.NodeID("40e4a748-2192-161a-0510-9bf59fe950b5")
nodeMeta := map[string]string{
"somekey": "somevalue",
}
a := &TestAgent{Name: t.Name(), HCL: `
node_id = "40e4a748-2192-161a-0510-9bf59fe950b5"
node_meta {
somekey = "somevalue"
}
`, NoInitialSync: true}
a.Start()
defer a.Shutdown()
@ -1367,9 +1371,9 @@ func TestAgentAntiEntropy_NodeInfo(t *testing.T) {
addrs := services.NodeServices.Node.TaggedAddresses
meta := services.NodeServices.Node.Meta
delete(meta, structs.MetaSegmentKey) // Added later, not in config.
if id != cfg.NodeID ||
!reflect.DeepEqual(addrs, cfg.TaggedAddresses) ||
!reflect.DeepEqual(meta, cfg.Meta) {
if id != nodeID ||
!reflect.DeepEqual(addrs, a.config.TaggedAddresses) ||
!reflect.DeepEqual(meta, nodeMeta) {
r.Fatalf("bad: %v", services.NodeServices.Node)
}
})
@ -1391,9 +1395,9 @@ func TestAgentAntiEntropy_NodeInfo(t *testing.T) {
addrs := services.NodeServices.Node.TaggedAddresses
meta := services.NodeServices.Node.Meta
delete(meta, structs.MetaSegmentKey) // Added later, not in config.
if id != cfg.NodeID ||
!reflect.DeepEqual(addrs, cfg.TaggedAddresses) ||
!reflect.DeepEqual(meta, cfg.Meta) {
if id != nodeID ||
!reflect.DeepEqual(addrs, a.config.TaggedAddresses) ||
!reflect.DeepEqual(meta, nodeMeta) {
r.Fatalf("bad: %v", services.NodeServices.Node)
}
})
@ -1428,7 +1432,7 @@ func TestAgent_serviceTokens(t *testing.T) {
tokens := new(token.Store)
tokens.UpdateUserToken("default")
l := NewLocalState(TestConfig(), nil, tokens)
l := NewLocalState(config.DefaultRuntimeConfig(`bind_addr = "127.0.0.1" data_dir = "dummy"`), nil, tokens)
l.AddService(&structs.NodeService{
ID: "redis",
@ -1457,7 +1461,7 @@ func TestAgent_checkTokens(t *testing.T) {
tokens := new(token.Store)
tokens.UpdateUserToken("default")
l := NewLocalState(TestConfig(), nil, tokens)
l := NewLocalState(config.DefaultRuntimeConfig(`bind_addr = "127.0.0.1" data_dir = "dummy"`), nil, tokens)
// Returns default when no token is set
if token := l.CheckToken("mem"); token != "default" {
@ -1479,8 +1483,7 @@ func TestAgent_checkTokens(t *testing.T) {
func TestAgent_checkCriticalTime(t *testing.T) {
t.Parallel()
cfg := TestConfig()
l := NewLocalState(cfg, nil, new(token.Store))
l := NewLocalState(config.DefaultRuntimeConfig(`bind_addr = "127.0.0.1" data_dir = "dummy"`), nil, new(token.Store))
svc := &structs.NodeService{ID: "redis", Service: "redis", Port: 8000}
l.AddService(svc, "")
@ -1542,8 +1545,7 @@ func TestAgent_checkCriticalTime(t *testing.T) {
func TestAgent_AddCheckFailure(t *testing.T) {
t.Parallel()
cfg := TestConfig()
l := NewLocalState(cfg, nil, new(token.Store))
l := NewLocalState(config.DefaultRuntimeConfig(`bind_addr = "127.0.0.1" data_dir = "dummy"`), nil, new(token.Store))
// Add a check for a service that does not exist and verify that it fails
checkID := types.CheckID("redis:1")
@ -1595,15 +1597,22 @@ func TestAgent_nestedPauseResume(t *testing.T) {
func TestAgent_sendCoordinate(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.SyncCoordinateRateTarget = 10.0 // updates/sec
cfg.SyncCoordinateIntervalMin = 1 * time.Millisecond
cfg.ConsulConfig.CoordinateUpdatePeriod = 100 * time.Millisecond
cfg.ConsulConfig.CoordinateUpdateBatchSize = 10
cfg.ConsulConfig.CoordinateUpdateMaxBatches = 1
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
sync_coordinate_interval_min = "1ms"
sync_coordinate_rate_target = 10.0
consul = {
coordinate = {
update_period = "100ms"
update_batch_size = 10
update_max_batches = 1
}
}
`)
defer a.Shutdown()
t.Logf("%d %d %s", a.consulConfig().CoordinateUpdateBatchSize, a.consulConfig().CoordinateUpdateMaxBatches,
a.consulConfig().CoordinateUpdatePeriod.String())
// Make sure the coordinate is present.
req := structs.DCSpecificRequest{
Datacenter: a.Config.Datacenter,

View File

@ -7,7 +7,6 @@ import (
"net/http/httptest"
"strings"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
@ -16,7 +15,7 @@ import (
func TestOperator_RaftConfiguration(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -43,7 +42,7 @@ func TestOperator_RaftConfiguration(t *testing.T) {
func TestOperator_RaftPeer(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -59,7 +58,7 @@ func TestOperator_RaftPeer(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -79,9 +78,9 @@ func TestOperator_KeyringInstall(t *testing.T) {
t.Parallel()
oldKey := "H3/9gBxcKKRf45CaI2DlRg=="
newKey := "z90lFx3sZZLtTOkutXcwYg=="
cfg := TestConfig()
cfg.EncryptKey = oldKey
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
encrypt = "`+oldKey+`"
`)
defer a.Shutdown()
body := bytes.NewBufferString(fmt.Sprintf("{\"Key\":\"%s\"}", newKey))
@ -114,9 +113,9 @@ func TestOperator_KeyringInstall(t *testing.T) {
func TestOperator_KeyringList(t *testing.T) {
t.Parallel()
key := "H3/9gBxcKKRf45CaI2DlRg=="
cfg := TestConfig()
cfg.EncryptKey = key
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
encrypt = "`+key+`"
`)
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/operator/keyring", nil)
@ -163,9 +162,9 @@ func TestOperator_KeyringRemove(t *testing.T) {
t.Parallel()
key := "H3/9gBxcKKRf45CaI2DlRg=="
tempKey := "z90lFx3sZZLtTOkutXcwYg=="
cfg := TestConfig()
cfg.EncryptKey = key
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
encrypt = "`+key+`"
`)
defer a.Shutdown()
_, err := a.InstallKey(tempKey, "", 0)
@ -221,9 +220,9 @@ func TestOperator_KeyringUse(t *testing.T) {
t.Parallel()
oldKey := "H3/9gBxcKKRf45CaI2DlRg=="
newKey := "z90lFx3sZZLtTOkutXcwYg=="
cfg := TestConfig()
cfg.EncryptKey = oldKey
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
encrypt = "`+oldKey+`"
`)
defer a.Shutdown()
if _, err := a.InstallKey(newKey, "", 0); err != nil {
@ -264,9 +263,9 @@ func TestOperator_KeyringUse(t *testing.T) {
func TestOperator_Keyring_InvalidRelayFactor(t *testing.T) {
t.Parallel()
key := "H3/9gBxcKKRf45CaI2DlRg=="
cfg := TestConfig()
cfg.EncryptKey = key
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
encrypt = "`+key+`"
`)
defer a.Shutdown()
cases := map[string]string{
@ -289,7 +288,7 @@ func TestOperator_Keyring_InvalidRelayFactor(t *testing.T) {
func TestOperator_AutopilotGetConfiguration(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -313,7 +312,7 @@ func TestOperator_AutopilotGetConfiguration(t *testing.T) {
func TestOperator_AutopilotSetConfiguration(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer([]byte(`{"CleanupDeadServers": false}`))
@ -341,7 +340,7 @@ func TestOperator_AutopilotSetConfiguration(t *testing.T) {
func TestOperator_AutopilotCASConfiguration(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer([]byte(`{"CleanupDeadServers": false}`))
@ -408,9 +407,9 @@ func TestOperator_AutopilotCASConfiguration(t *testing.T) {
func TestOperator_ServerHealth(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.RaftProtocol = 3
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
raft_protocol = 3
`)
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -440,11 +439,12 @@ func TestOperator_ServerHealth(t *testing.T) {
func TestOperator_ServerHealth_Unhealthy(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.RaftProtocol = 3
threshold := time.Duration(-1)
cfg.Autopilot.LastContactThreshold = &threshold
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
raft_protocol = 3
autopilot {
last_contact_threshold = "-1s"
}
`)
defer a.Shutdown()
body := bytes.NewBuffer(nil)

View File

@ -96,7 +96,7 @@ func (s *HTTPServer) preparedQueryExecute(id string, resp http.ResponseWriter, r
Agent: structs.QuerySource{
Node: s.agent.config.NodeName,
Datacenter: s.agent.config.Datacenter,
Segment: s.agent.config.Segment,
Segment: s.agent.config.SegmentName,
},
}
s.parseSource(req, &args.Source)
@ -141,7 +141,7 @@ func (s *HTTPServer) preparedQueryExplain(id string, resp http.ResponseWriter, r
Agent: structs.QuerySource{
Node: s.agent.config.NodeName,
Datacenter: s.agent.config.Datacenter,
Segment: s.agent.config.Segment,
Segment: s.agent.config.SegmentName,
},
}
s.parseSource(req, &args.Source)

View File

@ -70,7 +70,7 @@ func (m *MockPreparedQuery) Explain(args *structs.PreparedQueryExecuteRequest,
func TestPreparedQuery_Create(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -155,7 +155,7 @@ func TestPreparedQuery_Create(t *testing.T) {
func TestPreparedQuery_List(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -188,7 +188,7 @@ func TestPreparedQuery_List(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -238,7 +238,7 @@ func TestPreparedQuery_List(t *testing.T) {
func TestPreparedQuery_Execute(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -271,7 +271,7 @@ func TestPreparedQuery_Execute(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -327,7 +327,7 @@ func TestPreparedQuery_Execute(t *testing.T) {
// Ensure the proper params are set when no special args are passed
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -358,10 +358,10 @@ func TestPreparedQuery_Execute(t *testing.T) {
// Ensure WAN translation occurs for a response outside of the local DC.
t.Run("", func(t *testing.T) {
cfg := TestConfig()
cfg.Datacenter = "dc1"
cfg.TranslateWanAddrs = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
datacenter = "dc1"
translate_wan_addrs = true
`)
defer a.Shutdown()
m := MockPreparedQuery{
@ -408,10 +408,10 @@ func TestPreparedQuery_Execute(t *testing.T) {
// Ensure WAN translation doesn't occur for the local DC.
t.Run("", func(t *testing.T) {
cfg := TestConfig()
cfg.Datacenter = "dc1"
cfg.TranslateWanAddrs = true
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
datacenter = "dc1"
translate_wan_addrs = true
`)
defer a.Shutdown()
m := MockPreparedQuery{
@ -457,7 +457,7 @@ func TestPreparedQuery_Execute(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -475,7 +475,7 @@ func TestPreparedQuery_Execute(t *testing.T) {
func TestPreparedQuery_Explain(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -530,7 +530,7 @@ func TestPreparedQuery_Explain(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -548,7 +548,7 @@ func TestPreparedQuery_Explain(t *testing.T) {
func TestPreparedQuery_Get(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -596,7 +596,7 @@ func TestPreparedQuery_Get(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -613,7 +613,7 @@ func TestPreparedQuery_Get(t *testing.T) {
func TestPreparedQuery_Update(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -691,7 +691,7 @@ func TestPreparedQuery_Update(t *testing.T) {
func TestPreparedQuery_Delete(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
m := MockPreparedQuery{
@ -740,7 +740,7 @@ func TestPreparedQuery_Delete(t *testing.T) {
func TestPreparedQuery_BadMethods(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -755,7 +755,7 @@ func TestPreparedQuery_BadMethods(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -801,7 +801,7 @@ func TestPreparedQuery_parseLimit(t *testing.T) {
// correctly when calling through to the real endpoints.
func TestPreparedQuery_Integration(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Register a node and a service.

View File

@ -6,6 +6,7 @@ import (
"fmt"
"reflect"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
@ -95,40 +96,40 @@ func TestRexecWriter(t *testing.T) {
func TestRemoteExecGetSpec(t *testing.T) {
t.Parallel()
testRemoteExecGetSpec(t, nil, "", true)
testRemoteExecGetSpec(t, "", "", true)
}
func TestRemoteExecGetSpec_ACLToken(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLToken = "root"
cfg.ACLDefaultPolicy = "deny"
testRemoteExecGetSpec(t, cfg, "root", true)
testRemoteExecGetSpec(t, `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_token = "root"
acl_default_policy = "deny"
`, "root", true)
}
func TestRemoteExecGetSpec_ACLAgentToken(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLAgentToken = "root"
cfg.ACLDefaultPolicy = "deny"
testRemoteExecGetSpec(t, cfg, "root", true)
testRemoteExecGetSpec(t, `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_agent_token = "root"
acl_default_policy = "deny"
`, "root", true)
}
func TestRemoteExecGetSpec_ACLDeny(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLDefaultPolicy = "deny"
testRemoteExecGetSpec(t, cfg, "root", false)
testRemoteExecGetSpec(t, `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_default_policy = "deny"
`, "root", false)
}
func testRemoteExecGetSpec(t *testing.T, c *Config, token string, shouldSucceed bool) {
a := NewTestAgent(t.Name(), c)
func testRemoteExecGetSpec(t *testing.T, hcl string, token string, shouldSucceed bool) {
a := NewTestAgent(t.Name(), hcl)
defer a.Shutdown()
event := &remoteExecEvent{
@ -160,40 +161,40 @@ func testRemoteExecGetSpec(t *testing.T, c *Config, token string, shouldSucceed
func TestRemoteExecWrites(t *testing.T) {
t.Parallel()
testRemoteExecWrites(t, nil, "", true)
testRemoteExecWrites(t, "", "", true)
}
func TestRemoteExecWrites_ACLToken(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLToken = "root"
cfg.ACLDefaultPolicy = "deny"
testRemoteExecWrites(t, cfg, "root", true)
testRemoteExecWrites(t, `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_token = "root"
acl_default_policy = "deny"
`, "root", true)
}
func TestRemoteExecWrites_ACLAgentToken(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLAgentToken = "root"
cfg.ACLDefaultPolicy = "deny"
testRemoteExecWrites(t, cfg, "root", true)
testRemoteExecWrites(t, `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_agent_token = "root"
acl_default_policy = "deny"
`, "root", true)
}
func TestRemoteExecWrites_ACLDeny(t *testing.T) {
t.Parallel()
cfg := TestConfig()
cfg.ACLDatacenter = "dc1"
cfg.ACLMasterToken = "root"
cfg.ACLDefaultPolicy = "deny"
testRemoteExecWrites(t, cfg, "root", false)
testRemoteExecWrites(t, `
acl_datacenter = "dc1"
acl_master_token = "root"
acl_default_policy = "deny"
`, "root", false)
}
func testRemoteExecWrites(t *testing.T, c *Config, token string, shouldSucceed bool) {
a := NewTestAgent(t.Name(), c)
func testRemoteExecWrites(t *testing.T, hcl string, token string, shouldSucceed bool) {
a := NewTestAgent(t.Name(), hcl)
defer a.Shutdown()
event := &remoteExecEvent{
@ -250,7 +251,7 @@ func testRemoteExecWrites(t *testing.T, c *Config, token string, shouldSucceed b
}
func testHandleRemoteExec(t *testing.T, command string, expectedSubstring string, expectedReturnCode string) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
event := &remoteExecEvent{

View File

@ -12,9 +12,9 @@ import (
func (a *Agent) retryJoinLAN() {
r := &retryJoiner{
cluster: "LAN",
addrs: a.config.RetryJoin,
maxAttempts: a.config.RetryMaxAttempts,
interval: a.config.RetryInterval,
addrs: a.config.RetryJoinLAN,
maxAttempts: a.config.RetryJoinMaxAttemptsLAN,
interval: a.config.RetryJoinIntervalLAN,
join: a.JoinLAN,
logger: a.logger,
}
@ -26,9 +26,9 @@ func (a *Agent) retryJoinLAN() {
func (a *Agent) retryJoinWAN() {
r := &retryJoiner{
cluster: "WAN",
addrs: a.config.RetryJoinWan,
maxAttempts: a.config.RetryMaxAttemptsWan,
interval: a.config.RetryIntervalWan,
addrs: a.config.RetryJoinWAN,
maxAttempts: a.config.RetryJoinMaxAttemptsWAN,
interval: a.config.RetryJoinIntervalWAN,
join: a.JoinWAN,
logger: a.logger,
}

View File

@ -2,18 +2,14 @@
package agent
import (
"github.com/hashicorp/consul/agent/structs"
)
func ValidateSegments(conf *Config) error {
if conf.Segment != "" {
return structs.ErrSegmentsNotSupported
}
if len(conf.Segments) > 0 {
return structs.ErrSegmentsNotSupported
}
return nil
}
// done(fs): func ValidateSegments(conf *Config) error {
// done(fs): if conf.Segment != "" {
// done(fs): return structs.ErrSegmentsNotSupported
// done(fs): }
// done(fs):
// done(fs): if len(conf.Segments) > 0 {
// done(fs): return structs.ErrSegmentsNotSupported
// done(fs): }
// done(fs):
// done(fs): return nil
// done(fs): }

View File

@ -15,7 +15,7 @@ import (
func TestSessionCreate(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create a health check
@ -61,7 +61,7 @@ func TestSessionCreate(t *testing.T) {
func TestSessionCreateDelete(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Create a health check
@ -190,7 +190,7 @@ func makeTestSessionTTL(t *testing.T, srv *HTTPServer, ttl string) string {
func TestSessionDestroy(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
id := makeTestSession(t, a.srv)
@ -209,10 +209,9 @@ func TestSessionDestroy(t *testing.T) {
func TestSessionCustomTTL(t *testing.T) {
t.Parallel()
ttl := 250 * time.Millisecond
cfg := TestConfig()
cfg.SessionTTLMin = ttl
cfg.SessionTTLMinRaw = ttl.String()
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
session_ttl_min = "250ms"
`)
defer a.Shutdown()
id := makeTestSessionTTL(t, a.srv, ttl.String())
@ -251,10 +250,9 @@ func TestSessionCustomTTL(t *testing.T) {
func TestSessionTTLRenew(t *testing.T) {
// t.Parallel() // timing test. no parallel
ttl := 250 * time.Millisecond
cfg := TestConfig()
cfg.SessionTTLMin = ttl
cfg.SessionTTLMinRaw = ttl.String()
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
session_ttl_min = "250ms"
`)
defer a.Shutdown()
id := makeTestSessionTTL(t, a.srv, ttl.String())
@ -331,7 +329,7 @@ func TestSessionTTLRenew(t *testing.T) {
func TestSessionGet(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/session/info/adf4238a-882b-9ddc-4a9d-5b6758e4159e", nil)
@ -350,7 +348,7 @@ func TestSessionGet(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
id := makeTestSession(t, a.srv)
@ -374,7 +372,7 @@ func TestSessionGet(t *testing.T) {
func TestSessionList(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/session/list", nil)
@ -393,7 +391,7 @@ func TestSessionList(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
var ids []string
@ -420,7 +418,7 @@ func TestSessionList(t *testing.T) {
func TestSessionsForNode(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", "/v1/session/node/"+a.Config.NodeName, nil)
@ -439,7 +437,7 @@ func TestSessionsForNode(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
var ids []string
@ -465,7 +463,7 @@ func TestSessionsForNode(t *testing.T) {
func TestSessionDeleteDestroy(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
id := makeTestSessionDelete(t, a.srv)

View File

@ -15,7 +15,7 @@ func TestSnapshot(t *testing.T) {
t.Parallel()
var snap io.Reader
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -41,7 +41,7 @@ func TestSnapshot(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("PUT", "/v1/snapshot?token=root", snap)
@ -105,7 +105,7 @@ func TestSnapshot_Options(t *testing.T) {
func TestSnapshot_BadMethods(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)
@ -121,7 +121,7 @@ func TestSnapshot_BadMethods(t *testing.T) {
})
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
body := bytes.NewBuffer(nil)

View File

@ -1,12 +1,10 @@
package agent
import (
"testing"
)
import "testing"
func TestStatusLeader(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
obj, err := a.srv.StatusLeader(nil, nil)
@ -21,7 +19,7 @@ func TestStatusLeader(t *testing.T) {
func TestStatusPeers(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
obj, err := a.srv.StatusPeers(nil, nil)

View File

@ -3,10 +3,12 @@ package structs
import (
"reflect"
"testing"
"time"
"github.com/google/gofuzz"
"github.com/hashicorp/consul/api"
"github.com/mitchellh/reflectwalk"
"github.com/pascaldekloe/goe/verify"
)
func TestCheckDefinition_Defaults(t *testing.T) {
@ -70,3 +72,44 @@ func TestCheckDefinition_CheckType(t *testing.T) {
}
}
}
func TestCheckDefinitionToCheckType(t *testing.T) {
t.Parallel()
got := &CheckDefinition{
ID: "id",
Name: "name",
Status: "green",
Notes: "notes",
ServiceID: "svcid",
Token: "tok",
Script: "/bin/foo",
HTTP: "someurl",
TCP: "host:port",
Interval: 1 * time.Second,
DockerContainerID: "abc123",
Shell: "/bin/ksh",
TLSSkipVerify: true,
Timeout: 2 * time.Second,
TTL: 3 * time.Second,
DeregisterCriticalServiceAfter: 4 * time.Second,
}
want := &CheckType{
CheckID: "id",
Name: "name",
Status: "green",
Notes: "notes",
Script: "/bin/foo",
HTTP: "someurl",
TCP: "host:port",
Interval: 1 * time.Second,
DockerContainerID: "abc123",
Shell: "/bin/ksh",
TLSSkipVerify: true,
Timeout: 2 * time.Second,
TTL: 3 * time.Second,
DeregisterCriticalServiceAfter: 4 * time.Second,
}
verify.Values(t, "", got.CheckType(), want)
}

View File

@ -1,6 +1,7 @@
package structs
import (
"net"
"time"
"github.com/hashicorp/raft"
@ -223,3 +224,20 @@ type OperatorHealthReply struct {
// Servers holds the health of each server.
Servers []ServerHealth
}
// (Enterprise-only) NetworkSegment is the configuration for a network segment, which is an
// isolated serf group on the LAN.
type NetworkSegment struct {
// Name is the name of the segment.
Name string
// Bind is the bind address for this segment.
Bind *net.TCPAddr
// Advertise is the advertise address of this segment.
Advertise *net.TCPAddr
// RPCListener is whether to bind a separate RPC listener on the bind address
// for this segment.
RPCListener bool
}

View File

@ -15,13 +15,13 @@ import (
"strings"
"time"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/agent/consul"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/logger"
"github.com/hashicorp/consul/test/porter"
"github.com/hashicorp/consul/testutil/retry"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/consul/version"
uuid "github.com/hashicorp/go-uuid"
)
@ -40,12 +40,14 @@ type TestAgent struct {
// Name is an optional name of the agent.
Name string
HCL string
// Config is the agent configuration. If Config is nil then
// TestConfig() is used. If Config.DataDir is set then it is
// the callers responsibility to clean up the data directory.
// Otherwise, a temporary data directory is created and removed
// when Shutdown() is called.
Config *Config
Config *config.RuntimeConfig
// LogOutput is the sink for the logs. If nil, logs are written
// to os.Stderr.
@ -83,8 +85,8 @@ type TestAgent struct {
// configuration. It panics if the agent could not be started. The
// caller should call Shutdown() to stop the agent and remove temporary
// directories.
func NewTestAgent(name string, c *Config) *TestAgent {
a := &TestAgent{Name: name, Config: c}
func NewTestAgent(name string, hcl string) *TestAgent {
a := &TestAgent{Name: name, HCL: hcl}
a.Start()
return a
}
@ -99,13 +101,8 @@ func (a *TestAgent) Start() *TestAgent {
if a.Agent != nil {
panic("TestAgent already started")
}
if a.Config == nil {
a.Config = TestConfig()
}
if a.Config.DNSRecursor != "" {
a.Config.DNSRecursors = append(a.Config.DNSRecursors, a.Config.DNSRecursor)
}
if a.Config.DataDir == "" {
var hclDataDir string
if a.DataDir == "" {
name := "agent"
if a.Name != "" {
name = a.Name + "-agent"
@ -115,13 +112,16 @@ func (a *TestAgent) Start() *TestAgent {
if err != nil {
panic(fmt.Sprintf("Error creating data dir %s: %s", filepath.Join(TempDir, name), err))
}
a.DataDir = d
a.Config.DataDir = d
hclDataDir = `data_dir = "` + d + `"`
}
id := UniqueID()
for i := 10; i >= 0; i-- {
pickRandomPorts(a.Config)
a.Config = TestConfig(
config.Source{Name: a.Name, Format: "hcl", Data: a.HCL},
config.Source{Name: a.Name + ".data_dir", Format: "hcl", Data: hclDataDir},
randomPortsSource(),
)
// write the keyring
if a.Key != "" {
@ -146,7 +146,7 @@ func (a *TestAgent) Start() *TestAgent {
}
agent.LogOutput = logOutput
agent.LogWriter = a.LogWriter
agent.logger = log.New(logOutput, a.Name+" - ", log.LstdFlags)
agent.logger = log.New(logOutput, a.Name+" - ", log.LstdFlags|log.Lmicroseconds)
// we need the err var in the next exit condition
if err := agent.Start(); err == nil {
@ -182,7 +182,7 @@ func (a *TestAgent) Start() *TestAgent {
if len(a.httpServers) == 0 {
r.Fatal(a.Name, "waiting for server")
}
if a.Config.Bootstrap && a.Config.Server {
if a.Config.Bootstrap && a.Config.ServerMode {
// Ensure we have a leader and a node registration.
args := &structs.DCSpecificRequest{
Datacenter: a.Config.Datacenter,
@ -217,17 +217,28 @@ func (a *TestAgent) Start() *TestAgent {
// Shutdown stops the agent and removes the data directory if it is
// managed by the test agent.
func (a *TestAgent) Shutdown() error {
/* Removed this because it was breaking persistence tests where we would
persist a service and load it through a new agent with the same data-dir.
Not sure if we still need this for other things, everywhere we manually make
a data dir we already do 'defer os.RemoveAll()'
defer func() {
if a.DataDir != "" {
os.RemoveAll(a.DataDir)
}
}()
}()*/
// shutdown agent before endpoints
defer a.Agent.ShutdownEndpoints()
return a.Agent.ShutdownAgent()
}
func (a *TestAgent) DNSAddr() string {
if a.dns == nil {
return ""
}
return a.dns.Addr
}
func (a *TestAgent) HTTPAddr() string {
if a.srv == nil {
return ""
@ -275,12 +286,6 @@ func UniqueID() string {
return id
}
// TenPorts returns the first port number of a block of
// ten random ports.
func TenPorts() int {
return 1030 + int(rand.Int31n(6440))*10
}
// pickRandomPorts selects random ports from fixed size random blocks of
// ports. This does not eliminate the chance for port conflict but
// reduces it significanltly with little overhead. Furthermore, asking
@ -289,71 +294,85 @@ func TenPorts() int {
// chance of port conflicts for concurrently executed test binaries.
// Instead of relying on one set of ports to be sufficient we retry
// starting the agent with different ports on port conflict.
func pickRandomPorts(c *Config) {
port := TenPorts()
c.Ports.DNS = port + 1
c.Ports.HTTP = port + 2
// when we enable HTTPS then we need to fix finding the
// "first" HTTP server since that might be HTTPS server
// c.Ports.HTTPS = port + 3
c.Ports.SerfLan = port + 4
c.Ports.SerfWan = port + 5
c.Ports.Server = port + 6
func randomPortsSource() config.Source {
ports, err := porter.RandomPorts(5)
if err != nil {
panic(err)
}
return config.Source{
Name: "ports",
Format: "hcl",
Data: `
ports = {
dns = ` + strconv.Itoa(ports[0]) + `
http = ` + strconv.Itoa(ports[1]) + `
https = -1
serf_lan = ` + strconv.Itoa(ports[2]) + `
serf_wan = ` + strconv.Itoa(ports[3]) + `
server = ` + strconv.Itoa(ports[4]) + `
}
`,
}
}
func NodeID() string {
id, err := uuid.GenerateUUID()
if err != nil {
panic(err)
}
return id
}
// TestConfig returns a unique default configuration for testing an
// agent.
func TestConfig() *Config {
nodeID, err := uuid.GenerateUUID()
if err != nil {
panic(err)
func TestConfig(sources ...config.Source) *config.RuntimeConfig {
nodeID := NodeID()
testsrc := config.Source{
Name: "test",
Format: "hcl",
Data: `
bind_addr = "127.0.0.1"
advertise_addr = "127.0.0.1"
datacenter = "dc1"
bootstrap = true
server = true
node_id = "` + nodeID + `"
node_name = "Node ` + nodeID + `"
performance {
raft_multiplier = 1
}
`,
}
cfg := DefaultConfig()
b, err := config.NewBuilder(config.Flags{})
if err != nil {
panic("NewBuilder failed: " + err.Error())
}
b.Head = append(b.Head, testsrc)
b.Tail = append(b.Tail, config.DefaultConsulSource(), config.DevConsulSource())
b.Tail = append(b.Tail, sources...)
cfg.Version = version.Version
cfg.VersionPrerelease = "c.d"
cfg, err := b.BuildAndValidate()
if err != nil {
panic("Error building config: " + err.Error())
}
cfg.NodeID = types.NodeID(nodeID)
cfg.NodeName = "Node " + nodeID
cfg.BindAddr = "127.0.0.1"
cfg.AdvertiseAddr = "127.0.0.1"
cfg.Datacenter = "dc1"
cfg.Bootstrap = true
cfg.Server = true
for _, w := range b.Warnings {
fmt.Println("WARNING:", w)
}
ccfg := consul.DefaultConfig()
cfg.ConsulConfig = ccfg
ccfg.SerfLANConfig.MemberlistConfig.SuspicionMult = 3
ccfg.SerfLANConfig.MemberlistConfig.ProbeTimeout = 100 * time.Millisecond
ccfg.SerfLANConfig.MemberlistConfig.ProbeInterval = 100 * time.Millisecond
ccfg.SerfLANConfig.MemberlistConfig.GossipInterval = 100 * time.Millisecond
ccfg.SerfWANConfig.MemberlistConfig.SuspicionMult = 3
ccfg.SerfWANConfig.MemberlistConfig.ProbeTimeout = 100 * time.Millisecond
ccfg.SerfWANConfig.MemberlistConfig.ProbeInterval = 100 * time.Millisecond
ccfg.SerfWANConfig.MemberlistConfig.GossipInterval = 100 * time.Millisecond
ccfg.RaftConfig.LeaderLeaseTimeout = 20 * time.Millisecond
ccfg.RaftConfig.HeartbeatTimeout = 40 * time.Millisecond
ccfg.RaftConfig.ElectionTimeout = 40 * time.Millisecond
ccfg.CoordinateUpdatePeriod = 100 * time.Millisecond
ccfg.ServerHealthInterval = 10 * time.Millisecond
cfg.SetupTaggedAndAdvertiseAddrs()
return cfg
return &cfg
}
// TestACLConfig returns a default configuration for testing an agent
// with ACLs.
func TestACLConfig() *Config {
cfg := TestConfig()
cfg.ACLDatacenter = cfg.Datacenter
cfg.ACLDefaultPolicy = "deny"
cfg.ACLMasterToken = "root"
cfg.ACLAgentToken = "root"
cfg.ACLAgentMasterToken = "towel"
cfg.ACLEnforceVersion8 = Bool(true)
return cfg
func TestACLConfig() string {
return `
acl_datacenter = "dc1"
acl_default_policy = "deny"
acl_master_token = "root"
acl_agent_token = "root"
acl_agent_master_token = "towel"
acl_enforce_version_8 = true
`
}

28
agent/testagent_test.go Normal file
View File

@ -0,0 +1,28 @@
package agent
import (
"testing"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/hcl"
)
// TestDefaultConfig triggers a data race in the HCL parser.
func TestDefaultConfig(t *testing.T) {
for i := 0; i < 500; i++ {
t.Run("", func(t *testing.T) {
t.Parallel()
var c config.Config
data := config.DefaultSource().Data
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
hcl.Decode(&c, data)
})
}
}

View File

@ -10,7 +10,7 @@ import (
// depending on how the agent and the other node are configured. The dc
// parameter is the dc the datacenter this node is from.
func (a *Agent) TranslateAddress(dc string, addr string, taggedAddresses map[string]string) string {
if a.config.TranslateWanAddrs && (a.config.Datacenter != dc) {
if a.config.TranslateWANAddrs && (a.config.Datacenter != dc) {
wanAddr := taggedAddresses["wan"]
if wanAddr != "" {
addr = wanAddr
@ -34,7 +34,7 @@ func (a *Agent) TranslateAddresses(dc string, subj interface{}) {
// done. This also happens to skip looking at any of the incoming
// structure for the common case of not needing to translate, so it will
// skip a lot of work if no translation needs to be done.
if !a.config.TranslateWanAddrs || (a.config.Datacenter == dc) {
if !a.config.TranslateWANAddrs || (a.config.Datacenter == dc) {
return
}

View File

@ -14,7 +14,7 @@ import (
func TestTxnEndpoint_Bad_JSON(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
buf := bytes.NewBuffer([]byte("{"))
@ -33,7 +33,7 @@ func TestTxnEndpoint_Bad_JSON(t *testing.T) {
func TestTxnEndpoint_Bad_Method(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
buf := bytes.NewBuffer([]byte("{}"))
@ -49,20 +49,20 @@ func TestTxnEndpoint_Bad_Method(t *testing.T) {
func TestTxnEndpoint_Bad_Size_Item(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
buf := bytes.NewBuffer([]byte(fmt.Sprintf(`
[
{
"KV": {
"Verb": "set",
"Key": "key",
"Value": %q
}
}
]
`, strings.Repeat("bad", 2*maxKVSize))))
[
{
"KV": {
"Verb": "set",
"Key": "key",
"Value": %q
}
}
]
`, strings.Repeat("bad", 2*maxKVSize))))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
if _, err := a.srv.Txn(resp, req); err != nil {
@ -75,35 +75,35 @@ func TestTxnEndpoint_Bad_Size_Item(t *testing.T) {
func TestTxnEndpoint_Bad_Size_Net(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
value := strings.Repeat("X", maxKVSize/2)
buf := bytes.NewBuffer([]byte(fmt.Sprintf(`
[
{
"KV": {
"Verb": "set",
"Key": "key1",
"Value": %q
}
},
{
"KV": {
"Verb": "set",
"Key": "key1",
"Value": %q
}
},
{
"KV": {
"Verb": "set",
"Key": "key1",
"Value": %q
}
}
]
`, value, value, value)))
[
{
"KV": {
"Verb": "set",
"Key": "key1",
"Value": %q
}
},
{
"KV": {
"Verb": "set",
"Key": "key1",
"Value": %q
}
},
{
"KV": {
"Verb": "set",
"Key": "key1",
"Value": %q
}
}
]
`, value, value, value)))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
if _, err := a.srv.Txn(resp, req); err != nil {
@ -116,21 +116,21 @@ func TestTxnEndpoint_Bad_Size_Net(t *testing.T) {
func TestTxnEndpoint_Bad_Size_Ops(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
buf := bytes.NewBuffer([]byte(fmt.Sprintf(`
[
%s
{
"KV": {
"Verb": "set",
"Key": "key",
"Value": ""
}
}
]
`, strings.Repeat(`{ "KV": { "Verb": "get", "Key": "key" } },`, 2*maxTxnOps))))
[
%s
{
"KV": {
"Verb": "set",
"Key": "key",
"Value": ""
}
}
]
`, strings.Repeat(`{ "KV": { "Verb": "get", "Key": "key" } },`, 2*maxTxnOps))))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
if _, err := a.srv.Txn(resp, req); err != nil {
@ -144,7 +144,7 @@ func TestTxnEndpoint_Bad_Size_Ops(t *testing.T) {
func TestTxnEndpoint_KV_Actions(t *testing.T) {
t.Parallel()
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
// Make sure all incoming fields get converted properly to the internal
@ -153,24 +153,24 @@ func TestTxnEndpoint_KV_Actions(t *testing.T) {
id := makeTestSession(t, a.srv)
{
buf := bytes.NewBuffer([]byte(fmt.Sprintf(`
[
{
"KV": {
"Verb": "lock",
"Key": "key",
"Value": "aGVsbG8gd29ybGQ=",
"Flags": 23,
"Session": %q
}
},
{
"KV": {
"Verb": "get",
"Key": "key"
}
}
]
`, id)))
[
{
"KV": {
"Verb": "lock",
"Key": "key",
"Value": "aGVsbG8gd29ybGQ=",
"Flags": 23,
"Session": %q
}
},
{
"KV": {
"Verb": "get",
"Key": "key"
}
}
]
`, id)))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
obj, err := a.srv.Txn(resp, req)
@ -228,21 +228,21 @@ func TestTxnEndpoint_KV_Actions(t *testing.T) {
// fast-path endpoint.
{
buf := bytes.NewBuffer([]byte(`
[
{
"KV": {
"Verb": "get",
"Key": "key"
}
},
{
"KV": {
"Verb": "get-tree",
"Key": "key"
}
}
]
`))
[
{
"KV": {
"Verb": "get",
"Key": "key"
}
},
{
"KV": {
"Verb": "get-tree",
"Key": "key"
}
}
]
`))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
obj, err := a.srv.Txn(resp, req)
@ -310,23 +310,23 @@ func TestTxnEndpoint_KV_Actions(t *testing.T) {
// index field gets translated to the RPC format.
{
buf := bytes.NewBuffer([]byte(fmt.Sprintf(`
[
{
"KV": {
"Verb": "cas",
"Key": "key",
"Value": "Z29vZGJ5ZSB3b3JsZA==",
"Index": %d
}
},
{
"KV": {
"Verb": "get",
"Key": "key"
}
}
]
`, index)))
[
{
"KV": {
"Verb": "cas",
"Key": "key",
"Value": "Z29vZGJ5ZSB3b3JsZA==",
"Index": %d
}
},
{
"KV": {
"Verb": "get",
"Key": "key"
}
}
]
`, index)))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
obj, err := a.srv.Txn(resp, req)
@ -379,27 +379,27 @@ func TestTxnEndpoint_KV_Actions(t *testing.T) {
// Verify an error inside a transaction.
t.Run("", func(t *testing.T) {
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
buf := bytes.NewBuffer([]byte(`
[
{
"KV": {
"Verb": "lock",
"Key": "key",
"Value": "aGVsbG8gd29ybGQ=",
"Session": "nope"
}
},
{
"KV": {
"Verb": "get",
"Key": "key"
}
}
]
`))
[
{
"KV": {
"Verb": "lock",
"Key": "key",
"Value": "aGVsbG8gd29ybGQ=",
"Session": "nope"
}
},
{
"KV": {
"Verb": "get",
"Key": "key"
}
}
]
`))
req, _ := http.NewRequest("PUT", "/v1/txn", buf)
resp := httptest.NewRecorder()
if _, err := a.srv.Txn(resp, req); err != nil {

View File

@ -15,7 +15,7 @@ import (
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testutil"
"github.com/hashicorp/go-cleanhttp"
cleanhttp "github.com/hashicorp/go-cleanhttp"
)
func TestUiIndex(t *testing.T) {
@ -25,9 +25,9 @@ func TestUiIndex(t *testing.T) {
defer os.RemoveAll(uiDir)
// Make the server
cfg := TestConfig()
cfg.UIDir = uiDir
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), `
ui_dir = "`+uiDir+`"
`)
defer a.Shutdown()
// Create file
@ -63,7 +63,7 @@ func TestUiIndex(t *testing.T) {
func TestUiNodes(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
args := &structs.RegisterRequest{
@ -100,7 +100,7 @@ func TestUiNodes(t *testing.T) {
func TestUiNodeInfo(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
req, _ := http.NewRequest("GET", fmt.Sprintf("/v1/internal/ui/node/%s", a.Config.NodeName), nil)

View File

@ -209,7 +209,7 @@ func (a *Agent) ingestUserEvent(msg *UserEvent) {
// Special handling for internal events
switch msg.Name {
case remoteExecName:
if *a.config.DisableRemoteExec {
if a.config.DisableRemoteExec {
a.logger.Printf("[INFO] agent: ignoring remote exec event (%s), disabled.", msg.ID)
} else {
go a.handleRemoteExec(msg)

View File

@ -48,7 +48,7 @@ func TestValidateUserEventParams(t *testing.T) {
func TestShouldProcessUserEvent(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
srv1 := &structs.NodeService{
@ -117,7 +117,7 @@ func TestShouldProcessUserEvent(t *testing.T) {
func TestIngestUserEvent(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
for i := 0; i < 512; i++ {
@ -148,7 +148,7 @@ func TestIngestUserEvent(t *testing.T) {
func TestFireReceiveEvent(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), nil)
a := NewTestAgent(t.Name(), "")
defer a.Shutdown()
srv1 := &structs.NodeService{
@ -184,9 +184,9 @@ func TestFireReceiveEvent(t *testing.T) {
func TestUserEventToken(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.ACLDefaultPolicy = "deny" // Set the default policies to deny
a := NewTestAgent(t.Name(), cfg)
a := NewTestAgent(t.Name(), TestACLConfig()+`
acl_default_policy = "deny"
`)
defer a.Shutdown()
// Create an ACL token

View File

@ -6,7 +6,7 @@ import (
"fmt"
"math"
"os"
"os/user"
osuser "os/user"
"strconv"
"time"
@ -65,47 +65,34 @@ func checkIDHash(checkID types.CheckID) string {
return stringHash(string(checkID))
}
// FilePermissions is an interface which allows a struct to set
// ownership and permissions easily on a file it describes.
type FilePermissions interface {
// User returns a user ID or user name
User() string
// Group returns a group ID. Group names are not supported.
Group() string
// Mode returns a string of file mode bits e.g. "0644"
Mode() string
}
// setFilePermissions handles configuring ownership and permissions settings
// on a given file. It takes a path and any struct implementing the
// FilePermissions interface. All permission/ownership settings are optional.
// If no user or group is specified, the current user/group will be used. Mode
// is optional, and has no default (the operation is not performed if absent).
// User may be specified by name or ID, but group may only be specified by ID.
func setFilePermissions(path string, p FilePermissions) error {
// setFilePermissions handles configuring ownership and permissions
// settings on a given file. All permission/ownership settings are
// optional. If no user or group is specified, the current user/group
// will be used. Mode is optional, and has no default (the operation is
// not performed if absent). User may be specified by name or ID, but
// group may only be specified by ID.
func setFilePermissions(path string, user, group, mode string) error {
var err error
uid, gid := os.Getuid(), os.Getgid()
if p.User() != "" {
if uid, err = strconv.Atoi(p.User()); err == nil {
if user != "" {
if uid, err = strconv.Atoi(user); err == nil {
goto GROUP
}
// Try looking up the user by name
if u, err := user.Lookup(p.User()); err == nil {
if u, err := osuser.Lookup(user); err == nil {
uid, _ = strconv.Atoi(u.Uid)
goto GROUP
}
return fmt.Errorf("invalid user specified: %v", p.User())
return fmt.Errorf("invalid user specified: %v", user)
}
GROUP:
if p.Group() != "" {
if gid, err = strconv.Atoi(p.Group()); err != nil {
return fmt.Errorf("invalid group specified: %v", p.Group())
if group != "" {
if gid, err = strconv.Atoi(group); err != nil {
return fmt.Errorf("invalid group specified: %v", group)
}
}
if err := os.Chown(path, uid, gid); err != nil {
@ -113,10 +100,10 @@ GROUP:
uid, gid, path, err)
}
if p.Mode() != "" {
mode, err := strconv.ParseUint(p.Mode(), 8, 32)
if mode != "" {
mode, err := strconv.ParseUint(mode, 8, 32)
if err != nil {
return fmt.Errorf("invalid mode specified: %v", p.Mode())
return fmt.Errorf("invalid mode specified: %v", mode)
}
if err := os.Chmod(path, os.FileMode(mode)); err != nil {
return fmt.Errorf("failed setting permissions to %d on %q: %s",

View File

@ -46,22 +46,22 @@ func TestSetFilePermissions(t *testing.T) {
defer os.Remove(path)
// Bad UID fails
if err := setFilePermissions(path, UnixSocketPermissions{Usr: "%"}); err == nil {
if err := setFilePermissions(path, "%", "", ""); err == nil {
t.Fatalf("should fail")
}
// Bad GID fails
if err := setFilePermissions(path, UnixSocketPermissions{Grp: "%"}); err == nil {
if err := setFilePermissions(path, "", "%", ""); err == nil {
t.Fatalf("should fail")
}
// Bad mode fails
if err := setFilePermissions(path, UnixSocketPermissions{Perms: "%"}); err == nil {
if err := setFilePermissions(path, "", "", "%"); err == nil {
t.Fatalf("should fail")
}
// Allows omitting user/group/mode
if err := setFilePermissions(path, UnixSocketPermissions{}); err != nil {
if err := setFilePermissions(path, "", "", ""); err != nil {
t.Fatalf("err: %s", err)
}
@ -69,7 +69,7 @@ func TestSetFilePermissions(t *testing.T) {
if err := os.Chmod(path, 0700); err != nil {
t.Fatalf("err: %s", err)
}
if err := setFilePermissions(path, UnixSocketPermissions{}); err != nil {
if err := setFilePermissions(path, "", "", ""); err != nil {
t.Fatalf("err: %s", err)
}
fi, err := os.Stat(path)
@ -81,7 +81,7 @@ func TestSetFilePermissions(t *testing.T) {
}
// Changes mode if given
if err := setFilePermissions(path, UnixSocketPermissions{Perms: "0777"}); err != nil {
if err := setFilePermissions(path, "", "", "0777"); err != nil {
t.Fatalf("err: %s", err)
}
fi, err = os.Stat(path)

View File

@ -1,6 +1,7 @@
package api
import (
"fmt"
"io/ioutil"
"strings"
"testing"
@ -630,7 +631,9 @@ func TestAPI_AgentJoin(t *testing.T) {
}
// Join ourself
addr := info["Config"]["AdvertiseAddr"].(string)
ip := info["Config"]["SerfAdvertiseAddrLAN"].(map[string]interface{})["IP"].(string)
port := info["Config"]["SerfAdvertiseAddrLAN"].(map[string]interface{})["Port"].(float64)
addr := fmt.Sprintf("%s:%d", ip, int(port))
err = agent.Join(addr, false)
if err != nil {
t.Fatalf("err: %v", err)

View File

@ -446,6 +446,7 @@ func NewClient(config *Config) (*Client, error) {
if len(parts) == 2 {
switch parts[0] {
case "http":
config.Scheme = "http"
case "https":
config.Scheme = "https"
case "unix":
@ -466,10 +467,7 @@ func NewClient(config *Config) (*Client, error) {
config.Token = defConfig.Token
}
client := &Client{
config: *config,
}
return client, nil
return &Client{config: *config}, nil
}
// NewHttpClient returns an http client configured with the given Transport and TLS

View File

@ -52,6 +52,7 @@ func makeClientWithConfig(
// Create client
client, err := NewClient(conf)
if err != nil {
server.Stop()
t.Fatalf("err: %v", err)
}
@ -73,7 +74,11 @@ func testKey() string {
}
func TestAPI_DefaultConfig_env(t *testing.T) {
t.Parallel()
// t.Parallel() // DO NOT ENABLE !!!
// do not enable t.Parallel for this test since it modifies global state
// (environment) which has non-deterministic effects on the other tests
// which derive their default configuration from the environment
addr := "1.2.3.4:5678"
token := "abcd1234"
auth := "username:password"
@ -151,6 +156,7 @@ func TestAPI_DefaultConfig_env(t *testing.T) {
}
func TestAPI_SetupTLSConfig(t *testing.T) {
t.Parallel()
// A default config should result in a clean default client config.
tlsConfig := &TLSConfig{}
cc, err := SetupTLSConfig(tlsConfig)
@ -505,6 +511,7 @@ func TestAPI_UnixSocket(t *testing.T) {
}
func TestAPI_durToMsec(t *testing.T) {
t.Parallel()
if ms := durToMsec(0); ms != "0ms" {
t.Fatalf("bad: %s", ms)
}
@ -523,6 +530,7 @@ func TestAPI_durToMsec(t *testing.T) {
}
func TestAPI_IsServerError(t *testing.T) {
t.Parallel()
if IsServerError(nil) {
t.Fatalf("should not be a server error")
}

View File

@ -26,6 +26,7 @@ func TestAPI_CatalogDatacenters(t *testing.T) {
}
func TestAPI_CatalogNodes(t *testing.T) {
t.Parallel()
c, s := makeClient(t)
defer s.Stop()
@ -62,6 +63,7 @@ func TestAPI_CatalogNodes(t *testing.T) {
}
func TestAPI_CatalogNodes_MetaFilter(t *testing.T) {
t.Parallel()
meta := map[string]string{"somekey": "somevalue"}
c, s := makeClientWithConfig(t, nil, func(conf *testutil.TestServerConfig) {
conf.NodeMeta = meta
@ -137,6 +139,7 @@ func TestAPI_CatalogServices(t *testing.T) {
}
func TestAPI_CatalogServices_NodeMetaFilter(t *testing.T) {
t.Parallel()
meta := map[string]string{"somekey": "somevalue"}
c, s := makeClientWithConfig(t, nil, func(conf *testutil.TestServerConfig) {
conf.NodeMeta = meta

View File

@ -256,6 +256,7 @@ func TestAPI_HealthChecks_NodeMetaFilter(t *testing.T) {
}
func TestAPI_HealthService(t *testing.T) {
t.Parallel()
c, s := makeClient(t)
defer s.Stop()
@ -282,6 +283,7 @@ func TestAPI_HealthService(t *testing.T) {
}
func TestAPI_HealthService_NodeMetaFilter(t *testing.T) {
t.Parallel()
meta := map[string]string{"somekey": "somevalue"}
c, s := makeClientWithConfig(t, nil, func(conf *testutil.TestServerConfig) {
conf.NodeMeta = meta

View File

@ -7,9 +7,9 @@ import (
)
func TestAPI_OperatorKeyringInstallListPutRemove(t *testing.T) {
t.Parallel()
oldKey := "d8wu8CSUrqgtjVsvcBPmhQ=="
newKey := "qxycTi/SsePj/TZzCBmNXw=="
t.Parallel()
c, s := makeClientWithConfig(t, nil, func(c *testutil.TestServerConfig) {
c.Encrypt = oldKey
})

View File

@ -8,7 +8,6 @@ import (
"os"
"os/signal"
"path/filepath"
"reflect"
"regexp"
"strings"
"syscall"
@ -18,14 +17,10 @@ import (
"github.com/armon/go-metrics/circonus"
"github.com/armon/go-metrics/datadog"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/configutil"
"github.com/hashicorp/consul/ipaddr"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/logger"
"github.com/hashicorp/consul/watch"
"github.com/hashicorp/go-checkpoint"
discover "github.com/hashicorp/go-discover"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/logutils"
"github.com/mitchellh/cli"
@ -53,500 +48,30 @@ type AgentCommand struct {
// readConfig is responsible for setup of our configuration using
// the command line and any file configs
func (cmd *AgentCommand) readConfig() *agent.Config {
var cmdCfg agent.Config
var cfgFiles []string
var retryInterval string
var retryIntervalWan string
var dnsRecursors []string
var dev bool
var nodeMeta []string
f := cmd.BaseCommand.NewFlagSet(cmd)
f.Var((*configutil.AppendSliceValue)(&cfgFiles), "config-file",
"Path to a JSON file to read configuration from. This can be specified multiple times.")
f.Var((*configutil.AppendSliceValue)(&cfgFiles), "config-dir",
"Path to a directory to read configuration files from. This will read every file ending "+
"in '.json' as configuration in this directory in alphabetical order. This can be "+
"specified multiple times.")
f.Var((*configutil.AppendSliceValue)(&dnsRecursors), "recursor",
"Address of an upstream DNS server. Can be specified multiple times.")
f.Var((*configutil.AppendSliceValue)(&nodeMeta), "node-meta",
"An arbitrary metadata key/value pair for this node, of the format `key:value`. Can be specified multiple times.")
f.BoolVar(&dev, "dev", false, "Starts the agent in development mode.")
f.StringVar(&cmdCfg.LogLevel, "log-level", "", "Log level of the agent.")
f.StringVar(&cmdCfg.NodeName, "node", "", "Name of this node. Must be unique in the cluster.")
f.StringVar((*string)(&cmdCfg.NodeID), "node-id", "",
"A unique ID for this node across space and time. Defaults to a randomly-generated ID"+
" that persists in the data-dir.")
f.BoolVar(&cmdCfg.EnableScriptChecks, "enable-script-checks", false, "Enables health check scripts.")
var disableHostNodeID configutil.BoolValue
f.Var(&disableHostNodeID, "disable-host-node-id",
"Setting this to true will prevent Consul from using information from the"+
" host to generate a node ID, and will cause Consul to generate a"+
" random node ID instead.")
f.StringVar(&cmdCfg.Datacenter, "datacenter", "", "Datacenter of the agent.")
f.StringVar(&cmdCfg.DataDir, "data-dir", "", "Path to a data directory to store agent state.")
f.BoolVar(&cmdCfg.EnableUI, "ui", false, "Enables the built-in static web UI server.")
f.StringVar(&cmdCfg.UIDir, "ui-dir", "", "Path to directory containing the web UI resources.")
f.StringVar(&cmdCfg.PidFile, "pid-file", "", "Path to file to store agent PID.")
f.StringVar(&cmdCfg.EncryptKey, "encrypt", "", "Provides the gossip encryption key.")
f.BoolVar(&cmdCfg.DisableKeyringFile, "disable-keyring-file", false, "Disables the backing up "+
"of the keyring to a file.")
f.BoolVar(&cmdCfg.Server, "server", false, "Switches agent to server mode.")
f.BoolVar(&cmdCfg.NonVotingServer, "non-voting-server", false,
"(Enterprise-only) This flag is used to make the server not participate in the Raft quorum, "+
"and have it only receive the data replication stream. This can be used to add read scalability "+
"to a cluster in cases where a high volume of reads to servers are needed.")
f.BoolVar(&cmdCfg.Bootstrap, "bootstrap", false, "Sets server to bootstrap mode.")
f.IntVar(&cmdCfg.BootstrapExpect, "bootstrap-expect", 0, "Sets server to expect bootstrap mode.")
f.StringVar(&cmdCfg.Domain, "domain", "", "Domain to use for DNS interface.")
f.StringVar(&cmdCfg.ClientAddr, "client", "",
"Sets the address to bind for client access. This includes RPC, DNS, HTTP and HTTPS (if configured).")
f.StringVar(&cmdCfg.BindAddr, "bind", "", "Sets the bind address for cluster communication.")
f.StringVar(&cmdCfg.SerfWanBindAddr, "serf-wan-bind", "", "Address to bind Serf WAN listeners to.")
f.StringVar(&cmdCfg.SerfLanBindAddr, "serf-lan-bind", "", "Address to bind Serf LAN listeners to.")
f.IntVar(&cmdCfg.Ports.HTTP, "http-port", 0, "Sets the HTTP API port to listen on.")
f.IntVar(&cmdCfg.Ports.DNS, "dns-port", 0, "DNS port to use.")
f.StringVar(&cmdCfg.AdvertiseAddr, "advertise", "", "Sets the advertise address to use.")
f.StringVar(&cmdCfg.AdvertiseAddrWan, "advertise-wan", "",
"Sets address to advertise on WAN instead of -advertise address.")
f.StringVar(&cmdCfg.Segment, "segment", "", "(Enterprise-only) Sets the network segment to join.")
f.IntVar(&cmdCfg.Protocol, "protocol", -1,
"Sets the protocol version. Defaults to latest.")
f.IntVar(&cmdCfg.RaftProtocol, "raft-protocol", -1,
"Sets the Raft protocol version. Defaults to latest.")
f.BoolVar(&cmdCfg.EnableSyslog, "syslog", false,
"Enables logging to syslog.")
f.BoolVar(&cmdCfg.RejoinAfterLeave, "rejoin", false,
"Ignores a previous leave and attempts to rejoin the cluster.")
f.Var((*configutil.AppendSliceValue)(&cmdCfg.StartJoin), "join",
"Address of an agent to join at start time. Can be specified multiple times.")
f.Var((*configutil.AppendSliceValue)(&cmdCfg.StartJoinWan), "join-wan",
"Address of an agent to join -wan at start time. Can be specified multiple times.")
f.Var((*configutil.AppendSliceValue)(&cmdCfg.RetryJoin), "retry-join",
"Address of an agent to join at start time with retries enabled. Can be specified multiple times.")
f.IntVar(&cmdCfg.RetryMaxAttempts, "retry-max", 0,
"Maximum number of join attempts. Defaults to 0, which will retry indefinitely.")
f.StringVar(&retryInterval, "retry-interval", "",
"Time to wait between join attempts.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinEC2.Region, "retry-join-ec2-region", "",
"EC2 Region to discover servers in.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinEC2.TagKey, "retry-join-ec2-tag-key", "",
"EC2 tag key to filter on for server discovery.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinEC2.TagValue, "retry-join-ec2-tag-value", "",
"EC2 tag value to filter on for server discovery.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinGCE.ProjectName, "retry-join-gce-project-name", "",
"Google Compute Engine project to discover servers in.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinGCE.ZonePattern, "retry-join-gce-zone-pattern", "",
"Google Compute Engine region or zone to discover servers in (regex pattern).")
f.StringVar(&cmdCfg.DeprecatedRetryJoinGCE.TagValue, "retry-join-gce-tag-value", "",
"Google Compute Engine tag value to filter on for server discovery.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinGCE.CredentialsFile, "retry-join-gce-credentials-file", "",
"Path to credentials JSON file to use with Google Compute Engine.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinAzure.TagName, "retry-join-azure-tag-name", "",
"Azure tag name to filter on for server discovery.")
f.StringVar(&cmdCfg.DeprecatedRetryJoinAzure.TagValue, "retry-join-azure-tag-value", "",
"Azure tag value to filter on for server discovery.")
f.Var((*configutil.AppendSliceValue)(&cmdCfg.RetryJoinWan), "retry-join-wan",
"Address of an agent to join -wan at start time with retries enabled. "+
"Can be specified multiple times.")
f.IntVar(&cmdCfg.RetryMaxAttemptsWan, "retry-max-wan", 0,
"Maximum number of join -wan attempts. Defaults to 0, which will retry indefinitely.")
f.StringVar(&retryIntervalWan, "retry-interval-wan", "",
"Time to wait between join -wan attempts.")
// deprecated flags
var dcDeprecated string
var atlasJoin bool
var atlasInfrastructure, atlasToken, atlasEndpoint string
f.StringVar(&dcDeprecated, "dc", "",
"(deprecated) Datacenter of the agent (use 'datacenter' instead).")
f.StringVar(&atlasInfrastructure, "atlas", "",
"(deprecated) Sets the Atlas infrastructure name, enables SCADA.")
f.StringVar(&atlasToken, "atlas-token", "",
"(deprecated) Provides the Atlas API token.")
f.BoolVar(&atlasJoin, "atlas-join", false,
"(deprecated) Enables auto-joining the Atlas cluster.")
f.StringVar(&atlasEndpoint, "atlas-endpoint", "",
"(deprecated) The address of the endpoint for Atlas integration.")
func (cmd *AgentCommand) readConfig() *config.RuntimeConfig {
var flags config.Flags
fs := cmd.BaseCommand.NewFlagSet(cmd)
config.AddFlags(fs, &flags)
if err := cmd.BaseCommand.Parse(cmd.args); err != nil {
cmd.UI.Error(fmt.Sprintf("error parsing flags: %v", err))
return nil
}
// check deprecated flags
if atlasInfrastructure != "" {
cmd.UI.Warn("WARNING: 'atlas' is deprecated")
}
if atlasToken != "" {
cmd.UI.Warn("WARNING: 'atlas-token' is deprecated")
}
if atlasJoin {
cmd.UI.Warn("WARNING: 'atlas-join' is deprecated")
}
if atlasEndpoint != "" {
cmd.UI.Warn("WARNING: 'atlas-endpoint' is deprecated")
}
if dcDeprecated != "" && cmdCfg.Datacenter == "" {
cmd.UI.Warn("WARNING: 'dc' is deprecated. Use 'datacenter' instead")
cmdCfg.Datacenter = dcDeprecated
}
if retryInterval != "" {
dur, err := time.ParseDuration(retryInterval)
if err != nil {
cmd.UI.Error(fmt.Sprintf("Error: %s", err))
return nil
}
cmdCfg.RetryInterval = dur
}
if retryIntervalWan != "" {
dur, err := time.ParseDuration(retryIntervalWan)
if err != nil {
cmd.UI.Error(fmt.Sprintf("Error: %s", err))
return nil
}
cmdCfg.RetryIntervalWan = dur
}
if len(nodeMeta) > 0 {
cmdCfg.Meta = make(map[string]string)
for _, entry := range nodeMeta {
key, value := agent.ParseMetaPair(entry)
cmdCfg.Meta[key] = value
}
if err := structs.ValidateMetadata(cmdCfg.Meta, false); err != nil {
cmd.UI.Error(fmt.Sprintf("Failed to parse node metadata: %v", err))
return nil
}
}
cfg := agent.DefaultConfig()
if dev {
cfg = agent.DevConfig()
}
if len(cfgFiles) > 0 {
fileConfig, err := agent.ReadConfigPaths(cfgFiles)
if err != nil {
cmd.UI.Error(err.Error())
return nil
}
cfg = agent.MergeConfig(cfg, fileConfig)
}
cmdCfg.DNSRecursors = append(cmdCfg.DNSRecursors, dnsRecursors...)
cfg = agent.MergeConfig(cfg, &cmdCfg)
disableHostNodeID.Merge(cfg.DisableHostNodeID)
if cfg.NodeName == "" {
hostname, err := os.Hostname()
if err != nil {
cmd.UI.Error(fmt.Sprintf("Error determining node name: %s", err))
return nil
}
cfg.NodeName = hostname
}
cfg.NodeName = strings.TrimSpace(cfg.NodeName)
if cfg.NodeName == "" {
cmd.UI.Error("Node name can not be empty")
b, err := config.NewBuilder(flags)
if err != nil {
cmd.UI.Error(err.Error())
return nil
}
// Make sure LeaveOnTerm and SkipLeaveOnInt are set to the right
// defaults based on the agent's mode (client or server).
if cfg.LeaveOnTerm == nil {
cfg.LeaveOnTerm = agent.Bool(!cfg.Server)
}
if cfg.SkipLeaveOnInt == nil {
cfg.SkipLeaveOnInt = agent.Bool(cfg.Server)
}
// Ensure we have a data directory if we are not in dev mode.
if !dev {
if cfg.DataDir == "" {
cmd.UI.Error("Must specify data directory using -data-dir")
return nil
}
if finfo, err := os.Stat(cfg.DataDir); err != nil {
if !os.IsNotExist(err) {
cmd.UI.Error(fmt.Sprintf("Error getting data-dir: %s", err))
return nil
}
} else if !finfo.IsDir() {
cmd.UI.Error(fmt.Sprintf("The data-dir specified at %q is not a directory", cfg.DataDir))
return nil
}
}
// Ensure all endpoints are unique
if err := cfg.VerifyUniqueListeners(); err != nil {
cmd.UI.Error(fmt.Sprintf("All listening endpoints must be unique: %s", err))
cfg, err := b.BuildAndValidate()
if err != nil {
cmd.UI.Error(err.Error())
return nil
}
// Check the data dir for signs of an un-migrated Consul 0.5.x or older
// server. Consul refuses to start if this is present to protect a server
// with existing data from starting on a fresh data set.
if cfg.Server {
mdbPath := filepath.Join(cfg.DataDir, "mdb")
if _, err := os.Stat(mdbPath); !os.IsNotExist(err) {
if os.IsPermission(err) {
cmd.UI.Error(fmt.Sprintf("CRITICAL: Permission denied for data folder at %q!", mdbPath))
cmd.UI.Error("Consul will refuse to boot without access to this directory.")
cmd.UI.Error("Please correct permissions and try starting again.")
return nil
}
cmd.UI.Error(fmt.Sprintf("CRITICAL: Deprecated data folder found at %q!", mdbPath))
cmd.UI.Error("Consul will refuse to boot with this directory present.")
cmd.UI.Error("See https://www.consul.io/docs/upgrade-specific.html for more information.")
return nil
}
for _, w := range b.Warnings {
cmd.UI.Warn(w)
}
// Verify DNS settings
if cfg.DNSConfig.UDPAnswerLimit < 1 {
cmd.UI.Error(fmt.Sprintf("dns_config.udp_answer_limit %d too low, must always be greater than zero", cfg.DNSConfig.UDPAnswerLimit))
}
if cfg.EncryptKey != "" {
if _, err := cfg.EncryptBytes(); err != nil {
cmd.UI.Error(fmt.Sprintf("Invalid encryption key: %s", err))
return nil
}
keyfileLAN := filepath.Join(cfg.DataDir, agent.SerfLANKeyring)
if _, err := os.Stat(keyfileLAN); err == nil {
cmd.UI.Error("WARNING: LAN keyring exists but -encrypt given, using keyring")
}
if cfg.Server {
keyfileWAN := filepath.Join(cfg.DataDir, agent.SerfWANKeyring)
if _, err := os.Stat(keyfileWAN); err == nil {
cmd.UI.Error("WARNING: WAN keyring exists but -encrypt given, using keyring")
}
}
}
// Ensure the datacenter is always lowercased. The DNS endpoints automatically
// lowercase all queries, and internally we expect DC1 and dc1 to be the same.
cfg.Datacenter = strings.ToLower(cfg.Datacenter)
// Verify datacenter is valid
if !validDatacenter.MatchString(cfg.Datacenter) {
cmd.UI.Error("Datacenter must be alpha-numeric with underscores and hypens only")
return nil
}
// If 'acl_datacenter' is set, ensure it is lowercased.
if cfg.ACLDatacenter != "" {
cfg.ACLDatacenter = strings.ToLower(cfg.ACLDatacenter)
// Verify 'acl_datacenter' is valid
if !validDatacenter.MatchString(cfg.ACLDatacenter) {
cmd.UI.Error("ACL datacenter must be alpha-numeric with underscores and hypens only")
return nil
}
}
// Only allow bootstrap mode when acting as a server
if cfg.Bootstrap && !cfg.Server {
cmd.UI.Error("Bootstrap mode cannot be enabled when server mode is not enabled")
return nil
}
// Expect can only work when acting as a server
if cfg.BootstrapExpect != 0 && !cfg.Server {
cmd.UI.Error("Expect mode cannot be enabled when server mode is not enabled")
return nil
}
// Expect can only work when dev mode is off
if cfg.BootstrapExpect > 0 && cfg.DevMode {
cmd.UI.Error("Expect mode cannot be enabled when dev mode is enabled")
return nil
}
// Expect & Bootstrap are mutually exclusive
if cfg.BootstrapExpect != 0 && cfg.Bootstrap {
cmd.UI.Error("Bootstrap cannot be provided with an expected server count")
return nil
}
if ipaddr.IsAny(cfg.AdvertiseAddr) {
cmd.UI.Error("Advertise address cannot be " + cfg.AdvertiseAddr)
return nil
}
if ipaddr.IsAny(cfg.AdvertiseAddrWan) {
cmd.UI.Error("Advertise WAN address cannot be " + cfg.AdvertiseAddrWan)
return nil
}
if cfg.Server && cfg.Segment != "" {
cmd.UI.Error("Segment option can only be set on clients")
return nil
}
if !cfg.Server && len(cfg.Segments) > 0 {
cmd.UI.Error("Segments can only be configured on servers")
return nil
}
// patch deprecated retry-join-{gce,azure,ec2)-* parameters
// into -retry-join and issue warning.
// todo(fs): this should really be in DecodeConfig where it can be tested
if !reflect.DeepEqual(cfg.DeprecatedRetryJoinEC2, agent.RetryJoinEC2{}) {
m := discover.Config{
"provider": "aws",
"region": cfg.DeprecatedRetryJoinEC2.Region,
"tag_key": cfg.DeprecatedRetryJoinEC2.TagKey,
"tag_value": cfg.DeprecatedRetryJoinEC2.TagValue,
"access_key_id": cfg.DeprecatedRetryJoinEC2.AccessKeyID,
"secret_access_key": cfg.DeprecatedRetryJoinEC2.SecretAccessKey,
}
cfg.RetryJoin = append(cfg.RetryJoin, m.String())
cfg.DeprecatedRetryJoinEC2 = agent.RetryJoinEC2{}
// redact m before output
if m["access_key_id"] != "" {
m["access_key_id"] = "hidden"
}
if m["secret_access_key"] != "" {
m["secret_access_key"] = "hidden"
}
cmd.UI.Warn(fmt.Sprintf("==> DEPRECATION: retry_join_ec2 is deprecated. "+
"Please add %q to retry_join\n", m))
}
if !reflect.DeepEqual(cfg.DeprecatedRetryJoinAzure, agent.RetryJoinAzure{}) {
m := discover.Config{
"provider": "azure",
"tag_name": cfg.DeprecatedRetryJoinAzure.TagName,
"tag_value": cfg.DeprecatedRetryJoinAzure.TagValue,
"subscription_id": cfg.DeprecatedRetryJoinAzure.SubscriptionID,
"tenant_id": cfg.DeprecatedRetryJoinAzure.TenantID,
"client_id": cfg.DeprecatedRetryJoinAzure.ClientID,
"secret_access_key": cfg.DeprecatedRetryJoinAzure.SecretAccessKey,
}
cfg.RetryJoin = append(cfg.RetryJoin, m.String())
cfg.DeprecatedRetryJoinAzure = agent.RetryJoinAzure{}
// redact m before output
if m["subscription_id"] != "" {
m["subscription_id"] = "hidden"
}
if m["tenant_id"] != "" {
m["tenant_id"] = "hidden"
}
if m["client_id"] != "" {
m["client_id"] = "hidden"
}
if m["secret_access_key"] != "" {
m["secret_access_key"] = "hidden"
}
cmd.UI.Warn(fmt.Sprintf("==> DEPRECATION: retry_join_azure is deprecated. "+
"Please add %q to retry_join\n", m))
}
if !reflect.DeepEqual(cfg.DeprecatedRetryJoinGCE, agent.RetryJoinGCE{}) {
m := discover.Config{
"provider": "gce",
"project_name": cfg.DeprecatedRetryJoinGCE.ProjectName,
"zone_pattern": cfg.DeprecatedRetryJoinGCE.ZonePattern,
"tag_value": cfg.DeprecatedRetryJoinGCE.TagValue,
"credentials_file": cfg.DeprecatedRetryJoinGCE.CredentialsFile,
}
cfg.RetryJoin = append(cfg.RetryJoin, m.String())
cfg.DeprecatedRetryJoinGCE = agent.RetryJoinGCE{}
// redact m before output
if m["credentials_file"] != "" {
m["credentials_file"] = "hidden"
}
cmd.UI.Warn(fmt.Sprintf("==> DEPRECATION: retry_join_gce is deprecated. "+
"Please add %q to retry_join\n", m))
}
// Compile all the watches
for _, params := range cfg.Watches {
// Parse the watches, excluding the handler
wp, err := watch.ParseExempt(params, []string{"handler"})
if err != nil {
cmd.UI.Error(fmt.Sprintf("Failed to parse watch (%#v): %v", params, err))
return nil
}
// Get the handler
h := wp.Exempt["handler"]
if _, ok := h.(string); h == nil || !ok {
cmd.UI.Error("Watch handler must be a string")
return nil
}
// Store the watch plan
cfg.WatchPlans = append(cfg.WatchPlans, wp)
}
// Warn if we are in expect mode
if cfg.BootstrapExpect == 1 {
cmd.UI.Error("WARNING: BootstrapExpect Mode is specified as 1; this is the same as Bootstrap mode.")
cfg.BootstrapExpect = 0
cfg.Bootstrap = true
} else if cfg.BootstrapExpect > 0 {
cmd.UI.Error(fmt.Sprintf("WARNING: Expect Mode enabled, expecting %d servers", cfg.BootstrapExpect))
}
// Warn if we are expecting an even number of servers
if cfg.BootstrapExpect != 0 && cfg.BootstrapExpect%2 == 0 {
if cfg.BootstrapExpect == 2 {
cmd.UI.Error("WARNING: A cluster with 2 servers will provide no failure tolerance. See https://www.consul.io/docs/internals/consensus.html#deployment-table")
} else {
cmd.UI.Error("WARNING: A cluster with an even number of servers does not achieve optimum fault tolerance. See https://www.consul.io/docs/internals/consensus.html#deployment-table")
}
}
// Warn if we are in bootstrap mode
if cfg.Bootstrap {
cmd.UI.Error("WARNING: Bootstrap mode enabled! Do not enable unless necessary")
}
// It doesn't make sense to include both UI options.
if cfg.EnableUI == true && cfg.UIDir != "" {
cmd.UI.Error("Both the ui and ui-dir flags were specified, please provide only one")
cmd.UI.Error("If trying to use your own web UI resources, use the ui-dir flag")
cmd.UI.Error("If using Consul version 0.7.0 or later, the web UI is included in the binary so use ui to enable it")
return nil
}
// Set the version info
cfg.Revision = cmd.Revision
cfg.Version = cmd.Version
cfg.VersionPrerelease = cmd.VersionPrerelease
if err := cfg.ResolveTmplAddrs(); err != nil {
cmd.UI.Error(fmt.Sprintf("Failed to parse config: %v", err))
return nil
}
if err := cfg.SetupTaggedAndAdvertiseAddrs(); err != nil {
cmd.UI.Error(fmt.Sprintf("Failed to set up tagged and advertise addresses: %v", err))
return nil
}
return cfg
return &cfg
}
// checkpointResults is used to handler periodic results from our update checker
@ -568,7 +93,7 @@ func (cmd *AgentCommand) checkpointResults(results *checkpoint.CheckResponse, er
}
}
func (cmd *AgentCommand) startupUpdateCheck(config *agent.Config) {
func (cmd *AgentCommand) startupUpdateCheck(config *config.RuntimeConfig) {
version := config.Version
if config.VersionPrerelease != "" {
version += fmt.Sprintf("-%s", config.VersionPrerelease)
@ -592,13 +117,13 @@ func (cmd *AgentCommand) startupUpdateCheck(config *agent.Config) {
}
// startupJoin is invoked to handle any joins specified to take place at start time
func (cmd *AgentCommand) startupJoin(agent *agent.Agent, cfg *agent.Config) error {
if len(cfg.StartJoin) == 0 {
func (cmd *AgentCommand) startupJoin(agent *agent.Agent, cfg *config.RuntimeConfig) error {
if len(cfg.StartJoinAddrsLAN) == 0 {
return nil
}
cmd.UI.Output("Joining cluster...")
n, err := agent.JoinLAN(cfg.StartJoin)
n, err := agent.JoinLAN(cfg.StartJoinAddrsLAN)
if err != nil {
return err
}
@ -608,13 +133,13 @@ func (cmd *AgentCommand) startupJoin(agent *agent.Agent, cfg *agent.Config) erro
}
// startupJoinWan is invoked to handle any joins -wan specified to take place at start time
func (cmd *AgentCommand) startupJoinWan(agent *agent.Agent, cfg *agent.Config) error {
if len(cfg.StartJoinWan) == 0 {
func (cmd *AgentCommand) startupJoinWan(agent *agent.Agent, cfg *config.RuntimeConfig) error {
if len(cfg.StartJoinAddrsWAN) == 0 {
return nil
}
cmd.UI.Output("Joining -wan cluster...")
n, err := agent.JoinWAN(cfg.StartJoinWan)
n, err := agent.JoinWAN(cfg.StartJoinAddrsWAN)
if err != nil {
return err
}
@ -623,51 +148,51 @@ func (cmd *AgentCommand) startupJoinWan(agent *agent.Agent, cfg *agent.Config) e
return nil
}
func statsiteSink(config *agent.Config, hostname string) (metrics.MetricSink, error) {
if config.Telemetry.StatsiteAddr == "" {
func statsiteSink(config *config.RuntimeConfig, hostname string) (metrics.MetricSink, error) {
if config.TelemetryStatsiteAddr == "" {
return nil, nil
}
return metrics.NewStatsiteSink(config.Telemetry.StatsiteAddr)
return metrics.NewStatsiteSink(config.TelemetryStatsiteAddr)
}
func statsdSink(config *agent.Config, hostname string) (metrics.MetricSink, error) {
if config.Telemetry.StatsdAddr == "" {
func statsdSink(config *config.RuntimeConfig, hostname string) (metrics.MetricSink, error) {
if config.TelemetryStatsdAddr == "" {
return nil, nil
}
return metrics.NewStatsdSink(config.Telemetry.StatsdAddr)
return metrics.NewStatsdSink(config.TelemetryStatsdAddr)
}
func dogstatdSink(config *agent.Config, hostname string) (metrics.MetricSink, error) {
if config.Telemetry.DogStatsdAddr == "" {
func dogstatdSink(config *config.RuntimeConfig, hostname string) (metrics.MetricSink, error) {
if config.TelemetryDogstatsdAddr == "" {
return nil, nil
}
sink, err := datadog.NewDogStatsdSink(config.Telemetry.DogStatsdAddr, hostname)
sink, err := datadog.NewDogStatsdSink(config.TelemetryDogstatsdAddr, hostname)
if err != nil {
return nil, err
}
sink.SetTags(config.Telemetry.DogStatsdTags)
sink.SetTags(config.TelemetryDogstatsdTags)
return sink, nil
}
func circonusSink(config *agent.Config, hostname string) (metrics.MetricSink, error) {
if config.Telemetry.CirconusAPIToken == "" && config.Telemetry.CirconusCheckSubmissionURL == "" {
func circonusSink(config *config.RuntimeConfig, hostname string) (metrics.MetricSink, error) {
if config.TelemetryCirconusAPIToken == "" && config.TelemetryCirconusSubmissionURL == "" {
return nil, nil
}
cfg := &circonus.Config{}
cfg.Interval = config.Telemetry.CirconusSubmissionInterval
cfg.CheckManager.API.TokenKey = config.Telemetry.CirconusAPIToken
cfg.CheckManager.API.TokenApp = config.Telemetry.CirconusAPIApp
cfg.CheckManager.API.URL = config.Telemetry.CirconusAPIURL
cfg.CheckManager.Check.SubmissionURL = config.Telemetry.CirconusCheckSubmissionURL
cfg.CheckManager.Check.ID = config.Telemetry.CirconusCheckID
cfg.CheckManager.Check.ForceMetricActivation = config.Telemetry.CirconusCheckForceMetricActivation
cfg.CheckManager.Check.InstanceID = config.Telemetry.CirconusCheckInstanceID
cfg.CheckManager.Check.SearchTag = config.Telemetry.CirconusCheckSearchTag
cfg.CheckManager.Check.DisplayName = config.Telemetry.CirconusCheckDisplayName
cfg.CheckManager.Check.Tags = config.Telemetry.CirconusCheckTags
cfg.CheckManager.Broker.ID = config.Telemetry.CirconusBrokerID
cfg.CheckManager.Broker.SelectTag = config.Telemetry.CirconusBrokerSelectTag
cfg.Interval = config.TelemetryCirconusSubmissionInterval
cfg.CheckManager.API.TokenKey = config.TelemetryCirconusAPIToken
cfg.CheckManager.API.TokenApp = config.TelemetryCirconusAPIApp
cfg.CheckManager.API.URL = config.TelemetryCirconusAPIURL
cfg.CheckManager.Check.SubmissionURL = config.TelemetryCirconusSubmissionURL
cfg.CheckManager.Check.ID = config.TelemetryCirconusCheckID
cfg.CheckManager.Check.ForceMetricActivation = config.TelemetryCirconusCheckForceMetricActivation
cfg.CheckManager.Check.InstanceID = config.TelemetryCirconusCheckInstanceID
cfg.CheckManager.Check.SearchTag = config.TelemetryCirconusCheckSearchTag
cfg.CheckManager.Check.DisplayName = config.TelemetryCirconusCheckDisplayName
cfg.CheckManager.Check.Tags = config.TelemetryCirconusCheckTags
cfg.CheckManager.Broker.ID = config.TelemetryCirconusBrokerID
cfg.CheckManager.Broker.SelectTag = config.TelemetryCirconusBrokerSelectTag
if cfg.CheckManager.Check.DisplayName == "" {
cfg.CheckManager.Check.DisplayName = "Consul"
@ -689,19 +214,19 @@ func circonusSink(config *agent.Config, hostname string) (metrics.MetricSink, er
return sink, nil
}
func startupTelemetry(config *agent.Config) (*metrics.InmemSink, error) {
func startupTelemetry(conf *config.RuntimeConfig) (*metrics.InmemSink, error) {
// Setup telemetry
// Aggregate on 10 second intervals for 1 minute. Expose the
// metrics over stderr when there is a SIGUSR1 received.
memSink := metrics.NewInmemSink(10*time.Second, time.Minute)
metrics.DefaultInmemSignal(memSink)
metricsConf := metrics.DefaultConfig(config.Telemetry.StatsitePrefix)
metricsConf.EnableHostname = !config.Telemetry.DisableHostname
metricsConf.FilterDefault = *config.Telemetry.FilterDefault
metricsConf := metrics.DefaultConfig(conf.TelemetryStatsitePrefix)
metricsConf.EnableHostname = !conf.TelemetryDisableHostname
metricsConf.FilterDefault = conf.TelemetryFilterDefault
var sinks metrics.FanoutSink
addSink := func(name string, fn func(*agent.Config, string) (metrics.MetricSink, error)) error {
s, err := fn(config, metricsConf.HostName)
addSink := func(name string, fn func(*config.RuntimeConfig, string) (metrics.MetricSink, error)) error {
s, err := fn(conf, metricsConf.HostName)
if err != nil {
return err
}
@ -737,7 +262,7 @@ func startupTelemetry(config *agent.Config) (*metrics.InmemSink, error) {
func (cmd *AgentCommand) Run(args []string) int {
code := cmd.run(args)
if cmd.logger != nil {
cmd.logger.Println("[INFO] Exit code: ", code)
cmd.logger.Println("[INFO] Exit code:", code)
}
return code
}
@ -814,8 +339,8 @@ func (cmd *AgentCommand) run(args []string) int {
// Let the agent know we've finished registration
agent.StartSync()
segment := config.Segment
if config.Server {
segment := config.SegmentName
if config.ServerMode {
segment = "<all>"
}
@ -824,11 +349,11 @@ func (cmd *AgentCommand) run(args []string) int {
cmd.UI.Info(fmt.Sprintf(" Node ID: '%s'", config.NodeID))
cmd.UI.Info(fmt.Sprintf(" Node name: '%s'", config.NodeName))
cmd.UI.Info(fmt.Sprintf(" Datacenter: '%s' (Segment: '%s')", config.Datacenter, segment))
cmd.UI.Info(fmt.Sprintf(" Server: %v (Bootstrap: %v)", config.Server, config.Bootstrap))
cmd.UI.Info(fmt.Sprintf(" Client Addr: %v (HTTP: %d, HTTPS: %d, DNS: %d)", config.ClientAddr,
config.Ports.HTTP, config.Ports.HTTPS, config.Ports.DNS))
cmd.UI.Info(fmt.Sprintf(" Cluster Addr: %v (LAN: %d, WAN: %d)", config.AdvertiseAddr,
config.Ports.SerfLan, config.Ports.SerfWan))
cmd.UI.Info(fmt.Sprintf(" Server: %v (Bootstrap: %v)", config.ServerMode, config.Bootstrap))
cmd.UI.Info(fmt.Sprintf(" Client Addr: %v (HTTP: %d, HTTPS: %d, DNS: %d)", config.ClientAddrs,
config.HTTPPort, config.HTTPSPort, config.DNSPort))
cmd.UI.Info(fmt.Sprintf(" Cluster Addr: %v (LAN: %d, WAN: %d)", config.AdvertiseAddrLAN,
config.SerfPortLAN, config.SerfPortWAN))
cmd.UI.Info(fmt.Sprintf(" Encrypt: Gossip: %v, TLS-Outgoing: %v, TLS-Incoming: %v",
agent.GossipEncrypted(), config.VerifyOutgoing, config.VerifyIncoming))
@ -883,7 +408,7 @@ func (cmd *AgentCommand) run(args []string) int {
default:
cmd.logger.Println("[INFO] Caught signal: ", sig)
graceful := (sig == os.Interrupt && !(*config.SkipLeaveOnInt)) || (sig == syscall.SIGTERM && (*config.LeaveOnTerm))
graceful := (sig == os.Interrupt && !(config.SkipLeaveOnInt)) || (sig == syscall.SIGTERM && (config.LeaveOnTerm))
if !graceful {
cmd.logger.Println("[INFO] Graceful shutdown disabled. Exiting")
return 1
@ -916,7 +441,7 @@ func (cmd *AgentCommand) run(args []string) int {
}
// handleReload is invoked when we should reload our configs, e.g. SIGHUP
func (cmd *AgentCommand) handleReload(agent *agent.Agent, cfg *agent.Config) (*agent.Config, error) {
func (cmd *AgentCommand) handleReload(agent *agent.Agent, cfg *config.RuntimeConfig) (*config.RuntimeConfig, error) {
cmd.logger.Println("[INFO] Reloading configuration...")
var errs error
newCfg := cmd.readConfig()

View File

@ -6,7 +6,6 @@ import (
"os"
"os/exec"
"path/filepath"
"reflect"
"strings"
"testing"
@ -56,33 +55,45 @@ func TestValidDatacenter(t *testing.T) {
// TestConfigFail should test command line flags that lead to an immediate error.
func TestConfigFail(t *testing.T) {
t.Parallel()
dataDir := testutil.TempDir(t, "consul")
defer os.RemoveAll(dataDir)
tests := []struct {
args []string
out string
}{
{
args: []string{"agent", "-server", "-data-dir", "foo", "-advertise", "0.0.0.0"},
out: "==> Advertise address cannot be 0.0.0.0\n",
args: []string{"agent", "-server", "-bind=10.0.0.1", "-datacenter="},
out: "==> datacenter cannot be empty\n",
},
{
args: []string{"agent", "-server", "-data-dir", "foo", "-advertise", "::"},
out: "==> Advertise address cannot be ::\n",
args: []string{"agent", "-server", "-bind=10.0.0.1"},
out: "==> data_dir cannot be empty\n",
},
{
args: []string{"agent", "-server", "-data-dir", "foo", "-advertise", "[::]"},
out: "==> Advertise address cannot be [::]\n",
args: []string{"agent", "-server", "-data-dir", dataDir, "-advertise", "0.0.0.0", "-bind", "10.0.0.1"},
out: "==> Advertise address cannot be 0.0.0.0, :: or [::]\n",
},
{
args: []string{"agent", "-server", "-data-dir", "foo", "-advertise-wan", "0.0.0.0"},
out: "==> Advertise WAN address cannot be 0.0.0.0\n",
args: []string{"agent", "-server", "-data-dir", dataDir, "-advertise", "::", "-bind", "10.0.0.1"},
out: "==> Advertise address cannot be 0.0.0.0, :: or [::]\n",
},
{
args: []string{"agent", "-server", "-data-dir", "foo", "-advertise-wan", "::"},
out: "==> Advertise WAN address cannot be ::\n",
args: []string{"agent", "-server", "-data-dir", dataDir, "-advertise", "[::]", "-bind", "10.0.0.1"},
out: "==> Advertise address cannot be 0.0.0.0, :: or [::]\n",
},
{
args: []string{"agent", "-server", "-data-dir", "foo", "-advertise-wan", "[::]"},
out: "==> Advertise WAN address cannot be [::]\n",
args: []string{"agent", "-server", "-data-dir", dataDir, "-advertise-wan", "0.0.0.0", "-bind", "10.0.0.1"},
out: "==> Advertise WAN address cannot be 0.0.0.0, :: or [::]\n",
},
{
args: []string{"agent", "-server", "-data-dir", dataDir, "-advertise-wan", "::", "-bind", "10.0.0.1"},
out: "==> Advertise WAN address cannot be 0.0.0.0, :: or [::]\n",
},
{
args: []string{"agent", "-server", "-data-dir", dataDir, "-advertise-wan", "[::]", "-bind", "10.0.0.1"},
out: "==> Advertise WAN address cannot be 0.0.0.0, :: or [::]\n",
},
}
@ -103,7 +114,7 @@ func TestConfigFail(t *testing.T) {
func TestRetryJoin(t *testing.T) {
t.Skip("fs: skipping tests that use cmd.Run until signal handling is fixed")
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), "")
defer a.Shutdown()
cfg2 := agent.TestConfig()
@ -124,25 +135,15 @@ func TestRetryJoin(t *testing.T) {
BaseCommand: baseCommand(cli.NewMockUi()),
}
serfAddr := fmt.Sprintf(
"%s:%d",
a.Config.BindAddr,
a.Config.Ports.SerfLan)
serfWanAddr := fmt.Sprintf(
"%s:%d",
a.Config.BindAddr,
a.Config.Ports.SerfWan)
args := []string{
"-server",
"-bind", a.Config.BindAddr,
"-bind", a.Config.BindAddr.String(),
"-data-dir", tmpDir,
"-node", fmt.Sprintf(`"%s"`, cfg2.NodeName),
"-advertise", a.Config.BindAddr,
"-retry-join", serfAddr,
"-advertise", a.Config.BindAddr.String(),
"-retry-join", a.Config.SerfBindAddrLAN.String(),
"-retry-interval", "1s",
"-retry-join-wan", serfWanAddr,
"-retry-join-wan", a.Config.SerfBindAddrWAN.String(),
"-retry-interval-wan", "1s",
}
@ -162,181 +163,6 @@ func TestRetryJoin(t *testing.T) {
})
}
func TestReadCliConfig(t *testing.T) {
t.Parallel()
tmpDir := testutil.TempDir(t, "consul")
defer os.RemoveAll(tmpDir)
shutdownCh := make(chan struct{})
defer close(shutdownCh)
// Test config parse
{
cmd := &AgentCommand{
args: []string{
"-data-dir", tmpDir,
"-node", `"a"`,
"-bind", "1.2.3.4",
"-advertise-wan", "1.2.3.4",
"-serf-wan-bind", "4.3.2.1",
"-serf-lan-bind", "4.3.2.2",
"-node-meta", "somekey:somevalue",
},
ShutdownCh: shutdownCh,
BaseCommand: baseCommand(cli.NewMockUi()),
}
config := cmd.readConfig()
if config.AdvertiseAddrWan != "1.2.3.4" {
t.Fatalf("expected -advertise-addr-wan 1.2.3.4 got %s", config.AdvertiseAddrWan)
}
if config.SerfWanBindAddr != "4.3.2.1" {
t.Fatalf("expected -serf-wan-bind 4.3.2.1 got %s", config.SerfWanBindAddr)
}
if config.SerfLanBindAddr != "4.3.2.2" {
t.Fatalf("expected -serf-lan-bind 4.3.2.2 got %s", config.SerfLanBindAddr)
}
expected := map[string]string{
"somekey": "somevalue",
}
if !reflect.DeepEqual(config.Meta, expected) {
t.Fatalf("bad: %v %v", config.Meta, expected)
}
}
// Test multiple node meta flags
{
cmd := &AgentCommand{
args: []string{
"-data-dir", tmpDir,
"-node-meta", "somekey:somevalue",
"-node-meta", "otherkey:othervalue",
"-bind", "1.2.3.4",
},
ShutdownCh: shutdownCh,
BaseCommand: baseCommand(cli.NewMockUi()),
}
config := cmd.readConfig()
expected := map[string]string{
"somekey": "somevalue",
"otherkey": "othervalue",
}
if !reflect.DeepEqual(config.Meta, expected) {
t.Fatalf("bad: %v %v", config.Meta, expected)
}
}
// Test LeaveOnTerm and SkipLeaveOnInt defaults for server mode
{
ui := cli.NewMockUi()
cmd := &AgentCommand{
args: []string{
"-node", `"server1"`,
"-server",
"-data-dir", tmpDir,
"-bind", "1.2.3.4",
},
ShutdownCh: shutdownCh,
BaseCommand: baseCommand(ui),
}
config := cmd.readConfig()
if config == nil {
t.Fatalf(`Expected non-nil config object: %s`, ui.ErrorWriter.String())
}
if config.Server != true {
t.Errorf(`Expected -server to be true`)
}
if (*config.LeaveOnTerm) != false {
t.Errorf(`Expected LeaveOnTerm to be false in server mode`)
}
if (*config.SkipLeaveOnInt) != true {
t.Errorf(`Expected SkipLeaveOnInt to be true in server mode`)
}
}
// Test LeaveOnTerm and SkipLeaveOnInt defaults for client mode
{
ui := cli.NewMockUi()
cmd := &AgentCommand{
args: []string{
"-data-dir", tmpDir,
"-node", `"client"`,
"-bind", "1.2.3.4",
},
ShutdownCh: shutdownCh,
BaseCommand: baseCommand(ui),
}
config := cmd.readConfig()
if config == nil {
t.Fatalf(`Expected non-nil config object: %s`, ui.ErrorWriter.String())
}
if config.Server != false {
t.Errorf(`Expected server to be false`)
}
if (*config.LeaveOnTerm) != true {
t.Errorf(`Expected LeaveOnTerm to be true in client mode`)
}
if *config.SkipLeaveOnInt != false {
t.Errorf(`Expected SkipLeaveOnInt to be false in client mode`)
}
}
// Test empty node name
{
cmd := &AgentCommand{
args: []string{"-node", `""`},
ShutdownCh: shutdownCh,
BaseCommand: baseCommand(cli.NewMockUi()),
}
config := cmd.readConfig()
if config != nil {
t.Errorf(`Expected -node="" to fail`)
}
}
}
func TestAgent_HostBasedIDs(t *testing.T) {
t.Parallel()
tmpDir := testutil.TempDir(t, "consul")
defer os.RemoveAll(tmpDir)
// Host-based IDs are disabled by default.
{
cmd := &AgentCommand{
args: []string{
"-data-dir", tmpDir,
"-bind", "127.0.0.1",
},
BaseCommand: baseCommand(cli.NewMockUi()),
}
config := cmd.readConfig()
if *config.DisableHostNodeID != true {
t.Fatalf("expected host-based node IDs to be disabled")
}
}
// Try enabling host-based IDs.
{
cmd := &AgentCommand{
args: []string{
"-data-dir", tmpDir,
"-disable-host-node-id=false",
"-bind", "127.0.0.1",
},
BaseCommand: baseCommand(cli.NewMockUi()),
}
config := cmd.readConfig()
if *config.DisableHostNodeID != false {
t.Fatalf("expected host-based node IDs to be enabled")
}
}
}
func TestRetryJoinFail(t *testing.T) {
t.Skip("fs: skipping tests that use cmd.Run until signal handling is fixed")
t.Parallel()
@ -352,12 +178,10 @@ func TestRetryJoinFail(t *testing.T) {
BaseCommand: baseCommand(cli.NewMockUi()),
}
serfAddr := fmt.Sprintf("%s:%d", cfg.BindAddr, cfg.Ports.SerfLan)
args := []string{
"-bind", cfg.BindAddr,
"-bind", cfg.BindAddr.String(),
"-data-dir", tmpDir,
"-retry-join", serfAddr,
"-retry-join", cfg.SerfBindAddrLAN.String(),
"-retry-max", "1",
"-retry-interval", "10ms",
}
@ -382,13 +206,11 @@ func TestRetryJoinWanFail(t *testing.T) {
BaseCommand: baseCommand(cli.NewMockUi()),
}
serfAddr := fmt.Sprintf("%s:%d", cfg.BindAddr, cfg.Ports.SerfWan)
args := []string{
"-server",
"-bind", cfg.BindAddr,
"-bind", cfg.BindAddr.String(),
"-data-dir", tmpDir,
"-retry-join-wan", serfAddr,
"-retry-join-wan", cfg.SerfBindAddrWAN.String(),
"-retry-max-wan", "1",
"-retry-interval-wan", "10ms",
}
@ -410,7 +232,7 @@ func TestProtectDataDir(t *testing.T) {
cfgFile := testutil.TempFile(t, "consul")
defer os.Remove(cfgFile.Name())
content := fmt.Sprintf(`{"server": true, "data_dir": "%s"}`, dir)
content := fmt.Sprintf(`{"server": true, "bind_addr" : "10.0.0.1", "data_dir": "%s"}`, dir)
_, err := cfgFile.Write([]byte(content))
if err != nil {
t.Fatalf("err: %v", err)
@ -443,7 +265,7 @@ func TestBadDataDirPermissions(t *testing.T) {
ui := cli.NewMockUi()
cmd := &AgentCommand{
BaseCommand: baseCommand(ui),
args: []string{"-data-dir=" + dataDir, "-server=true"},
args: []string{"-data-dir=" + dataDir, "-server=true", "-bind=10.0.0.1"},
}
if conf := cmd.readConfig(); conf != nil {
t.Fatalf("Should fail with bad data directory permissions")

View File

@ -60,7 +60,7 @@ func TestCatalogListDatacentersCommand_Validation(t *testing.T) {
func TestCatalogListDatacentersCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testCatalogListDatacentersCommand(t)

View File

@ -60,7 +60,7 @@ func TestCatalogListNodesCommand_Validation(t *testing.T) {
func TestCatalogListNodesCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
t.Run("simple", func(t *testing.T) {

View File

@ -61,7 +61,7 @@ func TestCatalogListServicesCommand_Validation(t *testing.T) {
func TestCatalogListServicesCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
// Add another service with tags for testing

View File

@ -4,7 +4,7 @@ import (
"fmt"
"strings"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/configutil"
)
@ -52,11 +52,15 @@ func (c *ConfigTestCommand) Run(args []string) int {
return 1
}
_, err := agent.ReadConfigPaths(configFiles)
b, err := config.NewBuilder(config.Flags{ConfigFiles: configFiles})
if err != nil {
c.UI.Error(fmt.Sprintf("Config validation failed: %v", err.Error()))
return 1
}
if _, err := b.BuildAndValidate(); err != nil {
c.UI.Error(fmt.Sprintf("Config validation failed: %v", err.Error()))
return 1
}
return 0
}

View File

@ -41,29 +41,13 @@ func TestConfigTestCommandFailOnEmptyFile(t *testing.T) {
}
}
func TestConfigTestCommandSucceedOnEmptyDir(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
ui, cmd := testConfigTestCommand(t)
args := []string{
"-config-dir", td,
}
if code := cmd.Run(args); code != 0 {
t.Fatalf("bad: %d, %s", code, ui.ErrorWriter.String())
}
}
func TestConfigTestCommandSucceedOnMinimalConfigFile(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
fp := filepath.Join(td, "config.json")
err := ioutil.WriteFile(fp, []byte(`{}`), 0644)
err := ioutil.WriteFile(fp, []byte(`{"bind_addr":"10.0.0.1", "data_dir": "`+td+`"}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}
@ -84,28 +68,7 @@ func TestConfigTestCommandSucceedOnMinimalConfigDir(t *testing.T) {
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
err := ioutil.WriteFile(filepath.Join(td, "config.json"), []byte(`{}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}
_, cmd := testConfigTestCommand(t)
args := []string{
"-config-dir", td,
}
if code := cmd.Run(args); code != 0 {
t.Fatalf("bad: %d", code)
}
}
func TestConfigTestCommandSucceedOnConfigDirWithEmptyFile(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
err := ioutil.WriteFile(filepath.Join(td, "config.json"), []byte{}, 0644)
err := ioutil.WriteFile(filepath.Join(td, "config.json"), []byte(`{"bind_addr":"10.0.0.1", "data_dir": "`+td+`"}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}

View File

@ -15,7 +15,7 @@ func TestEventCommand_implements(t *testing.T) {
func TestEventCommandRun(t *testing.T) {
t.Parallel()
a1 := agent.NewTestAgent(t.Name(), nil)
a1 := agent.NewTestAgent(t.Name(), ``)
defer a1.Shutdown()
ui := cli.NewMockUi()

View File

@ -1,7 +1,6 @@
package command
import (
"fmt"
"strings"
"testing"
"time"
@ -29,9 +28,9 @@ func TestExecCommand_implements(t *testing.T) {
func TestExecCommandRun(t *testing.T) {
t.Parallel()
cfg := agent.TestConfig()
cfg.DisableRemoteExec = agent.Bool(false)
a := agent.NewTestAgent(t.Name(), cfg)
a := agent.NewTestAgent(t.Name(), `
disable_remote_exec = false
`)
defer a.Shutdown()
ui, c := testExecCommand(t)
@ -49,20 +48,19 @@ func TestExecCommandRun(t *testing.T) {
func TestExecCommandRun_CrossDC(t *testing.T) {
t.Parallel()
cfg1 := agent.TestConfig()
cfg1.DisableRemoteExec = agent.Bool(false)
a1 := agent.NewTestAgent(t.Name(), cfg1)
a1 := agent.NewTestAgent(t.Name(), `
disable_remote_exec = false
`)
defer a1.Shutdown()
cfg2 := agent.TestConfig()
cfg2.Datacenter = "dc2"
cfg2.DisableRemoteExec = agent.Bool(false)
a2 := agent.NewTestAgent(t.Name(), cfg2)
defer a1.Shutdown()
a2 := agent.NewTestAgent(t.Name(), `
datacenter = "dc2"
disable_remote_exec = false
`)
defer a2.Shutdown()
// Join over the WAN
wanAddr := fmt.Sprintf("%s:%d", a1.Config.BindAddr, a1.Config.Ports.SerfWan)
_, err := a2.JoinWAN([]string{wanAddr})
_, err := a2.JoinWAN([]string{a1.Config.SerfBindAddrWAN.String()})
if err != nil {
t.Fatalf("err: %v", err)
}
@ -127,9 +125,9 @@ func TestExecCommand_Validate(t *testing.T) {
func TestExecCommand_Sessions(t *testing.T) {
t.Parallel()
cfg := agent.TestConfig()
cfg.DisableRemoteExec = agent.Bool(false)
a := agent.NewTestAgent(t.Name(), cfg)
a := agent.NewTestAgent(t.Name(), `
disable_remote_exec = false
`)
defer a.Shutdown()
client := a.Client()
@ -166,9 +164,9 @@ func TestExecCommand_Sessions(t *testing.T) {
func TestExecCommand_Sessions_Foreign(t *testing.T) {
t.Parallel()
cfg := agent.TestConfig()
cfg.DisableRemoteExec = agent.Bool(false)
a := agent.NewTestAgent(t.Name(), cfg)
a := agent.NewTestAgent(t.Name(), `
disable_remote_exec = false
`)
defer a.Shutdown()
client := a.Client()
@ -216,9 +214,9 @@ func TestExecCommand_Sessions_Foreign(t *testing.T) {
func TestExecCommand_UploadDestroy(t *testing.T) {
t.Parallel()
cfg := agent.TestConfig()
cfg.DisableRemoteExec = agent.Bool(false)
a := agent.NewTestAgent(t.Name(), cfg)
a := agent.NewTestAgent(t.Name(), `
disable_remote_exec = false
`)
defer a.Shutdown()
client := a.Client()
@ -271,9 +269,9 @@ func TestExecCommand_UploadDestroy(t *testing.T) {
func TestExecCommand_StreamResults(t *testing.T) {
t.Parallel()
cfg := agent.TestConfig()
cfg.DisableRemoteExec = agent.Bool(false)
a := agent.NewTestAgent(t.Name(), cfg)
a := agent.NewTestAgent(t.Name(), `
disable_remote_exec = false
`)
defer a.Shutdown()
client := a.Client()

View File

@ -1,7 +1,6 @@
package command
import (
"fmt"
"strings"
"testing"
@ -28,13 +27,12 @@ func TestForceLeaveCommand_implements(t *testing.T) {
func TestForceLeaveCommandRun(t *testing.T) {
t.Parallel()
a1 := agent.NewTestAgent(t.Name(), nil)
a2 := agent.NewTestAgent(t.Name(), nil)
a1 := agent.NewTestAgent(t.Name(), ``)
a2 := agent.NewTestAgent(t.Name(), ``)
defer a1.Shutdown()
defer a2.Shutdown()
addr := fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan)
_, err := a1.JoinLAN([]string{addr})
_, err := a2.JoinLAN([]string{a1.Config.SerfBindAddrLAN.String()})
if err != nil {
t.Fatalf("err: %s", err)
}

View File

@ -15,7 +15,7 @@ func TestInfoCommand_implements(t *testing.T) {
func TestInfoCommandRun(t *testing.T) {
t.Parallel()
a1 := agent.NewTestAgent(t.Name(), nil)
a1 := agent.NewTestAgent(t.Name(), ``)
defer a1.Shutdown()
ui := cli.NewMockUi()

View File

@ -1,7 +1,6 @@
package command
import (
"fmt"
"strings"
"testing"
@ -26,15 +25,15 @@ func TestJoinCommand_implements(t *testing.T) {
func TestJoinCommandRun(t *testing.T) {
t.Parallel()
a1 := agent.NewTestAgent(t.Name(), nil)
a2 := agent.NewTestAgent(t.Name(), nil)
a1 := agent.NewTestAgent(t.Name(), ``)
a2 := agent.NewTestAgent(t.Name(), ``)
defer a1.Shutdown()
defer a2.Shutdown()
ui, c := testJoinCommand(t)
args := []string{
"-http-addr=" + a1.HTTPAddr(),
fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfLan),
a2.Config.SerfBindAddrLAN.String(),
}
code := c.Run(args)
@ -49,8 +48,8 @@ func TestJoinCommandRun(t *testing.T) {
func TestJoinCommandRun_wan(t *testing.T) {
t.Parallel()
a1 := agent.NewTestAgent(t.Name(), nil)
a2 := agent.NewTestAgent(t.Name(), nil)
a1 := agent.NewTestAgent(t.Name(), ``)
a2 := agent.NewTestAgent(t.Name(), ``)
defer a1.Shutdown()
defer a2.Shutdown()
@ -58,7 +57,7 @@ func TestJoinCommandRun_wan(t *testing.T) {
args := []string{
"-http-addr=" + a1.HTTPAddr(),
"-wan",
fmt.Sprintf("127.0.0.1:%d", a2.Config.Ports.SerfWan),
a2.Config.SerfBindAddrWAN.String(),
}
code := c.Run(args)

View File

@ -29,9 +29,9 @@ func TestKeyringCommandRun(t *testing.T) {
key2 := "kZyFABeAmc64UMTrm9XuKA=="
// Begin with a single key
cfg := agent.TestConfig()
cfg.EncryptKey = key1
a1 := agent.NewTestAgent(t.Name(), cfg)
a1 := agent.NewTestAgent(t.Name(), `
encrypt = "`+key1+`"
`)
defer a1.Shutdown()
// The LAN and WAN keyrings were initialized with key1

View File

@ -83,7 +83,7 @@ func TestKVDeleteCommand_Validation(t *testing.T) {
func TestKVDeleteCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -119,7 +119,7 @@ func TestKVDeleteCommand_Run(t *testing.T) {
func TestKVDeleteCommand_Recurse(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -162,7 +162,7 @@ func TestKVDeleteCommand_Recurse(t *testing.T) {
func TestKVDeleteCommand_CAS(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -12,7 +12,7 @@ import (
func TestKVExportCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -71,7 +71,7 @@ func TestKVGetCommand_Validation(t *testing.T) {
func TestKVGetCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -104,7 +104,7 @@ func TestKVGetCommand_Run(t *testing.T) {
func TestKVGetCommand_Missing(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
_, c := testKVGetCommand(t)
@ -122,7 +122,7 @@ func TestKVGetCommand_Missing(t *testing.T) {
func TestKVGetCommand_Empty(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -150,7 +150,7 @@ func TestKVGetCommand_Empty(t *testing.T) {
func TestKVGetCommand_Detailed(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -193,7 +193,7 @@ func TestKVGetCommand_Detailed(t *testing.T) {
func TestKVGetCommand_Keys(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -227,7 +227,7 @@ func TestKVGetCommand_Keys(t *testing.T) {
func TestKVGetCommand_Recurse(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -266,7 +266,7 @@ func TestKVGetCommand_Recurse(t *testing.T) {
func TestKVGetCommand_RecurseBase64(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -306,7 +306,7 @@ func TestKVGetCommand_RecurseBase64(t *testing.T) {
func TestKVGetCommand_DetailedBase64(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -10,7 +10,7 @@ import (
func TestKVImportCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -88,7 +88,7 @@ func TestKVPutCommand_Validation(t *testing.T) {
func TestKVPutCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -116,7 +116,7 @@ func TestKVPutCommand_Run(t *testing.T) {
func TestKVPutCommand_RunEmptyDataQuoted(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -144,7 +144,7 @@ func TestKVPutCommand_RunEmptyDataQuoted(t *testing.T) {
func TestKVPutCommand_RunBase64(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -180,7 +180,7 @@ func TestKVPutCommand_RunBase64(t *testing.T) {
func TestKVPutCommand_File(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -233,7 +233,7 @@ func TestKVPutCommand_FileNoExist(t *testing.T) {
func TestKVPutCommand_Stdin(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -269,7 +269,7 @@ func TestKVPutCommand_Stdin(t *testing.T) {
func TestKVPutCommand_NegativeVal(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -297,7 +297,7 @@ func TestKVPutCommand_NegativeVal(t *testing.T) {
func TestKVPutCommand_Flags(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()
@ -326,7 +326,7 @@ func TestKVPutCommand_Flags(t *testing.T) {
func TestKVPutCommand_CAS(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -25,7 +25,7 @@ func TestLeaveCommand_implements(t *testing.T) {
func TestLeaveCommandRun(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLeaveCommand(t)
@ -43,7 +43,7 @@ func TestLeaveCommandRun(t *testing.T) {
func TestLeaveCommandFailOnNonFlagArgs(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
_, c := testLeaveCommand(t)

View File

@ -48,7 +48,7 @@ func TestLockCommand_BadArgs(t *testing.T) {
func TestLockCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -70,7 +70,7 @@ func TestLockCommand_Run(t *testing.T) {
func TestLockCommand_Try_Lock(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -101,7 +101,7 @@ func TestLockCommand_Try_Lock(t *testing.T) {
func TestLockCommand_Try_Semaphore(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -132,7 +132,7 @@ func TestLockCommand_Try_Semaphore(t *testing.T) {
func TestLockCommand_MonitorRetry_Lock_Default(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -164,7 +164,7 @@ func TestLockCommand_MonitorRetry_Lock_Default(t *testing.T) {
func TestLockCommand_MonitorRetry_Semaphore_Default(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -196,7 +196,7 @@ func TestLockCommand_MonitorRetry_Semaphore_Default(t *testing.T) {
func TestLockCommand_MonitorRetry_Lock_Arg(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -228,7 +228,7 @@ func TestLockCommand_MonitorRetry_Lock_Arg(t *testing.T) {
func TestLockCommand_MonitorRetry_Semaphore_Arg(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testLockCommand(t)
@ -260,7 +260,7 @@ func TestLockCommand_MonitorRetry_Semaphore_Arg(t *testing.T) {
func TestLockCommand_ChildExitCode(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
t.Run("clean exit", func(t *testing.T) {

View File

@ -47,7 +47,7 @@ func TestMaintCommandRun_ConflictingArgs(t *testing.T) {
func TestMaintCommandRun_NoArgs(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
// Register the service and put it into maintenance mode
@ -94,7 +94,7 @@ func TestMaintCommandRun_NoArgs(t *testing.T) {
func TestMaintCommandRun_EnableNodeMaintenance(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMaintCommand(t)
@ -116,7 +116,7 @@ func TestMaintCommandRun_EnableNodeMaintenance(t *testing.T) {
func TestMaintCommandRun_DisableNodeMaintenance(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMaintCommand(t)
@ -137,7 +137,7 @@ func TestMaintCommandRun_DisableNodeMaintenance(t *testing.T) {
func TestMaintCommandRun_EnableServiceMaintenance(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
// Register the service
@ -169,7 +169,7 @@ func TestMaintCommandRun_EnableServiceMaintenance(t *testing.T) {
func TestMaintCommandRun_DisableServiceMaintenance(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
// Register the service
@ -200,7 +200,7 @@ func TestMaintCommandRun_DisableServiceMaintenance(t *testing.T) {
func TestMaintCommandRun_ServiceMaintenance_NoService(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMaintCommand(t)

View File

@ -26,7 +26,7 @@ func TestMembersCommand_implements(t *testing.T) {
func TestMembersCommandRun(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMembersCommand(t)
@ -55,7 +55,7 @@ func TestMembersCommandRun(t *testing.T) {
func TestMembersCommandRun_WAN(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMembersCommand(t)
@ -66,14 +66,14 @@ func TestMembersCommandRun_WAN(t *testing.T) {
t.Fatalf("bad: %d. %#v", code, ui.ErrorWriter.String())
}
if !strings.Contains(ui.OutputWriter.String(), fmt.Sprintf("%d", a.Config.Ports.SerfWan)) {
if !strings.Contains(ui.OutputWriter.String(), fmt.Sprintf("%d", a.Config.SerfPortWAN)) {
t.Fatalf("bad: %#v", ui.OutputWriter.String())
}
}
func TestMembersCommandRun_statusFilter(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMembersCommand(t)
@ -94,7 +94,7 @@ func TestMembersCommandRun_statusFilter(t *testing.T) {
func TestMembersCommandRun_statusFilter_failed(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui, c := testMembersCommand(t)

View File

@ -15,7 +15,7 @@ func TestOperator_Autopilot_Get_Implements(t *testing.T) {
func TestOperator_Autopilot_Get(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui := cli.NewMockUi()

View File

@ -17,7 +17,7 @@ func TestOperator_Autopilot_Set_Implements(t *testing.T) {
func TestOperator_Autopilot_Set(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
ui := cli.NewMockUi()

View File

@ -16,11 +16,11 @@ func TestOperator_Raft_ListPeers_Implements(t *testing.T) {
func TestOperator_Raft_ListPeers(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
expected := fmt.Sprintf("%s 127.0.0.1:%d 127.0.0.1:%d leader true 2",
a.Config.NodeName, a.Config.Ports.Server, a.Config.Ports.Server)
a.Config.NodeName, a.Config.ServerPort, a.Config.ServerPort)
// Test the legacy mode with 'consul operator raft -list-peers'
{

View File

@ -15,7 +15,7 @@ func TestOperator_Raft_RemovePeer_Implements(t *testing.T) {
func TestOperator_Raft_RemovePeer(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
// Test the legacy mode with 'consul operator raft -remove-peer'

View File

@ -15,7 +15,7 @@ func TestReloadCommand_implements(t *testing.T) {
func TestReloadCommandRun(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
// Setup a dummy response to errCh to simulate a successful reload

View File

@ -4,7 +4,6 @@ import (
"fmt"
"strings"
"testing"
"time"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/agent/structs"
@ -55,10 +54,13 @@ func TestRTTCommand_Run_BadArgs(t *testing.T) {
func TestRTTCommand_Run_LAN(t *testing.T) {
t.Parallel()
updatePeriod := 10 * time.Millisecond
cfg := agent.TestConfig()
cfg.ConsulConfig.CoordinateUpdatePeriod = updatePeriod
a := agent.NewTestAgent(t.Name(), cfg)
a := agent.NewTestAgent(t.Name(), `
consul = {
coordinate = {
update_period = "10ms"
}
}
`)
defer a.Shutdown()
// Inject some known coordinates.
@ -157,7 +159,7 @@ func TestRTTCommand_Run_LAN(t *testing.T) {
func TestRTTCommand_Run_WAN(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
node := fmt.Sprintf("%s.%s", a.Config.NodeName, a.Config.Datacenter)

View File

@ -73,7 +73,7 @@ func TestSnapshotInspectCommand_Validation(t *testing.T) {
func TestSnapshotInspectCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -73,7 +73,7 @@ func TestSnapshotRestoreCommand_Validation(t *testing.T) {
func TestSnapshotRestoreCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -72,7 +72,7 @@ func TestSnapshotSaveCommand_Validation(t *testing.T) {
func TestSnapshotSaveCommand_Run(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t.Name(), nil)
a := agent.NewTestAgent(t.Name(), ``)
defer a.Shutdown()
client := a.Client()

View File

@ -4,7 +4,7 @@ import (
"fmt"
"strings"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/agent/config"
"github.com/hashicorp/consul/configutil"
)
@ -46,6 +46,7 @@ func (c *ValidateCommand) Run(args []string) int {
c.BaseCommand.HideFlags("config-file", "config-dir")
if err := c.BaseCommand.Parse(args); err != nil {
c.UI.Error(err.Error())
return 1
}
@ -58,11 +59,15 @@ func (c *ValidateCommand) Run(args []string) int {
return 1
}
_, err := agent.ReadConfigPaths(configFiles)
b, err := config.NewBuilder(config.Flags{ConfigFiles: configFiles})
if err != nil {
c.UI.Error(fmt.Sprintf("Config validation failed: %v", err.Error()))
return 1
}
if _, err := b.BuildAndValidate(); err != nil {
c.UI.Error(fmt.Sprintf("Config validation failed: %v", err.Error()))
return 1
}
if !quiet {
c.UI.Output("Configuration is valid!")

View File

@ -39,27 +39,12 @@ func TestValidateCommandFailOnEmptyFile(t *testing.T) {
}
}
func TestValidateCommandSucceedOnEmptyDir(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
ui, cmd := testValidateCommand(t)
args := []string{td}
if code := cmd.Run(args); code != 0 {
t.Fatalf("bad: %d, %s", code, ui.ErrorWriter.String())
}
}
func TestValidateCommandSucceedOnMinimalConfigFile(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
fp := filepath.Join(td, "config.json")
err := ioutil.WriteFile(fp, []byte(`{}`), 0644)
err := ioutil.WriteFile(fp, []byte(`{"bind_addr":"10.0.0.1", "data_dir":"`+td+`"}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}
@ -74,30 +59,10 @@ func TestValidateCommandSucceedOnMinimalConfigFile(t *testing.T) {
}
func TestValidateCommandSucceedOnMinimalConfigDir(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
err := ioutil.WriteFile(filepath.Join(td, "config.json"), []byte(`{}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}
_, cmd := testValidateCommand(t)
args := []string{td}
if code := cmd.Run(args); code != 0 {
t.Fatalf("bad: %d", code)
}
}
func TestValidateCommandSucceedOnConfigDirWithEmptyFile(t *testing.T) {
t.Parallel()
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
err := ioutil.WriteFile(filepath.Join(td, "config.json"), []byte{}, 0644)
err := ioutil.WriteFile(filepath.Join(td, "config.json"), []byte(`{"bind_addr":"10.0.0.1", "data_dir":"`+td+`"}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}
@ -116,6 +81,12 @@ func TestValidateCommandQuiet(t *testing.T) {
td := testutil.TempDir(t, "consul")
defer os.RemoveAll(td)
fp := filepath.Join(td, "config.json")
err := ioutil.WriteFile(fp, []byte(`{"bind_addr":"10.0.0.1", "data_dir":"`+td+`"}`), 0644)
if err != nil {
t.Fatalf("err: %s", err)
}
ui, cmd := testValidateCommand(t)
args := []string{"-quiet", td}

Some files were not shown because too many files have changed in this diff Show More