tls: auto_encrypt enables automatic RPC cert provisioning for consul clients (#5597)

This commit is contained in:
Hans Hasselberg 2019-06-27 22:22:07 +02:00 committed by GitHub
parent 709d7bb36b
commit 73c4e9f07c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 1951 additions and 466 deletions

View File

@ -64,7 +64,7 @@ func NewTestACLAgent(name string, hcl string, resolveFn func(string) (acl.Author
config.Source{Name: a.Name + ".data_dir", Format: "hcl", Data: hclDataDir},
)
agent, err := New(a.Config)
agent, err := New(a.Config, nil)
if err != nil {
panic(fmt.Sprintf("Error creating agent: %v", err))
}

View File

@ -71,6 +71,12 @@ const (
"but no reason was provided. This is a default message."
defaultServiceMaintReason = "Maintenance mode is enabled for this " +
"service, but no reason was provided. This is a default message."
// ID of the roots watch
rootsWatchID = "roots"
// ID of the leaf watch
leafWatchID = "leaf"
)
type configSource int
@ -202,6 +208,8 @@ type Agent struct {
shutdownCh chan struct{}
shutdownLock sync.Mutex
InterruptStartCh chan struct{}
// joinLANNotifier is called after a successful JoinLAN.
joinLANNotifier notifier
@ -264,7 +272,7 @@ type Agent struct {
persistedTokensLock sync.RWMutex
}
func New(c *config.RuntimeConfig) (*Agent, error) {
func New(c *config.RuntimeConfig, logger *log.Logger) (*Agent, error) {
if c.Datacenter == "" {
return nil, fmt.Errorf("Must configure a Datacenter")
}
@ -272,32 +280,40 @@ func New(c *config.RuntimeConfig) (*Agent, error) {
return nil, fmt.Errorf("Must configure a DataDir")
}
a := &Agent{
config: c,
checkReapAfter: make(map[types.CheckID]time.Duration),
checkMonitors: make(map[types.CheckID]*checks.CheckMonitor),
checkTTLs: make(map[types.CheckID]*checks.CheckTTL),
checkHTTPs: make(map[types.CheckID]*checks.CheckHTTP),
checkTCPs: make(map[types.CheckID]*checks.CheckTCP),
checkGRPCs: make(map[types.CheckID]*checks.CheckGRPC),
checkDockers: make(map[types.CheckID]*checks.CheckDocker),
checkAliases: make(map[types.CheckID]*checks.CheckAlias),
eventCh: make(chan serf.UserEvent, 1024),
eventBuf: make([]*UserEvent, 256),
joinLANNotifier: &systemd.Notifier{},
reloadCh: make(chan chan error),
retryJoinCh: make(chan error),
shutdownCh: make(chan struct{}),
endpoints: make(map[string]string),
tokens: new(token.Store),
a := Agent{
config: c,
checkReapAfter: make(map[types.CheckID]time.Duration),
checkMonitors: make(map[types.CheckID]*checks.CheckMonitor),
checkTTLs: make(map[types.CheckID]*checks.CheckTTL),
checkHTTPs: make(map[types.CheckID]*checks.CheckHTTP),
checkTCPs: make(map[types.CheckID]*checks.CheckTCP),
checkGRPCs: make(map[types.CheckID]*checks.CheckGRPC),
checkDockers: make(map[types.CheckID]*checks.CheckDocker),
checkAliases: make(map[types.CheckID]*checks.CheckAlias),
eventCh: make(chan serf.UserEvent, 1024),
eventBuf: make([]*UserEvent, 256),
joinLANNotifier: &systemd.Notifier{},
reloadCh: make(chan chan error),
retryJoinCh: make(chan error),
shutdownCh: make(chan struct{}),
InterruptStartCh: make(chan struct{}),
endpoints: make(map[string]string),
tokens: new(token.Store),
logger: logger,
}
a.serviceManager = NewServiceManager(a)
a.serviceManager = NewServiceManager(&a)
if err := a.initializeACLs(); err != nil {
return nil, err
}
return a, nil
// Retrieve or generate the node ID before setting up the rest of the
// agent, which depends on it.
if err := a.setupNodeID(c); err != nil {
return nil, fmt.Errorf("Failed to setup node ID: %v", err)
}
return &a, nil
}
func LocalConfig(cfg *config.RuntimeConfig) local.Config {
@ -348,20 +364,6 @@ func (a *Agent) Start() error {
c := a.config
logOutput := a.LogOutput
if a.logger == nil {
if logOutput == nil {
logOutput = os.Stderr
}
a.logger = log.New(logOutput, "", log.LstdFlags)
}
// Retrieve or generate the node ID before setting up the rest of the
// agent, which depends on it.
if err := a.setupNodeID(c); err != nil {
return fmt.Errorf("Failed to setup node ID: %v", err)
}
// Warn if the node name is incompatible with DNS
if InvalidDnsRe.MatchString(a.config.NodeName) {
a.logger.Printf("[WARN] agent: Node name %q will not be discoverable "+
@ -433,6 +435,21 @@ func (a *Agent) Start() error {
// populated from above.
a.registerCache()
if a.config.AutoEncryptTLS && !a.config.ServerMode {
reply, err := a.setupClientAutoEncrypt()
if err != nil {
return fmt.Errorf("AutoEncrypt failed: %s", err)
}
rootsReq, leafReq, err := a.setupClientAutoEncryptCache(reply)
if err != nil {
return fmt.Errorf("AutoEncrypt failed: %s", err)
}
if err = a.setupClientAutoEncryptWatching(rootsReq, leafReq); err != nil {
return fmt.Errorf("AutoEncrypt failed: %s", err)
}
a.logger.Printf("[INFO] AutoEncrypt: upgraded to TLS")
}
// Load checks/services/metadata.
if err := a.loadServices(c); err != nil {
return err
@ -532,6 +549,158 @@ func (a *Agent) Start() error {
return nil
}
func (a *Agent) setupClientAutoEncrypt() (*structs.SignedResponse, error) {
client := a.delegate.(*consul.Client)
addrs := a.config.StartJoinAddrsLAN
disco, err := newDiscover()
if err != nil && len(addrs) == 0 {
return nil, err
}
addrs = append(addrs, retryJoinAddrs(disco, "LAN", a.config.RetryJoinLAN, a.logger)...)
reply, priv, err := client.RequestAutoEncryptCerts(addrs, a.config.ServerPort, a.tokens.AgentToken(), a.InterruptStartCh)
if err != nil {
return nil, err
}
connectCAPems := []string{}
for _, ca := range reply.ConnectCARoots.Roots {
connectCAPems = append(connectCAPems, ca.RootCert)
}
if err := a.tlsConfigurator.UpdateAutoEncrypt(reply.ManualCARoots, connectCAPems, reply.IssuedCert.CertPEM, priv, reply.VerifyServerHostname); err != nil {
return nil, err
}
return reply, nil
}
func (a *Agent) setupClientAutoEncryptCache(reply *structs.SignedResponse) (*structs.DCSpecificRequest, *cachetype.ConnectCALeafRequest, error) {
rootsReq := &structs.DCSpecificRequest{
Datacenter: a.config.Datacenter,
QueryOptions: structs.QueryOptions{Token: a.tokens.AgentToken()},
}
// prepolutate roots cache
rootRes := cache.FetchResult{Value: &reply.ConnectCARoots, Index: reply.ConnectCARoots.QueryMeta.Index}
if err := a.cache.Prepopulate(cachetype.ConnectCARootName, rootRes, a.config.Datacenter, a.tokens.AgentToken(), rootsReq.CacheInfo().Key); err != nil {
return nil, nil, err
}
leafReq := &cachetype.ConnectCALeafRequest{
Datacenter: a.config.Datacenter,
Token: a.tokens.AgentToken(),
Agent: a.config.NodeName,
}
// prepolutate leaf cache
certRes := cache.FetchResult{Value: &reply.IssuedCert, Index: reply.ConnectCARoots.QueryMeta.Index}
if err := a.cache.Prepopulate(cachetype.ConnectCALeafName, certRes, a.config.Datacenter, a.tokens.AgentToken(), leafReq.Key()); err != nil {
return nil, nil, err
}
return rootsReq, leafReq, nil
}
func (a *Agent) setupClientAutoEncryptWatching(rootsReq *structs.DCSpecificRequest, leafReq *cachetype.ConnectCALeafRequest) error {
// setup watches
ch := make(chan cache.UpdateEvent, 10)
ctx, cancel := context.WithCancel(context.Background())
// Watch for root changes
err := a.cache.Notify(ctx, cachetype.ConnectCARootName, rootsReq, rootsWatchID, ch)
if err != nil {
cancel()
return err
}
// Watch the leaf cert
err = a.cache.Notify(ctx, cachetype.ConnectCALeafName, leafReq, leafWatchID, ch)
if err != nil {
cancel()
return err
}
// Setup actions in case the watches are firing.
go func() {
for {
select {
case <-a.shutdownCh:
cancel()
return
case <-ctx.Done():
return
case u := <-ch:
switch u.CorrelationID {
case rootsWatchID:
roots, ok := u.Result.(*structs.IndexedCARoots)
if !ok {
err := fmt.Errorf("invalid type for roots response: %T", u.Result)
a.logger.Printf("[ERR] %s watch error: %s", u.CorrelationID, err)
continue
}
pems := []string{}
for _, root := range roots.Roots {
pems = append(pems, root.RootCert)
}
a.tlsConfigurator.UpdateAutoEncryptCA(pems)
case leafWatchID:
leaf, ok := u.Result.(*structs.IssuedCert)
if !ok {
err := fmt.Errorf("invalid type for leaf response: %T", u.Result)
a.logger.Printf("[ERR] %s watch error: %s", u.CorrelationID, err)
continue
}
a.tlsConfigurator.UpdateAutoEncryptCert(leaf.CertPEM, leaf.PrivateKeyPEM)
}
}
}
}()
// Setup safety net in case the auto_encrypt cert doesn't get renewed
// in time. The agent would be stuck in that case because the watches
// never use the AutoEncrypt.Sign endpoint.
go func() {
for {
// Check 10sec after cert expires. The agent cache
// should be handling the expiration and renew before
// it.
// If there is no cert, AutoEncryptCertNotAfter returns
// a value in the past which immediately triggers the
// renew, but this case shouldn't happen because at
// this point, auto_encrypt was just being setup
// successfully.
interval := a.tlsConfigurator.AutoEncryptCertNotAfter().Sub(time.Now().Add(10 * time.Second))
a.logger.Printf("[DEBUG] AutoEncrypt: client certificate expiration check in %s", interval)
select {
case <-a.shutdownCh:
return
case <-time.After(interval):
// check auto encrypt client cert expiration
if a.tlsConfigurator.AutoEncryptCertExpired() {
a.logger.Printf("[DEBUG] AutoEncrypt: client certificate expired.")
reply, err := a.setupClientAutoEncrypt()
if err != nil {
a.logger.Printf("[ERR] AutoEncrypt: client certificate expired, failed to renew: %s", err)
// in case of an error, try again in one minute
interval = time.Minute
continue
}
_, _, err = a.setupClientAutoEncryptCache(reply)
if err != nil {
a.logger.Printf("[ERR] AutoEncrypt: client certificate expired, failed to populate cache: %s", err)
// in case of an error, try again in one minute
interval = time.Minute
continue
}
}
}
}
}()
return nil
}
func (a *Agent) listenAndServeGRPC() error {
if len(a.config.GRPCAddrs) < 1 {
return nil
@ -1088,6 +1257,8 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
base.TLSCipherSuites = a.config.TLSCipherSuites
base.TLSPreferServerCipherSuites = a.config.TLSPreferServerCipherSuites
base.AutoEncryptAllowTLS = a.config.AutoEncryptAllowTLS
// Copy the Connect CA bootstrap config
if a.config.ConnectEnabled {
base.ConnectEnabled = true

View File

@ -3871,3 +3871,20 @@ func TestAgent_ReloadConfigTLSConfigFailure(t *testing.T) {
require.Len(t, tlsConf.ClientCAs.Subjects(), 1)
require.Len(t, tlsConf.RootCAs.Subjects(), 1)
}
func TestAgent_consulConfig(t *testing.T) {
t.Parallel()
dataDir := testutil.TempDir(t, "agent") // we manage the data dir
defer os.RemoveAll(dataDir)
hcl := `
data_dir = "` + dataDir + `"
verify_incoming = true
ca_file = "../test/ca/root.cer"
cert_file = "../test/key/ourdomain.cer"
key_file = "../test/key/ourdomain.key"
auto_encrypt { allow_tls = true }
`
a := NewTestAgent(t, t.Name(), hcl)
defer a.Shutdown()
require.True(t, a.consulConfig().AutoEncryptAllowTLS)
}

File diff suppressed because one or more lines are too long

View File

@ -501,12 +501,23 @@ func (c *ConnectCALeaf) generateNewLeaf(req *ConnectCALeafRequest,
return result, errors.New("cluster has no CA bootstrapped yet")
}
// Build the service ID
serviceID := &connect.SpiffeIDService{
Host: roots.TrustDomain,
Datacenter: req.Datacenter,
Namespace: "default",
Service: req.Service,
// Build the cert uri
var id connect.CertURI
if req.Service != "" {
id = &connect.SpiffeIDService{
Host: roots.TrustDomain,
Datacenter: req.Datacenter,
Namespace: "default",
Service: req.Service,
}
} else if req.Agent != "" {
id = &connect.SpiffeIDAgent{
Host: roots.TrustDomain,
Datacenter: req.Datacenter,
Agent: req.Agent,
}
} else {
return result, errors.New("URI must be either service or agent")
}
// Create a new private key
@ -516,7 +527,7 @@ func (c *ConnectCALeaf) generateNewLeaf(req *ConnectCALeafRequest,
}
// Create a CSR.
csr, err := connect.CreateCSR(serviceID, pk)
csr, err := connect.CreateCSR(id, pk)
if err != nil {
return result, err
}
@ -606,14 +617,23 @@ type ConnectCALeafRequest struct {
Token string
Datacenter string
Service string // Service name, not ID
Agent string // Agent name, not ID
MinQueryIndex uint64
MaxQueryTime time.Duration
}
func (r *ConnectCALeafRequest) Key() string {
if len(r.Agent) > 0 {
return fmt.Sprintf("agent:%s", r.Agent)
}
return fmt.Sprintf("service:%s", r.Service)
}
func (r *ConnectCALeafRequest) CacheInfo() cache.RequestInfo {
return cache.RequestInfo{
Token: r.Token,
Key: r.Service,
Key: r.Key(),
Datacenter: r.Datacenter,
MinIndex: r.MinQueryIndex,
Timeout: r.MaxQueryTime,

View File

@ -1016,3 +1016,10 @@ func (r *testGatedRootsRPC) RPC(method string, args interface{}, reply interface
*replyReal = <-r.ValueCh
return nil
}
func TestConnectCALeaf_Key(t *testing.T) {
r := ConnectCALeafRequest{Service: "web"}
require.Equal(t, "service:web", r.Key())
r = ConnectCALeafRequest{Agent: "abc"}
require.Equal(t, "agent:abc", r.Key())
}

31
agent/cache/cache.go vendored
View File

@ -387,7 +387,11 @@ RETRY_GET:
// entryKey returns the key for the entry in the cache. See the note
// about the entry key format in the structure docs for Cache.
func (c *Cache) entryKey(t string, r *RequestInfo) string {
return fmt.Sprintf("%s/%s/%s/%s", t, r.Datacenter, r.Token, r.Key)
return makeEntryKey(t, r.Datacenter, r.Token, r.Key)
}
func makeEntryKey(t, dc, token, key string) string {
return fmt.Sprintf("%s/%s/%s/%s", t, dc, token, key)
}
// fetch triggers a new background fetch for the given Request. If a
@ -745,3 +749,28 @@ func (c *Cache) Close() error {
}
return nil
}
// Prepopulate puts something in the cache manually. This is useful when the
// correct initial value is know and the cache shouldn't refetch the same thing
// on startup. It is used to set the ConnectRootCA and AgentLeafCert when
// AutoEncrypt.TLS is turned on. The cache itself cannot fetch that the first
// time because it requires a special RPCType. Subsequent runs are fine though.
func (c *Cache) Prepopulate(t string, res FetchResult, dc, token, k string) error {
// Check the type that we're prepolulating
c.typesLock.RLock()
tEntry, ok := c.types[t]
c.typesLock.RUnlock()
if !ok {
return fmt.Errorf("unknown type in cache: %s", t)
}
key := makeEntryKey(t, dc, token, k)
newEntry := cacheEntry{
Valid: true, Value: res.Value, State: res.State, Index: res.Index,
FetchedAt: time.Now(), Waiter: make(chan struct{}),
Expiry: &cacheEntryExpiry{Key: key, TTL: tEntry.Opts.LastGetTTL},
}
c.entriesLock.Lock()
c.entries[key] = newEntry
c.entriesLock.Unlock()
return nil
}

View File

@ -591,6 +591,13 @@ func (b *Builder) Build() (rt RuntimeConfig, err error) {
})
}
autoEncryptTLS := b.boolVal(c.AutoEncrypt.TLS)
autoEncryptAllowTLS := b.boolVal(c.AutoEncrypt.AllowTLS)
if autoEncryptAllowTLS {
connectEnabled = true
}
aclsEnabled := false
primaryDatacenter := strings.ToLower(b.stringVal(c.PrimaryDatacenter))
if c.ACLDatacenter != nil {
@ -793,6 +800,8 @@ func (b *Builder) Build() (rt RuntimeConfig, err error) {
Checks: checks,
ClientAddrs: clientAddrs,
ConfigEntryBootstrap: configEntries,
AutoEncryptTLS: autoEncryptTLS,
AutoEncryptAllowTLS: autoEncryptAllowTLS,
ConnectEnabled: connectEnabled,
ConnectCAProvider: connectCAProvider,
ConnectCAConfig: connectCAConfig,
@ -1101,6 +1110,12 @@ func (b *Builder) Validate(rt RuntimeConfig) error {
}
}
if rt.AutoEncryptAllowTLS {
if !rt.VerifyIncoming {
return fmt.Errorf("if auto_encrypt.allow_tls is turned on, TLS must be configured in order to work properly.")
}
}
// ----------------------------------------------------------------
// warnings
//

View File

@ -189,6 +189,7 @@ type Config struct {
Checks []CheckDefinition `json:"checks,omitempty" hcl:"checks" mapstructure:"checks"`
ClientAddr *string `json:"client_addr,omitempty" hcl:"client_addr" mapstructure:"client_addr"`
ConfigEntries ConfigEntries `json:"config_entries,omitempty" hcl:"config_entries" mapstructure:"config_entries"`
AutoEncrypt AutoEncrypt `json:"auto_encrypt,omitempty" hcl:"auto_encrypt" mapstructure:"auto_encrypt"`
Connect Connect `json:"connect,omitempty" hcl:"connect" mapstructure:"connect"`
DNS DNS `json:"dns_config,omitempty" hcl:"dns_config" mapstructure:"dns_config"`
DNSDomain *string `json:"domain,omitempty" hcl:"domain" mapstructure:"domain"`
@ -504,6 +505,16 @@ type Upstream struct {
Config map[string]interface{} `json:"config,omitempty" hcl:"config" mapstructure:"config"`
}
// AutoEncrypt is the agent-global auto_encrypt configuration.
type AutoEncrypt struct {
// TLS enables receiving certificates for clients from servers
TLS *bool `json:"tls,omitempty" hcl:"tls" mapstructure:"tls"`
// AllowTLS enables the RPC endpoint on the server to answer
// AutoEncrypt.Sign requests.
AllowTLS *bool `json:"allow_tls,omitempty" hcl:"allow_tls" mapstructure:"allow_tls"`
}
// Connect is the agent-global connect configuration.
type Connect struct {
// Enabled opts the agent into connect. It should be set on all clients and

View File

@ -519,6 +519,14 @@ type RuntimeConfig struct {
// If entries of the same Kind/Name exist already these will not update them.
ConfigEntryBootstrap []structs.ConfigEntry
// AutoEncryptTLS requires the client to acquire TLS certificates from
// servers.
AutoEncryptTLS bool
// AutoEncryptAllowTLS enables the server to respond to
// AutoEncrypt.Sign requests.
AutoEncryptAllowTLS bool
// ConnectEnabled opts the agent into connect. It should be set on all clients
// and servers in a cluster for correct connect operation.
ConnectEnabled bool
@ -1303,7 +1311,7 @@ type RuntimeConfig struct {
// hcl: skip_leave_on_interrupt = (true|false)
SkipLeaveOnInt bool
// StartJoinLAN is a list of addresses to attempt to join -wan when the
// StartJoinAddrsLAN is a list of addresses to attempt to join -lan when the
// agent starts. If Serf is unable to communicate with any of these
// addresses, then the agent will error and exit.
//
@ -1631,6 +1639,7 @@ func (c *RuntimeConfig) ToTLSUtilConfig() tlsutil.Config {
CipherSuites: c.TLSCipherSuites,
PreferServerCipherSuites: c.TLSPreferServerCipherSuites,
EnableAgentTLSForChecks: c.EnableAgentTLSForChecks,
AutoEncryptTLS: c.AutoEncryptTLS,
}
}

View File

@ -3161,6 +3161,10 @@ func TestFullConfig(t *testing.T) {
}
}
]
},
"auto_encrypt": {
"tls": true,
"allow_tls": true
},
"connect": {
"ca_provider": "consul",
@ -3737,6 +3741,10 @@ func TestFullConfig(t *testing.T) {
}
}
}
auto_encrypt = {
tls = true
allow_tls = true
}
connect {
ca_provider = "consul"
ca_config {
@ -4418,6 +4426,8 @@ func TestFullConfig(t *testing.T) {
},
},
},
AutoEncryptTLS: true,
AutoEncryptAllowTLS: true,
ConnectEnabled: true,
ConnectProxyBindMinPort: 2000,
ConnectProxyBindMaxPort: 3000,
@ -5210,6 +5220,8 @@ func TestSanitize(t *testing.T) {
}],
"ClientAddrs": [],
"ConfigEntryBootstrap": [],
"AutoEncryptTLS": false,
"AutoEncryptAllowTLS": false,
"ConnectCAConfig": {},
"ConnectCAProvider": "",
"ConnectEnabled": false,
@ -5668,24 +5680,26 @@ func TestRuntime_ToTLSUtilConfig(t *testing.T) {
TLSCipherSuites: []uint16{tls.TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305},
TLSPreferServerCipherSuites: true,
EnableAgentTLSForChecks: true,
AutoEncryptTLS: true,
}
r := c.ToTLSUtilConfig()
require.Equal(t, c.VerifyIncoming, r.VerifyIncoming)
require.Equal(t, c.VerifyIncomingRPC, r.VerifyIncomingRPC)
require.Equal(t, c.VerifyIncomingHTTPS, r.VerifyIncomingHTTPS)
require.Equal(t, c.VerifyOutgoing, r.VerifyOutgoing)
require.Equal(t, c.VerifyServerHostname, r.VerifyServerHostname)
require.Equal(t, c.CAFile, r.CAFile)
require.Equal(t, c.CAPath, r.CAPath)
require.Equal(t, c.CertFile, r.CertFile)
require.Equal(t, c.KeyFile, r.KeyFile)
require.Equal(t, c.NodeName, r.NodeName)
require.Equal(t, c.ServerName, r.ServerName)
require.Equal(t, c.DNSDomain, r.Domain)
require.Equal(t, c.TLSMinVersion, r.TLSMinVersion)
require.Equal(t, c.TLSCipherSuites, r.CipherSuites)
require.Equal(t, c.TLSPreferServerCipherSuites, r.PreferServerCipherSuites)
require.Equal(t, c.EnableAgentTLSForChecks, r.EnableAgentTLSForChecks)
require.True(t, r.VerifyIncoming)
require.True(t, r.VerifyIncomingRPC)
require.True(t, r.VerifyIncomingHTTPS)
require.True(t, r.VerifyOutgoing)
require.True(t, r.EnableAgentTLSForChecks)
require.True(t, r.AutoEncryptTLS)
require.True(t, r.VerifyServerHostname)
require.True(t, r.PreferServerCipherSuites)
require.Equal(t, "a", r.CAFile)
require.Equal(t, "b", r.CAPath)
require.Equal(t, "c", r.CertFile)
require.Equal(t, "d", r.KeyFile)
require.Equal(t, "e", r.NodeName)
require.Equal(t, "f", r.ServerName)
require.Equal(t, "g", r.Domain)
require.Equal(t, "tls12", r.TLSMinVersion)
require.Equal(t, []uint16{tls.TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305}, r.CipherSuites)
}
func TestReadPath(t *testing.T) {

View File

@ -349,8 +349,14 @@ func (c *ConsulProvider) Sign(csr *x509.CertificateRequest) (string, error) {
if err != nil {
return "", err
}
serviceId, ok := spiffeId.(*connect.SpiffeIDService)
if !ok {
subject := ""
switch id := spiffeId.(type) {
case *connect.SpiffeIDService:
subject = id.Service
case *connect.SpiffeIDAgent:
subject = id.Agent
default:
return "", fmt.Errorf("SPIFFE ID in CSR must be a service ID")
}
@ -373,7 +379,7 @@ func (c *ConsulProvider) Sign(csr *x509.CertificateRequest) (string, error) {
effectiveNow := time.Now().Add(-1 * time.Minute)
template := x509.Certificate{
SerialNumber: sn,
Subject: pkix.Name{CommonName: serviceId.Service},
Subject: pkix.Name{CommonName: subject},
URIs: csr.URIs,
Signature: csr.Signature,
SignatureAlgorithm: csr.SignatureAlgorithm,

View File

@ -181,6 +181,34 @@ func TestConsulCAProvider_SignLeaf(t *testing.T) {
require.True(parsed.NotAfter.Sub(time.Now()) < 3*24*time.Hour)
require.True(parsed.NotBefore.Before(time.Now()))
}
spiffeAgent := &connect.SpiffeIDAgent{
Host: "node1",
Datacenter: "dc1",
Agent: "uuid",
}
// Generate a leaf cert for an agent.
{
raw, _ := connect.TestCSR(t, spiffeAgent)
csr, err := connect.ParseCSR(raw)
require.NoError(err)
cert, err := provider.Sign(csr)
require.NoError(err)
parsed, err := connect.ParseCert(cert)
require.NoError(err)
require.Equal(spiffeAgent.URI(), parsed.URIs[0])
require.Equal("uuid", parsed.Subject.CommonName)
require.Equal(uint64(2), parsed.SerialNumber.Uint64())
// Ensure the cert is valid now and expires within the correct limit.
now := time.Now()
require.True(parsed.NotAfter.Sub(now) < time.Hour)
require.True(parsed.NotBefore.Before(now))
}
}
func TestConsulCAProvider_CrossSignCA(t *testing.T) {

View File

@ -31,6 +31,8 @@ type CertURI interface {
var (
spiffeIDServiceRegexp = regexp.MustCompile(
`^/ns/([^/]+)/dc/([^/]+)/svc/([^/]+)$`)
spiffeIDAgentRegexp = regexp.MustCompile(
`^/agent/client/dc/([^/]+)/id/([^/]+)$`)
)
// ParseCertURIFromString attempts to parse a string representation of a
@ -85,6 +87,27 @@ func ParseCertURI(input *url.URL) (CertURI, error) {
Datacenter: dc,
Service: service,
}, nil
} else if v := spiffeIDAgentRegexp.FindStringSubmatch(path); v != nil {
// Determine the values. We assume they're sane to save cycles,
// but if the raw path is not empty that means that something is
// URL encoded so we go to the slow path.
dc := v[1]
agent := v[2]
if input.RawPath != "" {
var err error
if dc, err = url.PathUnescape(v[1]); err != nil {
return nil, fmt.Errorf("Invalid datacenter: %s", err)
}
if agent, err = url.PathUnescape(v[2]); err != nil {
return nil, fmt.Errorf("Invalid node: %s", err)
}
}
return &SpiffeIDAgent{
Host: input.Host,
Datacenter: dc,
Agent: agent,
}, nil
}
// Test for signing ID
@ -98,5 +121,5 @@ func ParseCertURI(input *url.URL) (CertURI, error) {
}
}
return nil, fmt.Errorf("SPIFFE ID is not in the expected format")
return nil, fmt.Errorf("SPIFFE ID is not in the expected format: %s", input.String())
}

View File

@ -0,0 +1,29 @@
package connect
import (
"fmt"
"net/url"
"github.com/hashicorp/consul/agent/structs"
)
// SpiffeIDService is the structure to represent the SPIFFE ID for an agent.
type SpiffeIDAgent struct {
Host string
Datacenter string
Agent string
}
// URI returns the *url.URL for this SPIFFE ID.
func (id *SpiffeIDAgent) URI() *url.URL {
var result url.URL
result.Scheme = "spiffe"
result.Host = id.Host
result.Path = fmt.Sprintf("/agent/client/dc/%s/id/%s", id.Datacenter, id.Agent)
return &result
}
// CertURI impl.
func (id *SpiffeIDAgent) Authorize(_ *structs.Intention) (bool, bool) {
return false, false
}

View File

@ -0,0 +1,28 @@
package connect
import (
"testing"
"github.com/stretchr/testify/require"
)
func TestSpiffeIDAgentURI(t *testing.T) {
agent := &SpiffeIDAgent{
Host: "1234.consul",
Datacenter: "dc1",
Agent: "123",
}
require.Equal(t, "spiffe://1234.consul/agent/client/dc/dc1/id/123", agent.URI().String())
}
func TestSpiffeIDAgentAuthorize(t *testing.T) {
agent := &SpiffeIDAgent{
Host: "1234.consul",
Agent: "uuid",
}
auth, match := agent.Authorize(nil)
require.False(t, auth)
require.False(t, match)
}

View File

@ -33,6 +33,17 @@ var testCertURICases = []struct {
"",
},
{
"basic agent ID",
"spiffe://1234.consul/agent/client/dc/dc1/id/uuid",
&SpiffeIDAgent{
Host: "1234.consul",
Datacenter: "dc1",
Agent: "uuid",
},
"",
},
{
"service with URL-encoded values",
"spiffe://1234.consul/ns/foo%2Fbar/dc/bar%2Fbaz/svc/baz%2Fqux",

View File

@ -0,0 +1,194 @@
package consul
import (
"fmt"
"log"
"net"
"strings"
"time"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib"
"github.com/miekg/dns"
)
const (
dummyTrustDomain = "dummy.trustdomain"
retryJitterWindow = 30 * time.Second
)
func (c *Client) RequestAutoEncryptCerts(servers []string, port int, token string, interruptCh chan struct{}) (*structs.SignedResponse, string, error) {
errFn := func(err error) (*structs.SignedResponse, string, error) {
return nil, "", err
}
// Check if we know about a server already through gossip. Depending on
// how the agent joined, there might already be one. Also in case this
// gets called because the cert expired.
server := c.routers.FindServer()
if server != nil {
servers = []string{server.Addr.String()}
}
if len(servers) == 0 {
return errFn(fmt.Errorf("No servers to request AutoEncrypt.Sign"))
}
// We don't provide the correct host here, because we don't know any
// better at this point. Apart from the domain, we would need the
// ClusterID, which we don't have. This is why we go with
// dummyTrustDomain the first time. Subsequent CSRs will have the
// correct TrustDomain.
id := &connect.SpiffeIDAgent{
Host: dummyTrustDomain,
Datacenter: c.config.Datacenter,
Agent: string(c.config.NodeName),
}
// Create a new private key
pk, pkPEM, err := connect.GeneratePrivateKey()
if err != nil {
return errFn(err)
}
// Create a CSR.
csr, err := connect.CreateCSR(id, pk)
if err != nil {
return errFn(err)
}
// Prepare request and response so that it can be passed to
// RPCInsecure.
args := structs.CASignRequest{
WriteRequest: structs.WriteRequest{Token: token},
Datacenter: c.config.Datacenter,
CSR: csr,
}
var reply structs.SignedResponse
// Retry implementation modeled after https://github.com/hashicorp/consul/pull/5228.
// TLDR; there is a 30s window from which a random time is picked.
// Repeat until the call is successful.
attempts := 0
for {
select {
case <-interruptCh:
return errFn(fmt.Errorf("aborting AutoEncrypt because interrupted"))
default:
}
// Translate host to net.TCPAddr to make life easier for
// RPCInsecure.
for _, s := range servers {
ips, err := resolveAddr(s, c.logger)
if err != nil {
c.logger.Printf("[WARN] agent: AutoEncrypt resolveAddr failed: %v", err)
continue
}
for _, ip := range ips {
addr := net.TCPAddr{IP: ip, Port: port}
if err = c.connPool.RPC(c.config.Datacenter, &addr, 0, "AutoEncrypt.Sign", true, &args, &reply); err == nil {
return &reply, pkPEM, nil
} else {
c.logger.Printf("[WARN] agent: AutoEncrypt failed: %v", err)
}
}
}
attempts++
delay := lib.RandomStagger(retryJitterWindow)
interval := (time.Duration(attempts) * delay) + delay
c.logger.Printf("[WARN] agent: retrying AutoEncrypt in %v", interval)
select {
case <-time.After(interval):
continue
case <-interruptCh:
return errFn(fmt.Errorf("aborting AutoEncrypt because interrupted"))
case <-c.shutdownCh:
return errFn(fmt.Errorf("aborting AutoEncrypt because shutting down"))
}
}
}
// resolveAddr is used to resolve the address into an address,
// port, and error. If no port is given, use the default
func resolveAddr(rawHost string, logger *log.Logger) ([]net.IP, error) {
host, _, err := net.SplitHostPort(rawHost)
if err != nil && err.Error() != "missing port in address" {
return nil, err
}
if ip := net.ParseIP(host); ip != nil {
return []net.IP{ip}, nil
}
// First try TCP so we have the best chance for the largest list of
// hosts to join. If this fails it's not fatal since this isn't a standard
// way to query DNS, and we have a fallback below.
if ips, err := tcpLookupIP(host, logger); err != nil {
logger.Printf("[DEBUG] agent: TCP-first lookup failed for '%s', falling back to UDP: %s", host, err)
} else if len(ips) > 0 {
return ips, nil
}
// If TCP didn't yield anything then use the normal Go resolver which
// will try UDP, then might possibly try TCP again if the UDP response
// indicates it was truncated.
return net.LookupIP(host)
}
// tcpLookupIP is a helper to initiate a TCP-based DNS lookup for the given host.
// The built-in Go resolver will do a UDP lookup first, and will only use TCP if
// the response has the truncate bit set, which isn't common on DNS servers like
// Consul's. By doing the TCP lookup directly, we get the best chance for the
// largest list of hosts to join. Since joins are relatively rare events, it's ok
// to do this rather expensive operation.
func tcpLookupIP(host string, logger *log.Logger) ([]net.IP, error) {
// Don't attempt any TCP lookups against non-fully qualified domain
// names, since those will likely come from the resolv.conf file.
if !strings.Contains(host, ".") {
return nil, nil
}
// Make sure the domain name is terminated with a dot (we know there's
// at least one character at this point).
dn := host
if dn[len(dn)-1] != '.' {
dn = dn + "."
}
// See if we can find a server to try.
cc, err := dns.ClientConfigFromFile("/etc/resolv.conf")
if err != nil {
return nil, err
}
if len(cc.Servers) > 0 {
// Do the lookup.
c := new(dns.Client)
c.Net = "tcp"
msg := new(dns.Msg)
msg.SetQuestion(dn, dns.TypeANY)
in, _, err := c.Exchange(msg, cc.Servers[0])
if err != nil {
return nil, err
}
// Handle any IPs we get back that we can attempt to join.
var ips []net.IP
for _, r := range in.Answer {
switch rr := r.(type) {
case (*dns.A):
ips = append(ips, rr.A)
case (*dns.AAAA):
ips = append(ips, rr.AAAA)
case (*dns.CNAME):
logger.Printf("[DEBUG] agent: Ignoring CNAME RR in TCP-first answer for '%s'", host)
}
}
return ips, nil
}
return nil, nil
}

View File

@ -0,0 +1,51 @@
package consul
import (
"errors"
"github.com/hashicorp/consul/agent/structs"
)
var (
ErrAutoEncryptAllowTLSNotEnabled = errors.New("AutoEncrypt.AllowTLS must be enabled in order to use this endpoint")
)
type AutoEncrypt struct {
srv *Server
}
// Sign signs a certificate for an agent.
func (a *AutoEncrypt) Sign(
args *structs.CASignRequest,
reply *structs.SignedResponse) error {
if !a.srv.config.ConnectEnabled {
return ErrConnectNotEnabled
}
if !a.srv.config.AutoEncryptAllowTLS {
return ErrAutoEncryptAllowTLSNotEnabled
}
if done, err := a.srv.forward("AutoEncrypt.Sign", args, args, reply); done {
return err
}
// This is the ConnectCA endpoint which is reused here because it is
// exactly what is needed.
c := ConnectCA{srv: a.srv}
rootsArgs := structs.DCSpecificRequest{Datacenter: args.Datacenter}
roots := structs.IndexedCARoots{}
err := c.Roots(&rootsArgs, &roots)
cert := structs.IssuedCert{}
err = c.Sign(args, &cert)
if err != nil {
return err
}
reply.IssuedCert = cert
reply.ConnectCARoots = roots
reply.ManualCARoots = a.srv.tlsConfigurator.ManualCAPems()
reply.VerifyServerHostname = a.srv.tlsConfigurator.VerifyServerHostname()
return nil
}

View File

@ -0,0 +1,126 @@
package consul
import (
"crypto/x509"
"fmt"
"os"
"strings"
"testing"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/tlsutil"
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestAutoEncryptSign(t *testing.T) {
t.Parallel()
type test struct {
Name string
Config tlsutil.Config
ConnError bool
RPCError bool
Cert string
Key string
}
root := "../../test/ca/root.cer"
badRoot := "../../test/ca_path/cert1.crt"
tests := []test{
{Name: "Works with defaults", Config: tlsutil.Config{}, ConnError: false},
{Name: "Works with good root", Config: tlsutil.Config{CAFile: root}, ConnError: false},
{Name: "VerifyOutgoing fails because of bad root", Config: tlsutil.Config{CAFile: badRoot}, ConnError: true},
{Name: "VerifyServerHostname fails", Config: tlsutil.Config{VerifyServerHostname: true, CAFile: root}, ConnError: false, RPCError: true},
{Name: "VerifyServerHostname succeeds", Cert: "../../test/key/ourdomain_server.cer", Key: "../../test/key/ourdomain_server.key",
Config: tlsutil.Config{VerifyServerHostname: true, CAFile: root}, ConnError: false, RPCError: false},
}
for i, test := range tests {
t.Run(test.Name, func(t *testing.T) {
cert := test.Cert
key := test.Key
if cert == "" {
cert = "../../test/key/ourdomain.cer"
}
if key == "" {
key = "../../test/key/ourdomain.key"
}
dir, s := testServerWithConfig(t, func(c *Config) {
c.AutoEncryptAllowTLS = true
c.Bootstrap = true
c.CAFile = root
c.VerifyOutgoing = true
c.CertFile = cert
c.KeyFile = key
})
defer os.RemoveAll(dir)
defer s.Shutdown()
testrpc.WaitForLeader(t, s.RPC, "dc1")
info := fmt.Sprintf("case %d", i)
// Generate a CSR and request signing
id := &connect.SpiffeIDAgent{
Host: strings.TrimSuffix("domain", "."),
Datacenter: "dc1",
Agent: "uuid",
}
// Create a new private key
pk, _, err := connect.GeneratePrivateKey()
require.NoError(t, err, info)
// Create a CSR.
csr, err := connect.CreateCSR(id, pk)
require.NoError(t, err, info)
require.NotEmpty(t, csr, info)
args := &structs.CASignRequest{
Datacenter: "dc1",
CSR: csr,
}
cfg := test.Config
cfg.AutoEncryptTLS = true
cfg.Domain = "consul"
codec, err := insecureRPCClient(s, cfg)
if test.ConnError {
require.Error(t, err, info)
return
}
require.NoError(t, err, info)
var reply structs.SignedResponse
err = msgpackrpc.CallWithCodec(codec, "AutoEncrypt.Sign", args, &reply)
codec.Close()
if test.RPCError {
require.Error(t, err, info)
return
}
require.NoError(t, err, info)
// Get the current CA
state := s.fsm.State()
_, ca, err := state.CARootActive(nil)
require.NoError(t, err, info)
// Verify that the cert is signed by the CA
roots := x509.NewCertPool()
assert.True(t, roots.AppendCertsFromPEM([]byte(ca.RootCert)))
leaf, err := connect.ParseCert(reply.IssuedCert.CertPEM)
require.NoError(t, err, info)
_, err = leaf.Verify(x509.VerifyOptions{
Roots: roots,
})
require.NoError(t, err, info)
// Verify other fields
require.Equal(t, "uuid", reply.IssuedCert.Agent, info)
require.Len(t, reply.ManualCARoots, 1, info)
require.Len(t, reply.ConnectCARoots.Roots, 1, info)
})
}
}

View File

@ -84,6 +84,8 @@ type Client struct {
// embedded struct to hold all the enterprise specific data
EnterpriseClient
tlsConfigurator *tlsutil.Configurator
}
// NewClient is used to construct a new Consul client from the configuration,
@ -125,21 +127,22 @@ func NewClientLogger(config *Config, logger *log.Logger, tlsConfigurator *tlsuti
}
connPool := &pool.ConnPool{
SrcAddr: config.RPCSrcAddr,
LogOutput: config.LogOutput,
MaxTime: clientRPCConnMaxIdle,
MaxStreams: clientMaxStreams,
TLSWrapper: tlsConfigurator.OutgoingRPCWrapper(),
ForceTLS: config.VerifyOutgoing,
SrcAddr: config.RPCSrcAddr,
LogOutput: config.LogOutput,
MaxTime: clientRPCConnMaxIdle,
MaxStreams: clientMaxStreams,
TLSConfigurator: tlsConfigurator,
ForceTLS: config.VerifyOutgoing,
}
// Create client
c := &Client{
config: config,
connPool: connPool,
eventCh: make(chan serf.Event, serfEventBacklog),
logger: logger,
shutdownCh: make(chan struct{}),
config: config,
connPool: connPool,
eventCh: make(chan serf.Event, serfEventBacklog),
logger: logger,
shutdownCh: make(chan struct{}),
tlsConfigurator: tlsConfigurator,
}
c.rpcLimiter.Store(rate.NewLimiter(config.RPCRate, config.RPCMaxBurst))

View File

@ -417,17 +417,25 @@ type Config struct {
// ConfigEntryBootstrap contains a list of ConfigEntries to ensure are created
// If entries of the same Kind/Name exist already these will not update them.
ConfigEntryBootstrap []structs.ConfigEntry
// AutoEncryptAllowTLS is whether to enable the server responding to
// AutoEncrypt.Sign requests.
AutoEncryptAllowTLS bool
}
// ToTLSUtilConfig is only used by tests, usually the config is being passed
// down from the agent.
func (c *Config) ToTLSUtilConfig() tlsutil.Config {
return tlsutil.Config{
VerifyIncoming: c.VerifyIncoming,
VerifyOutgoing: c.VerifyOutgoing,
VerifyServerHostname: c.VerifyServerHostname,
CAFile: c.CAFile,
CAPath: c.CAPath,
CertFile: c.CertFile,
KeyFile: c.KeyFile,
NodeName: c.NodeName,
Domain: c.Domain,
ServerName: c.ServerName,
TLSMinVersion: c.TLSMinVersion,
CipherSuites: c.TLSCipherSuites,

View File

@ -410,10 +410,6 @@ func (s *ConnectCA) Sign(
if err != nil {
return err
}
serviceID, ok := spiffeID.(*connect.SpiffeIDService)
if !ok {
return fmt.Errorf("SPIFFE ID in CSR must be a service ID")
}
provider, caRoot := s.srv.getCAProvider()
if provider == nil {
@ -427,9 +423,17 @@ func (s *ConnectCA) Sign(
return err
}
signingID := connect.SpiffeIDSigningForCluster(config)
if !signingID.CanSign(serviceID) {
return fmt.Errorf("SPIFFE ID in CSR from a different trust domain: %s, "+
"we are %s", serviceID.Host, signingID.Host())
serviceID, isService := spiffeID.(*connect.SpiffeIDService)
agentID, isAgent := spiffeID.(*connect.SpiffeIDAgent)
if !isService && !isAgent {
return fmt.Errorf("SPIFFE ID in CSR must be a service or agent ID")
}
if isService {
if !signingID.CanSign(spiffeID) {
return fmt.Errorf("SPIFFE ID in CSR from a different trust domain: %s, "+
"we are %s", serviceID.Host, signingID.Host())
}
}
// Verify that the ACL token provided has permission to act as this service
@ -437,15 +441,21 @@ func (s *ConnectCA) Sign(
if err != nil {
return err
}
if rule != nil && !rule.ServiceWrite(serviceID.Service, nil) {
return acl.ErrPermissionDenied
}
if isService {
if rule != nil && !rule.ServiceWrite(serviceID.Service, nil) {
return acl.ErrPermissionDenied
}
// Verify that the DC in the service URI matches us. We might relax this
// requirement later but being restrictive for now is safer.
if serviceID.Datacenter != s.srv.config.Datacenter {
return fmt.Errorf("SPIFFE ID in CSR from a different datacenter: %s, "+
"we are %s", serviceID.Datacenter, s.srv.config.Datacenter)
// Verify that the DC in the service URI matches us. We might relax this
// requirement later but being restrictive for now is safer.
if serviceID.Datacenter != s.srv.config.Datacenter {
return fmt.Errorf("SPIFFE ID in CSR from a different datacenter: %s, "+
"we are %s", serviceID.Datacenter, s.srv.config.Datacenter)
}
} else if isAgent {
if rule != nil && !rule.NodeWrite(agentID.Agent, nil) {
return acl.ErrPermissionDenied
}
}
commonCfg, err := config.GetCommonConfig()
@ -529,8 +539,6 @@ func (s *ConnectCA) Sign(
*reply = structs.IssuedCert{
SerialNumber: connect.HexString(cert.SerialNumber.Bytes()),
CertPEM: pem,
Service: serviceID.Service,
ServiceURI: cert.URIs[0].String(),
ValidAfter: cert.NotBefore,
ValidBefore: cert.NotAfter,
RaftIndex: structs.RaftIndex{
@ -538,6 +546,13 @@ func (s *ConnectCA) Sign(
CreateIndex: modIdx,
},
}
if isService {
reply.Service = serviceID.Service
reply.ServiceURI = cert.URIs[0].String()
} else if isAgent {
reply.Agent = agentID.Agent
reply.AgentURI = cert.URIs[0].String()
}
return nil
}

View File

@ -84,7 +84,7 @@ func (s *Server) handleConn(conn net.Conn, isTLS bool) {
typ := pool.RPCType(buf[0])
// Enforce TLS if VerifyIncoming is set
if s.config.VerifyIncoming && !isTLS && typ != pool.RPCTLS {
if s.tlsConfigurator.VerifyIncomingRPC() && !isTLS && typ != pool.RPCTLS && typ != pool.RPCTLSInsecure {
s.logger.Printf("[WARN] consul.rpc: Non-TLS connection attempted with VerifyIncoming set %s", logConn(conn))
conn.Close()
return
@ -100,12 +100,7 @@ func (s *Server) handleConn(conn net.Conn, isTLS bool) {
s.raftLayer.Handoff(conn)
case pool.RPCTLS:
if s.rpcTLS == nil {
s.logger.Printf("[WARN] consul.rpc: TLS connection attempted, server not configured for TLS %s", logConn(conn))
conn.Close()
return
}
conn = tls.Server(conn, s.rpcTLS)
conn = tls.Server(conn, s.tlsConfigurator.IncomingRPCConfig())
s.handleConn(conn, true)
case pool.RPCMultiplexV2:
@ -114,6 +109,10 @@ func (s *Server) handleConn(conn net.Conn, isTLS bool) {
case pool.RPCSnapshot:
s.handleSnapshotConn(conn)
case pool.RPCTLSInsecure:
conn = tls.Server(conn, s.tlsConfigurator.IncomingInsecureRPCConfig())
s.handleInsecureConn(conn)
default:
if !s.handleEnterpriseRPCConn(typ, conn, isTLS) {
s.logger.Printf("[ERR] consul.rpc: unrecognized RPC byte: %v %s", typ, logConn(conn))
@ -163,6 +162,28 @@ func (s *Server) handleConsulConn(conn net.Conn) {
}
}
// handleInsecureConsulConn is used to service a single Consul INSECURERPC connection
func (s *Server) handleInsecureConn(conn net.Conn) {
defer conn.Close()
rpcCodec := msgpackrpc.NewServerCodec(conn)
for {
select {
case <-s.shutdownCh:
return
default:
}
if err := s.insecureRPCServer.ServeRequest(rpcCodec); err != nil {
if err != io.EOF && !strings.Contains(err.Error(), "closed") {
s.logger.Printf("[ERR] consul.rpc: INSECURERPC error: %v %s", err, logConn(conn))
metrics.IncrCounter([]string{"rpc", "request_error"}, 1)
}
return
}
metrics.IncrCounter([]string{"rpc", "request"}, 1)
}
}
// handleSnapshotConn is used to dispatch snapshot saves and restores, which
// stream so don't use the normal RPC mechanism.
func (s *Server) handleSnapshotConn(conn net.Conn) {

View File

@ -2,7 +2,6 @@ package consul
import (
"context"
"crypto/tls"
"errors"
"fmt"
"io"
@ -33,6 +32,7 @@ import (
"github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/raft"
raftboltdb "github.com/hashicorp/raft-boltdb"
"github.com/hashicorp/serf/serf"
@ -217,8 +217,16 @@ type Server struct {
Listener net.Listener
rpcServer *rpc.Server
// rpcTLS is the TLS config for incoming TLS requests
rpcTLS *tls.Config
// insecureRPCServer is a RPC server that is configure with
// IncomingInsecureRPCConfig to allow clients to call AutoEncrypt.Sign
// to request client certificates. At this point a client doesn't have
// a client cert and thus cannot present it. This is the only RPC
// Endpoint that is available at the time of writing.
insecureRPCServer *rpc.Server
// tlsConfigurator holds the agent configuration relevant to TLS and
// configures everything related to it.
tlsConfigurator *tlsutil.Configurator
// serfLAN is the Serf cluster maintained inside the DC
// which contains all the DC nodes
@ -332,33 +340,34 @@ func NewServerLogger(config *Config, logger *log.Logger, tokens *token.Store, tl
shutdownCh := make(chan struct{})
connPool := &pool.ConnPool{
SrcAddr: config.RPCSrcAddr,
LogOutput: config.LogOutput,
MaxTime: serverRPCCache,
MaxStreams: serverMaxStreams,
TLSWrapper: tlsConfigurator.OutgoingRPCWrapper(),
ForceTLS: config.VerifyOutgoing,
SrcAddr: config.RPCSrcAddr,
LogOutput: config.LogOutput,
MaxTime: serverRPCCache,
MaxStreams: serverMaxStreams,
TLSConfigurator: tlsConfigurator,
ForceTLS: config.VerifyOutgoing,
}
// Create server.
s := &Server{
config: config,
tokens: tokens,
connPool: connPool,
eventChLAN: make(chan serf.Event, serfEventChSize),
eventChWAN: make(chan serf.Event, serfEventChSize),
logger: logger,
leaveCh: make(chan struct{}),
reconcileCh: make(chan serf.Member, reconcileChSize),
router: router.NewRouter(logger, config.Datacenter),
rpcServer: rpc.NewServer(),
rpcTLS: tlsConfigurator.IncomingRPCConfig(),
reassertLeaderCh: make(chan chan error),
segmentLAN: make(map[string]*serf.Serf, len(config.Segments)),
sessionTimers: NewSessionTimers(),
tombstoneGC: gc,
serverLookup: NewServerLookup(),
shutdownCh: shutdownCh,
config: config,
tokens: tokens,
connPool: connPool,
eventChLAN: make(chan serf.Event, serfEventChSize),
eventChWAN: make(chan serf.Event, serfEventChSize),
logger: logger,
leaveCh: make(chan struct{}),
reconcileCh: make(chan serf.Member, reconcileChSize),
router: router.NewRouter(logger, config.Datacenter),
rpcServer: rpc.NewServer(),
insecureRPCServer: rpc.NewServer(),
tlsConfigurator: tlsConfigurator,
reassertLeaderCh: make(chan chan error),
segmentLAN: make(map[string]*serf.Serf, len(config.Segments)),
sessionTimers: NewSessionTimers(),
tombstoneGC: gc,
serverLookup: NewServerLookup(),
shutdownCh: shutdownCh,
}
// Initialize enterprise specific server functionality
@ -402,7 +411,7 @@ func NewServerLogger(config *Config, logger *log.Logger, tokens *token.Store, tl
}
// Initialize the RPC layer.
if err := s.setupRPC(tlsConfigurator.OutgoingRPCWrapper()); err != nil {
if err := s.setupRPC(); err != nil {
s.Shutdown()
return nil, fmt.Errorf("Failed to start RPC layer: %v", err)
}
@ -420,6 +429,10 @@ func NewServerLogger(config *Config, logger *log.Logger, tokens *token.Store, tl
return nil, fmt.Errorf("Failed to start Raft: %v", err)
}
if s.config.ConnectEnabled && s.config.AutoEncryptAllowTLS {
go s.trackAutoEncryptCARoots()
}
// Serf and dynamic bind ports
//
// The LAN serf cluster announces the port of the WAN serf cluster
@ -512,6 +525,33 @@ func NewServerLogger(config *Config, logger *log.Logger, tokens *token.Store, tl
return s, nil
}
func (s *Server) trackAutoEncryptCARoots() {
for {
select {
case <-s.shutdownCh:
s.logger.Printf("[DEBUG] agent: shutting down trackAutoEncryptCARoots because shutdown")
return
default:
}
ws := memdb.NewWatchSet()
state := s.fsm.State()
ws.Add(state.AbandonCh())
_, cas, err := state.CARoots(ws)
if err != nil {
s.logger.Printf("[DEBUG] agent: Failed to watch AutoEncrypt CARoot: %v", err)
return
}
caPems := []string{}
for _, ca := range cas {
caPems = append(caPems, ca.RootCert)
}
if err := s.tlsConfigurator.UpdateAutoEncryptCA(caPems); err != nil {
s.logger.Printf("[DEBUG] agent: Failed to update AutoEncrypt CAPems: %v", err)
}
ws.Watch(nil)
}
}
// setupRaft is used to setup and initialize Raft
func (s *Server) setupRaft() error {
// If we have an unclean exit then attempt to close the Raft store.
@ -704,11 +744,16 @@ func registerEndpoint(fn factory) {
}
// setupRPC is used to setup the RPC listener
func (s *Server) setupRPC(tlsWrap tlsutil.DCWrapper) error {
func (s *Server) setupRPC() error {
for _, fn := range endpoints {
s.rpcServer.Register(fn(s))
}
// Only register AutoEncrypt on the insecure RPC server. Insecure only
// means that verify incoming is turned off even though it might have
// been configured.
s.insecureRPCServer.Register(&AutoEncrypt{srv: s})
ln, err := net.ListenTCP("tcp", s.config.RPCAddr)
if err != nil {
return err
@ -730,7 +775,7 @@ func (s *Server) setupRPC(tlsWrap tlsutil.DCWrapper) error {
// Provide a DC specific wrapper. Raft replication is only
// ever done in the same datacenter, so we can provide it as a constant.
wrapper := tlsutil.SpecificDC(s.config.Datacenter, tlsWrap)
wrapper := tlsutil.SpecificDC(s.config.Datacenter, s.tlsConfigurator.OutgoingRPCWrapper())
// Define a callback for determining whether to wrap a connection with TLS
tlsFunc := func(address raft.ServerAddress) bool {

View File

@ -9,6 +9,7 @@ import (
"github.com/hashicorp/consul/agent/pool"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/net-rpc-msgpackrpc"
)
@ -24,6 +25,23 @@ func rpcClient(t *testing.T, s *Server) rpc.ClientCodec {
return msgpackrpc.NewClientCodec(conn)
}
func insecureRPCClient(s *Server, c tlsutil.Config) (rpc.ClientCodec, error) {
addr := s.config.RPCAdvertise
configurator, err := tlsutil.NewConfigurator(c, nil)
if err != nil {
return nil, err
}
wrapper := configurator.OutgoingRPCWrapper()
if wrapper == nil {
return nil, err
}
conn, _, err := pool.DialTimeoutWithRPCType(s.config.Datacenter, addr, nil, time.Second, true, wrapper, pool.RPCTLSInsecure)
if err != nil {
return nil, err
}
return msgpackrpc.NewClientCodec(conn), nil
}
func TestStatusLeader(t *testing.T) {
t.Parallel()
dir1, s1 := testServer(t)

View File

@ -4,7 +4,6 @@ type RPCType byte
const (
// keep numbers unique.
// iota depends on order
RPCConsul RPCType = 0
RPCRaft = 1
RPCMultiplex = 2 // Old Muxado byte, no longer supported.
@ -12,4 +11,10 @@ const (
RPCMultiplexV2 = 4
RPCSnapshot = 5
RPCGossip = 6
// RPCTLSInsecure is used to flag RPC calls that require verify
// incoming to be disabled, even when it is turned on in the
// configuration. At the time of writing there is only AutoEncryt.Sign
// that is supported and it might be the only one there
// ever is.
RPCTLSInsecure = 7
)

View File

@ -133,8 +133,8 @@ type ConnPool struct {
// The maximum number of open streams to keep
MaxStreams int
// TLS wrapper
TLSWrapper tlsutil.DCWrapper
// TLSConfigurator
TLSConfigurator *tlsutil.Configurator
// ForceTLS is used to enforce outgoing TLS verification
ForceTLS bool
@ -198,7 +198,7 @@ func (p *ConnPool) acquire(dc string, addr net.Addr, version int, useTLS bool) (
addrStr := addr.String()
// Check to see if there's a pooled connection available. This is up
// here since it should the the vastly more common case than the rest
// here since it should the vastly more common case than the rest
// of the code here.
p.Lock()
c := p.pool[addrStr]
@ -266,13 +266,31 @@ type HalfCloser interface {
CloseWrite() error
}
// DialTimeout is used to establish a raw connection to the given server, with a
// given connection timeout.
// DialTimeout is used to establish a raw connection to the given server, with
// given connection timeout. It also writes RPCTLS as the first byte.
func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration, useTLS bool) (net.Conn, HalfCloser, error) {
p.once.Do(p.init)
return DialTimeoutWithRPCType(dc, addr, p.SrcAddr, timeout, useTLS || p.ForceTLS, p.TLSConfigurator.OutgoingRPCWrapper(), RPCTLS)
}
// DialTimeoutInsecure is used to establish a raw connection to the given
// server, with given connection timeout. It also writes RPCTLSInsecure as the
// first byte to indicate that the client cannot provide a certificate. This is
// so far only used for AutoEncrypt.Sign.
func (p *ConnPool) DialTimeoutInsecure(dc string, addr net.Addr, timeout time.Duration, wrapper tlsutil.DCWrapper) (net.Conn, HalfCloser, error) {
p.once.Do(p.init)
if wrapper == nil {
return nil, nil, fmt.Errorf("wrapper cannot be nil")
}
return DialTimeoutWithRPCType(dc, addr, p.SrcAddr, timeout, true, wrapper, RPCTLSInsecure)
}
func DialTimeoutWithRPCType(dc string, addr net.Addr, src *net.TCPAddr, timeout time.Duration, useTLS bool, wrapper tlsutil.DCWrapper, rpcType RPCType) (net.Conn, HalfCloser, error) {
// Try to dial the conn
d := &net.Dialer{LocalAddr: p.SrcAddr, Timeout: timeout}
d := &net.Dialer{LocalAddr: src, Timeout: timeout}
conn, err := d.Dial("tcp", addr.String())
if err != nil {
return nil, nil, err
@ -287,15 +305,15 @@ func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration,
}
// Check if TLS is enabled
if (useTLS || p.ForceTLS) && p.TLSWrapper != nil {
if (useTLS) && wrapper != nil {
// Switch the connection into TLS mode
if _, err := conn.Write([]byte{byte(RPCTLS)}); err != nil {
if _, err := conn.Write([]byte{byte(rpcType)}); err != nil {
conn.Close()
return nil, nil, err
}
// Wrap the connection in a TLS client
tlsConn, err := p.TLSWrapper(dc, conn)
tlsConn, err := wrapper(dc, conn)
if err != nil {
conn.Close()
return nil, nil, err
@ -402,6 +420,36 @@ START:
// RPC is used to make an RPC call to a remote host
func (p *ConnPool) RPC(dc string, addr net.Addr, version int, method string, useTLS bool, args interface{}, reply interface{}) error {
if method == "AutoEncrypt.Sign" {
return p.rpcInsecure(dc, addr, method, args, reply)
} else {
return p.rpc(dc, addr, version, method, useTLS, args, reply)
}
}
// rpcInsecure is used to make an RPC call to a remote host.
// It doesn't actually use any of the pooling, it is here so that it is
// transparent for the consumer. The pool cannot be used because
// AutoEncrypt.Sign is a one-off call and it doesn't make sense to pool that
// connection if it is not being reused.
func (p *ConnPool) rpcInsecure(dc string, addr net.Addr, method string, args interface{}, reply interface{}) error {
var codec rpc.ClientCodec
conn, _, err := p.DialTimeoutInsecure(dc, addr, 1*time.Second, p.TLSConfigurator.OutgoingRPCWrapper())
if err != nil {
return fmt.Errorf("rpcinsecure error establishing connection: %v", err)
}
codec = msgpackrpc.NewClientCodec(conn)
// Make the RPC call
err = msgpackrpc.CallWithCodec(codec, method, args, reply)
if err != nil {
return fmt.Errorf("rpcinsecure error making call: %v", err)
}
return nil
}
func (p *ConnPool) rpc(dc string, addr net.Addr, version int, method string, useTLS bool, args interface{}, reply interface{}) error {
p.once.Do(p.init)
// Get a usable client

View File

@ -39,6 +39,47 @@ func (a *Agent) retryJoinWAN() {
}
}
func newDiscover() (*discover.Discover, error) {
providers := make(map[string]discover.Provider)
for k, v := range discover.Providers {
providers[k] = v
}
providers["k8s"] = &discoverk8s.Provider{}
return discover.New(
discover.WithUserAgent(lib.UserAgent()),
discover.WithProviders(providers),
)
}
func retryJoinAddrs(disco *discover.Discover, cluster string, retryJoin []string, logger *log.Logger) []string {
addrs := []string{}
if disco == nil {
return addrs
}
for _, addr := range retryJoin {
switch {
case strings.Contains(addr, "provider="):
servers, err := disco.Addrs(addr, logger)
if err != nil {
if logger != nil {
logger.Printf("[ERR] agent: Cannot discover %s %s: %s", cluster, addr, err)
}
} else {
addrs = append(addrs, servers...)
if logger != nil {
logger.Printf("[INFO] agent: Discovered %s servers: %s", cluster, strings.Join(servers, " "))
}
}
default:
addrs = append(addrs, addr)
}
}
return addrs
}
// retryJoiner is used to handle retrying a join until it succeeds or all
// retries are exhausted.
type retryJoiner struct {
@ -69,17 +110,7 @@ func (r *retryJoiner) retryJoin() error {
return nil
}
// Copy the default providers, and then add the non-default
providers := make(map[string]discover.Provider)
for k, v := range discover.Providers {
providers[k] = v
}
providers["k8s"] = &discoverk8s.Provider{}
disco, err := discover.New(
discover.WithUserAgent(lib.UserAgent()),
discover.WithProviders(providers),
)
disco, err := newDiscover()
if err != nil {
return err
}
@ -88,34 +119,14 @@ func (r *retryJoiner) retryJoin() error {
r.logger.Printf("[INFO] agent: Joining %s cluster...", r.cluster)
attempt := 0
for {
var addrs []string
var err error
for _, addr := range r.addrs {
switch {
case strings.Contains(addr, "provider="):
servers, err := disco.Addrs(addr, r.logger)
if err != nil {
r.logger.Printf("[ERR] agent: Join %s: %s", r.cluster, err)
} else {
addrs = append(addrs, servers...)
r.logger.Printf("[INFO] agent: Discovered %s servers: %s", r.cluster, strings.Join(servers, " "))
}
default:
addrs = append(addrs, addr)
}
}
addrs := retryJoinAddrs(disco, r.cluster, r.addrs, r.logger)
if len(addrs) > 0 {
n, err := r.join(addrs)
if err == nil {
r.logger.Printf("[INFO] agent: Join %s completed. Synced with %d initial agents", r.cluster, n)
return nil
}
}
if len(addrs) == 0 {
} else if len(addrs) == 0 {
err = fmt.Errorf("No servers to join")
}

View File

@ -1,20 +1,58 @@
package agent
import (
"reflect"
"bytes"
"log"
"testing"
discover "github.com/hashicorp/go-discover"
"github.com/stretchr/testify/require"
)
func TestGoDiscoverRegistration(t *testing.T) {
d, err := discover.New()
if err != nil {
t.Fatal(err)
}
got := d.Names()
want := []string{"aliyun", "aws", "azure", "digitalocean", "gce", "mdns", "os", "packet", "scaleway", "softlayer", "triton", "vsphere"}
if !reflect.DeepEqual(got, want) {
t.Fatalf("got go-discover providers %v want %v", got, want)
func TestAgentRetryNewDiscover(t *testing.T) {
d, err := newDiscover()
require.NoError(t, err)
expected := []string{
"aliyun", "aws", "azure", "digitalocean", "gce", "k8s", "mdns",
"os", "packet", "scaleway", "softlayer", "triton", "vsphere",
}
require.Equal(t, expected, d.Names())
}
func TestAgentRetryJoinAddrs(t *testing.T) {
d, err := newDiscover()
require.NoError(t, err)
tests := []struct {
name string
input []string
expected []string
}{
{"handles nil", nil, []string{}},
{"handles empty input", []string{}, []string{}},
{"handles one element",
[]string{"192.168.0.12"},
[]string{"192.168.0.12"},
},
{"handles two elements",
[]string{"192.168.0.12", "192.168.0.13"},
[]string{"192.168.0.12", "192.168.0.13"},
},
{"tries to resolve aws things, which fails but that is fine",
[]string{"192.168.0.12", "provider=aws region=eu-west-1 tag_key=consul tag_value=tag access_key_id=a secret_access_key=a"},
[]string{"192.168.0.12"},
},
}
for i, test := range tests {
t.Run(test.name, func(t *testing.T) {
var buf bytes.Buffer
logger := log.New(&buf, "logger: ", log.Lshortfile)
require.Equal(t, test.expected, retryJoinAddrs(d, "LAN", test.input, logger), buf.String())
if i == 4 {
require.Contains(t, buf.String(), `Using provider "aws"`)
}
})
}
t.Run("handles nil discover", func(t *testing.T) {
require.Equal(t, []string{}, retryJoinAddrs(nil, "LAN", []string{"a"}, nil))
})
}

View File

@ -0,0 +1,9 @@
package structs
type SignedResponse struct {
IssuedCert IssuedCert `json:",omitempty"`
ConnectCARoots IndexedCARoots `json:",omitempty"`
ManualCARoots []string `json:",omitempty"`
GossipKey string `json:",omitempty"`
VerifyServerHostname bool `json:",omitempty"`
}

View File

@ -140,8 +140,13 @@ type IssuedCert struct {
// Service is the name of the service for which the cert was issued.
// ServiceURI is the cert URI value.
Service string
ServiceURI string
Service string `json:",omitempty"`
ServiceURI string `json:",omitempty"`
// Agent is the name of the node for which the cert was issued.
// AgentURI is the cert URI value.
Agent string `json:",omitempty"`
AgentURI string `json:",omitempty"`
// ValidAfter and ValidBefore are the validity periods for the
// certificate.

View File

@ -104,7 +104,7 @@ func NewTestAgent(t *testing.T, name string, hcl string) *TestAgent {
func NewUnstartedAgent(t *testing.T, name string, hcl string) (*Agent, error) {
c := TestConfig(config.Source{Name: name, Format: "hcl", Data: hcl})
a, err := New(c)
a, err := New(c, nil)
if err != nil {
return nil, err
}
@ -147,16 +147,17 @@ func (a *TestAgent) Start(t *testing.T) *TestAgent {
writeKey(a.Key, SerfWANKeyring)
}
agent, err := New(a.Config)
require.NoError(err, fmt.Sprintf("Error creating agent: %s", err))
logOutput := a.LogOutput
if logOutput == nil {
logOutput = os.Stderr
}
agentLogger := log.New(logOutput, a.Name+" - ", log.LstdFlags|log.Lmicroseconds)
agent, err := New(a.Config, agentLogger)
require.NoError(err, fmt.Sprintf("Error creating agent: %s", err))
agent.LogOutput = logOutput
agent.LogWriter = a.LogWriter
agent.logger = log.New(logOutput, a.Name+" - ", log.LstdFlags|log.Lmicroseconds)
agent.MemSink = metrics.NewInmemSink(1*time.Second, time.Minute)
// we need the err var in the next exit condition

View File

@ -214,7 +214,7 @@ func (c *cmd) run(args []string) int {
// Create the agent
c.UI.Output("Starting Consul agent...")
agent, err := agent.New(config)
agent, err := agent.New(config, c.logger)
if err != nil {
c.UI.Error(fmt.Sprintf("Error creating agent: %s", err))
return 1
@ -223,7 +223,66 @@ func (c *cmd) run(args []string) int {
agent.LogWriter = logWriter
agent.MemSink = memSink
if err := agent.Start(); err != nil {
segment := config.SegmentName
if config.ServerMode {
segment = "<all>"
}
c.UI.Info(fmt.Sprintf(" Version: '%s'", c.versionHuman))
c.UI.Info(fmt.Sprintf(" Node ID: '%s'", config.NodeID))
c.UI.Info(fmt.Sprintf(" Node name: '%s'", config.NodeName))
c.UI.Info(fmt.Sprintf(" Datacenter: '%s' (Segment: '%s')", config.Datacenter, segment))
c.UI.Info(fmt.Sprintf(" Server: %v (Bootstrap: %v)", config.ServerMode, config.Bootstrap))
c.UI.Info(fmt.Sprintf(" Client Addr: %v (HTTP: %d, HTTPS: %d, gRPC: %d, DNS: %d)", config.ClientAddrs,
config.HTTPPort, config.HTTPSPort, config.GRPCPort, config.DNSPort))
c.UI.Info(fmt.Sprintf(" Cluster Addr: %v (LAN: %d, WAN: %d)", config.AdvertiseAddrLAN,
config.SerfPortLAN, config.SerfPortWAN))
c.UI.Info(fmt.Sprintf(" Encrypt: Gossip: %v, TLS-Outgoing: %v, TLS-Incoming: %v, Auto-Encrypt-TLS: %t",
config.EncryptKey, config.VerifyOutgoing, config.VerifyIncoming, config.AutoEncryptTLS || config.AutoEncryptAllowTLS))
// Enable log streaming
c.UI.Info("")
c.UI.Output("Log data will now stream in as it occurs:\n")
logGate.Flush()
// wait for signal
signalCh := make(chan os.Signal, 10)
stopCh := make(chan struct{})
signal.Notify(signalCh, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP, syscall.SIGPIPE)
go func() {
for {
var sig os.Signal
select {
case s := <-signalCh:
sig = s
case <-stopCh:
return
}
switch sig {
case syscall.SIGPIPE:
continue
case syscall.SIGHUP:
err := fmt.Errorf("cannot reload before agent started")
c.logger.Printf("[ERR] agent: Caught signal: %s err: %s\n", sig, err)
default:
c.logger.Println("[INFO] agent: Caught signal: ", sig)
agent.InterruptStartCh <- struct{}{}
return
}
}
}()
err = agent.Start()
signal.Stop(signalCh)
select {
case stopCh <- struct{}{}:
default:
}
if err != nil {
c.UI.Error(fmt.Sprintf("Error starting agent: %s", err))
return 1
}
@ -249,31 +308,10 @@ func (c *cmd) run(args []string) int {
// Let the agent know we've finished registration
agent.StartSync()
segment := config.SegmentName
if config.ServerMode {
segment = "<all>"
}
c.UI.Output("Consul agent running!")
c.UI.Info(fmt.Sprintf(" Version: '%s'", c.versionHuman))
c.UI.Info(fmt.Sprintf(" Node ID: '%s'", config.NodeID))
c.UI.Info(fmt.Sprintf(" Node name: '%s'", config.NodeName))
c.UI.Info(fmt.Sprintf(" Datacenter: '%s' (Segment: '%s')", config.Datacenter, segment))
c.UI.Info(fmt.Sprintf(" Server: %v (Bootstrap: %v)", config.ServerMode, config.Bootstrap))
c.UI.Info(fmt.Sprintf(" Client Addr: %v (HTTP: %d, HTTPS: %d, gRPC: %d, DNS: %d)", config.ClientAddrs,
config.HTTPPort, config.HTTPSPort, config.GRPCPort, config.DNSPort))
c.UI.Info(fmt.Sprintf(" Cluster Addr: %v (LAN: %d, WAN: %d)", config.AdvertiseAddrLAN,
config.SerfPortLAN, config.SerfPortWAN))
c.UI.Info(fmt.Sprintf(" Encrypt: Gossip: %v, TLS-Outgoing: %v, TLS-Incoming: %v",
agent.GossipEncrypted(), config.VerifyOutgoing, config.VerifyIncoming))
// Enable log streaming
c.UI.Info("")
c.UI.Output("Log data will now stream in as it occurs:\n")
logGate.Flush()
// wait for signal
signalCh := make(chan os.Signal, 10)
signalCh = make(chan os.Signal, 10)
signal.Notify(signalCh, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP, syscall.SIGPIPE)
for {

View File

@ -7,6 +7,7 @@ import (
"github.com/hashicorp/consul/command/flags"
"github.com/hashicorp/consul/command/tls"
"github.com/hashicorp/consul/tlsutil"
"github.com/mitchellh/cli"
)
@ -61,12 +62,12 @@ func (c *cmd) Run(args []string) int {
return 1
}
sn, err := tls.GenerateSerialNumber()
sn, err := tlsutil.GenerateSerialNumber()
if err != nil {
c.UI.Error(err.Error())
return 1
}
s, pk, err := tls.GeneratePrivateKey()
s, pk, err := tlsutil.GeneratePrivateKey()
if err != nil {
c.UI.Error(err.Error())
}
@ -74,7 +75,7 @@ func (c *cmd) Run(args []string) int {
if c.constraint {
constraints = append(c.additionalConstraints, []string{c.domain, "localhost"}...)
}
ca, err := tls.GenerateCA(s, sn, c.days, constraints)
ca, err := tlsutil.GenerateCA(s, sn, c.days, constraints)
if err != nil {
c.UI.Error(err.Error())
}

View File

@ -11,6 +11,7 @@ import (
"github.com/hashicorp/consul/command/flags"
"github.com/hashicorp/consul/command/tls"
"github.com/hashicorp/consul/tlsutil"
"github.com/mitchellh/cli"
)
@ -154,25 +155,25 @@ func (c *cmd) Run(args []string) int {
}
c.UI.Info("==> Using " + caFile + " and " + keyFile)
signer, err := tls.ParseSigner(string(key))
signer, err := tlsutil.ParseSigner(string(key))
if err != nil {
c.UI.Error(err.Error())
return 1
}
sn, err := tls.GenerateSerialNumber()
sn, err := tlsutil.GenerateSerialNumber()
if err != nil {
c.UI.Error(err.Error())
return 1
}
pub, priv, err := tls.GenerateCert(signer, string(cert), sn, name, c.days, DNSNames, IPAddresses, extKeyUsage)
pub, priv, err := tlsutil.GenerateCert(signer, string(cert), sn, name, c.days, DNSNames, IPAddresses, extKeyUsage)
if err != nil {
c.UI.Error(err.Error())
return 1
}
if err = tls.Verify(string(cert), pub, name); err != nil {
if err = tlsutil.Verify(string(cert), pub, name); err != nil {
c.UI.Error("==> " + err.Error())
return 1
}

View File

@ -306,7 +306,7 @@ func (s *Service) Ready() bool {
return s.tlsCfg.Ready()
}
// ReadyWait returns a chan that is closed when the the Service becomes ready
// ReadyWait returns a chan that is closed when the Service becomes ready
// for use for the first time. Note that if the Service is ready when it is
// called it returns a nil chan. Ready means that it has root and leaf
// certificates configured which we assume are valid. The service may

View File

@ -0,0 +1,23 @@
-----BEGIN CERTIFICATE-----
MIIDyTCCArGgAwIBAgIURPvvB7dOIjTd54ojjHIw2imSpEgwDQYJKoZIhvcNAQEL
BQAwgZgxCzAJBgNVBAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEBxMNU2FuIEZy
YW5jaXNjbzEcMBoGA1UEChMTSGFzaGlDb3JwIFRlc3QgQ2VydDEMMAoGA1UECxMD
RGV2MRYwFAYDVQQDEw10ZXN0LmludGVybmFsMSAwHgYJKoZIhvcNAQkBFhF0ZXN0
QGludGVybmFsLmNvbTAeFw0xOTA1MjAxMjE5MDBaFw0yOTA1MTcxMjE5MDBaMAAw
ggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQC9G2dzg4w9ZqgiINgZJXDV
7ueWUE42wf6qS8WjdPZVci+ondLI4QTvbGE5wwy5EF/GTRTnVQup3VK8axaNqDqB
ThBrh7MUMeUgqhnElKxC7tgJOxvKE6JpyAz+e2jXwYMx8wFxDNd9Ve36yA1R6cc1
T/lupisRl6ARgNOXb/l89fkpM8aHKpwWFFaoXxabsH2Jgnfai2wXLmf7vRnvAM7m
GwtioWRjt2UCOltZbh/AYL/HVpNeD/IVRaOUHdw72lpAwPHMhvEbqQQkklj4Aldz
/xs8U82XjyeDhAbeNqAWps2EjFwkmHy+dVSXK9a2hW+Qgw2CQq+By10DmR7rPZWl
AgMBAAGjgaEwgZ4wDgYDVR0PAQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMB
BggrBgEFBQcDAjAMBgNVHRMBAf8EAjAAMB0GA1UdDgQWBBTXwQO3LMRTMtINOYgD
sMREIqezCTAfBgNVHSMEGDAWgBSj+es5+q9t57ZWSVUogWXJARu4lTAfBgNVHREB
Af8EFTATghFzZXJ2ZXIuZGMxLmNvbnN1bDANBgkqhkiG9w0BAQsFAAOCAQEAHrvb
XYi87gpvyrXUCwihHCFDjKdvo+3b9a3MoFzn+/e3gcUwz9fLnup4kGq+Fh+iyitn
YtraUvFLb0QSZmROTRYRzGwmLLYYOguIcbadRKJkd3NkiJ5QDw4+S8OXE+1/hZpm
sjM3fnzQPhspxweSZZPtDvgI6JFA8IKCvh2jUfdKdWVfhBonEb/le2ox/7RfqQ+M
JZ7rorpvSCat+NOQKKIbIAlNcntflXGBE2ken/IHihtUREk5pYKpplMf4PqkyFkB
MuATqNoXuARYcN3u8HYWv6ewwHqBuUB86NurYArCw3Antl0hJBLDPQ7X8ukJZDoB
lhe56raXzfZxn9BIrw==
-----END CERTIFICATE-----

View File

@ -0,0 +1,27 @@
-----BEGIN RSA PRIVATE KEY-----
MIIEowIBAAKCAQEAvRtnc4OMPWaoIiDYGSVw1e7nllBONsH+qkvFo3T2VXIvqJ3S
yOEE72xhOcMMuRBfxk0U51ULqd1SvGsWjag6gU4Qa4ezFDHlIKoZxJSsQu7YCTsb
yhOiacgM/nto18GDMfMBcQzXfVXt+sgNUenHNU/5bqYrEZegEYDTl2/5fPX5KTPG
hyqcFhRWqF8Wm7B9iYJ32otsFy5n+70Z7wDO5hsLYqFkY7dlAjpbWW4fwGC/x1aT
Xg/yFUWjlB3cO9paQMDxzIbxG6kEJJJY+AJXc/8bPFPNl48ng4QG3jagFqbNhIxc
JJh8vnVUlyvWtoVvkIMNgkKvgctdA5ke6z2VpQIDAQABAoIBAEhSerYK0U+KOzMS
LJMxZn3q6FbsT31Ro/utDuD8klkLWn66HSsGI8UNRgG5PtxoDrjgMeZm+Bb2tfWp
Xu6+L+HT4SO+uhY3HtHnPAOaeGcAwU6GdJJMAAlEoPOSbv5U27/2l86Mgr1EZ8dJ
Sw7QwYb6FwLiOgZ4XpdTFZkNEfVf1CeZSlxTIpvlJMJJvVg7crhVnS4boI6d8z24
b0LerEbvVC97N++sKhSGtSF/QX0wW7BkbTquruaRbWsL/e0o8Bydy1jRVFLTet4l
GO9MNcN8Cdt7eOptsK00Ma9S9AZfTvaXdrPERimJ0JDtqjMSyfly6M7fK7Yx2gUn
HcPSOAECgYEAy8IQ5+jw16qJDfAhc9jer5i0E/fj2xohYfUF6IniGFYT9Ih6D/2T
D55W1m8PSoNM5gfYkLZ6dvjzcF6kDCD5vppVr07B3RlS7caMkM1GGmLxOgX/YIa7
OpEzaTr1kIefvsrR4c4b2wBzZm1XuGDE/lw5yCQmoBAbzyq4X2Mlr+UCgYEA7Zey
0vUxCPxBp+spdPuZuH5oW9b2L5D2c87em7+tObuf20wQic7vXfZ4TDTDojSkeEJk
S7aUKEivzMePdNJ+QYxK+/0w4klYUS62Xk5422zg90dvQ5P86aYHagrQ61jmpUIe
rthLgP+fHtXEJyb2Ud2fc+yx+Sn38ptWi52AcsECgYBIQe0O4Ouv/5kU6Uhjtc/a
w7NwtWMHiy+1dlf/DA6zBKuU92UaEJm7WzJ+Xuo0SXXWyYeCYkPxtv1VpypT5snK
Tx79yVc0mktvaQ7mNvWaW+Yh5oiW6ZCyB1YNBkyZUY9T8McXZak5M+K4uyP1jdOu
RHR1RmSwNKY4BHVX1mhCBQKBgQCNmuBFw0Uaad5nykzHID2aLBzev9uytd9tXlpm
0XLY+e5osYkZ2W/ovMEuCjSfNGjiFA/a4FKlP80na7kgk2QUhR0b2ueLttMgb4rZ
4kM95EKgnr69tDIEv6OjnBawbifpTuMiql813yRjKFzkSOB+ImyqluPr8QuKWPLX
+2NXAQKBgGAij1zH8M6vV486KA5l4PUyN9iaiakC262IH9bX3js93Wfwd1bGDtQA
rgxbhQNYifx3ID3HqxyJxWSennczHfEPTfkU4BooFzlqxhKPi+TT+Xd+N+fY5hGT
S8lS78V8hT8eOEIgGbsM5bckHQqNapGRCNsImIsNftLwmUCSQMDG
-----END RSA PRIVATE KEY-----

View File

@ -0,0 +1,18 @@
-----BEGIN CERTIFICATE-----
MIIC8TCCApegAwIBAgIQAZlDb2Sj7IqEyY7VVYVD2DAKBggqhkjOPQQDAjCBuTEL
MAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQHEw1TYW4gRnJhbmNpc2Nv
MRowGAYDVQQJExExMDEgU2Vjb25kIFN0cmVldDEOMAwGA1UEERMFOTQxMDUxFzAV
BgNVBAoTDkhhc2hpQ29ycCBJbmMuMUAwPgYDVQQDEzdDb25zdWwgQWdlbnQgQ0Eg
MTM3OTAxMDI0NDM2OTM2OTAyMTIyMzQwODcxMDg3MzI1OTIxMDkwMB4XDTE5MDYy
NjE1MDU1MVoXDTE5MDYyNjE1MDU1MVowGTEXMBUGA1UEAxMOY2xpLmRjMS5jb25z
dWwwWTATBgcqhkjOPQIBBggqhkjOPQMBBwNCAAQwvl+mLmsP78y0wxTP8HMLH2k3
DJ0xWeO7jJW6XapkYJfYjSWcMbpJXIP4cFMgSRkmc2u69lnHAcxlJ2I67QoVo4IB
HjCCARowDgYDVR0PAQH/BAQDAgWgMAwGA1UdEwEB/wQCMAAwaAYDVR0OBGEEX2E2
OjQyOmMzOjFlOmQ4OjllOjM4OjM4OmM1OmQxOjgyOmI3OjU5OmY4OmRjOmM3Ojdl
OjNjOmE0OmY5OjQ0OjAxOjgwOjkxOjUwOjk2OjViOmZkOmY5OjUxOmI5OjZiMGoG
A1UdIwRjMGGAXzA1OmY3OmVhOjQ5OjFmOjNlOjVkOjAwOmY1OmJkOjk5OmJjOmU3
OmUyOmQ4OjVhOjZhOjBmOjRhOjYxOjY5OjY0OjA3OjVhOmQ1Ojc4OjVjOjFmOjhi
OjIzOmMwOmM0MCQGA1UdEQQdMBuCDmNsaS5kYzEuY29uc3Vsgglsb2NhbGhvc3Qw
CgYIKoZIzj0EAwIDSAAwRQIhANLqOUqIqejDqsvBpm9BPDgyHBqyHUClg43zAICR
z9mXAiAhKVL/+RLOtYnlyJ3CVgvPuDMT20eJAFUitgCuy8AJzQ==
-----END CERTIFICATE-----

View File

@ -0,0 +1,5 @@
-----BEGIN EC PRIVATE KEY-----
MHcCAQEEIPciLigI6zq/mTMJ5koV6BIymAUAvx1t084Zc5Bm1pCnoAoGCCqGSM49
AwEHoUQDQgAEML5fpi5rD+/MtMMUz/BzCx9pNwydMVnju4yVul2qZGCX2I0lnDG6
SVyD+HBTIEkZJnNruvZZxwHMZSdiOu0KFQ==
-----END EC PRIVATE KEY-----

View File

@ -4,13 +4,14 @@ import (
"crypto/tls"
"crypto/x509"
"fmt"
"io/ioutil"
"log"
"net"
"os"
"path/filepath"
"strings"
"sync"
"time"
"github.com/hashicorp/go-rootcerts"
)
// DCWrapper is a function that is used to wrap a non-TLS connection
@ -112,6 +113,10 @@ type Config struct {
// the server using the same TLS configuration as the agent (CA, cert,
// and key).
EnableAgentTLSForChecks bool
// AutoEncryptTLS opts the agent into provisioning agent
// TLS certificates.
AutoEncryptTLS bool
}
// KeyPair is used to open and parse a certificate and key file
@ -130,13 +135,31 @@ func SpecificDC(dc string, tlsWrap DCWrapper) Wrapper {
}
}
type autoEncrypt struct {
manualCAPems []string
connectCAPems []string
cert *tls.Certificate
verifyServerHostname bool
}
func (a *autoEncrypt) caPems() []string {
return append(a.manualCAPems, a.connectCAPems...)
}
type manual struct {
caPems []string
cert *tls.Certificate
}
// Configurator holds a Config and is responsible for generating all the
// *tls.Config necessary for Consul. Except the one in the api package.
type Configurator struct {
sync.RWMutex
base *Config
cert *tls.Certificate
cas *x509.CertPool
base *Config
autoEncrypt *autoEncrypt
manual *manual
caPool *x509.CertPool
logger *log.Logger
version int
}
@ -144,7 +167,7 @@ type Configurator struct {
// NewConfigurator creates a new Configurator and sets the provided
// configuration.
func NewConfigurator(config Config, logger *log.Logger) (*Configurator, error) {
c := &Configurator{logger: logger}
c := &Configurator{logger: logger, manual: &manual{}, autoEncrypt: &autoEncrypt{}}
err := c.Update(config)
if err != nil {
return nil, err
@ -152,33 +175,140 @@ func NewConfigurator(config Config, logger *log.Logger) (*Configurator, error) {
return c, nil
}
// CAPems returns the currently loaded CAs in PEM format.
func (c *Configurator) CAPems() []string {
c.RLock()
defer c.RUnlock()
return append(c.manual.caPems, c.autoEncrypt.caPems()...)
}
// ManualCAPems returns the currently loaded CAs in PEM format.
func (c *Configurator) ManualCAPems() []string {
c.RLock()
defer c.RUnlock()
return c.manual.caPems
}
// Update updates the internal configuration which is used to generate
// *tls.Config.
// This function acquires a write lock because it writes the new config.
func (c *Configurator) Update(config Config) error {
c.Lock()
// order of defers matters because log acquires a RLock()
defer c.log("Update")
defer c.Unlock()
cert, err := loadKeyPair(config.CertFile, config.KeyFile)
if err != nil {
return err
}
cas, err := loadCAs(config.CAFile, config.CAPath)
pems, err := loadCAs(config.CAFile, config.CAPath)
if err != nil {
return err
}
if err = c.check(config, cas, cert); err != nil {
pool, err := pool(append(pems, c.autoEncrypt.caPems()...))
if err != nil {
return err
}
if err = c.check(config, pool, cert); err != nil {
return err
}
c.Lock()
c.base = &config
c.cert = cert
c.cas = cas
c.manual.cert = cert
c.manual.caPems = pems
c.caPool = pool
c.version++
c.Unlock()
c.log("Update")
return nil
}
func (c *Configurator) check(config Config, cas *x509.CertPool, cert *tls.Certificate) error {
// UpdateAutoEncryptCA updates the autoEncrypt.caPems. This is supposed to be called
// from the server in order to be able to accept TLS connections with TLS
// certificates.
// Or it is being called on the client side when CA changes are detected.
func (c *Configurator) UpdateAutoEncryptCA(connectCAPems []string) error {
c.Lock()
// order of defers matters because log acquires a RLock()
defer c.log("UpdateAutoEncryptCA")
defer c.Unlock()
pool, err := pool(append(c.manual.caPems, append(c.autoEncrypt.manualCAPems, connectCAPems...)...))
if err != nil {
c.RUnlock()
return err
}
if err = c.check(*c.base, pool, c.manual.cert); err != nil {
c.RUnlock()
return err
}
c.autoEncrypt.connectCAPems = connectCAPems
c.caPool = pool
c.version++
return nil
}
// UpdateAutoEncryptCert
func (c *Configurator) UpdateAutoEncryptCert(pub, priv string) error {
// order of defers matters because log acquires a RLock()
defer c.log("UpdateAutoEncryptCert")
cert, err := tls.X509KeyPair([]byte(pub), []byte(priv))
if err != nil {
return fmt.Errorf("Failed to load cert/key pair: %v", err)
}
c.Lock()
defer c.Unlock()
c.autoEncrypt.cert = &cert
c.version++
return nil
}
// UpdateAutoEncrypt sets everything under autoEncrypt. This is being called on the
// client when it received its cert from AutoEncrypt endpoint.
func (c *Configurator) UpdateAutoEncrypt(manualCAPems, connectCAPems []string, pub, priv string, verifyServerHostname bool) error {
// order of defers matters because log acquires a RLock()
defer c.log("UpdateAutoEncrypt")
cert, err := tls.X509KeyPair([]byte(pub), []byte(priv))
if err != nil {
return fmt.Errorf("Failed to load cert/key pair: %v", err)
}
c.Lock()
defer c.Unlock()
pool, err := pool(append(c.manual.caPems, append(manualCAPems, connectCAPems...)...))
if err != nil {
return err
}
c.autoEncrypt.manualCAPems = manualCAPems
c.autoEncrypt.connectCAPems = connectCAPems
c.autoEncrypt.cert = &cert
c.caPool = pool
c.autoEncrypt.verifyServerHostname = verifyServerHostname
c.version++
return nil
}
func (c *Configurator) Base() Config {
c.RLock()
defer c.RUnlock()
return *c.base
}
func pool(pems []string) (*x509.CertPool, error) {
pool := x509.NewCertPool()
for _, pem := range pems {
if !pool.AppendCertsFromPEM([]byte(pem)) {
return nil, fmt.Errorf("Couldn't parse PEM %s", pem)
}
}
if len(pool.Subjects()) == 0 {
return nil, nil
}
return pool, nil
}
func (c *Configurator) check(config Config, pool *x509.CertPool, cert *tls.Certificate) error {
// Check if a minimum TLS version was set
if config.TLSMinVersion != "" {
if _, ok := TLSLookup[config.TLSMinVersion]; !ok {
@ -187,25 +317,41 @@ func (c *Configurator) check(config Config, cas *x509.CertPool, cert *tls.Certif
}
// Ensure we have a CA if VerifyOutgoing is set
if config.VerifyOutgoing && cas == nil {
if config.VerifyOutgoing && pool == nil {
return fmt.Errorf("VerifyOutgoing set, and no CA certificate provided!")
}
// Ensure we have a CA and cert if VerifyIncoming is set
if config.VerifyIncoming || config.VerifyIncomingRPC || config.VerifyIncomingHTTPS {
if cas == nil {
if config.anyVerifyIncoming() {
if pool == nil {
return fmt.Errorf("VerifyIncoming set, and no CA certificate provided!")
}
if cert == nil {
if cert == nil || cert.Certificate == nil {
return fmt.Errorf("VerifyIncoming set, and no Cert/Key pair provided!")
}
}
return nil
}
func (c Config) anyVerifyIncoming() bool {
return c.baseVerifyIncoming() || c.VerifyIncomingRPC || c.VerifyIncomingHTTPS
}
func (c Config) verifyIncomingRPC() bool {
return c.baseVerifyIncoming() || c.VerifyIncomingRPC
}
func (c Config) verifyIncomingHTTPS() bool {
return c.baseVerifyIncoming() || c.VerifyIncomingHTTPS
}
func (c *Config) baseVerifyIncoming() bool {
return c.VerifyIncoming
}
func loadKeyPair(certFile, keyFile string) (*tls.Certificate, error) {
if certFile == "" || keyFile == "" {
return nil, nil
return &tls.Certificate{}, nil
}
cert, err := tls.LoadX509KeyPair(certFile, keyFile)
if err != nil {
@ -214,33 +360,64 @@ func loadKeyPair(certFile, keyFile string) (*tls.Certificate, error) {
return &cert, nil
}
func loadCAs(caFile, caPath string) (*x509.CertPool, error) {
if caFile != "" {
return rootcerts.LoadCAFile(caFile)
} else if caPath != "" {
pool, err := rootcerts.LoadCAPath(caPath)
if err != nil {
return nil, err
}
// make sure to not return an empty pool because this is not
// the users intention when providing a path for CAs.
if len(pool.Subjects()) == 0 {
return nil, fmt.Errorf("Error loading CA: path %q has no CAs", caPath)
}
return pool, nil
func loadCAs(caFile, caPath string) ([]string, error) {
if caFile == "" && caPath == "" {
return nil, nil
}
return nil, nil
pems := []string{}
readFn := func(path string) error {
pem, err := ioutil.ReadFile(path)
if err != nil {
return fmt.Errorf("Error loading from %s: %s", path, err)
}
pems = append(pems, string(pem))
return nil
}
walkFn := func(path string, info os.FileInfo, err error) error {
if err != nil {
return err
}
if !info.IsDir() {
if err := readFn(path); err != nil {
return err
}
}
return nil
}
if caFile != "" {
err := readFn(caFile)
if err != nil {
return pems, err
}
} else if caPath != "" {
err := filepath.Walk(caPath, walkFn)
if err != nil {
return pems, err
}
if len(pems) == 0 {
return pems, fmt.Errorf("Error loading from CAPath: no CAs found")
}
}
return pems, nil
}
// commonTLSConfig generates a *tls.Config from the base configuration the
// Configurator has. It accepts an additional flag in case a config is needed
// for incoming TLS connections.
// This function acquires a read lock because it reads from the config.
func (c *Configurator) commonTLSConfig(additionalVerifyIncomingFlag bool) *tls.Config {
func (c *Configurator) commonTLSConfig(verifyIncoming bool) *tls.Config {
// this needs to be outside of RLock because it acquires an RLock itself
verifyServerHostname := c.VerifyServerHostname()
c.RLock()
defer c.RUnlock()
tlsConfig := &tls.Config{
InsecureSkipVerify: !c.base.VerifyServerHostname,
InsecureSkipVerify: !verifyServerHostname,
}
// Set the cipher suites
@ -250,15 +427,28 @@ func (c *Configurator) commonTLSConfig(additionalVerifyIncomingFlag bool) *tls.C
tlsConfig.PreferServerCipherSuites = c.base.PreferServerCipherSuites
// GetCertificate is used when acting as a server and responding to
// client requests. Always return the manually configured cert, because
// on the server this is all we have. And on the client, this is the
// only sensitive option.
tlsConfig.GetCertificate = func(*tls.ClientHelloInfo) (*tls.Certificate, error) {
return c.cert, nil
}
tlsConfig.GetClientCertificate = func(*tls.CertificateRequestInfo) (*tls.Certificate, error) {
return c.cert, nil
return c.manual.cert, nil
}
tlsConfig.ClientCAs = c.cas
tlsConfig.RootCAs = c.cas
// GetClientCertificate is used when acting as a client and responding
// to a server requesting a certificate. Return the autoEncrypt certificate
// if possible, otherwise default to the manually provisioned one.
tlsConfig.GetClientCertificate = func(*tls.CertificateRequestInfo) (*tls.Certificate, error) {
cert := c.autoEncrypt.cert
if cert == nil {
cert = c.manual.cert
}
return cert, nil
}
tlsConfig.ClientCAs = c.caPool
tlsConfig.RootCAs = c.caPool
// This is possible because TLSLookup also contains "" with golang's
// default (tls10). And because the initial check makes sure the
@ -266,7 +456,7 @@ func (c *Configurator) commonTLSConfig(additionalVerifyIncomingFlag bool) *tls.C
tlsConfig.MinVersion = TLSLookup[c.base.TLSMinVersion]
// Set ClientAuth if necessary
if c.base.VerifyIncoming || additionalVerifyIncomingFlag {
if verifyIncoming {
tlsConfig.ClientAuth = tls.RequireAndVerifyClientCert
}
@ -274,31 +464,63 @@ func (c *Configurator) commonTLSConfig(additionalVerifyIncomingFlag bool) *tls.C
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) outgoingRPCTLSDisabled() bool {
func (c *Configurator) VerifyIncomingRPC() bool {
c.RLock()
defer c.RUnlock()
return c.cas == nil && !c.base.VerifyOutgoing
return c.base.verifyIncomingRPC()
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) someValuesFromConfig() (bool, bool, string) {
func (c *Configurator) outgoingRPCTLSDisabled() bool {
c.RLock()
defer c.RUnlock()
return c.base.VerifyServerHostname, c.base.VerifyOutgoing, c.base.Domain
// if AutoEncrypt enabled, always use TLS
if c.base.AutoEncryptTLS {
return false
}
// if CAs are provided or VerifyOutgoing is set, use TLS
if c.caPool != nil || c.base.VerifyOutgoing {
return false
}
return true
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) verifyOutgoing() bool {
c.RLock()
defer c.RUnlock()
// If AutoEncryptTLS is enabled and there is a CA, then verify
// outgoing.
if c.base.AutoEncryptTLS && c.caPool != nil {
return true
}
return c.base.VerifyOutgoing
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) domain() string {
c.RLock()
defer c.RUnlock()
return c.base.Domain
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) verifyIncomingRPC() bool {
c.RLock()
defer c.RUnlock()
return c.base.VerifyIncomingRPC
return c.base.verifyIncomingRPC()
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) verifyIncomingHTTPS() bool {
c.RLock()
defer c.RUnlock()
return c.base.VerifyIncomingHTTPS
return c.base.verifyIncomingHTTPS()
}
// This function acquires a read lock because it reads from the config.
@ -318,6 +540,13 @@ func (c *Configurator) serverNameOrNodeName() string {
return c.base.NodeName
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) VerifyServerHostname() bool {
c.RLock()
defer c.RUnlock()
return c.base.VerifyServerHostname || c.autoEncrypt.verifyServerHostname
}
// IncomingRPCConfig generates a *tls.Config for incoming RPC connections.
func (c *Configurator) IncomingRPCConfig() *tls.Config {
c.log("IncomingRPCConfig")
@ -328,6 +557,20 @@ func (c *Configurator) IncomingRPCConfig() *tls.Config {
return config
}
// IncomingInsecureRPCConfig means that it doesn't verify incoming even thought
// it might have been configured. This is only supposed to be used by the
// servers for the insecure RPC server. At the time of writing only the
// AutoEncrypt.Sign call is supported on that server. And it might be the only
// usecase ever.
func (c *Configurator) IncomingInsecureRPCConfig() *tls.Config {
c.log("IncomingInsecureRPCConfig")
config := c.commonTLSConfig(false)
config.GetConfigForClient = func(*tls.ClientHelloInfo) (*tls.Config, error) {
return c.IncomingInsecureRPCConfig(), nil
}
return config
}
// IncomingHTTPSConfig generates a *tls.Config for incoming HTTPS connections.
func (c *Configurator) IncomingHTTPSConfig() *tls.Config {
c.log("IncomingHTTPSConfig")
@ -383,6 +626,27 @@ func (c *Configurator) OutgoingRPCWrapper() DCWrapper {
}
}
// AutoEncryptCertNotAfter returns NotAfter from the auto_encrypt cert. In case
// there is no cert, it will return a time in the past.
func (c *Configurator) AutoEncryptCertNotAfter() time.Time {
c.RLock()
tlsCert := c.autoEncrypt.cert
c.RUnlock()
if tlsCert == nil {
return time.Now().AddDate(0, 0, -1)
}
cert, err := x509.ParseCertificate(tlsCert.Certificate[0])
if err != nil {
return time.Now().AddDate(0, 0, -1)
}
return cert.NotAfter
}
// AutoEncryptCertExpired returns if the auto_encrypt cert is expired.
func (c *Configurator) AutoEncryptCertExpired() bool {
return c.AutoEncryptCertNotAfter().Before(time.Now())
}
// This function acquires a read lock because it reads from the config.
func (c *Configurator) log(name string) {
if c.logger != nil {
@ -403,18 +667,17 @@ func (c *Configurator) log(name string) {
// no longer supports this mode of operation, we have to do it
// manually.
func (c *Configurator) wrapTLSClient(dc string, conn net.Conn) (net.Conn, error) {
var err error
var tlsConn *tls.Conn
config := c.OutgoingRPCConfig()
verifyServerHostname, verifyOutgoing, domain := c.someValuesFromConfig()
verifyServerHostname := c.VerifyServerHostname()
verifyOutgoing := c.verifyOutgoing()
domain := c.domain()
if verifyServerHostname {
// Strip the trailing '.' from the domain if any
domain = strings.TrimSuffix(domain, ".")
config.ServerName = "server." + dc + "." + domain
}
tlsConn = tls.Client(conn, config)
tlsConn := tls.Client(conn, config)
// If crypto/tls is doing verification, there's no need to do
// our own.
@ -427,7 +690,8 @@ func (c *Configurator) wrapTLSClient(dc string, conn net.Conn) (net.Conn, error)
return tlsConn, nil
}
if err = tlsConn.Handshake(); err != nil {
err := tlsConn.Handshake()
if err != nil {
tlsConn.Close()
return nil, err
}

View File

@ -293,34 +293,40 @@ func TestConfigurator_loadKeyPair(t *testing.T) {
cert, key string
shoulderr bool
isnil bool
isempty bool
}
variants := []variant{
{"", "", false, true},
{"bogus", "", false, true},
{"", "bogus", false, true},
{"../test/key/ourdomain.cer", "", false, true},
{"", "../test/key/ourdomain.key", false, true},
{"bogus", "bogus", true, true},
{"", "", false, false, true},
{"bogus", "", false, false, true},
{"", "bogus", false, false, true},
{"../test/key/ourdomain.cer", "", false, false, true},
{"", "../test/key/ourdomain.key", false, false, true},
{"bogus", "bogus", true, true, false},
{"../test/key/ourdomain.cer", "../test/key/ourdomain.key",
false, false},
false, false, false},
}
for _, v := range variants {
for i, v := range variants {
info := fmt.Sprintf("case %d", i)
cert1, err1 := loadKeyPair(v.cert, v.key)
config := &Config{CertFile: v.cert, KeyFile: v.key}
cert2, err2 := config.KeyPair()
if v.shoulderr {
require.Error(t, err1)
require.Error(t, err2)
require.Error(t, err1, info)
require.Error(t, err2, info)
} else {
require.NoError(t, err1)
require.NoError(t, err2)
require.NoError(t, err1, info)
require.NoError(t, err2, info)
}
if v.isempty {
require.Empty(t, cert1.Certificate, info)
require.Empty(t, cert2.Certificate, info)
}
if v.isnil {
require.Nil(t, cert1)
require.Nil(t, cert2)
require.Nil(t, cert1, info)
require.Nil(t, cert2, info)
} else {
require.NotNil(t, cert1)
require.NotNil(t, cert2)
require.NotNil(t, cert1, info)
require.NotNil(t, cert2, info)
}
}
}
@ -359,43 +365,43 @@ func TestConfigurator_ErrorPropagation(t *testing.T) {
certfile := "../test/key/ourdomain.cer"
keyfile := "../test/key/ourdomain.key"
variants := []variant{
{Config{}, false, false},
{Config{TLSMinVersion: "tls9"}, true, false},
{Config{TLSMinVersion: ""}, false, false},
{Config{TLSMinVersion: "tls10"}, false, false},
{Config{TLSMinVersion: "tls11"}, false, false},
{Config{TLSMinVersion: "tls12"}, false, false},
{Config{VerifyOutgoing: true, CAFile: "", CAPath: ""}, true, false},
{Config{VerifyOutgoing: false, CAFile: "", CAPath: ""}, false, false},
{Config{}, false, false}, // 1
{Config{TLSMinVersion: "tls9"}, true, false}, // 1
{Config{TLSMinVersion: ""}, false, false}, // 2
{Config{TLSMinVersion: "tls10"}, false, false}, // 3
{Config{TLSMinVersion: "tls11"}, false, false}, // 4
{Config{TLSMinVersion: "tls12"}, false, false}, // 5
{Config{VerifyOutgoing: true, CAFile: "", CAPath: ""}, true, false}, // 6
{Config{VerifyOutgoing: false, CAFile: "", CAPath: ""}, false, false}, // 7
{Config{VerifyOutgoing: false, CAFile: cafile, CAPath: ""},
false, false},
false, false}, // 8
{Config{VerifyOutgoing: false, CAFile: "", CAPath: capath},
false, false},
false, false}, // 9
{Config{VerifyOutgoing: false, CAFile: cafile, CAPath: capath},
false, false},
false, false}, // 10
{Config{VerifyOutgoing: true, CAFile: cafile, CAPath: ""},
false, false},
false, false}, // 11
{Config{VerifyOutgoing: true, CAFile: "", CAPath: capath},
false, false},
false, false}, // 12
{Config{VerifyOutgoing: true, CAFile: cafile, CAPath: capath},
false, false},
{Config{VerifyIncoming: true, CAFile: "", CAPath: ""}, true, false},
false, false}, // 13
{Config{VerifyIncoming: true, CAFile: "", CAPath: ""}, true, false}, // 14
{Config{VerifyIncomingRPC: true, CAFile: "", CAPath: ""},
true, false},
true, false}, // 15
{Config{VerifyIncomingHTTPS: true, CAFile: "", CAPath: ""},
true, false},
{Config{VerifyIncoming: true, CAFile: cafile, CAPath: ""}, true, false},
{Config{VerifyIncoming: true, CAFile: "", CAPath: capath}, true, false},
true, false}, // 16
{Config{VerifyIncoming: true, CAFile: cafile, CAPath: ""}, true, false}, // 17
{Config{VerifyIncoming: true, CAFile: "", CAPath: capath}, true, false}, // 18
{Config{VerifyIncoming: true, CAFile: "", CAPath: capath,
CertFile: certfile, KeyFile: keyfile}, false, false},
{Config{CertFile: "bogus", KeyFile: "bogus"}, true, true},
{Config{CAFile: "bogus"}, true, true},
{Config{CAPath: "bogus"}, true, true},
CertFile: certfile, KeyFile: keyfile}, false, false}, // 19
{Config{CertFile: "bogus", KeyFile: "bogus"}, true, true}, // 20
{Config{CAFile: "bogus"}, true, true}, // 21
{Config{CAPath: "bogus"}, true, true}, // 22
}
c := &Configurator{}
c := Configurator{autoEncrypt: &autoEncrypt{}, manual: &manual{}}
for i, v := range variants {
info := fmt.Sprintf("case %d", i)
info := fmt.Sprintf("case %d, config: %+v", i, v.config)
_, err1 := NewConfigurator(v.config, nil)
err2 := c.Update(v.config)
@ -403,9 +409,11 @@ func TestConfigurator_ErrorPropagation(t *testing.T) {
if !v.excludeCheck {
cert, err := v.config.KeyPair()
require.NoError(t, err, info)
cas, _ := loadCAs(v.config.CAFile, v.config.CAPath)
pems, err := loadCAs(v.config.CAFile, v.config.CAPath)
require.NoError(t, err, info)
err3 = c.check(v.config, cas, cert)
pool, err := pool(pems)
require.NoError(t, err, info)
err3 = c.check(v.config, pool, cert)
}
if v.shouldErr {
require.Error(t, err1, info)
@ -461,18 +469,24 @@ func TestConfigurator_loadCAs(t *testing.T) {
{"../test/ca/root.cer", "../test/ca_path", false, false, 1},
}
for i, v := range variants {
cas, err := loadCAs(v.cafile, v.capath)
pems, err1 := loadCAs(v.cafile, v.capath)
pool, err2 := pool(pems)
info := fmt.Sprintf("case %d", i)
if v.shouldErr {
require.Error(t, err, info)
if err1 == nil && err2 == nil {
t.Fatal("An error is expected but got nil.")
}
} else {
require.NoError(t, err, info)
require.NoError(t, err1, info)
require.NoError(t, err2, info)
}
if v.isNil {
require.Nil(t, cas, info)
require.Nil(t, pool, info)
} else {
require.NotNil(t, cas, info)
require.Len(t, cas.Subjects(), v.count, info)
require.NotEmpty(t, pems, info)
require.NotNil(t, pool, info)
require.Len(t, pool.Subjects(), v.count, info)
require.Len(t, pems, v.count, info)
}
}
}
@ -526,16 +540,16 @@ func TestConfigurator_CommonTLSConfigGetClientCertificate(t *testing.T) {
cert, err := c.commonTLSConfig(false).GetCertificate(nil)
require.NoError(t, err)
require.Nil(t, cert)
require.Nil(t, cert.Certificate)
c.cert = &tls.Certificate{}
c.manual.cert = &tls.Certificate{}
cert, err = c.commonTLSConfig(false).GetCertificate(nil)
require.NoError(t, err)
require.Equal(t, c.cert, cert)
require.Equal(t, c.manual.cert, cert)
cert, err = c.commonTLSConfig(false).GetClientCertificate(nil)
require.NoError(t, err)
require.Equal(t, c.cert, cert)
require.Equal(t, c.manual.cert, cert)
}
func TestConfigurator_CommonTLSConfigCAs(t *testing.T) {
@ -544,9 +558,9 @@ func TestConfigurator_CommonTLSConfigCAs(t *testing.T) {
require.Nil(t, c.commonTLSConfig(false).ClientCAs)
require.Nil(t, c.commonTLSConfig(false).RootCAs)
c.cas = &x509.CertPool{}
require.Equal(t, c.cas, c.commonTLSConfig(false).ClientCAs)
require.Equal(t, c.cas, c.commonTLSConfig(false).RootCAs)
c.caPool = &x509.CertPool{}
require.Equal(t, c.caPool, c.commonTLSConfig(false).ClientCAs)
require.Equal(t, c.caPool, c.commonTLSConfig(false).RootCAs)
}
func TestConfigurator_CommonTLSConfigTLSMinVersion(t *testing.T) {
@ -565,67 +579,48 @@ func TestConfigurator_CommonTLSConfigTLSMinVersion(t *testing.T) {
}
func TestConfigurator_CommonTLSConfigVerifyIncoming(t *testing.T) {
c := Configurator{base: &Config{}}
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
type variant struct {
verify bool
additional bool
expected tls.ClientAuthType
verify bool
expected tls.ClientAuthType
}
variants := []variant{
{false, false, tls.NoClientCert},
{true, false, tls.RequireAndVerifyClientCert},
{false, true, tls.RequireAndVerifyClientCert},
{true, true, tls.RequireAndVerifyClientCert},
{true, tls.RequireAndVerifyClientCert},
{false, tls.NoClientCert},
}
for _, v := range variants {
c.base.VerifyIncoming = v.verify
require.Equal(t, v.expected,
c.commonTLSConfig(v.additional).ClientAuth)
require.Equal(t, v.expected, c.commonTLSConfig(v.verify).ClientAuth)
}
}
func TestConfigurator_OutgoingRPCTLSDisabled(t *testing.T) {
c := Configurator{base: &Config{}}
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
type variant struct {
verify bool
file string
path string
expected bool
verify bool
autoEncryptTLS bool
pool *x509.CertPool
expected bool
}
cafile := "../test/ca/root.cer"
capath := "../test/ca_path"
variants := []variant{
{false, "", "", true},
{false, cafile, "", false},
{false, "", capath, false},
{false, cafile, capath, false},
{true, "", "", false},
{true, cafile, "", false},
{true, "", capath, false},
{true, cafile, capath, false},
{false, false, nil, true},
{true, false, nil, false},
{false, true, nil, false},
{true, true, nil, false},
{false, false, &x509.CertPool{}, false},
{true, false, &x509.CertPool{}, false},
{false, true, &x509.CertPool{}, false},
{true, true, &x509.CertPool{}, false},
}
for i, v := range variants {
info := fmt.Sprintf("case %d", i)
cas, err := loadCAs(v.file, v.path)
require.NoError(t, err, info)
c.cas = cas
c.caPool = v.pool
c.base.VerifyOutgoing = v.verify
c.base.AutoEncryptTLS = v.autoEncryptTLS
require.Equal(t, v.expected, c.outgoingRPCTLSDisabled(), info)
}
}
func TestConfigurator_SomeValuesFromConfig(t *testing.T) {
c := Configurator{base: &Config{
VerifyServerHostname: true,
VerifyOutgoing: true,
Domain: "abc.de",
}}
one, two, three := c.someValuesFromConfig()
require.Equal(t, c.base.VerifyServerHostname, one)
require.Equal(t, c.base.VerifyOutgoing, two)
require.Equal(t, c.base.Domain, three)
}
func TestConfigurator_VerifyIncomingRPC(t *testing.T) {
c := Configurator{base: &Config{
VerifyIncomingRPC: true,
@ -667,7 +662,7 @@ func TestConfigurator_IncomingRPCConfig(t *testing.T) {
}
func TestConfigurator_IncomingHTTPSConfig(t *testing.T) {
c := Configurator{base: &Config{}}
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
require.Equal(t, []string{"h2", "http/1.1"}, c.IncomingHTTPSConfig().NextProtos)
}
@ -675,7 +670,7 @@ func TestConfigurator_OutgoingTLSConfigForChecks(t *testing.T) {
c := Configurator{base: &Config{
TLSMinVersion: "tls12",
EnableAgentTLSForChecks: false,
}}
}, autoEncrypt: &autoEncrypt{}}
tlsConf := c.OutgoingTLSConfigForCheck(true)
require.Equal(t, true, tlsConf.InsecureSkipVerify)
require.Equal(t, uint16(0), tlsConf.MinVersion)
@ -689,14 +684,14 @@ func TestConfigurator_OutgoingTLSConfigForChecks(t *testing.T) {
}
func TestConfigurator_OutgoingRPCConfig(t *testing.T) {
c := Configurator{base: &Config{}}
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
require.Nil(t, c.OutgoingRPCConfig())
c.base.VerifyOutgoing = true
require.NotNil(t, c.OutgoingRPCConfig())
}
func TestConfigurator_OutgoingRPCWrapper(t *testing.T) {
c := Configurator{base: &Config{}}
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
require.Nil(t, c.OutgoingRPCWrapper())
c.base.VerifyOutgoing = true
wrap := c.OutgoingRPCWrapper()
@ -719,8 +714,8 @@ func TestConfigurator_UpdateChecks(t *testing.T) {
func TestConfigurator_UpdateSetsStuff(t *testing.T) {
c, err := NewConfigurator(Config{}, nil)
require.NoError(t, err)
require.Nil(t, c.cas)
require.Nil(t, c.cert)
require.Nil(t, c.caPool)
require.Nil(t, c.manual.cert.Certificate)
require.Equal(t, c.base, &Config{})
require.Equal(t, 1, c.version)
@ -733,9 +728,9 @@ func TestConfigurator_UpdateSetsStuff(t *testing.T) {
KeyFile: "../test/key/ourdomain.key",
}
require.NoError(t, c.Update(config))
require.NotNil(t, c.cas)
require.Len(t, c.cas.Subjects(), 1)
require.NotNil(t, c.cert)
require.NotNil(t, c.caPool)
require.Len(t, c.caPool.Subjects(), 1)
require.NotNil(t, c.manual.cert)
require.Equal(t, c.base, &config)
require.Equal(t, 2, c.version)
}
@ -757,3 +752,68 @@ func TestConfigurator_ServerNameOrNodeName(t *testing.T) {
require.Equal(t, v.expected, c.serverNameOrNodeName())
}
}
func TestConfigurator_VerifyOutgoing(t *testing.T) {
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
type variant struct {
verify bool
autoEncryptTLS bool
pool *x509.CertPool
expected bool
}
variants := []variant{
{false, false, nil, false},
{true, false, nil, true},
{false, true, nil, false},
{true, true, nil, true},
{false, false, &x509.CertPool{}, false},
{true, false, &x509.CertPool{}, true},
{false, true, &x509.CertPool{}, true},
{true, true, &x509.CertPool{}, true},
}
for i, v := range variants {
info := fmt.Sprintf("case %d", i)
c.caPool = v.pool
c.base.VerifyOutgoing = v.verify
c.base.AutoEncryptTLS = v.autoEncryptTLS
require.Equal(t, v.expected, c.verifyOutgoing(), info)
}
}
func TestConfigurator_Domain(t *testing.T) {
c := Configurator{base: &Config{Domain: "something"}}
require.Equal(t, "something", c.domain())
}
func TestConfigurator_VerifyServerHostname(t *testing.T) {
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
require.False(t, c.VerifyServerHostname())
c.base.VerifyServerHostname = true
c.autoEncrypt.verifyServerHostname = false
require.True(t, c.VerifyServerHostname())
c.base.VerifyServerHostname = false
c.autoEncrypt.verifyServerHostname = true
require.True(t, c.VerifyServerHostname())
c.base.VerifyServerHostname = true
c.autoEncrypt.verifyServerHostname = true
require.True(t, c.VerifyServerHostname())
}
func TestConfigurator_AutoEncrytCertExpired(t *testing.T) {
c := Configurator{base: &Config{}, autoEncrypt: &autoEncrypt{}}
require.True(t, c.AutoEncryptCertExpired())
cert, err := loadKeyPair("../test/key/something_expired.cer", "../test/key/something_expired.key")
require.NoError(t, err)
c.autoEncrypt.cert = cert
require.True(t, c.AutoEncryptCertExpired())
cert, err = loadKeyPair("../test/key/ourdomain.cer", "../test/key/ourdomain.key")
require.NoError(t, err)
c.autoEncrypt.cert = cert
require.False(t, c.AutoEncryptCertExpired())
}

View File

@ -1,4 +1,4 @@
package tls
package tlsutil
import (
"bytes"

View File

@ -1,4 +1,4 @@
package tls
package tlsutil
import (
"crypto"

View File

@ -804,6 +804,15 @@ default will automatically work with some tooling.
until enough newer-versioned servers have been added to the cluster before promoting any of them to voters. Defaults
to `false`.
* <a name="auto_encrypt"></a><a href="#auto_encrypt">`auto_encrypt`</a>
This object allows setting options for the `auto_encrypt` feature.
The following sub-keys are available:
* <a name="allow_tls"></a><a href="#allow_tls">`allow_tls`</a> (Defaults to `false`) This option enables `auto_encrypt` on the servers and allows them to automatically distribute certificates from the Connect CA to the clients. If enabled, the server can accept incoming connections from both the built-in CA and the Connect CA, as well as their certificates. Note, the server will only present the built-in CA and certificate, which the client can verify using the CA it received from `auto_encrypt` endpoint. If disabled, a client configured with `auto_encrypt.tls` will be unable to start.
* <a name="tls"></a><a href="#tls">`tls`</a> (Defaults to `false`) Allows the client to request the Connect CA and certificates from the servers, for encrypting RPC communication. The client will make the request to any servers listed in the `-join` or `-retry-join` option. This requires that every server to have `auto_encrypt.allow_tls` enabled. When both `auto_encrypt` options are used, it allows clients to receive certificates that are generated on the servers. If the `-server-port` is not the default one, it has to be provided to the client as well. Usually this is discovered through LAN gossip, but `auto_encrypt` provision happens before the information can be distributed through gossip. The most secure `auto_encrypt` setup is when the client is provided with the built-in CA, `verify_server_hostname` is turned on, and when an ACL token with `node.write` permissions is setup. It is also possible to use `auto_encrypt` with a CA and ACL, but without `verify_server_hostname`, or only with a ACL enabled, or only with CA and `verify_server_hostname`, or only with a CA, or finally without a CA and without ACL enabled. In any case, the communication to the `auto_encrypt` endpoint is always TLS encrypted.
* <a name="bootstrap"></a><a href="#bootstrap">`bootstrap`</a> Equivalent to the
[`-bootstrap` command-line flag](#_bootstrap).