From dad0f189a2e551257e24bf669b04644525d252ed Mon Sep 17 00:00:00 2001 From: Matt Keeler Date: Tue, 28 Jul 2020 15:31:48 -0400 Subject: [PATCH] Agent Auto Config: Implement Certificate Generation (#8360) Most of the groundwork was laid in previous PRs between adding the cert-monitor package to extracting the logic of signing certificates out of the connect_ca_endpoint.go code and into a method on the server. This also refactors the auto-config package a bit to split things out into multiple files. --- agent/agent.go | 62 +- agent/agent_test.go | 37 +- agent/auto-config/auto_config.go | 401 ++-- agent/auto-config/auto_config_test.go | 391 +++- agent/auto-config/builder.go | 30 + agent/auto-config/config.go | 115 ++ agent/auto-config/config_translate.go | 68 + agent/cert-monitor/cert_monitor.go | 1 + agent/config/builder.go | 4 + agent/config/runtime.go | 4 - agent/config/runtime_test.go | 35 +- agent/consul/auto_config_endpoint.go | 196 +- agent/consul/auto_config_endpoint_test.go | 529 +++-- agent/consul/server.go | 20 +- proto/pbautoconf/auto_config.pb.go | 289 ++- proto/pbautoconf/auto_config.proto | 15 + proto/pbconnect/connect.pb.binary.go | 38 + proto/pbconnect/connect.pb.go | 2271 +++++++++++++++++++++ proto/pbconnect/connect.proto | 147 ++ proto/translate.go | 56 + proto/translate_test.go | 65 + 21 files changed, 4325 insertions(+), 449 deletions(-) create mode 100644 agent/auto-config/builder.go create mode 100644 agent/auto-config/config.go create mode 100644 proto/pbconnect/connect.pb.binary.go create mode 100644 proto/pbconnect/connect.pb.go create mode 100644 proto/pbconnect/connect.proto create mode 100644 proto/translate.go create mode 100644 proto/translate_test.go diff --git a/agent/agent.go b/agent/agent.go index d9565d997..9c75a6f51 100644 --- a/agent/agent.go +++ b/agent/agent.go @@ -460,6 +460,10 @@ func New(options ...AgentOption) (*Agent, error) { // loaded any auto-config sources yet. a.config = config + // create the cache using the rate limiting settings from the config. Note that this means + // that these limits are not reloadable. + a.cache = cache.New(a.config.Cache) + if flat.logger == nil { logConf := &logging.Config{ LogLevel: config.LogLevel, @@ -525,14 +529,34 @@ func New(options ...AgentOption) (*Agent, error) { return nil, fmt.Errorf("Failed to setup node ID: %v", err) } - acOpts := []autoconf.Option{ - autoconf.WithDirectRPC(a.connPool), - autoconf.WithTLSConfigurator(a.tlsConfigurator), - autoconf.WithBuilderOpts(flat.builderOpts), - autoconf.WithLogger(a.logger), - autoconf.WithOverrides(flat.overrides...), + // We used to do this in the Start method. However it doesn't need to go + // there any longer. Originally it did because we passed the agent + // delegate to some of the cache registrations. Now we just + // pass the agent itself so its safe to move here. + a.registerCache() + + cmConf := new(certmon.Config). + WithCache(a.cache). + WithTLSConfigurator(a.tlsConfigurator). + WithDNSSANs(a.config.AutoConfig.DNSSANs). + WithIPSANs(a.config.AutoConfig.IPSANs). + WithDatacenter(a.config.Datacenter). + WithNodeName(a.config.NodeName). + WithFallback(a.autoConfigFallbackTLS). + WithLogger(a.logger.Named(logging.AutoConfig)). + WithTokens(a.tokens) + acCertMon, err := certmon.New(cmConf) + if err != nil { + return nil, err } - ac, err := autoconf.New(acOpts...) + + acConf := new(autoconf.Config). + WithDirectRPC(a.connPool). + WithBuilderOpts(flat.builderOpts). + WithLogger(a.logger). + WithOverrides(flat.overrides...). + WithCertMonitor(acCertMon) + ac, err := autoconf.New(acConf) if err != nil { return nil, err } @@ -663,9 +687,6 @@ func (a *Agent) Start(ctx context.Context) error { // regular and on-demand state synchronizations (anti-entropy). a.sync = ae.NewStateSyncer(a.State, c.AEInterval, a.shutdownCh, a.logger) - // create the cache - a.cache = cache.New(c.Cache) - // create the config for the rpc server/client consulCfg, err := a.consulConfig() if err != nil { @@ -714,10 +735,6 @@ func (a *Agent) Start(ctx context.Context) error { a.State.Delegate = a.delegate a.State.TriggerSyncChanges = a.sync.SyncChanges.Trigger - // Register the cache. We do this much later so the delegate is - // populated from above. - a.registerCache() - if a.config.AutoEncryptTLS && !a.config.ServerMode { reply, err := a.autoEncryptInitialCertificate(ctx) if err != nil { @@ -755,6 +772,9 @@ func (a *Agent) Start(ctx context.Context) error { 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) + } a.serviceManager.Start() // Load checks/services/metadata. @@ -867,6 +887,10 @@ func (a *Agent) autoEncryptInitialCertificate(ctx context.Context) (*structs.Sig return client.RequestAutoEncryptCerts(ctx, addrs, a.config.ServerPort, a.tokens.AgentToken(), a.config.AutoEncryptDNSSAN, a.config.AutoEncryptIPSAN) } +func (a *Agent) autoConfigFallbackTLS(ctx context.Context) (*structs.SignedResponse, error) { + return a.autoConf.FallbackTLS(ctx) +} + func (a *Agent) listenAndServeGRPC() error { if len(a.config.GRPCAddrs) < 1 { return nil @@ -1827,6 +1851,16 @@ func (a *Agent) ShutdownAgent() error { // Stop the watches to avoid any notification/state change during shutdown a.stopAllWatches() + // this would be cancelled anyways (by the closing of the shutdown ch) but + // 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() diff --git a/agent/agent_test.go b/agent/agent_test.go index 149f3610b..29fee024d 100644 --- a/agent/agent_test.go +++ b/agent/agent_test.go @@ -4645,7 +4645,8 @@ func TestAutoConfig_Integration(t *testing.T) { // eventually this test should really live with integration tests // the goal here is to have one test server and another test client // spin up both agents and allow the server to authorize the auto config - // request and then see the client joined + // request and then see the client joined. Finally we force a CA roots + // update and wait to see that the agents TLS certificate gets updated. cfgDir := testutil.TempDir(t, "auto-config") @@ -4683,7 +4684,6 @@ func TestAutoConfig_Integration(t *testing.T) { cert_file = "` + certFile + `" key_file = "` + keyFile + `" connect { enabled = true } - auto_encrypt { allow_tls = true } auto_config { authorization { enabled = true @@ -4740,11 +4740,44 @@ func TestAutoConfig_Integration(t *testing.T) { defer client.Shutdown() + retry.Run(t, func(r *retry.R) { + require.NotNil(r, client.Agent.tlsConfigurator.Cert()) + }) + // when this is successful we managed to get the gossip key and serf addresses to bind to // and then connect. Additionally we would have to have certificates or else the // verify_incoming config on the server would not let it work. testrpc.WaitForTestAgent(t, client.RPC, "dc1", testrpc.WithToken(TestDefaultMasterToken)) + // grab the existing cert + cert1 := client.Agent.tlsConfigurator.Cert() + require.NotNil(t, cert1) + + // force a roots rotation by updating the CA config + t.Logf("Forcing roots rotation on the server") + ca := connect.TestCA(t, nil) + req := &structs.CARequest{ + Datacenter: "dc1", + WriteRequest: structs.WriteRequest{Token: TestDefaultMasterToken}, + Config: &structs.CAConfiguration{ + Provider: "consul", + Config: map[string]interface{}{ + "LeafCertTTL": "1h", + "PrivateKey": ca.SigningKey, + "RootCert": ca.RootCert, + "RotationPeriod": "6h", + "IntermediateCertTTL": "3h", + }, + }, + } + var reply interface{} + require.NoError(t, srv.RPC("ConnectCA.ConfigurationSet", &req, &reply)) + + // ensure that a new cert gets generated and pushed into the TLS configurator + retry.Run(t, func(r *retry.R) { + require.NotEqual(r, cert1, client.Agent.tlsConfigurator.Cert()) + }) + // spot check that we now have an ACL token require.NotEmpty(t, client.tokens.AgentToken()) } diff --git a/agent/auto-config/auto_config.go b/agent/auto-config/auto_config.go index 047fbc324..61587c39a 100644 --- a/agent/auto-config/auto_config.go +++ b/agent/auto-config/auto_config.go @@ -13,84 +13,38 @@ import ( "time" "github.com/hashicorp/consul/agent/config" + "github.com/hashicorp/consul/agent/connect" + "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/logging" "github.com/hashicorp/consul/proto/pbautoconf" - "github.com/hashicorp/consul/tlsutil" "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" ) -// DirectRPC is the interface that needs to be satisifed for AutoConfig to be able to perform -// direct RPCs against individual servers. This should not use -type DirectRPC interface { - RPC(dc string, node string, addr net.Addr, method string, args interface{}, reply interface{}) error -} - -type options struct { - logger hclog.Logger - directRPC DirectRPC - tlsConfigurator *tlsutil.Configurator - builderOpts config.BuilderOpts - waiter *lib.RetryWaiter - overrides []config.Source -} - -// Option represents one point of configurability for the New function -// when creating a new AutoConfig object -type Option func(*options) - -// WithLogger will cause the created AutoConfig type to use the provided logger -func WithLogger(logger hclog.Logger) Option { - return func(opt *options) { - opt.logger = logger +var ( + pbMarshaler = &jsonpb.Marshaler{ + OrigName: false, + EnumsAsInts: false, + Indent: " ", + EmitDefaults: true, } -} -// WithTLSConfigurator will cause the created AutoConfig type to use the provided configurator -func WithTLSConfigurator(tlsConfigurator *tlsutil.Configurator) Option { - return func(opt *options) { - opt.tlsConfigurator = tlsConfigurator + pbUnmarshaler = &jsonpb.Unmarshaler{ + AllowUnknownFields: false, } -} - -// WithConnectionPool will cause the created AutoConfig type to use the provided connection pool -func WithDirectRPC(directRPC DirectRPC) Option { - return func(opt *options) { - opt.directRPC = directRPC - } -} - -// WithBuilderOpts will cause the created AutoConfig type to use the provided CLI builderOpts -func WithBuilderOpts(builderOpts config.BuilderOpts) Option { - return func(opt *options) { - opt.builderOpts = builderOpts - } -} - -// WithRetryWaiter will cause the created AutoConfig type to use the provided retry waiter -func WithRetryWaiter(waiter *lib.RetryWaiter) Option { - return func(opt *options) { - opt.waiter = waiter - } -} - -// WithOverrides is used to provide a config source to append to the tail sources -// during config building. It is really only useful for testing to tune non-user -// configurable tunables to make various tests converge more quickly than they -// could otherwise. -func WithOverrides(overrides ...config.Source) Option { - return func(opt *options) { - opt.overrides = overrides - } -} +) // 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. @@ -101,86 +55,52 @@ func WithOverrides(overrides ...config.Source) Option { // 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 { - config *config.RuntimeConfig - builderOpts config.BuilderOpts - logger hclog.Logger - directRPC DirectRPC - tlsConfigurator *tlsutil.Configurator - autoConfigData string - waiter *lib.RetryWaiter - overrides []config.Source -} - -func flattenOptions(opts []Option) options { - var flat options - for _, opt := range opts { - opt(&flat) - } - return flat + builderOpts config.BuilderOpts + logger hclog.Logger + directRPC DirectRPC + waiter *lib.RetryWaiter + overrides []config.Source + certMonitor CertMonitor + config *config.RuntimeConfig + autoConfigData string + cancel context.CancelFunc } // New creates a new AutoConfig object for providing automatic // Consul configuration. -func New(options ...Option) (*AutoConfig, error) { - flat := flattenOptions(options) +func New(config *Config) (*AutoConfig, error) { + if config == nil { + return nil, fmt.Errorf("must provide a config struct") + } - if flat.directRPC == nil { + if config.DirectRPC == nil { return nil, fmt.Errorf("must provide a direct RPC delegate") } - if flat.tlsConfigurator == nil { - return nil, fmt.Errorf("must provide a TLS configurator") - } - - logger := flat.logger + logger := config.Logger if logger == nil { logger = hclog.NewNullLogger() } else { logger = logger.Named(logging.AutoConfig) } - waiter := flat.waiter + waiter := config.Waiter if waiter == nil { waiter = lib.NewRetryWaiter(1, 0, 10*time.Minute, lib.NewJitterRandomStagger(25)) } ac := &AutoConfig{ - builderOpts: flat.builderOpts, - logger: logger, - directRPC: flat.directRPC, - tlsConfigurator: flat.tlsConfigurator, - waiter: waiter, - overrides: flat.overrides, + builderOpts: config.BuilderOpts, + logger: logger, + directRPC: config.DirectRPC, + waiter: waiter, + overrides: config.Overrides, + certMonitor: config.CertMonitor, } return ac, nil } -// LoadConfig will build the configuration including the extraHead source injected -// after all other defaults but before any user supplied configuration and the overrides -// source injected as the final source in the configuration parsing chain. -func LoadConfig(builderOpts config.BuilderOpts, extraHead config.Source, overrides ...config.Source) (*config.RuntimeConfig, []string, error) { - b, err := config.NewBuilder(builderOpts) - if err != nil { - return nil, nil, err - } - - if extraHead.Data != "" { - b.Head = append(b.Head, extraHead) - } - - if len(overrides) != 0 { - b.Tail = append(b.Tail, overrides...) - } - - cfg, err := b.BuildAndValidate() - if err != nil { - return nil, nil, err - } - - return &cfg, b.Warnings, 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) { @@ -219,8 +139,18 @@ func (ac *AutoConfig) restorePersistedAutoConfig() (bool, error) { 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) - ac.autoConfigData = string(content) return true, nil } @@ -305,12 +235,12 @@ func (ac *AutoConfig) introToken() (string, error) { return token, nil } -// autoConfigHosts is responsible for taking the list of server addresses and +// 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) autoConfigHosts() ([]string, error) { +func (ac *AutoConfig) serverHosts() ([]string, error) { servers := ac.config.AutoConfig.ServerAddresses providers := make(map[string]discover.Provider) @@ -388,31 +318,20 @@ func (ac *AutoConfig) resolveHost(hostPort string) []net.TCPAddr { return addrs } -// recordAutoConfigReply takes an AutoConfig RPC reply records it with the agent +// 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) recordAutoConfigReply(reply *pbautoconf.AutoConfigResponse) error { - // overwrite the auto encrypt DNS SANs with the ones specified in the auto_config stanza - if len(ac.config.AutoConfig.DNSSANs) > 0 && reply.Config.AutoEncrypt != nil { - reply.Config.AutoEncrypt.DNSSAN = ac.config.AutoConfig.DNSSANs - } - - // overwrite the auto encrypt IP SANs with the ones specified in the auto_config stanza - if len(ac.config.AutoConfig.IPSANs) > 0 && reply.Config.AutoEncrypt != nil { - var ips []string - for _, ip := range ac.config.AutoConfig.IPSANs { - ips = append(ips, ip.String()) - } - reply.Config.AutoEncrypt.IPSAN = ips - } - - conf, err := json.Marshal(translateConfig(reply.Config)) +func (ac *AutoConfig) recordResponse(resp *pbautoconf.AutoConfigResponse) error { + serialized, err := pbMarshaler.MarshalToString(resp) if err != nil { - return fmt.Errorf("failed to encode auto-config configuration as JSON: %w", err) + return fmt.Errorf("failed to encode auto-config response as JSON: %w", err) } - ac.autoConfigData = string(conf) + if err := ac.update(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 @@ -420,7 +339,7 @@ func (ac *AutoConfig) recordAutoConfigReply(reply *pbautoconf.AutoConfigResponse path := filepath.Join(ac.config.DataDir, autoConfigFileName) - err = ioutil.WriteFile(path, conf, 0660) + err = ioutil.WriteFile(path, []byte(serialized), 0660) if err != nil { return fmt.Errorf("failed to write auto-config configurations: %w", err) } @@ -435,10 +354,10 @@ func (ac *AutoConfig) recordAutoConfigReply(reply *pbautoconf.AutoConfigResponse // successful the bool return will be true and the err value will indicate whether we // successfully recorded the auto config settings (persisted to disk and stored internally // on the AutoConfig object) -func (ac *AutoConfig) getInitialConfigurationOnce(ctx context.Context) (bool, error) { +func (ac *AutoConfig) getInitialConfigurationOnce(ctx context.Context, csr string, key string) (*pbautoconf.AutoConfigResponse, error) { token, err := ac.introToken() if err != nil { - return false, err + return nil, err } request := pbautoconf.AutoConfigRequest{ @@ -446,50 +365,63 @@ func (ac *AutoConfig) getInitialConfigurationOnce(ctx context.Context) (bool, er Node: ac.config.NodeName, Segment: ac.config.SegmentName, JWT: token, + CSR: csr, } - var reply pbautoconf.AutoConfigResponse + var resp pbautoconf.AutoConfigResponse - servers, err := ac.autoConfigHosts() + servers, err := ac.serverHosts() if err != nil { - return false, err + 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 false, ctx.Err() + return nil, ctx.Err() } ac.logger.Debug("making AutoConfig.InitialConfiguration RPC", "addr", addr.String()) - if err = ac.directRPC.RPC(ac.config.Datacenter, ac.config.NodeName, &addr, "AutoConfig.InitialConfiguration", &request, &reply); err != nil { + if err = ac.directRPC.RPC(ac.config.Datacenter, ac.config.NodeName, &addr, "AutoConfig.InitialConfiguration", &request, &resp); err != nil { ac.logger.Error("AutoConfig.InitialConfiguration RPC failed", "addr", addr.String(), "error", err) continue } - return true, ac.recordAutoConfigReply(&reply) + // update the Certificate with the private key we generated locally + if resp.Certificate != nil { + resp.Certificate.PrivateKeyPEM = key + } + + return &resp, nil } } - return false, ctx.Err() + return nil, ctx.Err() } // 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 { + // generate a CSR + csr, key, err := ac.generateCSR() + if err != nil { + return err + } + // this resets the failures so that we will perform immediate request wait := ac.waiter.Success() for { select { case <-wait: - done, err := ac.getInitialConfigurationOnce(ctx) - if done { - return err - } - if err != nil { + resp, err := ac.getInitialConfigurationOnce(ctx, csr, key) + if resp != nil { + return ac.recordResponse(resp) + } 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") } wait = ac.waiter.Failed() case <-ctx.Done(): @@ -498,3 +430,164 @@ func (ac *AutoConfig) getInitialConfiguration(ctx context.Context) error { } } } + +// 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 { + if err := ac.updateConfigFromResponse(resp); err != nil { + return err + } + + if err := ac.updateTLSFromResponse(resp); err != nil { + return err + } + + return nil +} + +// updateConfigFromResponse is responsible for generating the JSON compatible with the +// agent/config.Config struct +func (ac *AutoConfig) updateConfigFromResponse(resp *pbautoconf.AutoConfigResponse) error { + // here we want to serialize the translated configuration for use in injecting into the normal + // configuration parsing chain. + conf, err := json.Marshal(translateConfig(resp.Config)) + if err != nil { + return fmt.Errorf("failed to encode auto-config configuration as JSON: %w", err) + } + + ac.autoConfigData = string(conf) + 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 { + return nil + } + + if !ac.config.AutoConfig.Enabled { + return nil + } + + _, err := ac.certMonitor.Start(ctx) + return err +} + +func (ac *AutoConfig) Stop() bool { + if ac.certMonitor == nil { + return false + } + + if !ac.config.AutoConfig.Enabled { + return false + } + + 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) +} diff --git a/agent/auto-config/auto_config_test.go b/agent/auto-config/auto_config_test.go index 254ce9512..4583d803b 100644 --- a/agent/auto-config/auto_config_test.go +++ b/agent/auto-config/auto_config_test.go @@ -2,21 +2,23 @@ package autoconf import ( "context" - "encoding/json" "fmt" "io/ioutil" "net" "os" "path/filepath" + "strings" "testing" "time" + "github.com/gogo/protobuf/types" "github.com/hashicorp/consul/agent/config" + "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/proto/pbautoconf" "github.com/hashicorp/consul/proto/pbconfig" + "github.com/hashicorp/consul/proto/pbconnect" "github.com/hashicorp/consul/sdk/testutil" - "github.com/hashicorp/consul/tlsutil" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" ) @@ -26,7 +28,17 @@ type mockDirectRPC struct { } func (m *mockDirectRPC) RPC(dc string, node string, addr net.Addr, method string, args interface{}, reply interface{}) error { - retValues := m.Called(dc, node, addr, method, args, reply) + 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 { + retValues = m.Called(dc, node, addr, method, args, reply) + } + switch ret := retValues.Get(0).(type) { case error: return ret @@ -38,30 +50,48 @@ func (m *mockDirectRPC) RPC(dc string, node string, addr net.Addr, method string } } +type mockCertMonitor struct { + mock.Mock +} + +func (m *mockCertMonitor) Start(_ context.Context) (<-chan struct{}, error) { + ret := m.Called() + ch := ret.Get(0).(<-chan struct{}) + return ch, ret.Error(1) +} + +func (m *mockCertMonitor) Stop() bool { + return m.Called().Bool(0) +} + +func (m *mockCertMonitor) Update(resp *structs.SignedResponse) error { + var privKey string + // filter out real certificates as we cannot predict their values + if resp != nil && strings.HasPrefix(resp.IssuedCert.PrivateKeyPEM, "-----BEGIN") { + privKey = resp.IssuedCert.PrivateKeyPEM + resp.IssuedCert.PrivateKeyPEM = "" + } + err := m.Called(resp).Error(0) + if privKey != "" { + resp.IssuedCert.PrivateKeyPEM = privKey + } + return err +} + func TestNew(t *testing.T) { type testCase struct { - opts []Option + config Config err string validate func(t *testing.T, ac *AutoConfig) } cases := map[string]testCase{ "no-direct-rpc": { - opts: []Option{ - WithTLSConfigurator(&tlsutil.Configurator{}), - }, err: "must provide a direct RPC delegate", }, - "no-tls-configurator": { - opts: []Option{ - WithDirectRPC(&mockDirectRPC{}), - }, - err: "must provide a TLS configurator", - }, "ok": { - opts: []Option{ - WithTLSConfigurator(&tlsutil.Configurator{}), - WithDirectRPC(&mockDirectRPC{}), + config: Config{ + DirectRPC: &mockDirectRPC{}, }, validate: func(t *testing.T, ac *AutoConfig) { t.Helper() @@ -72,7 +102,7 @@ func TestNew(t *testing.T) { for name, tcase := range cases { t.Run(name, func(t *testing.T) { - ac, err := New(tcase.opts...) + ac, err := New(&tcase.config) if tcase.err != "" { testutil.RequireErrorContains(t, err, tcase.err) } else { @@ -173,7 +203,10 @@ func TestInitialConfiguration_disabled(t *testing.T) { builderOpts.ConfigFiles = append(builderOpts.ConfigFiles, cfgFile) directRPC := mockDirectRPC{} - ac, err := New(WithBuilderOpts(builderOpts), WithTLSConfigurator(&tlsutil.Configurator{}), WithDirectRPC(&directRPC)) + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(&directRPC) + ac, err := New(conf) require.NoError(t, err) require.NotNil(t, ac) @@ -208,7 +241,10 @@ func TestInitialConfiguration_cancelled(t *testing.T) { } directRPC.On("RPC", "dc1", "autoconf", &net.TCPAddr{IP: net.IPv4(127, 0, 0, 1), Port: 8300}, "AutoConfig.InitialConfiguration", &expectedRequest, mock.Anything).Return(fmt.Errorf("injected error")).Times(0) - ac, err := New(WithBuilderOpts(builderOpts), WithTLSConfigurator(&tlsutil.Configurator{}), WithDirectRPC(&directRPC)) + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(&directRPC) + ac, err := New(conf) require.NoError(t, err) require.NotNil(t, ac) @@ -235,26 +271,97 @@ func TestInitialConfiguration_restored(t *testing.T) { // persist an auto config response to the data dir where it is expected persistedFile := filepath.Join(dataDir, autoConfigFileName) - response := &pbconfig.Config{ - PrimaryDatacenter: "primary", + response := &pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + PrimaryDatacenter: "primary", + TLS: &pbconfig.TLS{ + VerifyServerHostname: true, + }, + }, + CARoots: &pbconnect.CARoots{ + ActiveRootID: "active", + TrustDomain: "trust", + Roots: []*pbconnect.CARoot{ + { + ID: "active", + Name: "foo", + SerialNumber: 42, + SigningKeyID: "blarg", + NotBefore: &types.Timestamp{Seconds: 5000, Nanos: 100}, + NotAfter: &types.Timestamp{Seconds: 10000, Nanos: 9009}, + RootCert: "not an actual cert", + Active: true, + }, + }, + }, + Certificate: &pbconnect.IssuedCert{ + SerialNumber: "1234", + CertPEM: "not a cert", + PrivateKeyPEM: "private", + Agent: "foo", + AgentURI: "spiffe://blarg/agent/client/dc/foo/id/foo", + ValidAfter: &types.Timestamp{Seconds: 6000}, + ValidBefore: &types.Timestamp{Seconds: 7000}, + }, + ExtraCACertificates: []string{"blarg"}, } - data, err := json.Marshal(translateConfig(response)) + data, err := pbMarshaler.MarshalToString(response) require.NoError(t, err) - require.NoError(t, ioutil.WriteFile(persistedFile, data, 0600)) + require.NoError(t, ioutil.WriteFile(persistedFile, []byte(data), 0600)) directRPC := mockDirectRPC{} - ac, err := New(WithBuilderOpts(builderOpts), WithTLSConfigurator(&tlsutil.Configurator{}), WithDirectRPC(&directRPC)) + // setup the mock certificate monitor to ensure that the initial state gets + // updated appropriately during config restoration. + certMon := mockCertMonitor{} + certMon.On("Update", &structs.SignedResponse{ + IssuedCert: structs.IssuedCert{ + SerialNumber: "1234", + CertPEM: "not a cert", + PrivateKeyPEM: "private", + Agent: "foo", + AgentURI: "spiffe://blarg/agent/client/dc/foo/id/foo", + ValidAfter: time.Unix(6000, 0), + ValidBefore: time.Unix(7000, 0), + }, + ConnectCARoots: structs.IndexedCARoots{ + ActiveRootID: "active", + TrustDomain: "trust", + Roots: []*structs.CARoot{ + { + ID: "active", + Name: "foo", + SerialNumber: 42, + SigningKeyID: "blarg", + NotBefore: time.Unix(5000, 100), + NotAfter: time.Unix(10000, 9009), + RootCert: "not an actual cert", + Active: true, + // the decoding process doesn't leave this nil + IntermediateCerts: []string{}, + }, + }, + }, + ManualCARoots: []string{"blarg"}, + VerifyServerHostname: true, + }).Return(nil).Once() + + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(&directRPC). + WithCertMonitor(&certMon) + ac, err := New(conf) require.NoError(t, err) require.NotNil(t, ac) cfg, err := ac.InitialConfiguration(context.Background()) - require.NoError(t, err) + require.NoError(t, err, data) require.NotNil(t, cfg) require.Equal(t, "primary", cfg.PrimaryDatacenter) // ensure no RPC was made directRPC.AssertExpectations(t) + certMon.AssertExpectations(t) } func TestInitialConfiguration_success(t *testing.T) { @@ -275,7 +382,36 @@ func TestInitialConfiguration_success(t *testing.T) { require.True(t, ok) resp.Config = &pbconfig.Config{ PrimaryDatacenter: "primary", + TLS: &pbconfig.TLS{ + VerifyServerHostname: true, + }, } + + resp.CARoots = &pbconnect.CARoots{ + ActiveRootID: "active", + TrustDomain: "trust", + Roots: []*pbconnect.CARoot{ + { + ID: "active", + Name: "foo", + SerialNumber: 42, + SigningKeyID: "blarg", + NotBefore: &types.Timestamp{Seconds: 5000, Nanos: 100}, + NotAfter: &types.Timestamp{Seconds: 10000, Nanos: 9009}, + RootCert: "not an actual cert", + Active: true, + }, + }, + } + resp.Certificate = &pbconnect.IssuedCert{ + SerialNumber: "1234", + CertPEM: "not a cert", + Agent: "foo", + AgentURI: "spiffe://blarg/agent/client/dc/foo/id/foo", + ValidAfter: &types.Timestamp{Seconds: 6000}, + ValidBefore: &types.Timestamp{Seconds: 7000}, + } + resp.ExtraCACertificates = []string{"blarg"} } expectedRequest := pbautoconf.AutoConfigRequest{ @@ -293,7 +429,44 @@ func TestInitialConfiguration_success(t *testing.T) { &expectedRequest, &pbautoconf.AutoConfigResponse{}).Return(populateResponse) - ac, err := New(WithBuilderOpts(builderOpts), WithTLSConfigurator(&tlsutil.Configurator{}), WithDirectRPC(&directRPC)) + // setup the mock certificate monitor to ensure that the initial state gets + // updated appropriately during config restoration. + certMon := mockCertMonitor{} + certMon.On("Update", &structs.SignedResponse{ + IssuedCert: structs.IssuedCert{ + SerialNumber: "1234", + CertPEM: "not a cert", + PrivateKeyPEM: "", // the mock + Agent: "foo", + AgentURI: "spiffe://blarg/agent/client/dc/foo/id/foo", + ValidAfter: time.Unix(6000, 0), + ValidBefore: time.Unix(7000, 0), + }, + ConnectCARoots: structs.IndexedCARoots{ + ActiveRootID: "active", + TrustDomain: "trust", + Roots: []*structs.CARoot{ + { + ID: "active", + Name: "foo", + SerialNumber: 42, + SigningKeyID: "blarg", + NotBefore: time.Unix(5000, 100), + NotAfter: time.Unix(10000, 9009), + RootCert: "not an actual cert", + Active: true, + }, + }, + }, + ManualCARoots: []string{"blarg"}, + VerifyServerHostname: true, + }).Return(nil).Once() + + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(&directRPC). + WithCertMonitor(&certMon) + ac, err := New(conf) require.NoError(t, err) require.NotNil(t, ac) @@ -307,6 +480,7 @@ func TestInitialConfiguration_success(t *testing.T) { // ensure no RPC was made directRPC.AssertExpectations(t) + certMon.AssertExpectations(t) } func TestInitialConfiguration_retries(t *testing.T) { @@ -381,7 +555,11 @@ func TestInitialConfiguration_retries(t *testing.T) { &pbautoconf.AutoConfigResponse{}).Return(populateResponse) waiter := lib.NewRetryWaiter(2, 0, 1*time.Millisecond, nil) - ac, err := New(WithBuilderOpts(builderOpts), WithTLSConfigurator(&tlsutil.Configurator{}), WithDirectRPC(&directRPC), WithRetryWaiter(waiter)) + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(&directRPC). + WithRetryWaiter(waiter) + ac, err := New(conf) require.NoError(t, err) require.NotNil(t, ac) @@ -396,3 +574,162 @@ func TestInitialConfiguration_retries(t *testing.T) { // ensure no RPC was made directRPC.AssertExpectations(t) } + +func TestAutoConfig_StartStop(t *testing.T) { + // currently the only thing running for autoconf is just the cert monitor + // so this test only needs to ensure that the cert monitor is started and + // stopped and not that anything with regards to running the cert monitor + // actually work. Those are tested in the cert-monitor package. + + _, configDir, builderOpts := testSetupAutoConf(t) + + cfgFile := filepath.Join(configDir, "test.json") + require.NoError(t, ioutil.WriteFile(cfgFile, []byte(`{ + "auto_config": {"enabled": true, "intro_token": "blarg", "server_addresses": ["198.18.0.1", "198.18.0.2:8398", "198.18.0.3:8399", "127.0.0.1:1234"]}, "verify_outgoing": true + }`), 0600)) + + builderOpts.ConfigFiles = append(builderOpts.ConfigFiles, cfgFile) + directRPC := &mockDirectRPC{} + certMon := &mockCertMonitor{} + + certMon.On("Start").Return((<-chan struct{})(nil), nil).Once() + certMon.On("Stop").Return(true).Once() + + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(directRPC). + WithCertMonitor(certMon) + + ac, err := New(conf) + require.NoError(t, err) + require.NotNil(t, ac) + cfg, err := ac.ReadConfig() + require.NoError(t, err) + ac.config = cfg + + require.NoError(t, ac.Start(context.Background())) + require.True(t, ac.Stop()) + + certMon.AssertExpectations(t) + directRPC.AssertExpectations(t) +} + +func TestFallBackTLS(t *testing.T) { + _, configDir, builderOpts := testSetupAutoConf(t) + + cfgFile := filepath.Join(configDir, "test.json") + require.NoError(t, ioutil.WriteFile(cfgFile, []byte(`{ + "auto_config": {"enabled": true, "intro_token": "blarg", "server_addresses": ["127.0.0.1:8300"]}, "verify_outgoing": true + }`), 0600)) + + builderOpts.ConfigFiles = append(builderOpts.ConfigFiles, cfgFile) + + directRPC := mockDirectRPC{} + + populateResponse := func(val interface{}) { + resp, ok := val.(*pbautoconf.AutoConfigResponse) + require.True(t, ok) + resp.Config = &pbconfig.Config{ + PrimaryDatacenter: "primary", + TLS: &pbconfig.TLS{ + VerifyServerHostname: true, + }, + } + + resp.CARoots = &pbconnect.CARoots{ + ActiveRootID: "active", + TrustDomain: "trust", + Roots: []*pbconnect.CARoot{ + { + ID: "active", + Name: "foo", + SerialNumber: 42, + SigningKeyID: "blarg", + NotBefore: &types.Timestamp{Seconds: 5000, Nanos: 100}, + NotAfter: &types.Timestamp{Seconds: 10000, Nanos: 9009}, + RootCert: "not an actual cert", + Active: true, + }, + }, + } + resp.Certificate = &pbconnect.IssuedCert{ + SerialNumber: "1234", + CertPEM: "not a cert", + Agent: "foo", + AgentURI: "spiffe://blarg/agent/client/dc/foo/id/foo", + ValidAfter: &types.Timestamp{Seconds: 6000}, + ValidBefore: &types.Timestamp{Seconds: 7000}, + } + resp.ExtraCACertificates = []string{"blarg"} + } + + expectedRequest := pbautoconf.AutoConfigRequest{ + Datacenter: "dc1", + Node: "autoconf", + JWT: "blarg", + } + + directRPC.On( + "RPC", + "dc1", + "autoconf", + &net.TCPAddr{IP: net.IPv4(127, 0, 0, 1), Port: 8300}, + "AutoConfig.InitialConfiguration", + &expectedRequest, + &pbautoconf.AutoConfigResponse{}).Return(populateResponse) + + // setup the mock certificate monitor we don't expect it to be used + // as the FallbackTLS method is mainly used by the certificate monitor + // if for some reason it fails to renew the TLS certificate in time. + certMon := mockCertMonitor{} + + conf := new(Config). + WithBuilderOpts(builderOpts). + WithDirectRPC(&directRPC). + WithCertMonitor(&certMon) + ac, err := New(conf) + require.NoError(t, err) + require.NotNil(t, ac) + ac.config, err = ac.ReadConfig() + require.NoError(t, err) + + actual, err := ac.FallbackTLS(context.Background()) + require.NoError(t, err) + expected := &structs.SignedResponse{ + ConnectCARoots: structs.IndexedCARoots{ + ActiveRootID: "active", + TrustDomain: "trust", + Roots: []*structs.CARoot{ + { + ID: "active", + Name: "foo", + SerialNumber: 42, + SigningKeyID: "blarg", + NotBefore: time.Unix(5000, 100), + NotAfter: time.Unix(10000, 9009), + RootCert: "not an actual cert", + Active: true, + }, + }, + }, + IssuedCert: structs.IssuedCert{ + SerialNumber: "1234", + CertPEM: "not a cert", + Agent: "foo", + AgentURI: "spiffe://blarg/agent/client/dc/foo/id/foo", + ValidAfter: time.Unix(6000, 0), + ValidBefore: time.Unix(7000, 0), + }, + ManualCARoots: []string{"blarg"}, + VerifyServerHostname: true, + } + // have to just verify that the private key was put in here but we then + // must zero it out so that the remaining equality check will pass + require.NotEmpty(t, actual.IssuedCert.PrivateKeyPEM) + actual.IssuedCert.PrivateKeyPEM = "" + require.Equal(t, expected, actual) + + // ensure no RPC was made + directRPC.AssertExpectations(t) + certMon.AssertExpectations(t) +} diff --git a/agent/auto-config/builder.go b/agent/auto-config/builder.go new file mode 100644 index 000000000..56feb268a --- /dev/null +++ b/agent/auto-config/builder.go @@ -0,0 +1,30 @@ +package autoconf + +import ( + "github.com/hashicorp/consul/agent/config" +) + +// LoadConfig will build the configuration including the extraHead source injected +// after all other defaults but before any user supplied configuration and the overrides +// source injected as the final source in the configuration parsing chain. +func LoadConfig(builderOpts config.BuilderOpts, extraHead config.Source, overrides ...config.Source) (*config.RuntimeConfig, []string, error) { + b, err := config.NewBuilder(builderOpts) + if err != nil { + return nil, nil, err + } + + if extraHead.Data != "" { + b.Head = append(b.Head, extraHead) + } + + if len(overrides) != 0 { + b.Tail = append(b.Tail, overrides...) + } + + cfg, err := b.BuildAndValidate() + if err != nil { + return nil, nil, err + } + + return &cfg, b.Warnings, nil +} diff --git a/agent/auto-config/config.go b/agent/auto-config/config.go new file mode 100644 index 000000000..9510ba6c9 --- /dev/null +++ b/agent/auto-config/config.go @@ -0,0 +1,115 @@ +package autoconf + +import ( + "context" + "net" + + "github.com/hashicorp/consul/agent/config" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/lib" + "github.com/hashicorp/go-hclog" +) + +// DirectRPC is the interface that needs to be satisifed for AutoConfig to be able to perform +// direct RPCs against individual servers. This will not be used for any ongoing RPCs as once +// the agent gets configured, it can go through the normal RPC means of selecting a available +// server automatically. +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 +} + +// Config contains all the tunables for AutoConfig +type Config struct { + // Logger is any logger that should be utilized. If not provided, + // then no logs will be emitted. + Logger hclog.Logger + + // DirectRPC is the interface to be used by AutoConfig to make the + // AutoConfig.InitialConfiguration RPCs for generating the bootstrap + // configuration. Setting this field is required. + DirectRPC DirectRPC + + // BuilderOpts are any configuration building options that should be + // used when loading the Consul configuration. This is mostly a pass + // through from what the CLI will generate. While this option is + // not strictly required, not setting it will prevent AutoConfig + // from doing anything useful. Enabling AutoConfig requires a + // CLI flag or a config file (also specified by the CLI) flag. + // So without providing the opts its equivalent to using the + // configuration of not specifying anything to the consul agent + // cli. + BuilderOpts config.BuilderOpts + + // 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 + // is trying the RPC once against each configured server addr). The + // waiting implements some backoff to prevent from retrying these RPCs + // to often. This field is not required and if left unset a waiter will + // be used that has a max wait duration of 10 minutes and a randomized + // jitter of 25% of the wait time. Setting this is mainly useful for + // testing purposes to allow testing out the retrying functionality without + // having the test take minutes/hours to complete. + Waiter *lib.RetryWaiter + + // Overrides are a list of configuration sources to append to the tail of + // the config builder. This field is optional and mainly useful for testing + // to override values that would be otherwise not user-settable. + Overrides []config.Source + + // 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 +} + +// WithLogger will cause the created AutoConfig type to use the provided logger +func (c *Config) WithLogger(logger hclog.Logger) *Config { + c.Logger = logger + return c +} + +// WithConnectionPool will cause the created AutoConfig type to use the provided connection pool +func (c *Config) WithDirectRPC(directRPC DirectRPC) *Config { + c.DirectRPC = directRPC + return c +} + +// WithBuilderOpts will cause the created AutoConfig type to use the provided CLI builderOpts +func (c *Config) WithBuilderOpts(builderOpts config.BuilderOpts) *Config { + c.BuilderOpts = builderOpts + return c +} + +// WithRetryWaiter will cause the created AutoConfig type to use the provided retry waiter +func (c *Config) WithRetryWaiter(waiter *lib.RetryWaiter) *Config { + c.Waiter = waiter + return c +} + +// WithOverrides is used to provide a config source to append to the tail sources +// during config building. It is really only useful for testing to tune non-user +// configurable tunables to make various tests converge more quickly than they +// could otherwise. +func (c *Config) WithOverrides(overrides ...config.Source) *Config { + c.Overrides = overrides + return c +} + +// WithCertMonitor is used to provide a certificate monitor to the auto-config. +// This monitor is responsible for renewing the agents TLS certificate and keeping +// the connect CA roots up to date. +func (c *Config) WithCertMonitor(certMonitor CertMonitor) *Config { + c.CertMonitor = certMonitor + return c +} diff --git a/agent/auto-config/config_translate.go b/agent/auto-config/config_translate.go index fa56fcb57..313b37a47 100644 --- a/agent/auto-config/config_translate.go +++ b/agent/auto-config/config_translate.go @@ -1,7 +1,14 @@ package autoconf import ( + "fmt" + + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/proto" + "github.com/hashicorp/consul/proto/pbautoconf" "github.com/hashicorp/consul/proto/pbconfig" + "github.com/hashicorp/consul/proto/pbconnect" + "github.com/mitchellh/mapstructure" ) // translateAgentConfig is meant to take in a proto/pbconfig.Config type @@ -158,3 +165,64 @@ func translateConfig(c *pbconfig.Config) map[string]interface{} { return out } + +func extractSignedResponse(resp *pbautoconf.AutoConfigResponse) (*structs.SignedResponse, error) { + roots, err := translateCARootsToStructs(resp.CARoots) + if err != nil { + return nil, err + } + + cert, err := translateIssuedCertToStructs(resp.Certificate) + if err != nil { + return nil, err + } + + out := &structs.SignedResponse{ + IssuedCert: *cert, + ConnectCARoots: *roots, + ManualCARoots: resp.ExtraCACertificates, + } + + if resp.Config != nil && resp.Config.TLS != nil { + out.VerifyServerHostname = resp.Config.TLS.VerifyServerHostname + } + + return out, err +} + +// translateCARootsToStructs will create a structs.IndexedCARoots object from the corresponding +// protobuf struct. Those structs are intended to be identical so the conversion just uses +// mapstructure to go from one to the other. +func translateCARootsToStructs(in *pbconnect.CARoots) (*structs.IndexedCARoots, error) { + var out structs.IndexedCARoots + if err := mapstructureTranslateToStructs(in, &out); err != nil { + return nil, fmt.Errorf("Failed to re-encode CA Roots: %w", err) + } + + return &out, nil +} + +// translateIssuedCertToStructs will create a structs.IssuedCert object from the corresponding +// protobuf struct. Those structs are intended to be identical so the conversion just uses +// mapstructure to go from one to the other. +func translateIssuedCertToStructs(in *pbconnect.IssuedCert) (*structs.IssuedCert, error) { + var out structs.IssuedCert + if err := mapstructureTranslateToStructs(in, &out); err != nil { + return nil, fmt.Errorf("Failed to re-encode CA Roots: %w", err) + } + + return &out, nil +} + +func mapstructureTranslateToStructs(in interface{}, out interface{}) error { + decoder, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ + DecodeHook: proto.HookPBTimestampToTime, + Result: out, + }) + + if err != nil { + return err + } + + return decoder.Decode(in) +} diff --git a/agent/cert-monitor/cert_monitor.go b/agent/cert-monitor/cert_monitor.go index 44fd3b7f2..b65a490d3 100644 --- a/agent/cert-monitor/cert_monitor.go +++ b/agent/cert-monitor/cert_monitor.go @@ -230,6 +230,7 @@ func (m *CertMonitor) Start(ctx context.Context) (<-chan struct{}, error) { exit := make(chan struct{}) go m.run(ctx, exit) + m.logger.Info("certificate monitor started") return exit, nil } diff --git a/agent/config/builder.go b/agent/config/builder.go index b3c7620f7..4eb3afef1 100644 --- a/agent/config/builder.go +++ b/agent/config/builder.go @@ -1297,6 +1297,10 @@ func (b *Builder) Validate(rt RuntimeConfig) error { return err } + if rt.AutoConfig.Enabled && rt.AutoEncryptTLS { + return fmt.Errorf("both auto_encrypt.tls and auto_config.enabled cannot be set to true.") + } + if err := b.validateAutoConfig(rt); err != nil { return err } diff --git a/agent/config/runtime.go b/agent/config/runtime.go index db3f926a0..55055ffb3 100644 --- a/agent/config/runtime.go +++ b/agent/config/runtime.go @@ -1692,10 +1692,6 @@ func (c *RuntimeConfig) ClientAddress() (unixAddr, httpAddr, httpsAddr string) { } func (c *RuntimeConfig) ConnectCAConfiguration() (*structs.CAConfiguration, error) { - if !c.ConnectEnabled { - return nil, nil - } - ca := &structs.CAConfiguration{ Provider: "consul", Config: map[string]interface{}{ diff --git a/agent/config/runtime_test.go b/agent/config/runtime_test.go index d3a564df2..dc98c074d 100644 --- a/agent/config/runtime_test.go +++ b/agent/config/runtime_test.go @@ -3803,6 +3803,35 @@ func TestConfigFlagsAndEdgecases(t *testing.T) { /////////////////////////////////// // Auto Config related tests + { + desc: "auto config and auto encrypt error", + args: []string{ + `-data-dir=` + dataDir, + }, + hcl: []string{` + auto_config { + enabled = true + intro_token = "blah" + server_addresses = ["198.18.0.1"] + } + auto_encrypt { + tls = true + } + verify_outgoing = true + `}, + json: []string{`{ + "auto_config": { + "enabled": true, + "intro_token": "blah", + "server_addresses": ["198.18.0.1"] + }, + "auto_encrypt": { + "tls": true + }, + "verify_outgoing": true + }`}, + err: "both auto_encrypt.tls and auto_config.enabled cannot be set to true.", + }, { desc: "auto config not allowed for servers", args: []string{ @@ -7470,12 +7499,6 @@ func TestConnectCAConfiguration(t *testing.T) { } cases := map[string]testCase{ - "connect-disabled": { - config: RuntimeConfig{ - ConnectEnabled: false, - }, - expected: nil, - }, "defaults": { config: RuntimeConfig{ ConnectEnabled: true, diff --git a/agent/consul/auto_config_endpoint.go b/agent/consul/auto_config_endpoint.go index 3260157b5..6cbf0a539 100644 --- a/agent/consul/auto_config_endpoint.go +++ b/agent/consul/auto_config_endpoint.go @@ -2,22 +2,31 @@ package consul import ( "context" + "crypto/x509" "encoding/base64" "fmt" "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/proto" + + "github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/consul/authmethod/ssoauth" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/lib/template" "github.com/hashicorp/consul/proto/pbautoconf" - config "github.com/hashicorp/consul/proto/pbconfig" + "github.com/hashicorp/consul/proto/pbconfig" + "github.com/hashicorp/consul/proto/pbconnect" "github.com/hashicorp/consul/tlsutil" bexpr "github.com/hashicorp/go-bexpr" + "github.com/mitchellh/mapstructure" ) type AutoConfigOptions struct { NodeName string SegmentName string + + CSR *x509.CertificateRequest + SpiffeID *connect.SpiffeIDAgent } type AutoConfigAuthorizer interface { @@ -73,16 +82,35 @@ func (a *jwtAuthorizer) Authorize(req *pbautoconf.AutoConfigRequest) (AutoConfig } } - return AutoConfigOptions{ + opts := AutoConfigOptions{ NodeName: req.Node, SegmentName: req.Segment, - }, nil + } + + if req.CSR != "" { + csr, id, err := parseAutoConfigCSR(req.CSR) + if err != nil { + return AutoConfigOptions{}, err + } + + if id.Agent != req.Node { + return AutoConfigOptions{}, fmt.Errorf("Spiffe ID agent name (%s) of the certificate signing request is not for the correct node (%s)", id.Agent, req.Node) + } + + opts.CSR = csr + opts.SpiffeID = id + } + + return opts, nil } type AutoConfigBackend interface { CreateACLToken(template *structs.ACLToken) (*structs.ACLToken, error) DatacenterJoinAddresses(segment string) ([]string, error) ForwardRPC(method string, info structs.RPCInfo, args, reply interface{}) (bool, error) + + GetCARoots() (*structs.IndexedCARoots, error) + SignCertificate(csr *x509.CertificateRequest, id connect.CertURI) (*structs.IssuedCert, error) } // AutoConfig endpoint is used for cluster auto configuration operations @@ -114,15 +142,51 @@ func NewAutoConfig(conf *Config, tlsConfigurator *tlsutil.Configurator, backend // made aware of its certificates are populated. This will only work if connect is enabled and // in some cases only if auto_encrypt is enabled on the servers. This endpoint has the option // to configure auto_encrypt or potentially in the future to generate the certificates inline. -func (ac *AutoConfig) updateTLSCertificatesInConfig(opts AutoConfigOptions, conf *config.Config) error { - conf.AutoEncrypt = &config.AutoEncrypt{TLS: ac.config.AutoEncryptAllowTLS} +func (ac *AutoConfig) updateTLSCertificatesInConfig(opts AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error { + // nothing to be done as we cannot generate certificates + if !ac.config.ConnectEnabled { + return nil + } + + if opts.CSR != nil { + cert, err := ac.backend.SignCertificate(opts.CSR, opts.SpiffeID) + if err != nil { + return fmt.Errorf("Failed to sign CSR: %w", err) + } + + // convert to the protobuf form of the issued certificate + pbcert, err := translateIssuedCertToProtobuf(cert) + if err != nil { + return err + } + resp.Certificate = pbcert + } + + connectRoots, err := ac.backend.GetCARoots() + if err != nil { + return fmt.Errorf("Failed to lookup the CA roots: %w", err) + } + + // convert to the protobuf form of the issued certificate + pbroots, err := translateCARootsToProtobuf(connectRoots) + if err != nil { + return err + } + + resp.CARoots = pbroots + + // get the non-connect CA certs from the TLS Configurator + if ac.tlsConfigurator != nil { + resp.ExtraCACertificates = ac.tlsConfigurator.ManualCAPems() + } + return nil } // updateACLtokensInConfig will configure all of the agents ACL settings and will populate // the configuration with an agent token usable for all default agent operations. -func (ac *AutoConfig) updateACLsInConfig(opts AutoConfigOptions, conf *config.Config) error { - acl := &config.ACL{ +func (ac *AutoConfig) updateACLsInConfig(opts AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error { + acl := &pbconfig.ACL{ Enabled: ac.config.ACLsEnabled, PolicyTTL: ac.config.ACLPolicyTTL.String(), RoleTTL: ac.config.ACLRoleTTL.String(), @@ -153,48 +217,48 @@ func (ac *AutoConfig) updateACLsInConfig(opts AutoConfigOptions, conf *config.Co return fmt.Errorf("Failed to generate an ACL token for node %q - %w", opts.NodeName, err) } - acl.Tokens = &config.ACLTokens{Agent: token.SecretID} + acl.Tokens = &pbconfig.ACLTokens{Agent: token.SecretID} } - conf.ACL = acl + resp.Config.ACL = acl return nil } // updateJoinAddressesInConfig determines the correct gossip endpoints that clients should // be connecting to for joining the cluster based on the segment given in the opts parameter. -func (ac *AutoConfig) updateJoinAddressesInConfig(opts AutoConfigOptions, conf *config.Config) error { +func (ac *AutoConfig) updateJoinAddressesInConfig(opts AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error { joinAddrs, err := ac.backend.DatacenterJoinAddresses(opts.SegmentName) if err != nil { return err } - if conf.Gossip == nil { - conf.Gossip = &config.Gossip{} + if resp.Config.Gossip == nil { + resp.Config.Gossip = &pbconfig.Gossip{} } - conf.Gossip.RetryJoinLAN = joinAddrs + resp.Config.Gossip.RetryJoinLAN = joinAddrs return nil } // updateGossipEncryptionInConfig will populate the gossip encryption configuration settings -func (ac *AutoConfig) updateGossipEncryptionInConfig(_ AutoConfigOptions, conf *config.Config) error { +func (ac *AutoConfig) updateGossipEncryptionInConfig(_ AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error { // Add gossip encryption settings if there is any key loaded memberlistConfig := ac.config.SerfLANConfig.MemberlistConfig if lanKeyring := memberlistConfig.Keyring; lanKeyring != nil { - if conf.Gossip == nil { - conf.Gossip = &config.Gossip{} + if resp.Config.Gossip == nil { + resp.Config.Gossip = &pbconfig.Gossip{} } - if conf.Gossip.Encryption == nil { - conf.Gossip.Encryption = &config.GossipEncryption{} + if resp.Config.Gossip.Encryption == nil { + resp.Config.Gossip.Encryption = &pbconfig.GossipEncryption{} } pk := lanKeyring.GetPrimaryKey() if len(pk) > 0 { - conf.Gossip.Encryption.Key = base64.StdEncoding.EncodeToString(pk) + resp.Config.Gossip.Encryption.Key = base64.StdEncoding.EncodeToString(pk) } - conf.Gossip.Encryption.VerifyIncoming = memberlistConfig.GossipVerifyIncoming - conf.Gossip.Encryption.VerifyOutgoing = memberlistConfig.GossipVerifyOutgoing + resp.Config.Gossip.Encryption.VerifyIncoming = memberlistConfig.GossipVerifyIncoming + resp.Config.Gossip.Encryption.VerifyOutgoing = memberlistConfig.GossipVerifyOutgoing } return nil @@ -202,44 +266,44 @@ func (ac *AutoConfig) updateGossipEncryptionInConfig(_ AutoConfigOptions, conf * // updateTLSSettingsInConfig will populate the TLS configuration settings but will not // populate leaf or ca certficiates. -func (ac *AutoConfig) updateTLSSettingsInConfig(_ AutoConfigOptions, conf *config.Config) error { +func (ac *AutoConfig) updateTLSSettingsInConfig(_ AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error { if ac.tlsConfigurator == nil { // TLS is not enabled? return nil } // add in TLS configuration - if conf.TLS == nil { - conf.TLS = &config.TLS{} + if resp.Config.TLS == nil { + resp.Config.TLS = &pbconfig.TLS{} } - conf.TLS.VerifyServerHostname = ac.tlsConfigurator.VerifyServerHostname() + resp.Config.TLS.VerifyServerHostname = ac.tlsConfigurator.VerifyServerHostname() base := ac.tlsConfigurator.Base() - conf.TLS.VerifyOutgoing = base.VerifyOutgoing - conf.TLS.MinVersion = base.TLSMinVersion - conf.TLS.PreferServerCipherSuites = base.PreferServerCipherSuites + resp.Config.TLS.VerifyOutgoing = base.VerifyOutgoing + resp.Config.TLS.MinVersion = base.TLSMinVersion + resp.Config.TLS.PreferServerCipherSuites = base.PreferServerCipherSuites var err error - conf.TLS.CipherSuites, err = tlsutil.CipherString(base.CipherSuites) + resp.Config.TLS.CipherSuites, err = tlsutil.CipherString(base.CipherSuites) return err } // baseConfig will populate the configuration with some base settings such as the // datacenter names, node name etc. -func (ac *AutoConfig) baseConfig(opts AutoConfigOptions, conf *config.Config) error { +func (ac *AutoConfig) baseConfig(opts AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error { if opts.NodeName == "" { return fmt.Errorf("Cannot generate auto config response without a node name") } - conf.Datacenter = ac.config.Datacenter - conf.PrimaryDatacenter = ac.config.PrimaryDatacenter - conf.NodeName = opts.NodeName - conf.SegmentName = opts.SegmentName + resp.Config.Datacenter = ac.config.Datacenter + resp.Config.PrimaryDatacenter = ac.config.PrimaryDatacenter + resp.Config.NodeName = opts.NodeName + resp.Config.SegmentName = opts.SegmentName return nil } -type autoConfigUpdater func(c *AutoConfig, opts AutoConfigOptions, conf *config.Config) error +type autoConfigUpdater func(c *AutoConfig, opts AutoConfigOptions, resp *pbautoconf.AutoConfigResponse) error var ( // variable holding the list of config updating functions to execute when generating @@ -290,15 +354,71 @@ func (ac *AutoConfig) InitialConfiguration(req *pbautoconf.AutoConfigRequest, re return err } - conf := &config.Config{} + resp.Config = &pbconfig.Config{} // update all the configurations for _, configFn := range autoConfigUpdaters { - if err := configFn(ac, opts, conf); err != nil { + if err := configFn(ac, opts, resp); err != nil { return err } } - resp.Config = conf return nil } + +func parseAutoConfigCSR(csr string) (*x509.CertificateRequest, *connect.SpiffeIDAgent, error) { + // Parse the CSR string into the x509 CertificateRequest struct + x509CSR, err := connect.ParseCSR(csr) + if err != nil { + return nil, nil, fmt.Errorf("Failed to parse CSR: %w", err) + } + + // ensure that a URI SAN is present + if len(x509CSR.URIs) < 1 { + return nil, nil, fmt.Errorf("CSR didn't include any URI SANs") + } + + // Parse the SPIFFE ID + spiffeID, err := connect.ParseCertURI(x509CSR.URIs[0]) + if err != nil { + return nil, nil, fmt.Errorf("Failed to parse the SPIFFE URI: %w", err) + } + + agentID, isAgent := spiffeID.(*connect.SpiffeIDAgent) + if !isAgent { + return nil, nil, fmt.Errorf("SPIFFE ID is not an Agent ID") + } + + return x509CSR, agentID, nil +} + +func translateCARootsToProtobuf(in *structs.IndexedCARoots) (*pbconnect.CARoots, error) { + var out pbconnect.CARoots + if err := mapstructureTranslateToProtobuf(in, &out); err != nil { + return nil, fmt.Errorf("Failed to re-encode CA Roots: %w", err) + } + + return &out, nil +} + +func translateIssuedCertToProtobuf(in *structs.IssuedCert) (*pbconnect.IssuedCert, error) { + var out pbconnect.IssuedCert + if err := mapstructureTranslateToProtobuf(in, &out); err != nil { + return nil, fmt.Errorf("Failed to re-encode CA Roots: %w", err) + } + + return &out, nil +} + +func mapstructureTranslateToProtobuf(in interface{}, out interface{}) error { + decoder, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ + DecodeHook: proto.HookTimeToPBTimestamp, + Result: out, + }) + + if err != nil { + return err + } + + return decoder.Decode(in) +} diff --git a/agent/consul/auto_config_endpoint_test.go b/agent/consul/auto_config_endpoint_test.go index 927474eb6..49c386b21 100644 --- a/agent/consul/auto_config_endpoint_test.go +++ b/agent/consul/auto_config_endpoint_test.go @@ -1,6 +1,7 @@ package consul import ( + "crypto/x509" "encoding/base64" "fmt" "io/ioutil" @@ -11,6 +12,7 @@ import ( "testing" "time" + "github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/internal/go-sso/oidcauth/oidcauthtest" "github.com/hashicorp/consul/proto/pbautoconf" @@ -48,6 +50,18 @@ func (m *mockAutoConfigBackend) ForwardRPC(method string, info structs.RPCInfo, return ret.Bool(0), ret.Error(1) } +func (m *mockAutoConfigBackend) GetCARoots() (*structs.IndexedCARoots, error) { + ret := m.Called() + roots, _ := ret.Get(0).(*structs.IndexedCARoots) + return roots, ret.Error(1) +} + +func (m *mockAutoConfigBackend) SignCertificate(csr *x509.CertificateRequest, id connect.CertURI) (*structs.IssuedCert, error) { + ret := m.Called(csr, id) + cert, _ := ret.Get(0).(*structs.IssuedCert) + return cert, ret.Error(1) +} + func testJWTStandardClaims() jwt.Claims { now := time.Now() @@ -79,13 +93,6 @@ func signJWTWithStandardClaims(t *testing.T, privKey string, claims interface{}) // * Each of the individual config generation functions. These can be unit tested separately and should NOT // require running test servers func TestAutoConfigInitialConfiguration(t *testing.T) { - type testCase struct { - request pbautoconf.AutoConfigRequest - expected pbautoconf.AutoConfigResponse - patchResponse func(t *testing.T, srv *Server, resp *pbautoconf.AutoConfigResponse) - err string - } - gossipKey := make([]byte, 32) // this is not cryptographic randomness and is not secure but for the sake of this test its all we need. n, err := rand.Read(gossipKey) @@ -105,85 +112,21 @@ func TestAutoConfigInitialConfiguration(t *testing.T) { _, altpriv, err := oidcauthtest.GenerateKey() require.NoError(t, err) - cases := map[string]testCase{ - "wrong-datacenter": { - request: pbautoconf.AutoConfigRequest{ - Datacenter: "no-such-dc", - }, - err: `invalid datacenter "no-such-dc" - agent auto configuration cannot target a remote datacenter`, - }, - "unverifiable": { - request: pbautoconf.AutoConfigRequest{ - Node: "test-node", - // this is signed using an incorrect private key - JWT: signJWTWithStandardClaims(t, altpriv, map[string]interface{}{"consul_node_name": "test-node"}), - }, - err: "Permission denied: Failed JWT authorization: no known key successfully validated the token signature", - }, - "claim-assertion-failed": { - request: pbautoconf.AutoConfigRequest{ - Node: "test-node", - JWT: signJWTWithStandardClaims(t, priv, map[string]interface{}{"wrong_claim": "test-node"}), - }, - err: "Permission denied: Failed JWT claim assertion", - }, - "good": { - request: pbautoconf.AutoConfigRequest{ - Node: "test-node", - JWT: signJWTWithStandardClaims(t, priv, map[string]interface{}{"consul_node_name": "test-node"}), - }, - expected: pbautoconf.AutoConfigResponse{ - Config: &pbconfig.Config{ - Datacenter: "dc1", - PrimaryDatacenter: "dc1", - NodeName: "test-node", - AutoEncrypt: &pbconfig.AutoEncrypt{ - TLS: true, - }, - ACL: &pbconfig.ACL{ - Enabled: true, - PolicyTTL: "30s", - TokenTTL: "30s", - RoleTTL: "30s", - DisabledTTL: "0s", - DownPolicy: "extend-cache", - DefaultPolicy: "deny", - Tokens: &pbconfig.ACLTokens{ - Agent: "patched-secret", - }, - }, - Gossip: &pbconfig.Gossip{ - Encryption: &pbconfig.GossipEncryption{ - Key: gossipKeyEncoded, - VerifyIncoming: true, - VerifyOutgoing: true, - }, - }, - TLS: &pbconfig.TLS{ - VerifyOutgoing: true, - VerifyServerHostname: true, - MinVersion: "tls12", - PreferServerCipherSuites: true, - }, - }, - }, - patchResponse: func(t *testing.T, srv *Server, resp *pbautoconf.AutoConfigResponse) { - // we are expecting an ACL token but cannot check anything for equality - // so here we check that it was set and overwrite it - require.NotNil(t, resp.Config) - require.NotNil(t, resp.Config.ACL) - require.NotNil(t, resp.Config.ACL.Tokens) - require.NotEmpty(t, resp.Config.ACL.Tokens.Agent) - resp.Config.ACL.Tokens.Agent = "patched-secret" - - // we don't know the expected join address until we start up the test server - joinAddr := &net.TCPAddr{IP: net.IPv4(127, 0, 0, 1), Port: srv.config.SerfLANConfig.MemberlistConfig.AdvertisePort} - require.NotNil(t, resp.Config.Gossip) - require.Equal(t, []string{joinAddr.String()}, resp.Config.Gossip.RetryJoinLAN) - resp.Config.Gossip.RetryJoinLAN = nil - }, - }, + // this CSR is what gets sent in the request + csrID := connect.SpiffeIDAgent{ + Host: "dummy.trustdomain", + Agent: "test-node", + Datacenter: "dc1", } + csr, _ := connect.TestCSR(t, &csrID) + + altCSRID := connect.SpiffeIDAgent{ + Host: "dummy.trustdomain", + Agent: "alt", + Datacenter: "dc1", + } + + altCSR, _ := connect.TestCSR(t, &altCSRID) _, s, _ := testACLServerWithConfig(t, func(c *Config) { c.Domain = "consul" @@ -248,6 +191,126 @@ func TestAutoConfigInitialConfiguration(t *testing.T) { waitForLeaderEstablishment(t, s) + roots, err := s.GetCARoots() + require.NoError(t, err) + + pbroots, err := translateCARootsToProtobuf(roots) + require.NoError(t, err) + + joinAddr := &net.TCPAddr{IP: net.IPv4(127, 0, 0, 1), Port: s.config.SerfLANConfig.MemberlistConfig.AdvertisePort} + + // ------------------------------------------------------------------------- + // Common test setup is now complete + // ------------------------------------------------------------------------- + + type testCase struct { + request pbautoconf.AutoConfigRequest + expected pbautoconf.AutoConfigResponse + patchResponse func(t *testing.T, srv *Server, resp *pbautoconf.AutoConfigResponse) + err string + } + + cases := map[string]testCase{ + "wrong-datacenter": { + request: pbautoconf.AutoConfigRequest{ + Datacenter: "no-such-dc", + }, + err: `invalid datacenter "no-such-dc" - agent auto configuration cannot target a remote datacenter`, + }, + "unverifiable": { + request: pbautoconf.AutoConfigRequest{ + Node: "test-node", + // this is signed using an incorrect private key + JWT: signJWTWithStandardClaims(t, altpriv, map[string]interface{}{"consul_node_name": "test-node"}), + }, + err: "Permission denied: Failed JWT authorization: no known key successfully validated the token signature", + }, + "claim-assertion-failed": { + request: pbautoconf.AutoConfigRequest{ + Node: "test-node", + JWT: signJWTWithStandardClaims(t, priv, map[string]interface{}{"wrong_claim": "test-node"}), + }, + err: "Permission denied: Failed JWT claim assertion", + }, + "bad-csr-id": { + request: pbautoconf.AutoConfigRequest{ + Node: "test-node", + JWT: signJWTWithStandardClaims(t, priv, map[string]interface{}{"consul_node_name": "test-node"}), + CSR: altCSR, + }, + err: "Spiffe ID agent name (alt) of the certificate signing request is not for the correct node (test-node)", + }, + "good": { + request: pbautoconf.AutoConfigRequest{ + Node: "test-node", + JWT: signJWTWithStandardClaims(t, priv, map[string]interface{}{"consul_node_name": "test-node"}), + CSR: csr, + }, + expected: pbautoconf.AutoConfigResponse{ + CARoots: pbroots, + ExtraCACertificates: []string{cacert}, + Config: &pbconfig.Config{ + Datacenter: "dc1", + PrimaryDatacenter: "dc1", + NodeName: "test-node", + ACL: &pbconfig.ACL{ + Enabled: true, + PolicyTTL: "30s", + TokenTTL: "30s", + RoleTTL: "30s", + DisabledTTL: "0s", + DownPolicy: "extend-cache", + DefaultPolicy: "deny", + Tokens: &pbconfig.ACLTokens{ + Agent: "patched-secret", + }, + }, + Gossip: &pbconfig.Gossip{ + Encryption: &pbconfig.GossipEncryption{ + Key: gossipKeyEncoded, + VerifyIncoming: true, + VerifyOutgoing: true, + }, + RetryJoinLAN: []string{joinAddr.String()}, + }, + TLS: &pbconfig.TLS{ + VerifyOutgoing: true, + VerifyServerHostname: true, + MinVersion: "tls12", + PreferServerCipherSuites: true, + }, + }, + }, + patchResponse: func(t *testing.T, srv *Server, resp *pbautoconf.AutoConfigResponse) { + // we are expecting an ACL token but cannot check anything for equality + // so here we check that it was set and overwrite it + require.NotNil(t, resp.Config) + require.NotNil(t, resp.Config.ACL) + require.NotNil(t, resp.Config.ACL.Tokens) + require.NotEmpty(t, resp.Config.ACL.Tokens.Agent) + resp.Config.ACL.Tokens.Agent = "patched-secret" + + require.NotNil(t, resp.Certificate) + require.NotEmpty(t, resp.Certificate.SerialNumber) + require.NotEmpty(t, resp.Certificate.CertPEM) + require.Empty(t, resp.Certificate.Service) + require.Empty(t, resp.Certificate.ServiceURI) + require.Equal(t, "test-node", resp.Certificate.Agent) + + expectedID := connect.SpiffeIDAgent{ + Host: roots.TrustDomain, + Agent: "test-node", + Datacenter: "dc1", + } + + require.Equal(t, expectedID.URI().String(), resp.Certificate.AgentURI) + + // nil this out so we don't check it for equality + resp.Certificate = nil + }, + }, + } + for testName, tcase := range cases { t.Run(testName, func(t *testing.T) { var reply pbautoconf.AutoConfigResponse @@ -269,7 +332,7 @@ func TestAutoConfig_baseConfig(t *testing.T) { type testCase struct { serverConfig Config opts AutoConfigOptions - expected pbconfig.Config + expected pbautoconf.AutoConfigResponse err string } @@ -283,11 +346,13 @@ func TestAutoConfig_baseConfig(t *testing.T) { NodeName: "lBdc0lsH", SegmentName: "HZiwlWpi", }, - expected: pbconfig.Config{ - Datacenter: "oSWzfhnU", - PrimaryDatacenter: "53XO9mx4", - NodeName: "lBdc0lsH", - SegmentName: "HZiwlWpi", + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + Datacenter: "oSWzfhnU", + PrimaryDatacenter: "53XO9mx4", + NodeName: "lBdc0lsH", + SegmentName: "HZiwlWpi", + }, }, }, "no-node-name": { @@ -305,7 +370,7 @@ func TestAutoConfig_baseConfig(t *testing.T) { config: &tcase.serverConfig, } - var actual pbconfig.Config + actual := pbautoconf.AutoConfigResponse{Config: &pbconfig.Config{}} err := ac.baseConfig(tcase.opts, &actual) if tcase.err == "" { require.NoError(t, err) @@ -317,6 +382,13 @@ func TestAutoConfig_baseConfig(t *testing.T) { } } +func parseCiphers(t *testing.T, cipherStr string) []uint16 { + t.Helper() + ciphers, err := tlsutil.ParseCiphers(cipherStr) + require.NoError(t, err) + return ciphers +} + func TestAutoConfig_updateTLSSettingsInConfig(t *testing.T) { _, _, cacert, err := testTLSCertificates("server.dc1.consul") require.NoError(t, err) @@ -328,16 +400,9 @@ func TestAutoConfig_updateTLSSettingsInConfig(t *testing.T) { err = ioutil.WriteFile(cafile, []byte(cacert), 0600) require.NoError(t, err) - parseCiphers := func(t *testing.T, cipherStr string) []uint16 { - t.Helper() - ciphers, err := tlsutil.ParseCiphers(cipherStr) - require.NoError(t, err) - return ciphers - } - type testCase struct { tlsConfig tlsutil.Config - expected pbconfig.Config + expected pbautoconf.AutoConfigResponse } cases := map[string]testCase{ @@ -350,13 +415,15 @@ func TestAutoConfig_updateTLSSettingsInConfig(t *testing.T) { CAFile: cafile, CipherSuites: parseCiphers(t, "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"), }, - expected: pbconfig.Config{ - TLS: &pbconfig.TLS{ - VerifyOutgoing: true, - VerifyServerHostname: true, - MinVersion: "tls12", - PreferServerCipherSuites: true, - CipherSuites: "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + TLS: &pbconfig.TLS{ + VerifyOutgoing: true, + VerifyServerHostname: true, + MinVersion: "tls12", + PreferServerCipherSuites: true, + CipherSuites: "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", + }, }, }, }, @@ -369,13 +436,15 @@ func TestAutoConfig_updateTLSSettingsInConfig(t *testing.T) { CAFile: cafile, CipherSuites: parseCiphers(t, "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"), }, - expected: pbconfig.Config{ - TLS: &pbconfig.TLS{ - VerifyOutgoing: true, - VerifyServerHostname: false, - MinVersion: "tls10", - PreferServerCipherSuites: false, - CipherSuites: "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + TLS: &pbconfig.TLS{ + VerifyOutgoing: true, + VerifyServerHostname: false, + MinVersion: "tls10", + PreferServerCipherSuites: false, + CipherSuites: "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", + }, }, }, }, @@ -391,7 +460,7 @@ func TestAutoConfig_updateTLSSettingsInConfig(t *testing.T) { tlsConfigurator: configurator, } - var actual pbconfig.Config + actual := pbautoconf.AutoConfigResponse{Config: &pbconfig.Config{}} err = ac.updateTLSSettingsInConfig(AutoConfigOptions{}, &actual) require.NoError(t, err) require.Equal(t, tcase.expected, actual) @@ -402,7 +471,7 @@ func TestAutoConfig_updateTLSSettingsInConfig(t *testing.T) { func TestAutoConfig_updateGossipEncryptionInConfig(t *testing.T) { type testCase struct { conf memberlist.Config - expected pbconfig.Config + expected pbautoconf.AutoConfigResponse } gossipKey := make([]byte, 32) @@ -422,12 +491,14 @@ func TestAutoConfig_updateGossipEncryptionInConfig(t *testing.T) { GossipVerifyIncoming: true, GossipVerifyOutgoing: true, }, - expected: pbconfig.Config{ - Gossip: &pbconfig.Gossip{ - Encryption: &pbconfig.GossipEncryption{ - Key: gossipKeyEncoded, - VerifyIncoming: true, - VerifyOutgoing: true, + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + Gossip: &pbconfig.Gossip{ + Encryption: &pbconfig.GossipEncryption{ + Key: gossipKeyEncoded, + VerifyIncoming: true, + VerifyOutgoing: true, + }, }, }, }, @@ -438,12 +509,14 @@ func TestAutoConfig_updateGossipEncryptionInConfig(t *testing.T) { GossipVerifyIncoming: false, GossipVerifyOutgoing: false, }, - expected: pbconfig.Config{ - Gossip: &pbconfig.Gossip{ - Encryption: &pbconfig.GossipEncryption{ - Key: gossipKeyEncoded, - VerifyIncoming: false, - VerifyOutgoing: false, + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + Gossip: &pbconfig.Gossip{ + Encryption: &pbconfig.GossipEncryption{ + Key: gossipKeyEncoded, + VerifyIncoming: false, + VerifyOutgoing: false, + }, }, }, }, @@ -451,6 +524,9 @@ func TestAutoConfig_updateGossipEncryptionInConfig(t *testing.T) { "encryption-disabled": { // zero values all around - if no keyring is configured then the gossip // encryption settings should not be set. + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{}, + }, }, } @@ -463,7 +539,7 @@ func TestAutoConfig_updateGossipEncryptionInConfig(t *testing.T) { config: cfg, } - var actual pbconfig.Config + actual := pbautoconf.AutoConfigResponse{Config: &pbconfig.Config{}} err := ac.updateGossipEncryptionInConfig(AutoConfigOptions{}, &actual) require.NoError(t, err) require.Equal(t, tcase.expected, actual) @@ -472,40 +548,149 @@ func TestAutoConfig_updateGossipEncryptionInConfig(t *testing.T) { } func TestAutoConfig_updateTLSCertificatesInConfig(t *testing.T) { + now := time.Now() + later := now.Add(time.Hour) + + // Generate a Test CA + ca := connect.TestCA(t, nil) + + // roots will be returned by the mock backend + roots := structs.IndexedCARoots{ + ActiveRootID: ca.ID, + TrustDomain: connect.TestClusterID + ".consul", + Roots: []*structs.CARoot{ + ca, + }, + } + + // this CSR is what gets put into the opts for the + // function to look at an process + csrID := connect.SpiffeIDAgent{ + Host: roots.TrustDomain, + Agent: "test", + Datacenter: "dc1", + } + csrStr, _ := connect.TestCSR(t, &csrID) + + csr, err := connect.ParseCSR(csrStr) + require.NoError(t, err) + + // fake certificate response for the backend + fakeCert := structs.IssuedCert{ + SerialNumber: "1", + CertPEM: "not-currently-decoded", + ValidAfter: now, + ValidBefore: later, + EnterpriseMeta: *structs.DefaultEnterpriseMeta(), + RaftIndex: structs.RaftIndex{ + ModifyIndex: 10, + CreateIndex: 10, + }, + } + + // translate the fake cert to the protobuf equivalent + // for embedding in expected results + pbcert, err := translateIssuedCertToProtobuf(&fakeCert) + require.NoError(t, err) + + // generate a CA certificate to use for specifying non-Connect + // certificates which come back differently in the response + _, _, cacert, err := testTLSCertificates("server.dc1.consul") + require.NoError(t, err) + + // write out that ca cert to disk - it is unfortunate that + // this is necessary but creation of the tlsutil.Configurator + // will error if it cannot load the CA certificate from disk + dir := testutil.TempDir(t, "auto-config-tls-certificate") + t.Cleanup(func() { os.RemoveAll(dir) }) + + cafile := path.Join(dir, "cacert.pem") + err = ioutil.WriteFile(cafile, []byte(cacert), 0600) + require.NoError(t, err) + + // translate the roots response to protobuf to be embedded + // into the expected results + pbroots, err := translateCARootsToProtobuf(&roots) + require.NoError(t, err) + type testCase struct { serverConfig Config - expected pbconfig.Config + tlsConfig tlsutil.Config + + opts AutoConfigOptions + expected pbautoconf.AutoConfigResponse } cases := map[string]testCase{ - "auto_encrypt-enabled": { + "no-csr": { serverConfig: Config{ - ConnectEnabled: true, - AutoEncryptAllowTLS: true, + ConnectEnabled: true, }, - expected: pbconfig.Config{ - AutoEncrypt: &pbconfig.AutoEncrypt{TLS: true}, + tlsConfig: tlsutil.Config{ + VerifyOutgoing: true, + VerifyServerHostname: true, + TLSMinVersion: "tls12", + PreferServerCipherSuites: true, + CAFile: cafile, + CipherSuites: parseCiphers(t, "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"), + }, + expected: pbautoconf.AutoConfigResponse{ + CARoots: pbroots, + ExtraCACertificates: []string{cacert}, + Config: &pbconfig.Config{}, }, }, - "auto_encrypt-disabled": { + "signed-certificate": { serverConfig: Config{ - ConnectEnabled: true, - AutoEncryptAllowTLS: false, + ConnectEnabled: true, }, - expected: pbconfig.Config{ - AutoEncrypt: &pbconfig.AutoEncrypt{TLS: false}, + tlsConfig: tlsutil.Config{ + VerifyOutgoing: true, + VerifyServerHostname: true, + TLSMinVersion: "tls12", + PreferServerCipherSuites: true, + CAFile: cafile, + CipherSuites: parseCiphers(t, "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"), + }, + opts: AutoConfigOptions{ + NodeName: "test", + CSR: csr, + SpiffeID: &csrID, + }, + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{}, + CARoots: pbroots, + ExtraCACertificates: []string{cacert}, + Certificate: pbcert, + }, + }, + "connect-disabled": { + serverConfig: Config{ + ConnectEnabled: false, + }, + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{}, }, }, } for name, tcase := range cases { t.Run(name, func(t *testing.T) { + backend := &mockAutoConfigBackend{} + backend.On("GetCARoots").Return(&roots, nil) + backend.On("SignCertificate", tcase.opts.CSR, tcase.opts.SpiffeID).Return(&fakeCert, nil) + + tlsConfigurator, err := tlsutil.NewConfigurator(tcase.tlsConfig, testutil.Logger(t)) + require.NoError(t, err) + ac := AutoConfig{ - config: &tcase.serverConfig, + config: &tcase.serverConfig, + tlsConfigurator: tlsConfigurator, + backend: backend, } - var actual pbconfig.Config - err := ac.updateTLSCertificatesInConfig(AutoConfigOptions{}, &actual) + actual := pbautoconf.AutoConfigResponse{Config: &pbconfig.Config{}} + err = ac.updateTLSCertificatesInConfig(tcase.opts, &actual) require.NoError(t, err) require.Equal(t, tcase.expected, actual) }) @@ -515,7 +700,7 @@ func TestAutoConfig_updateTLSCertificatesInConfig(t *testing.T) { func TestAutoConfig_updateACLsInConfig(t *testing.T) { type testCase struct { config Config - expected pbconfig.Config + expected pbautoconf.AutoConfigResponse expectACLToken bool err error } @@ -542,18 +727,20 @@ func TestAutoConfig_updateACLsInConfig(t *testing.T) { ACLEnableKeyListPolicy: true, }, expectACLToken: true, - expected: pbconfig.Config{ - ACL: &pbconfig.ACL{ - Enabled: true, - PolicyTTL: "7s", - RoleTTL: "10s", - TokenTTL: "12s", - DisabledTTL: "31s", - DownPolicy: "deny", - DefaultPolicy: "allow", - EnableKeyListPolicy: true, - Tokens: &pbconfig.ACLTokens{ - Agent: tokenSecret, + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + ACL: &pbconfig.ACL{ + Enabled: true, + PolicyTTL: "7s", + RoleTTL: "10s", + TokenTTL: "12s", + DisabledTTL: "31s", + DownPolicy: "deny", + DefaultPolicy: "allow", + EnableKeyListPolicy: true, + Tokens: &pbconfig.ACLTokens{ + Agent: tokenSecret, + }, }, }, }, @@ -572,16 +759,18 @@ func TestAutoConfig_updateACLsInConfig(t *testing.T) { ACLEnableKeyListPolicy: true, }, expectACLToken: false, - expected: pbconfig.Config{ - ACL: &pbconfig.ACL{ - Enabled: false, - PolicyTTL: "7s", - RoleTTL: "10s", - TokenTTL: "12s", - DisabledTTL: "31s", - DownPolicy: "deny", - DefaultPolicy: "allow", - EnableKeyListPolicy: true, + expected: pbautoconf.AutoConfigResponse{ + Config: &pbconfig.Config{ + ACL: &pbconfig.ACL{ + Enabled: false, + PolicyTTL: "7s", + RoleTTL: "10s", + TokenTTL: "12s", + DisabledTTL: "31s", + DownPolicy: "deny", + DefaultPolicy: "allow", + EnableKeyListPolicy: true, + }, }, }, }, @@ -630,7 +819,7 @@ func TestAutoConfig_updateACLsInConfig(t *testing.T) { ac := AutoConfig{config: &tcase.config, backend: backend} - var actual pbconfig.Config + actual := pbautoconf.AutoConfigResponse{Config: &pbconfig.Config{}} err := ac.updateACLsInConfig(AutoConfigOptions{NodeName: "something"}, &actual) if tcase.err != nil { testutil.RequireErrorContains(t, err, tcase.err.Error()) @@ -651,12 +840,12 @@ func TestAutoConfig_updateJoinAddressesInConfig(t *testing.T) { ac := AutoConfig{backend: backend} - var actual pbconfig.Config + actual := pbautoconf.AutoConfigResponse{Config: &pbconfig.Config{}} err := ac.updateJoinAddressesInConfig(AutoConfigOptions{}, &actual) require.NoError(t, err) - require.NotNil(t, actual.Gossip) - require.ElementsMatch(t, addrs, actual.Gossip.RetryJoinLAN) + require.NotNil(t, actual.Config.Gossip) + require.ElementsMatch(t, addrs, actual.Config.Gossip.RetryJoinLAN) backend.AssertExpectations(t) } diff --git a/agent/consul/server.go b/agent/consul/server.go index d55f306b9..f6263a9ef 100644 --- a/agent/consul/server.go +++ b/agent/consul/server.go @@ -1,6 +1,7 @@ package consul import ( + "context" "errors" "fmt" "io" @@ -523,8 +524,8 @@ func NewServerWithOptions(config *Config, options ...ConsulOption) (*Server, err return nil, fmt.Errorf("Failed to start Raft: %v", err) } - if s.config.ConnectEnabled && s.config.AutoEncryptAllowTLS { - go s.trackAutoEncryptCARoots() + if s.config.ConnectEnabled && (s.config.AutoEncryptAllowTLS || s.config.AutoConfigAuthzEnabled) { + go s.connectCARootsMonitor(&lib.StopChannelContext{StopCh: s.shutdownCh}) } if s.gatewayLocator != nil { @@ -632,18 +633,11 @@ func NewServerWithOptions(config *Config, options ...ConsulOption) (*Server, err return s, nil } -func (s *Server) trackAutoEncryptCARoots() { +func (s *Server) connectCARootsMonitor(ctx context.Context) { for { - select { - case <-s.shutdownCh: - s.logger.Debug("shutting down trackAutoEncryptCARoots because shutdown") - return - default: - } ws := memdb.NewWatchSet() state := s.fsm.State() ws.Add(state.AbandonCh()) - ws.Add(s.shutdownCh) _, cas, err := state.CARoots(ws) if err != nil { s.logger.Error("Failed to watch AutoEncrypt CARoot", "error", err) @@ -656,7 +650,11 @@ func (s *Server) trackAutoEncryptCARoots() { if err := s.tlsConfigurator.UpdateAutoEncryptCA(caPems); err != nil { s.logger.Error("Failed to update AutoEncrypt CAPems", "error", err) } - ws.Watch(nil) + + if err := ws.WatchCtx(ctx); err == context.Canceled { + s.logger.Info("shutting down Connect CA roots monitor") + return + } } } diff --git a/proto/pbautoconf/auto_config.pb.go b/proto/pbautoconf/auto_config.pb.go index f354153c5..8d1df2d73 100644 --- a/proto/pbautoconf/auto_config.pb.go +++ b/proto/pbautoconf/auto_config.pb.go @@ -7,6 +7,7 @@ import ( fmt "fmt" proto "github.com/golang/protobuf/proto" pbconfig "github.com/hashicorp/consul/proto/pbconfig" + pbconnect "github.com/hashicorp/consul/proto/pbconnect" io "io" math "math" ) @@ -39,7 +40,10 @@ type AutoConfigRequest struct { JWT string `protobuf:"bytes,5,opt,name=JWT,proto3" json:"JWT,omitempty"` // ConsulToken is a Consul ACL token that the agent requesting the // configuration already has. - ConsulToken string `protobuf:"bytes,6,opt,name=ConsulToken,proto3" json:"ConsulToken,omitempty"` + ConsulToken string `protobuf:"bytes,6,opt,name=ConsulToken,proto3" json:"ConsulToken,omitempty"` + // CSR is a certificate signing request to be used when generating the + // agents TLS certificate + CSR string `protobuf:"bytes,7,opt,name=CSR,proto3" json:"CSR,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -113,12 +117,27 @@ func (m *AutoConfigRequest) GetConsulToken() string { return "" } +func (m *AutoConfigRequest) GetCSR() string { + if m != nil { + return m.CSR + } + return "" +} + // AutoConfigResponse is the data structure sent in response to a AutoConfig.InitialConfiguration request type AutoConfigResponse struct { - Config *pbconfig.Config `protobuf:"bytes,1,opt,name=Config,proto3" json:"Config,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + // Config is the partial Consul configuration to inject into the agents own configuration + Config *pbconfig.Config `protobuf:"bytes,1,opt,name=Config,proto3" json:"Config,omitempty"` + // CARoots is the current list of Connect CA Roots + CARoots *pbconnect.CARoots `protobuf:"bytes,2,opt,name=CARoots,proto3" json:"CARoots,omitempty"` + // Certificate is the TLS certificate issued for the agent + Certificate *pbconnect.IssuedCert `protobuf:"bytes,3,opt,name=Certificate,proto3" json:"Certificate,omitempty"` + // ExtraCACertificates holds non-Connect certificates that may be necessary + // to verify TLS connections with the Consul servers + ExtraCACertificates []string `protobuf:"bytes,4,rep,name=ExtraCACertificates,proto3" json:"ExtraCACertificates,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *AutoConfigResponse) Reset() { *m = AutoConfigResponse{} } @@ -161,31 +180,60 @@ func (m *AutoConfigResponse) GetConfig() *pbconfig.Config { return nil } +func (m *AutoConfigResponse) GetCARoots() *pbconnect.CARoots { + if m != nil { + return m.CARoots + } + return nil +} + +func (m *AutoConfigResponse) GetCertificate() *pbconnect.IssuedCert { + if m != nil { + return m.Certificate + } + return nil +} + +func (m *AutoConfigResponse) GetExtraCACertificates() []string { + if m != nil { + return m.ExtraCACertificates + } + return nil +} + func init() { proto.RegisterType((*AutoConfigRequest)(nil), "autoconf.AutoConfigRequest") proto.RegisterType((*AutoConfigResponse)(nil), "autoconf.AutoConfigResponse") } -func init() { proto.RegisterFile("proto/pbautoconf/auto_config.proto", fileDescriptor_ccc5af992e5daf69) } +func init() { + proto.RegisterFile("proto/pbautoconf/auto_config.proto", fileDescriptor_ccc5af992e5daf69) +} var fileDescriptor_ccc5af992e5daf69 = []byte{ - // 256 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x2a, 0x28, 0xca, 0x2f, - 0xc9, 0xd7, 0x2f, 0x48, 0x4a, 0x2c, 0x2d, 0xc9, 0x4f, 0xce, 0xcf, 0x4b, 0xd3, 0x07, 0x31, 0xe2, - 0x41, 0xac, 0xcc, 0x74, 0x3d, 0xb0, 0xa4, 0x10, 0x07, 0x4c, 0x4e, 0x4a, 0x1a, 0xa6, 0x1a, 0x22, - 0xaf, 0x8f, 0xac, 0x4c, 0x69, 0x2a, 0x23, 0x97, 0xa0, 0x63, 0x69, 0x49, 0xbe, 0x33, 0x58, 0x30, - 0x28, 0xb5, 0xb0, 0x34, 0xb5, 0xb8, 0x44, 0x48, 0x8e, 0x8b, 0xcb, 0x25, 0xb1, 0x24, 0x31, 0x39, - 0x35, 0xaf, 0x24, 0xb5, 0x48, 0x82, 0x51, 0x81, 0x51, 0x83, 0x33, 0x08, 0x49, 0x44, 0x48, 0x88, - 0x8b, 0xc5, 0x2f, 0x3f, 0x25, 0x55, 0x82, 0x09, 0x2c, 0x03, 0x66, 0x0b, 0x49, 0x70, 0xb1, 0x07, - 0xa7, 0xa6, 0xe7, 0xa6, 0xe6, 0x95, 0x48, 0xb0, 0x80, 0x85, 0x61, 0x5c, 0x21, 0x01, 0x2e, 0x66, - 0xaf, 0xf0, 0x10, 0x09, 0x56, 0xb0, 0x28, 0x88, 0x29, 0xa4, 0xc0, 0xc5, 0xed, 0x9c, 0x9f, 0x57, - 0x5c, 0x9a, 0x13, 0x92, 0x9f, 0x9d, 0x9a, 0x27, 0xc1, 0x06, 0x96, 0x41, 0x16, 0x52, 0xb2, 0xe1, - 0x12, 0x42, 0x76, 0x56, 0x71, 0x41, 0x7e, 0x5e, 0x71, 0xaa, 0x90, 0x1a, 0x17, 0x1b, 0x44, 0x04, - 0xec, 0x26, 0x6e, 0x23, 0x3e, 0x3d, 0xa8, 0x67, 0xa0, 0xea, 0xa0, 0xb2, 0x4e, 0x76, 0x27, 0x1e, - 0xc9, 0x31, 0x5e, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x8c, 0xc7, 0x72, 0x0c, 0x51, - 0x3a, 0xe9, 0x99, 0x25, 0x19, 0xa5, 0x49, 0x7a, 0xc9, 0xf9, 0xb9, 0xfa, 0x19, 0x89, 0xc5, 0x19, - 0x99, 0xc9, 0xf9, 0x45, 0x05, 0xa0, 0xa0, 0x28, 0x2e, 0xcd, 0xd1, 0x47, 0x0f, 0xce, 0x24, 0x36, - 0xb0, 0x88, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0x46, 0x7e, 0xde, 0xed, 0x69, 0x01, 0x00, 0x00, + // 347 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x92, 0xc1, 0x4a, 0xc3, 0x30, + 0x18, 0xc7, 0x8d, 0x9b, 0x9b, 0x4b, 0x41, 0x66, 0x76, 0x09, 0x13, 0xcb, 0xd8, 0x41, 0x86, 0x48, + 0x2b, 0x13, 0xaf, 0xc2, 0x8c, 0x1e, 0xf4, 0xe0, 0x21, 0x1b, 0x08, 0x5e, 0xa4, 0xeb, 0xb2, 0xad, + 0xb8, 0x25, 0xb5, 0xf9, 0x02, 0x3e, 0x8a, 0x6f, 0xe0, 0xab, 0x78, 0xd4, 0x37, 0x90, 0xf9, 0x22, + 0xd2, 0xb4, 0x95, 0x20, 0x9e, 0xfa, 0xef, 0xff, 0xf7, 0xfb, 0xe0, 0x6b, 0x13, 0xdc, 0x4f, 0x33, + 0x05, 0x2a, 0x4c, 0xa7, 0x91, 0x01, 0x15, 0x2b, 0x39, 0x0f, 0xf3, 0xf0, 0x98, 0xa7, 0x64, 0x11, + 0x58, 0x48, 0x76, 0x2b, 0xd6, 0x3d, 0xa8, 0xec, 0x82, 0x87, 0xae, 0xd6, 0x3d, 0x74, 0xa0, 0x14, + 0x31, 0x84, 0xe5, 0xb3, 0xc0, 0xfd, 0x37, 0x84, 0xf7, 0x47, 0x06, 0x14, 0xb3, 0x33, 0x5c, 0x3c, + 0x1b, 0xa1, 0x81, 0xf8, 0x18, 0x5f, 0x45, 0x10, 0xc5, 0x42, 0x82, 0xc8, 0x28, 0xea, 0xa1, 0x41, + 0x8b, 0x3b, 0x0d, 0x21, 0xb8, 0x7e, 0xa7, 0x66, 0x82, 0x6e, 0x5b, 0x62, 0x33, 0xa1, 0xb8, 0x39, + 0x16, 0x8b, 0xb5, 0x90, 0x40, 0xeb, 0xb6, 0xae, 0x5e, 0x49, 0x1b, 0xd7, 0x6e, 0xef, 0x27, 0x74, + 0xc7, 0xb6, 0x79, 0x24, 0x3d, 0xec, 0x31, 0x25, 0xb5, 0x59, 0x4d, 0xd4, 0x93, 0x90, 0xb4, 0x61, + 0x89, 0x5b, 0xe5, 0x33, 0x6c, 0xcc, 0x69, 0xb3, 0x98, 0x61, 0x63, 0xde, 0xff, 0x44, 0x98, 0xb8, + 0x9b, 0xea, 0x54, 0x49, 0x2d, 0xc8, 0x11, 0x6e, 0x14, 0x8d, 0x5d, 0xd3, 0x1b, 0xee, 0x05, 0xe5, + 0xe7, 0x97, 0x5e, 0x49, 0xc9, 0x31, 0x6e, 0xb2, 0x11, 0x57, 0x0a, 0xb4, 0xdd, 0xda, 0x1b, 0xb6, + 0x83, 0xea, 0x4f, 0x94, 0x3d, 0xaf, 0x04, 0x72, 0x8e, 0x3d, 0x26, 0x32, 0x48, 0xe6, 0x49, 0x1c, + 0x81, 0xa0, 0x35, 0xeb, 0x77, 0x7e, 0xfd, 0x1b, 0xad, 0x8d, 0x98, 0xe5, 0x06, 0x77, 0x3d, 0x72, + 0x8a, 0x3b, 0xd7, 0x2f, 0x90, 0x45, 0x6c, 0xe4, 0xb4, 0x9a, 0xd6, 0x7b, 0xb5, 0x41, 0x8b, 0xff, + 0x87, 0x2e, 0x2f, 0xde, 0x37, 0x3e, 0xfa, 0xd8, 0xf8, 0xe8, 0x6b, 0xe3, 0xa3, 0xd7, 0x6f, 0x7f, + 0xeb, 0xe1, 0x64, 0x91, 0xc0, 0xd2, 0x4c, 0x83, 0x58, 0xad, 0xc3, 0x65, 0xa4, 0x97, 0x49, 0xac, + 0xb2, 0x34, 0x3f, 0x33, 0x6d, 0x56, 0xe1, 0xdf, 0x5b, 0x31, 0x6d, 0xd8, 0xe6, 0xec, 0x27, 0x00, + 0x00, 0xff, 0xff, 0xe2, 0x1d, 0x6e, 0x48, 0x30, 0x02, 0x00, 0x00, } func (m *AutoConfigRequest) Marshal() (dAtA []byte, err error) { @@ -233,6 +281,12 @@ func (m *AutoConfigRequest) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintAutoConfig(dAtA, i, uint64(len(m.ConsulToken))) i += copy(dAtA[i:], m.ConsulToken) } + if len(m.CSR) > 0 { + dAtA[i] = 0x3a + i++ + i = encodeVarintAutoConfig(dAtA, i, uint64(len(m.CSR))) + i += copy(dAtA[i:], m.CSR) + } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -264,6 +318,41 @@ func (m *AutoConfigResponse) MarshalTo(dAtA []byte) (int, error) { } i += n1 } + if m.CARoots != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintAutoConfig(dAtA, i, uint64(m.CARoots.Size())) + n2, err := m.CARoots.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + } + if m.Certificate != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintAutoConfig(dAtA, i, uint64(m.Certificate.Size())) + n3, err := m.Certificate.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + } + if len(m.ExtraCACertificates) > 0 { + for _, s := range m.ExtraCACertificates { + dAtA[i] = 0x22 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -305,6 +394,10 @@ func (m *AutoConfigRequest) Size() (n int) { if l > 0 { n += 1 + l + sovAutoConfig(uint64(l)) } + l = len(m.CSR) + if l > 0 { + n += 1 + l + sovAutoConfig(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -321,6 +414,20 @@ func (m *AutoConfigResponse) Size() (n int) { l = m.Config.Size() n += 1 + l + sovAutoConfig(uint64(l)) } + if m.CARoots != nil { + l = m.CARoots.Size() + n += 1 + l + sovAutoConfig(uint64(l)) + } + if m.Certificate != nil { + l = m.Certificate.Size() + n += 1 + l + sovAutoConfig(uint64(l)) + } + if len(m.ExtraCACertificates) > 0 { + for _, s := range m.ExtraCACertificates { + l = len(s) + n += 1 + l + sovAutoConfig(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -529,6 +636,38 @@ func (m *AutoConfigRequest) Unmarshal(dAtA []byte) error { } m.ConsulToken = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CSR", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAutoConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthAutoConfig + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthAutoConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CSR = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipAutoConfig(dAtA[iNdEx:]) @@ -619,6 +758,110 @@ func (m *AutoConfigResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CARoots", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAutoConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthAutoConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthAutoConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CARoots == nil { + m.CARoots = &pbconnect.CARoots{} + } + if err := m.CARoots.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Certificate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAutoConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthAutoConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthAutoConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Certificate == nil { + m.Certificate = &pbconnect.IssuedCert{} + } + if err := m.Certificate.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraCACertificates", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAutoConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthAutoConfig + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthAutoConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraCACertificates = append(m.ExtraCACertificates, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipAutoConfig(dAtA[iNdEx:]) diff --git a/proto/pbautoconf/auto_config.proto b/proto/pbautoconf/auto_config.proto index 979edf23d..aa7f4e381 100644 --- a/proto/pbautoconf/auto_config.proto +++ b/proto/pbautoconf/auto_config.proto @@ -5,6 +5,7 @@ package autoconf; option go_package = "github.com/hashicorp/consul/proto/pbautoconf"; import "proto/pbconfig/config.proto"; +import "proto/pbconnect/connect.proto"; // AutoConfigRequest is the data structure to be sent along with the // AutoConfig.InitialConfiguration RPC @@ -28,9 +29,23 @@ message AutoConfigRequest { // ConsulToken is a Consul ACL token that the agent requesting the // configuration already has. string ConsulToken = 6; + + // CSR is a certificate signing request to be used when generating the + // agents TLS certificate + string CSR = 7; } // AutoConfigResponse is the data structure sent in response to a AutoConfig.InitialConfiguration request message AutoConfigResponse { + // Config is the partial Consul configuration to inject into the agents own configuration config.Config Config = 1; + + // CARoots is the current list of Connect CA Roots + connect.CARoots CARoots = 2; + // Certificate is the TLS certificate issued for the agent + connect.IssuedCert Certificate = 3; + + // ExtraCACertificates holds non-Connect certificates that may be necessary + // to verify TLS connections with the Consul servers + repeated string ExtraCACertificates = 4; } \ No newline at end of file diff --git a/proto/pbconnect/connect.pb.binary.go b/proto/pbconnect/connect.pb.binary.go new file mode 100644 index 000000000..ff4d44ee7 --- /dev/null +++ b/proto/pbconnect/connect.pb.binary.go @@ -0,0 +1,38 @@ +// Code generated by protoc-gen-go-binary. DO NOT EDIT. +// source: proto/pbconnect/connect.proto + +package pbconnect + +import ( + "github.com/golang/protobuf/proto" +) + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *CARoots) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *CARoots) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *CARoot) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *CARoot) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *IssuedCert) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *IssuedCert) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} diff --git a/proto/pbconnect/connect.pb.go b/proto/pbconnect/connect.pb.go new file mode 100644 index 000000000..ba36492c5 --- /dev/null +++ b/proto/pbconnect/connect.pb.go @@ -0,0 +1,2271 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: proto/pbconnect/connect.proto + +package pbconnect + +import ( + fmt "fmt" + types "github.com/gogo/protobuf/types" + proto "github.com/golang/protobuf/proto" + pbcommon "github.com/hashicorp/consul/proto/pbcommon" + io "io" + math "math" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +// CARoots is the list of all currently trusted CA Roots. +type CARoots struct { + // ActiveRootID is the ID of a root in Roots that is the active CA root. + // Other roots are still valid if they're in the Roots list but are in + // the process of being rotated out. + ActiveRootID string `protobuf:"bytes,1,opt,name=ActiveRootID,proto3" json:"ActiveRootID,omitempty"` + // TrustDomain is the identification root for this Consul cluster. All + // certificates signed by the cluster's CA must have their identifying URI in + // this domain. + // + // This does not include the protocol (currently spiffe://) since we may + // implement other protocols in future with equivalent semantics. It should be + // compared against the "authority" section of a URI (i.e. host:port). + // + // We need to support migrating a cluster between trust domains to support + // Multi-DC migration in Enterprise. In this case the current trust domain is + // here but entries in Roots may also have ExternalTrustDomain set to a + // non-empty value implying they were previous roots that are still trusted + // but under a different trust domain. + // + // Note that we DON'T validate trust domain during AuthZ since it causes + // issues of loss of connectivity during migration between trust domains. The + // only time the additional validation adds value is where the cluster shares + // an external root (e.g. organization-wide root) with another distinct Consul + // cluster or PKI system. In this case, x509 Name Constraints can be added to + // enforce that Consul's CA can only validly sign or trust certs within the + // same trust-domain. Name constraints as enforced by TLS handshake also allow + // seamless rotation between trust domains thanks to cross-signing. + TrustDomain string `protobuf:"bytes,2,opt,name=TrustDomain,proto3" json:"TrustDomain,omitempty"` + // Roots is a list of root CA certs to trust. + Roots []*CARoot `protobuf:"bytes,3,rep,name=Roots,proto3" json:"Roots,omitempty"` + // QueryMeta here is mainly used to contain the latest Raft Index that could + // be used to perform a blocking query. + QueryMeta *pbcommon.QueryMeta `protobuf:"bytes,4,opt,name=QueryMeta,proto3" json:"QueryMeta,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CARoots) Reset() { *m = CARoots{} } +func (m *CARoots) String() string { return proto.CompactTextString(m) } +func (*CARoots) ProtoMessage() {} +func (*CARoots) Descriptor() ([]byte, []int) { + return fileDescriptor_80627e709958eb04, []int{0} +} +func (m *CARoots) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CARoots) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CARoots.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *CARoots) XXX_Merge(src proto.Message) { + xxx_messageInfo_CARoots.Merge(m, src) +} +func (m *CARoots) XXX_Size() int { + return m.Size() +} +func (m *CARoots) XXX_DiscardUnknown() { + xxx_messageInfo_CARoots.DiscardUnknown(m) +} + +var xxx_messageInfo_CARoots proto.InternalMessageInfo + +func (m *CARoots) GetActiveRootID() string { + if m != nil { + return m.ActiveRootID + } + return "" +} + +func (m *CARoots) GetTrustDomain() string { + if m != nil { + return m.TrustDomain + } + return "" +} + +func (m *CARoots) GetRoots() []*CARoot { + if m != nil { + return m.Roots + } + return nil +} + +func (m *CARoots) GetQueryMeta() *pbcommon.QueryMeta { + if m != nil { + return m.QueryMeta + } + return nil +} + +type CARoot struct { + // ID is a globally unique ID (UUID) representing this CA root. + ID string `protobuf:"bytes,1,opt,name=ID,proto3" json:"ID,omitempty"` + // Name is a human-friendly name for this CA root. This value is + // opaque to Consul and is not used for anything internally. + Name string `protobuf:"bytes,2,opt,name=Name,proto3" json:"Name,omitempty"` + // SerialNumber is the x509 serial number of the certificate. + SerialNumber uint64 `protobuf:"varint,3,opt,name=SerialNumber,proto3" json:"SerialNumber,omitempty"` + // SigningKeyID is the ID of the public key that corresponds to the private + // key used to sign leaf certificates. Is is the HexString format of the + // raw AuthorityKeyID bytes. + SigningKeyID string `protobuf:"bytes,4,opt,name=SigningKeyID,proto3" json:"SigningKeyID,omitempty"` + // ExternalTrustDomain is the trust domain this root was generated under. It + // is usually empty implying "the current cluster trust-domain". It is set + // only in the case that a cluster changes trust domain and then all old roots + // that are still trusted have the old trust domain set here. + // + // We currently DON'T validate these trust domains explicitly anywhere, see + // IndexedRoots.TrustDomain doc. We retain this information for debugging and + // future flexibility. + ExternalTrustDomain string `protobuf:"bytes,5,opt,name=ExternalTrustDomain,proto3" json:"ExternalTrustDomain,omitempty"` + // Time validity bounds. + NotBefore *types.Timestamp `protobuf:"bytes,6,opt,name=NotBefore,proto3" json:"NotBefore,omitempty"` + NotAfter *types.Timestamp `protobuf:"bytes,7,opt,name=NotAfter,proto3" json:"NotAfter,omitempty"` + // RootCert is the PEM-encoded public certificate. + RootCert string `protobuf:"bytes,8,opt,name=RootCert,proto3" json:"RootCert,omitempty"` + // IntermediateCerts is a list of PEM-encoded intermediate certs to + // attach to any leaf certs signed by this CA. + IntermediateCerts []string `protobuf:"bytes,9,rep,name=IntermediateCerts,proto3" json:"IntermediateCerts,omitempty"` + // SigningCert is the PEM-encoded signing certificate and SigningKey + // is the PEM-encoded private key for the signing certificate. These + // may actually be empty if the CA plugin in use manages these for us. + SigningCert string `protobuf:"bytes,10,opt,name=SigningCert,proto3" json:"SigningCert,omitempty"` + SigningKey string `protobuf:"bytes,11,opt,name=SigningKey,proto3" json:"SigningKey,omitempty"` + // Active is true if this is the current active CA. This must only + // be true for exactly one CA. For any method that modifies roots in the + // state store, tests should be written to verify that multiple roots + // cannot be active. + Active bool `protobuf:"varint,12,opt,name=Active,proto3" json:"Active,omitempty"` + // RotatedOutAt is the time at which this CA was removed from the state. + // This will only be set on roots that have been rotated out from being the + // active root. + RotatedOutAt *types.Timestamp `protobuf:"bytes,13,opt,name=RotatedOutAt,proto3" json:"RotatedOutAt,omitempty"` + // PrivateKeyType is the type of the private key used to sign certificates. It + // may be "rsa" or "ec". This is provided as a convenience to avoid parsing + // the public key to from the certificate to infer the type. + PrivateKeyType string `protobuf:"bytes,14,opt,name=PrivateKeyType,proto3" json:"PrivateKeyType,omitempty"` + // PrivateKeyBits is the length of the private key used to sign certificates. + // This is provided as a convenience to avoid parsing the public key from the + // certificate to infer the type. + PrivateKeyBits int32 `protobuf:"varint,15,opt,name=PrivateKeyBits,proto3" json:"PrivateKeyBits,omitempty"` + RaftIndex *pbcommon.RaftIndex `protobuf:"bytes,16,opt,name=RaftIndex,proto3" json:"RaftIndex,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CARoot) Reset() { *m = CARoot{} } +func (m *CARoot) String() string { return proto.CompactTextString(m) } +func (*CARoot) ProtoMessage() {} +func (*CARoot) Descriptor() ([]byte, []int) { + return fileDescriptor_80627e709958eb04, []int{1} +} +func (m *CARoot) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CARoot) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CARoot.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *CARoot) XXX_Merge(src proto.Message) { + xxx_messageInfo_CARoot.Merge(m, src) +} +func (m *CARoot) XXX_Size() int { + return m.Size() +} +func (m *CARoot) XXX_DiscardUnknown() { + xxx_messageInfo_CARoot.DiscardUnknown(m) +} + +var xxx_messageInfo_CARoot proto.InternalMessageInfo + +func (m *CARoot) GetID() string { + if m != nil { + return m.ID + } + return "" +} + +func (m *CARoot) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *CARoot) GetSerialNumber() uint64 { + if m != nil { + return m.SerialNumber + } + return 0 +} + +func (m *CARoot) GetSigningKeyID() string { + if m != nil { + return m.SigningKeyID + } + return "" +} + +func (m *CARoot) GetExternalTrustDomain() string { + if m != nil { + return m.ExternalTrustDomain + } + return "" +} + +func (m *CARoot) GetNotBefore() *types.Timestamp { + if m != nil { + return m.NotBefore + } + return nil +} + +func (m *CARoot) GetNotAfter() *types.Timestamp { + if m != nil { + return m.NotAfter + } + return nil +} + +func (m *CARoot) GetRootCert() string { + if m != nil { + return m.RootCert + } + return "" +} + +func (m *CARoot) GetIntermediateCerts() []string { + if m != nil { + return m.IntermediateCerts + } + return nil +} + +func (m *CARoot) GetSigningCert() string { + if m != nil { + return m.SigningCert + } + return "" +} + +func (m *CARoot) GetSigningKey() string { + if m != nil { + return m.SigningKey + } + return "" +} + +func (m *CARoot) GetActive() bool { + if m != nil { + return m.Active + } + return false +} + +func (m *CARoot) GetRotatedOutAt() *types.Timestamp { + if m != nil { + return m.RotatedOutAt + } + return nil +} + +func (m *CARoot) GetPrivateKeyType() string { + if m != nil { + return m.PrivateKeyType + } + return "" +} + +func (m *CARoot) GetPrivateKeyBits() int32 { + if m != nil { + return m.PrivateKeyBits + } + return 0 +} + +func (m *CARoot) GetRaftIndex() *pbcommon.RaftIndex { + if m != nil { + return m.RaftIndex + } + return nil +} + +type IssuedCert struct { + // SerialNumber is the unique serial number for this certificate. + // This is encoded in standard hex separated by :. + SerialNumber string `protobuf:"bytes,1,opt,name=SerialNumber,proto3" json:"SerialNumber,omitempty"` + // CertPEM and PrivateKeyPEM are the PEM-encoded certificate and private + // key for that cert, respectively. This should not be stored in the + // state store, but is present in the sign API response. + CertPEM string `protobuf:"bytes,2,opt,name=CertPEM,proto3" json:"CertPEM,omitempty"` + PrivateKeyPEM string `protobuf:"bytes,3,opt,name=PrivateKeyPEM,proto3" json:"PrivateKeyPEM,omitempty"` + // Service is the name of the service for which the cert was issued. + // ServiceURI is the cert URI value. + Service string `protobuf:"bytes,4,opt,name=Service,proto3" json:"Service,omitempty"` + ServiceURI string `protobuf:"bytes,5,opt,name=ServiceURI,proto3" json:"ServiceURI,omitempty"` + // Agent is the name of the node for which the cert was issued. + // AgentURI is the cert URI value. + Agent string `protobuf:"bytes,6,opt,name=Agent,proto3" json:"Agent,omitempty"` + AgentURI string `protobuf:"bytes,7,opt,name=AgentURI,proto3" json:"AgentURI,omitempty"` + // ValidAfter and ValidBefore are the validity periods for the + // certificate. + ValidAfter *types.Timestamp `protobuf:"bytes,8,opt,name=ValidAfter,proto3" json:"ValidAfter,omitempty"` + ValidBefore *types.Timestamp `protobuf:"bytes,9,opt,name=ValidBefore,proto3" json:"ValidBefore,omitempty"` + // EnterpriseMeta is the Consul Enterprise specific metadata + EnterpriseMeta *pbcommon.EnterpriseMeta `protobuf:"bytes,10,opt,name=EnterpriseMeta,proto3" json:"EnterpriseMeta,omitempty"` + RaftIndex *pbcommon.RaftIndex `protobuf:"bytes,11,opt,name=RaftIndex,proto3" json:"RaftIndex,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *IssuedCert) Reset() { *m = IssuedCert{} } +func (m *IssuedCert) String() string { return proto.CompactTextString(m) } +func (*IssuedCert) ProtoMessage() {} +func (*IssuedCert) Descriptor() ([]byte, []int) { + return fileDescriptor_80627e709958eb04, []int{2} +} +func (m *IssuedCert) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IssuedCert) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_IssuedCert.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *IssuedCert) XXX_Merge(src proto.Message) { + xxx_messageInfo_IssuedCert.Merge(m, src) +} +func (m *IssuedCert) XXX_Size() int { + return m.Size() +} +func (m *IssuedCert) XXX_DiscardUnknown() { + xxx_messageInfo_IssuedCert.DiscardUnknown(m) +} + +var xxx_messageInfo_IssuedCert proto.InternalMessageInfo + +func (m *IssuedCert) GetSerialNumber() string { + if m != nil { + return m.SerialNumber + } + return "" +} + +func (m *IssuedCert) GetCertPEM() string { + if m != nil { + return m.CertPEM + } + return "" +} + +func (m *IssuedCert) GetPrivateKeyPEM() string { + if m != nil { + return m.PrivateKeyPEM + } + return "" +} + +func (m *IssuedCert) GetService() string { + if m != nil { + return m.Service + } + return "" +} + +func (m *IssuedCert) GetServiceURI() string { + if m != nil { + return m.ServiceURI + } + return "" +} + +func (m *IssuedCert) GetAgent() string { + if m != nil { + return m.Agent + } + return "" +} + +func (m *IssuedCert) GetAgentURI() string { + if m != nil { + return m.AgentURI + } + return "" +} + +func (m *IssuedCert) GetValidAfter() *types.Timestamp { + if m != nil { + return m.ValidAfter + } + return nil +} + +func (m *IssuedCert) GetValidBefore() *types.Timestamp { + if m != nil { + return m.ValidBefore + } + return nil +} + +func (m *IssuedCert) GetEnterpriseMeta() *pbcommon.EnterpriseMeta { + if m != nil { + return m.EnterpriseMeta + } + return nil +} + +func (m *IssuedCert) GetRaftIndex() *pbcommon.RaftIndex { + if m != nil { + return m.RaftIndex + } + return nil +} + +func init() { + proto.RegisterType((*CARoots)(nil), "connect.CARoots") + proto.RegisterType((*CARoot)(nil), "connect.CARoot") + proto.RegisterType((*IssuedCert)(nil), "connect.IssuedCert") +} + +func init() { proto.RegisterFile("proto/pbconnect/connect.proto", fileDescriptor_80627e709958eb04) } + +var fileDescriptor_80627e709958eb04 = []byte{ + // 667 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0xdf, 0x6a, 0x13, 0x4f, + 0x14, 0xfe, 0x6d, 0xf3, 0x77, 0x4f, 0xda, 0xf4, 0xd7, 0xf9, 0xfd, 0x28, 0x43, 0xc4, 0x74, 0x09, + 0x2a, 0x01, 0x25, 0x2b, 0x15, 0x44, 0x44, 0x0b, 0x69, 0xd3, 0x8b, 0x50, 0x1a, 0xeb, 0xb4, 0x7a, + 0xe1, 0x8d, 0x6c, 0x92, 0x93, 0x74, 0x20, 0xbb, 0x13, 0x66, 0x67, 0x4b, 0xf3, 0x26, 0xbe, 0x81, + 0x8f, 0xa2, 0x97, 0x3e, 0x82, 0xd4, 0xe7, 0x10, 0x64, 0x66, 0x77, 0x93, 0xdd, 0x58, 0xc8, 0x55, + 0xe6, 0x7c, 0xdf, 0x37, 0xb3, 0xe7, 0xcc, 0xf7, 0x65, 0xe0, 0xe1, 0x5c, 0x0a, 0x25, 0xdc, 0xf9, + 0x70, 0x24, 0x82, 0x00, 0x47, 0xca, 0x4d, 0x7e, 0x3b, 0x06, 0x27, 0x95, 0xa4, 0x6c, 0x1c, 0x4c, + 0x85, 0x98, 0xce, 0xd0, 0x35, 0xf0, 0x30, 0x9a, 0xb8, 0x8a, 0xfb, 0x18, 0x2a, 0xcf, 0x9f, 0xc7, + 0xca, 0xc6, 0x83, 0xd5, 0x41, 0xbe, 0x2f, 0x02, 0x37, 0xfe, 0x49, 0xc8, 0x83, 0x7b, 0xc9, 0xcf, + 0x22, 0x0c, 0x63, 0x41, 0xeb, 0xab, 0x05, 0x95, 0x93, 0x2e, 0x13, 0x42, 0x85, 0xa4, 0x05, 0xdb, + 0xdd, 0x91, 0xe2, 0x37, 0xa8, 0xcb, 0x7e, 0x8f, 0x5a, 0x8e, 0xd5, 0xb6, 0x59, 0x0e, 0x23, 0x0e, + 0xd4, 0xae, 0x64, 0x14, 0xaa, 0x9e, 0xf0, 0x3d, 0x1e, 0xd0, 0x2d, 0x23, 0xc9, 0x42, 0xe4, 0x31, + 0x94, 0xcc, 0x71, 0xb4, 0xe0, 0x14, 0xda, 0xb5, 0xc3, 0xdd, 0x4e, 0x3a, 0x58, 0xfc, 0x19, 0x16, + 0xb3, 0xc4, 0x05, 0xfb, 0x7d, 0x84, 0x72, 0x71, 0x8e, 0xca, 0xa3, 0x45, 0xc7, 0x6a, 0xd7, 0x0e, + 0xf7, 0x3a, 0x49, 0xef, 0x4b, 0x82, 0xad, 0x34, 0xad, 0xdf, 0x45, 0x28, 0xc7, 0x47, 0x90, 0x3a, + 0x6c, 0x2d, 0xdb, 0xdb, 0xea, 0xf7, 0x08, 0x81, 0xe2, 0xc0, 0xf3, 0x31, 0xe9, 0xc6, 0xac, 0xf5, + 0x30, 0x97, 0x28, 0xb9, 0x37, 0x1b, 0x44, 0xfe, 0x10, 0x25, 0x2d, 0x38, 0x56, 0xbb, 0xc8, 0x72, + 0x98, 0xd1, 0xf0, 0x69, 0xc0, 0x83, 0xe9, 0x19, 0x2e, 0xfa, 0x3d, 0xd3, 0x86, 0xcd, 0x72, 0x18, + 0x79, 0x0e, 0xff, 0x9d, 0xde, 0x2a, 0x94, 0x81, 0x37, 0xcb, 0x0e, 0x5e, 0x32, 0xd2, 0xfb, 0x28, + 0xf2, 0x0a, 0xec, 0x81, 0x50, 0xc7, 0x38, 0x11, 0x12, 0x69, 0xd9, 0x4c, 0xd6, 0xe8, 0xc4, 0x2e, + 0x76, 0x52, 0x17, 0x3b, 0x57, 0xa9, 0x8b, 0x6c, 0x25, 0x26, 0x2f, 0xa1, 0x3a, 0x10, 0xaa, 0x3b, + 0x51, 0x28, 0x69, 0x65, 0xe3, 0xc6, 0xa5, 0x96, 0x34, 0xa0, 0xaa, 0xef, 0xe5, 0x04, 0xa5, 0xa2, + 0x55, 0xd3, 0xd8, 0xb2, 0x26, 0xcf, 0x60, 0xaf, 0x1f, 0x28, 0x94, 0x3e, 0x8e, 0xb9, 0xa7, 0x50, + 0x63, 0x21, 0xb5, 0x9d, 0x42, 0xdb, 0x66, 0x7f, 0x13, 0xda, 0xde, 0x64, 0x7a, 0x73, 0x18, 0xc4, + 0xf6, 0x66, 0x20, 0xd2, 0x04, 0x58, 0xdd, 0x0f, 0xad, 0x19, 0x41, 0x06, 0x21, 0xfb, 0x50, 0x8e, + 0x03, 0x43, 0xb7, 0x1d, 0xab, 0x5d, 0x65, 0x49, 0x45, 0x8e, 0x60, 0x9b, 0x09, 0xe5, 0x29, 0x1c, + 0xbf, 0x8b, 0x54, 0x57, 0xd1, 0x9d, 0x8d, 0xf3, 0xe5, 0xf4, 0xe4, 0x09, 0xd4, 0x2f, 0x24, 0xbf, + 0xf1, 0x14, 0x9e, 0xe1, 0xe2, 0x6a, 0x31, 0x47, 0x5a, 0x37, 0xdf, 0x5e, 0x43, 0xf3, 0xba, 0x63, + 0xae, 0x42, 0xba, 0xeb, 0x58, 0xed, 0x12, 0x5b, 0x43, 0x75, 0xfe, 0x98, 0x37, 0x51, 0xfd, 0x60, + 0x8c, 0xb7, 0xf4, 0xdf, 0x7c, 0xfe, 0x96, 0x04, 0x5b, 0x69, 0x5a, 0xdf, 0x0a, 0x00, 0xfd, 0x30, + 0x8c, 0x70, 0x6c, 0xee, 0x61, 0x3d, 0x5f, 0xc9, 0x9f, 0x25, 0x97, 0x2f, 0x0a, 0x15, 0xad, 0xbd, + 0x38, 0x3d, 0x4f, 0xa2, 0x99, 0x96, 0xe4, 0x11, 0xec, 0xac, 0xfa, 0xd1, 0x7c, 0xc1, 0xf0, 0x79, + 0x50, 0xef, 0xbf, 0x44, 0x79, 0xc3, 0x47, 0x98, 0x44, 0x33, 0x2d, 0x8d, 0x0b, 0xf1, 0xf2, 0x03, + 0xeb, 0x27, 0x61, 0xcc, 0x20, 0xe4, 0x7f, 0x28, 0x75, 0xa7, 0x18, 0x28, 0x93, 0x3f, 0x9b, 0xc5, + 0x85, 0xce, 0x89, 0x59, 0xe8, 0x3d, 0x95, 0x38, 0x27, 0x69, 0x4d, 0x5e, 0x03, 0x7c, 0xf4, 0x66, + 0x7c, 0x1c, 0xa7, 0xaf, 0xba, 0xd1, 0x9d, 0x8c, 0x9a, 0xbc, 0x81, 0x9a, 0xa9, 0x92, 0xcc, 0xdb, + 0x1b, 0x37, 0x67, 0xe5, 0xe4, 0x08, 0xea, 0xa7, 0x3a, 0x88, 0x73, 0xc9, 0x43, 0x34, 0xcf, 0x01, + 0x98, 0x03, 0xf6, 0x53, 0x3b, 0xf2, 0x2c, 0x5b, 0x53, 0xe7, 0x9d, 0xac, 0x6d, 0x76, 0xf2, 0xf8, + 0xed, 0xf7, 0xbb, 0xa6, 0xf5, 0xe3, 0xae, 0x69, 0xfd, 0xbc, 0x6b, 0x5a, 0x5f, 0x7e, 0x35, 0xff, + 0xf9, 0xf4, 0x74, 0xca, 0xd5, 0x75, 0x34, 0xd4, 0xbb, 0xdc, 0x6b, 0x2f, 0xbc, 0xe6, 0x23, 0x21, + 0xe7, 0xfa, 0x45, 0x0e, 0xa3, 0x99, 0xbb, 0xf6, 0x50, 0x0f, 0xcb, 0x06, 0x78, 0xf1, 0x27, 0x00, + 0x00, 0xff, 0xff, 0xcf, 0xc8, 0x31, 0xb3, 0xc2, 0x05, 0x00, 0x00, +} + +func (m *CARoots) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CARoots) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.ActiveRootID) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.ActiveRootID))) + i += copy(dAtA[i:], m.ActiveRootID) + } + if len(m.TrustDomain) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.TrustDomain))) + i += copy(dAtA[i:], m.TrustDomain) + } + if len(m.Roots) > 0 { + for _, msg := range m.Roots { + dAtA[i] = 0x1a + i++ + i = encodeVarintConnect(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if m.QueryMeta != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.QueryMeta.Size())) + n1, err := m.QueryMeta.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + } + if m.XXX_unrecognized != nil { + i += copy(dAtA[i:], m.XXX_unrecognized) + } + return i, nil +} + +func (m *CARoot) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CARoot) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.ID) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.ID))) + i += copy(dAtA[i:], m.ID) + } + if len(m.Name) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.Name))) + i += copy(dAtA[i:], m.Name) + } + if m.SerialNumber != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.SerialNumber)) + } + if len(m.SigningKeyID) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.SigningKeyID))) + i += copy(dAtA[i:], m.SigningKeyID) + } + if len(m.ExternalTrustDomain) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.ExternalTrustDomain))) + i += copy(dAtA[i:], m.ExternalTrustDomain) + } + if m.NotBefore != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.NotBefore.Size())) + n2, err := m.NotBefore.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + } + if m.NotAfter != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.NotAfter.Size())) + n3, err := m.NotAfter.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + } + if len(m.RootCert) > 0 { + dAtA[i] = 0x42 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.RootCert))) + i += copy(dAtA[i:], m.RootCert) + } + if len(m.IntermediateCerts) > 0 { + for _, s := range m.IntermediateCerts { + dAtA[i] = 0x4a + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.SigningCert) > 0 { + dAtA[i] = 0x52 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.SigningCert))) + i += copy(dAtA[i:], m.SigningCert) + } + if len(m.SigningKey) > 0 { + dAtA[i] = 0x5a + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.SigningKey))) + i += copy(dAtA[i:], m.SigningKey) + } + if m.Active { + dAtA[i] = 0x60 + i++ + if m.Active { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.RotatedOutAt != nil { + dAtA[i] = 0x6a + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.RotatedOutAt.Size())) + n4, err := m.RotatedOutAt.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n4 + } + if len(m.PrivateKeyType) > 0 { + dAtA[i] = 0x72 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.PrivateKeyType))) + i += copy(dAtA[i:], m.PrivateKeyType) + } + if m.PrivateKeyBits != 0 { + dAtA[i] = 0x78 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.PrivateKeyBits)) + } + if m.RaftIndex != nil { + dAtA[i] = 0x82 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.RaftIndex.Size())) + n5, err := m.RaftIndex.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n5 + } + if m.XXX_unrecognized != nil { + i += copy(dAtA[i:], m.XXX_unrecognized) + } + return i, nil +} + +func (m *IssuedCert) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *IssuedCert) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.SerialNumber) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.SerialNumber))) + i += copy(dAtA[i:], m.SerialNumber) + } + if len(m.CertPEM) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.CertPEM))) + i += copy(dAtA[i:], m.CertPEM) + } + if len(m.PrivateKeyPEM) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.PrivateKeyPEM))) + i += copy(dAtA[i:], m.PrivateKeyPEM) + } + if len(m.Service) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.Service))) + i += copy(dAtA[i:], m.Service) + } + if len(m.ServiceURI) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.ServiceURI))) + i += copy(dAtA[i:], m.ServiceURI) + } + if len(m.Agent) > 0 { + dAtA[i] = 0x32 + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.Agent))) + i += copy(dAtA[i:], m.Agent) + } + if len(m.AgentURI) > 0 { + dAtA[i] = 0x3a + i++ + i = encodeVarintConnect(dAtA, i, uint64(len(m.AgentURI))) + i += copy(dAtA[i:], m.AgentURI) + } + if m.ValidAfter != nil { + dAtA[i] = 0x42 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.ValidAfter.Size())) + n6, err := m.ValidAfter.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } + if m.ValidBefore != nil { + dAtA[i] = 0x4a + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.ValidBefore.Size())) + n7, err := m.ValidBefore.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n7 + } + if m.EnterpriseMeta != nil { + dAtA[i] = 0x52 + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.EnterpriseMeta.Size())) + n8, err := m.EnterpriseMeta.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n8 + } + if m.RaftIndex != nil { + dAtA[i] = 0x5a + i++ + i = encodeVarintConnect(dAtA, i, uint64(m.RaftIndex.Size())) + n9, err := m.RaftIndex.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n9 + } + if m.XXX_unrecognized != nil { + i += copy(dAtA[i:], m.XXX_unrecognized) + } + return i, nil +} + +func encodeVarintConnect(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *CARoots) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ActiveRootID) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.TrustDomain) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if len(m.Roots) > 0 { + for _, e := range m.Roots { + l = e.Size() + n += 1 + l + sovConnect(uint64(l)) + } + } + if m.QueryMeta != nil { + l = m.QueryMeta.Size() + n += 1 + l + sovConnect(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CARoot) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ID) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if m.SerialNumber != 0 { + n += 1 + sovConnect(uint64(m.SerialNumber)) + } + l = len(m.SigningKeyID) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.ExternalTrustDomain) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if m.NotBefore != nil { + l = m.NotBefore.Size() + n += 1 + l + sovConnect(uint64(l)) + } + if m.NotAfter != nil { + l = m.NotAfter.Size() + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.RootCert) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if len(m.IntermediateCerts) > 0 { + for _, s := range m.IntermediateCerts { + l = len(s) + n += 1 + l + sovConnect(uint64(l)) + } + } + l = len(m.SigningCert) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.SigningKey) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if m.Active { + n += 2 + } + if m.RotatedOutAt != nil { + l = m.RotatedOutAt.Size() + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.PrivateKeyType) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if m.PrivateKeyBits != 0 { + n += 1 + sovConnect(uint64(m.PrivateKeyBits)) + } + if m.RaftIndex != nil { + l = m.RaftIndex.Size() + n += 2 + l + sovConnect(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *IssuedCert) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.SerialNumber) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.CertPEM) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.PrivateKeyPEM) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.Service) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.ServiceURI) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.Agent) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + l = len(m.AgentURI) + if l > 0 { + n += 1 + l + sovConnect(uint64(l)) + } + if m.ValidAfter != nil { + l = m.ValidAfter.Size() + n += 1 + l + sovConnect(uint64(l)) + } + if m.ValidBefore != nil { + l = m.ValidBefore.Size() + n += 1 + l + sovConnect(uint64(l)) + } + if m.EnterpriseMeta != nil { + l = m.EnterpriseMeta.Size() + n += 1 + l + sovConnect(uint64(l)) + } + if m.RaftIndex != nil { + l = m.RaftIndex.Size() + n += 1 + l + sovConnect(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovConnect(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozConnect(x uint64) (n int) { + return sovConnect(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *CARoots) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CARoots: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CARoots: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ActiveRootID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ActiveRootID = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TrustDomain", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TrustDomain = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Roots", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Roots = append(m.Roots, &CARoot{}) + if err := m.Roots[len(m.Roots)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryMeta", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.QueryMeta == nil { + m.QueryMeta = &pbcommon.QueryMeta{} + } + if err := m.QueryMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipConnect(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthConnect + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthConnect + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CARoot) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CARoot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CARoot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ID = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SerialNumber", wireType) + } + m.SerialNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SerialNumber |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SigningKeyID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SigningKeyID = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExternalTrustDomain", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExternalTrustDomain = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NotBefore", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NotBefore == nil { + m.NotBefore = &types.Timestamp{} + } + if err := m.NotBefore.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NotAfter", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NotAfter == nil { + m.NotAfter = &types.Timestamp{} + } + if err := m.NotAfter.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootCert", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RootCert = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IntermediateCerts", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IntermediateCerts = append(m.IntermediateCerts, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SigningCert", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SigningCert = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SigningKey", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SigningKey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Active", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Active = bool(v != 0) + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RotatedOutAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RotatedOutAt == nil { + m.RotatedOutAt = &types.Timestamp{} + } + if err := m.RotatedOutAt.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrivateKeyType", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrivateKeyType = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PrivateKeyBits", wireType) + } + m.PrivateKeyBits = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PrivateKeyBits |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RaftIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RaftIndex == nil { + m.RaftIndex = &pbcommon.RaftIndex{} + } + if err := m.RaftIndex.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipConnect(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthConnect + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthConnect + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *IssuedCert) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: IssuedCert: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IssuedCert: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SerialNumber", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SerialNumber = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CertPEM", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CertPEM = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrivateKeyPEM", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrivateKeyPEM = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Service", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Service = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ServiceURI", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ServiceURI = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Agent", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Agent = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentURI", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AgentURI = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidAfter", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidAfter == nil { + m.ValidAfter = &types.Timestamp{} + } + if err := m.ValidAfter.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidBefore", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidBefore == nil { + m.ValidBefore = &types.Timestamp{} + } + if err := m.ValidBefore.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EnterpriseMeta", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EnterpriseMeta == nil { + m.EnterpriseMeta = &pbcommon.EnterpriseMeta{} + } + if err := m.EnterpriseMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RaftIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowConnect + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthConnect + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthConnect + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RaftIndex == nil { + m.RaftIndex = &pbcommon.RaftIndex{} + } + if err := m.RaftIndex.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipConnect(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthConnect + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthConnect + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipConnect(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowConnect + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowConnect + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowConnect + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthConnect + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthConnect + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowConnect + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipConnect(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthConnect + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthConnect = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowConnect = fmt.Errorf("proto: integer overflow") +) diff --git a/proto/pbconnect/connect.proto b/proto/pbconnect/connect.proto new file mode 100644 index 000000000..897cfec8f --- /dev/null +++ b/proto/pbconnect/connect.proto @@ -0,0 +1,147 @@ +syntax = "proto3"; + +package connect; + +option go_package = "github.com/hashicorp/consul/proto/pbconnect"; + +import "google/protobuf/timestamp.proto"; +import "proto/pbcommon/common.proto"; +import "proto/pbcommon/common_oss.proto"; + +// CARoots is the list of all currently trusted CA Roots. +message CARoots { + // ActiveRootID is the ID of a root in Roots that is the active CA root. + // Other roots are still valid if they're in the Roots list but are in + // the process of being rotated out. + string ActiveRootID = 1; + + // TrustDomain is the identification root for this Consul cluster. All + // certificates signed by the cluster's CA must have their identifying URI in + // this domain. + // + // This does not include the protocol (currently spiffe://) since we may + // implement other protocols in future with equivalent semantics. It should be + // compared against the "authority" section of a URI (i.e. host:port). + // + // We need to support migrating a cluster between trust domains to support + // Multi-DC migration in Enterprise. In this case the current trust domain is + // here but entries in Roots may also have ExternalTrustDomain set to a + // non-empty value implying they were previous roots that are still trusted + // but under a different trust domain. + // + // Note that we DON'T validate trust domain during AuthZ since it causes + // issues of loss of connectivity during migration between trust domains. The + // only time the additional validation adds value is where the cluster shares + // an external root (e.g. organization-wide root) with another distinct Consul + // cluster or PKI system. In this case, x509 Name Constraints can be added to + // enforce that Consul's CA can only validly sign or trust certs within the + // same trust-domain. Name constraints as enforced by TLS handshake also allow + // seamless rotation between trust domains thanks to cross-signing. + string TrustDomain = 2; + + // Roots is a list of root CA certs to trust. + repeated CARoot Roots = 3; + + // QueryMeta here is mainly used to contain the latest Raft Index that could + // be used to perform a blocking query. + common.QueryMeta QueryMeta = 4; +} + +message CARoot { + // ID is a globally unique ID (UUID) representing this CA root. + string ID = 1; + + // Name is a human-friendly name for this CA root. This value is + // opaque to Consul and is not used for anything internally. + string Name = 2; + + // SerialNumber is the x509 serial number of the certificate. + uint64 SerialNumber = 3; + + // SigningKeyID is the ID of the public key that corresponds to the private + // key used to sign leaf certificates. Is is the HexString format of the + // raw AuthorityKeyID bytes. + string SigningKeyID = 4; + + // ExternalTrustDomain is the trust domain this root was generated under. It + // is usually empty implying "the current cluster trust-domain". It is set + // only in the case that a cluster changes trust domain and then all old roots + // that are still trusted have the old trust domain set here. + // + // We currently DON'T validate these trust domains explicitly anywhere, see + // IndexedRoots.TrustDomain doc. We retain this information for debugging and + // future flexibility. + string ExternalTrustDomain = 5; + + // Time validity bounds. + google.protobuf.Timestamp NotBefore = 6; + google.protobuf.Timestamp NotAfter = 7; + + // RootCert is the PEM-encoded public certificate. + string RootCert = 8; + + // IntermediateCerts is a list of PEM-encoded intermediate certs to + // attach to any leaf certs signed by this CA. + repeated string IntermediateCerts = 9; + + // SigningCert is the PEM-encoded signing certificate and SigningKey + // is the PEM-encoded private key for the signing certificate. These + // may actually be empty if the CA plugin in use manages these for us. + string SigningCert = 10; + string SigningKey = 11; + + // Active is true if this is the current active CA. This must only + // be true for exactly one CA. For any method that modifies roots in the + // state store, tests should be written to verify that multiple roots + // cannot be active. + bool Active = 12; + + // RotatedOutAt is the time at which this CA was removed from the state. + // This will only be set on roots that have been rotated out from being the + // active root. + google.protobuf.Timestamp RotatedOutAt = 13; + + // PrivateKeyType is the type of the private key used to sign certificates. It + // may be "rsa" or "ec". This is provided as a convenience to avoid parsing + // the public key to from the certificate to infer the type. + string PrivateKeyType = 14; + + // PrivateKeyBits is the length of the private key used to sign certificates. + // This is provided as a convenience to avoid parsing the public key from the + // certificate to infer the type. + int32 PrivateKeyBits = 15; + + common.RaftIndex RaftIndex = 16; +} + +message IssuedCert { + // SerialNumber is the unique serial number for this certificate. + // This is encoded in standard hex separated by :. + string SerialNumber = 1; + + // CertPEM and PrivateKeyPEM are the PEM-encoded certificate and private + // key for that cert, respectively. This should not be stored in the + // state store, but is present in the sign API response. + string CertPEM = 2; + string PrivateKeyPEM = 3; + + // Service is the name of the service for which the cert was issued. + // ServiceURI is the cert URI value. + string Service = 4; + string ServiceURI = 5; + + // Agent is the name of the node for which the cert was issued. + // AgentURI is the cert URI value. + string Agent = 6; + string AgentURI = 7; + + // ValidAfter and ValidBefore are the validity periods for the + // certificate. + google.protobuf.Timestamp ValidAfter = 8; + google.protobuf.Timestamp ValidBefore = 9; + + // EnterpriseMeta is the Consul Enterprise specific metadata + common.EnterpriseMeta EnterpriseMeta = 10; + + common.RaftIndex RaftIndex = 11; +} \ No newline at end of file diff --git a/proto/translate.go b/proto/translate.go new file mode 100644 index 000000000..3619a0e6e --- /dev/null +++ b/proto/translate.go @@ -0,0 +1,56 @@ +package proto + +import ( + "reflect" + "time" + + "github.com/gogo/protobuf/types" +) + +var ( + tsType = reflect.TypeOf((*types.Timestamp)(nil)) + timePtrType = reflect.TypeOf((*time.Time)(nil)) + timeType = timePtrType.Elem() + mapStrInf = reflect.TypeOf((map[string]interface{})(nil)) +) + +// HookPBTimestampToTime is a mapstructure decode hook to translate a protobuf timestamp +// to a time.Time value +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 + } + + return data, nil +} + +// HookTimeToPBtimestamp is a mapstructure decode hook to translate a time.Time value to +// a protobuf Timestamp value. +func HookTimeToPBTimestamp(from, to reflect.Type, data interface{}) (interface{}, error) { + // Note that mapstructure doesn't do direct struct to struct conversion in this case. I + // still don't completely understand why converting the PB TS to time.Time does but + // I suspect it has something to do with the struct containing a concrete time.Time + // as opposed to a pointer to a time.Time. Regardless this path through mapstructure + // first will decode the concrete time.Time into a map[string]interface{} before + // eventually decoding that map[string]interface{} into the *types.Timestamp. One + // other note is that mapstructure ends up creating a new Value and sets it it to + // the time.Time value and thats what gets passed to us. That is why we end up + // seeing a *time.Time instead of a time.Time. + if from == timePtrType && to == mapStrInf { + ts := data.(*time.Time) + nanos := ts.UnixNano() + if nanos < 0 { + return map[string]interface{}{}, nil + } + + seconds := nanos / 1000000000 + nanos = nanos % 1000000000 + + return map[string]interface{}{ + "Seconds": seconds, + "Nanos": int32(nanos), + }, nil + } + return data, nil +} diff --git a/proto/translate_test.go b/proto/translate_test.go new file mode 100644 index 000000000..cd88d8933 --- /dev/null +++ b/proto/translate_test.go @@ -0,0 +1,65 @@ +package proto + +import ( + "testing" + "time" + + "github.com/gogo/protobuf/types" + "github.com/mitchellh/mapstructure" + + "github.com/stretchr/testify/require" +) + +type pbTSWrapper struct { + Timestamp *types.Timestamp +} + +type timeTSWrapper struct { + Timestamp time.Time +} + +func TestHookPBTimestampToTime(t *testing.T) { + in := pbTSWrapper{ + Timestamp: &types.Timestamp{ + Seconds: 1000, + Nanos: 42, + }, + } + + expected := timeTSWrapper{ + Timestamp: time.Unix(1000, 42), + } + + var actual timeTSWrapper + decoder, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ + DecodeHook: HookPBTimestampToTime, + Result: &actual, + }) + require.NoError(t, err) + require.NoError(t, decoder.Decode(in)) + + require.Equal(t, expected, actual) +} + +func TestHookTimeToPBTimestamp(t *testing.T) { + in := timeTSWrapper{ + Timestamp: time.Unix(999999, 123456), + } + + expected := pbTSWrapper{ + Timestamp: &types.Timestamp{ + Seconds: 999999, + Nanos: 123456, + }, + } + + 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) +}