Merge pull request #9570 from hashicorp/bugfix/9498

This commit is contained in:
Matt Keeler 2021-01-19 16:30:04 -05:00 committed by GitHub
commit 2d7a4073e3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 601 additions and 121 deletions

15
.changelog/9498.txt Normal file
View File

@ -0,0 +1,15 @@
```release-note:bug
leader: Fixed a bug that could cause Connect CA initialization failures from allowing leader establishment to complete resulting in potentially infinite leader elections.
```
```release-note:bug
connect: Fixed a bug in the Vault Connect CA provider that could cause the intermediate PKI path to be deleted after reconfiguring the CA
```
```release-note:bug
connect: Fixed a bug in the AWS PCA Connect CA provider that could cause the intermediate PKI path to be deleted after reconfiguring the CA
```
```release-note:bug
connect: Fixed an issue that would prevent updating the Connect CA configuration if the CA provider didn't complete initialization previously.
```

View File

@ -55,13 +55,13 @@ func (_m *MockProvider) ActiveRoot() (string, error) {
return r0, r1
}
// Cleanup provides a mock function with given fields:
func (_m *MockProvider) Cleanup() error {
ret := _m.Called()
// Cleanup provides a mock function with given fields: providerTypeChange, config
func (_m *MockProvider) Cleanup(providerTypeChange bool, config map[string]interface{}) error {
ret := _m.Called(providerTypeChange, config)
var r0 error
if rf, ok := ret.Get(0).(func() error); ok {
r0 = rf()
if rf, ok := ret.Get(0).(func(bool, map[string]interface{}) error); ok {
r0 = rf(providerTypeChange, config)
} else {
r0 = ret.Error(0)
}

View File

@ -161,8 +161,13 @@ type Provider interface {
// Cleanup performs any necessary cleanup that should happen when the provider
// is shut down permanently, such as removing a temporary PKI backend in Vault
// created for an intermediate CA.
Cleanup() error
// created for an intermediate CA. Whether the CA provider type is changing
// and the other providers raw configuration is passed along so that the provider
// instance can determine which cleanup steps to perform. For example, when the
// Vault provider is in use and there is no type change occuring, the Vault
// provider should check if the intermediate PKI path is changing. If it is not
// changing then the provider should not remove that path from Vault.
Cleanup(providerTypeChange bool, otherConfig map[string]interface{}) error
}
// NeedsLogger is an optional interface that allows a CA provider to use the

View File

@ -303,6 +303,7 @@ func (a *AWSProvider) createPCA() error {
a.logger.Debug("creating new PCA", "common_name", commonName)
createOutput, err := a.client.CreateCertificateAuthority(&createInput)
if err != nil {
a.logger.Error("failed to create new PCA", "common_name", commonName, "error", err)
return err
}
@ -649,12 +650,26 @@ func (a *AWSProvider) deletePCA() error {
}
// Cleanup implements Provider
func (a *AWSProvider) Cleanup() error {
func (a *AWSProvider) Cleanup(providerTypeChange bool, otherConfig map[string]interface{}) error {
old := atomic.SwapUint32(&a.stopped, 1)
if old == 0 {
close(a.stopCh)
}
if !providerTypeChange {
awsConfig, err := ParseAWSCAConfig(otherConfig)
if err != nil {
return err
}
// if the provider is being replaced and using an existing PCA instance
// then prevent deletion of that instance if the new provider uses
// the same instance.
if a.config.ExistingARN == awsConfig.ExistingARN {
return nil
}
}
if a.config.DeleteOnExit {
if err := a.disablePCA(); err != nil {
// Log the error but continue trying to delete as some errors may still

View File

@ -5,6 +5,8 @@ import (
"strconv"
"testing"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/service/acmpca"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/stretchr/testify/require"
@ -39,7 +41,7 @@ func TestAWSBootstrapAndSignPrimary(t *testing.T) {
"PrivateKeyBits": tc.KeyBits,
}
provider := testAWSProvider(t, testProviderConfigPrimary(t, cfg))
defer provider.Cleanup()
defer provider.Cleanup(true, nil)
// Generate the root
require.NoError(provider.GenerateRoot())
@ -89,12 +91,12 @@ func TestAWSBootstrapAndSignSecondary(t *testing.T) {
skipIfAWSNotConfigured(t)
p1 := testAWSProvider(t, testProviderConfigPrimary(t, nil))
defer p1.Cleanup()
defer p1.Cleanup(true, nil)
rootPEM, err := p1.ActiveRoot()
require.NoError(t, err)
p2 := testAWSProvider(t, testProviderConfigSecondary(t, nil))
defer p2.Cleanup()
defer p2.Cleanup(true, nil)
testSignIntermediateCrossDC(t, p1, p2)
@ -191,14 +193,14 @@ func TestAWSBootstrapAndSignSecondaryConsul(t *testing.T) {
require.NoError(t, p1.GenerateRoot())
p2 := testAWSProvider(t, testProviderConfigSecondary(t, nil))
defer p2.Cleanup()
defer p2.Cleanup(true, nil)
testSignIntermediateCrossDC(t, p1, p2)
})
t.Run("pri=aws,sec=consul", func(t *testing.T) {
p1 := testAWSProvider(t, testProviderConfigPrimary(t, nil))
defer p1.Cleanup()
defer p1.Cleanup(true, nil)
require.NoError(t, p1.GenerateRoot())
conf := testConsulCAConfig()
@ -217,7 +219,7 @@ func TestAWSNoCrossSigning(t *testing.T) {
skipIfAWSNotConfigured(t)
p1 := testAWSProvider(t, testProviderConfigPrimary(t, nil))
defer p1.Cleanup()
defer p1.Cleanup(true, nil)
// Don't bother initializing a PCA as that is slow and unnecessary for this
// test
@ -235,6 +237,142 @@ func TestAWSNoCrossSigning(t *testing.T) {
require.Contains(t, err.Error(), "not implemented")
}
func TestAWSProvider_Cleanup(t *testing.T) {
// Note not parallel since we could easily hit AWS limits of too many CAs if
// all of these tests run at once.
skipIfAWSNotConfigured(t)
describeCA := func(t *testing.T, provider *AWSProvider) (bool, error) {
t.Helper()
state, err := provider.State()
require.NoError(t, err)
// Load from the resource.
input := &acmpca.DescribeCertificateAuthorityInput{
CertificateAuthorityArn: aws.String(state[AWSStateCAARNKey]),
}
output, err := provider.client.DescribeCertificateAuthority(input)
if err != nil {
return false, err
}
require.NotNil(t, output)
require.NotNil(t, output.CertificateAuthority)
require.NotNil(t, output.CertificateAuthority.Status)
return *output.CertificateAuthority.Status == acmpca.CertificateAuthorityStatusDeleted, nil
}
requirePCADeleted := func(t *testing.T, provider *AWSProvider) {
deleted, err := describeCA(t, provider)
require.True(t, err != nil || deleted, "The AWS PCA instance has not been deleted")
}
requirePCANotDeleted := func(t *testing.T, provider *AWSProvider) {
deleted, err := describeCA(t, provider)
require.NoError(t, err)
require.False(t, deleted, "The AWS PCA instance should not have been deleted")
}
t.Run("provider-change", func(t *testing.T) {
// create a provider with the default config which will create the CA
p1Conf := testProviderConfigPrimary(t, nil)
p1 := testAWSProvider(t, p1Conf)
p1.GenerateRoot()
t.Cleanup(func() {
// This is a fail safe just in case the Cleanup routine of the
// second provider fails to delete the CA. In that case we want
// to request that the main provider delete it during Cleanup.
if deleted, err := describeCA(t, p1); err == nil && deleted {
p1.Cleanup(false, p1Conf.RawConfig)
} else {
p1.Cleanup(true, nil)
}
})
// just ensure that it got created
requirePCANotDeleted(t, p1)
state, err := p1.State()
require.NoError(t, err)
p2Conf := testProviderConfigPrimary(t, map[string]interface{}{
"ExistingARN": state[AWSStateCAARNKey],
})
p2 := testAWSProvider(t, p2Conf)
// provider change should trigger deletion of the CA
require.NoError(t, p2.Cleanup(true, nil))
requirePCADeleted(t, p1)
})
t.Run("arn-change", func(t *testing.T) {
// create a provider with the default config which will create the CA
p1Conf := testProviderConfigPrimary(t, nil)
p1 := testAWSProvider(t, p1Conf)
p1.GenerateRoot()
t.Cleanup(func() {
// This is a fail safe just in case the Cleanup routine of the
// second provider fails to delete the CA. In that case we want
// to request that the main provider delete it during Cleanup.
if deleted, err := describeCA(t, p1); err == nil || deleted {
p1.Cleanup(false, p1Conf.RawConfig)
} else {
p1.Cleanup(true, nil)
}
})
// just ensure that it got created
requirePCANotDeleted(t, p1)
state, err := p1.State()
require.NoError(t, err)
p2Conf := testProviderConfigPrimary(t, map[string]interface{}{
"ExistingARN": state[AWSStateCAARNKey],
})
p2 := testAWSProvider(t, p2Conf)
// changing the ARN should cause the other CA to be deleted
p2ConfAltARN := testProviderConfigPrimary(t, map[string]interface{}{
"ExistingARN": "doesnt-need-to-be-real",
})
require.NoError(t, p2.Cleanup(false, p2ConfAltARN.RawConfig))
requirePCADeleted(t, p1)
})
t.Run("arn-not-changed", func(t *testing.T) {
// create a provider with the default config which will create the CA
p1Conf := testProviderConfigPrimary(t, nil)
p1 := testAWSProvider(t, p1Conf)
p1.GenerateRoot()
t.Cleanup(func() {
// the p2 provider should not remove the CA but we need to ensure that
// we do clean it up
p1.Cleanup(true, nil)
})
// just ensure that it got created
requirePCANotDeleted(t, p1)
state, err := p1.State()
require.NoError(t, err)
p2Conf := testProviderConfigPrimary(t, map[string]interface{}{
"ExistingARN": state[AWSStateCAARNKey],
})
p2 := testAWSProvider(t, p2Conf)
// because the ARN isn't changing we don't want to remove the CA
require.NoError(t, p2.Cleanup(false, p2Conf.RawConfig))
requirePCANotDeleted(t, p1)
})
}
func testAWSProvider(t *testing.T, cfg ProviderConfig) *AWSProvider {
p := &AWSProvider{}
logger := testutil.Logger(t)

View File

@ -310,7 +310,11 @@ func (c *ConsulProvider) GenerateIntermediate() (string, error) {
}
// Remove the state store entry for this provider instance.
func (c *ConsulProvider) Cleanup() error {
func (c *ConsulProvider) Cleanup(_ bool, _ map[string]interface{}) error {
// This method only gets called for final cleanup. Therefore we don't
// need to worry about the case where a ca config update is made to
// change the cert ttls but leaving the private key and root cert the
// same. Changing those would change the id field on the provider.
args := &structs.CARequest{
Op: structs.CAOpDeleteProviderState,
ProviderState: &structs.CAConsulProviderState{ID: c.id},

View File

@ -523,10 +523,31 @@ func (c *VaultProvider) SupportsCrossSigning() (bool, error) {
// Cleanup unmounts the configured intermediate PKI backend. It's fine to tear
// this down and recreate it on small config changes because the intermediate
// certs get bundled with the leaf certs, so there's no cost to the CA changing.
func (v *VaultProvider) Cleanup() error {
func (v *VaultProvider) Cleanup(providerTypeChange bool, otherConfig map[string]interface{}) error {
v.Stop()
return v.client.Sys().Unmount(v.config.IntermediatePKIPath)
if !providerTypeChange {
newConfig, err := ParseVaultCAConfig(otherConfig)
if err != nil {
return err
}
// if the intermeidate PKI path isn't changing we don't want to delete it as
// Cleanup is called after initializing the new provider
if newConfig.IntermediatePKIPath == v.config.IntermediatePKIPath {
return nil
}
}
err := v.client.Sys().Unmount(v.config.IntermediatePKIPath)
switch err {
case ErrBackendNotMounted, ErrBackendNotInitialized:
// suppress these errors if we didn't finish initialization before
return nil
default:
return err
}
}
// Stop shuts down the token renew goroutine.

View File

@ -359,6 +359,86 @@ func TestVaultProvider_SignIntermediateConsul(t *testing.T) {
})
}
func TestVaultProvider_Cleanup(t *testing.T) {
t.Parallel()
SkipIfVaultNotPresent(t)
testVault, err := runTestVault(t)
require.NoError(t, err)
testVault.WaitUntilReady(t)
t.Run("provider-change", func(t *testing.T) {
provider, err := createVaultProvider(t, true, testVault.Addr, testVault.RootToken, nil)
require.NoError(t, err)
// ensure that the intermediate PKI mount exists
mounts, err := provider.client.Sys().ListMounts()
require.NoError(t, err)
require.Contains(t, mounts, provider.config.IntermediatePKIPath)
// call cleanup with a provider change - this should cause removal of the mount
require.NoError(t, provider.Cleanup(true, nil))
// verify the mount was removed
mounts, err = provider.client.Sys().ListMounts()
require.NoError(t, err)
require.NotContains(t, mounts, provider.config.IntermediatePKIPath)
})
t.Run("pki-path-change", func(t *testing.T) {
provider, err := createVaultProvider(t, true, testVault.Addr, testVault.RootToken, nil)
require.NoError(t, err)
// ensure that the intermediate PKI mount exists
mounts, err := provider.client.Sys().ListMounts()
require.NoError(t, err)
require.Contains(t, mounts, provider.config.IntermediatePKIPath)
// call cleanup with an intermediate pki path change - this should cause removal of the mount
require.NoError(t, provider.Cleanup(false, map[string]interface{}{
"Address": testVault.Addr,
"Token": testVault.RootToken,
"RootPKIPath": "pki-root/",
//
"IntermediatePKIPath": "pki-intermediate2/",
// Tests duration parsing after msgpack type mangling during raft apply.
"LeafCertTTL": []uint8("72h"),
}))
// verify the mount was removed
mounts, err = provider.client.Sys().ListMounts()
require.NoError(t, err)
require.NotContains(t, mounts, provider.config.IntermediatePKIPath)
})
t.Run("pki-path-unchanged", func(t *testing.T) {
provider, err := createVaultProvider(t, true, testVault.Addr, testVault.RootToken, nil)
require.NoError(t, err)
// ensure that the intermediate PKI mount exists
mounts, err := provider.client.Sys().ListMounts()
require.NoError(t, err)
require.Contains(t, mounts, provider.config.IntermediatePKIPath)
// call cleanup with no config changes - this should not cause removal of the intermediate pki path
require.NoError(t, provider.Cleanup(false, map[string]interface{}{
"Address": testVault.Addr,
"Token": testVault.RootToken,
"RootPKIPath": "pki-root/",
"IntermediatePKIPath": "pki-intermediate/",
// Tests duration parsing after msgpack type mangling during raft apply.
"LeafCertTTL": []uint8("72h"),
}))
// verify the mount was NOT removed
mounts, err = provider.client.Sys().ListMounts()
require.NoError(t, err)
require.Contains(t, mounts, provider.config.IntermediatePKIPath)
})
}
func getIntermediateCertTTL(t *testing.T, caConf *structs.CAConfiguration) time.Duration {
t.Helper()

View File

@ -343,10 +343,6 @@ func (s *Server) establishLeadership(ctx context.Context) error {
s.getOrCreateAutopilotConfig()
s.autopilot.Start(ctx)
if err := s.caManager.InitializeCA(); err != nil {
return err
}
s.startConfigReplication()
s.startFederationStateReplication()
@ -391,7 +387,7 @@ func (s *Server) revokeLeadership() {
s.stopConnectLeader()
s.caManager.setCAProvider(nil, nil)
s.caManager.setState(CAStateUninitialized, false)
s.caManager.setState(caStateUninitialized, false)
s.stopACLTokenReaping()

View File

@ -34,21 +34,16 @@ func (s *Server) startConnectLeader() error {
return nil
}
// Start the Connect secondary DC actions if enabled.
if s.config.Datacenter != s.config.PrimaryDatacenter {
s.leaderRoutineManager.Start(secondaryCARootWatchRoutineName, s.caManager.secondaryCARootWatch)
}
s.leaderRoutineManager.Start(intermediateCertRenewWatchRoutineName, s.caManager.intermediateCertRenewalWatch)
s.caManager.Start()
s.leaderRoutineManager.Start(caRootPruningRoutineName, s.runCARootPruning)
return s.startIntentionConfigEntryMigration()
}
// stopConnectLeader stops connect specific leader functions.
func (s *Server) stopConnectLeader() {
s.caManager.Stop()
s.leaderRoutineManager.Stop(intentionMigrationRoutineName)
s.leaderRoutineManager.Stop(secondaryCARootWatchRoutineName)
s.leaderRoutineManager.Stop(intermediateCertRenewWatchRoutineName)
s.leaderRoutineManager.Stop(caRootPruningRoutineName)
// If the provider implements NeedsStop, we call Stop to perform any shutdown actions.

View File

@ -17,14 +17,14 @@ import (
uuid "github.com/hashicorp/go-uuid"
)
type CAState string
type caState string
const (
CAStateUninitialized CAState = "UNINITIALIZED"
CAStateInitializing = "INITIALIZING"
CAStateReady = "READY"
CAStateRenewIntermediate = "RENEWING"
CAStateReconfig = "RECONFIGURING"
caStateUninitialized caState = "UNINITIALIZED"
caStateInitializing = "INITIALIZING"
caStateInitialized = "INITIALIZED"
caStateRenewIntermediate = "RENEWING"
caStateReconfig = "RECONFIGURING"
)
// caServerDelegate is an interface for server operations for facilitating
@ -61,9 +61,11 @@ type CAManager struct {
// stateLock protects the internal state used for administrative CA tasks.
stateLock sync.Mutex
state CAState
state caState
primaryRoots structs.IndexedCARoots // The most recently seen state of the root CAs from the primary datacenter.
actingSecondaryCA bool // True if this datacenter has been initialized as a secondary CA.
leaderRoutineManager *LeaderRoutineManager
}
type caDelegateWithState struct {
@ -74,36 +76,47 @@ func (c *caDelegateWithState) State() *state.Store {
return c.fsm.State()
}
func NewCAManager(delegate caServerDelegate, logger hclog.Logger, config *Config) *CAManager {
func NewCAManager(delegate caServerDelegate, leaderRoutineManager *LeaderRoutineManager, logger hclog.Logger, config *Config) *CAManager {
return &CAManager{
delegate: delegate,
logger: logger,
serverConf: config,
state: CAStateUninitialized,
delegate: delegate,
logger: logger,
serverConf: config,
state: caStateUninitialized,
leaderRoutineManager: leaderRoutineManager,
}
}
func (c *CAManager) reset() {
c.state = CAStateUninitialized
c.state = caStateUninitialized
c.primaryRoots = structs.IndexedCARoots{}
c.actingSecondaryCA = false
c.setCAProvider(nil, nil)
}
// setState attempts to update the CA state to the given state.
// If the current state is not READY, this will fail. The only exception is when
// the current state is UNINITIALIZED, and the function is called with CAStateInitializing.
func (c *CAManager) setState(newState CAState, validateState bool) error {
// Valid state transitions are:
//
// caStateInitialized -> <any state>
// caStateUninitialized -> caStateInitializing
// caStateUninitialized -> caStateReconfig
//
// Other state transitions may be forced if the validateState parameter is set to false.
// This will mainly be used in deferred functions which aim to set the final status based
// a successful/error return.
func (c *CAManager) setState(newState caState, validateState bool) (caState, error) {
c.stateLock.Lock()
defer c.stateLock.Unlock()
state := c.state
if !validateState || state == CAStateReady || (state == CAStateUninitialized && newState == CAStateInitializing) {
if !validateState ||
state == caStateInitialized ||
(state == caStateUninitialized && newState == caStateInitializing) ||
(state == caStateUninitialized && newState == caStateReconfig) {
c.state = newState
} else {
return fmt.Errorf("CA is already in state %q", state)
return state, fmt.Errorf("CA is already in state %q", state)
}
return nil
return state, nil
}
// setPrimaryRoots updates the most recently seen roots from the primary.
@ -111,7 +124,7 @@ func (c *CAManager) setPrimaryRoots(newRoots structs.IndexedCARoots) error {
c.stateLock.Lock()
defer c.stateLock.Unlock()
if c.state == CAStateInitializing || c.state == CAStateReconfig {
if c.state == caStateInitializing || c.state == caStateReconfig {
c.primaryRoots = newRoots
} else {
return fmt.Errorf("Cannot update primary roots in state %q", c.state)
@ -234,21 +247,87 @@ func (c *CAManager) setCAProvider(newProvider ca.Provider, root *structs.CARoot)
c.providerLock.Unlock()
}
func (c *CAManager) Start() {
// Attempt to initialize the Connect CA now. This will
// happen during leader establishment and it would be great
// if the CA was ready to go once that process was finished.
if err := c.InitializeCA(); err != nil {
c.logger.Error("Failed to initialize Connect CA", "error", err)
// we failed to fully initialize the CA so we need to spawn a
// go routine to retry this process until it succeeds or we lose
// leadership and the go routine gets stopped.
c.leaderRoutineManager.Start(backgroundCAInitializationRoutineName, c.backgroundCAInitialization)
} else {
// We only start these if CA initialization was successful. If not the completion of the
// background CA initialization will start these routines.
c.startPostInitializeRoutines()
}
}
func (c *CAManager) Stop() {
c.leaderRoutineManager.Stop(secondaryCARootWatchRoutineName)
c.leaderRoutineManager.Stop(intermediateCertRenewWatchRoutineName)
c.leaderRoutineManager.Stop(backgroundCAInitializationRoutineName)
}
func (c *CAManager) startPostInitializeRoutines() {
// Start the Connect secondary DC actions if enabled.
if c.serverConf.Datacenter != c.serverConf.PrimaryDatacenter {
c.leaderRoutineManager.Start(secondaryCARootWatchRoutineName, c.secondaryCARootWatch)
}
c.leaderRoutineManager.Start(intermediateCertRenewWatchRoutineName, c.intermediateCertRenewalWatch)
}
func (c *CAManager) backgroundCAInitialization(ctx context.Context) error {
retryLoopBackoffAbortOnSuccess(ctx, c.InitializeCA, func(err error) {
c.logger.Error("Failed to initialize Connect CA",
"routine", backgroundCAInitializationRoutineName,
"error", err,
)
})
if err := ctx.Err(); err != nil {
return err
}
c.logger.Info("Successfully initialized the Connect CA")
c.startPostInitializeRoutines()
return nil
}
// InitializeCA sets up the CA provider when gaining leadership, either bootstrapping
// the CA if this is the primary DC or making a remote RPC for intermediate signing
// if this is a secondary DC.
func (c *CAManager) InitializeCA() error {
func (c *CAManager) InitializeCA() (reterr error) {
// Bail if connect isn't enabled.
if !c.serverConf.ConnectEnabled {
return nil
}
// Update the state before doing anything else.
err := c.setState(CAStateInitializing, true)
oldState, err := c.setState(caStateInitializing, true)
// if we were already in the initialized state then there is nothing to be done.
if oldState == caStateInitialized {
return nil
}
if err != nil {
return err
}
defer c.setState(CAStateReady, false)
defer func() {
// Using named return values in deferred funcs isnt too common in our code
// but it is first class Go functionality. The error erturned from the
// main func will be available by its given name within deferred functions.
// See: https://blog.golang.org/defer-panic-and-recover
if reterr == nil {
c.setState(caStateInitialized, false)
} else {
c.setState(caStateUninitialized, false)
}
}()
// Initialize the provider based on the current config.
conf, err := c.initializeCAConfig()
@ -613,12 +692,29 @@ func (c *CAManager) persistNewRootAndConfig(provider ca.Provider, newActiveRoot
return nil
}
func (c *CAManager) UpdateConfiguration(args *structs.CARequest) error {
func (c *CAManager) UpdateConfiguration(args *structs.CARequest) (reterr error) {
// Attempt to update the state first.
if err := c.setState(CAStateReconfig, true); err != nil {
oldState, err := c.setState(caStateReconfig, true)
if err != nil {
return err
}
defer func() {
// Using named return values in deferred funcs isnt too common in our code
// but it is first class Go functionality. The error erturned from the
// main func will be available by its given name within deferred functions.
// See: https://blog.golang.org/defer-panic-and-recover
if reterr == nil {
c.setState(caStateInitialized, false)
} else {
c.setState(oldState, false)
}
}()
// Attempt to initialize the config if we failed to do so in InitializeCA for some reason
_, err = c.initializeCAConfig()
if err != nil {
return err
}
defer c.setState(CAStateReady, false)
// Exit early if it's a no-op change
state := c.delegate.State()
@ -674,7 +770,7 @@ func (c *CAManager) UpdateConfiguration(args *structs.CARequest) error {
cleanupNewProvider := true
defer func() {
if cleanupNewProvider {
if err := newProvider.Cleanup(); err != nil {
if err := newProvider.Cleanup(args.Config.Provider != config.Provider, args.Config.Config); err != nil {
c.logger.Warn("failed to clean up CA provider while handling startup failure", "provider", newProvider, "error", err)
}
}
@ -739,51 +835,60 @@ func (c *CAManager) UpdateConfiguration(args *structs.CARequest) error {
return nil
}
// At this point, we know the config change has trigged a root rotation,
// either by swapping the provider type or changing the provider's config
// to use a different root certificate.
// First up, sanity check that the current provider actually supports
// cross-signing.
// get the old CA provider to be used for Cross Signing and to clean it up at the end
// of the functi8on.
oldProvider, _ := c.getCAProvider()
if oldProvider == nil {
return fmt.Errorf("internal error: CA provider is nil")
}
canXSign, err := oldProvider.SupportsCrossSigning()
if err != nil {
return fmt.Errorf("CA provider error: %s", err)
}
if !canXSign && !args.Config.ForceWithoutCrossSigning {
return errors.New("The current CA Provider does not support cross-signing. " +
"You can try again with ForceWithoutCrossSigningSet but this may cause " +
"disruption - see documentation for more.")
}
if !canXSign && args.Config.ForceWithoutCrossSigning {
c.logger.Warn("current CA doesn't support cross signing but " +
"CA reconfiguration forced anyway with ForceWithoutCrossSigning")
}
// If it's a config change that would trigger a rotation (different provider/root):
// 1. Get the root from the new provider.
// 2. Call CrossSignCA on the old provider to sign the new root with the old one to
// get a cross-signed certificate.
// 3. Take the active root for the new provider and append the intermediate from step 2
// to its list of intermediates.
newRoot, err := connect.ParseCert(newRootPEM)
if err != nil {
return err
}
if canXSign {
// Have the old provider cross-sign the new root
xcCert, err := oldProvider.CrossSignCA(newRoot)
// We only even think about cross signing if the current provider has a root cert
// In some cases such as having a bad CA configuration during startup the provider
// may not have been able to generate a cert. We then want to be able to prevent
// an attempt to cross sign the cert which will definitely fail.
if root != nil {
// If it's a config change that would trigger a rotation (different provider/root):
// 1. Get the root from the new provider.
// 2. Call CrossSignCA on the old provider to sign the new root with the old one to
// get a cross-signed certificate.
// 3. Take the active root for the new provider and append the intermediate from step 2
// to its list of intermediates.
newRoot, err := connect.ParseCert(newRootPEM)
if err != nil {
return err
}
// Add the cross signed cert to the new CA's intermediates (to be attached
// to leaf certs).
newActiveRoot.IntermediateCerts = []string{xcCert}
// At this point, we know the config change has triggered a root rotation,
// either by swapping the provider type or changing the provider's config
// to use a different root certificate.
// First up, check that the current provider actually supports
// cross-signing.
canXSign, err := oldProvider.SupportsCrossSigning()
if err != nil {
return fmt.Errorf("CA provider error: %s", err)
}
if !canXSign && !args.Config.ForceWithoutCrossSigning {
return errors.New("The current CA Provider does not support cross-signing. " +
"You can try again with ForceWithoutCrossSigningSet but this may cause " +
"disruption - see documentation for more.")
}
if !canXSign && args.Config.ForceWithoutCrossSigning {
c.logger.Warn("current CA doesn't support cross signing but " +
"CA reconfiguration forced anyway with ForceWithoutCrossSigning")
}
if canXSign {
// Have the old provider cross-sign the new root
xcCert, err := oldProvider.CrossSignCA(newRoot)
if err != nil {
return err
}
// Add the cross signed cert to the new CA's intermediates (to be attached
// to leaf certs).
newActiveRoot.IntermediateCerts = []string{xcCert}
}
}
intermediate, err := newProvider.GenerateIntermediate()
@ -831,8 +936,8 @@ func (c *CAManager) UpdateConfiguration(args *structs.CARequest) error {
cleanupNewProvider = false
c.setCAProvider(newProvider, newActiveRoot)
if err := oldProvider.Cleanup(); err != nil {
c.logger.Warn("failed to clean up old provider", "provider", config.Provider)
if err := oldProvider.Cleanup(args.Config.Provider != config.Provider, args.Config.Config); err != nil {
c.logger.Warn("failed to clean up old provider", "provider", config.Provider, "error", err)
}
c.logger.Info("CA rotated to new root under provider", "provider", args.Config.Provider)
@ -924,10 +1029,10 @@ func (c *CAManager) intermediateCertRenewalWatch(ctx context.Context) error {
func (c *CAManager) RenewIntermediate(ctx context.Context, isPrimary bool) error {
// Grab the 'lock' right away so the provider/config can't be changed out while we check
// the intermediate.
if err := c.setState(CAStateRenewIntermediate, true); err != nil {
if _, err := c.setState(caStateRenewIntermediate, true); err != nil {
return err
}
defer c.setState(CAStateReady, false)
defer c.setState(caStateInitialized, false)
provider, _ := c.getCAProvider()
if provider == nil {
@ -1053,10 +1158,10 @@ func (c *CAManager) secondaryCARootWatch(ctx context.Context) error {
// certificate if necessary.
func (c *CAManager) UpdateRoots(roots structs.IndexedCARoots) error {
// Update the state first to claim the 'lock'.
if err := c.setState(CAStateReconfig, true); err != nil {
if _, err := c.setState(caStateReconfig, true); err != nil {
return err
}
defer c.setState(CAStateReady, false)
defer c.setState(caStateInitialized, false)
// Update the cached primary roots now that the lock is held.
if err := c.setPrimaryRoots(roots); err != nil {
@ -1125,7 +1230,7 @@ func (c *CAManager) setSecondaryCA() error {
c.stateLock.Lock()
defer c.stateLock.Unlock()
if c.state == CAStateInitializing || c.state == CAStateReconfig {
if c.state == caStateInitializing || c.state == caStateReconfig {
c.actingSecondaryCA = true
} else {
return fmt.Errorf("Cannot update secondary CA flag in state %q", c.state)

View File

@ -138,7 +138,7 @@ func (m *mockCAProvider) Sign(*x509.CertificateRequest) (string, error)
func (m *mockCAProvider) SignIntermediate(*x509.CertificateRequest) (string, error) { return "", nil }
func (m *mockCAProvider) CrossSignCA(*x509.Certificate) (string, error) { return "", nil }
func (m *mockCAProvider) SupportsCrossSigning() (bool, error) { return false, nil }
func (m *mockCAProvider) Cleanup() error { return nil }
func (m *mockCAProvider) Cleanup(_ bool, _ map[string]interface{}) error { return nil }
func waitForCh(t *testing.T, ch chan string, expected string) {
select {
@ -202,18 +202,18 @@ func TestCAManager_Initialize(t *testing.T) {
conf.PrimaryDatacenter = "dc1"
conf.Datacenter = "dc2"
delegate := NewMockCAServerDelegate(t, conf)
manager := NewCAManager(delegate, testutil.Logger(t), conf)
manager := NewCAManager(delegate, nil, testutil.Logger(t), conf)
// Call InitializeCA and then confirm the RPCs and provider calls
// happen in the expected order.
require.EqualValues(t, CAStateUninitialized, manager.state)
require.EqualValues(t, caStateUninitialized, manager.state)
errCh := make(chan error)
go func() {
errCh <- manager.InitializeCA()
}()
waitForCh(t, delegate.callbackCh, "forwardDC/ConnectCA.Roots")
require.EqualValues(t, CAStateInitializing, manager.state)
require.EqualValues(t, caStateInitializing, manager.state)
waitForCh(t, delegate.callbackCh, "provider/GenerateIntermediateCSR")
waitForCh(t, delegate.callbackCh, "forwardDC/ConnectCA.SignIntermediate")
waitForCh(t, delegate.callbackCh, "provider/SetIntermediate")
@ -228,7 +228,7 @@ func TestCAManager_Initialize(t *testing.T) {
t.Fatal("never got result from errCh")
}
require.EqualValues(t, CAStateReady, manager.state)
require.EqualValues(t, caStateInitialized, manager.state)
}
func TestCAManager_UpdateConfigWhileRenewIntermediate(t *testing.T) {
@ -252,7 +252,7 @@ func TestCAManager_UpdateConfigWhileRenewIntermediate(t *testing.T) {
conf.PrimaryDatacenter = "dc1"
conf.Datacenter = "dc2"
delegate := NewMockCAServerDelegate(t, conf)
manager := NewCAManager(delegate, testutil.Logger(t), conf)
manager := NewCAManager(delegate, nil, testutil.Logger(t), conf)
initTestManager(t, manager, delegate)
// Wait half the TTL for the cert to need renewing.
@ -270,7 +270,7 @@ func TestCAManager_UpdateConfigWhileRenewIntermediate(t *testing.T) {
// Call UpdateConfiguration while RenewIntermediate is still in-flight to
// make sure we get an error about the state being occupied.
go func() {
require.EqualValues(t, CAStateRenewIntermediate, manager.state)
require.EqualValues(t, caStateRenewIntermediate, manager.state)
require.Error(t, errors.New("already in state"), manager.UpdateConfiguration(&structs.CARequest{}))
}()
@ -287,5 +287,5 @@ func TestCAManager_UpdateConfigWhileRenewIntermediate(t *testing.T) {
t.Fatal("never got result from errCh")
}
require.EqualValues(t, CAStateReady, manager.state)
require.EqualValues(t, caStateInitialized, manager.state)
}

View File

@ -1204,3 +1204,104 @@ func TestLeader_retryLoopBackoffHandleSuccess(t *testing.T) {
})
}
}
func TestLeader_Vault_BadCAConfigShouldntPreventLeaderEstablishment(t *testing.T) {
ca.SkipIfVaultNotPresent(t)
testVault := ca.NewTestVaultServer(t)
defer testVault.Stop()
_, s1 := testServerWithConfig(t, func(c *Config) {
c.Build = "1.9.1"
c.PrimaryDatacenter = "dc1"
c.CAConfig = &structs.CAConfiguration{
Provider: "vault",
Config: map[string]interface{}{
"Address": testVault.Addr,
"Token": "not-the-root",
"RootPKIPath": "pki-root/",
"IntermediatePKIPath": "pki-intermediate/",
},
}
})
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
rootsList, activeRoot, err := getTestRoots(s1, "dc1")
require.NoError(t, err)
require.Empty(t, rootsList.Roots)
require.Nil(t, activeRoot)
// Now that the leader is up and we have verified that there are no roots / CA init failed,
// verify that we can reconfigure away from the bad configuration.
newConfig := &structs.CAConfiguration{
Provider: "vault",
Config: map[string]interface{}{
"Address": testVault.Addr,
"Token": testVault.RootToken,
"RootPKIPath": "pki-root/",
"IntermediatePKIPath": "pki-intermediate/",
},
}
{
args := &structs.CARequest{
Datacenter: "dc1",
Config: newConfig,
}
var reply interface{}
retry.Run(t, func(r *retry.R) {
require.NoError(r, s1.RPC("ConnectCA.ConfigurationSet", args, &reply))
})
}
rootsList, activeRoot, err = getTestRoots(s1, "dc1")
require.NoError(t, err)
require.NotEmpty(t, rootsList.Roots)
require.NotNil(t, activeRoot)
}
func TestLeader_Consul_BadCAConfigShouldntPreventLeaderEstablishment(t *testing.T) {
ca.SkipIfVaultNotPresent(t)
_, s1 := testServerWithConfig(t, func(c *Config) {
c.Build = "1.9.1"
c.PrimaryDatacenter = "dc1"
c.CAConfig = &structs.CAConfiguration{
Provider: "consul",
Config: map[string]interface{}{
"RootCert": "garbage",
},
}
})
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
rootsList, activeRoot, err := getTestRoots(s1, "dc1")
require.NoError(t, err)
require.Empty(t, rootsList.Roots)
require.Nil(t, activeRoot)
newConfig := &structs.CAConfiguration{
Provider: "consul",
Config: map[string]interface{}{},
}
{
args := &structs.CARequest{
Datacenter: "dc1",
Config: newConfig,
}
var reply interface{}
retry.Run(t, func(r *retry.R) {
require.NoError(r, s1.RPC("ConnectCA.ConfigurationSet", args, &reply))
})
}
rootsList, activeRoot, err = getTestRoots(s1, "dc1")
require.NoError(t, err)
require.NotEmpty(t, rootsList.Roots)
require.NotNil(t, activeRoot)
}

View File

@ -108,6 +108,7 @@ const (
intentionMigrationRoutineName = "intention config entry migration"
secondaryCARootWatchRoutineName = "secondary CA roots watch"
intermediateCertRenewWatchRoutineName = "intermediate cert renew watch"
backgroundCAInitializationRoutineName = "CA initialization"
)
var (
@ -468,7 +469,7 @@ func NewServer(config *Config, flat Deps) (*Server, error) {
return nil, fmt.Errorf("Failed to start Raft: %v", err)
}
s.caManager = NewCAManager(&caDelegateWithState{s}, s.loggers.Named(logging.Connect), s.config)
s.caManager = NewCAManager(&caDelegateWithState{s}, s.leaderRoutineManager, s.loggers.Named(logging.Connect), s.config)
if s.config.ConnectEnabled && (s.config.AutoEncryptAllowTLS || s.config.AutoConfigAuthzEnabled) {
go s.connectCARootsMonitor(&lib.StopChannelContext{StopCh: s.shutdownCh})
}

View File

@ -134,9 +134,9 @@ func (s *Server) getCARoots(ws memdb.WatchSet, state *state.Store) (*structs.Ind
func (s *Server) SignCertificate(csr *x509.CertificateRequest, spiffeID connect.CertURI) (*structs.IssuedCert, error) {
provider, caRoot := s.caManager.getCAProvider()
if provider == nil {
return nil, fmt.Errorf("internal error: CA provider is nil")
return nil, fmt.Errorf("CA is uninitialized and unable to sign certificates yet: provider is nil")
} else if caRoot == nil {
return nil, fmt.Errorf("internal error: CA root is nil")
return nil, fmt.Errorf("CA is uninitialized and unable to sign certificates yet: no root certificate")
}
// Verify that the CSR entity is in the cluster's trust domain

View File

@ -6,6 +6,7 @@ import (
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
@ -19,7 +20,6 @@ func TestIntentionListCommand_noTabs(t *testing.T) {
func TestIntentionListCommand(t *testing.T) {
t.Parallel()
require := require.New(t)
a := agent.NewTestAgent(t, ``)
defer a.Shutdown()
client := a.Client()
@ -28,13 +28,17 @@ func TestIntentionListCommand(t *testing.T) {
var id string
{
var err error
//nolint:staticcheck
id, _, err = client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",
Action: api.IntentionActionAllow,
}, nil)
require.NoError(err)
// This needs to be in a retry in 1.9+ due to the potential to get errors about
// intentions being read only during intention -> config entry migration.
retry.Run(t, func(r *retry.R) {
//nolint:staticcheck
id, _, err = client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",
Action: api.IntentionActionAllow,
}, nil)
require.NoError(r, err)
})
}
// List all intentions
@ -42,6 +46,6 @@ func TestIntentionListCommand(t *testing.T) {
cmd := New(ui)
args := []string{"-http-addr=" + a.HTTPAddr()}
require.Equal(0, cmd.Run(args), ui.ErrorWriter.String())
require.Contains(ui.OutputWriter.String(), id)
require.Equal(t, 0, cmd.Run(args), ui.ErrorWriter.String())
require.Contains(t, ui.OutputWriter.String(), id)
}