ca: move provider creation into CAManager

This further decouples the CAManager from Server. It reduces the interface between them and
removes the need for the SetLogger method on providers.
This commit is contained in:
Daniel Nephin 2021-06-21 17:51:37 -04:00 committed by Dhia Ayachi
parent b1877660d5
commit fc14f5ab14
12 changed files with 52 additions and 114 deletions

View File

@ -3,8 +3,6 @@ package ca
import ( import (
"crypto/x509" "crypto/x509"
"errors" "errors"
"github.com/hashicorp/go-hclog"
) )
//go:generate mockery -name Provider -inpkg //go:generate mockery -name Provider -inpkg
@ -171,13 +169,6 @@ type Provider interface {
Cleanup(providerTypeChange bool, otherConfig map[string]interface{}) error Cleanup(providerTypeChange bool, otherConfig map[string]interface{}) error
} }
// NeedsLogger is an optional interface that allows a CA provider to use the
// Consul logger to output diagnostic messages.
type NeedsLogger interface {
// SetLogger will pass a configured Logger to the provider.
SetLogger(logger hclog.Logger)
}
// NeedsStop is an optional interface that allows a CA to define a function // NeedsStop is an optional interface that allows a CA to define a function
// to be called when the CA instance is no longer in use. This is different // to be called when the CA instance is no longer in use. This is different
// from Cleanup(), as only the local provider instance is being shut down // from Cleanup(), as only the local provider instance is being shut down

View File

@ -18,7 +18,6 @@ import (
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
) )
const ( const (
@ -78,12 +77,9 @@ type AWSProvider struct {
logger hclog.Logger logger hclog.Logger
} }
// SetLogger implements NeedsLogger // NewAWSProvider returns a new AWSProvider
func (a *AWSProvider) SetLogger(logger hclog.Logger) { func NewAWSProvider(logger hclog.Logger) *AWSProvider {
a.logger = logger. return &AWSProvider{logger: logger}
ResetNamed(logging.Connect).
Named(logging.CA).
Named(logging.AWS)
} }
// Configure implements Provider // Configure implements Provider

View File

@ -7,9 +7,10 @@ import (
"github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/service/acmpca" "github.com/aws/aws-sdk-go/service/acmpca"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/sdk/testutil" "github.com/hashicorp/consul/sdk/testutil"
"github.com/stretchr/testify/require"
) )
// skipIfAWSNotConfigured skips the test unless ENABLE_AWS_PCA_TESTS=true. // skipIfAWSNotConfigured skips the test unless ENABLE_AWS_PCA_TESTS=true.
@ -375,9 +376,7 @@ func TestAWSProvider_Cleanup(t *testing.T) {
} }
func testAWSProvider(t *testing.T, cfg ProviderConfig) *AWSProvider { func testAWSProvider(t *testing.T, cfg ProviderConfig) *AWSProvider {
p := &AWSProvider{} p := &AWSProvider{logger: testutil.Logger(t)}
logger := testutil.Logger(t)
p.SetLogger(logger)
require.NoError(t, p.Configure(cfg)) require.NoError(t, p.Configure(cfg))
return p return p
} }

View File

@ -19,7 +19,6 @@ import (
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
) )
var ( var (
@ -38,7 +37,7 @@ type ConsulProvider struct {
clusterID string clusterID string
isPrimary bool isPrimary bool
spiffeID *connect.SpiffeIDSigning spiffeID *connect.SpiffeIDSigning
logger hclog.Logger Logger hclog.Logger
// testState is only used to test Consul leader's handling of providers that // testState is only used to test Consul leader's handling of providers that
// need to persist state. Consul provider actually manages it's state directly // need to persist state. Consul provider actually manages it's state directly
@ -127,7 +126,7 @@ func (c *ConsulProvider) Configure(cfg ProviderConfig) error {
return err return err
} }
c.logger.Debug("consul CA provider configured", c.Logger.Debug("consul CA provider configured",
"id", c.id, "id", c.id,
"is_primary", c.isPrimary, "is_primary", c.isPrimary,
) )
@ -667,14 +666,6 @@ func (c *ConsulProvider) generateCA(privateKey string, sn uint64) (string, error
return buf.String(), nil return buf.String(), nil
} }
// SetLogger implements the NeedsLogger interface so the provider can log important messages.
func (c *ConsulProvider) SetLogger(logger hclog.Logger) {
c.logger = logger.
ResetNamed(logging.Connect).
Named(logging.CA).
Named(logging.Consul)
}
func (c *ConsulProvider) parseTestState(rawConfig map[string]interface{}, state map[string]string) { func (c *ConsulProvider) parseTestState(rawConfig map[string]interface{}, state map[string]string) {
c.testState = nil c.testState = nil
if rawTestState, ok := rawConfig["test_state"]; ok { if rawTestState, ok := rawConfig["test_state"]; ok {

View File

@ -17,7 +17,6 @@ import (
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
) )
const VaultCALeafCertRole = "leaf-cert" const VaultCALeafCertRole = "leaf-cert"
@ -38,8 +37,11 @@ type VaultProvider struct {
logger hclog.Logger logger hclog.Logger
} }
func NewVaultProvider() *VaultProvider { func NewVaultProvider(logger hclog.Logger) *VaultProvider {
return &VaultProvider{shutdown: func() {}} return &VaultProvider{
shutdown: func() {},
logger: logger,
}
} }
func vaultTLSConfig(config *structs.VaultCAProviderConfig) *vaultapi.TLSConfig { func vaultTLSConfig(config *structs.VaultCAProviderConfig) *vaultapi.TLSConfig {
@ -143,14 +145,6 @@ func (v *VaultProvider) renewToken(ctx context.Context, watcher *vaultapi.Lifeti
} }
} }
// SetLogger implements the NeedsLogger interface so the provider can log important messages.
func (v *VaultProvider) SetLogger(logger hclog.Logger) {
v.logger = logger.
ResetNamed(logging.Connect).
Named(logging.CA).
Named(logging.Vault)
}
// State implements Provider. Vault provider needs no state other than the // State implements Provider. Vault provider needs no state other than the
// user-provided config currently. // user-provided config currently.
func (v *VaultProvider) State() (map[string]string, error) { func (v *VaultProvider) State() (map[string]string, error) {

View File

@ -8,12 +8,13 @@ import (
"testing" "testing"
"time" "time"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
vaultapi "github.com/hashicorp/vault/api" vaultapi "github.com/hashicorp/vault/api"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil/retry"
) )
func TestVaultCAProvider_VaultTLSConfig(t *testing.T) { func TestVaultCAProvider_VaultTLSConfig(t *testing.T) {
@ -485,7 +486,7 @@ func createVaultProvider(t *testing.T, isPrimary bool, addr, token string, rawCo
conf[k] = v conf[k] = v
} }
provider := NewVaultProvider() provider := NewVaultProvider(hclog.New(nil))
cfg := ProviderConfig{ cfg := ProviderConfig{
ClusterID: connect.TestClusterID, ClusterID: connect.TestClusterID,
@ -494,11 +495,6 @@ func createVaultProvider(t *testing.T, isPrimary bool, addr, token string, rawCo
RawConfig: conf, RawConfig: conf,
} }
logger := hclog.New(&hclog.LoggerOptions{
Output: ioutil.Discard,
})
provider.SetLogger(logger)
if !isPrimary { if !isPrimary {
cfg.IsPrimary = false cfg.IsPrimary = false
cfg.Datacenter = "dc2" cfg.Datacenter = "dc2"

View File

@ -7,14 +7,15 @@ import (
"os/exec" "os/exec"
"sync" "sync"
"github.com/hashicorp/go-hclog"
vaultapi "github.com/hashicorp/vault/api"
"github.com/mitchellh/go-testing-interface"
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/freeport" "github.com/hashicorp/consul/sdk/freeport"
"github.com/hashicorp/consul/sdk/testutil/retry" "github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/go-hclog"
vaultapi "github.com/hashicorp/vault/api"
"github.com/mitchellh/go-testing-interface"
) )
// KeyTestCases is a list of the important CA key types that we should test // KeyTestCases is a list of the important CA key types that we should test
@ -75,13 +76,9 @@ func CASigningKeyTypeCases() []CASigningKeyTypes {
// TestConsulProvider creates a new ConsulProvider, taking care to stub out it's // TestConsulProvider creates a new ConsulProvider, taking care to stub out it's
// Logger so that logging calls don't panic. If logging output is important // Logger so that logging calls don't panic. If logging output is important
// SetLogger can be called again with another logger to capture logs.
func TestConsulProvider(t testing.T, d ConsulProviderStateDelegate) *ConsulProvider { func TestConsulProvider(t testing.T, d ConsulProviderStateDelegate) *ConsulProvider {
provider := &ConsulProvider{Delegate: d} logger := hclog.New(&hclog.LoggerOptions{Output: ioutil.Discard})
logger := hclog.New(&hclog.LoggerOptions{ provider := &ConsulProvider{Delegate: d, Logger: logger}
Output: ioutil.Discard,
})
provider.SetLogger(logger)
return provider return provider
} }

View File

@ -1,20 +0,0 @@
package consul
import (
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs"
)
// consulCADelegate providers callbacks for the Consul CA provider
// to use the state store for its operations.
type consulCADelegate struct {
srv *Server
}
func (c *consulCADelegate) State() *state.Store {
return c.srv.fsm.State()
}
func (c *consulCADelegate) ApplyCARequest(req *structs.CARequest) (interface{}, error) {
return c.srv.raftApply(structs.ConnectCARequestType, req)
}

View File

@ -2,12 +2,10 @@ package consul
import ( import (
"context" "context"
"fmt"
"time" "time"
"golang.org/x/time/rate" "golang.org/x/time/rate"
"github.com/hashicorp/consul/agent/connect/ca"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging" "github.com/hashicorp/consul/logging"
) )
@ -51,28 +49,6 @@ func (s *Server) stopConnectLeader() {
s.leaderRoutineManager.Stop(caSigningMetricRoutineName) s.leaderRoutineManager.Stop(caSigningMetricRoutineName)
} }
// createProvider returns a connect CA provider from the given config.
func (s *Server) createCAProvider(conf *structs.CAConfiguration) (ca.Provider, error) {
var p ca.Provider
switch conf.Provider {
case structs.ConsulCAProvider:
p = &ca.ConsulProvider{Delegate: &consulCADelegate{s}}
case structs.VaultCAProvider:
p = ca.NewVaultProvider()
case structs.AWSCAProvider:
p = &ca.AWSProvider{}
default:
return nil, fmt.Errorf("unknown CA provider %q", conf.Provider)
}
// If the provider implements NeedsLogger, we give it our logger.
if needsLogger, ok := p.(ca.NeedsLogger); ok {
needsLogger.SetLogger(s.logger)
}
return p, nil
}
func (s *Server) runCARootPruning(ctx context.Context) error { func (s *Server) runCARootPruning(ctx context.Context) error {
ticker := time.NewTicker(caRootPruneInterval) ticker := time.NewTicker(caRootPruneInterval)
defer ticker.Stop() defer ticker.Stop()

View File

@ -32,10 +32,9 @@ const (
// caServerDelegate is an interface for server operations for facilitating // caServerDelegate is an interface for server operations for facilitating
// easier testing. // easier testing.
type caServerDelegate interface { type caServerDelegate interface {
State() *state.Store ca.ConsulProviderStateDelegate
IsLeader() bool IsLeader() bool
createCAProvider(conf *structs.CAConfiguration) (ca.Provider, error)
forwardDC(method, dc string, args interface{}, reply interface{}) error forwardDC(method, dc string, args interface{}, reply interface{}) error
generateCASignRequest(csr string) *structs.CASignRequest generateCASignRequest(csr string) *structs.CASignRequest
raftApply(t structs.MessageType, msg interface{}) (interface{}, error) raftApply(t structs.MessageType, msg interface{}) (interface{}, error)
@ -68,6 +67,8 @@ type CAManager struct {
actingSecondaryCA bool // True if this datacenter has been initialized as a secondary CA. actingSecondaryCA bool // True if this datacenter has been initialized as a secondary CA.
leaderRoutineManager *routine.Manager leaderRoutineManager *routine.Manager
// providerShim is used to test CAManager with a fake provider.
providerShim ca.Provider
} }
type caDelegateWithState struct { type caDelegateWithState struct {
@ -78,6 +79,10 @@ func (c *caDelegateWithState) State() *state.Store {
return c.fsm.State() return c.fsm.State()
} }
func (c *caDelegateWithState) ApplyCARequest(req *structs.CARequest) (interface{}, error) {
return c.Server.raftApplyMsgpack(structs.ConnectCARequestType, req)
}
func NewCAManager(delegate caServerDelegate, leaderRoutineManager *routine.Manager, logger hclog.Logger, config *Config) *CAManager { func NewCAManager(delegate caServerDelegate, leaderRoutineManager *routine.Manager, logger hclog.Logger, config *Config) *CAManager {
return &CAManager{ return &CAManager{
delegate: delegate, delegate: delegate,
@ -344,7 +349,7 @@ func (c *CAManager) InitializeCA() (reterr error) {
if err != nil { if err != nil {
return err return err
} }
provider, err := c.delegate.createCAProvider(conf) provider, err := c.newProvider(conf)
if err != nil { if err != nil {
return err return err
} }
@ -394,6 +399,24 @@ func (c *CAManager) InitializeCA() (reterr error) {
return nil return nil
} }
// createProvider returns a connect CA provider from the given config.
func (c *CAManager) newProvider(conf *structs.CAConfiguration) (ca.Provider, error) {
logger := c.logger.Named(conf.Provider)
switch conf.Provider {
case structs.ConsulCAProvider:
return &ca.ConsulProvider{Delegate: c.delegate, Logger: logger}, nil
case structs.VaultCAProvider:
return ca.NewVaultProvider(logger), nil
case structs.AWSCAProvider:
return ca.NewAWSProvider(logger), nil
default:
if c.providerShim != nil {
return c.providerShim, nil
}
return nil, fmt.Errorf("unknown CA provider %q", conf.Provider)
}
}
// initializeRootCA runs the initialization logic for a root CA. It should only // initializeRootCA runs the initialization logic for a root CA. It should only
// be called while the state lock is held by setting the state to non-ready. // be called while the state lock is held by setting the state to non-ready.
func (c *CAManager) initializeRootCA(provider ca.Provider, conf *structs.CAConfiguration) error { func (c *CAManager) initializeRootCA(provider ca.Provider, conf *structs.CAConfiguration) error {
@ -774,7 +797,7 @@ func (c *CAManager) UpdateConfiguration(args *structs.CARequest) (reterr error)
// and get the current active root CA. This acts as a good validation // and get the current active root CA. This acts as a good validation
// of the config and makes sure the provider is functioning correctly // of the config and makes sure the provider is functioning correctly
// before we commit any changes to Raft. // before we commit any changes to Raft.
newProvider, err := c.delegate.createCAProvider(args.Config) newProvider, err := c.newProvider(args.Config)
if err != nil { if err != nil {
return fmt.Errorf("could not initialize provider: %v", err) return fmt.Errorf("could not initialize provider: %v", err)
} }

View File

@ -473,7 +473,7 @@ func NewServer(config *Config, flat Deps) (*Server, error) {
return nil, fmt.Errorf("Failed to start Raft: %v", err) return nil, fmt.Errorf("Failed to start Raft: %v", err)
} }
s.caManager = NewCAManager(&caDelegateWithState{s}, s.leaderRoutineManager, s.loggers.Named(logging.Connect), s.config) s.caManager = NewCAManager(&caDelegateWithState{s}, s.leaderRoutineManager, s.logger.ResetNamed("connect.ca"), s.config)
if s.config.ConnectEnabled && (s.config.AutoEncryptAllowTLS || s.config.AutoConfigAuthzEnabled) { if s.config.ConnectEnabled && (s.config.AutoEncryptAllowTLS || s.config.AutoConfigAuthzEnabled) {
go s.connectCARootsMonitor(&lib.StopChannelContext{StopCh: s.shutdownCh}) go s.connectCARootsMonitor(&lib.StopChannelContext{StopCh: s.shutdownCh})
} }

View File

@ -16,7 +16,6 @@ import (
"github.com/hashicorp/memberlist" "github.com/hashicorp/memberlist"
"github.com/hashicorp/raft" "github.com/hashicorp/raft"
"github.com/hashicorp/consul/agent/connect/ca"
"github.com/hashicorp/consul/ipaddr" "github.com/hashicorp/consul/ipaddr"
"github.com/hashicorp/go-uuid" "github.com/hashicorp/go-uuid"
@ -1645,10 +1644,6 @@ func TestServer_CALogging(t *testing.T) {
defer s1.Shutdown() defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1") testrpc.WaitForLeader(t, s1.RPC, "dc1")
if _, ok := s1.caManager.provider.(ca.NeedsLogger); !ok {
t.Fatalf("provider does not implement NeedsLogger")
}
// Wait til CA root is setup // Wait til CA root is setup
retry.Run(t, func(r *retry.R) { retry.Run(t, func(r *retry.R) {
var out structs.IndexedCARoots var out structs.IndexedCARoots