Merge of auto-config and auto-encrypt code (#8523)
auto-encrypt is now handled as a special case of auto-config. This also is moving all the cert-monitor code into the auto-config package.
This commit is contained in:
parent
bd4e480a78
commit
335c604ced
|
@ -31,7 +31,6 @@ import (
|
|||
autoconf "github.com/hashicorp/consul/agent/auto-config"
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
certmon "github.com/hashicorp/consul/agent/cert-monitor"
|
||||
"github.com/hashicorp/consul/agent/checks"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/consul"
|
||||
|
@ -162,8 +161,6 @@ type notifier interface {
|
|||
type Agent struct {
|
||||
autoConf *autoconf.AutoConfig
|
||||
|
||||
certMonitor *certmon.CertMonitor
|
||||
|
||||
// config is the agent configuration.
|
||||
config *config.RuntimeConfig
|
||||
|
||||
|
@ -373,6 +370,11 @@ func New(bd BaseDeps) (*Agent, error) {
|
|||
// pass the agent itself so its safe to move here.
|
||||
a.registerCache()
|
||||
|
||||
// TODO: move to newBaseDeps
|
||||
// TODO: handle error
|
||||
a.loadTokens(a.config)
|
||||
a.loadEnterpriseTokens(a.config)
|
||||
|
||||
return &a, nil
|
||||
}
|
||||
|
||||
|
@ -426,11 +428,6 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
return fmt.Errorf("Failed to load TLS configurations after applying auto-config settings: %w", err)
|
||||
}
|
||||
|
||||
// TODO: move to newBaseDeps
|
||||
// TODO: handle error
|
||||
a.loadTokens(a.config)
|
||||
a.loadEnterpriseTokens(a.config)
|
||||
|
||||
// create the local state
|
||||
a.State = local.NewState(LocalConfig(c), a.logger, a.tokens)
|
||||
|
||||
|
@ -495,43 +492,6 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
a.State.Delegate = a.delegate
|
||||
a.State.TriggerSyncChanges = a.sync.SyncChanges.Trigger
|
||||
|
||||
if a.config.AutoEncryptTLS && !a.config.ServerMode {
|
||||
reply, err := a.autoEncryptInitialCertificate(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("AutoEncrypt failed: %s", err)
|
||||
}
|
||||
|
||||
cmConfig := new(certmon.Config).
|
||||
WithCache(a.cache).
|
||||
WithLogger(a.logger.Named(logging.AutoEncrypt)).
|
||||
WithTLSConfigurator(a.tlsConfigurator).
|
||||
WithTokens(a.tokens).
|
||||
WithFallback(a.autoEncryptInitialCertificate).
|
||||
WithDNSSANs(a.config.AutoEncryptDNSSAN).
|
||||
WithIPSANs(a.config.AutoEncryptIPSAN).
|
||||
WithDatacenter(a.config.Datacenter).
|
||||
WithNodeName(a.config.NodeName)
|
||||
|
||||
monitor, err := certmon.New(cmConfig)
|
||||
if err != nil {
|
||||
return fmt.Errorf("AutoEncrypt failed to setup certificate monitor: %w", err)
|
||||
}
|
||||
if err := monitor.Update(reply); err != nil {
|
||||
return fmt.Errorf("AutoEncrypt failed to setup certificate monitor: %w", err)
|
||||
}
|
||||
a.certMonitor = monitor
|
||||
|
||||
// we don't need to worry about ever calling Stop as we have tied the go routines
|
||||
// to the agents lifetime by using the StopCh. Also the agent itself doesn't have
|
||||
// a need of ensuring that the go routine was stopped before performing any action
|
||||
// so we can ignore the chan in the return.
|
||||
if _, err := a.certMonitor.Start(&lib.StopChannelContext{StopCh: a.shutdownCh}); err != nil {
|
||||
return fmt.Errorf("AutoEncrypt failed to start certificate monitor: %w", err)
|
||||
}
|
||||
|
||||
a.logger.Info("automatically upgraded to TLS")
|
||||
}
|
||||
|
||||
if err := a.autoConf.Start(&lib.StopChannelContext{StopCh: a.shutdownCh}); err != nil {
|
||||
return fmt.Errorf("AutoConf failed to start certificate monitor: %w", err)
|
||||
}
|
||||
|
@ -645,19 +605,6 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (a *Agent) autoEncryptInitialCertificate(ctx context.Context) (*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, retryJoinSerfVariant, "LAN", a.config.RetryJoinLAN, a.logger)...)
|
||||
|
||||
return client.RequestAutoEncryptCerts(ctx, addrs, a.config.ServerPort, a.tokens.AgentToken(), a.config.AutoEncryptDNSSAN, a.config.AutoEncryptIPSAN)
|
||||
}
|
||||
|
||||
func (a *Agent) listenAndServeGRPC() error {
|
||||
if len(a.config.GRPCAddrs) < 1 {
|
||||
return nil
|
||||
|
@ -1380,12 +1327,6 @@ func (a *Agent) ShutdownAgent() error {
|
|||
// this should help them to be stopped more quickly
|
||||
a.autoConf.Stop()
|
||||
|
||||
if a.certMonitor != nil {
|
||||
// this would be cancelled anyways (by the closing of the shutdown ch)
|
||||
// but this should help them to be stopped more quickly
|
||||
a.certMonitor.Stop()
|
||||
}
|
||||
|
||||
// Stop the service manager (must happen before we take the stateLock to avoid deadlock)
|
||||
if a.serviceManager != nil {
|
||||
a.serviceManager.Stop()
|
||||
|
|
|
@ -4,62 +4,54 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/consul/logging"
|
||||
"github.com/hashicorp/consul/proto/pbautoconf"
|
||||
"github.com/hashicorp/go-discover"
|
||||
discoverk8s "github.com/hashicorp/go-discover/provider/k8s"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
)
|
||||
|
||||
const (
|
||||
// autoConfigFileName is the name of the file that the agent auto-config settings are
|
||||
// stored in within the data directory
|
||||
autoConfigFileName = "auto-config.json"
|
||||
|
||||
dummyTrustDomain = "dummytrustdomain"
|
||||
)
|
||||
|
||||
var (
|
||||
pbMarshaler = &jsonpb.Marshaler{
|
||||
OrigName: false,
|
||||
EnumsAsInts: false,
|
||||
Indent: " ",
|
||||
EmitDefaults: true,
|
||||
}
|
||||
|
||||
pbUnmarshaler = &jsonpb.Unmarshaler{
|
||||
AllowUnknownFields: false,
|
||||
}
|
||||
)
|
||||
|
||||
// AutoConfig is all the state necessary for being able to parse a configuration
|
||||
// as well as perform the necessary RPCs to perform Agent Auto Configuration.
|
||||
//
|
||||
// NOTE: This struct and methods on it are not currently thread/goroutine safe.
|
||||
// However it doesn't spawn any of its own go routines yet and is used in a
|
||||
// synchronous fashion. In the future if either of those two conditions change
|
||||
// then we will need to add some locking here. I am deferring that for now
|
||||
// to help ease the review of this already large PR.
|
||||
type AutoConfig struct {
|
||||
sync.Mutex
|
||||
|
||||
acConfig Config
|
||||
logger hclog.Logger
|
||||
certMonitor CertMonitor
|
||||
cache Cache
|
||||
waiter *lib.RetryWaiter
|
||||
config *config.RuntimeConfig
|
||||
autoConfigResponse *pbautoconf.AutoConfigResponse
|
||||
autoConfigSource config.Source
|
||||
|
||||
running bool
|
||||
done chan struct{}
|
||||
// cancel is used to cancel the entire AutoConfig
|
||||
// go routine. This is the main field protected
|
||||
// by the mutex as it being non-nil indicates that
|
||||
// the go routine has been started and is stoppable.
|
||||
// note that it doesn't indcate that the go routine
|
||||
// is currently running.
|
||||
cancel context.CancelFunc
|
||||
|
||||
// cancelWatches is used to cancel the existing
|
||||
// cache watches regarding the agents certificate. This is
|
||||
// mainly only necessary when the Agent token changes.
|
||||
cancelWatches context.CancelFunc
|
||||
|
||||
// cacheUpdates is the chan used to have the cache
|
||||
// send us back events
|
||||
cacheUpdates chan cache.UpdateEvent
|
||||
|
||||
// tokenUpdates is the struct used to receive
|
||||
// events from the token store when the Agent
|
||||
// token is updated.
|
||||
tokenUpdates token.Notifier
|
||||
}
|
||||
|
||||
// New creates a new AutoConfig object for providing automatic Consul configuration.
|
||||
|
@ -69,6 +61,19 @@ func New(config Config) (*AutoConfig, error) {
|
|||
return nil, fmt.Errorf("must provide a config loader")
|
||||
case config.DirectRPC == nil:
|
||||
return nil, fmt.Errorf("must provide a direct RPC delegate")
|
||||
case config.Cache == nil:
|
||||
return nil, fmt.Errorf("must provide a cache")
|
||||
case config.TLSConfigurator == nil:
|
||||
return nil, fmt.Errorf("must provide a TLS configurator")
|
||||
case config.Tokens == nil:
|
||||
return nil, fmt.Errorf("must provide a token store")
|
||||
}
|
||||
|
||||
if config.FallbackLeeway == 0 {
|
||||
config.FallbackLeeway = 10 * time.Second
|
||||
}
|
||||
if config.FallbackRetry == 0 {
|
||||
config.FallbackRetry = time.Minute
|
||||
}
|
||||
|
||||
logger := config.Logger
|
||||
|
@ -83,15 +88,16 @@ func New(config Config) (*AutoConfig, error) {
|
|||
}
|
||||
|
||||
return &AutoConfig{
|
||||
acConfig: config,
|
||||
logger: logger,
|
||||
certMonitor: config.CertMonitor,
|
||||
acConfig: config,
|
||||
logger: logger,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// ReadConfig will parse the current configuration and inject any
|
||||
// auto-config sources if present into the correct place in the parsing chain.
|
||||
func (ac *AutoConfig) ReadConfig() (*config.RuntimeConfig, error) {
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
cfg, warnings, err := ac.acConfig.Loader(ac.autoConfigSource)
|
||||
if err != nil {
|
||||
return cfg, err
|
||||
|
@ -105,46 +111,6 @@ func (ac *AutoConfig) ReadConfig() (*config.RuntimeConfig, error) {
|
|||
return cfg, nil
|
||||
}
|
||||
|
||||
// restorePersistedAutoConfig will attempt to load the persisted auto-config
|
||||
// settings from the data directory. It returns true either when there was an
|
||||
// unrecoverable error or when the configuration was successfully loaded from
|
||||
// disk. Recoverable errors, such as "file not found" are suppressed and this
|
||||
// method will return false for the first boolean.
|
||||
func (ac *AutoConfig) restorePersistedAutoConfig() (bool, error) {
|
||||
if ac.config.DataDir == "" {
|
||||
// no data directory means we don't have anything to potentially load
|
||||
return false, nil
|
||||
}
|
||||
|
||||
path := filepath.Join(ac.config.DataDir, autoConfigFileName)
|
||||
ac.logger.Debug("attempting to restore any persisted configuration", "path", path)
|
||||
|
||||
content, err := ioutil.ReadFile(path)
|
||||
if err == nil {
|
||||
rdr := strings.NewReader(string(content))
|
||||
|
||||
var resp pbautoconf.AutoConfigResponse
|
||||
if err := pbUnmarshaler.Unmarshal(rdr, &resp); err != nil {
|
||||
return false, fmt.Errorf("failed to decode persisted auto-config data: %w", err)
|
||||
}
|
||||
|
||||
if err := ac.update(&resp); err != nil {
|
||||
return false, fmt.Errorf("error restoring persisted auto-config response: %w", err)
|
||||
}
|
||||
|
||||
ac.logger.Info("restored persisted configuration", "path", path)
|
||||
return true, nil
|
||||
}
|
||||
|
||||
if !os.IsNotExist(err) {
|
||||
return true, fmt.Errorf("failed to load %s: %w", path, err)
|
||||
}
|
||||
|
||||
// ignore non-existence errors as that is an indicator that we haven't
|
||||
// performed the auto configuration before
|
||||
return false, nil
|
||||
}
|
||||
|
||||
// InitialConfiguration will perform a one-time RPC request to the configured servers
|
||||
// to retrieve various cluster wide configurations. See the proto/pbautoconf/auto_config.proto
|
||||
// file for a complete reference of what configurations can be applied in this manner.
|
||||
|
@ -164,30 +130,49 @@ func (ac *AutoConfig) InitialConfiguration(ctx context.Context) (*config.Runtime
|
|||
ac.config = config
|
||||
}
|
||||
|
||||
if !ac.config.AutoConfig.Enabled {
|
||||
return ac.config, nil
|
||||
}
|
||||
|
||||
ready, err := ac.restorePersistedAutoConfig()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if !ready {
|
||||
ac.logger.Info("retrieving initial agent auto configuration remotely")
|
||||
if err := ac.getInitialConfiguration(ctx); err != nil {
|
||||
switch {
|
||||
case ac.config.AutoConfig.Enabled:
|
||||
resp, err := ac.readPersistedAutoConfig()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// re-read the configuration now that we have our initial auto-config
|
||||
config, err := ac.ReadConfig()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if resp == nil {
|
||||
ac.logger.Info("retrieving initial agent auto configuration remotely")
|
||||
resp, err = ac.getInitialConfiguration(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
ac.config = config
|
||||
return ac.config, nil
|
||||
ac.logger.Debug("updating auto-config settings")
|
||||
if err = ac.recordInitialConfiguration(resp); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// re-read the configuration now that we have our initial auto-config
|
||||
config, err := ac.ReadConfig()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ac.config = config
|
||||
return ac.config, nil
|
||||
case ac.config.AutoEncryptTLS:
|
||||
certs, err := ac.autoEncryptInitialCerts(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := ac.setInitialTLSCertificates(certs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ac.logger.Info("automatically upgraded to TLS")
|
||||
return ac.config, nil
|
||||
default:
|
||||
return ac.config, nil
|
||||
}
|
||||
}
|
||||
|
||||
// introToken is responsible for determining the correct intro token to use
|
||||
|
@ -217,118 +202,45 @@ func (ac *AutoConfig) introToken() (string, error) {
|
|||
return token, nil
|
||||
}
|
||||
|
||||
// serverHosts is responsible for taking the list of server addresses and
|
||||
// resolving any go-discover provider invocations. It will then return a list
|
||||
// of hosts. These might be hostnames and is expected that DNS resolution may
|
||||
// be performed after this function runs. Additionally these may contain ports
|
||||
// so SplitHostPort could also be necessary.
|
||||
func (ac *AutoConfig) serverHosts() ([]string, error) {
|
||||
servers := ac.config.AutoConfig.ServerAddresses
|
||||
// recordInitialConfiguration is responsible for recording the AutoConfigResponse from
|
||||
// the AutoConfig.InitialConfiguration RPC. It is an all-in-one function to do the following
|
||||
// * update the Agent token in the token store
|
||||
func (ac *AutoConfig) recordInitialConfiguration(resp *pbautoconf.AutoConfigResponse) error {
|
||||
ac.autoConfigResponse = resp
|
||||
|
||||
providers := make(map[string]discover.Provider)
|
||||
for k, v := range discover.Providers {
|
||||
providers[k] = v
|
||||
ac.autoConfigSource = config.LiteralSource{
|
||||
Name: autoConfigFileName,
|
||||
Config: translateConfig(resp.Config),
|
||||
}
|
||||
providers["k8s"] = &discoverk8s.Provider{}
|
||||
|
||||
disco, err := discover.New(
|
||||
discover.WithUserAgent(lib.UserAgent()),
|
||||
discover.WithProviders(providers),
|
||||
)
|
||||
|
||||
// we need to re-read the configuration to determine what the correct ACL
|
||||
// token to push into the token store is. Any user provided token will override
|
||||
// any AutoConfig generated token.
|
||||
config, err := ac.ReadConfig()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("Failed to create go-discover resolver: %w", err)
|
||||
return fmt.Errorf("failed to fully resolve configuration: %w", err)
|
||||
}
|
||||
|
||||
var addrs []string
|
||||
for _, addr := range servers {
|
||||
switch {
|
||||
case strings.Contains(addr, "provider="):
|
||||
resolved, err := disco.Addrs(addr, ac.logger.StandardLogger(&hclog.StandardLoggerOptions{InferLevels: true}))
|
||||
if err != nil {
|
||||
ac.logger.Error("failed to resolve go-discover auto-config servers", "configuration", addr, "err", err)
|
||||
continue
|
||||
}
|
||||
// ignoring the return value which would indicate a change in the token
|
||||
_ = ac.acConfig.Tokens.UpdateAgentToken(config.ACLAgentToken, token.TokenSourceConfig)
|
||||
|
||||
addrs = append(addrs, resolved...)
|
||||
ac.logger.Debug("discovered auto-config servers", "servers", resolved)
|
||||
default:
|
||||
addrs = append(addrs, addr)
|
||||
}
|
||||
}
|
||||
|
||||
if len(addrs) == 0 {
|
||||
return nil, fmt.Errorf("no auto-config server addresses available for use")
|
||||
}
|
||||
|
||||
return addrs, nil
|
||||
}
|
||||
|
||||
// resolveHost will take a single host string and convert it to a list of TCPAddrs
|
||||
// This will process any port in the input as well as looking up the hostname using
|
||||
// normal DNS resolution.
|
||||
func (ac *AutoConfig) resolveHost(hostPort string) []net.TCPAddr {
|
||||
port := ac.config.ServerPort
|
||||
host, portStr, err := net.SplitHostPort(hostPort)
|
||||
// extra a structs.SignedResponse from the AutoConfigResponse for use in cache prepopulation
|
||||
signed, err := extractSignedResponse(resp)
|
||||
if err != nil {
|
||||
if strings.Contains(err.Error(), "missing port in address") {
|
||||
host = hostPort
|
||||
} else {
|
||||
ac.logger.Warn("error splitting host address into IP and port", "address", hostPort, "error", err)
|
||||
return nil
|
||||
}
|
||||
} else {
|
||||
port, err = strconv.Atoi(portStr)
|
||||
if err != nil {
|
||||
ac.logger.Warn("Parsed port is not an integer", "port", portStr, "error", err)
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("failed to extract certificates from the auto-config response: %w", err)
|
||||
}
|
||||
|
||||
// resolve the host to a list of IPs
|
||||
ips, err := net.LookupIP(host)
|
||||
if err != nil {
|
||||
ac.logger.Warn("IP resolution failed", "host", host, "error", err)
|
||||
return nil
|
||||
// prepopulate the cache
|
||||
if err = ac.populateCertificateCache(signed); err != nil {
|
||||
return fmt.Errorf("failed to populate the cache with certificate responses: %w", err)
|
||||
}
|
||||
|
||||
var addrs []net.TCPAddr
|
||||
for _, ip := range ips {
|
||||
addrs = append(addrs, net.TCPAddr{IP: ip, Port: port})
|
||||
}
|
||||
|
||||
return addrs
|
||||
}
|
||||
|
||||
// recordResponse takes an AutoConfig RPC response records it with the agent
|
||||
// This will persist the configuration to disk (unless in dev mode running without
|
||||
// a data dir) and will reload the configuration.
|
||||
func (ac *AutoConfig) recordResponse(resp *pbautoconf.AutoConfigResponse) error {
|
||||
serialized, err := pbMarshaler.MarshalToString(resp)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to encode auto-config response as JSON: %w", err)
|
||||
}
|
||||
|
||||
if err := ac.update(resp); err != nil {
|
||||
// update the TLS configurator with the latest certificates
|
||||
if err := ac.updateTLSFromResponse(resp); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// now that we know the configuration is generally fine including TLS certs go ahead and persist it to disk.
|
||||
if ac.config.DataDir == "" {
|
||||
ac.logger.Debug("not persisting auto-config settings because there is no data directory")
|
||||
return nil
|
||||
}
|
||||
|
||||
path := filepath.Join(ac.config.DataDir, autoConfigFileName)
|
||||
|
||||
err = ioutil.WriteFile(path, []byte(serialized), 0660)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to write auto-config configurations: %w", err)
|
||||
}
|
||||
|
||||
ac.logger.Debug("auto-config settings were persisted to disk")
|
||||
|
||||
return nil
|
||||
return ac.persistAutoConfig(resp)
|
||||
}
|
||||
|
||||
// getInitialConfigurationOnce will perform full server to TCPAddr resolution and
|
||||
|
@ -352,7 +264,7 @@ func (ac *AutoConfig) getInitialConfigurationOnce(ctx context.Context, csr strin
|
|||
|
||||
var resp pbautoconf.AutoConfigResponse
|
||||
|
||||
servers, err := ac.serverHosts()
|
||||
servers, err := ac.autoConfigHosts()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -369,6 +281,7 @@ func (ac *AutoConfig) getInitialConfigurationOnce(ctx context.Context, csr strin
|
|||
ac.logger.Error("AutoConfig.InitialConfiguration RPC failed", "addr", addr.String(), "error", err)
|
||||
continue
|
||||
}
|
||||
ac.logger.Debug("AutoConfig.InitialConfiguration RPC was successful")
|
||||
|
||||
// update the Certificate with the private key we generated locally
|
||||
if resp.Certificate != nil {
|
||||
|
@ -379,17 +292,17 @@ func (ac *AutoConfig) getInitialConfigurationOnce(ctx context.Context, csr strin
|
|||
}
|
||||
}
|
||||
|
||||
return nil, ctx.Err()
|
||||
return nil, fmt.Errorf("No server successfully responded to the auto-config request")
|
||||
}
|
||||
|
||||
// getInitialConfiguration implements a loop to retry calls to getInitialConfigurationOnce.
|
||||
// It uses the RetryWaiter on the AutoConfig object to control how often to attempt
|
||||
// the initial configuration process. It is also canceallable by cancelling the provided context.
|
||||
func (ac *AutoConfig) getInitialConfiguration(ctx context.Context) error {
|
||||
func (ac *AutoConfig) getInitialConfiguration(ctx context.Context) (*pbautoconf.AutoConfigResponse, error) {
|
||||
// generate a CSR
|
||||
csr, key, err := ac.generateCSR()
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// this resets the failures so that we will perform immediate request
|
||||
|
@ -397,183 +310,95 @@ func (ac *AutoConfig) getInitialConfiguration(ctx context.Context) error {
|
|||
for {
|
||||
select {
|
||||
case <-wait:
|
||||
resp, err := ac.getInitialConfigurationOnce(ctx, csr, key)
|
||||
if resp != nil {
|
||||
return ac.recordResponse(resp)
|
||||
if resp, err := ac.getInitialConfigurationOnce(ctx, csr, key); err == nil && resp != nil {
|
||||
return resp, nil
|
||||
} else if err != nil {
|
||||
ac.logger.Error(err.Error())
|
||||
} else {
|
||||
ac.logger.Error("No error returned when fetching the initial auto-configuration but no response was either")
|
||||
ac.logger.Error("No error returned when fetching configuration from the servers but no response was either")
|
||||
}
|
||||
|
||||
wait = ac.acConfig.Waiter.Failed()
|
||||
case <-ctx.Done():
|
||||
ac.logger.Info("interrupted during initial auto configuration", "err", ctx.Err())
|
||||
return ctx.Err()
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// generateCSR will generate a CSR for an Agent certificate. This should
|
||||
// be sent along with the AutoConfig.InitialConfiguration RPC. The generated
|
||||
// CSR does NOT have a real trust domain as when generating this we do
|
||||
// not yet have the CA roots. The server will update the trust domain
|
||||
// for us though.
|
||||
func (ac *AutoConfig) generateCSR() (csr string, key string, err error) {
|
||||
// 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{
|
||||
// will be replaced
|
||||
Host: dummyTrustDomain,
|
||||
Datacenter: ac.config.Datacenter,
|
||||
Agent: ac.config.NodeName,
|
||||
}
|
||||
|
||||
caConfig, err := ac.config.ConnectCAConfiguration()
|
||||
if err != nil {
|
||||
return "", "", fmt.Errorf("Cannot generate CSR: %w", err)
|
||||
}
|
||||
|
||||
conf, err := caConfig.GetCommonConfig()
|
||||
if err != nil {
|
||||
return "", "", fmt.Errorf("Failed to load common CA configuration: %w", err)
|
||||
}
|
||||
|
||||
if conf.PrivateKeyType == "" {
|
||||
conf.PrivateKeyType = connect.DefaultPrivateKeyType
|
||||
}
|
||||
if conf.PrivateKeyBits == 0 {
|
||||
conf.PrivateKeyBits = connect.DefaultPrivateKeyBits
|
||||
}
|
||||
|
||||
// Create a new private key
|
||||
pk, pkPEM, err := connect.GeneratePrivateKeyWithConfig(conf.PrivateKeyType, conf.PrivateKeyBits)
|
||||
if err != nil {
|
||||
return "", "", fmt.Errorf("Failed to generate private key: %w", err)
|
||||
}
|
||||
|
||||
dnsNames := append([]string{"localhost"}, ac.config.AutoConfig.DNSSANs...)
|
||||
ipAddresses := append([]net.IP{net.ParseIP("127.0.0.1"), net.ParseIP("::")}, ac.config.AutoConfig.IPSANs...)
|
||||
|
||||
// Create a CSR.
|
||||
//
|
||||
// The Common Name includes the dummy trust domain for now but Server will
|
||||
// override this when it is signed anyway so it's OK.
|
||||
cn := connect.AgentCN(ac.config.NodeName, dummyTrustDomain)
|
||||
csr, err = connect.CreateCSR(id, cn, pk, dnsNames, ipAddresses)
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
|
||||
return csr, pkPEM, nil
|
||||
}
|
||||
|
||||
// update will take an AutoConfigResponse and do all things necessary
|
||||
// to restore those settings. This currently involves updating the
|
||||
// config data to be used during a call to ReadConfig, updating the
|
||||
// tls Configurator and prepopulating the cache.
|
||||
func (ac *AutoConfig) update(resp *pbautoconf.AutoConfigResponse) error {
|
||||
ac.autoConfigResponse = resp
|
||||
|
||||
ac.autoConfigSource = config.LiteralSource{
|
||||
Name: autoConfigFileName,
|
||||
Config: translateConfig(resp.Config),
|
||||
}
|
||||
|
||||
if err := ac.updateTLSFromResponse(resp); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// updateTLSFromResponse will update the TLS certificate and roots in the shared
|
||||
// TLS configurator.
|
||||
func (ac *AutoConfig) updateTLSFromResponse(resp *pbautoconf.AutoConfigResponse) error {
|
||||
if ac.certMonitor == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
roots, err := translateCARootsToStructs(resp.CARoots)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
cert, err := translateIssuedCertToStructs(resp.Certificate)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
update := &structs.SignedResponse{
|
||||
IssuedCert: *cert,
|
||||
ConnectCARoots: *roots,
|
||||
ManualCARoots: resp.ExtraCACertificates,
|
||||
}
|
||||
|
||||
if resp.Config != nil && resp.Config.TLS != nil {
|
||||
update.VerifyServerHostname = resp.Config.TLS.VerifyServerHostname
|
||||
}
|
||||
|
||||
if err := ac.certMonitor.Update(update); err != nil {
|
||||
return fmt.Errorf("failed to update the certificate monitor: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) Start(ctx context.Context) error {
|
||||
if ac.certMonitor == nil {
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
|
||||
if !ac.config.AutoConfig.Enabled && !ac.config.AutoEncryptTLS {
|
||||
return nil
|
||||
}
|
||||
|
||||
if !ac.config.AutoConfig.Enabled {
|
||||
return nil
|
||||
if ac.running || ac.cancel != nil {
|
||||
return fmt.Errorf("AutoConfig is already running")
|
||||
}
|
||||
|
||||
_, err := ac.certMonitor.Start(ctx)
|
||||
return err
|
||||
// create the top level context to control the go
|
||||
// routine executing the `run` method
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
|
||||
// create the channel to get cache update events through
|
||||
// really we should only ever get 10 updates
|
||||
ac.cacheUpdates = make(chan cache.UpdateEvent, 10)
|
||||
|
||||
// setup the cache watches
|
||||
cancelCertWatches, err := ac.setupCertificateCacheWatches(ctx)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return fmt.Errorf("error setting up cache watches: %w", err)
|
||||
}
|
||||
|
||||
// start the token update notifier
|
||||
ac.tokenUpdates = ac.acConfig.Tokens.Notify(token.TokenKindAgent)
|
||||
|
||||
// store the cancel funcs
|
||||
ac.cancel = cancel
|
||||
ac.cancelWatches = cancelCertWatches
|
||||
|
||||
ac.running = true
|
||||
ac.done = make(chan struct{})
|
||||
go ac.run(ctx, ac.done)
|
||||
|
||||
ac.logger.Info("auto-config started")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) Done() <-chan struct{} {
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
|
||||
if ac.done != nil {
|
||||
return ac.done
|
||||
}
|
||||
|
||||
// return a closed channel to indicate that we are already done
|
||||
done := make(chan struct{})
|
||||
close(done)
|
||||
return done
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) IsRunning() bool {
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
return ac.running
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) Stop() bool {
|
||||
if ac.certMonitor == nil {
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
|
||||
if !ac.running {
|
||||
return false
|
||||
}
|
||||
|
||||
if !ac.config.AutoConfig.Enabled {
|
||||
return false
|
||||
if ac.cancel != nil {
|
||||
ac.cancel()
|
||||
}
|
||||
|
||||
return ac.certMonitor.Stop()
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) FallbackTLS(ctx context.Context) (*structs.SignedResponse, error) {
|
||||
// generate a CSR
|
||||
csr, key, err := ac.generateCSR()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
resp, err := ac.getInitialConfigurationOnce(ctx, csr, key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return extractSignedResponse(resp)
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) RecordUpdatedCerts(resp *structs.SignedResponse) error {
|
||||
var err error
|
||||
ac.autoConfigResponse.ExtraCACertificates = resp.ManualCARoots
|
||||
ac.autoConfigResponse.CARoots, err = translateCARootsToProtobuf(&resp.ConnectCARoots)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ac.autoConfigResponse.Certificate, err = translateIssuedCertToProtobuf(&resp.IssuedCert)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return ac.recordResponse(ac.autoConfigResponse)
|
||||
return true
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,111 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"strings"
|
||||
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
)
|
||||
|
||||
func (ac *AutoConfig) autoEncryptInitialCerts(ctx context.Context) (*structs.SignedResponse, error) {
|
||||
// generate a CSR
|
||||
csr, key, err := ac.generateCSR()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// this resets the failures so that we will perform immediate request
|
||||
wait := ac.acConfig.Waiter.Success()
|
||||
for {
|
||||
select {
|
||||
case <-wait:
|
||||
if resp, err := ac.autoEncryptInitialCertsOnce(ctx, csr, key); err == nil && resp != nil {
|
||||
return resp, nil
|
||||
} else if err != nil {
|
||||
ac.logger.Error(err.Error())
|
||||
} else {
|
||||
ac.logger.Error("No error returned when fetching certificates from the servers but no response was either")
|
||||
}
|
||||
|
||||
wait = ac.acConfig.Waiter.Failed()
|
||||
case <-ctx.Done():
|
||||
ac.logger.Info("interrupted during retrieval of auto-encrypt certificates", "err", ctx.Err())
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) autoEncryptInitialCertsOnce(ctx context.Context, csr, key string) (*structs.SignedResponse, error) {
|
||||
request := structs.CASignRequest{
|
||||
WriteRequest: structs.WriteRequest{Token: ac.acConfig.Tokens.AgentToken()},
|
||||
Datacenter: ac.config.Datacenter,
|
||||
CSR: csr,
|
||||
}
|
||||
var resp structs.SignedResponse
|
||||
|
||||
servers, err := ac.autoEncryptHosts()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, s := range servers {
|
||||
// try each IP to see if we can successfully make the request
|
||||
for _, addr := range ac.resolveHost(s) {
|
||||
if ctx.Err() != nil {
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
|
||||
ac.logger.Debug("making AutoEncrypt.Sign RPC", "addr", addr.String())
|
||||
err = ac.acConfig.DirectRPC.RPC(ac.config.Datacenter, ac.config.NodeName, &addr, "AutoEncrypt.Sign", &request, &resp)
|
||||
if err != nil {
|
||||
ac.logger.Error("AutoEncrypt.Sign RPC failed", "addr", addr.String(), "error", err)
|
||||
continue
|
||||
}
|
||||
|
||||
resp.IssuedCert.PrivateKeyPEM = key
|
||||
return &resp, nil
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("No servers successfully responded to the auto-encrypt request")
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) autoEncryptHosts() ([]string, error) {
|
||||
// use servers known to gossip if there are any
|
||||
if ac.acConfig.ServerProvider != nil {
|
||||
if srv := ac.acConfig.ServerProvider.FindLANServer(); srv != nil {
|
||||
return []string{srv.Addr.String()}, nil
|
||||
}
|
||||
}
|
||||
|
||||
hosts, err := ac.discoverServers(ac.config.RetryJoinLAN)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var addrs []string
|
||||
|
||||
// The addresses we use for auto-encrypt are the retry join and start join
|
||||
// addresses. These are for joining serf and therefore we cannot rely on the
|
||||
// ports for these. This loop strips any port that may have been specified and
|
||||
// will let subsequent resolveAddr calls add on the default RPC port.
|
||||
for _, addr := range append(ac.config.StartJoinAddrsLAN, hosts...) {
|
||||
host, _, err := net.SplitHostPort(addr)
|
||||
if err != nil {
|
||||
if strings.Contains(err.Error(), "missing port in address") {
|
||||
host = addr
|
||||
} else {
|
||||
ac.logger.Warn("error splitting host address into IP and port", "address", addr, "error", err)
|
||||
continue
|
||||
}
|
||||
}
|
||||
addrs = append(addrs, host)
|
||||
}
|
||||
|
||||
if len(addrs) == 0 {
|
||||
return nil, fmt.Errorf("no auto-encrypt server addresses available for use")
|
||||
}
|
||||
|
||||
return addrs, nil
|
||||
}
|
|
@ -0,0 +1,562 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/x509"
|
||||
"crypto/x509/pkix"
|
||||
"encoding/asn1"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestAutoEncrypt_generateCSR(t *testing.T) {
|
||||
type testCase struct {
|
||||
conf *config.RuntimeConfig
|
||||
|
||||
// to validate the csr
|
||||
expectedSubject pkix.Name
|
||||
expectedSigAlg x509.SignatureAlgorithm
|
||||
expectedPubAlg x509.PublicKeyAlgorithm
|
||||
expectedDNSNames []string
|
||||
expectedIPs []net.IP
|
||||
expectedURIs []*url.URL
|
||||
}
|
||||
|
||||
cases := map[string]testCase{
|
||||
"ip-sans": {
|
||||
conf: &config.RuntimeConfig{
|
||||
Datacenter: "dc1",
|
||||
NodeName: "test-node",
|
||||
AutoEncryptTLS: true,
|
||||
AutoEncryptIPSAN: []net.IP{net.IPv4(198, 18, 0, 1), net.IPv4(198, 18, 0, 2)},
|
||||
},
|
||||
expectedSubject: pkix.Name{
|
||||
CommonName: connect.AgentCN("test-node", unknownTrustDomain),
|
||||
Names: []pkix.AttributeTypeAndValue{
|
||||
{
|
||||
// 2,5,4,3 is the CommonName type ASN1 identifier
|
||||
Type: asn1.ObjectIdentifier{2, 5, 4, 3},
|
||||
Value: "testnode.agnt.unknown.consul",
|
||||
},
|
||||
},
|
||||
},
|
||||
expectedSigAlg: x509.ECDSAWithSHA256,
|
||||
expectedPubAlg: x509.ECDSA,
|
||||
expectedDNSNames: defaultDNSSANs,
|
||||
expectedIPs: append(defaultIPSANs,
|
||||
net.IP{198, 18, 0, 1},
|
||||
net.IP{198, 18, 0, 2},
|
||||
),
|
||||
expectedURIs: []*url.URL{
|
||||
{
|
||||
Scheme: "spiffe",
|
||||
Host: unknownTrustDomain,
|
||||
Path: "/agent/client/dc/dc1/id/test-node",
|
||||
},
|
||||
},
|
||||
},
|
||||
"dns-sans": {
|
||||
conf: &config.RuntimeConfig{
|
||||
Datacenter: "dc1",
|
||||
NodeName: "test-node",
|
||||
AutoEncryptTLS: true,
|
||||
AutoEncryptDNSSAN: []string{"foo.local", "bar.local"},
|
||||
},
|
||||
expectedSubject: pkix.Name{
|
||||
CommonName: connect.AgentCN("test-node", unknownTrustDomain),
|
||||
Names: []pkix.AttributeTypeAndValue{
|
||||
{
|
||||
// 2,5,4,3 is the CommonName type ASN1 identifier
|
||||
Type: asn1.ObjectIdentifier{2, 5, 4, 3},
|
||||
Value: "testnode.agnt.unknown.consul",
|
||||
},
|
||||
},
|
||||
},
|
||||
expectedSigAlg: x509.ECDSAWithSHA256,
|
||||
expectedPubAlg: x509.ECDSA,
|
||||
expectedDNSNames: append(defaultDNSSANs, "foo.local", "bar.local"),
|
||||
expectedIPs: defaultIPSANs,
|
||||
expectedURIs: []*url.URL{
|
||||
{
|
||||
Scheme: "spiffe",
|
||||
Host: unknownTrustDomain,
|
||||
Path: "/agent/client/dc/dc1/id/test-node",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for name, tcase := range cases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
ac := AutoConfig{config: tcase.conf}
|
||||
|
||||
csr, _, err := ac.generateCSR()
|
||||
require.NoError(t, err)
|
||||
|
||||
request, err := connect.ParseCSR(csr)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, request)
|
||||
|
||||
require.Equal(t, tcase.expectedSubject, request.Subject)
|
||||
require.Equal(t, tcase.expectedSigAlg, request.SignatureAlgorithm)
|
||||
require.Equal(t, tcase.expectedPubAlg, request.PublicKeyAlgorithm)
|
||||
require.Equal(t, tcase.expectedDNSNames, request.DNSNames)
|
||||
require.Equal(t, tcase.expectedIPs, request.IPAddresses)
|
||||
require.Equal(t, tcase.expectedURIs, request.URIs)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_hosts(t *testing.T) {
|
||||
type testCase struct {
|
||||
serverProvider ServerProvider
|
||||
config *config.RuntimeConfig
|
||||
|
||||
hosts []string
|
||||
err string
|
||||
}
|
||||
|
||||
providerNone := newMockServerProvider(t)
|
||||
providerNone.On("FindLANServer").Return(nil).Times(0)
|
||||
|
||||
providerWithServer := newMockServerProvider(t)
|
||||
providerWithServer.On("FindLANServer").Return(&metadata.Server{Addr: &net.TCPAddr{IP: net.IPv4(198, 18, 0, 1), Port: 1234}}).Times(0)
|
||||
|
||||
cases := map[string]testCase{
|
||||
"router-override": {
|
||||
serverProvider: providerWithServer,
|
||||
config: &config.RuntimeConfig{
|
||||
RetryJoinLAN: []string{"127.0.0.1:9876"},
|
||||
StartJoinAddrsLAN: []string{"192.168.1.2:4321"},
|
||||
},
|
||||
hosts: []string{"198.18.0.1:1234"},
|
||||
},
|
||||
"various-addresses": {
|
||||
serverProvider: providerNone,
|
||||
config: &config.RuntimeConfig{
|
||||
RetryJoinLAN: []string{"198.18.0.1", "foo.com", "[2001:db8::1234]:1234", "abc.local:9876"},
|
||||
StartJoinAddrsLAN: []string{"192.168.1.1:5432", "start.local", "[::ffff:172.16.5.4]", "main.dev:6789"},
|
||||
},
|
||||
hosts: []string{
|
||||
"192.168.1.1",
|
||||
"start.local",
|
||||
"[::ffff:172.16.5.4]",
|
||||
"main.dev",
|
||||
"198.18.0.1",
|
||||
"foo.com",
|
||||
"2001:db8::1234",
|
||||
"abc.local",
|
||||
},
|
||||
},
|
||||
"split-host-port-error": {
|
||||
serverProvider: providerNone,
|
||||
config: &config.RuntimeConfig{
|
||||
StartJoinAddrsLAN: []string{"this-is-not:a:ip:and_port"},
|
||||
},
|
||||
err: "no auto-encrypt server addresses available for use",
|
||||
},
|
||||
}
|
||||
|
||||
for name, tcase := range cases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
ac := AutoConfig{
|
||||
config: tcase.config,
|
||||
logger: testutil.Logger(t),
|
||||
acConfig: Config{
|
||||
ServerProvider: tcase.serverProvider,
|
||||
},
|
||||
}
|
||||
|
||||
hosts, err := ac.autoEncryptHosts()
|
||||
if tcase.err != "" {
|
||||
testutil.RequireErrorContains(t, err, tcase.err)
|
||||
} else {
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, tcase.hosts, hosts)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_InitialCerts(t *testing.T) {
|
||||
token := "1a148388-3dd7-4db4-9eea-520424b4a86a"
|
||||
datacenter := "foo"
|
||||
nodeName := "bar"
|
||||
|
||||
mcfg := newMockedConfig(t)
|
||||
|
||||
_, indexedRoots, cert := testCerts(t, nodeName, datacenter)
|
||||
|
||||
// The following are called once for each round through the auto-encrypt initial certs outer loop
|
||||
// (not the per-host direct rpc attempts but the one involving the RetryWaiter)
|
||||
mcfg.tokens.On("AgentToken").Return(token).Times(2)
|
||||
mcfg.serverProvider.On("FindLANServer").Return(nil).Times(2)
|
||||
|
||||
request := structs.CASignRequest{
|
||||
WriteRequest: structs.WriteRequest{Token: token},
|
||||
Datacenter: datacenter,
|
||||
// this gets removed by the mock code as its non-deterministic what it will be
|
||||
CSR: "",
|
||||
}
|
||||
|
||||
// first failure
|
||||
mcfg.directRPC.On("RPC",
|
||||
datacenter,
|
||||
nodeName,
|
||||
&net.TCPAddr{IP: net.IPv4(198, 18, 0, 1), Port: 8300},
|
||||
"AutoEncrypt.Sign",
|
||||
&request,
|
||||
&structs.SignedResponse{},
|
||||
).Once().Return(fmt.Errorf("injected error"))
|
||||
// second failure
|
||||
mcfg.directRPC.On("RPC",
|
||||
datacenter,
|
||||
nodeName,
|
||||
&net.TCPAddr{IP: net.IPv4(198, 18, 0, 2), Port: 8300},
|
||||
"AutoEncrypt.Sign",
|
||||
&request,
|
||||
&structs.SignedResponse{},
|
||||
).Once().Return(fmt.Errorf("injected error"))
|
||||
// third times is successfuly (second attempt to first server)
|
||||
mcfg.directRPC.On("RPC",
|
||||
datacenter,
|
||||
nodeName,
|
||||
&net.TCPAddr{IP: net.IPv4(198, 18, 0, 1), Port: 8300},
|
||||
"AutoEncrypt.Sign",
|
||||
&request,
|
||||
&structs.SignedResponse{},
|
||||
).Once().Return(nil).Run(func(args mock.Arguments) {
|
||||
resp, ok := args.Get(5).(*structs.SignedResponse)
|
||||
require.True(t, ok)
|
||||
resp.ConnectCARoots = *indexedRoots
|
||||
resp.IssuedCert = *cert
|
||||
resp.VerifyServerHostname = true
|
||||
})
|
||||
|
||||
mcfg.Config.Waiter = lib.NewRetryWaiter(2, 0, 1*time.Millisecond, nil)
|
||||
|
||||
ac := AutoConfig{
|
||||
config: &config.RuntimeConfig{
|
||||
Datacenter: datacenter,
|
||||
NodeName: nodeName,
|
||||
RetryJoinLAN: []string{"198.18.0.1:1234", "198.18.0.2:3456"},
|
||||
ServerPort: 8300,
|
||||
},
|
||||
acConfig: mcfg.Config,
|
||||
logger: testutil.Logger(t),
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
defer cancel()
|
||||
resp, err := ac.autoEncryptInitialCerts(ctx)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, resp)
|
||||
require.True(t, resp.VerifyServerHostname)
|
||||
require.NotEmpty(t, resp.IssuedCert.PrivateKeyPEM)
|
||||
resp.IssuedCert.PrivateKeyPEM = ""
|
||||
cert.PrivateKeyPEM = ""
|
||||
require.Equal(t, cert, &resp.IssuedCert)
|
||||
require.Equal(t, indexedRoots, &resp.ConnectCARoots)
|
||||
require.Empty(t, resp.ManualCARoots)
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_InitialConfiguration(t *testing.T) {
|
||||
token := "010494ae-ee45-4433-903c-a58c91297714"
|
||||
nodeName := "auto-encrypt"
|
||||
datacenter := "dc1"
|
||||
|
||||
mcfg := newMockedConfig(t)
|
||||
loader := setupRuntimeConfig(t)
|
||||
loader.addConfigHCL(`
|
||||
auto_encrypt {
|
||||
tls = true
|
||||
}
|
||||
`)
|
||||
loader.opts.Config.NodeName = &nodeName
|
||||
mcfg.Config.Loader = loader.Load
|
||||
|
||||
indexedRoots, cert, extraCerts := mcfg.setupInitialTLS(t, nodeName, datacenter, token)
|
||||
|
||||
// prepopulation is going to grab the token to populate the correct cache key
|
||||
mcfg.tokens.On("AgentToken").Return(token).Times(0)
|
||||
|
||||
// no server provider
|
||||
mcfg.serverProvider.On("FindLANServer").Return(&metadata.Server{Addr: &net.TCPAddr{IP: net.IPv4(127, 0, 0, 1), Port: 8300}}).Times(1)
|
||||
|
||||
populateResponse := func(args mock.Arguments) {
|
||||
resp, ok := args.Get(5).(*structs.SignedResponse)
|
||||
require.True(t, ok)
|
||||
*resp = structs.SignedResponse{
|
||||
VerifyServerHostname: true,
|
||||
ConnectCARoots: *indexedRoots,
|
||||
IssuedCert: *cert,
|
||||
ManualCARoots: extraCerts,
|
||||
}
|
||||
}
|
||||
|
||||
expectedRequest := structs.CASignRequest{
|
||||
WriteRequest: structs.WriteRequest{Token: token},
|
||||
Datacenter: datacenter,
|
||||
// TODO (autoconf) Maybe in the future we should populate a CSR
|
||||
// and do some manual parsing/verification of the contents. The
|
||||
// bits not having to do with the signing key such as the requested
|
||||
// SANs and CN. For now though the mockDirectRPC type will empty
|
||||
// the CSR so we have to pass in an empty string to the expectation.
|
||||
CSR: "",
|
||||
}
|
||||
|
||||
mcfg.directRPC.On(
|
||||
"RPC",
|
||||
datacenter,
|
||||
nodeName,
|
||||
&net.TCPAddr{IP: net.IPv4(127, 0, 0, 1), Port: 8300},
|
||||
"AutoEncrypt.Sign",
|
||||
&expectedRequest,
|
||||
&structs.SignedResponse{}).Return(nil).Run(populateResponse)
|
||||
|
||||
ac, err := New(mcfg.Config)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, ac)
|
||||
|
||||
cfg, err := ac.InitialConfiguration(context.Background())
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, cfg)
|
||||
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_TokenUpdate(t *testing.T) {
|
||||
testAC := startedAutoConfig(t, true)
|
||||
|
||||
newToken := "1a4cc445-86ed-46b4-a355-bbf5a11dddb0"
|
||||
|
||||
rootsCtx, rootsCancel := context.WithCancel(context.Background())
|
||||
testAC.mcfg.cache.On("Notify",
|
||||
mock.Anything,
|
||||
cachetype.ConnectCARootName,
|
||||
&structs.DCSpecificRequest{Datacenter: testAC.ac.config.Datacenter},
|
||||
rootsWatchID,
|
||||
mock.Anything,
|
||||
).Return(nil).Once().Run(func(args mock.Arguments) {
|
||||
rootsCancel()
|
||||
})
|
||||
|
||||
leafCtx, leafCancel := context.WithCancel(context.Background())
|
||||
testAC.mcfg.cache.On("Notify",
|
||||
mock.Anything,
|
||||
cachetype.ConnectCALeafName,
|
||||
&cachetype.ConnectCALeafRequest{
|
||||
Datacenter: "dc1",
|
||||
Agent: "autoconf",
|
||||
Token: newToken,
|
||||
DNSSAN: defaultDNSSANs,
|
||||
IPSAN: defaultIPSANs,
|
||||
},
|
||||
leafWatchID,
|
||||
mock.Anything,
|
||||
).Return(nil).Once().Run(func(args mock.Arguments) {
|
||||
leafCancel()
|
||||
})
|
||||
|
||||
// this will be retrieved once when resetting the leaf cert watch
|
||||
testAC.mcfg.tokens.On("AgentToken").Return(newToken).Once()
|
||||
|
||||
// send the notification about the token update
|
||||
testAC.tokenUpdates <- struct{}{}
|
||||
|
||||
// wait for the leaf cert watches
|
||||
require.True(t, waitForChans(100*time.Millisecond, leafCtx.Done(), rootsCtx.Done()), "New cache watches were not started within 100ms")
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_RootsUpdate(t *testing.T) {
|
||||
testAC := startedAutoConfig(t, true)
|
||||
|
||||
secondCA := connect.TestCA(t, testAC.initialRoots.Roots[0])
|
||||
secondRoots := structs.IndexedCARoots{
|
||||
ActiveRootID: secondCA.ID,
|
||||
TrustDomain: connect.TestClusterID,
|
||||
Roots: []*structs.CARoot{
|
||||
secondCA,
|
||||
testAC.initialRoots.Roots[0],
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{
|
||||
Index: 99,
|
||||
},
|
||||
}
|
||||
|
||||
updatedCtx, cancel := context.WithCancel(context.Background())
|
||||
testAC.mcfg.tlsCfg.On("UpdateAutoTLSCA",
|
||||
[]string{secondCA.RootCert, testAC.initialRoots.Roots[0].RootCert},
|
||||
).Return(nil).Once().Run(func(args mock.Arguments) {
|
||||
cancel()
|
||||
})
|
||||
|
||||
// when a cache event comes in we end up recalculating the fallback timer which requires this call
|
||||
testAC.mcfg.tlsCfg.On("AutoEncryptCertNotAfter").Return(time.Now().Add(10 * time.Minute)).Once()
|
||||
|
||||
req := structs.DCSpecificRequest{Datacenter: "dc1"}
|
||||
require.True(t, testAC.mcfg.cache.sendNotification(context.Background(), req.CacheInfo().Key, cache.UpdateEvent{
|
||||
CorrelationID: rootsWatchID,
|
||||
Result: &secondRoots,
|
||||
Meta: cache.ResultMeta{
|
||||
Index: secondRoots.Index,
|
||||
},
|
||||
}))
|
||||
|
||||
require.True(t, waitForChans(100*time.Millisecond, updatedCtx.Done()), "TLS certificates were not updated within the alotted time")
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_CertUpdate(t *testing.T) {
|
||||
testAC := startedAutoConfig(t, true)
|
||||
secondCert := newLeaf(t, "autoconf", "dc1", testAC.initialRoots.Roots[0], 99, 10*time.Minute)
|
||||
|
||||
updatedCtx, cancel := context.WithCancel(context.Background())
|
||||
testAC.mcfg.tlsCfg.On("UpdateAutoTLSCert",
|
||||
secondCert.CertPEM,
|
||||
"redacted",
|
||||
).Return(nil).Once().Run(func(args mock.Arguments) {
|
||||
cancel()
|
||||
})
|
||||
|
||||
// when a cache event comes in we end up recalculating the fallback timer which requires this call
|
||||
testAC.mcfg.tlsCfg.On("AutoEncryptCertNotAfter").Return(secondCert.ValidBefore).Once()
|
||||
|
||||
req := cachetype.ConnectCALeafRequest{
|
||||
Datacenter: "dc1",
|
||||
Agent: "autoconf",
|
||||
Token: testAC.originalToken,
|
||||
DNSSAN: defaultDNSSANs,
|
||||
IPSAN: defaultIPSANs,
|
||||
}
|
||||
require.True(t, testAC.mcfg.cache.sendNotification(context.Background(), req.CacheInfo().Key, cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: secondCert,
|
||||
Meta: cache.ResultMeta{
|
||||
Index: secondCert.ModifyIndex,
|
||||
},
|
||||
}))
|
||||
|
||||
require.True(t, waitForChans(100*time.Millisecond, updatedCtx.Done()), "TLS certificates were not updated within the alotted time")
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_Fallback(t *testing.T) {
|
||||
testAC := startedAutoConfig(t, true)
|
||||
|
||||
// at this point everything is operating normally and we are just
|
||||
// waiting for events. We are going to send a new cert that is basically
|
||||
// already expired and then allow the fallback routine to kick in.
|
||||
secondCert := newLeaf(t, "autoconf", "dc1", testAC.initialRoots.Roots[0], 100, time.Nanosecond)
|
||||
secondCA := connect.TestCA(t, testAC.initialRoots.Roots[0])
|
||||
secondRoots := structs.IndexedCARoots{
|
||||
ActiveRootID: secondCA.ID,
|
||||
TrustDomain: connect.TestClusterID,
|
||||
Roots: []*structs.CARoot{
|
||||
secondCA,
|
||||
testAC.initialRoots.Roots[0],
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{
|
||||
Index: 101,
|
||||
},
|
||||
}
|
||||
thirdCert := newLeaf(t, "autoconf", "dc1", secondCA, 102, 10*time.Minute)
|
||||
|
||||
// setup the expectation for when the certs get updated initially
|
||||
updatedCtx, updateCancel := context.WithCancel(context.Background())
|
||||
testAC.mcfg.tlsCfg.On("UpdateAutoTLSCert",
|
||||
secondCert.CertPEM,
|
||||
"redacted",
|
||||
).Return(nil).Once().Run(func(args mock.Arguments) {
|
||||
updateCancel()
|
||||
})
|
||||
|
||||
// when a cache event comes in we end up recalculating the fallback timer which requires this call
|
||||
testAC.mcfg.tlsCfg.On("AutoEncryptCertNotAfter").Return(secondCert.ValidBefore).Once()
|
||||
testAC.mcfg.tlsCfg.On("AutoEncryptCertExpired").Return(true).Once()
|
||||
|
||||
fallbackCtx, fallbackCancel := context.WithCancel(context.Background())
|
||||
|
||||
// also testing here that we can change server IPs for ongoing operations
|
||||
testAC.mcfg.serverProvider.On("FindLANServer").Once().Return(&metadata.Server{
|
||||
Addr: &net.TCPAddr{IP: net.IPv4(198, 18, 23, 2), Port: 8300},
|
||||
})
|
||||
|
||||
// after sending the notification for the cert update another InitialConfiguration RPC
|
||||
// will be made to pull down the latest configuration. So we need to set up the response
|
||||
// for the second RPC
|
||||
populateResponse := func(args mock.Arguments) {
|
||||
resp, ok := args.Get(5).(*structs.SignedResponse)
|
||||
require.True(t, ok)
|
||||
*resp = structs.SignedResponse{
|
||||
VerifyServerHostname: true,
|
||||
ConnectCARoots: secondRoots,
|
||||
IssuedCert: *thirdCert,
|
||||
ManualCARoots: testAC.extraCerts,
|
||||
}
|
||||
|
||||
fallbackCancel()
|
||||
}
|
||||
|
||||
expectedRequest := structs.CASignRequest{
|
||||
WriteRequest: structs.WriteRequest{Token: testAC.originalToken},
|
||||
Datacenter: "dc1",
|
||||
// TODO (autoconf) Maybe in the future we should populate a CSR
|
||||
// and do some manual parsing/verification of the contents. The
|
||||
// bits not having to do with the signing key such as the requested
|
||||
// SANs and CN. For now though the mockDirectRPC type will empty
|
||||
// the CSR so we have to pass in an empty string to the expectation.
|
||||
CSR: "",
|
||||
}
|
||||
|
||||
// the fallback routine to perform auto-encrypt again will need to grab this
|
||||
testAC.mcfg.tokens.On("AgentToken").Return(testAC.originalToken).Once()
|
||||
|
||||
testAC.mcfg.directRPC.On(
|
||||
"RPC",
|
||||
"dc1",
|
||||
"autoconf",
|
||||
&net.TCPAddr{IP: net.IPv4(198, 18, 23, 2), Port: 8300},
|
||||
"AutoEncrypt.Sign",
|
||||
&expectedRequest,
|
||||
&structs.SignedResponse{}).Return(nil).Run(populateResponse).Once()
|
||||
|
||||
testAC.mcfg.expectInitialTLS(t, "autoconf", "dc1", testAC.originalToken, secondCA, &secondRoots, thirdCert, testAC.extraCerts)
|
||||
|
||||
// after the second RPC we now will use the new certs validity period in the next run loop iteration
|
||||
testAC.mcfg.tlsCfg.On("AutoEncryptCertNotAfter").Return(time.Now().Add(10 * time.Minute)).Once()
|
||||
|
||||
// now that all the mocks are set up we can trigger the whole thing by sending the second expired cert
|
||||
// as a cache update event.
|
||||
req := cachetype.ConnectCALeafRequest{
|
||||
Datacenter: "dc1",
|
||||
Agent: "autoconf",
|
||||
Token: testAC.originalToken,
|
||||
DNSSAN: defaultDNSSANs,
|
||||
IPSAN: defaultIPSANs,
|
||||
}
|
||||
require.True(t, testAC.mcfg.cache.sendNotification(context.Background(), req.CacheInfo().Key, cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: secondCert,
|
||||
Meta: cache.ResultMeta{
|
||||
Index: secondCert.ModifyIndex,
|
||||
},
|
||||
}))
|
||||
|
||||
// wait for the TLS certificates to get updated
|
||||
require.True(t, waitForChans(100*time.Millisecond, updatedCtx.Done()), "TLS certificates were not updated within the alotted time")
|
||||
|
||||
// now wait for the fallback routine to be invoked
|
||||
require.True(t, waitForChans(100*time.Millisecond, fallbackCtx.Done()), "fallback routines did not get invoked within the alotted time")
|
||||
}
|
|
@ -3,9 +3,12 @@ package autoconf
|
|||
import (
|
||||
"context"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
@ -18,12 +21,35 @@ type DirectRPC interface {
|
|||
RPC(dc string, node string, addr net.Addr, method string, args interface{}, reply interface{}) error
|
||||
}
|
||||
|
||||
// CertMonitor is the interface that needs to be satisfied for AutoConfig to be able to
|
||||
// setup monitoring of the Connect TLS certificate after we first get it.
|
||||
type CertMonitor interface {
|
||||
Update(*structs.SignedResponse) error
|
||||
Start(context.Context) (<-chan struct{}, error)
|
||||
Stop() bool
|
||||
// Cache is an interface to represent the methods of the
|
||||
// agent/cache.Cache struct that we care about
|
||||
type Cache interface {
|
||||
Notify(ctx context.Context, t string, r cache.Request, correlationID string, ch chan<- cache.UpdateEvent) error
|
||||
Prepopulate(t string, result cache.FetchResult, dc string, token string, key string) error
|
||||
}
|
||||
|
||||
// ServerProvider is an interface that can be used to find one server in the local DC known to
|
||||
// the agent via Gossip
|
||||
type ServerProvider interface {
|
||||
FindLANServer() *metadata.Server
|
||||
}
|
||||
|
||||
// TLSConfigurator is an interface of the methods on the tlsutil.Configurator that we will require at
|
||||
// runtime.
|
||||
type TLSConfigurator interface {
|
||||
UpdateAutoTLS(manualCAPEMs, connectCAPEMs []string, pub, priv string, verifyServerHostname bool) error
|
||||
UpdateAutoTLSCA([]string) error
|
||||
UpdateAutoTLSCert(pub, priv string) error
|
||||
AutoEncryptCertNotAfter() time.Time
|
||||
AutoEncryptCertExpired() bool
|
||||
}
|
||||
|
||||
// TokenStore is an interface of the methods we will need to use from the token.Store.
|
||||
type TokenStore interface {
|
||||
AgentToken() string
|
||||
UpdateAgentToken(secret string, source token.TokenSource) bool
|
||||
Notify(kind token.TokenKind) token.Notifier
|
||||
StopNotify(notifier token.Notifier)
|
||||
}
|
||||
|
||||
// Config contains all the tunables for AutoConfig
|
||||
|
@ -37,6 +63,10 @@ type Config struct {
|
|||
// configuration. Setting this field is required.
|
||||
DirectRPC DirectRPC
|
||||
|
||||
// ServerProvider is the interfaced to be used by AutoConfig to find any
|
||||
// known servers during fallback operations.
|
||||
ServerProvider ServerProvider
|
||||
|
||||
// Waiter is a RetryWaiter to be used during retrieval of the
|
||||
// initial configuration. When a round of requests fails we will
|
||||
// wait and eventually make another round of requests (1 round
|
||||
|
@ -49,14 +79,28 @@ type Config struct {
|
|||
// having the test take minutes/hours to complete.
|
||||
Waiter *lib.RetryWaiter
|
||||
|
||||
// CertMonitor is the Connect TLS Certificate Monitor to be used for ongoing
|
||||
// certificate renewals and connect CA roots updates. This field is not
|
||||
// strictly required but if not provided the TLS certificates retrieved
|
||||
// through by the AutoConfig.InitialConfiguration RPC will not be used
|
||||
// or renewed.
|
||||
CertMonitor CertMonitor
|
||||
|
||||
// Loader merges source with the existing FileSources and returns the complete
|
||||
// RuntimeConfig.
|
||||
Loader func(source config.Source) (cfg *config.RuntimeConfig, warnings []string, err error)
|
||||
|
||||
// TLSConfigurator is the shared TLS Configurator. AutoConfig will update the
|
||||
// auto encrypt/auto config certs as they are renewed.
|
||||
TLSConfigurator TLSConfigurator
|
||||
|
||||
// Cache is an object implementing our Cache interface. The Cache
|
||||
// used at runtime must be able to handle Roots and Leaf Cert watches
|
||||
Cache Cache
|
||||
|
||||
// FallbackLeeway is the amount of time after certificate expiration before
|
||||
// invoking the fallback routine. If not set this will default to 10s.
|
||||
FallbackLeeway time.Duration
|
||||
|
||||
// FallbackRetry is the duration between Fallback invocations when the configured
|
||||
// fallback routine returns an error. If not set this will default to 1m.
|
||||
FallbackRetry time.Duration
|
||||
|
||||
// Tokens is the shared token store. It is used to retrieve the current
|
||||
// agent token as well as getting notifications when that token is updated.
|
||||
// This field is required.
|
||||
Tokens TokenStore
|
||||
}
|
||||
|
|
|
@ -22,9 +22,9 @@ import (
|
|||
// package cannot import the agent/config package without running into import cycles.
|
||||
func translateConfig(c *pbconfig.Config) config.Config {
|
||||
result := config.Config{
|
||||
Datacenter: &c.Datacenter,
|
||||
PrimaryDatacenter: &c.PrimaryDatacenter,
|
||||
NodeName: &c.NodeName,
|
||||
Datacenter: stringPtrOrNil(c.Datacenter),
|
||||
PrimaryDatacenter: stringPtrOrNil(c.PrimaryDatacenter),
|
||||
NodeName: stringPtrOrNil(c.NodeName),
|
||||
// only output the SegmentName in the configuration if its non-empty
|
||||
// this will avoid a warning later when parsing the persisted configuration
|
||||
SegmentName: stringPtrOrNil(c.SegmentName),
|
||||
|
@ -42,13 +42,13 @@ func translateConfig(c *pbconfig.Config) config.Config {
|
|||
if a := c.ACL; a != nil {
|
||||
result.ACL = config.ACL{
|
||||
Enabled: &a.Enabled,
|
||||
PolicyTTL: &a.PolicyTTL,
|
||||
RoleTTL: &a.RoleTTL,
|
||||
TokenTTL: &a.TokenTTL,
|
||||
DownPolicy: &a.DownPolicy,
|
||||
DefaultPolicy: &a.DefaultPolicy,
|
||||
PolicyTTL: stringPtrOrNil(a.PolicyTTL),
|
||||
RoleTTL: stringPtrOrNil(a.RoleTTL),
|
||||
TokenTTL: stringPtrOrNil(a.TokenTTL),
|
||||
DownPolicy: stringPtrOrNil(a.DownPolicy),
|
||||
DefaultPolicy: stringPtrOrNil(a.DefaultPolicy),
|
||||
EnableKeyListPolicy: &a.EnableKeyListPolicy,
|
||||
DisabledTTL: &a.DisabledTTL,
|
||||
DisabledTTL: stringPtrOrNil(a.DisabledTTL),
|
||||
EnableTokenPersistence: &a.EnableTokenPersistence,
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,7 @@ func translateConfig(c *pbconfig.Config) config.Config {
|
|||
result.RetryJoinLAN = g.RetryJoinLAN
|
||||
|
||||
if e := c.Gossip.Encryption; e != nil {
|
||||
result.EncryptKey = &e.Key
|
||||
result.EncryptKey = stringPtrOrNil(e.Key)
|
||||
result.EncryptVerifyIncoming = &e.VerifyIncoming
|
||||
result.EncryptVerifyOutgoing = &e.VerifyOutgoing
|
||||
}
|
||||
|
|
|
@ -1,10 +1,13 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
pbconfig "github.com/hashicorp/consul/proto/pbconfig"
|
||||
"github.com/hashicorp/consul/proto/pbconnect"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
|
@ -16,6 +19,38 @@ func boolPointer(b bool) *bool {
|
|||
return &b
|
||||
}
|
||||
|
||||
func translateCARootToProtobuf(in *structs.CARoot) (*pbconnect.CARoot, error) {
|
||||
var out pbconnect.CARoot
|
||||
if err := mapstructureTranslateToProtobuf(in, &out); err != nil {
|
||||
return nil, fmt.Errorf("Failed to re-encode CA Roots: %w", err)
|
||||
}
|
||||
return &out, nil
|
||||
}
|
||||
|
||||
func mustTranslateCARootToProtobuf(t *testing.T, in *structs.CARoot) *pbconnect.CARoot {
|
||||
out, err := translateCARootToProtobuf(in)
|
||||
require.NoError(t, err)
|
||||
return out
|
||||
}
|
||||
|
||||
func mustTranslateCARootsToStructs(t *testing.T, in *pbconnect.CARoots) *structs.IndexedCARoots {
|
||||
out, err := translateCARootsToStructs(in)
|
||||
require.NoError(t, err)
|
||||
return out
|
||||
}
|
||||
|
||||
func mustTranslateCARootsToProtobuf(t *testing.T, in *structs.IndexedCARoots) *pbconnect.CARoots {
|
||||
out, err := translateCARootsToProtobuf(in)
|
||||
require.NoError(t, err)
|
||||
return out
|
||||
}
|
||||
|
||||
func mustTranslateIssuedCertToProtobuf(t *testing.T, in *structs.IssuedCert) *pbconnect.IssuedCert {
|
||||
out, err := translateIssuedCertToProtobuf(in)
|
||||
require.NoError(t, err)
|
||||
return out
|
||||
}
|
||||
|
||||
func TestTranslateConfig(t *testing.T) {
|
||||
original := pbconfig.Config{
|
||||
Datacenter: "abc",
|
||||
|
@ -119,3 +154,9 @@ func TestTranslateConfig(t *testing.T) {
|
|||
translated := translateConfig(&original)
|
||||
require.Equal(t, expected, translated)
|
||||
}
|
||||
|
||||
func TestCArootsTranslation(t *testing.T) {
|
||||
_, indexedRoots, _ := testCerts(t, "autoconf", "dc1")
|
||||
protoRoots := mustTranslateCARootsToProtobuf(t, indexedRoots)
|
||||
require.Equal(t, indexedRoots, mustTranslateCARootsToStructs(t, protoRoots))
|
||||
}
|
||||
|
|
|
@ -0,0 +1,337 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/proto/pbautoconf"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
"github.com/stretchr/testify/mock"
|
||||
)
|
||||
|
||||
type mockDirectRPC struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
func newMockDirectRPC(t *testing.T) *mockDirectRPC {
|
||||
m := mockDirectRPC{}
|
||||
m.Test(t)
|
||||
return &m
|
||||
}
|
||||
|
||||
func (m *mockDirectRPC) RPC(dc string, node string, addr net.Addr, method string, args interface{}, reply interface{}) error {
|
||||
var retValues mock.Arguments
|
||||
if method == "AutoConfig.InitialConfiguration" {
|
||||
req := args.(*pbautoconf.AutoConfigRequest)
|
||||
csr := req.CSR
|
||||
req.CSR = ""
|
||||
retValues = m.Called(dc, node, addr, method, args, reply)
|
||||
req.CSR = csr
|
||||
} else if method == "AutoEncrypt.Sign" {
|
||||
req := args.(*structs.CASignRequest)
|
||||
csr := req.CSR
|
||||
req.CSR = ""
|
||||
retValues = m.Called(dc, node, addr, method, args, reply)
|
||||
req.CSR = csr
|
||||
} else {
|
||||
retValues = m.Called(dc, node, addr, method, args, reply)
|
||||
}
|
||||
|
||||
return retValues.Error(0)
|
||||
}
|
||||
|
||||
type mockTLSConfigurator struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
func newMockTLSConfigurator(t *testing.T) *mockTLSConfigurator {
|
||||
m := mockTLSConfigurator{}
|
||||
m.Test(t)
|
||||
return &m
|
||||
}
|
||||
|
||||
func (m *mockTLSConfigurator) UpdateAutoTLS(manualCAPEMs, connectCAPEMs []string, pub, priv string, verifyServerHostname bool) error {
|
||||
if priv != "" {
|
||||
priv = "redacted"
|
||||
}
|
||||
|
||||
ret := m.Called(manualCAPEMs, connectCAPEMs, pub, priv, verifyServerHostname)
|
||||
return ret.Error(0)
|
||||
}
|
||||
|
||||
func (m *mockTLSConfigurator) UpdateAutoTLSCA(pems []string) error {
|
||||
ret := m.Called(pems)
|
||||
return ret.Error(0)
|
||||
}
|
||||
func (m *mockTLSConfigurator) UpdateAutoTLSCert(pub, priv string) error {
|
||||
if priv != "" {
|
||||
priv = "redacted"
|
||||
}
|
||||
ret := m.Called(pub, priv)
|
||||
return ret.Error(0)
|
||||
}
|
||||
func (m *mockTLSConfigurator) AutoEncryptCertNotAfter() time.Time {
|
||||
ret := m.Called()
|
||||
ts, _ := ret.Get(0).(time.Time)
|
||||
|
||||
return ts
|
||||
}
|
||||
func (m *mockTLSConfigurator) AutoEncryptCertExpired() bool {
|
||||
ret := m.Called()
|
||||
return ret.Bool(0)
|
||||
}
|
||||
|
||||
type mockServerProvider struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
func newMockServerProvider(t *testing.T) *mockServerProvider {
|
||||
m := mockServerProvider{}
|
||||
m.Test(t)
|
||||
return &m
|
||||
}
|
||||
|
||||
func (m *mockServerProvider) FindLANServer() *metadata.Server {
|
||||
ret := m.Called()
|
||||
srv, _ := ret.Get(0).(*metadata.Server)
|
||||
return srv
|
||||
}
|
||||
|
||||
type mockWatcher struct {
|
||||
ch chan<- cache.UpdateEvent
|
||||
done <-chan struct{}
|
||||
}
|
||||
|
||||
type mockCache struct {
|
||||
mock.Mock
|
||||
|
||||
lock sync.Mutex
|
||||
watchers map[string][]mockWatcher
|
||||
}
|
||||
|
||||
func newMockCache(t *testing.T) *mockCache {
|
||||
m := mockCache{
|
||||
watchers: make(map[string][]mockWatcher),
|
||||
}
|
||||
m.Test(t)
|
||||
return &m
|
||||
}
|
||||
|
||||
func (m *mockCache) Notify(ctx context.Context, t string, r cache.Request, correlationID string, ch chan<- cache.UpdateEvent) error {
|
||||
ret := m.Called(ctx, t, r, correlationID, ch)
|
||||
|
||||
err := ret.Error(0)
|
||||
if err == nil {
|
||||
m.lock.Lock()
|
||||
key := r.CacheInfo().Key
|
||||
m.watchers[key] = append(m.watchers[key], mockWatcher{ch: ch, done: ctx.Done()})
|
||||
m.lock.Unlock()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (m *mockCache) Prepopulate(t string, result cache.FetchResult, dc string, token string, key string) error {
|
||||
var restore string
|
||||
cert, ok := result.Value.(*structs.IssuedCert)
|
||||
if ok {
|
||||
// we cannot know what the private key is prior to it being injected into the cache.
|
||||
// therefore redact it here and all mock expectations should take that into account
|
||||
restore = cert.PrivateKeyPEM
|
||||
cert.PrivateKeyPEM = "redacted"
|
||||
}
|
||||
|
||||
ret := m.Called(t, result, dc, token, key)
|
||||
|
||||
if ok && restore != "" {
|
||||
cert.PrivateKeyPEM = restore
|
||||
}
|
||||
return ret.Error(0)
|
||||
}
|
||||
|
||||
func (m *mockCache) sendNotification(ctx context.Context, key string, u cache.UpdateEvent) bool {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
|
||||
watchers, ok := m.watchers[key]
|
||||
if !ok || len(m.watchers) < 1 {
|
||||
return false
|
||||
}
|
||||
|
||||
var newWatchers []mockWatcher
|
||||
|
||||
for _, watcher := range watchers {
|
||||
select {
|
||||
case watcher.ch <- u:
|
||||
newWatchers = append(newWatchers, watcher)
|
||||
case <-watcher.done:
|
||||
// do nothing, this watcher will be removed from the list
|
||||
case <-ctx.Done():
|
||||
// return doesn't matter here really, the test is being cancelled
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
// this removes any already cancelled watches from being sent to
|
||||
m.watchers[key] = newWatchers
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
type mockTokenStore struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
func newMockTokenStore(t *testing.T) *mockTokenStore {
|
||||
m := mockTokenStore{}
|
||||
m.Test(t)
|
||||
return &m
|
||||
}
|
||||
|
||||
func (m *mockTokenStore) AgentToken() string {
|
||||
ret := m.Called()
|
||||
return ret.String(0)
|
||||
}
|
||||
|
||||
func (m *mockTokenStore) UpdateAgentToken(secret string, source token.TokenSource) bool {
|
||||
return m.Called(secret, source).Bool(0)
|
||||
}
|
||||
|
||||
func (m *mockTokenStore) Notify(kind token.TokenKind) token.Notifier {
|
||||
ret := m.Called(kind)
|
||||
n, _ := ret.Get(0).(token.Notifier)
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *mockTokenStore) StopNotify(notifier token.Notifier) {
|
||||
m.Called(notifier)
|
||||
}
|
||||
|
||||
type mockedConfig struct {
|
||||
Config
|
||||
|
||||
directRPC *mockDirectRPC
|
||||
serverProvider *mockServerProvider
|
||||
cache *mockCache
|
||||
tokens *mockTokenStore
|
||||
tlsCfg *mockTLSConfigurator
|
||||
}
|
||||
|
||||
func newMockedConfig(t *testing.T) *mockedConfig {
|
||||
directRPC := newMockDirectRPC(t)
|
||||
serverProvider := newMockServerProvider(t)
|
||||
mcache := newMockCache(t)
|
||||
tokens := newMockTokenStore(t)
|
||||
tlsCfg := newMockTLSConfigurator(t)
|
||||
|
||||
// I am not sure it is well defined behavior but in testing it
|
||||
// out it does appear like Cleanup functions can fail tests
|
||||
// Adding in the mock expectations assertions here saves us
|
||||
// a bunch of code in the other test functions.
|
||||
t.Cleanup(func() {
|
||||
if !t.Failed() {
|
||||
directRPC.AssertExpectations(t)
|
||||
serverProvider.AssertExpectations(t)
|
||||
mcache.AssertExpectations(t)
|
||||
tokens.AssertExpectations(t)
|
||||
tlsCfg.AssertExpectations(t)
|
||||
}
|
||||
})
|
||||
|
||||
return &mockedConfig{
|
||||
Config: Config{
|
||||
DirectRPC: directRPC,
|
||||
ServerProvider: serverProvider,
|
||||
Cache: mcache,
|
||||
Tokens: tokens,
|
||||
TLSConfigurator: tlsCfg,
|
||||
Logger: testutil.Logger(t),
|
||||
},
|
||||
directRPC: directRPC,
|
||||
serverProvider: serverProvider,
|
||||
cache: mcache,
|
||||
tokens: tokens,
|
||||
tlsCfg: tlsCfg,
|
||||
}
|
||||
}
|
||||
|
||||
func (m *mockedConfig) expectInitialTLS(t *testing.T, agentName, datacenter, token string, ca *structs.CARoot, indexedRoots *structs.IndexedCARoots, cert *structs.IssuedCert, extraCerts []string) {
|
||||
var pems []string
|
||||
for _, root := range indexedRoots.Roots {
|
||||
pems = append(pems, root.RootCert)
|
||||
}
|
||||
|
||||
// we should update the TLS configurator with the proper certs
|
||||
m.tlsCfg.On("UpdateAutoTLS",
|
||||
extraCerts,
|
||||
pems,
|
||||
cert.CertPEM,
|
||||
// auto-config handles the CSR and Key so our tests don't have
|
||||
// a way to know that the key is correct or not. We do replace
|
||||
// a non empty PEM with "redacted" so we can ensure that some
|
||||
// certificate is being sent
|
||||
"redacted",
|
||||
true,
|
||||
).Return(nil).Once()
|
||||
|
||||
rootRes := cache.FetchResult{Value: indexedRoots, Index: indexedRoots.QueryMeta.Index}
|
||||
rootsReq := structs.DCSpecificRequest{Datacenter: datacenter}
|
||||
|
||||
// we should prepopulate the cache with the CA roots
|
||||
m.cache.On("Prepopulate",
|
||||
cachetype.ConnectCARootName,
|
||||
rootRes,
|
||||
datacenter,
|
||||
"",
|
||||
rootsReq.CacheInfo().Key,
|
||||
).Return(nil).Once()
|
||||
|
||||
leafReq := cachetype.ConnectCALeafRequest{
|
||||
Token: token,
|
||||
Agent: agentName,
|
||||
Datacenter: datacenter,
|
||||
}
|
||||
|
||||
// copy the cert and redact the private key for the mock expectation
|
||||
// the actual private key will not correspond to the cert but thats
|
||||
// because AutoConfig is generated a key/csr internally and sending that
|
||||
// on up with the request.
|
||||
copy := *cert
|
||||
copy.PrivateKeyPEM = "redacted"
|
||||
leafRes := cache.FetchResult{
|
||||
Value: ©,
|
||||
Index: copy.RaftIndex.ModifyIndex,
|
||||
State: cachetype.ConnectCALeafSuccess(ca.SigningKeyID),
|
||||
}
|
||||
|
||||
// we should prepopulate the cache with the agents cert
|
||||
m.cache.On("Prepopulate",
|
||||
cachetype.ConnectCALeafName,
|
||||
leafRes,
|
||||
datacenter,
|
||||
token,
|
||||
leafReq.Key(),
|
||||
).Return(nil).Once()
|
||||
|
||||
// when prepopulating the cert in the cache we grab the token so
|
||||
// we should expec that here
|
||||
m.tokens.On("AgentToken").Return(token).Once()
|
||||
}
|
||||
|
||||
func (m *mockedConfig) setupInitialTLS(t *testing.T, agentName, datacenter, token string) (*structs.IndexedCARoots, *structs.IssuedCert, []string) {
|
||||
ca, indexedRoots, cert := testCerts(t, agentName, datacenter)
|
||||
|
||||
ca2 := connect.TestCA(t, nil)
|
||||
extraCerts := []string{ca2.RootCert}
|
||||
|
||||
m.expectInitialTLS(t, agentName, datacenter, token, ca, indexedRoots, cert, extraCerts)
|
||||
return indexedRoots, cert, extraCerts
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
"github.com/hashicorp/consul/proto/pbautoconf"
|
||||
)
|
||||
|
||||
const (
|
||||
// autoConfigFileName is the name of the file that the agent auto-config settings are
|
||||
// stored in within the data directory
|
||||
autoConfigFileName = "auto-config.json"
|
||||
)
|
||||
|
||||
var (
|
||||
pbMarshaler = &jsonpb.Marshaler{
|
||||
OrigName: false,
|
||||
EnumsAsInts: false,
|
||||
Indent: " ",
|
||||
EmitDefaults: true,
|
||||
}
|
||||
|
||||
pbUnmarshaler = &jsonpb.Unmarshaler{
|
||||
AllowUnknownFields: false,
|
||||
}
|
||||
)
|
||||
|
||||
func (ac *AutoConfig) readPersistedAutoConfig() (*pbautoconf.AutoConfigResponse, error) {
|
||||
if ac.config.DataDir == "" {
|
||||
// no data directory means we don't have anything to potentially load
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
path := filepath.Join(ac.config.DataDir, autoConfigFileName)
|
||||
ac.logger.Debug("attempting to restore any persisted configuration", "path", path)
|
||||
|
||||
content, err := ioutil.ReadFile(path)
|
||||
if err == nil {
|
||||
rdr := strings.NewReader(string(content))
|
||||
|
||||
var resp pbautoconf.AutoConfigResponse
|
||||
if err := pbUnmarshaler.Unmarshal(rdr, &resp); err != nil {
|
||||
return nil, fmt.Errorf("failed to decode persisted auto-config data: %w", err)
|
||||
}
|
||||
|
||||
ac.logger.Info("read persisted configuration", "path", path)
|
||||
return &resp, nil
|
||||
}
|
||||
|
||||
if !os.IsNotExist(err) {
|
||||
return nil, fmt.Errorf("failed to load %s: %w", path, err)
|
||||
}
|
||||
|
||||
// ignore non-existence errors as that is an indicator that we haven't
|
||||
// performed the auto configuration before
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) persistAutoConfig(resp *pbautoconf.AutoConfigResponse) error {
|
||||
// now that we know the configuration is generally fine including TLS certs go ahead and persist it to disk.
|
||||
if ac.config.DataDir == "" {
|
||||
ac.logger.Debug("not persisting auto-config settings because there is no data directory")
|
||||
return nil
|
||||
}
|
||||
|
||||
serialized, err := pbMarshaler.MarshalToString(resp)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to encode auto-config response as JSON: %w", err)
|
||||
}
|
||||
|
||||
path := filepath.Join(ac.config.DataDir, autoConfigFileName)
|
||||
|
||||
err = ioutil.WriteFile(path, []byte(serialized), 0660)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to write auto-config configurations: %w", err)
|
||||
}
|
||||
|
||||
ac.logger.Debug("auto-config settings were persisted to disk")
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,192 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
)
|
||||
|
||||
// handleCacheEvent is used to handle event notifications from the cache for the roots
|
||||
// or leaf cert watches.
|
||||
func (ac *AutoConfig) handleCacheEvent(u cache.UpdateEvent) error {
|
||||
switch u.CorrelationID {
|
||||
case rootsWatchID:
|
||||
ac.logger.Debug("roots watch fired - updating CA certificates")
|
||||
if u.Err != nil {
|
||||
return fmt.Errorf("root watch returned an error: %w", u.Err)
|
||||
}
|
||||
|
||||
roots, ok := u.Result.(*structs.IndexedCARoots)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for roots watch response: %T", u.Result)
|
||||
}
|
||||
|
||||
return ac.updateCARoots(roots)
|
||||
case leafWatchID:
|
||||
ac.logger.Debug("leaf certificate watch fired - updating TLS certificate")
|
||||
if u.Err != nil {
|
||||
return fmt.Errorf("leaf watch returned an error: %w", u.Err)
|
||||
}
|
||||
|
||||
leaf, ok := u.Result.(*structs.IssuedCert)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for agent leaf cert watch response: %T", u.Result)
|
||||
}
|
||||
|
||||
return ac.updateLeafCert(leaf)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// handleTokenUpdate is used when a notification about the agent token being updated
|
||||
// is received and various watches need cancelling/restarting to use the new token.
|
||||
func (ac *AutoConfig) handleTokenUpdate(ctx context.Context) error {
|
||||
ac.logger.Debug("Agent token updated - resetting watches")
|
||||
|
||||
// TODO (autoencrypt) Prepopulate the cache with the new token with
|
||||
// the existing cache entry with the old token. The certificate doesn't
|
||||
// need to change just because the token has. However there isn't a
|
||||
// good way to make that happen and this behavior is benign enough
|
||||
// that I am going to push off implementing it.
|
||||
|
||||
// the agent token has been updated so we must update our leaf cert watch.
|
||||
// this cancels the current watches before setting up new ones
|
||||
ac.cancelWatches()
|
||||
|
||||
// recreate the chan for cache updates. This is a precautionary measure to ensure
|
||||
// that we don't accidentally get notified for the new watches being setup before
|
||||
// a blocking query in the cache returns and sends data to the old chan. In theory
|
||||
// the code in agent/cache/watch.go should prevent this where we specifically check
|
||||
// for context cancellation prior to sending the event. However we could cancel
|
||||
// it after that check and finish setting up the new watches before getting the old
|
||||
// events. Both the go routine scheduler and the OS thread scheduler would have to
|
||||
// be acting up for this to happen. Regardless the way to ensure we don't get events
|
||||
// for the old watches is to simply replace the chan we are expecting them from.
|
||||
close(ac.cacheUpdates)
|
||||
ac.cacheUpdates = make(chan cache.UpdateEvent, 10)
|
||||
|
||||
// restart watches - this will be done with the correct token
|
||||
cancelWatches, err := ac.setupCertificateCacheWatches(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to restart watches after agent token update: %w", err)
|
||||
}
|
||||
ac.cancelWatches = cancelWatches
|
||||
return nil
|
||||
}
|
||||
|
||||
// handleFallback is used when the current TLS certificate has expired and the normal
|
||||
// updating mechanisms have failed to renew it quickly enough. This function will
|
||||
// use the configured fallback mechanism to retrieve a new cert and start monitoring
|
||||
// that one.
|
||||
func (ac *AutoConfig) handleFallback(ctx context.Context) error {
|
||||
ac.logger.Warn("agent's client certificate has expired")
|
||||
// Background because the context is mainly useful when the agent is first starting up.
|
||||
switch {
|
||||
case ac.config.AutoConfig.Enabled:
|
||||
resp, err := ac.getInitialConfiguration(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error while retrieving new agent certificates via auto-config: %w", err)
|
||||
}
|
||||
|
||||
return ac.recordInitialConfiguration(resp)
|
||||
case ac.config.AutoEncryptTLS:
|
||||
reply, err := ac.autoEncryptInitialCerts(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error while retrieving new agent certificate via auto-encrypt: %w", err)
|
||||
}
|
||||
return ac.setInitialTLSCertificates(reply)
|
||||
default:
|
||||
return fmt.Errorf("logic error: either auto-encrypt or auto-config must be enabled")
|
||||
}
|
||||
}
|
||||
|
||||
// run is the private method to be spawn by the Start method for
|
||||
// executing the main monitoring loop.
|
||||
func (ac *AutoConfig) run(ctx context.Context, exit chan struct{}) {
|
||||
// The fallbackTimer is used to notify AFTER the agents
|
||||
// leaf certificate has expired and where we need
|
||||
// to fall back to the less secure RPC endpoint just like
|
||||
// if the agent was starting up new.
|
||||
//
|
||||
// Check 10sec (fallback leeway duration) after cert
|
||||
// expires. The agent cache should be handling the expiration
|
||||
// and renew it before then.
|
||||
//
|
||||
// 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.
|
||||
calcFallbackInterval := func() time.Duration {
|
||||
certExpiry := ac.acConfig.TLSConfigurator.AutoEncryptCertNotAfter()
|
||||
return certExpiry.Add(ac.acConfig.FallbackLeeway).Sub(time.Now())
|
||||
}
|
||||
fallbackTimer := time.NewTimer(calcFallbackInterval())
|
||||
|
||||
// cleanup for once we are stopped
|
||||
defer func() {
|
||||
// cancel the go routines performing the cache watches
|
||||
ac.cancelWatches()
|
||||
// ensure we don't leak the timers go routine
|
||||
fallbackTimer.Stop()
|
||||
// stop receiving notifications for token updates
|
||||
ac.acConfig.Tokens.StopNotify(ac.tokenUpdates)
|
||||
|
||||
ac.logger.Debug("auto-config has been stopped")
|
||||
|
||||
ac.Lock()
|
||||
ac.cancel = nil
|
||||
ac.running = false
|
||||
// this should be the final cleanup task as its what notifies
|
||||
// the rest of the world that this go routine has exited.
|
||||
close(exit)
|
||||
ac.Unlock()
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
ac.logger.Debug("stopping auto-config")
|
||||
return
|
||||
case <-ac.tokenUpdates.Ch:
|
||||
ac.logger.Debug("handling a token update event")
|
||||
|
||||
if err := ac.handleTokenUpdate(ctx); err != nil {
|
||||
ac.logger.Error("error in handling token update event", "error", err)
|
||||
}
|
||||
case u := <-ac.cacheUpdates:
|
||||
ac.logger.Debug("handling a cache update event", "correlation_id", u.CorrelationID)
|
||||
|
||||
if err := ac.handleCacheEvent(u); err != nil {
|
||||
ac.logger.Error("error in handling cache update event", "error", err)
|
||||
}
|
||||
|
||||
// reset the fallback timer as the certificate may have been updated
|
||||
fallbackTimer.Stop()
|
||||
fallbackTimer = time.NewTimer(calcFallbackInterval())
|
||||
case <-fallbackTimer.C:
|
||||
// This is a safety net in case the cert doesn't get renewed
|
||||
// in time. The agent would be stuck in that case because the watches
|
||||
// never use the AutoEncrypt.Sign endpoint.
|
||||
|
||||
// check auto encrypt client cert expiration
|
||||
if ac.acConfig.TLSConfigurator.AutoEncryptCertExpired() {
|
||||
if err := ac.handleFallback(ctx); err != nil {
|
||||
ac.logger.Error("error when handling a certificate expiry event", "error", err)
|
||||
fallbackTimer = time.NewTimer(ac.acConfig.FallbackRetry)
|
||||
} else {
|
||||
fallbackTimer = time.NewTimer(calcFallbackInterval())
|
||||
}
|
||||
} else {
|
||||
// this shouldn't be possible. We calculate the timer duration to be the certificate
|
||||
// expiration time + some leeway (10s default). So whenever we get here the certificate
|
||||
// should be expired. Regardless its probably worth resetting the timer.
|
||||
fallbackTimer = time.NewTimer(calcFallbackInterval())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/go-discover"
|
||||
discoverk8s "github.com/hashicorp/go-discover/provider/k8s"
|
||||
|
||||
"github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
||||
func (ac *AutoConfig) discoverServers(servers []string) ([]string, error) {
|
||||
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),
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("Failed to create go-discover resolver: %w", err)
|
||||
}
|
||||
|
||||
var addrs []string
|
||||
for _, addr := range servers {
|
||||
switch {
|
||||
case strings.Contains(addr, "provider="):
|
||||
resolved, err := disco.Addrs(addr, ac.logger.StandardLogger(&hclog.StandardLoggerOptions{InferLevels: true}))
|
||||
if err != nil {
|
||||
ac.logger.Error("failed to resolve go-discover auto-config servers", "configuration", addr, "err", err)
|
||||
continue
|
||||
}
|
||||
|
||||
addrs = append(addrs, resolved...)
|
||||
ac.logger.Debug("discovered auto-config servers", "servers", resolved)
|
||||
default:
|
||||
addrs = append(addrs, addr)
|
||||
}
|
||||
}
|
||||
|
||||
return addrs, nil
|
||||
}
|
||||
|
||||
// autoConfigHosts is responsible for taking the list of server addresses
|
||||
// and resolving any go-discover provider invocations. It will then return
|
||||
// a list of hosts. These might be hostnames and is expected that DNS resolution
|
||||
// may be performed after this function runs. Additionally these may contain
|
||||
// ports so SplitHostPort could also be necessary.
|
||||
func (ac *AutoConfig) autoConfigHosts() ([]string, error) {
|
||||
// use servers known to gossip if there are any
|
||||
if ac.acConfig.ServerProvider != nil {
|
||||
if srv := ac.acConfig.ServerProvider.FindLANServer(); srv != nil {
|
||||
return []string{srv.Addr.String()}, nil
|
||||
}
|
||||
}
|
||||
|
||||
addrs, err := ac.discoverServers(ac.config.AutoConfig.ServerAddresses)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(addrs) == 0 {
|
||||
return nil, fmt.Errorf("no auto-config server addresses available for use")
|
||||
}
|
||||
|
||||
return addrs, nil
|
||||
}
|
||||
|
||||
// resolveHost will take a single host string and convert it to a list of TCPAddrs
|
||||
// This will process any port in the input as well as looking up the hostname using
|
||||
// normal DNS resolution.
|
||||
func (ac *AutoConfig) resolveHost(hostPort string) []net.TCPAddr {
|
||||
port := ac.config.ServerPort
|
||||
host, portStr, err := net.SplitHostPort(hostPort)
|
||||
if err != nil {
|
||||
if strings.Contains(err.Error(), "missing port in address") {
|
||||
host = hostPort
|
||||
} else {
|
||||
ac.logger.Warn("error splitting host address into IP and port", "address", hostPort, "error", err)
|
||||
return nil
|
||||
}
|
||||
} else {
|
||||
port, err = strconv.Atoi(portStr)
|
||||
if err != nil {
|
||||
ac.logger.Warn("Parsed port is not an integer", "port", portStr, "error", err)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// resolve the host to a list of IPs
|
||||
ips, err := net.LookupIP(host)
|
||||
if err != nil {
|
||||
ac.logger.Warn("IP resolution failed", "host", host, "error", err)
|
||||
return nil
|
||||
}
|
||||
|
||||
var addrs []net.TCPAddr
|
||||
for _, ip := range ips {
|
||||
addrs = append(addrs, net.TCPAddr{IP: ip, Port: port})
|
||||
}
|
||||
|
||||
return addrs
|
||||
}
|
|
@ -0,0 +1,280 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto/pbautoconf"
|
||||
)
|
||||
|
||||
const (
|
||||
// ID of the roots watch
|
||||
rootsWatchID = "roots"
|
||||
|
||||
// ID of the leaf watch
|
||||
leafWatchID = "leaf"
|
||||
|
||||
unknownTrustDomain = "unknown"
|
||||
)
|
||||
|
||||
var (
|
||||
defaultDNSSANs = []string{"localhost"}
|
||||
|
||||
defaultIPSANs = []net.IP{{127, 0, 0, 1}, net.ParseIP("::1")}
|
||||
)
|
||||
|
||||
func extractPEMs(roots *structs.IndexedCARoots) []string {
|
||||
var pems []string
|
||||
for _, root := range roots.Roots {
|
||||
pems = append(pems, root.RootCert)
|
||||
}
|
||||
return pems
|
||||
}
|
||||
|
||||
// updateTLSFromResponse will update the TLS certificate and roots in the shared
|
||||
// TLS configurator.
|
||||
func (ac *AutoConfig) updateTLSFromResponse(resp *pbautoconf.AutoConfigResponse) error {
|
||||
var pems []string
|
||||
for _, root := range resp.GetCARoots().GetRoots() {
|
||||
pems = append(pems, root.RootCert)
|
||||
}
|
||||
|
||||
err := ac.acConfig.TLSConfigurator.UpdateAutoTLS(
|
||||
resp.ExtraCACertificates,
|
||||
pems,
|
||||
resp.Certificate.GetCertPEM(),
|
||||
resp.Certificate.GetPrivateKeyPEM(),
|
||||
resp.Config.GetTLS().GetVerifyServerHostname(),
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("Failed to update the TLS configurator with new certificates: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) setInitialTLSCertificates(certs *structs.SignedResponse) error {
|
||||
if certs == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := ac.populateCertificateCache(certs); err != nil {
|
||||
return fmt.Errorf("error populating cache with certificates: %w", err)
|
||||
}
|
||||
|
||||
connectCAPems := extractPEMs(&certs.ConnectCARoots)
|
||||
|
||||
err := ac.acConfig.TLSConfigurator.UpdateAutoTLS(
|
||||
certs.ManualCARoots,
|
||||
connectCAPems,
|
||||
certs.IssuedCert.CertPEM,
|
||||
certs.IssuedCert.PrivateKeyPEM,
|
||||
certs.VerifyServerHostname,
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("error updating TLS configurator with certificates: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) populateCertificateCache(certs *structs.SignedResponse) error {
|
||||
cert, err := connect.ParseCert(certs.IssuedCert.CertPEM)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Failed to parse certificate: %w", err)
|
||||
}
|
||||
|
||||
// prepolutate roots cache
|
||||
rootRes := cache.FetchResult{Value: &certs.ConnectCARoots, Index: certs.ConnectCARoots.QueryMeta.Index}
|
||||
rootsReq := ac.caRootsRequest()
|
||||
// getting the roots doesn't require a token so in order to potentially share the cache with another
|
||||
if err := ac.acConfig.Cache.Prepopulate(cachetype.ConnectCARootName, rootRes, ac.config.Datacenter, "", rootsReq.CacheInfo().Key); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
leafReq := ac.leafCertRequest()
|
||||
|
||||
// prepolutate leaf cache
|
||||
certRes := cache.FetchResult{
|
||||
Value: &certs.IssuedCert,
|
||||
Index: certs.IssuedCert.RaftIndex.ModifyIndex,
|
||||
State: cachetype.ConnectCALeafSuccess(connect.EncodeSigningKeyID(cert.AuthorityKeyId)),
|
||||
}
|
||||
if err := ac.acConfig.Cache.Prepopulate(cachetype.ConnectCALeafName, certRes, leafReq.Datacenter, leafReq.Token, leafReq.Key()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) setupCertificateCacheWatches(ctx context.Context) (context.CancelFunc, error) {
|
||||
notificationCtx, cancel := context.WithCancel(ctx)
|
||||
|
||||
rootsReq := ac.caRootsRequest()
|
||||
err := ac.acConfig.Cache.Notify(notificationCtx, cachetype.ConnectCARootName, &rootsReq, rootsWatchID, ac.cacheUpdates)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
leafReq := ac.leafCertRequest()
|
||||
err = ac.acConfig.Cache.Notify(notificationCtx, cachetype.ConnectCALeafName, &leafReq, leafWatchID, ac.cacheUpdates)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return cancel, nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) updateCARoots(roots *structs.IndexedCARoots) error {
|
||||
switch {
|
||||
case ac.config.AutoConfig.Enabled:
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
var err error
|
||||
ac.autoConfigResponse.CARoots, err = translateCARootsToProtobuf(roots)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := ac.updateTLSFromResponse(ac.autoConfigResponse); err != nil {
|
||||
return err
|
||||
}
|
||||
return ac.persistAutoConfig(ac.autoConfigResponse)
|
||||
case ac.config.AutoEncryptTLS:
|
||||
pems := extractPEMs(roots)
|
||||
|
||||
if err := ac.acConfig.TLSConfigurator.UpdateAutoTLSCA(pems); err != nil {
|
||||
return fmt.Errorf("failed to update Connect CA certificates: %w", err)
|
||||
}
|
||||
return nil
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) updateLeafCert(cert *structs.IssuedCert) error {
|
||||
switch {
|
||||
case ac.config.AutoConfig.Enabled:
|
||||
ac.Lock()
|
||||
defer ac.Unlock()
|
||||
var err error
|
||||
ac.autoConfigResponse.Certificate, err = translateIssuedCertToProtobuf(cert)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := ac.updateTLSFromResponse(ac.autoConfigResponse); err != nil {
|
||||
return err
|
||||
}
|
||||
return ac.persistAutoConfig(ac.autoConfigResponse)
|
||||
case ac.config.AutoEncryptTLS:
|
||||
if err := ac.acConfig.TLSConfigurator.UpdateAutoTLSCert(cert.CertPEM, cert.PrivateKeyPEM); err != nil {
|
||||
return fmt.Errorf("failed to update the agent leaf cert: %w", err)
|
||||
}
|
||||
return nil
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) caRootsRequest() structs.DCSpecificRequest {
|
||||
return structs.DCSpecificRequest{Datacenter: ac.config.Datacenter}
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) leafCertRequest() cachetype.ConnectCALeafRequest {
|
||||
return cachetype.ConnectCALeafRequest{
|
||||
Datacenter: ac.config.Datacenter,
|
||||
Agent: ac.config.NodeName,
|
||||
DNSSAN: ac.getDNSSANs(),
|
||||
IPSAN: ac.getIPSANs(),
|
||||
Token: ac.acConfig.Tokens.AgentToken(),
|
||||
}
|
||||
}
|
||||
|
||||
// generateCSR will generate a CSR for an Agent certificate. This should
|
||||
// be sent along with the AutoConfig.InitialConfiguration RPC or the
|
||||
// AutoEncrypt.Sign RPC. The generated CSR does NOT have a real trust domain
|
||||
// as when generating this we do not yet have the CA roots. The server will
|
||||
// update the trust domain for us though.
|
||||
func (ac *AutoConfig) generateCSR() (csr string, key string, err error) {
|
||||
// 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
|
||||
// unknownTrustDomain the first time. Subsequent CSRs will have the
|
||||
// correct TrustDomain.
|
||||
id := &connect.SpiffeIDAgent{
|
||||
// will be replaced
|
||||
Host: unknownTrustDomain,
|
||||
Datacenter: ac.config.Datacenter,
|
||||
Agent: ac.config.NodeName,
|
||||
}
|
||||
|
||||
caConfig, err := ac.config.ConnectCAConfiguration()
|
||||
if err != nil {
|
||||
return "", "", fmt.Errorf("Cannot generate CSR: %w", err)
|
||||
}
|
||||
|
||||
conf, err := caConfig.GetCommonConfig()
|
||||
if err != nil {
|
||||
return "", "", fmt.Errorf("Failed to load common CA configuration: %w", err)
|
||||
}
|
||||
|
||||
if conf.PrivateKeyType == "" {
|
||||
conf.PrivateKeyType = connect.DefaultPrivateKeyType
|
||||
}
|
||||
if conf.PrivateKeyBits == 0 {
|
||||
conf.PrivateKeyBits = connect.DefaultPrivateKeyBits
|
||||
}
|
||||
|
||||
// Create a new private key
|
||||
pk, pkPEM, err := connect.GeneratePrivateKeyWithConfig(conf.PrivateKeyType, conf.PrivateKeyBits)
|
||||
if err != nil {
|
||||
return "", "", fmt.Errorf("Failed to generate private key: %w", err)
|
||||
}
|
||||
|
||||
dnsNames := ac.getDNSSANs()
|
||||
ipAddresses := ac.getIPSANs()
|
||||
|
||||
// Create a CSR.
|
||||
//
|
||||
// The Common Name includes the dummy trust domain for now but Server will
|
||||
// override this when it is signed anyway so it's OK.
|
||||
cn := connect.AgentCN(ac.config.NodeName, unknownTrustDomain)
|
||||
csr, err = connect.CreateCSR(id, cn, pk, dnsNames, ipAddresses)
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
|
||||
return csr, pkPEM, nil
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) getDNSSANs() []string {
|
||||
sans := defaultDNSSANs
|
||||
switch {
|
||||
case ac.config.AutoConfig.Enabled:
|
||||
sans = append(sans, ac.config.AutoConfig.DNSSANs...)
|
||||
case ac.config.AutoEncryptTLS:
|
||||
sans = append(sans, ac.config.AutoEncryptDNSSAN...)
|
||||
}
|
||||
return sans
|
||||
}
|
||||
|
||||
func (ac *AutoConfig) getIPSANs() []net.IP {
|
||||
sans := defaultIPSANs
|
||||
switch {
|
||||
case ac.config.AutoConfig.Enabled:
|
||||
sans = append(sans, ac.config.AutoConfig.IPSANs...)
|
||||
case ac.config.AutoEncryptTLS:
|
||||
sans = append(sans, ac.config.AutoEncryptIPSAN...)
|
||||
}
|
||||
return sans
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
package autoconf
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func newLeaf(t *testing.T, agentName, datacenter string, ca *structs.CARoot, idx uint64, expiration time.Duration) *structs.IssuedCert {
|
||||
t.Helper()
|
||||
|
||||
pub, priv, err := connect.TestAgentLeaf(t, agentName, datacenter, ca, expiration)
|
||||
require.NoError(t, err)
|
||||
cert, err := connect.ParseCert(pub)
|
||||
require.NoError(t, err)
|
||||
|
||||
spiffeID, err := connect.ParseCertURI(cert.URIs[0])
|
||||
require.NoError(t, err)
|
||||
|
||||
agentID, ok := spiffeID.(*connect.SpiffeIDAgent)
|
||||
require.True(t, ok, "certificate doesn't have an agent leaf cert URI")
|
||||
|
||||
return &structs.IssuedCert{
|
||||
SerialNumber: cert.SerialNumber.String(),
|
||||
CertPEM: pub,
|
||||
PrivateKeyPEM: priv,
|
||||
ValidAfter: cert.NotBefore,
|
||||
ValidBefore: cert.NotAfter,
|
||||
Agent: agentID.Agent,
|
||||
AgentURI: agentID.URI().String(),
|
||||
EnterpriseMeta: *structs.DefaultEnterpriseMeta(),
|
||||
RaftIndex: structs.RaftIndex{
|
||||
CreateIndex: idx,
|
||||
ModifyIndex: idx,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func testCerts(t *testing.T, agentName, datacenter string) (*structs.CARoot, *structs.IndexedCARoots, *structs.IssuedCert) {
|
||||
ca := connect.TestCA(t, nil)
|
||||
ca.IntermediateCerts = make([]string, 0)
|
||||
cert := newLeaf(t, agentName, datacenter, ca, 1, 10*time.Minute)
|
||||
indexedRoots := structs.IndexedCARoots{
|
||||
ActiveRootID: ca.ID,
|
||||
TrustDomain: connect.TestClusterID,
|
||||
Roots: []*structs.CARoot{
|
||||
ca,
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{Index: 1},
|
||||
}
|
||||
|
||||
return ca, &indexedRoots, cert
|
||||
}
|
|
@ -1,505 +0,0 @@
|
|||
package certmon
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/tlsutil"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
||||
const (
|
||||
// ID of the roots watch
|
||||
rootsWatchID = "roots"
|
||||
|
||||
// ID of the leaf watch
|
||||
leafWatchID = "leaf"
|
||||
)
|
||||
|
||||
// Cache is an interface to represent the methods of the
|
||||
// agent/cache.Cache struct that we care about
|
||||
type Cache interface {
|
||||
Notify(ctx context.Context, t string, r cache.Request, correlationID string, ch chan<- cache.UpdateEvent) error
|
||||
Prepopulate(t string, result cache.FetchResult, dc string, token string, key string) error
|
||||
}
|
||||
|
||||
// CertMonitor will setup the proper watches to ensure that
|
||||
// the Agent's Connect TLS certificate remains up to date
|
||||
type CertMonitor struct {
|
||||
logger hclog.Logger
|
||||
cache Cache
|
||||
tlsConfigurator *tlsutil.Configurator
|
||||
tokens *token.Store
|
||||
leafReq cachetype.ConnectCALeafRequest
|
||||
rootsReq structs.DCSpecificRequest
|
||||
persist PersistFunc
|
||||
fallback FallbackFunc
|
||||
fallbackLeeway time.Duration
|
||||
fallbackRetry time.Duration
|
||||
|
||||
l sync.Mutex
|
||||
running bool
|
||||
// cancel is used to cancel the entire CertMonitor
|
||||
// go routine. This is the main field protected
|
||||
// by the mutex as it being non-nil indicates that
|
||||
// the go routine has been started and is stoppable.
|
||||
// note that it doesn't indcate that the go routine
|
||||
// is currently running.
|
||||
cancel context.CancelFunc
|
||||
|
||||
// cancelWatches is used to cancel the existing
|
||||
// cache watches. This is mainly only necessary
|
||||
// when the Agent token changes
|
||||
cancelWatches context.CancelFunc
|
||||
|
||||
// cacheUpdates is the chan used to have the cache
|
||||
// send us back events
|
||||
cacheUpdates chan cache.UpdateEvent
|
||||
// tokenUpdates is the struct used to receive
|
||||
// events from the token store when the Agent
|
||||
// token is updated.
|
||||
tokenUpdates token.Notifier
|
||||
|
||||
// this is used to keep a local copy of the certs
|
||||
// keys and ca certs. It will be used to persist
|
||||
// all of the local state at once.
|
||||
certs structs.SignedResponse
|
||||
}
|
||||
|
||||
// New creates a new CertMonitor for automatically rotating
|
||||
// an Agent's Connect Certificate
|
||||
func New(config *Config) (*CertMonitor, error) {
|
||||
logger := config.Logger
|
||||
if logger == nil {
|
||||
logger = hclog.New(&hclog.LoggerOptions{
|
||||
Level: 0,
|
||||
Output: ioutil.Discard,
|
||||
})
|
||||
}
|
||||
|
||||
if config.FallbackLeeway == 0 {
|
||||
config.FallbackLeeway = 10 * time.Second
|
||||
}
|
||||
if config.FallbackRetry == 0 {
|
||||
config.FallbackRetry = time.Minute
|
||||
}
|
||||
|
||||
if config.Cache == nil {
|
||||
return nil, fmt.Errorf("CertMonitor creation requires a Cache")
|
||||
}
|
||||
|
||||
if config.TLSConfigurator == nil {
|
||||
return nil, fmt.Errorf("CertMonitor creation requires a TLS Configurator")
|
||||
}
|
||||
|
||||
if config.Fallback == nil {
|
||||
return nil, fmt.Errorf("CertMonitor creation requires specifying a FallbackFunc")
|
||||
}
|
||||
|
||||
if config.Datacenter == "" {
|
||||
return nil, fmt.Errorf("CertMonitor creation requires specifying the datacenter")
|
||||
}
|
||||
|
||||
if config.NodeName == "" {
|
||||
return nil, fmt.Errorf("CertMonitor creation requires specifying the agent's node name")
|
||||
}
|
||||
|
||||
if config.Tokens == nil {
|
||||
return nil, fmt.Errorf("CertMonitor creation requires specifying a token store")
|
||||
}
|
||||
|
||||
return &CertMonitor{
|
||||
logger: logger,
|
||||
cache: config.Cache,
|
||||
tokens: config.Tokens,
|
||||
tlsConfigurator: config.TLSConfigurator,
|
||||
persist: config.Persist,
|
||||
fallback: config.Fallback,
|
||||
fallbackLeeway: config.FallbackLeeway,
|
||||
fallbackRetry: config.FallbackRetry,
|
||||
rootsReq: structs.DCSpecificRequest{Datacenter: config.Datacenter},
|
||||
leafReq: cachetype.ConnectCALeafRequest{
|
||||
Datacenter: config.Datacenter,
|
||||
Agent: config.NodeName,
|
||||
DNSSAN: config.DNSSANs,
|
||||
IPSAN: config.IPSANs,
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Update is responsible for priming the cache with the certificates
|
||||
// as well as injecting them into the TLS configurator
|
||||
func (m *CertMonitor) Update(certs *structs.SignedResponse) error {
|
||||
if certs == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
m.certs = *certs
|
||||
|
||||
if err := m.populateCache(certs); err != nil {
|
||||
return fmt.Errorf("error populating cache with certificates: %w", err)
|
||||
}
|
||||
|
||||
connectCAPems := []string{}
|
||||
for _, ca := range certs.ConnectCARoots.Roots {
|
||||
connectCAPems = append(connectCAPems, ca.RootCert)
|
||||
}
|
||||
|
||||
// Note that its expected that the private key be within the IssuedCert in the
|
||||
// SignedResponse. This isn't how a server would send back the response and requires
|
||||
// that the recipient of the response who also has access to the private key will
|
||||
// have filled it in. The Cache definitely does this but auto-encrypt/auto-config
|
||||
// will need to ensure the original response is setup this way too.
|
||||
err := m.tlsConfigurator.UpdateAutoTLS(
|
||||
certs.ManualCARoots,
|
||||
connectCAPems,
|
||||
certs.IssuedCert.CertPEM,
|
||||
certs.IssuedCert.PrivateKeyPEM,
|
||||
certs.VerifyServerHostname)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("error updating TLS configurator with certificates: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// populateCache is responsible for inserting the certificates into the cache
|
||||
func (m *CertMonitor) populateCache(resp *structs.SignedResponse) error {
|
||||
cert, err := connect.ParseCert(resp.IssuedCert.CertPEM)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Failed to parse certificate: %w", err)
|
||||
}
|
||||
|
||||
// prepolutate roots cache
|
||||
rootRes := cache.FetchResult{Value: &resp.ConnectCARoots, Index: resp.ConnectCARoots.QueryMeta.Index}
|
||||
// getting the roots doesn't require a token so in order to potentially share the cache with another
|
||||
if err := m.cache.Prepopulate(cachetype.ConnectCARootName, rootRes, m.rootsReq.Datacenter, "", m.rootsReq.CacheInfo().Key); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// copy the template and update the token
|
||||
leafReq := m.leafReq
|
||||
leafReq.Token = m.tokens.AgentToken()
|
||||
|
||||
// prepolutate leaf cache
|
||||
certRes := cache.FetchResult{
|
||||
Value: &resp.IssuedCert,
|
||||
Index: resp.ConnectCARoots.QueryMeta.Index,
|
||||
State: cachetype.ConnectCALeafSuccess(connect.EncodeSigningKeyID(cert.AuthorityKeyId)),
|
||||
}
|
||||
if err := m.cache.Prepopulate(cachetype.ConnectCALeafName, certRes, leafReq.Datacenter, leafReq.Token, leafReq.Key()); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Start spawns the go routine to monitor the certificate and ensure it is
|
||||
// rotated/renewed as necessary. The chan will indicate once the started
|
||||
// go routine has exited
|
||||
func (m *CertMonitor) Start(ctx context.Context) (<-chan struct{}, error) {
|
||||
m.l.Lock()
|
||||
defer m.l.Unlock()
|
||||
|
||||
if m.running || m.cancel != nil {
|
||||
return nil, fmt.Errorf("the CertMonitor is already running")
|
||||
}
|
||||
|
||||
// create the top level context to control the go
|
||||
// routine executing the `run` method
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
|
||||
// create the channel to get cache update events through
|
||||
// really we should only ever get 10 updates
|
||||
m.cacheUpdates = make(chan cache.UpdateEvent, 10)
|
||||
|
||||
// setup the cache watches
|
||||
cancelWatches, err := m.setupCacheWatches(ctx)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, fmt.Errorf("error setting up cache watches: %w", err)
|
||||
}
|
||||
|
||||
// start the token update notifier
|
||||
m.tokenUpdates = m.tokens.Notify(token.TokenKindAgent)
|
||||
|
||||
// store the cancel funcs
|
||||
m.cancel = cancel
|
||||
m.cancelWatches = cancelWatches
|
||||
|
||||
m.running = true
|
||||
exit := make(chan struct{})
|
||||
go m.run(ctx, exit)
|
||||
|
||||
m.logger.Info("certificate monitor started")
|
||||
return exit, nil
|
||||
}
|
||||
|
||||
// Stop manually stops the go routine spawned by Start and
|
||||
// returns whether the go routine was still running before
|
||||
// cancelling.
|
||||
//
|
||||
// Note that cancelling the context passed into Start will
|
||||
// also cause the go routine to stop
|
||||
func (m *CertMonitor) Stop() bool {
|
||||
m.l.Lock()
|
||||
defer m.l.Unlock()
|
||||
|
||||
if !m.running {
|
||||
return false
|
||||
}
|
||||
|
||||
if m.cancel != nil {
|
||||
m.cancel()
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// IsRunning returns whether the go routine to perform certificate monitoring
|
||||
// is already running.
|
||||
func (m *CertMonitor) IsRunning() bool {
|
||||
m.l.Lock()
|
||||
defer m.l.Unlock()
|
||||
return m.running
|
||||
}
|
||||
|
||||
// setupCacheWatches will start both the roots and leaf cert watch with a new child
|
||||
// context and an up to date ACL token. The watches are started with a new child context
|
||||
// whose CancelFunc is also returned.
|
||||
func (m *CertMonitor) setupCacheWatches(ctx context.Context) (context.CancelFunc, error) {
|
||||
notificationCtx, cancel := context.WithCancel(ctx)
|
||||
|
||||
// copy the request
|
||||
rootsReq := m.rootsReq
|
||||
|
||||
err := m.cache.Notify(notificationCtx, cachetype.ConnectCARootName, &rootsReq, rootsWatchID, m.cacheUpdates)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// copy the request
|
||||
leafReq := m.leafReq
|
||||
leafReq.Token = m.tokens.AgentToken()
|
||||
|
||||
err = m.cache.Notify(notificationCtx, cachetype.ConnectCALeafName, &leafReq, leafWatchID, m.cacheUpdates)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return cancel, nil
|
||||
}
|
||||
|
||||
// handleCacheEvent is used to handle event notifications from the cache for the roots
|
||||
// or leaf cert watches.
|
||||
func (m *CertMonitor) handleCacheEvent(u cache.UpdateEvent) error {
|
||||
switch u.CorrelationID {
|
||||
case rootsWatchID:
|
||||
m.logger.Debug("roots watch fired - updating CA certificates")
|
||||
if u.Err != nil {
|
||||
return fmt.Errorf("root watch returned an error: %w", u.Err)
|
||||
}
|
||||
|
||||
roots, ok := u.Result.(*structs.IndexedCARoots)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for roots watch response: %T", u.Result)
|
||||
}
|
||||
|
||||
m.certs.ConnectCARoots = *roots
|
||||
|
||||
var pems []string
|
||||
for _, root := range roots.Roots {
|
||||
pems = append(pems, root.RootCert)
|
||||
}
|
||||
|
||||
if err := m.tlsConfigurator.UpdateAutoTLSCA(pems); err != nil {
|
||||
return fmt.Errorf("failed to update Connect CA certificates: %w", err)
|
||||
}
|
||||
|
||||
if m.persist != nil {
|
||||
copy := m.certs
|
||||
if err := m.persist(©); err != nil {
|
||||
return fmt.Errorf("failed to persist certificate package: %w", err)
|
||||
}
|
||||
}
|
||||
case leafWatchID:
|
||||
m.logger.Debug("leaf certificate watch fired - updating TLS certificate")
|
||||
if u.Err != nil {
|
||||
return fmt.Errorf("leaf watch returned an error: %w", u.Err)
|
||||
}
|
||||
|
||||
leaf, ok := u.Result.(*structs.IssuedCert)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for agent leaf cert watch response: %T", u.Result)
|
||||
}
|
||||
|
||||
m.certs.IssuedCert = *leaf
|
||||
|
||||
if err := m.tlsConfigurator.UpdateAutoTLSCert(leaf.CertPEM, leaf.PrivateKeyPEM); err != nil {
|
||||
return fmt.Errorf("failed to update the agent leaf cert: %w", err)
|
||||
}
|
||||
|
||||
if m.persist != nil {
|
||||
copy := m.certs
|
||||
if err := m.persist(©); err != nil {
|
||||
return fmt.Errorf("failed to persist certificate package: %w", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// handleTokenUpdate is used when a notification about the agent token being updated
|
||||
// is received and various watches need cancelling/restarting to use the new token.
|
||||
func (m *CertMonitor) handleTokenUpdate(ctx context.Context) error {
|
||||
m.logger.Debug("Agent token updated - resetting watches")
|
||||
|
||||
// TODO (autoencrypt) Prepopulate the cache with the new token with
|
||||
// the existing cache entry with the old token. The certificate doesn't
|
||||
// need to change just because the token has. However there isn't a
|
||||
// good way to make that happen and this behavior is benign enough
|
||||
// that I am going to push off implementing it.
|
||||
|
||||
// the agent token has been updated so we must update our leaf cert watch.
|
||||
// this cancels the current watches before setting up new ones
|
||||
m.cancelWatches()
|
||||
|
||||
// recreate the chan for cache updates. This is a precautionary measure to ensure
|
||||
// that we don't accidentally get notified for the new watches being setup before
|
||||
// a blocking query in the cache returns and sends data to the old chan. In theory
|
||||
// the code in agent/cache/watch.go should prevent this where we specifically check
|
||||
// for context cancellation prior to sending the event. However we could cancel
|
||||
// it after that check and finish setting up the new watches before getting the old
|
||||
// events. Both the go routine scheduler and the OS thread scheduler would have to
|
||||
// be acting up for this to happen. Regardless the way to ensure we don't get events
|
||||
// for the old watches is to simply replace the chan we are expecting them from.
|
||||
close(m.cacheUpdates)
|
||||
m.cacheUpdates = make(chan cache.UpdateEvent, 10)
|
||||
|
||||
// restart watches - this will be done with the correct token
|
||||
cancelWatches, err := m.setupCacheWatches(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to restart watches after agent token update: %w", err)
|
||||
}
|
||||
m.cancelWatches = cancelWatches
|
||||
return nil
|
||||
}
|
||||
|
||||
// handleFallback is used when the current TLS certificate has expired and the normal
|
||||
// updating mechanisms have failed to renew it quickly enough. This function will
|
||||
// use the configured fallback mechanism to retrieve a new cert and start monitoring
|
||||
// that one.
|
||||
func (m *CertMonitor) handleFallback(ctx context.Context) error {
|
||||
m.logger.Warn("agent's client certificate has expired")
|
||||
// Background because the context is mainly useful when the agent is first starting up.
|
||||
reply, err := m.fallback(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error when getting new agent certificate: %w", err)
|
||||
}
|
||||
|
||||
if m.persist != nil {
|
||||
if err := m.persist(reply); err != nil {
|
||||
return fmt.Errorf("failed to persist certificate package: %w", err)
|
||||
}
|
||||
}
|
||||
return m.Update(reply)
|
||||
}
|
||||
|
||||
// run is the private method to be spawn by the Start method for
|
||||
// executing the main monitoring loop.
|
||||
func (m *CertMonitor) run(ctx context.Context, exit chan struct{}) {
|
||||
// The fallbackTimer is used to notify AFTER the agents
|
||||
// leaf certificate has expired and where we need
|
||||
// to fall back to the less secure RPC endpoint just like
|
||||
// if the agent was starting up new.
|
||||
//
|
||||
// Check 10sec (fallback leeway duration) after cert
|
||||
// expires. The agent cache should be handling the expiration
|
||||
// and renew it before then.
|
||||
//
|
||||
// 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.
|
||||
calcFallbackInterval := func() time.Duration {
|
||||
certExpiry := m.tlsConfigurator.AutoEncryptCertNotAfter()
|
||||
return certExpiry.Add(m.fallbackLeeway).Sub(time.Now())
|
||||
}
|
||||
fallbackTimer := time.NewTimer(calcFallbackInterval())
|
||||
|
||||
// cleanup for once we are stopped
|
||||
defer func() {
|
||||
// cancel the go routines performing the cache watches
|
||||
m.cancelWatches()
|
||||
// ensure we don't leak the timers go routine
|
||||
fallbackTimer.Stop()
|
||||
// stop receiving notifications for token updates
|
||||
m.tokens.StopNotify(m.tokenUpdates)
|
||||
|
||||
m.logger.Debug("certificate monitor has been stopped")
|
||||
|
||||
m.l.Lock()
|
||||
m.cancel = nil
|
||||
m.running = false
|
||||
m.l.Unlock()
|
||||
|
||||
// this should be the final cleanup task as its what notifies
|
||||
// the rest of the world that this go routine has exited.
|
||||
close(exit)
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
m.logger.Debug("stopping the certificate monitor")
|
||||
return
|
||||
case <-m.tokenUpdates.Ch:
|
||||
m.logger.Debug("handling a token update event")
|
||||
|
||||
if err := m.handleTokenUpdate(ctx); err != nil {
|
||||
m.logger.Error("error in handling token update event", "error", err)
|
||||
}
|
||||
case u := <-m.cacheUpdates:
|
||||
m.logger.Debug("handling a cache update event", "correlation_id", u.CorrelationID)
|
||||
|
||||
if err := m.handleCacheEvent(u); err != nil {
|
||||
m.logger.Error("error in handling cache update event", "error", err)
|
||||
}
|
||||
|
||||
// reset the fallback timer as the certificate may have been updated
|
||||
fallbackTimer.Stop()
|
||||
fallbackTimer = time.NewTimer(calcFallbackInterval())
|
||||
case <-fallbackTimer.C:
|
||||
// This is a 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.
|
||||
|
||||
// check auto encrypt client cert expiration
|
||||
if m.tlsConfigurator.AutoEncryptCertExpired() {
|
||||
if err := m.handleFallback(ctx); err != nil {
|
||||
m.logger.Error("error when handling a certificate expiry event", "error", err)
|
||||
fallbackTimer = time.NewTimer(m.fallbackRetry)
|
||||
} else {
|
||||
fallbackTimer = time.NewTimer(calcFallbackInterval())
|
||||
}
|
||||
} else {
|
||||
// this shouldn't be possible. We calculate the timer duration to be the certificate
|
||||
// expiration time + some leeway (10s default). So whenever we get here the certificate
|
||||
// should be expired. Regardless its probably worth resetting the timer.
|
||||
fallbackTimer = time.NewTimer(calcFallbackInterval())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,731 +0,0 @@
|
|||
package certmon
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"net"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
"github.com/hashicorp/consul/sdk/testutil/retry"
|
||||
"github.com/hashicorp/consul/tlsutil"
|
||||
"github.com/hashicorp/go-uuid"
|
||||
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
type mockFallback struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
func (m *mockFallback) fallback(ctx context.Context) (*structs.SignedResponse, error) {
|
||||
ret := m.Called()
|
||||
resp, _ := ret.Get(0).(*structs.SignedResponse)
|
||||
return resp, ret.Error(1)
|
||||
}
|
||||
|
||||
type mockPersist struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
func (m *mockPersist) persist(resp *structs.SignedResponse) error {
|
||||
return m.Called(resp).Error(0)
|
||||
}
|
||||
|
||||
type mockWatcher struct {
|
||||
ch chan<- cache.UpdateEvent
|
||||
done <-chan struct{}
|
||||
}
|
||||
|
||||
type mockCache struct {
|
||||
mock.Mock
|
||||
|
||||
lock sync.Mutex
|
||||
watchers map[string][]mockWatcher
|
||||
}
|
||||
|
||||
func (m *mockCache) Notify(ctx context.Context, t string, r cache.Request, correlationID string, ch chan<- cache.UpdateEvent) error {
|
||||
m.lock.Lock()
|
||||
key := r.CacheInfo().Key
|
||||
m.watchers[key] = append(m.watchers[key], mockWatcher{ch: ch, done: ctx.Done()})
|
||||
m.lock.Unlock()
|
||||
ret := m.Called(t, r, correlationID)
|
||||
return ret.Error(0)
|
||||
}
|
||||
|
||||
func (m *mockCache) Prepopulate(t string, result cache.FetchResult, dc string, token string, key string) error {
|
||||
ret := m.Called(t, result, dc, token, key)
|
||||
return ret.Error(0)
|
||||
}
|
||||
|
||||
func (m *mockCache) sendNotification(ctx context.Context, key string, u cache.UpdateEvent) bool {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
|
||||
watchers, ok := m.watchers[key]
|
||||
if !ok || len(m.watchers) < 1 {
|
||||
return false
|
||||
}
|
||||
|
||||
var newWatchers []mockWatcher
|
||||
|
||||
for _, watcher := range watchers {
|
||||
select {
|
||||
case watcher.ch <- u:
|
||||
newWatchers = append(newWatchers, watcher)
|
||||
case <-watcher.done:
|
||||
// do nothing, this watcher will be removed from the list
|
||||
case <-ctx.Done():
|
||||
// return doesn't matter here really, the test is being cancelled
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
// this removes any already cancelled watches from being sent to
|
||||
m.watchers[key] = newWatchers
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func newMockCache(t *testing.T) *mockCache {
|
||||
mcache := mockCache{watchers: make(map[string][]mockWatcher)}
|
||||
mcache.Test(t)
|
||||
return &mcache
|
||||
}
|
||||
|
||||
func waitForChan(timer *time.Timer, ch <-chan struct{}) bool {
|
||||
select {
|
||||
case <-timer.C:
|
||||
return false
|
||||
case <-ch:
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
func waitForChans(timeout time.Duration, chans ...<-chan struct{}) bool {
|
||||
timer := time.NewTimer(timeout)
|
||||
defer timer.Stop()
|
||||
|
||||
for _, ch := range chans {
|
||||
if !waitForChan(timer, ch) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func testTLSConfigurator(t *testing.T) *tlsutil.Configurator {
|
||||
t.Helper()
|
||||
logger := testutil.Logger(t)
|
||||
cfg, err := tlsutil.NewConfigurator(tlsutil.Config{AutoTLS: true}, logger)
|
||||
require.NoError(t, err)
|
||||
return cfg
|
||||
}
|
||||
|
||||
func newLeaf(t *testing.T, ca *structs.CARoot, idx uint64, expiration time.Duration) *structs.IssuedCert {
|
||||
t.Helper()
|
||||
|
||||
pub, priv, err := connect.TestAgentLeaf(t, "node", "foo", ca, expiration)
|
||||
require.NoError(t, err)
|
||||
cert, err := connect.ParseCert(pub)
|
||||
require.NoError(t, err)
|
||||
|
||||
spiffeID, err := connect.ParseCertURI(cert.URIs[0])
|
||||
require.NoError(t, err)
|
||||
|
||||
agentID, ok := spiffeID.(*connect.SpiffeIDAgent)
|
||||
require.True(t, ok, "certificate doesn't have an agent leaf cert URI")
|
||||
|
||||
return &structs.IssuedCert{
|
||||
SerialNumber: cert.SerialNumber.String(),
|
||||
CertPEM: pub,
|
||||
PrivateKeyPEM: priv,
|
||||
ValidAfter: cert.NotBefore,
|
||||
ValidBefore: cert.NotAfter,
|
||||
Agent: agentID.Agent,
|
||||
AgentURI: agentID.URI().String(),
|
||||
EnterpriseMeta: *structs.DefaultEnterpriseMeta(),
|
||||
RaftIndex: structs.RaftIndex{
|
||||
CreateIndex: idx,
|
||||
ModifyIndex: idx,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
type testCertMonitor struct {
|
||||
monitor *CertMonitor
|
||||
mcache *mockCache
|
||||
tls *tlsutil.Configurator
|
||||
tokens *token.Store
|
||||
fallback *mockFallback
|
||||
persist *mockPersist
|
||||
|
||||
extraCACerts []string
|
||||
initialCert *structs.IssuedCert
|
||||
initialRoots *structs.IndexedCARoots
|
||||
|
||||
// these are some variables that the CertMonitor was created with
|
||||
datacenter string
|
||||
nodeName string
|
||||
dns []string
|
||||
ips []net.IP
|
||||
verifyServerHostname bool
|
||||
}
|
||||
|
||||
func newTestCertMonitor(t *testing.T) testCertMonitor {
|
||||
t.Helper()
|
||||
|
||||
tlsConfigurator := testTLSConfigurator(t)
|
||||
tokens := new(token.Store)
|
||||
|
||||
id, err := uuid.GenerateUUID()
|
||||
require.NoError(t, err)
|
||||
tokens.UpdateAgentToken(id, token.TokenSourceConfig)
|
||||
|
||||
ca := connect.TestCA(t, nil)
|
||||
manualCA := connect.TestCA(t, nil)
|
||||
// this cert is setup to not expire quickly. this will prevent
|
||||
// the test from accidentally running the fallback routine
|
||||
// before we want to force that to happen.
|
||||
issued := newLeaf(t, ca, 1, 10*time.Minute)
|
||||
|
||||
indexedRoots := structs.IndexedCARoots{
|
||||
ActiveRootID: ca.ID,
|
||||
TrustDomain: connect.TestClusterID,
|
||||
Roots: []*structs.CARoot{
|
||||
ca,
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{
|
||||
Index: 1,
|
||||
},
|
||||
}
|
||||
|
||||
initialCerts := &structs.SignedResponse{
|
||||
ConnectCARoots: indexedRoots,
|
||||
IssuedCert: *issued,
|
||||
ManualCARoots: []string{manualCA.RootCert},
|
||||
VerifyServerHostname: true,
|
||||
}
|
||||
|
||||
dnsSANs := []string{"test.dev"}
|
||||
ipSANs := []net.IP{net.IPv4(198, 18, 0, 1)}
|
||||
|
||||
fallback := &mockFallback{}
|
||||
fallback.Test(t)
|
||||
persist := &mockPersist{}
|
||||
persist.Test(t)
|
||||
|
||||
mcache := newMockCache(t)
|
||||
rootRes := cache.FetchResult{Value: &indexedRoots, Index: 1}
|
||||
rootsReq := structs.DCSpecificRequest{Datacenter: "foo"}
|
||||
mcache.On("Prepopulate", cachetype.ConnectCARootName, rootRes, "foo", "", rootsReq.CacheInfo().Key).Return(nil).Once()
|
||||
|
||||
leafReq := cachetype.ConnectCALeafRequest{
|
||||
Token: tokens.AgentToken(),
|
||||
Agent: "node",
|
||||
Datacenter: "foo",
|
||||
DNSSAN: dnsSANs,
|
||||
IPSAN: ipSANs,
|
||||
}
|
||||
leafRes := cache.FetchResult{
|
||||
Value: issued,
|
||||
Index: 1,
|
||||
State: cachetype.ConnectCALeafSuccess(ca.SigningKeyID),
|
||||
}
|
||||
mcache.On("Prepopulate", cachetype.ConnectCALeafName, leafRes, "foo", tokens.AgentToken(), leafReq.Key()).Return(nil).Once()
|
||||
|
||||
// we can assert more later but this should always be done.
|
||||
defer mcache.AssertExpectations(t)
|
||||
|
||||
cfg := new(Config).
|
||||
WithCache(mcache).
|
||||
WithLogger(testutil.Logger(t)).
|
||||
WithTLSConfigurator(tlsConfigurator).
|
||||
WithTokens(tokens).
|
||||
WithFallback(fallback.fallback).
|
||||
WithDNSSANs(dnsSANs).
|
||||
WithIPSANs(ipSANs).
|
||||
WithDatacenter("foo").
|
||||
WithNodeName("node").
|
||||
WithFallbackLeeway(time.Nanosecond).
|
||||
WithFallbackRetry(time.Millisecond).
|
||||
WithPersistence(persist.persist)
|
||||
|
||||
monitor, err := New(cfg)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, monitor)
|
||||
|
||||
require.NoError(t, monitor.Update(initialCerts))
|
||||
|
||||
return testCertMonitor{
|
||||
monitor: monitor,
|
||||
tls: tlsConfigurator,
|
||||
tokens: tokens,
|
||||
mcache: mcache,
|
||||
persist: persist,
|
||||
fallback: fallback,
|
||||
extraCACerts: []string{manualCA.RootCert},
|
||||
initialCert: issued,
|
||||
initialRoots: &indexedRoots,
|
||||
datacenter: "foo",
|
||||
nodeName: "node",
|
||||
dns: dnsSANs,
|
||||
ips: ipSANs,
|
||||
verifyServerHostname: true,
|
||||
}
|
||||
}
|
||||
|
||||
func tlsCertificateFromIssued(t *testing.T, issued *structs.IssuedCert) *tls.Certificate {
|
||||
t.Helper()
|
||||
|
||||
cert, err := tls.X509KeyPair([]byte(issued.CertPEM), []byte(issued.PrivateKeyPEM))
|
||||
require.NoError(t, err)
|
||||
return &cert
|
||||
}
|
||||
|
||||
// convenience method to get a TLS Certificate from the intial issued certificate and priv key
|
||||
func (cm *testCertMonitor) initialTLSCertificate(t *testing.T) *tls.Certificate {
|
||||
t.Helper()
|
||||
return tlsCertificateFromIssued(t, cm.initialCert)
|
||||
}
|
||||
|
||||
// just a convenience method to get a list of all the CA pems that we set up regardless
|
||||
// of manual vs connect.
|
||||
func (cm *testCertMonitor) initialCACerts() []string {
|
||||
pems := cm.extraCACerts
|
||||
for _, root := range cm.initialRoots.Roots {
|
||||
pems = append(pems, root.RootCert)
|
||||
}
|
||||
return pems
|
||||
}
|
||||
|
||||
func (cm *testCertMonitor) assertExpectations(t *testing.T) {
|
||||
cm.mcache.AssertExpectations(t)
|
||||
cm.fallback.AssertExpectations(t)
|
||||
cm.persist.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestCertMonitor_InitialCerts(t *testing.T) {
|
||||
// this also ensures that the cache was prepopulated properly
|
||||
cm := newTestCertMonitor(t)
|
||||
|
||||
// verify that the certificate was injected into the TLS configurator correctly
|
||||
require.Equal(t, cm.initialTLSCertificate(t), cm.tls.Cert())
|
||||
// verify that the CA certs (both Connect and manual ones) were injected correctly
|
||||
require.ElementsMatch(t, cm.initialCACerts(), cm.tls.CAPems())
|
||||
// verify that the auto-tls verify server hostname setting was injected correctly
|
||||
require.Equal(t, cm.verifyServerHostname, cm.tls.VerifyServerHostname())
|
||||
}
|
||||
|
||||
func TestCertMonitor_GoRoutineManagement(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
cm := newTestCertMonitor(t)
|
||||
|
||||
// ensure that the monitor is not running
|
||||
require.False(t, cm.monitor.IsRunning())
|
||||
|
||||
// ensure that nothing bad happens and that it reports as stopped
|
||||
require.False(t, cm.monitor.Stop())
|
||||
|
||||
// we will never send notifications so these just ignore everything
|
||||
cm.mcache.On("Notify", cachetype.ConnectCARootName, &structs.DCSpecificRequest{Datacenter: cm.datacenter}, rootsWatchID).Return(nil).Times(2)
|
||||
cm.mcache.On("Notify", cachetype.ConnectCALeafName,
|
||||
&cachetype.ConnectCALeafRequest{
|
||||
Token: cm.tokens.AgentToken(),
|
||||
Datacenter: cm.datacenter,
|
||||
Agent: cm.nodeName,
|
||||
DNSSAN: cm.dns,
|
||||
IPSAN: cm.ips,
|
||||
},
|
||||
leafWatchID,
|
||||
).Return(nil).Times(2)
|
||||
|
||||
done, err := cm.monitor.Start(ctx)
|
||||
require.NoError(t, err)
|
||||
require.True(t, cm.monitor.IsRunning())
|
||||
_, err = cm.monitor.Start(ctx)
|
||||
testutil.RequireErrorContains(t, err, "the CertMonitor is already running")
|
||||
require.True(t, cm.monitor.Stop())
|
||||
|
||||
require.True(t, waitForChans(100*time.Millisecond, done), "monitor didn't shut down")
|
||||
require.False(t, cm.monitor.IsRunning())
|
||||
done, err = cm.monitor.Start(ctx)
|
||||
require.NoError(t, err)
|
||||
|
||||
// ensure that context cancellation causes us to stop as well
|
||||
cancel()
|
||||
require.True(t, waitForChans(100*time.Millisecond, done))
|
||||
|
||||
cm.assertExpectations(t)
|
||||
}
|
||||
|
||||
func startedCertMonitor(t *testing.T) (context.Context, testCertMonitor) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
t.Cleanup(cancel)
|
||||
|
||||
cm := newTestCertMonitor(t)
|
||||
|
||||
rootsCtx, rootsCancel := context.WithCancel(ctx)
|
||||
defer rootsCancel()
|
||||
leafCtx, leafCancel := context.WithCancel(ctx)
|
||||
defer leafCancel()
|
||||
|
||||
// initial roots watch
|
||||
cm.mcache.On("Notify", cachetype.ConnectCARootName,
|
||||
&structs.DCSpecificRequest{
|
||||
Datacenter: cm.datacenter,
|
||||
},
|
||||
rootsWatchID).
|
||||
Return(nil).
|
||||
Once().
|
||||
Run(func(_ mock.Arguments) {
|
||||
rootsCancel()
|
||||
})
|
||||
// the initial watch after starting the monitor
|
||||
cm.mcache.On("Notify", cachetype.ConnectCALeafName,
|
||||
&cachetype.ConnectCALeafRequest{
|
||||
Token: cm.tokens.AgentToken(),
|
||||
Datacenter: cm.datacenter,
|
||||
Agent: cm.nodeName,
|
||||
DNSSAN: cm.dns,
|
||||
IPSAN: cm.ips,
|
||||
},
|
||||
leafWatchID).
|
||||
Return(nil).
|
||||
Once().
|
||||
Run(func(_ mock.Arguments) {
|
||||
leafCancel()
|
||||
})
|
||||
|
||||
done, err := cm.monitor.Start(ctx)
|
||||
require.NoError(t, err)
|
||||
// this prevents logs after the test finishes
|
||||
t.Cleanup(func() {
|
||||
cm.monitor.Stop()
|
||||
<-done
|
||||
})
|
||||
|
||||
require.True(t,
|
||||
waitForChans(100*time.Millisecond, rootsCtx.Done(), leafCtx.Done()),
|
||||
"not all watches were started within the alotted time")
|
||||
|
||||
return ctx, cm
|
||||
}
|
||||
|
||||
// This test ensures that the cache watches are restarted with the updated
|
||||
// token after receiving a token update
|
||||
func TestCertMonitor_TokenUpdate(t *testing.T) {
|
||||
ctx, cm := startedCertMonitor(t)
|
||||
|
||||
rootsCtx, rootsCancel := context.WithCancel(ctx)
|
||||
defer rootsCancel()
|
||||
leafCtx, leafCancel := context.WithCancel(ctx)
|
||||
defer leafCancel()
|
||||
|
||||
newToken := "8e4fe8db-162d-42d8-81ca-710fb2280ad0"
|
||||
|
||||
// we expect a new roots watch because when the leaf cert watch is restarted so is the root cert watch
|
||||
cm.mcache.On("Notify", cachetype.ConnectCARootName,
|
||||
&structs.DCSpecificRequest{
|
||||
Datacenter: cm.datacenter,
|
||||
},
|
||||
rootsWatchID).
|
||||
Return(nil).
|
||||
Once().
|
||||
Run(func(_ mock.Arguments) {
|
||||
rootsCancel()
|
||||
})
|
||||
|
||||
secondWatch := &cachetype.ConnectCALeafRequest{
|
||||
Token: newToken,
|
||||
Datacenter: cm.datacenter,
|
||||
Agent: cm.nodeName,
|
||||
DNSSAN: cm.dns,
|
||||
IPSAN: cm.ips,
|
||||
}
|
||||
// the new watch after updating the token
|
||||
cm.mcache.On("Notify", cachetype.ConnectCALeafName, secondWatch, leafWatchID).
|
||||
Return(nil).
|
||||
Once().
|
||||
Run(func(args mock.Arguments) {
|
||||
leafCancel()
|
||||
})
|
||||
|
||||
cm.tokens.UpdateAgentToken(newToken, token.TokenSourceAPI)
|
||||
|
||||
require.True(t,
|
||||
waitForChans(100*time.Millisecond, rootsCtx.Done(), leafCtx.Done()),
|
||||
"not all watches were restarted within the alotted time")
|
||||
|
||||
cm.assertExpectations(t)
|
||||
}
|
||||
|
||||
func TestCertMonitor_RootsUpdate(t *testing.T) {
|
||||
ctx, cm := startedCertMonitor(t)
|
||||
|
||||
secondCA := connect.TestCA(t, cm.initialRoots.Roots[0])
|
||||
secondRoots := structs.IndexedCARoots{
|
||||
ActiveRootID: secondCA.ID,
|
||||
TrustDomain: connect.TestClusterID,
|
||||
Roots: []*structs.CARoot{
|
||||
secondCA,
|
||||
cm.initialRoots.Roots[0],
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{
|
||||
Index: 99,
|
||||
},
|
||||
}
|
||||
|
||||
cm.persist.On("persist", &structs.SignedResponse{
|
||||
IssuedCert: *cm.initialCert,
|
||||
ManualCARoots: cm.extraCACerts,
|
||||
ConnectCARoots: secondRoots,
|
||||
VerifyServerHostname: cm.verifyServerHostname,
|
||||
}).Return(nil).Once()
|
||||
|
||||
// assert value of the CA certs prior to updating
|
||||
require.ElementsMatch(t, cm.initialCACerts(), cm.tls.CAPems())
|
||||
|
||||
req := structs.DCSpecificRequest{Datacenter: cm.datacenter}
|
||||
require.True(t, cm.mcache.sendNotification(ctx, req.CacheInfo().Key, cache.UpdateEvent{
|
||||
CorrelationID: rootsWatchID,
|
||||
Result: &secondRoots,
|
||||
Meta: cache.ResultMeta{
|
||||
Index: secondRoots.Index,
|
||||
},
|
||||
}))
|
||||
|
||||
expectedCAs := append(cm.extraCACerts, secondCA.RootCert, cm.initialRoots.Roots[0].RootCert)
|
||||
|
||||
// this will wait up to 200ms (8 x 25 ms waits between the 9 requests)
|
||||
retry.RunWith(&retry.Counter{Count: 9, Wait: 25 * time.Millisecond}, t, func(r *retry.R) {
|
||||
require.ElementsMatch(r, expectedCAs, cm.tls.CAPems())
|
||||
})
|
||||
|
||||
cm.assertExpectations(t)
|
||||
}
|
||||
|
||||
func TestCertMonitor_CertUpdate(t *testing.T) {
|
||||
ctx, cm := startedCertMonitor(t)
|
||||
|
||||
secondCert := newLeaf(t, cm.initialRoots.Roots[0], 100, 10*time.Minute)
|
||||
|
||||
cm.persist.On("persist", &structs.SignedResponse{
|
||||
IssuedCert: *secondCert,
|
||||
ManualCARoots: cm.extraCACerts,
|
||||
ConnectCARoots: *cm.initialRoots,
|
||||
VerifyServerHostname: cm.verifyServerHostname,
|
||||
}).Return(nil).Once()
|
||||
|
||||
// assert value of cert prior to updating the leaf
|
||||
require.Equal(t, cm.initialTLSCertificate(t), cm.tls.Cert())
|
||||
|
||||
key := cm.monitor.leafReq.CacheInfo().Key
|
||||
|
||||
// send the new certificate - this notifies only the watchers utilizing
|
||||
// the new ACL token
|
||||
require.True(t, cm.mcache.sendNotification(ctx, key, cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: secondCert,
|
||||
Meta: cache.ResultMeta{
|
||||
Index: secondCert.ModifyIndex,
|
||||
},
|
||||
}))
|
||||
|
||||
tlsCert := tlsCertificateFromIssued(t, secondCert)
|
||||
|
||||
// this will wait up to 200ms (8 x 25 ms waits between the 9 requests)
|
||||
retry.RunWith(&retry.Counter{Count: 9, Wait: 25 * time.Millisecond}, t, func(r *retry.R) {
|
||||
require.Equal(r, tlsCert, cm.tls.Cert())
|
||||
})
|
||||
|
||||
cm.assertExpectations(t)
|
||||
}
|
||||
|
||||
func TestCertMonitor_Fallback(t *testing.T) {
|
||||
ctx, cm := startedCertMonitor(t)
|
||||
|
||||
// at this point everything is operating normally and the monitor is just
|
||||
// waiting for events. We are going to send a new cert that is basically
|
||||
// already expired and then allow the fallback routine to kick in.
|
||||
secondCert := newLeaf(t, cm.initialRoots.Roots[0], 100, time.Nanosecond)
|
||||
secondCA := connect.TestCA(t, cm.initialRoots.Roots[0])
|
||||
secondRoots := structs.IndexedCARoots{
|
||||
ActiveRootID: secondCA.ID,
|
||||
TrustDomain: connect.TestClusterID,
|
||||
Roots: []*structs.CARoot{
|
||||
secondCA,
|
||||
cm.initialRoots.Roots[0],
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{
|
||||
Index: 101,
|
||||
},
|
||||
}
|
||||
thirdCert := newLeaf(t, secondCA, 102, 10*time.Minute)
|
||||
|
||||
// inject a fallback routine error to check that we rerun it quickly
|
||||
cm.fallback.On("fallback").Return(nil, fmt.Errorf("induced error")).Once()
|
||||
|
||||
fallbackResp := &structs.SignedResponse{
|
||||
ConnectCARoots: secondRoots,
|
||||
IssuedCert: *thirdCert,
|
||||
ManualCARoots: cm.extraCACerts,
|
||||
VerifyServerHostname: true,
|
||||
}
|
||||
// expect the fallback routine to be executed and setup the return
|
||||
cm.fallback.On("fallback").Return(fallbackResp, nil).Once()
|
||||
|
||||
cm.persist.On("persist", &structs.SignedResponse{
|
||||
IssuedCert: *secondCert,
|
||||
ConnectCARoots: *cm.initialRoots,
|
||||
ManualCARoots: cm.extraCACerts,
|
||||
VerifyServerHostname: cm.verifyServerHostname,
|
||||
}).Return(nil).Once()
|
||||
|
||||
cm.persist.On("persist", fallbackResp).Return(nil).Once()
|
||||
|
||||
// Add another roots cache prepopulation expectation which should happen
|
||||
// in response to executing the fallback mechanism
|
||||
rootRes := cache.FetchResult{Value: &secondRoots, Index: 101}
|
||||
rootsReq := structs.DCSpecificRequest{Datacenter: cm.datacenter}
|
||||
cm.mcache.On("Prepopulate", cachetype.ConnectCARootName, rootRes, cm.datacenter, "", rootsReq.CacheInfo().Key).Return(nil).Once()
|
||||
|
||||
// add another leaf cert cache prepopulation expectation which should happen
|
||||
// in response to executing the fallback mechanism
|
||||
leafReq := cachetype.ConnectCALeafRequest{
|
||||
Token: cm.tokens.AgentToken(),
|
||||
Agent: cm.nodeName,
|
||||
Datacenter: cm.datacenter,
|
||||
DNSSAN: cm.dns,
|
||||
IPSAN: cm.ips,
|
||||
}
|
||||
leafRes := cache.FetchResult{
|
||||
Value: thirdCert,
|
||||
Index: 101,
|
||||
State: cachetype.ConnectCALeafSuccess(secondCA.SigningKeyID),
|
||||
}
|
||||
cm.mcache.On("Prepopulate", cachetype.ConnectCALeafName, leafRes, leafReq.Datacenter, leafReq.Token, leafReq.Key()).Return(nil).Once()
|
||||
|
||||
// nothing in the monitor should be looking at this as its only done
|
||||
// in response to sending token updates, no need to synchronize
|
||||
key := cm.monitor.leafReq.CacheInfo().Key
|
||||
// send the new certificate - this notifies only the watchers utilizing
|
||||
// the new ACL token
|
||||
require.True(t, cm.mcache.sendNotification(ctx, key, cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: secondCert,
|
||||
Meta: cache.ResultMeta{
|
||||
Index: secondCert.ModifyIndex,
|
||||
},
|
||||
}))
|
||||
|
||||
// if all went well we would have updated the first certificate which was pretty much expired
|
||||
// causing the fallback handler to be invoked almost immediately. The fallback routine will
|
||||
// return the response containing the third cert and second CA roots so now we should wait
|
||||
// a little while and ensure they were applied to the TLS Configurator
|
||||
tlsCert := tlsCertificateFromIssued(t, thirdCert)
|
||||
expectedCAs := append(cm.extraCACerts, secondCA.RootCert, cm.initialRoots.Roots[0].RootCert)
|
||||
|
||||
// this will wait up to 200ms (8 x 25 ms waits between the 9 requests)
|
||||
retry.RunWith(&retry.Counter{Count: 9, Wait: 25 * time.Millisecond}, t, func(r *retry.R) {
|
||||
require.Equal(r, tlsCert, cm.tls.Cert())
|
||||
require.ElementsMatch(r, expectedCAs, cm.tls.CAPems())
|
||||
})
|
||||
|
||||
cm.assertExpectations(t)
|
||||
}
|
||||
|
||||
func TestCertMonitor_New_Errors(t *testing.T) {
|
||||
type testCase struct {
|
||||
cfg Config
|
||||
err string
|
||||
}
|
||||
|
||||
fallback := func(_ context.Context) (*structs.SignedResponse, error) {
|
||||
return nil, fmt.Errorf("Unimplemented")
|
||||
}
|
||||
|
||||
tokens := new(token.Store)
|
||||
|
||||
cases := map[string]testCase{
|
||||
"no-cache": {
|
||||
cfg: Config{
|
||||
TLSConfigurator: testTLSConfigurator(t),
|
||||
Fallback: fallback,
|
||||
Tokens: tokens,
|
||||
Datacenter: "foo",
|
||||
NodeName: "bar",
|
||||
},
|
||||
err: "CertMonitor creation requires a Cache",
|
||||
},
|
||||
"no-tls-configurator": {
|
||||
cfg: Config{
|
||||
Cache: cache.New(cache.Options{}),
|
||||
Fallback: fallback,
|
||||
Tokens: tokens,
|
||||
Datacenter: "foo",
|
||||
NodeName: "bar",
|
||||
},
|
||||
err: "CertMonitor creation requires a TLS Configurator",
|
||||
},
|
||||
"no-fallback": {
|
||||
cfg: Config{
|
||||
Cache: cache.New(cache.Options{}),
|
||||
TLSConfigurator: testTLSConfigurator(t),
|
||||
Tokens: tokens,
|
||||
Datacenter: "foo",
|
||||
NodeName: "bar",
|
||||
},
|
||||
err: "CertMonitor creation requires specifying a FallbackFunc",
|
||||
},
|
||||
"no-tokens": {
|
||||
cfg: Config{
|
||||
Cache: cache.New(cache.Options{}),
|
||||
TLSConfigurator: testTLSConfigurator(t),
|
||||
Fallback: fallback,
|
||||
Datacenter: "foo",
|
||||
NodeName: "bar",
|
||||
},
|
||||
err: "CertMonitor creation requires specifying a token store",
|
||||
},
|
||||
"no-datacenter": {
|
||||
cfg: Config{
|
||||
Cache: cache.New(cache.Options{}),
|
||||
TLSConfigurator: testTLSConfigurator(t),
|
||||
Fallback: fallback,
|
||||
Tokens: tokens,
|
||||
NodeName: "bar",
|
||||
},
|
||||
err: "CertMonitor creation requires specifying the datacenter",
|
||||
},
|
||||
"no-node-name": {
|
||||
cfg: Config{
|
||||
Cache: cache.New(cache.Options{}),
|
||||
TLSConfigurator: testTLSConfigurator(t),
|
||||
Fallback: fallback,
|
||||
Tokens: tokens,
|
||||
Datacenter: "foo",
|
||||
},
|
||||
err: "CertMonitor creation requires specifying the agent's node name",
|
||||
},
|
||||
}
|
||||
|
||||
for name, tcase := range cases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
monitor, err := New(&tcase.cfg)
|
||||
testutil.RequireErrorContains(t, err, tcase.err)
|
||||
require.Nil(t, monitor)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -1,150 +0,0 @@
|
|||
package certmon
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/tlsutil"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
||||
// FallbackFunc is used when the normal cache watch based Certificate
|
||||
// updating fails to update the Certificate in time and a different
|
||||
// method of updating the certificate is required.
|
||||
type FallbackFunc func(context.Context) (*structs.SignedResponse, error)
|
||||
|
||||
// PersistFunc is used to persist the data from a signed response
|
||||
type PersistFunc func(*structs.SignedResponse) error
|
||||
|
||||
type Config struct {
|
||||
// Logger is the logger to be used while running. If not set
|
||||
// then no logging will be performed.
|
||||
Logger hclog.Logger
|
||||
|
||||
// TLSConfigurator is where the certificates and roots are set when
|
||||
// they are updated. This field is required.
|
||||
TLSConfigurator *tlsutil.Configurator
|
||||
|
||||
// Cache is an object implementing our Cache interface. The Cache
|
||||
// used at runtime must be able to handle Roots and Leaf Cert watches
|
||||
Cache Cache
|
||||
|
||||
// Tokens is the shared token store. It is used to retrieve the current
|
||||
// agent token as well as getting notifications when that token is updated.
|
||||
// This field is required.
|
||||
Tokens *token.Store
|
||||
|
||||
// Persist is a function to run when there are new certs or keys
|
||||
Persist PersistFunc
|
||||
|
||||
// Fallback is a function to run when the normal cache updating of the
|
||||
// agent's certificates has failed to work for one reason or another.
|
||||
// This field is required.
|
||||
Fallback FallbackFunc
|
||||
|
||||
// FallbackLeeway is the amount of time after certificate expiration before
|
||||
// invoking the fallback routine. If not set this will default to 10s.
|
||||
FallbackLeeway time.Duration
|
||||
|
||||
// FallbackRetry is the duration between Fallback invocations when the configured
|
||||
// fallback routine returns an error. If not set this will default to 1m.
|
||||
FallbackRetry time.Duration
|
||||
|
||||
// DNSSANs is a list of DNS SANs that certificate requests should include. This
|
||||
// field is optional and no extra DNS SANs will be requested if unset. 'localhost'
|
||||
// is unconditionally requested by the cache implementation.
|
||||
DNSSANs []string
|
||||
|
||||
// IPSANs is a list of IP SANs to include in the certificate signing request. This
|
||||
// field is optional and no extra IP SANs will be requested if unset. Both '127.0.0.1'
|
||||
// and '::1' IP SANs are unconditionally requested by the cache implementation.
|
||||
IPSANs []net.IP
|
||||
|
||||
// Datacenter is the datacenter to request certificates within. This filed is required
|
||||
Datacenter string
|
||||
|
||||
// NodeName is the agent's node name to use when requesting certificates. This field
|
||||
// is required.
|
||||
NodeName string
|
||||
}
|
||||
|
||||
// WithCache will cause the created CertMonitor type to use the provided Cache
|
||||
func (cfg *Config) WithCache(cache Cache) *Config {
|
||||
cfg.Cache = cache
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithLogger will cause the created CertMonitor type to use the provided logger
|
||||
func (cfg *Config) WithLogger(logger hclog.Logger) *Config {
|
||||
cfg.Logger = logger
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithTLSConfigurator will cause the created CertMonitor type to use the provided configurator
|
||||
func (cfg *Config) WithTLSConfigurator(tlsConfigurator *tlsutil.Configurator) *Config {
|
||||
cfg.TLSConfigurator = tlsConfigurator
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithTokens will cause the created CertMonitor type to use the provided token store
|
||||
func (cfg *Config) WithTokens(tokens *token.Store) *Config {
|
||||
cfg.Tokens = tokens
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithFallback configures a fallback function to use if the normal update mechanisms
|
||||
// fail to renew the certificate in time.
|
||||
func (cfg *Config) WithFallback(fallback FallbackFunc) *Config {
|
||||
cfg.Fallback = fallback
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithDNSSANs configures the CertMonitor to request these DNS SANs when requesting a new
|
||||
// certificate
|
||||
func (cfg *Config) WithDNSSANs(sans []string) *Config {
|
||||
cfg.DNSSANs = sans
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithIPSANs configures the CertMonitor to request these IP SANs when requesting a new
|
||||
// certificate
|
||||
func (cfg *Config) WithIPSANs(sans []net.IP) *Config {
|
||||
cfg.IPSANs = sans
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithDatacenter configures the CertMonitor to request Certificates in this DC
|
||||
func (cfg *Config) WithDatacenter(dc string) *Config {
|
||||
cfg.Datacenter = dc
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithNodeName configures the CertMonitor to request Certificates with this agent name
|
||||
func (cfg *Config) WithNodeName(name string) *Config {
|
||||
cfg.NodeName = name
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithFallbackLeeway configures how long after a certificate expires before attempting to
|
||||
// generarte a new certificate using the fallback mechanism. The default is 10s.
|
||||
func (cfg *Config) WithFallbackLeeway(leeway time.Duration) *Config {
|
||||
cfg.FallbackLeeway = leeway
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithFallbackRetry controls how quickly we will make subsequent invocations of
|
||||
// the fallback func in the case of it erroring out.
|
||||
func (cfg *Config) WithFallbackRetry(after time.Duration) *Config {
|
||||
cfg.FallbackRetry = after
|
||||
return cfg
|
||||
}
|
||||
|
||||
// WithPersistence will configure the CertMonitor to use this callback for persisting
|
||||
// a new TLS configuration.
|
||||
func (cfg *Config) WithPersistence(persist PersistFunc) *Config {
|
||||
cfg.Persist = persist
|
||||
return cfg
|
||||
}
|
|
@ -1,239 +0,0 @@
|
|||
package consul
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/miekg/dns"
|
||||
)
|
||||
|
||||
const (
|
||||
dummyTrustDomain = "dummy.trustdomain"
|
||||
retryJitterWindow = 30 * time.Second
|
||||
)
|
||||
|
||||
func (c *Client) autoEncryptCSR(extraDNSSANs []string, extraIPSANs []net.IP) (string, string, error) {
|
||||
// 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: c.config.NodeName,
|
||||
}
|
||||
|
||||
conf, err := c.config.CAConfig.GetCommonConfig()
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
|
||||
if conf.PrivateKeyType == "" {
|
||||
conf.PrivateKeyType = connect.DefaultPrivateKeyType
|
||||
}
|
||||
if conf.PrivateKeyBits == 0 {
|
||||
conf.PrivateKeyBits = connect.DefaultPrivateKeyBits
|
||||
}
|
||||
|
||||
// Create a new private key
|
||||
pk, pkPEM, err := connect.GeneratePrivateKeyWithConfig(conf.PrivateKeyType, conf.PrivateKeyBits)
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
|
||||
dnsNames := append([]string{"localhost"}, extraDNSSANs...)
|
||||
ipAddresses := append([]net.IP{net.ParseIP("127.0.0.1"), net.ParseIP("::1")}, extraIPSANs...)
|
||||
|
||||
// Create a CSR.
|
||||
//
|
||||
// The Common Name includes the dummy trust domain for now but Server will
|
||||
// override this when it is signed anyway so it's OK.
|
||||
cn := connect.AgentCN(c.config.NodeName, dummyTrustDomain)
|
||||
csr, err := connect.CreateCSR(id, cn, pk, dnsNames, ipAddresses)
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
|
||||
return pkPEM, csr, nil
|
||||
}
|
||||
|
||||
func (c *Client) RequestAutoEncryptCerts(ctx context.Context, servers []string, port int, token string, extraDNSSANs []string, extraIPSANs []net.IP) (*structs.SignedResponse, error) {
|
||||
errFn := func(err error) (*structs.SignedResponse, 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.router.FindLANServer()
|
||||
if server != nil {
|
||||
servers = []string{server.Addr.String()}
|
||||
}
|
||||
|
||||
if len(servers) == 0 {
|
||||
return errFn(fmt.Errorf("No servers to request AutoEncrypt.Sign"))
|
||||
}
|
||||
|
||||
pkPEM, csr, err := c.autoEncryptCSR(extraDNSSANs, extraIPSANs)
|
||||
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 <-ctx.Done():
|
||||
return errFn(fmt.Errorf("aborting AutoEncrypt because interrupted: %w", ctx.Err()))
|
||||
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.Warn("AutoEncrypt resolveAddr failed", "error", err)
|
||||
continue
|
||||
}
|
||||
|
||||
for _, ip := range ips {
|
||||
addr := net.TCPAddr{IP: ip, Port: port}
|
||||
|
||||
if err = c.connPool.RPC(c.config.Datacenter, c.config.NodeName, &addr, "AutoEncrypt.Sign", &args, &reply); err == nil {
|
||||
reply.IssuedCert.PrivateKeyPEM = pkPEM
|
||||
return &reply, nil
|
||||
} else {
|
||||
c.logger.Warn("AutoEncrypt failed", "error", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
attempts++
|
||||
|
||||
delay := lib.RandomStagger(retryJitterWindow)
|
||||
interval := (time.Duration(attempts) * delay) + delay
|
||||
c.logger.Warn("retrying AutoEncrypt", "retry_interval", interval)
|
||||
select {
|
||||
case <-time.After(interval):
|
||||
continue
|
||||
case <-ctx.Done():
|
||||
return errFn(fmt.Errorf("aborting AutoEncrypt because interrupted: %w", ctx.Err()))
|
||||
case <-c.shutdownCh:
|
||||
return errFn(fmt.Errorf("aborting AutoEncrypt because shutting down"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func missingPortError(host string, err error) bool {
|
||||
return err != nil && err.Error() == fmt.Sprintf("address %s: missing port in address", host)
|
||||
}
|
||||
|
||||
// resolveAddr is used to resolve the host into IPs and error.
|
||||
func resolveAddr(rawHost string, logger hclog.Logger) ([]net.IP, error) {
|
||||
host, _, err := net.SplitHostPort(rawHost)
|
||||
if err != nil {
|
||||
// In case we encounter this error, we proceed with the
|
||||
// rawHost. This is fine since -start-join and -retry-join
|
||||
// take only hosts anyways and this is an expected case.
|
||||
if missingPortError(rawHost, err) {
|
||||
host = rawHost
|
||||
} else {
|
||||
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.Debug("TCP-first lookup failed for host, falling back to UDP", "host", host, "error", 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.
|
||||
ips, err := net.LookupIP(host)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ips, nil
|
||||
}
|
||||
|
||||
// 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 hclog.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.Debug("Ignoring CNAME RR in TCP-first answer for host", "host", host)
|
||||
}
|
||||
}
|
||||
return ips, nil
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -1,205 +0,0 @@
|
|||
package consul
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/x509"
|
||||
"crypto/x509/pkix"
|
||||
"encoding/asn1"
|
||||
"net"
|
||||
"net/url"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestAutoEncrypt_resolveAddr(t *testing.T) {
|
||||
type args struct {
|
||||
rawHost string
|
||||
logger hclog.Logger
|
||||
}
|
||||
logger := testutil.Logger(t)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
ips []net.IP
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "host without port",
|
||||
args: args{
|
||||
"127.0.0.1",
|
||||
logger,
|
||||
},
|
||||
ips: []net.IP{net.IPv4(127, 0, 0, 1)},
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "host with port",
|
||||
args: args{
|
||||
"127.0.0.1:1234",
|
||||
logger,
|
||||
},
|
||||
ips: []net.IP{net.IPv4(127, 0, 0, 1)},
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "host with broken port",
|
||||
args: args{
|
||||
"127.0.0.1:xyz",
|
||||
logger,
|
||||
},
|
||||
ips: []net.IP{net.IPv4(127, 0, 0, 1)},
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "not an address",
|
||||
args: args{
|
||||
"abc",
|
||||
logger,
|
||||
},
|
||||
ips: nil,
|
||||
wantErr: true,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
ips, err := resolveAddr(tt.args.rawHost, tt.args.logger)
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("resolveAddr error: %v, wantErr: %v", err, tt.wantErr)
|
||||
return
|
||||
}
|
||||
require.Equal(t, tt.ips, ips)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_missingPortError(t *testing.T) {
|
||||
host := "127.0.0.1"
|
||||
_, _, err := net.SplitHostPort(host)
|
||||
require.True(t, missingPortError(host, err))
|
||||
|
||||
host = "127.0.0.1:1234"
|
||||
_, _, err = net.SplitHostPort(host)
|
||||
require.False(t, missingPortError(host, err))
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_RequestAutoEncryptCerts(t *testing.T) {
|
||||
dir1, c1 := testClient(t)
|
||||
defer os.RemoveAll(dir1)
|
||||
defer c1.Shutdown()
|
||||
servers := []string{"localhost"}
|
||||
port := 8301
|
||||
token := ""
|
||||
|
||||
ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(75*time.Millisecond))
|
||||
defer cancel()
|
||||
|
||||
doneCh := make(chan struct{})
|
||||
var err error
|
||||
go func() {
|
||||
_, err = c1.RequestAutoEncryptCerts(ctx, servers, port, token, nil, nil)
|
||||
close(doneCh)
|
||||
}()
|
||||
select {
|
||||
case <-doneCh:
|
||||
// since there are no servers at this port, we shouldn't be
|
||||
// done and this should be an error of some sorts that happened
|
||||
// in the setup phase before entering the for loop in
|
||||
// RequestAutoEncryptCerts.
|
||||
require.NoError(t, err)
|
||||
case <-ctx.Done():
|
||||
// this is the happy case since auto encrypt is in its loop to
|
||||
// try to request certs.
|
||||
}
|
||||
}
|
||||
|
||||
func TestAutoEncrypt_autoEncryptCSR(t *testing.T) {
|
||||
type testCase struct {
|
||||
conf *Config
|
||||
extraDNSSANs []string
|
||||
extraIPSANs []net.IP
|
||||
err string
|
||||
|
||||
// to validate the csr
|
||||
expectedSubject pkix.Name
|
||||
expectedSigAlg x509.SignatureAlgorithm
|
||||
expectedPubAlg x509.PublicKeyAlgorithm
|
||||
expectedDNSNames []string
|
||||
expectedIPs []net.IP
|
||||
expectedURIs []*url.URL
|
||||
}
|
||||
|
||||
cases := map[string]testCase{
|
||||
"sans": {
|
||||
conf: &Config{
|
||||
Datacenter: "dc1",
|
||||
NodeName: "test-node",
|
||||
CAConfig: &structs.CAConfiguration{},
|
||||
},
|
||||
extraDNSSANs: []string{"foo.local", "bar.local"},
|
||||
extraIPSANs: []net.IP{net.IPv4(198, 18, 0, 1), net.IPv4(198, 18, 0, 2)},
|
||||
expectedSubject: pkix.Name{
|
||||
CommonName: connect.AgentCN("test-node", dummyTrustDomain),
|
||||
Names: []pkix.AttributeTypeAndValue{
|
||||
{
|
||||
// 2,5,4,3 is the CommonName type ASN1 identifier
|
||||
Type: asn1.ObjectIdentifier{2, 5, 4, 3},
|
||||
Value: "testnode.agnt.dummy.tr.consul",
|
||||
},
|
||||
},
|
||||
},
|
||||
expectedSigAlg: x509.ECDSAWithSHA256,
|
||||
expectedPubAlg: x509.ECDSA,
|
||||
expectedDNSNames: []string{
|
||||
"localhost",
|
||||
"foo.local",
|
||||
"bar.local",
|
||||
},
|
||||
expectedIPs: []net.IP{
|
||||
{127, 0, 0, 1},
|
||||
net.ParseIP("::1"),
|
||||
{198, 18, 0, 1},
|
||||
{198, 18, 0, 2},
|
||||
},
|
||||
expectedURIs: []*url.URL{
|
||||
{
|
||||
Scheme: "spiffe",
|
||||
Host: dummyTrustDomain,
|
||||
Path: "/agent/client/dc/dc1/id/test-node",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for name, tcase := range cases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
client := Client{config: tcase.conf}
|
||||
|
||||
_, csr, err := client.autoEncryptCSR(tcase.extraDNSSANs, tcase.extraIPSANs)
|
||||
if tcase.err == "" {
|
||||
require.NoError(t, err)
|
||||
|
||||
request, err := connect.ParseCSR(csr)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, request)
|
||||
|
||||
require.Equal(t, tcase.expectedSubject, request.Subject)
|
||||
require.Equal(t, tcase.expectedSigAlg, request.SignatureAlgorithm)
|
||||
require.Equal(t, tcase.expectedPubAlg, request.PublicKeyAlgorithm)
|
||||
require.Equal(t, tcase.expectedDNSNames, request.DNSNames)
|
||||
require.Equal(t, tcase.expectedIPs, request.IPAddresses)
|
||||
require.Equal(t, tcase.expectedURIs, request.URIs)
|
||||
} else {
|
||||
require.Error(t, err)
|
||||
require.Empty(t, csr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
|
@ -1,7 +1,6 @@
|
|||
package agent
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
|
@ -10,11 +9,9 @@ import (
|
|||
|
||||
autoconf "github.com/hashicorp/consul/agent/auto-config"
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
certmon "github.com/hashicorp/consul/agent/cert-monitor"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/router"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/ipaddr"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
|
@ -86,38 +83,21 @@ func NewBaseDeps(configLoader ConfigLoader, logOut io.Writer) (BaseDeps, error)
|
|||
d.Cache = cache.New(cfg.Cache)
|
||||
d.ConnPool = newConnPool(cfg, d.Logger, d.TLSConfigurator)
|
||||
|
||||
deferredAC := &deferredAutoConfig{}
|
||||
|
||||
d.Router = router.NewRouter(d.Logger, cfg.Datacenter, fmt.Sprintf("%s.%s", cfg.NodeName, cfg.Datacenter))
|
||||
|
||||
cmConf := new(certmon.Config).
|
||||
WithCache(d.Cache).
|
||||
WithTLSConfigurator(d.TLSConfigurator).
|
||||
WithDNSSANs(cfg.AutoConfig.DNSSANs).
|
||||
WithIPSANs(cfg.AutoConfig.IPSANs).
|
||||
WithDatacenter(cfg.Datacenter).
|
||||
WithNodeName(cfg.NodeName).
|
||||
WithFallback(deferredAC.autoConfigFallbackTLS).
|
||||
WithLogger(d.Logger.Named(logging.AutoConfig)).
|
||||
WithTokens(d.Tokens).
|
||||
WithPersistence(deferredAC.autoConfigPersist)
|
||||
acCertMon, err := certmon.New(cmConf)
|
||||
if err != nil {
|
||||
return d, err
|
||||
}
|
||||
|
||||
acConf := autoconf.Config{
|
||||
DirectRPC: d.ConnPool,
|
||||
Logger: d.Logger,
|
||||
CertMonitor: acCertMon,
|
||||
Loader: configLoader,
|
||||
DirectRPC: d.ConnPool,
|
||||
Logger: d.Logger,
|
||||
Loader: configLoader,
|
||||
ServerProvider: d.Router,
|
||||
TLSConfigurator: d.TLSConfigurator,
|
||||
Cache: d.Cache,
|
||||
Tokens: d.Tokens,
|
||||
}
|
||||
d.AutoConfig, err = autoconf.New(acConf)
|
||||
if err != nil {
|
||||
return d, err
|
||||
}
|
||||
// TODO: can this cyclic dependency be un-cycled?
|
||||
deferredAC.autoConf = d.AutoConfig
|
||||
|
||||
return d, nil
|
||||
}
|
||||
|
@ -144,21 +124,3 @@ func newConnPool(config *config.RuntimeConfig, logger hclog.Logger, tls *tlsutil
|
|||
}
|
||||
return pool
|
||||
}
|
||||
|
||||
type deferredAutoConfig struct {
|
||||
autoConf *autoconf.AutoConfig // TODO: use an interface
|
||||
}
|
||||
|
||||
func (a *deferredAutoConfig) autoConfigFallbackTLS(ctx context.Context) (*structs.SignedResponse, error) {
|
||||
if a.autoConf == nil {
|
||||
return nil, fmt.Errorf("AutoConfig manager has not been created yet")
|
||||
}
|
||||
return a.autoConf.FallbackTLS(ctx)
|
||||
}
|
||||
|
||||
func (a *deferredAutoConfig) autoConfigPersist(resp *structs.SignedResponse) error {
|
||||
if a.autoConf == nil {
|
||||
return fmt.Errorf("AutoConfig manager has not been created yet")
|
||||
}
|
||||
return a.autoConf.RecordUpdatedCerts(resp)
|
||||
}
|
||||
|
|
|
@ -12,6 +12,8 @@ var (
|
|||
timePtrType = reflect.TypeOf((*time.Time)(nil))
|
||||
timeType = timePtrType.Elem()
|
||||
mapStrInf = reflect.TypeOf((map[string]interface{})(nil))
|
||||
|
||||
epoch1970 = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
)
|
||||
|
||||
// HookPBTimestampToTime is a mapstructure decode hook to translate a protobuf timestamp
|
||||
|
@ -19,7 +21,10 @@ var (
|
|||
func HookPBTimestampToTime(from, to reflect.Type, data interface{}) (interface{}, error) {
|
||||
if to == timeType && from == tsType {
|
||||
ts := data.(*types.Timestamp)
|
||||
return time.Unix(ts.Seconds, int64(ts.Nanos)), nil
|
||||
if ts.Seconds == 0 && ts.Nanos == 0 {
|
||||
return time.Time{}, nil
|
||||
}
|
||||
return time.Unix(ts.Seconds, int64(ts.Nanos)).UTC(), nil
|
||||
}
|
||||
|
||||
return data, nil
|
||||
|
@ -39,6 +44,13 @@ func HookTimeToPBTimestamp(from, to reflect.Type, data interface{}) (interface{}
|
|||
// seeing a *time.Time instead of a time.Time.
|
||||
if from == timePtrType && to == mapStrInf {
|
||||
ts := data.(*time.Time)
|
||||
|
||||
// protobuf only supports times from Jan 1 1970 onward but the time.Time type
|
||||
// can represent values back to year 1. Basically
|
||||
if ts.Before(epoch1970) {
|
||||
return map[string]interface{}{}, nil
|
||||
}
|
||||
|
||||
nanos := ts.UnixNano()
|
||||
if nanos < 0 {
|
||||
return map[string]interface{}{}, nil
|
||||
|
|
|
@ -27,7 +27,7 @@ func TestHookPBTimestampToTime(t *testing.T) {
|
|||
}
|
||||
|
||||
expected := timeTSWrapper{
|
||||
Timestamp: time.Unix(1000, 42),
|
||||
Timestamp: time.Unix(1000, 42).UTC(),
|
||||
}
|
||||
|
||||
var actual timeTSWrapper
|
||||
|
@ -43,7 +43,7 @@ func TestHookPBTimestampToTime(t *testing.T) {
|
|||
|
||||
func TestHookTimeToPBTimestamp(t *testing.T) {
|
||||
in := timeTSWrapper{
|
||||
Timestamp: time.Unix(999999, 123456),
|
||||
Timestamp: time.Unix(999999, 123456).UTC(),
|
||||
}
|
||||
|
||||
expected := pbTSWrapper{
|
||||
|
@ -63,3 +63,24 @@ func TestHookTimeToPBTimestamp(t *testing.T) {
|
|||
|
||||
require.Equal(t, expected, actual)
|
||||
}
|
||||
|
||||
func TestHookTimeToPBTimestamp_ZeroTime(t *testing.T) {
|
||||
in := timeTSWrapper{}
|
||||
|
||||
expected := pbTSWrapper{
|
||||
Timestamp: &types.Timestamp{
|
||||
Seconds: 0,
|
||||
Nanos: 0,
|
||||
},
|
||||
}
|
||||
|
||||
var actual pbTSWrapper
|
||||
decoder, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{
|
||||
DecodeHook: HookTimeToPBTimestamp,
|
||||
Result: &actual,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, decoder.Decode(in))
|
||||
|
||||
require.Equal(t, expected, actual)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue