From fa00fbfdd8c44eda7d1e48cb180929e0e3d43713 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 27 Nov 2019 15:41:45 -0600 Subject: [PATCH 01/27] client: enable nomad client to request and set SI tokens for tasks When a job is configured with Consul Connect aware tasks (i.e. sidecar), the Nomad Client should be able to request from Consul (through Nomad Server) Service Identity tokens specific to those tasks. --- client/allocrunner/alloc_runner.go | 23 ++- client/allocrunner/config.go | 3 + client/allocrunner/taskrunner/sids_hook.go | 193 ++++++++++++++++++ .../allocrunner/taskrunner/sids_hook_test.go | 122 +++++++++++ client/allocrunner/taskrunner/task_runner.go | 14 +- .../taskrunner/task_runner_hooks.go | 24 ++- .../taskrunner/task_runner_test.go | 73 ++++++- client/allocrunner/testing.go | 1 + client/client.go | 132 +++++++++--- client/client_test.go | 68 ++++++ client/consul/consul.go | 23 +++ client/consul/consul_testing.go | 1 - client/consul/identities.go | 32 +++ client/consul/identities_test.go | 31 +++ client/consul/identities_testing.go | 85 ++++++++ client/vaultclient/vaultclient_testing.go | 7 +- nomad/structs/structs.go | 23 ++- 17 files changed, 810 insertions(+), 45 deletions(-) create mode 100644 client/allocrunner/taskrunner/sids_hook.go create mode 100644 client/allocrunner/taskrunner/sids_hook_test.go create mode 100644 client/consul/identities.go create mode 100644 client/consul/identities_test.go create mode 100644 client/consul/identities_testing.go diff --git a/client/allocrunner/alloc_runner.go b/client/allocrunner/alloc_runner.go index eee412cc725..344b5204c61 100644 --- a/client/allocrunner/alloc_runner.go +++ b/client/allocrunner/alloc_runner.go @@ -62,6 +62,10 @@ type allocRunner struct { // registering services and checks consulClient consul.ConsulServiceAPI + // sidsClient is the client used by the service identity hook for + // managing SI tokens + sidsClient consul.ServiceIdentityAPI + // vaultClient is the used to manage Vault tokens vaultClient vaultclient.VaultClient @@ -157,6 +161,7 @@ func NewAllocRunner(config *Config) (*allocRunner, error) { alloc: alloc, clientConfig: config.ClientConfig, consulClient: config.Consul, + sidsClient: config.ConsulSI, vaultClient: config.Vault, tasks: make(map[string]*taskrunner.TaskRunner, len(tg.Tasks)), waitCh: make(chan struct{}), @@ -202,14 +207,16 @@ func NewAllocRunner(config *Config) (*allocRunner, error) { func (ar *allocRunner) initTaskRunners(tasks []*structs.Task) error { for _, task := range tasks { config := &taskrunner.Config{ - Alloc: ar.alloc, - ClientConfig: ar.clientConfig, - Task: task, - TaskDir: ar.allocDir.NewTaskDir(task.Name), - Logger: ar.logger, - StateDB: ar.stateDB, - StateUpdater: ar, - Consul: ar.consulClient, + Alloc: ar.alloc, + ClientConfig: ar.clientConfig, + Task: task, + TaskDir: ar.allocDir.NewTaskDir(task.Name), + Logger: ar.logger, + StateDB: ar.stateDB, + StateUpdater: ar, + Consul: ar.consulClient, + ConsulSI: ar.sidsClient, + Vault: ar.vaultClient, DeviceStatsReporter: ar.deviceStatsReporter, DeviceManager: ar.devicemanager, diff --git a/client/allocrunner/config.go b/client/allocrunner/config.go index 42cea978ea6..a9240b3a312 100644 --- a/client/allocrunner/config.go +++ b/client/allocrunner/config.go @@ -30,6 +30,9 @@ type Config struct { // Consul is the Consul client used to register task services and checks Consul consul.ConsulServiceAPI + // ConsulSI is the Consul client used to manage service identity tokens. + ConsulSI consul.ServiceIdentityAPI + // Vault is the Vault client to use to retrieve Vault tokens Vault vaultclient.VaultClient diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go new file mode 100644 index 00000000000..caa548cd9d8 --- /dev/null +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -0,0 +1,193 @@ +package taskrunner + +import ( + "context" + "io/ioutil" + "os" + "path/filepath" + "sync" + "time" + + hclog "github.com/hashicorp/go-hclog" + "github.com/hashicorp/nomad/client/allocrunner/interfaces" + "github.com/hashicorp/nomad/client/consul" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/pkg/errors" +) + +const ( + // the name of this hook, used in logs + sidsHookName = "consul_sids" + + // sidsBackoffBaseline is the baseline time for exponential backoff when + // attempting to retrieve a Consul SI token + sidsBackoffBaseline = 5 * time.Second + + // sidsBackoffLimit is the limit of the exponential backoff when attempting + // to retrieve a Consul SI token + sidsBackoffLimit = 3 * time.Minute + + // sidsTokenFile is the name of the file holding the Consul SI token inside + // the task's secret directory + sidsTokenFile = "sids_token" + + // sidsTokenFilePerms is the level of file permissions granted on the file + // in the secrets directory for the task + sidsTokenFilePerms = 0440 +) + +type sidsHookConfig struct { + alloc *structs.Allocation + task *structs.Task + sidsClient consul.ServiceIdentityAPI + logger hclog.Logger +} + +// Service Identities hook for managing SI tokens of connect enabled tasks. +type sidsHook struct { + alloc *structs.Allocation + taskName string + sidsClient consul.ServiceIdentityAPI + logger hclog.Logger + + lock sync.Mutex + firstRun bool +} + +func newSIDSHook(c sidsHookConfig) *sidsHook { + return &sidsHook{ + alloc: c.alloc, + taskName: c.task.Name, + sidsClient: c.sidsClient, + logger: c.logger.Named(sidsHookName), + firstRun: true, + } +} + +func (h *sidsHook) Name() string { + return sidsHookName +} + +func (h *sidsHook) Prestart( + ctx context.Context, + req *interfaces.TaskPrestartRequest, + _ *interfaces.TaskPrestartResponse) error { + + h.lock.Lock() + defer h.lock.Unlock() + + // do nothing if we have already done things + if h.earlyExit() { + return nil + } + + // optimistically try to recover token from disk + token, err := h.recoverToken(req.TaskDir.SecretsDir) + if err != nil { + return err + } + + // need to ask for a new SI token & persist it to disk + if token == "" { + if token, err = h.deriveSIToken(ctx); err != nil { + return err + } + if err := h.writeToken(req.TaskDir.SecretsDir, token); err != nil { + return err + } + } + + return nil +} + +// earlyExit returns true if the Prestart hook has already been executed during +// the instantiation of this task runner. +// +// assumes h is locked +func (h *sidsHook) earlyExit() bool { + if h.firstRun { + h.firstRun = false + return false + } + return true +} + +// writeToken writes token into the secrets directory for the task. +func (h *sidsHook) writeToken(dir string, token string) error { + tokenPath := filepath.Join(dir, sidsTokenFile) + if err := ioutil.WriteFile(tokenPath, []byte(token), sidsTokenFilePerms); err != nil { + return errors.Wrap(err, "failed to write SI token") + } + return nil +} + +// recoverToken returns the token saved to disk in the secrets directory for the +// task if it exists, or the empty string if the file does not exist. an error +// is returned only for some other (e.g. disk IO) error. +func (h *sidsHook) recoverToken(dir string) (string, error) { + tokenPath := filepath.Join(dir, sidsTokenFile) + token, err := ioutil.ReadFile(tokenPath) + if err != nil { + if !os.IsNotExist(err) { + return "", errors.Wrap(err, "failed to recover SI token") + } + h.logger.Trace("no pre-existing SI token to recover", "task", h.taskName) + return "", nil // token file does not exist yet + } + h.logger.Trace("recovered pre-existing SI token", "task", h.taskName) + return string(token), nil +} + +// deriveSIToken spawns and waits on a goroutine which will make attempts to +// derive an SI token until a token is successfully created, or ctx is signaled +// done. +func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { + tokenCh := make(chan string) + + // keep trying to get the token in the background + go h.tryDerive(ctx, tokenCh) + + // wait until we get a token, or we get a signal to quit + for { + select { + case token := <-tokenCh: + return token, nil + case <-ctx.Done(): + return "", ctx.Err() + } + } +} + +// tryDerive loops forever until a token is created, or ctx is done. +func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- string) { + for i := 0; backoff(ctx, i); i++ { + tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{h.taskName}) + if err != nil { + h.logger.Warn("failed to derive SI token", "attempt", i, "error", err) + continue + } + ch <- tokens[h.taskName] + return + } +} + +func backoff(ctx context.Context, i int) bool { + next := computeBackoff(i) + select { + case <-ctx.Done(): + return false + case <-time.After(next): + return true + } +} + +func computeBackoff(attempt int) time.Duration { + switch { + case attempt <= 0: + return 0 + case attempt >= 4: + return sidsBackoffLimit + default: + return (1 << (2 * uint(attempt))) * sidsBackoffBaseline + } +} diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go new file mode 100644 index 00000000000..cbb056634dc --- /dev/null +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -0,0 +1,122 @@ +package taskrunner + +import ( + "context" + "io/ioutil" + "os" + "testing" + "time" + + "github.com/hashicorp/nomad/client/allocrunner/interfaces" + "github.com/hashicorp/nomad/client/consul" + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +var _ interfaces.TaskPrestartHook = (*sidsHook)(nil) + +func tmpDir(t *testing.T) string { + dir, err := ioutil.TempDir("", "sids-") + require.NoError(t, err) + return dir +} + +func cleanupDir(t *testing.T, dir string) { + err := os.RemoveAll(dir) + require.NoError(t, err) +} + +func TestSIDSHook_recoverToken(t *testing.T) { + t.Parallel() + + r := require.New(t) + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + + h := newSIDSHook(sidsHookConfig{ + task: &structs.Task{Name: "task1"}, + logger: testlog.HCLogger(t), + }) + + expected := "12345678-1234-1234-1234-1234567890" + err := h.writeToken(secrets, expected) + r.NoError(err) + + token, err := h.recoverToken(secrets) + r.NoError(err) + r.Equal(expected, token) +} + +func TestSIDSHook_recoverToken_empty(t *testing.T) { + t.Parallel() + + r := require.New(t) + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + + h := newSIDSHook(sidsHookConfig{ + task: &structs.Task{Name: "task1"}, + logger: testlog.HCLogger(t), + }) + + token, err := h.recoverToken(secrets) + r.NoError(err) + r.Empty(token) +} + +func TestSIDSHook_deriveSIToken(t *testing.T) { + t.Parallel() + + r := require.New(t) + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + + h := newSIDSHook(sidsHookConfig{ + alloc: &structs.Allocation{ID: "a1"}, + task: &structs.Task{Name: "task1"}, + logger: testlog.HCLogger(t), + sidsClient: consul.NewMockServiceIdentitiesClient(), + }) + + ctx := context.Background() + token, err := h.deriveSIToken(ctx) + r.NoError(err) + r.True(helper.IsUUID(token)) +} + +func TestSIDSHook_computeBackoff(t *testing.T) { + t.Parallel() + + try := func(i int, exp time.Duration) { + result := computeBackoff(i) + require.Equal(t, exp, result) + } + + try(0, time.Duration(0)) + try(1, 20*time.Second) + try(2, 80*time.Second) + try(3, 320*time.Second) + try(4, sidsBackoffLimit) +} + +func TestSIDSHook_backoff(t *testing.T) { + t.Parallel() + r := require.New(t) + + ctx := context.Background() + stop := !backoff(ctx, 0) + r.False(stop) +} + +func TestSIDSHook_backoffKilled(t *testing.T) { + t.Parallel() + r := require.New(t) + + ctx, cancel := context.WithTimeout(context.Background(), 1) + defer cancel() + + stop := !backoff(ctx, 1000) + r.True(stop) +} diff --git a/client/allocrunner/taskrunner/task_runner.go b/client/allocrunner/taskrunner/task_runner.go index 9f6e5ac34cb..52721f12dad 100644 --- a/client/allocrunner/taskrunner/task_runner.go +++ b/client/allocrunner/taskrunner/task_runner.go @@ -50,7 +50,7 @@ const ( // giving up and potentially leaking resources. killFailureLimit = 5 - // triggerUpdatechCap is the capacity for the triggerUpdateCh used for + // triggerUpdateChCap is the capacity for the triggerUpdateCh used for // triggering updates. It should be exactly 1 as even if multiple // updates have come in since the last one was handled, we only need to // handle the last one. @@ -158,6 +158,10 @@ type TaskRunner struct { // registering services and checks consulClient consul.ConsulServiceAPI + // sidsClient is the client used by the service identity hook for managing + // service identity tokens + siClient consul.ServiceIdentityAPI + // vaultClient is the client to use to derive and renew Vault tokens vaultClient vaultclient.VaultClient @@ -210,11 +214,16 @@ type TaskRunner struct { type Config struct { Alloc *structs.Allocation ClientConfig *config.Config - Consul consul.ConsulServiceAPI Task *structs.Task TaskDir *allocdir.TaskDir Logger log.Logger + // Consul is the client to use for managing Consul service registrations + Consul consul.ConsulServiceAPI + + // ConsulSI is the client to use for managing Consul SI tokens + ConsulSI consul.ServiceIdentityAPI + // Vault is the client to use to derive and renew Vault tokens Vault vaultclient.VaultClient @@ -271,6 +280,7 @@ func NewTaskRunner(config *Config) (*TaskRunner, error) { taskLeader: config.Task.Leader, envBuilder: envBuilder, consulClient: config.Consul, + siClient: config.ConsulSI, vaultClient: config.Vault, state: tstate, localState: state.NewLocalState(), diff --git a/client/allocrunner/taskrunner/task_runner_hooks.go b/client/allocrunner/taskrunner/task_runner_hooks.go index 2f5723197e5..46c5b5e8958 100644 --- a/client/allocrunner/taskrunner/task_runner_hooks.go +++ b/client/allocrunner/taskrunner/task_runner_hooks.go @@ -45,7 +45,7 @@ func (h *hookResources) getMounts() []*drivers.MountConfig { return h.Mounts } -// initHooks intializes the tasks hooks. +// initHooks initializes the tasks hooks. func (tr *TaskRunner) initHooks() { hookLogger := tr.logger.Named("task_hook") task := tr.Task() @@ -96,7 +96,7 @@ func (tr *TaskRunner) initHooks() { })) } - // If there are any services, add the hook + // If there are any services, add the service hook if len(task.Services) != 0 { tr.runnerHooks = append(tr.runnerHooks, newServiceHook(serviceHookConfig{ alloc: tr.Alloc(), @@ -107,6 +107,15 @@ func (tr *TaskRunner) initHooks() { })) } + if usesConnect(tr.alloc.Job.LookupTaskGroup(tr.alloc.TaskGroup)) { + tr.runnerHooks = append(tr.runnerHooks, newSIDSHook(sidsHookConfig{ + alloc: tr.Alloc(), + task: tr.Task(), + sidsClient: tr.siClient, + logger: hookLogger, + })) + } + // If there are any script checks, add the hook scriptCheckHook := newScriptCheckHook(scriptCheckHookConfig{ alloc: tr.Alloc(), @@ -117,6 +126,15 @@ func (tr *TaskRunner) initHooks() { tr.runnerHooks = append(tr.runnerHooks, scriptCheckHook) } +func usesConnect(tg *structs.TaskGroup) bool { + for _, service := range tg.Services { + if service.Connect != nil { + return true + } + } + return false +} + func (tr *TaskRunner) emitHookError(err error, hookName string) { var taskEvent *structs.TaskEvent if herr, ok := err.(*hookError); ok { @@ -131,7 +149,7 @@ func (tr *TaskRunner) emitHookError(err error, hookName string) { // prestart is used to run the runners prestart hooks. func (tr *TaskRunner) prestart() error { - // Determine if the allocation is terminaland we should avoid running + // Determine if the allocation is terminal and we should avoid running // prestart hooks. alloc := tr.Alloc() if alloc.TerminalStatus() { diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 25124c3fdbf..d2dad453478 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -97,10 +97,11 @@ func testTaskRunnerConfig(t *testing.T, alloc *structs.Allocation, taskName stri conf := &Config{ Alloc: alloc, ClientConfig: clientConf, - Consul: consulapi.NewMockConsulServiceClient(t, logger), Task: thisTask, TaskDir: taskDir, Logger: clientConf.Logger, + Consul: consulapi.NewMockConsulServiceClient(t, logger), + ConsulSI: consulapi.NewMockServiceIdentitiesClient(), Vault: vaultclient.NewMockVaultClient(), StateDB: cstate.NoopDB{}, StateUpdater: NewMockTaskStateUpdater(), @@ -1085,6 +1086,76 @@ func TestTaskRunner_CheckWatcher_Restart(t *testing.T) { require.True(t, state.Failed, pretty.Sprint(state)) } +func TestTaskRunner_BlockForSIDS(t *testing.T) { + t.Parallel() + r := require.New(t) + + // setup a connect enabled batch job that wants to exit immediately, which + // makes testing the prestart lifecycle easier + alloc := mock.BatchAlloc() + tg := alloc.Job.TaskGroups[0] + tg.Tasks[0].Config = map[string]interface{}{"run_for": "0s"} + tg.Services = []*structs.Service{{ + Name: "testconnect", + PortLabel: "9999", + Connect: &structs.ConsulConnect{ + SidecarService: &structs.ConsulSidecarService{}, + }}, + } + taskName := tg.Tasks[0].Name + + trConfig, cleanup := testTaskRunnerConfig(t, alloc, taskName) + defer cleanup() + + // control when we get a Consul SI token + token := "12345678-1234-1234-1234-1234567890" + waitCh := make(chan struct{}) + deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { + <-waitCh + return map[string]string{taskName: token}, nil + } + siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) + siClient.DeriveTokenFn = deriveFn + + // start the task runner + tr, err := NewTaskRunner(trConfig) + r.NoError(err) + defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) + go tr.Run() + + // assert task runner blocks on SI token + select { + case <-tr.WaitCh(): + r.Fail("task_runner exited before si unblocked") + case <-time.After(100 * time.Millisecond): + } + + // assert task state is still pending + r.Equal(structs.TaskStatePending, tr.TaskState().State) + + // unblock service identity token + close(waitCh) + + // task runner should exit now that it has been unblocked and it is a batch + // job with a zero sleep time + select { + case <-tr.WaitCh(): + case <-time.After(15 * time.Second * time.Duration(testutil.TestMultiplier())): + r.Fail("timed out waiting for batch task to exist") + } + + // assert task exited successfully + finalState := tr.TaskState() + r.Equal(structs.TaskStateDead, finalState.State) + r.False(finalState.Failed) + + // assert the token is on disk + tokenPath := filepath.Join(trConfig.TaskDir.SecretsDir, sidsTokenFile) + data, err := ioutil.ReadFile(tokenPath) + r.NoError(err) + r.Equal(token, string(data)) +} + // TestTaskRunner_BlockForVault asserts tasks do not start until a vault token // is derived. func TestTaskRunner_BlockForVault(t *testing.T) { diff --git a/client/allocrunner/testing.go b/client/allocrunner/testing.go index 75806644bb8..02751687bb4 100644 --- a/client/allocrunner/testing.go +++ b/client/allocrunner/testing.go @@ -60,6 +60,7 @@ func testAllocRunnerConfig(t *testing.T, alloc *structs.Allocation) (*Config, fu ClientConfig: clientConf, StateDB: state.NoopDB{}, Consul: consul.NewMockConsulServiceClient(t, clientConf.Logger), + ConsulSI: consul.NewMockServiceIdentitiesClient(), Vault: vaultclient.NewMockVaultClient(), StateUpdater: &MockStateUpdater{}, PrevAllocWatcher: allocwatcher.NoopPrevAlloc{}, diff --git a/client/client.go b/client/client.go index fe7447dcacf..0b5346dc0fa 100644 --- a/client/client.go +++ b/client/client.go @@ -1,7 +1,6 @@ package client import ( - "errors" "fmt" "io/ioutil" "net" @@ -46,6 +45,7 @@ import ( "github.com/hashicorp/nomad/plugins/device" "github.com/hashicorp/nomad/plugins/drivers" vaultapi "github.com/hashicorp/vault/api" + "github.com/pkg/errors" "github.com/shirou/gopsutil/host" ) @@ -236,6 +236,10 @@ type Client struct { // Shutdown() blocks on Wait() after closing shutdownCh. shutdownGroup group + // tokensClient is Nomad Client's custom Consul client for requesting Consul + // Service Identity tokens through Nomad Server. + tokensClient consulApi.ServiceIdentityAPI + // vaultClient is used to interact with Vault for token and secret renewals vaultClient vaultclient.VaultClient @@ -445,6 +449,10 @@ func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulServic } } + if err := c.setupConsulTokenClient(); err != nil { + return nil, errors.Wrap(err, "failed to setup consul tokens client") + } + // Setup the vault client for token and secret renewals if err := c.setupVaultClient(); err != nil { return nil, fmt.Errorf("failed to setup vault client: %v", err) @@ -1042,6 +1050,7 @@ func (c *Client) restoreState() error { StateUpdater: c, DeviceStatsReporter: c, Consul: c.consulService, + ConsulSI: c.tokensClient, // todo(shoenig), keep plumbing! Vault: c.vaultClient, PrevAllocWatcher: prevAllocWatcher, PrevAllocMigrator: prevAllocMigrator, @@ -2291,6 +2300,7 @@ func (c *Client) addAlloc(alloc *structs.Allocation, migrateToken string) error ClientConfig: c.configCopy, StateDB: c.stateDB, Consul: c.consulService, + ConsulSI: c.tokensClient, // todo(shoenig), keep plumbing! Vault: c.vaultClient, StateUpdater: c, DeviceStatsReporter: c, @@ -2313,6 +2323,14 @@ func (c *Client) addAlloc(alloc *structs.Allocation, migrateToken string) error return nil } +// setupConsulTokenClient configures a tokenClient for managing consul service +// identity tokens. +func (c *Client) setupConsulTokenClient() error { + tc := consulApi.NewIdentitiesClient(c.logger, c.deriveSIToken) + c.tokensClient = tc + return nil +} + // setupVaultClient creates an object to periodically renew tokens and secrets // with vault. func (c *Client) setupVaultClient() error { @@ -2338,33 +2356,10 @@ func (c *Client) setupVaultClient() error { // client and returns a map of unwrapped tokens, indexed by the task name. func (c *Client) deriveToken(alloc *structs.Allocation, taskNames []string, vclient *vaultapi.Client) (map[string]string, error) { vlogger := c.logger.Named("vault") - if alloc == nil { - return nil, fmt.Errorf("nil allocation") - } - - if taskNames == nil || len(taskNames) == 0 { - return nil, fmt.Errorf("missing task names") - } - - group := alloc.Job.LookupTaskGroup(alloc.TaskGroup) - if group == nil { - return nil, fmt.Errorf("group name in allocation is not present in job") - } - verifiedTasks := []string{} - // Check if the given task names actually exist in the allocation - for _, taskName := range taskNames { - found := false - for _, task := range group.Tasks { - if task.Name == taskName { - found = true - } - } - if !found { - vlogger.Error("task not found in the allocation", "task_name", taskName) - return nil, fmt.Errorf("task %q not found in the allocation", taskName) - } - verifiedTasks = append(verifiedTasks, taskName) + verifiedTasks, err := verifiedTasks(vlogger, alloc, taskNames) + if err != nil { + return nil, err } // DeriveVaultToken of nomad server can take in a set of tasks and @@ -2439,6 +2434,89 @@ func (c *Client) deriveToken(alloc *structs.Allocation, taskNames []string, vcli return unwrappedTokens, nil } +// deriveSIToken takes an allocation and a set of tasks and derives Consul +// Service Identity tokens for each of the tasks by requesting them from the +// Nomad Server. +func (c *Client) deriveSIToken(alloc *structs.Allocation, taskNames []string) (map[string]string, error) { + tasks, err := verifiedTasks(c.logger, alloc, taskNames) + if err != nil { + return nil, err + } + + req := &structs.DeriveSITokenRequest{ + NodeID: c.NodeID(), + AllocID: alloc.ID, + Tasks: tasks, + QueryOptions: structs.QueryOptions{Region: c.Region()}, + } + + // Nicely ask Nomad Server for the tokens. + var resp structs.DeriveSITokenResponse + if err := c.RPC("Node.DeriveSIToken", &req, &resp); err != nil { + c.logger.Error("error making derive token RPC", "error", err) + return nil, fmt.Errorf("DeriveSIToken RPC failed: %v", err) + } + if err := resp.Error; err != nil { + c.logger.Error("error deriving SI tokens", "error", err) + return nil, structs.NewWrappedServerError(err) + } + if len(resp.Tokens) == 0 { + c.logger.Error("error deriving SI tokens", "error", "invalid_response") + return nil, fmt.Errorf("failed to derive SI tokens: invalid response") + } + + // NOTE: Unlike with the Vault integration, Nomad Server replies with the + // actual Consul SI token (.SecretID), because otherwise each Nomad + // Client would need to be blessed with 'acl:write' permissions to read the + // secret value given the .AccessorID, which does not fit well in the Consul + // security model. + // + // https://www.consul.io/api/acl/tokens.html#read-a-token + // https://www.consul.io/docs/internals/security.html + + m := helper.CopyMapStringString(resp.Tokens) + return m, nil +} + +// verifiedTasks asserts each task in taskNames actually exists in the given alloc, +// otherwise an error is returned. +func verifiedTasks(logger hclog.Logger, alloc *structs.Allocation, taskNames []string) ([]string, error) { + if alloc == nil { + return nil, fmt.Errorf("nil allocation") + } + + if len(taskNames) == 0 { + return nil, fmt.Errorf("missing task names") + } + + group := alloc.Job.LookupTaskGroup(alloc.TaskGroup) + if group == nil { + return nil, fmt.Errorf("group name in allocation is not present in job") + } + + verifiedTasks := make([]string, 0, len(taskNames)) + + // confirm the requested task names actually exist in the allocation + for _, taskName := range taskNames { + if !taskIsPresent(taskName, group.Tasks) { + logger.Error("task not found in the allocation", "task_name", taskName) + return nil, fmt.Errorf("task %q not found in allocation", taskName) + } + verifiedTasks = append(verifiedTasks, taskName) + } + + return verifiedTasks, nil +} + +func taskIsPresent(taskName string, tasks []*structs.Task) bool { + for _, task := range tasks { + if task.Name == taskName { + return true + } + } + return false +} + // triggerDiscovery causes a Consul discovery to begin (if one hasn't already) func (c *Client) triggerDiscovery() { select { diff --git a/client/client_test.go b/client/client_test.go index 204c067ece0..95bf51069ad 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -1620,3 +1620,71 @@ func TestClient_hasLocalState(t *testing.T) { require.True(t, c.hasLocalState(alloc)) }) } + +func Test_verifiedTasks(t *testing.T) { + t.Parallel() + logger := testlog.HCLogger(t) + + // produce a result and check against expected tasks and/or error output + try := func(t *testing.T, a *structs.Allocation, tasks, expTasks []string, expErr string) { + result, err := verifiedTasks(logger, a, tasks) + if expErr != "" { + require.EqualError(t, err, expErr) + } else { + require.NoError(t, err) + require.Equal(t, expTasks, result) + } + } + + // create an alloc with TaskGroup=g1, tasks configured given g1Tasks + alloc := func(g1Tasks []string) *structs.Allocation { + var tasks []*structs.Task + for _, taskName := range g1Tasks { + tasks = append(tasks, &structs.Task{Name: taskName}) + } + + return &structs.Allocation{ + Job: &structs.Job{ + TaskGroups: []*structs.TaskGroup{ + {Name: "g0", Tasks: []*structs.Task{{Name: "g0t1"}}}, + {Name: "g1", Tasks: tasks}, + }, + }, + TaskGroup: "g1", + } + } + + t.Run("nil alloc", func(t *testing.T) { + tasks := []string{"g1t1"} + try(t, nil, tasks, nil, "nil allocation") + }) + + t.Run("missing task names", func(t *testing.T) { + var tasks []string + tgTasks := []string{"g1t1"} + try(t, alloc(tgTasks), tasks, nil, "missing task names") + }) + + t.Run("missing group", func(t *testing.T) { + tasks := []string{"g1t1"} + a := alloc(tasks) + a.TaskGroup = "other" + try(t, a, tasks, nil, "group name in allocation is not present in job") + }) + + t.Run("nonexistent task", func(t *testing.T) { + tasks := []string{"missing"} + try(t, alloc([]string{"task1"}), tasks, nil, `task "missing" not found in allocation`) + }) + + t.Run("matching task", func(t *testing.T) { + tasks := []string{"g1t1"} + try(t, alloc(tasks), tasks, tasks, "") + }) + + t.Run("matching task subset", func(t *testing.T) { + tasks := []string{"g1t1", "g1t3"} + tgTasks := []string{"g1t1", "g1t2", "g1t3"} + try(t, alloc(tgTasks), tasks, tasks, "") + }) +} diff --git a/client/consul/consul.go b/client/consul/consul.go index 251165ff35c..f8348c2202e 100644 --- a/client/consul/consul.go +++ b/client/consul/consul.go @@ -2,14 +2,37 @@ package consul import ( "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/nomad/structs" ) // ConsulServiceAPI is the interface the Nomad Client uses to register and // remove services and checks from Consul. type ConsulServiceAPI interface { + // RegisterWorkload with Consul. Adds all service entries and checks to Consul. RegisterWorkload(*consul.WorkloadServices) error + + // RemoveWorkload from Consul. Removes all service entries and checks. RemoveWorkload(*consul.WorkloadServices) + + // UpdateWorkload in Consul. Does not alter the service if only checks have + // changed. UpdateWorkload(old, newTask *consul.WorkloadServices) error + + // AllocRegistrations returns the registrations for the given allocation. AllocRegistrations(allocID string) (*consul.AllocRegistration, error) + + // UpdateTTL is used to update the TTL of a check. UpdateTTL(id, output, status string) error } + +// TokenDeriverFunc takes an allocation and a set of tasks and derives a +// service identity token for each. Requests go through nomad server. +type TokenDeriverFunc func(*structs.Allocation, []string) (map[string]string, error) + +// ServiceIdentityAPI is the interface the Nomad Client uses to request Consul +// Service Identity tokens through Nomad Server. +type ServiceIdentityAPI interface { + // DeriveSITokens contacts the nomad server and requests consul service + // identity tokens be generated for tasks in the allocation. + DeriveSITokens(alloc *structs.Allocation, tasks []string) (map[string]string, error) +} diff --git a/client/consul/consul_testing.go b/client/consul/consul_testing.go index 75307eae810..0384e4c9541 100644 --- a/client/consul/consul_testing.go +++ b/client/consul/consul_testing.go @@ -6,7 +6,6 @@ import ( "time" log "github.com/hashicorp/go-hclog" - "github.com/hashicorp/nomad/command/agent/consul" testing "github.com/mitchellh/go-testing-interface" ) diff --git a/client/consul/identities.go b/client/consul/identities.go new file mode 100644 index 00000000000..e07dfaf33dd --- /dev/null +++ b/client/consul/identities.go @@ -0,0 +1,32 @@ +package consul + +import ( + "github.com/hashicorp/go-hclog" + "github.com/hashicorp/nomad/nomad/structs" +) + +// Implementation of ServiceIdentityAPI used to interact with Nomad Server from +// Nomad Client for acquiring Consul Service Identity tokens. +// +// This client is split from the other consul client(s) to avoid a circular +// dependency between themselves and client.Client +type identitiesClient struct { + tokenDeriver TokenDeriverFunc + logger hclog.Logger +} + +func NewIdentitiesClient(logger hclog.Logger, tokenDeriver TokenDeriverFunc) *identitiesClient { + return &identitiesClient{ + tokenDeriver: tokenDeriver, + logger: logger, + } +} + +func (c *identitiesClient) DeriveSITokens(alloc *structs.Allocation, tasks []string) (map[string]string, error) { + tokens, err := c.tokenDeriver(alloc, tasks) + if err != nil { + c.logger.Error("error deriving SI token", "error", err, "alloc_id", alloc.ID, "task_names", tasks) + return nil, err + } + return tokens, nil +} diff --git a/client/consul/identities_test.go b/client/consul/identities_test.go new file mode 100644 index 00000000000..e56000d4a71 --- /dev/null +++ b/client/consul/identities_test.go @@ -0,0 +1,31 @@ +package consul + +import ( + "errors" + "testing" + + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +func TestCSI_DeriveTokens(t *testing.T) { + logger := testlog.HCLogger(t) + dFunc := func(alloc *structs.Allocation, taskNames []string) (map[string]string, error) { + return map[string]string{"a": "b"}, nil + } + tc := NewIdentitiesClient(logger, dFunc) + tokens, err := tc.DeriveSITokens(nil, nil) + require.NoError(t, err) + require.Equal(t, map[string]string{"a": "b"}, tokens) +} + +func TestCSI_DeriveTokens_error(t *testing.T) { + logger := testlog.HCLogger(t) + dFunc := func(alloc *structs.Allocation, taskNames []string) (map[string]string, error) { + return nil, errors.New("some failure") + } + tc := NewIdentitiesClient(logger, dFunc) + _, err := tc.DeriveSITokens(&structs.Allocation{ID: "a1"}, nil) + require.Error(t, err) +} diff --git a/client/consul/identities_testing.go b/client/consul/identities_testing.go new file mode 100644 index 00000000000..2d4258d2535 --- /dev/null +++ b/client/consul/identities_testing.go @@ -0,0 +1,85 @@ +package consul + +import ( + "fmt" + "sync" + + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/structs" +) + +// MockServiceIdentitiesClient is used for testing the client for managing consul service +// identity tokens. +type MockServiceIdentitiesClient struct { + // deriveTokenErrors maps an allocation ID and tasks to an error when the + // token is derived + deriveTokenErrors map[string]map[string]error + + // DeriveTokenFn allows the caller to control the DeriveToken function. If + // not set an error is returned if found in DeriveTokenErrors and otherwise + // a token is generated and returned + DeriveTokenFn TokenDeriverFunc + + // lock around everything + lock sync.Mutex +} + +var _ ServiceIdentityAPI = (*MockServiceIdentitiesClient)(nil) + +// NewMockServiceIdentitiesClient returns a MockServiceIdentitiesClient for testing. +func NewMockServiceIdentitiesClient() *MockServiceIdentitiesClient { + return &MockServiceIdentitiesClient{ + deriveTokenErrors: make(map[string]map[string]error), + } +} + +func (mtc *MockServiceIdentitiesClient) DeriveSITokens(alloc *structs.Allocation, tasks []string) (map[string]string, error) { + mtc.lock.Lock() + defer mtc.lock.Unlock() + + fmt.Println("MockServiceIdentitiesClient.DeriveSITokens running!") + + // if the DeriveTokenFn is explicitly set, use that + if mtc.DeriveTokenFn != nil { + return mtc.DeriveTokenFn(alloc, tasks) + } + + // generate a token for each task, unless the mock has an error ready for + // one or more of the tasks in which case return that + tokens := make(map[string]string, len(tasks)) + for _, task := range tasks { + if m, ok := mtc.deriveTokenErrors[alloc.ID]; ok { + if err, ok := m[task]; ok { + return nil, err + } + } + tokens[task] = uuid.Generate() + } + return tokens, nil +} + +func (mtc *MockServiceIdentitiesClient) SetDeriveTokenError(allocID string, tasks []string, err error) { + mtc.lock.Lock() + defer mtc.lock.Unlock() + + if _, ok := mtc.deriveTokenErrors[allocID]; !ok { + mtc.deriveTokenErrors[allocID] = make(map[string]error, 10) + } + + for _, task := range tasks { + mtc.deriveTokenErrors[allocID][task] = err + } +} + +func (mtc *MockServiceIdentitiesClient) DeriveTokenErrors() map[string]map[string]error { + mtc.lock.Lock() + defer mtc.lock.Unlock() + + m := make(map[string]map[string]error) + for aID, tasks := range mtc.deriveTokenErrors { + for task, err := range tasks { + m[aID][task] = err + } + } + return m +} diff --git a/client/vaultclient/vaultclient_testing.go b/client/vaultclient/vaultclient_testing.go index fea31a31d30..08adaa119ea 100644 --- a/client/vaultclient/vaultclient_testing.go +++ b/client/vaultclient/vaultclient_testing.go @@ -67,6 +67,7 @@ func (vc *MockVaultClient) SetDeriveTokenError(allocID string, tasks []string, e vc.deriveTokenErrors = make(map[string]map[string]error, 10) } + // todo(shoenig): this seems like a bug if _, ok := vc.renewTokenErrors[allocID]; !ok { vc.deriveTokenErrors[allocID] = make(map[string]error, 10) } @@ -111,8 +112,10 @@ func (vc *MockVaultClient) StopRenewToken(token string) error { return nil } -func (vc *MockVaultClient) Start() {} -func (vc *MockVaultClient) Stop() {} +func (vc *MockVaultClient) Start() {} + +func (vc *MockVaultClient) Stop() {} + func (vc *MockVaultClient) GetConsulACL(string, string) (*vaultapi.Secret, error) { return nil, nil } // StoppedTokens tracks the tokens that have stopped renewing diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 90f48c33165..5aa0a1370f3 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -920,7 +920,28 @@ type DeriveVaultTokenResponse struct { Tasks map[string]string // Error stores any error that occurred. Errors are stored here so we can - // communicate whether it is retriable + // communicate whether it is retryable + Error *RecoverableError + + QueryMeta +} + +// DeriveSITokenRequest is used to request Consul Service Identity tokens from +// the Nomad Server for the named tasks in the given allocation. +type DeriveSITokenRequest struct { + NodeID string + SecretID string + AllocID string + Tasks []string + QueryOptions +} + +type DeriveSITokenResponse struct { + // Tokens maps from Task Name to its associated SI token + Tokens map[string]string + + // Error stores any error that occurred. Errors are stored here so we can + // communicate whether it is retryable Error *RecoverableError QueryMeta From ad3afe5c44a4f94223a5d11313a894e2dc71c9e9 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Fri, 6 Dec 2019 14:46:46 -0600 Subject: [PATCH 02/27] nomad: proxy requests for Service Identity tokens between Clients and Consul Nomad jobs may be configured with a TaskGroup which contains a Service definition that is Consul Connect enabled. These service definitions end up establishing a Consul Connect Proxy Task (e.g. envoy, by default). In the case where Consul ACLs are enabled, a Service Identity token is required for these tasks to run & connect, etc. This changeset enables the Nomad Server to recieve RPC requests for the derivation of SI tokens on behalf of instances of Consul Connect using Tasks. Those tokens are then relayed back to the requesting Client, which then injects the tokens in the secrets directory of the Task. --- client/allocrunner/alloc_runner.go | 19 +- client/allocrunner/taskrunner/sids_hook.go | 79 +++- .../allocrunner/taskrunner/sids_hook_test.go | 9 +- .../taskrunner/task_runner_hooks.go | 12 +- .../taskrunner/task_runner_test.go | 157 ++++++- client/client.go | 5 +- client/consul/identities_testing.go | 3 - client/vaultclient/vaultclient_testing.go | 3 +- command/agent/agent.go | 10 +- command/agent/command.go | 9 +- command/agent/config.go | 2 +- command/agent/consul/acl_testing.go | 230 ++++++++++ command/agent/consul/catalog_testing.go | 10 +- command/agent/consul/client.go | 14 + nomad/consul.go | 301 +++++++++++++ nomad/consul_test.go | 339 +++++++++++++++ nomad/fsm.go | 66 +++ nomad/fsm_test.go | 73 ++++ nomad/job_endpoint.go | 40 +- nomad/job_endpoint_hook_connect.go | 2 +- nomad/job_endpoint_test.go | 81 ++++ nomad/leader.go | 59 ++- nomad/leader_test.go | 37 +- nomad/mock/mock.go | 109 +++++ nomad/node_endpoint.go | 399 +++++++++++++++--- nomad/node_endpoint_test.go | 211 +++++++++ nomad/server.go | 22 +- nomad/state/schema.go | 41 +- nomad/state/state_store.go | 139 ++++++ nomad/state/state_store_test.go | 195 +++++++++ nomad/structs/config/consul.go | 13 +- nomad/structs/service_identities.go | 64 +++ nomad/structs/structs.go | 62 ++- nomad/structs/structs_test.go | 62 +++ nomad/testing.go | 4 +- 35 files changed, 2701 insertions(+), 180 deletions(-) create mode 100644 command/agent/consul/acl_testing.go create mode 100644 nomad/consul.go create mode 100644 nomad/consul_test.go create mode 100644 nomad/structs/service_identities.go diff --git a/client/allocrunner/alloc_runner.go b/client/allocrunner/alloc_runner.go index 344b5204c61..00515439ac8 100644 --- a/client/allocrunner/alloc_runner.go +++ b/client/allocrunner/alloc_runner.go @@ -207,16 +207,15 @@ func NewAllocRunner(config *Config) (*allocRunner, error) { func (ar *allocRunner) initTaskRunners(tasks []*structs.Task) error { for _, task := range tasks { config := &taskrunner.Config{ - Alloc: ar.alloc, - ClientConfig: ar.clientConfig, - Task: task, - TaskDir: ar.allocDir.NewTaskDir(task.Name), - Logger: ar.logger, - StateDB: ar.stateDB, - StateUpdater: ar, - Consul: ar.consulClient, - ConsulSI: ar.sidsClient, - + Alloc: ar.alloc, + ClientConfig: ar.clientConfig, + Task: task, + TaskDir: ar.allocDir.NewTaskDir(task.Name), + Logger: ar.logger, + StateDB: ar.stateDB, + StateUpdater: ar, + Consul: ar.consulClient, + ConsulSI: ar.sidsClient, Vault: ar.vaultClient, DeviceStatsReporter: ar.deviceStatsReporter, DeviceManager: ar.devicemanager, diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index caa548cd9d8..aca6dbb93b0 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -10,6 +10,7 @@ import ( hclog "github.com/hashicorp/go-hclog" "github.com/hashicorp/nomad/client/allocrunner/interfaces" + ti "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces" "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/pkg/errors" @@ -29,7 +30,7 @@ const ( // sidsTokenFile is the name of the file holding the Consul SI token inside // the task's secret directory - sidsTokenFile = "sids_token" + sidsTokenFile = "si_token" // sidsTokenFilePerms is the level of file permissions granted on the file // in the secrets directory for the task @@ -40,17 +41,31 @@ type sidsHookConfig struct { alloc *structs.Allocation task *structs.Task sidsClient consul.ServiceIdentityAPI + lifecycle ti.TaskLifecycle logger hclog.Logger } // Service Identities hook for managing SI tokens of connect enabled tasks. type sidsHook struct { - alloc *structs.Allocation - taskName string + // alloc is the allocation + alloc *structs.Allocation + + // taskName is the name of the task + taskName string + + // sidsClient is the Consul client [proxy] for requesting SI tokens sidsClient consul.ServiceIdentityAPI - logger hclog.Logger - lock sync.Mutex + // lifecycle is used to signal, restart, and kill a task + lifecycle ti.TaskLifecycle + + // logger is used to log + logger hclog.Logger + + // lock variables that can be manipulated after hook creation + lock sync.Mutex + // firstRun keeps track of whether the hook is being called for the first + // time (for this task) during the lifespan of the Nomad Client process. firstRun bool } @@ -59,6 +74,7 @@ func newSIDSHook(c sidsHookConfig) *sidsHook { alloc: c.alloc, taskName: c.task.Name, sidsClient: c.sidsClient, + lifecycle: c.lifecycle, logger: c.logger.Named(sidsHookName), firstRun: true, } @@ -158,16 +174,42 @@ func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { } } +func (h *sidsHook) kill(ctx context.Context, err error) { + _ = h.lifecycle.Kill( + ctx, + structs.NewTaskEvent(structs.TaskKilling). + SetFailsTask(). + SetDisplayMessage(err.Error()), + ) +} + // tryDerive loops forever until a token is created, or ctx is done. func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- string) { - for i := 0; backoff(ctx, i); i++ { + for attempt := 0; backoff(ctx, attempt); attempt++ { + tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{h.taskName}) - if err != nil { - h.logger.Warn("failed to derive SI token", "attempt", i, "error", err) - continue + + switch { + + case err == nil: + // nothing broke and we can return the token for the task + ch <- tokens[h.taskName] + return + + case structs.IsServerSide(err): + // the error is known to be a server problem, just die + h.logger.Error("failed to derive SI token", "error", err, "server_side", true) + h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) + + case !structs.IsRecoverable(err): + // the error is known not to be recoverable, just die + h.logger.Error("failed to derive SI token", "error", err, "recoverable", false) + h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) + + default: + // the error is marked recoverable, retry after some backoff + h.logger.Error("failed to derive SI token", "error", err, "recoverable", true) } - ch <- tokens[h.taskName] - return } } @@ -182,12 +224,17 @@ func backoff(ctx context.Context, i int) bool { } func computeBackoff(attempt int) time.Duration { - switch { - case attempt <= 0: + switch attempt { + case 0: return 0 - case attempt >= 4: - return sidsBackoffLimit + case 1: + // go fast on first retry, because a unit test should be fast + return 100 * time.Millisecond default: - return (1 << (2 * uint(attempt))) * sidsBackoffBaseline + wait := time.Duration(attempt) * sidsBackoffBaseline + if wait > sidsBackoffLimit { + wait = sidsBackoffLimit + } + return wait } } diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go index cbb056634dc..859a008b293 100644 --- a/client/allocrunner/taskrunner/sids_hook_test.go +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -95,10 +95,11 @@ func TestSIDSHook_computeBackoff(t *testing.T) { } try(0, time.Duration(0)) - try(1, 20*time.Second) - try(2, 80*time.Second) - try(3, 320*time.Second) - try(4, sidsBackoffLimit) + try(1, 100*time.Millisecond) + try(2, 10*time.Second) + try(3, 15*time.Second) + try(4, 20*time.Second) + try(5, 25*time.Second) } func TestSIDSHook_backoff(t *testing.T) { diff --git a/client/allocrunner/taskrunner/task_runner_hooks.go b/client/allocrunner/taskrunner/task_runner_hooks.go index 46c5b5e8958..365cbd3250e 100644 --- a/client/allocrunner/taskrunner/task_runner_hooks.go +++ b/client/allocrunner/taskrunner/task_runner_hooks.go @@ -107,11 +107,12 @@ func (tr *TaskRunner) initHooks() { })) } - if usesConnect(tr.alloc.Job.LookupTaskGroup(tr.alloc.TaskGroup)) { + if task.UsesConnect() { tr.runnerHooks = append(tr.runnerHooks, newSIDSHook(sidsHookConfig{ alloc: tr.Alloc(), task: tr.Task(), sidsClient: tr.siClient, + lifecycle: tr, logger: hookLogger, })) } @@ -126,15 +127,6 @@ func (tr *TaskRunner) initHooks() { tr.runnerHooks = append(tr.runnerHooks, scriptCheckHook) } -func usesConnect(tg *structs.TaskGroup) bool { - for _, service := range tg.Services { - if service.Connect != nil { - return true - } - } - return false -} - func (tr *TaskRunner) emitHookError(err error, hookName string) { var taskEvent *structs.TaskEvent if herr, ok := err.(*hookError); ok { diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index d2dad453478..142ca0b027b 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -2,6 +2,7 @@ package taskrunner import ( "context" + "errors" "fmt" "io/ioutil" "net/http" @@ -1086,25 +1087,41 @@ func TestTaskRunner_CheckWatcher_Restart(t *testing.T) { require.True(t, state.Failed, pretty.Sprint(state)) } -func TestTaskRunner_BlockForSIDS(t *testing.T) { +type mockEnvoyBootstrapHook struct{} + +func (mockEnvoyBootstrapHook) Name() string { + return "mock_envoy_bootstrap" +} + +func (*mockEnvoyBootstrapHook) Prestart(_ context.Context, _ *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error { + resp.Done = true + return nil +} + +// The envoy bootstrap hook tries to connect to consul and run the envoy +// bootstrap command, so turn it off when testing connect jobs that are not +// using envoy (for now?). +func disableEnvoyBootstrapHook(tr *TaskRunner) { + for i, hook := range tr.runnerHooks { + if _, ok := hook.(*envoyBootstrapHook); ok { + tr.runnerHooks[i] = new(mockEnvoyBootstrapHook) + } + } +} + +// TestTaskRunner_BlockForSIDSToken asserts tasks do not start until a Consul +// Service Identity token is derived. +func TestTaskRunner_BlockForSIDSToken(t *testing.T) { t.Parallel() r := require.New(t) - // setup a connect enabled batch job that wants to exit immediately, which - // makes testing the prestart lifecycle easier - alloc := mock.BatchAlloc() - tg := alloc.Job.TaskGroups[0] - tg.Tasks[0].Config = map[string]interface{}{"run_for": "0s"} - tg.Services = []*structs.Service{{ - Name: "testconnect", - PortLabel: "9999", - Connect: &structs.ConsulConnect{ - SidecarService: &structs.ConsulSidecarService{}, - }}, + alloc := mock.BatchConnectAlloc() + task := alloc.Job.TaskGroups[0].Tasks[0] + task.Config = map[string]interface{}{ + "run_for": "0s", } - taskName := tg.Tasks[0].Name - trConfig, cleanup := testTaskRunnerConfig(t, alloc, taskName) + trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) defer cleanup() // control when we get a Consul SI token @@ -1112,7 +1129,7 @@ func TestTaskRunner_BlockForSIDS(t *testing.T) { waitCh := make(chan struct{}) deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { <-waitCh - return map[string]string{taskName: token}, nil + return map[string]string{task.Name: token}, nil } siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) siClient.DeriveTokenFn = deriveFn @@ -1121,6 +1138,7 @@ func TestTaskRunner_BlockForSIDS(t *testing.T) { tr, err := NewTaskRunner(trConfig) r.NoError(err) defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) + disableEnvoyBootstrapHook(tr) // turn off envoy bootstrap go tr.Run() // assert task runner blocks on SI token @@ -1156,9 +1174,114 @@ func TestTaskRunner_BlockForSIDS(t *testing.T) { r.Equal(token, string(data)) } -// TestTaskRunner_BlockForVault asserts tasks do not start until a vault token +func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { + t.Parallel() + r := require.New(t) + + alloc := mock.BatchConnectAlloc() + task := alloc.Job.TaskGroups[0].Tasks[0] + task.Config = map[string]interface{}{ + "run_for": "0s", + } + + trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) + defer cleanup() + + // control when we get a Consul SI token + token := "12345678-1234-1234-1234-1234567890" + deriveCount := 0 + deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { + if deriveCount > 0 { + return map[string]string{task.Name: token}, nil + } + deriveCount++ + return nil, structs.NewRecoverableError(errors.New("try again later"), true) + } + siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) + siClient.DeriveTokenFn = deriveFn + + // start the task runner + tr, err := NewTaskRunner(trConfig) + r.NoError(err) + defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) + disableEnvoyBootstrapHook(tr) // turn off envoy bootstrap + go tr.Run() + + // assert task runner blocks on SI token + select { + case <-tr.WaitCh(): + case <-time.After(time.Duration(testutil.TestMultiplier()*15) * time.Second): + r.Fail("timed out waiting for task runner") + } + + // assert task exited successfully + finalState := tr.TaskState() + r.Equal(structs.TaskStateDead, finalState.State) + r.False(finalState.Failed) + + // assert the token is on disk + tokenPath := filepath.Join(trConfig.TaskDir.SecretsDir, sidsTokenFile) + data, err := ioutil.ReadFile(tokenPath) + r.NoError(err) + r.Equal(token, string(data)) +} + +// TestTaskRunner_DeriveSIToken_Unrecoverable asserts that an unrecoverable error +// from deriving a service identity token will fail a task. +func TestTaskRunner_DeriveSIToken_Unrecoverable(t *testing.T) { + t.Parallel() + r := require.New(t) + + alloc := mock.BatchConnectAlloc() + tg := alloc.Job.TaskGroups[0] + tg.RestartPolicy.Attempts = 0 + tg.RestartPolicy.Interval = 0 + tg.RestartPolicy.Delay = 0 + tg.RestartPolicy.Mode = structs.RestartPolicyModeFail + task := tg.Tasks[0] + task.Config = map[string]interface{}{ + "run_for": "0s", + } + + trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) + defer cleanup() + + // SI token derivation suffers a non-retryable error + siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) + siClient.SetDeriveTokenError(alloc.ID, []string{task.Name}, errors.New("non-recoverable")) + + tr, err := NewTaskRunner(trConfig) + r.NoError(err) + + defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) + disableEnvoyBootstrapHook(tr) // turn off envoy bootstrap + go tr.Run() + + // Wait for the task to die + select { + case <-tr.WaitCh(): + case <-time.After(time.Duration(testutil.TestMultiplier()*15) * time.Second): + require.Fail(t, "timed out waiting for task runner to fail") + } + + // assert we have died and failed + finalState := tr.TaskState() + r.Equal(structs.TaskStateDead, finalState.State) + r.True(finalState.Failed) + r.Equal(5, len(finalState.Events)) + /* + + event: Task received by client + + event: Building Task Directory + + event: consul: failed to derive SI token: non-recoverable + + event: consul_sids: context canceled + + event: Policy allows no restarts + */ + r.Equal("true", finalState.Events[2].Details["fails_task"]) +} + +// TestTaskRunner_BlockForVaultToken asserts tasks do not start until a vault token // is derived. -func TestTaskRunner_BlockForVault(t *testing.T) { +func TestTaskRunner_BlockForVaultToken(t *testing.T) { t.Parallel() alloc := mock.BatchAlloc() diff --git a/client/client.go b/client/client.go index 0b5346dc0fa..8ed8b0ea107 100644 --- a/client/client.go +++ b/client/client.go @@ -1050,7 +1050,7 @@ func (c *Client) restoreState() error { StateUpdater: c, DeviceStatsReporter: c, Consul: c.consulService, - ConsulSI: c.tokensClient, // todo(shoenig), keep plumbing! + ConsulSI: c.tokensClient, Vault: c.vaultClient, PrevAllocWatcher: prevAllocWatcher, PrevAllocMigrator: prevAllocMigrator, @@ -2300,7 +2300,7 @@ func (c *Client) addAlloc(alloc *structs.Allocation, migrateToken string) error ClientConfig: c.configCopy, StateDB: c.stateDB, Consul: c.consulService, - ConsulSI: c.tokensClient, // todo(shoenig), keep plumbing! + ConsulSI: c.tokensClient, Vault: c.vaultClient, StateUpdater: c, DeviceStatsReporter: c, @@ -2445,6 +2445,7 @@ func (c *Client) deriveSIToken(alloc *structs.Allocation, taskNames []string) (m req := &structs.DeriveSITokenRequest{ NodeID: c.NodeID(), + SecretID: c.secretNodeID(), AllocID: alloc.ID, Tasks: tasks, QueryOptions: structs.QueryOptions{Region: c.Region()}, diff --git a/client/consul/identities_testing.go b/client/consul/identities_testing.go index 2d4258d2535..a091c006063 100644 --- a/client/consul/identities_testing.go +++ b/client/consul/identities_testing.go @@ -1,7 +1,6 @@ package consul import ( - "fmt" "sync" "github.com/hashicorp/nomad/helper/uuid" @@ -37,8 +36,6 @@ func (mtc *MockServiceIdentitiesClient) DeriveSITokens(alloc *structs.Allocation mtc.lock.Lock() defer mtc.lock.Unlock() - fmt.Println("MockServiceIdentitiesClient.DeriveSITokens running!") - // if the DeriveTokenFn is explicitly set, use that if mtc.DeriveTokenFn != nil { return mtc.DeriveTokenFn(alloc, tasks) diff --git a/client/vaultclient/vaultclient_testing.go b/client/vaultclient/vaultclient_testing.go index 08adaa119ea..c3c25c90d45 100644 --- a/client/vaultclient/vaultclient_testing.go +++ b/client/vaultclient/vaultclient_testing.go @@ -67,8 +67,7 @@ func (vc *MockVaultClient) SetDeriveTokenError(allocID string, tasks []string, e vc.deriveTokenErrors = make(map[string]map[string]error, 10) } - // todo(shoenig): this seems like a bug - if _, ok := vc.renewTokenErrors[allocID]; !ok { + if _, ok := vc.deriveTokenErrors[allocID]; !ok { vc.deriveTokenErrors[allocID] = make(map[string]error, 10) } diff --git a/command/agent/agent.go b/command/agent/agent.go index f8d85f5623a..7a4256bd800 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -64,6 +64,9 @@ type Agent struct { // consulCatalog is the subset of Consul's Catalog API Nomad uses. consulCatalog consul.CatalogAPI + // consulACLs is Nomad's subset of Consul's ACL API Nomad uses. + consulACLs consul.ACLsAPI + // client is the launched Nomad Client. Can be nil if the agent isn't // configured to run a client. client *client.Client @@ -584,7 +587,7 @@ func (a *Agent) setupServer() error { } // Create the server - server, err := nomad.NewServer(conf, a.consulCatalog) + server, err := nomad.NewServer(conf, a.consulCatalog, a.consulACLs) if err != nil { return fmt.Errorf("server setup failed: %v", err) } @@ -1023,11 +1026,12 @@ func (a *Agent) setupConsul(consulConfig *config.ConsulConfig) error { return err } - // Determine version for TLSSkipVerify - // Create Consul Catalog client for service discovery. a.consulCatalog = client.Catalog() + // Create Consul ACL client for managing tokens. + a.consulACLs = client.ACL() + // Create Consul Service client for service advertisement and checks. isClient := false if a.config.Client != nil && a.config.Client.Enabled { diff --git a/command/agent/command.go b/command/agent/command.go index 8a886377cc9..32c46862f3b 100644 --- a/command/agent/command.go +++ b/command/agent/command.go @@ -251,9 +251,12 @@ func (c *Command) readConfig() *Config { if config.Client == nil { config.Client = &ClientConfig{} } - if config.Server == nil { - config.Server = &ServerConfig{} - } + + // todo(shoenig): disable the server in dev mode just for my own demo stuff + // this should not be merged! + //if config.Server == nil { + // config.Server = &ServerConfig{} + //} // Merge any CLI options over config file options config = config.Merge(cmdConfig) diff --git a/command/agent/config.go b/command/agent/config.go index 39259abb2c4..b195fc83ac9 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -769,7 +769,7 @@ func DevConfig(mode *devModeConfig) *Config { conf.BindAddr = mode.bindAddr conf.LogLevel = "DEBUG" conf.Client.Enabled = true - conf.Server.Enabled = true + conf.Server.Enabled = false conf.DevMode = mode != nil conf.EnableDebug = true conf.DisableAnonymousSignature = true diff --git a/command/agent/consul/acl_testing.go b/command/agent/consul/acl_testing.go new file mode 100644 index 00000000000..05045ae7aa6 --- /dev/null +++ b/command/agent/consul/acl_testing.go @@ -0,0 +1,230 @@ +package consul + +import ( + "errors" + "sync" + "time" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/go-hclog" + "github.com/hashicorp/nomad/helper/uuid" +) + +var _ ACLsAPI = (*MockACLsAPI)(nil) + +// MockACLsAPI is a mock of consul.ACLsAPI +type MockACLsAPI struct { + logger hclog.Logger + + lock sync.Mutex + state struct { + index uint64 + error error + tokens map[string]*api.ACLToken + } +} + +func NewMockACLsAPI(l hclog.Logger) *MockACLsAPI { + return &MockACLsAPI{ + logger: l.Named("mock_consul"), + state: struct { + index uint64 + error error + tokens map[string]*api.ACLToken + }{tokens: make(map[string]*api.ACLToken)}, + } +} + +const ( + ExamplePolicyID1 = "a7c86856-0af5-4ab5-8834-03f4517e5564" + ExamplePolicyID2 = "ffa1b66c-967d-4468-8775-c687b5cfc16e" + ExamplePolicyID3 = "f68f0c36-51f8-4343-97dd-f0d4816c915f" +) + +func (m *MockACLsAPI) PolicyRead(policyID string, _ *api.QueryOptions) (*api.ACLPolicy, *api.QueryMeta, error) { + switch policyID { + + case ExamplePolicyID1: + return &api.ACLPolicy{ + ID: ExamplePolicyID1, + Rules: `service "service1" { policy = "write" }`, + }, nil, nil + + case ExamplePolicyID2: + return &api.ACLPolicy{ + ID: ExamplePolicyID2, + Rules: `service_prefix "foo-" { policy = "write" }`, + }, nil, nil + + case ExamplePolicyID3: + return &api.ACLPolicy{ + ID: ExamplePolicyID3, + Rules: ` +service "service1" { policy = "read" } +service "service2" { policy = "write" }`, + }, nil, nil + + default: + return nil, nil, errors.New("no such policy") + } +} + +const ( + ExampleOperatorToken1 = "59c219c2-47e4-43f3-bb45-258fd13f59d5" + ExampleOperatorToken2 = "868cc216-e123-4c2b-b362-f4d4c087de8e" + ExampleOperatorToken3 = "6177d1b9-c0f6-4118-b891-d818a3cb80b1" +) + +func (m *MockACLsAPI) TokenReadSelf(q *api.QueryOptions) (*api.ACLToken, *api.QueryMeta, error) { + switch q.Token { + + case ExampleOperatorToken1: + return &api.ACLToken{ + SecretID: ExampleOperatorToken1, + AccessorID: "e341bacd-535e-417c-8f45-f88d7faffcaf", + Description: "operator token 1", + Policies: []*api.ACLTokenPolicyLink{{ + ID: ExamplePolicyID1, + }}, + }, nil, nil + + case ExampleOperatorToken2: + return &api.ACLToken{ + SecretID: ExampleOperatorToken2, + AccessorID: "615b4d77-5164-4ec6-b616-24c0b24ac9cb", + Description: "operator token 2", + Policies: []*api.ACLTokenPolicyLink{{ + ID: ExamplePolicyID2, + }}, + }, nil, nil + + case ExampleOperatorToken3: + return &api.ACLToken{ + SecretID: ExampleOperatorToken3, + AccessorID: "6b7de0d7-15f7-45b4-95eb-fb775bfe3fdc", + Description: "operator token 3", + Policies: []*api.ACLTokenPolicyLink{{ + ID: ExamplePolicyID3, + }}, + }, nil, nil + + default: + return nil, nil, errors.New("no such token") + } +} + +// SetError is a helper method for configuring an error that will be returned +// on future calls to mocked methods. +func (m *MockACLsAPI) SetError(err error) { + m.lock.Lock() + defer m.lock.Unlock() + m.state.error = err +} + +// TokenCreate is a mock of ACLsAPI.TokenCreate +func (m *MockACLsAPI) TokenCreate(token *api.ACLToken, opts *api.WriteOptions) (*api.ACLToken, *api.WriteMeta, error) { + index, created, meta, err := m.tokenCreate(token, opts) + + services := func(token *api.ACLToken) []string { + if token == nil { + return nil + } + var names []string + for _, id := range token.ServiceIdentities { + names = append(names, id.ServiceName) + } + return names + }(created) + + description := func(token *api.ACLToken) string { + if token == nil { + return "" + } + return token.Description + }(created) + + accessor := func(token *api.ACLToken) string { + if token == nil { + return "" + } + return token.AccessorID + }(created) + + secret := func(token *api.ACLToken) string { + if token == nil { + return "" + } + return token.SecretID + }(created) + + m.logger.Trace("TokenCreate()", "description", description, "service_identities", services, "accessor", accessor, "secret", secret, "index", index, "error", err) + return created, meta, err +} + +func (m *MockACLsAPI) tokenCreate(token *api.ACLToken, _ *api.WriteOptions) (uint64, *api.ACLToken, *api.WriteMeta, error) { + m.lock.Lock() + defer m.lock.Unlock() + + m.state.index++ + + if m.state.error != nil { + return m.state.index, nil, nil, m.state.error + } + + secret := &api.ACLToken{ + CreateIndex: m.state.index, + ModifyIndex: m.state.index, + AccessorID: uuid.Generate(), + SecretID: uuid.Generate(), + Description: token.Description, + ServiceIdentities: token.ServiceIdentities, + CreateTime: time.Now(), + } + + m.state.tokens[secret.AccessorID] = secret + + w := &api.WriteMeta{ + RequestTime: 1 * time.Millisecond, + } + + return m.state.index, secret, w, nil +} + +// TokenDelete is a mock of ACLsAPI.TokenDelete +func (m *MockACLsAPI) TokenDelete(accessorID string, opts *api.WriteOptions) (*api.WriteMeta, error) { + meta, err := m.tokenDelete(accessorID, opts) + m.logger.Trace("TokenDelete()", "accessor", accessorID, "error", err) + return meta, err +} + +func (m *MockACLsAPI) tokenDelete(tokenID string, _ *api.WriteOptions) (*api.WriteMeta, error) { + m.lock.Lock() + defer m.lock.Unlock() + + m.state.index++ + + if m.state.error != nil { + return nil, m.state.error + } + + if _, exists := m.state.tokens[tokenID]; !exists { + return nil, errors.New("token does not exist") + } + + delete(m.state.tokens, tokenID) + + m.logger.Trace("TokenDelete()") + + return nil, nil +} + +// TokenList is a mock of ACLsAPI.TokenList +func (m *MockACLsAPI) TokenList(_ *api.QueryOptions) ([]*api.ACLTokenListEntry, *api.QueryMeta, error) { + m.lock.Lock() + defer m.lock.Unlock() + + //todo(shoenig): will need this for background token reconciliation + // coming in another issue + + return nil, nil, nil +} diff --git a/command/agent/consul/catalog_testing.go b/command/agent/consul/catalog_testing.go index 621d4143992..cd7977131e2 100644 --- a/command/agent/consul/catalog_testing.go +++ b/command/agent/consul/catalog_testing.go @@ -4,19 +4,17 @@ import ( "fmt" "sync" - log "github.com/hashicorp/go-hclog" - "github.com/hashicorp/consul/api" + "github.com/hashicorp/go-hclog" ) // MockCatalog can be used for testing where the CatalogAPI is needed. type MockCatalog struct { - logger log.Logger + logger hclog.Logger } -func NewMockCatalog(l log.Logger) *MockCatalog { - l = l.Named("mock_consul") - return &MockCatalog{logger: l} +func NewMockCatalog(l hclog.Logger) *MockCatalog { + return &MockCatalog{logger: l.Named("mock_consul")} } func (m *MockCatalog) Datacenters() ([]string, error) { diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index e7480a8b758..f20e86b8435 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -93,6 +93,20 @@ type AgentAPI interface { UpdateTTL(id, output, status string) error } +// ACLsAPI is the consul/api.ACL API used by Nomad Server. +type ACLsAPI interface { + // todo: RoleRead (...) + + // We are looking up by [operator token] SecretID, which implies we need + // to use this method instead of the normal TokenRead, which can only be + // used to lookup tokens by their AccessorID. + TokenReadSelf(q *api.QueryOptions) (*api.ACLToken, *api.QueryMeta, error) + PolicyRead(policyID string, q *api.QueryOptions) (*api.ACLPolicy, *api.QueryMeta, error) + TokenCreate(partial *api.ACLToken, q *api.WriteOptions) (*api.ACLToken, *api.WriteMeta, error) + TokenDelete(accessorID string, q *api.WriteOptions) (*api.WriteMeta, error) + TokenList(q *api.QueryOptions) ([]*api.ACLTokenListEntry, *api.QueryMeta, error) +} + func agentServiceUpdateRequired(reg *api.AgentServiceRegistration, svc *api.AgentService) bool { return !(reg.Kind == svc.Kind && reg.ID == svc.ID && diff --git a/nomad/consul.go b/nomad/consul.go new file mode 100644 index 00000000000..9df1409e6d9 --- /dev/null +++ b/nomad/consul.go @@ -0,0 +1,301 @@ +package nomad + +import ( + "context" + "fmt" + "strings" + "time" + + "github.com/armon/go-metrics" + "github.com/hashicorp/consul/api" + "github.com/hashicorp/go-hclog" + "github.com/hashicorp/hcl" + "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/pkg/errors" + "golang.org/x/time/rate" +) + +const ( + // siTokenDescriptionFmt is the format for the .Description field of + // service identity tokens generated on behalf of Nomad. + siTokenDescriptionFmt = "_nomad_si [%s] [%s] [%s]" + + // siTokenRequestRateLimit is the maximum number of requests per second Nomad + // will make against Consul for requesting SI tokens. + siTokenRequestRateLimit rate.Limit = 500 + + // siTokenMaxParallelRevokes is the maximum number of parallel SI token + // revocation requests Nomad will make against Consul. + siTokenMaxParallelRevokes = 64 + + // todo: more revocation things +) + +const ( + // ConsulPolicyWrite is the literal text of the policy field of a Consul Policy + // Rule that we check when validating an Operator Consul token against the + // necessary permissions for creating a Service Identity token for a given + // service. + // + // The rule may be: + // - service. + // - service."*" (wildcard) + // - service_prefix. (including empty string) + // + // e.g. + // service "web" { policy = "write" } + // service_prefix "" { policy = "write" } + ConsulPolicyWrite = "write" +) + +// ConsulServiceRule represents a policy for a service +type ConsulServiceRule struct { + Name string `hcl:",key"` + Policy string +} + +type ConsulPolicy struct { + Services []*ConsulServiceRule `hcl:"service,expand"` + ServicePrefixes []*ConsulServiceRule `hcl:"service_prefix,expand"` +} + +func (cp *ConsulPolicy) IsEmpty() bool { + if cp == nil { + return true + } + return len(cp.Services) == 0 && len(cp.ServicePrefixes) == 0 +} + +func ParseConsulPolicy(s string) (*ConsulPolicy, error) { + cp := new(ConsulPolicy) + if err := hcl.Decode(cp, s); err != nil { + return nil, errors.Wrap(err, "failed to parse ACL policy") + } + if cp.IsEmpty() { + // the only use case for now, may as well validate asap + return nil, errors.New("consul policy contains no service rules") + } + return cp, nil +} + +type ServiceIdentityIndex struct { + ClusterID string + AllocID string + TaskName string +} + +func (sii ServiceIdentityIndex) Validate() error { + switch { + case sii.ClusterID == "": + return errors.New("cluster id not set") + case sii.AllocID == "": + return errors.New("alloc id not set") + case sii.TaskName == "": + return errors.New("task name not set") + default: + return nil + } +} + +func (sii ServiceIdentityIndex) Description() string { + return fmt.Sprintf(siTokenDescriptionFmt, sii.ClusterID, sii.AllocID, sii.TaskName) +} + +// ConsulACLsAPI is an abstraction over the consul/api.ACL API used by Nomad +// Server. +type ConsulACLsAPI interface { + + // CheckSIPolicy checks that the given operator token has the equivalent ACL + // permissiveness that a Service Identity token policy for task would have. + CheckSIPolicy(ctx context.Context, task, secretID string) error + + // Create instructs Consul to create a Service Identity token. + CreateToken(context.Context, ServiceIdentityIndex) (*structs.SIToken, error) + + // RevokeTokens instructs Consul to revoke the given token accessors. + RevokeTokens(context.Context, []*structs.SITokenAccessor) error + + // ListTokens lists every token in Consul. + // + // To be used for reconciliation (later). + ListTokens() ([]string, error) +} + +type consulACLsAPI struct { + // aclClient is the API subset of the real consul client we need for + // managing Service Identity tokens. + aclClient consul.ACLsAPI + + // limiter is used to rate limit requests to consul + limiter *rate.Limiter + + // logger is used to log messages + logger hclog.Logger +} + +func NewConsulACLsAPI(aclClient consul.ACLsAPI, logger hclog.Logger) (ConsulACLsAPI, error) { + c := &consulACLsAPI{ + aclClient: aclClient, + logger: logger.Named("consul_acl"), + limiter: rate.NewLimiter(requestRateLimit, int(requestRateLimit)), + } + return c, nil +} + +func (c *consulACLsAPI) CheckSIPolicy(_ context.Context, task, secretID string) error { + if id := strings.TrimSpace(secretID); id == "" { + // todo: check in tests + return errors.New("missing consul token") + } + + // todo: log request time, result, etc. + + // todo: use ctx + + // todo: use rate limiting + + opToken, meta, err := c.aclClient.TokenReadSelf(&api.QueryOptions{ + AllowStale: false, + Token: secretID, + }) + if err != nil { + return errors.Wrap(err, "unable to validate operator consul token") + } + + _ = meta + + allowable, err := c.hasSufficientPolicy(task, opToken) + if err != nil { + return errors.Wrap(err, "unable to validate operator consul token") + } + if !allowable { + return errors.Errorf("permission denied for %q", task) + } + + return nil +} + +func (c *consulACLsAPI) hasSufficientPolicy(task string, token *api.ACLToken) (bool, error) { + + for _, policyRef := range token.Policies { + if allowable, err := c.policyAllowsServiceWrite(task, policyRef.ID); err != nil { + return false, err + } else if allowable { + return true, nil + } + } + + // todo: probably also need to go through each role and check all those + + return false, nil +} + +// policyAllowsServiceWrite +func (c *consulACLsAPI) policyAllowsServiceWrite(task string, policyID string) (bool, error) { + policy, _, err := c.aclClient.PolicyRead(policyID, &api.QueryOptions{ + AllowStale: false, + }) + if err != nil { + return false, err + } + + // compare policy to the necessary permission for service write + // e.g. service "db" { policy = "write" } + // e.g. service_prefix "" { policy == "write" } + cp, err := ParseConsulPolicy(policy.Rules) + if err != nil { + return false, err + } + + if c.allowsServiceWrite(task, cp) { + return true, nil + } + + return false, nil +} + +const ( + serviceNameWildcard = "*" +) + +func (_ *consulACLsAPI) allowsServiceWrite(task string, cp *ConsulPolicy) bool { + for _, service := range cp.Services { + name := strings.ToLower(service.Name) + policy := strings.ToLower(service.Policy) + if policy == ConsulPolicyWrite { + if name == task || name == serviceNameWildcard { + return true + } + } + } + + for _, servicePrefix := range cp.ServicePrefixes { + prefix := strings.ToLower(servicePrefix.Name) + policy := strings.ToLower(servicePrefix.Policy) + if policy == ConsulPolicyWrite { + if strings.HasPrefix(task, prefix) { + return true + } + } + } + return false +} + +func (c *consulACLsAPI) CreateToken(ctx context.Context, sii ServiceIdentityIndex) (*structs.SIToken, error) { + defer metrics.MeasureSince([]string{"nomad", "consul", "create_token"}, time.Now()) + + // sanity check the metadata for the token we want + if err := sii.Validate(); err != nil { + return nil, err + } + + // todo: use ctx + + // todo: rate limiting + + partial := &api.ACLToken{ + Description: sii.Description(), + ServiceIdentities: []*api.ACLServiceIdentity{{ServiceName: sii.TaskName}}, + } + + token, _, err := c.aclClient.TokenCreate(partial, nil) + if err != nil { + return nil, err + } + + return &structs.SIToken{ + TaskName: sii.TaskName, + AccessorID: token.AccessorID, + SecretID: token.SecretID, + }, nil +} + +func (c *consulACLsAPI) RevokeTokens(ctx context.Context, accessors []*structs.SITokenAccessor) error { + defer metrics.MeasureSince([]string{"nomad", "consul", "revoke_tokens"}, time.Now()) + + // todo: use ctx + + // todo: rate limiting + + for _, accessor := range accessors { + if err := c.revokeToken(ctx, accessor); err != nil { + // todo: accumulate errors and IDs that are going to need another attempt + return err + } + } + + return nil +} + +func (c *consulACLsAPI) revokeToken(_ context.Context, accessor *structs.SITokenAccessor) error { + c.logger.Trace("revoke SI token", "task", accessor.TaskName, "alloc_id", accessor.AllocID, "node_id", accessor.NodeID) + _, err := c.aclClient.TokenDelete(accessor.AccessorID, nil) + return err +} + +func (c *consulACLsAPI) ListTokens() ([]string, error) { + defer metrics.MeasureSince([]string{"nomad", "consul", "list_tokens"}, time.Now()) + + return nil, errors.New("not yet implemented") +} diff --git a/nomad/consul_test.go b/nomad/consul_test.go new file mode 100644 index 00000000000..8a5d844e4d5 --- /dev/null +++ b/nomad/consul_test.go @@ -0,0 +1,339 @@ +package nomad + +import ( + "context" + "errors" + "testing" + + "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +func TestConsulPolicy_ParseConsulPolicy(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, text string, expPolicy *ConsulPolicy, expErr string) { + policy, err := ParseConsulPolicy(text) + if expErr != "" { + require.EqualError(t, err, expErr) + require.True(t, policy.IsEmpty()) + } else { + require.NoError(t, err) + require.Equal(t, expPolicy, policy) + } + } + + t.Run("service", func(t *testing.T) { + text := `service "web" { policy = "read" }` + exp := &ConsulPolicy{ + Services: []*ConsulServiceRule{{Name: "web", Policy: "read"}}, + ServicePrefixes: []*ConsulServiceRule(nil), + } + try(t, text, exp, "") + }) + + t.Run("service_prefix", func(t *testing.T) { + text := `service_prefix "data" { policy = "write" }` + exp := &ConsulPolicy{ + Services: []*ConsulServiceRule(nil), + ServicePrefixes: []*ConsulServiceRule{{Name: "data", Policy: "write"}}, + } + try(t, text, exp, "") + }) + + t.Run("empty", func(t *testing.T) { + text := `` + expErr := "consul policy contains no service rules" + try(t, text, nil, expErr) + }) + + t.Run("malformed", func(t *testing.T) { + text := `this is not valid HCL!` + expErr := "failed to parse ACL policy: At 1:22: illegal char" + try(t, text, nil, expErr) + }) +} + +func TestConsulPolicy_IsEmpty(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, cp *ConsulPolicy, exp bool) { + result := cp.IsEmpty() + require.Equal(t, exp, result) + } + + t.Run("nil", func(t *testing.T) { + cp := (*ConsulPolicy)(nil) + try(t, cp, true) + }) + + t.Run("empty slices", func(t *testing.T) { + cp := &ConsulPolicy{ + Services: []*ConsulServiceRule(nil), + ServicePrefixes: []*ConsulServiceRule(nil), + } + try(t, cp, true) + }) + + t.Run("services nonempty", func(t *testing.T) { + cp := &ConsulPolicy{ + Services: []*ConsulServiceRule{{Name: "example", Policy: "write"}}, + } + try(t, cp, false) + }) + + t.Run("service_prefixes nonempty", func(t *testing.T) { + cp := &ConsulPolicy{ + ServicePrefixes: []*ConsulServiceRule{{Name: "pre", Policy: "read"}}, + } + try(t, cp, false) + }) +} + +var _ ConsulACLsAPI = (*consulACLsAPI)(nil) +var _ ConsulACLsAPI = (*mockConsulACLsAPI)(nil) + +type mockConsulACLsAPI struct { + revokeRequests []string +} + +func (m *mockConsulACLsAPI) CheckSIPolicy(_ context.Context, _, _ string) error { + panic("not implemented yet") +} + +func (m *mockConsulACLsAPI) CreateToken(_ context.Context, _ ServiceIdentityIndex) (*structs.SIToken, error) { + panic("not implemented yet") +} + +func (m *mockConsulACLsAPI) ListTokens() ([]string, error) { + panic("not implemented yet") +} + +func (m *mockConsulACLsAPI) RevokeTokens(_ context.Context, accessors []*structs.SITokenAccessor) error { + for _, accessor := range accessors { + m.revokeRequests = append(m.revokeRequests, accessor.AccessorID) + } + return nil +} + +func TestConsulACLsAPI_CreateToken(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, expErr error) { + logger := testlog.HCLogger(t) + aclAPI := consul.NewMockACLsAPI(logger) + aclAPI.SetError(expErr) + + c, err := NewConsulACLsAPI(aclAPI, logger) + require.NoError(t, err) + + ctx := context.Background() + sii := ServiceIdentityIndex{ + AllocID: uuid.Generate(), + ClusterID: uuid.Generate(), + TaskName: "my-task1", + } + + token, err := c.CreateToken(ctx, sii) + + if expErr != nil { + require.Equal(t, expErr, err) + require.Nil(t, token) + } else { + require.NoError(t, err) + require.Equal(t, "my-task1", token.TaskName) + require.True(t, helper.IsUUID(token.AccessorID)) + require.True(t, helper.IsUUID(token.SecretID)) + } + } + + t.Run("create token success", func(t *testing.T) { + try(t, nil) + }) + + t.Run("create token error", func(t *testing.T) { + try(t, errors.New("consul broke")) + }) +} + +func TestConsulACLsAPI_RevokeTokens(t *testing.T) { + t.Parallel() + + setup := func(t *testing.T, exp error) (context.Context, ConsulACLsAPI, *structs.SIToken) { + logger := testlog.HCLogger(t) + aclAPI := consul.NewMockACLsAPI(logger) + + c, err := NewConsulACLsAPI(aclAPI, logger) + require.NoError(t, err) + + ctx := context.Background() + generated, err := c.CreateToken(ctx, ServiceIdentityIndex{ + ClusterID: uuid.Generate(), + AllocID: uuid.Generate(), + TaskName: "task1", + }) + require.NoError(t, err) + + // set the mock error after calling CreateToken for setting up + aclAPI.SetError(exp) + + return context.Background(), c, generated + } + + accessors := func(ids ...string) (result []*structs.SITokenAccessor) { + for _, id := range ids { + result = append(result, &structs.SITokenAccessor{AccessorID: id}) + } + return + } + + t.Run("revoke token success", func(t *testing.T) { + ctx, c, token := setup(t, nil) + err := c.RevokeTokens(ctx, accessors(token.AccessorID)) + require.NoError(t, err) + }) + + t.Run("revoke token non-existent", func(t *testing.T) { + ctx, c, _ := setup(t, nil) + err := c.RevokeTokens(ctx, accessors(uuid.Generate())) + require.EqualError(t, err, "token does not exist") + }) + + t.Run("revoke token error", func(t *testing.T) { + exp := errors.New("consul broke") + ctx, c, token := setup(t, exp) + err := c.RevokeTokens(ctx, accessors(token.AccessorID)) + require.EqualError(t, err, exp.Error()) + }) +} + +func TestConsulACLsAPI_CheckSIPolicy(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, service, token string, expErr string) { + logger := testlog.HCLogger(t) + aclAPI := consul.NewMockACLsAPI(logger) + cAPI, err := NewConsulACLsAPI(aclAPI, logger) + require.NoError(t, err) + + err = cAPI.CheckSIPolicy(context.Background(), service, token) + if expErr != "" { + require.EqualError(t, err, expErr) + } else { + require.NoError(t, err) + } + } + + t.Run("operator has service write", func(t *testing.T) { + try(t, "service1", consul.ExampleOperatorToken1, "") + }) + + t.Run("operator has service_prefix write", func(t *testing.T) { + try(t, "foo-service1", consul.ExampleOperatorToken2, "") + }) + + t.Run("operator permissions insufficient", func(t *testing.T) { + try(t, "service1", consul.ExampleOperatorToken3, + "permission denied for \"service1\"", + ) + }) + + t.Run("no token provided", func(t *testing.T) { + try(t, "service1", "", "missing consul token") + }) + + t.Run("nonsense token provided", func(t *testing.T) { + try(t, "service1", "f1682bde-1e71-90b1-9204-85d35467ba61", + "unable to validate operator consul token: no such token", + ) + }) +} + +func TestConsulACLsAPI_allowsServiceWrite(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, task string, cp *ConsulPolicy, exp bool) { + cAPI := new(consulACLsAPI) + result := cAPI.allowsServiceWrite(task, cp) + require.Equal(t, exp, result) + } + + makeCP := func(services [][2]string, prefixes [][2]string) *ConsulPolicy { + serviceRules := make([]*ConsulServiceRule, 0, len(services)) + for _, service := range services { + serviceRules = append(serviceRules, &ConsulServiceRule{Name: service[0], Policy: service[1]}) + } + prefixRules := make([]*ConsulServiceRule, 0, len(prefixes)) + for _, prefix := range prefixes { + prefixRules = append(prefixRules, &ConsulServiceRule{Name: prefix[0], Policy: prefix[1]}) + } + return &ConsulPolicy{Services: serviceRules, ServicePrefixes: prefixRules} + } + + t.Run("matching service policy write", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"task1", "write"}}, + nil, + ), true) + }) + + t.Run("matching service policy read", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"task1", "read"}}, + nil, + ), false) + }) + + t.Run("wildcard service policy write", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"*", "write"}}, + nil, + ), true) + }) + + t.Run("wrong service policy write", func(t *testing.T) { + try(t, "other1", makeCP( + [][2]string{{"task1", "write"}}, + nil, + ), false) + }) + + t.Run("matching prefix policy write", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"task-", "write"}}, + ), true) + }) + + t.Run("matching prefix policy read", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"task-", "read"}}, + ), false) + }) + + t.Run("empty prefix policy write", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"", "write"}}, + ), true) + }) + + t.Run("late matching service", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"task0", "write"}, {"task1", "write"}}, + nil, + ), true) + }) + + t.Run("late matching prefix", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"foo-", "write"}, {"task-", "write"}}, + ), true) + }) +} diff --git a/nomad/fsm.go b/nomad/fsm.go index c9c29d5b1b1..37e89a9395e 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -15,6 +15,7 @@ import ( "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" "github.com/hashicorp/raft" + "github.com/pkg/errors" "github.com/ugorji/go/codec" ) @@ -46,6 +47,7 @@ const ( ACLTokenSnapshot SchedulerConfigSnapshot ClusterMetadataSnapshot + ServiceIdentityTokenAccessorSnapshot ) // LogApplier is the definition of a function that can apply a Raft log @@ -254,6 +256,10 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { return n.applyDeregisterNodeBatch(buf[1:], log.Index) case structs.ClusterMetadataRequestType: return n.applyClusterMetadata(buf[1:], log.Index) + case structs.ServiceIdentityAccessorRegisterRequestType: + return n.applyUpsertSIAccessor(buf[1:], log.Index) + case structs.ServiceIdentityAccessorDeregisterRequestType: + return n.applyDeregisterSIAccessor(buf[1:], log.Index) } // Check enterprise only message types. @@ -865,6 +871,36 @@ func (n *nomadFSM) applyDeregisterVaultAccessor(buf []byte, index uint64) interf return nil } +func (n *nomadFSM) applyUpsertSIAccessor(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "upsert_si_accessor"}, time.Now()) + var request structs.SITokenAccessorsRequest + if err := structs.Decode(buf, &request); err != nil { + panic(errors.Wrap(err, "failed to decode request")) + } + + if err := n.state.UpsertSITokenAccessors(index, request.Accessors); err != nil { + n.logger.Error("UpsertSITokenAccessors failed", "error", err) + return err + } + + return nil +} + +func (n *nomadFSM) applyDeregisterSIAccessor(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "deregister_si_accessor"}, time.Now()) + var request structs.SITokenAccessorsRequest + if err := structs.Decode(buf, &request); err != nil { + panic(errors.Wrap(err, "failed to decode request")) + } + + if err := n.state.DeleteSITokenAccessors(index, request.Accessors); err != nil { + n.logger.Error("DeregisterSITokenAccessor failed", "error", err) + return err + } + + return nil +} + // applyPlanApply applies the results of a plan application func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_plan_results"}, time.Now()) @@ -1229,6 +1265,15 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { return err } + case ServiceIdentityTokenAccessorSnapshot: + accessor := new(structs.SITokenAccessor) + if err := dec.Decode(accessor); err != nil { + return err + } + if err := restore.SITokenAccessorRestore(accessor); err != nil { + return err + } + case JobVersionSnapshot: version := new(structs.Job) if err := dec.Decode(version); err != nil { @@ -1533,6 +1578,10 @@ func (s *nomadSnapshot) Persist(sink raft.SnapshotSink) error { sink.Cancel() return err } + if err := s.persistSITokenAccessors(sink, encoder); err != nil { + sink.Cancel() + return err + } if err := s.persistJobVersions(sink, encoder); err != nil { sink.Cancel() return err @@ -1781,6 +1830,23 @@ func (s *nomadSnapshot) persistVaultAccessors(sink raft.SnapshotSink, return nil } +func (s *nomadSnapshot) persistSITokenAccessors(sink raft.SnapshotSink, encoder *codec.Encoder) error { + ws := memdb.NewWatchSet() + accessors, err := s.snap.SITokenAccessors(ws) + if err != nil { + return err + } + + for raw := accessors.Next(); raw != nil; raw = accessors.Next() { + accessor := raw.(*structs.SITokenAccessor) + sink.Write([]byte{byte(ServiceIdentityTokenAccessorSnapshot)}) + if err := encoder.Encode(accessor); err != nil { + return err + } + } + return nil +} + func (s *nomadSnapshot) persistJobVersions(sink raft.SnapshotSink, encoder *codec.Encoder) error { // Get all the jobs diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 0cfb72480e3..eb5e2abefb3 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1662,6 +1662,79 @@ func TestFSM_DeregisterVaultAccessor(t *testing.T) { } } +func TestFSM_UpsertSITokenAccessor(t *testing.T) { + t.Parallel() + r := require.New(t) + + fsm := testFSM(t) + fsm.blockedEvals.SetEnabled(true) + + a1 := mock.SITokenAccessor() + a2 := mock.SITokenAccessor() + request := structs.SITokenAccessorsRequest{ + Accessors: []*structs.SITokenAccessor{a1, a2}, + } + buf, err := structs.Encode(structs.ServiceIdentityAccessorRegisterRequestType, request) + r.NoError(err) + + response := fsm.Apply(makeLog(buf)) + r.Nil(response) + + // Verify the accessors got registered + ws := memdb.NewWatchSet() + result1, err := fsm.State().SITokenAccessor(ws, a1.AccessorID) + r.NoError(err) + r.NotNil(result1) + r.Equal(uint64(1), result1.CreateIndex) + + result2, err := fsm.State().SITokenAccessor(ws, a2.AccessorID) + r.NoError(err) + r.NotNil(result2) + r.Equal(uint64(1), result2.CreateIndex) + + tt := fsm.TimeTable() + latestIndex := tt.NearestIndex(time.Now()) + r.Equal(uint64(1), latestIndex) +} + +func TestFSM_DeregisterSITokenAccessor(t *testing.T) { + t.Parallel() + r := require.New(t) + + fsm := testFSM(t) + fsm.blockedEvals.SetEnabled(true) + + a1 := mock.SITokenAccessor() + a2 := mock.SITokenAccessor() + accessors := []*structs.SITokenAccessor{a1, a2} + var err error + + // Insert the accessors + err = fsm.State().UpsertSITokenAccessors(1000, accessors) + r.NoError(err) + + request := structs.SITokenAccessorsRequest{Accessors: accessors} + buf, err := structs.Encode(structs.ServiceIdentityAccessorDeregisterRequestType, request) + r.NoError(err) + + response := fsm.Apply(makeLog(buf)) + r.Nil(response) + + ws := memdb.NewWatchSet() + + result1, err := fsm.State().SITokenAccessor(ws, a1.AccessorID) + r.NoError(err) + r.Nil(result1) // should have been deleted + + result2, err := fsm.State().SITokenAccessor(ws, a2.AccessorID) + r.NoError(err) + r.Nil(result2) // should have been deleted + + tt := fsm.TimeTable() + latestIndex := tt.NearestIndex(time.Now()) + r.Equal(uint64(1), latestIndex) +} + func TestFSM_ApplyPlanResults(t *testing.T) { t.Parallel() fsm := testFSM(t) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 5829d886188..215bb54ef58 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -20,6 +20,7 @@ import ( "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" + "github.com/pkg/errors" ) const ( @@ -33,7 +34,6 @@ const ( ) var ( - // allowRescheduleTransition is the transition that allows failed // allocations to be force rescheduled. We create a one off // variable to avoid creating a new object for every request. @@ -104,7 +104,7 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis return structs.ErrPermissionDenied } - // Validate Volume Permsissions + // Validate Volume Permissions for _, tg := range args.Job.TaskGroups { for _, vol := range tg.Volumes { if vol.Type != structs.VolumeTypeHost { @@ -215,6 +215,42 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis } } + // helper function that checks if the "operator token" supplied with the + // job has sufficient ACL permissions for establishing consul connect services + checkOperatorToken := func(task string) error { + fmt.Println("@@ check op token for:", task, "au:", j.srv.config.ConsulConfig.AllowsUnauthenticated()) + + if j.srv.config.ConsulConfig.AllowsUnauthenticated() { + // if consul.allow_unauthenticated is enabled (which is the default) + // just let the Job through without checking anything. + fmt.Println("@@ allows unauthenticated = true, not checking") + return nil + } + proxiedTask := strings.TrimPrefix(task, structs.ConnectProxyPrefix+"-") + fmt.Println("@@ proxiedTask:", proxiedTask) + ctx := context.Background() + if err := j.srv.consulACLs.CheckSIPolicy(ctx, proxiedTask, args.Job.ConsulToken); err != nil { + // not much in the way of exported error types, we could parse + // the content, but all errors are going to be failures anyway + fmt.Println("stopped in check for", proxiedTask) + return errors.Wrap(err, "operator token denied") + } + fmt.Println("got passed check for", proxiedTask) + return nil + } + + // Enforce that the operator has necessary Consul ACL permissions + connectTasks := args.Job.ConnectTasks() + if len(connectTasks) > 0 { + for _, tg := range connectTasks { + for _, task := range tg { + if err := checkOperatorToken(task); err != nil { + return err + } + } + } + } + // Enforce Sentinel policies policyWarnings, err := j.enforceSubmitJob(args.PolicyOverride, args.Job) if err != nil { diff --git a/nomad/job_endpoint_hook_connect.go b/nomad/job_endpoint_hook_connect.go index 80eec7e920c..b18b4c1ec0f 100644 --- a/nomad/job_endpoint_hook_connect.go +++ b/nomad/job_endpoint_hook_connect.go @@ -42,7 +42,7 @@ var ( } ) -// jobConnectHook implements a job Mutating and Validating admission controller +// jobConnectHook implements a job Mutating and Validating admission controller type jobConnectHook struct{} func (jobConnectHook) Name() string { diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 8b1bd9e95c6..257961fca32 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -11,6 +11,7 @@ import ( memdb "github.com/hashicorp/go-memdb" msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc" "github.com/hashicorp/nomad/acl" + "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" @@ -275,6 +276,86 @@ func TestJobEndpoint_Register_ConnectWithSidecarTask(t *testing.T) { } +// TestJobEndpoint_Register_Connect_AllowUnauthenticatedFalse asserts that a job +// submission fails allow_unauthenticated is false, and either an invalid or no +// operator Consul token is provided. +func TestJobEndpoint_Register_Connect_AllowUnauthenticatedFalse(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + c.ConsulConfig.AllowUnauthenticated = helper.BoolToPtr(false) + }) + defer cleanupS1() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the register request + job := mock.Job() + job.TaskGroups[0].Networks = structs.Networks{ + { + Mode: "bridge", + }, + } + job.TaskGroups[0].Services = []*structs.Service{ + { + Name: "service1", // matches consul.ExamplePolicyID1 + PortLabel: "8080", + Connect: &structs.ConsulConnect{ + SidecarService: &structs.ConsulSidecarService{}, + }, + }, + } + + newRequest := func(job *structs.Job) *structs.JobRegisterRequest { + return &structs.JobRegisterRequest{ + Job: job, + WriteRequest: structs.WriteRequest{ + Region: "global", + Namespace: job.Namespace, + }, + } + } + + // Each variation of the provided Consul operator token + noOpToken := "" + unrecognizedOpToken := uuid.Generate() + unauthorizedOpToken := consul.ExampleOperatorToken3 + authorizedOpToken := consul.ExampleOperatorToken1 + + t.Run("no token provided", func(t *testing.T) { + request := newRequest(job) + request.Job.ConsulToken = noOpToken + var response structs.JobRegisterResponse + err := msgpackrpc.CallWithCodec(codec, "Job.Register", request, &response) + require.EqualError(t, err, "operator token denied: unable to validate operator consul token: no such token") + }) + + t.Run("unknown token provided", func(t *testing.T) { + request := newRequest(job) + request.Job.ConsulToken = unrecognizedOpToken + var response structs.JobRegisterResponse + err := msgpackrpc.CallWithCodec(codec, "Job.Register", request, &response) + require.EqualError(t, err, "operator token denied: unable to validate operator consul token: no such token") + }) + + t.Run("unauthorized token provided", func(t *testing.T) { + request := newRequest(job) + request.Job.ConsulToken = unauthorizedOpToken + var response structs.JobRegisterResponse + err := msgpackrpc.CallWithCodec(codec, "Job.Register", request, &response) + require.EqualError(t, err, "operator token denied: permission denied for \"service1\"") + }) + + t.Run("authorized token provided", func(t *testing.T) { + request := newRequest(job) + request.Job.ConsulToken = authorizedOpToken + var response structs.JobRegisterResponse + err := msgpackrpc.CallWithCodec(codec, "Job.Register", request, &response) + require.NoError(t, err) + }) +} + func TestJobEndpoint_Register_ACL(t *testing.T) { t.Parallel() diff --git a/nomad/leader.go b/nomad/leader.go index c6eb91d6692..8f02c179a26 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -233,7 +233,13 @@ func (s *Server) establishLeadership(stopCh chan struct{}) error { // Activate the vault client s.vault.SetActive(true) - if err := s.restoreRevokingAccessors(); err != nil { + // Cleanup orphaned Vault token accessors + if err := s.revokeVaultAccessorsOnRestore(); err != nil { + return err + } + + // Cleanup orphaned Service Identity token accessors + if err := s.revokeSITokenAccessorsOnRestore(); err != nil { return err } @@ -322,9 +328,9 @@ func (s *Server) restoreEvals() error { return nil } -// restoreRevokingAccessors is used to restore Vault accessors that should be +// revokeVaultAccessorsOnRestore is used to restore Vault accessors that should be // revoked. -func (s *Server) restoreRevokingAccessors() error { +func (s *Server) revokeVaultAccessorsOnRestore() error { // An accessor should be revoked if its allocation or node is terminal ws := memdb.NewWatchSet() state := s.fsm.State() @@ -374,6 +380,53 @@ func (s *Server) restoreRevokingAccessors() error { return nil } +// revokeSITokenAccessorsOnRestore is used to revoke Service Identity token +// accessors on behalf of allocs that are now gone / terminal. +func (s *Server) revokeSITokenAccessorsOnRestore() error { + ws := memdb.NewWatchSet() + fsmState := s.fsm.State() + iter, err := fsmState.SITokenAccessors(ws) + if err != nil { + return errors.Wrap(err, "failed to get SI token accessors") + } + + var toRevoke []*structs.SITokenAccessor + for raw := iter.Next(); raw != nil; raw = iter.Next() { + accessor := raw.(*structs.SITokenAccessor) + + // Check the allocation + alloc, err := fsmState.AllocByID(ws, accessor.AllocID) + if err != nil { + return errors.Wrapf(err, "failed to lookup alloc %q", accessor.AllocID) + } + if alloc == nil || alloc.Terminated() { + // no longer running and associated accessors should be revoked + toRevoke = append(toRevoke, accessor) + continue + } + + // Check the node + node, err := fsmState.NodeByID(ws, accessor.NodeID) + if err != nil { + return errors.Wrapf(err, "failed to lookup node %q", accessor.NodeID) + } + if node == nil || node.TerminalStatus() { + // node is terminal and associated accessors should be revoked + toRevoke = append(toRevoke, accessor) + continue + } + } + + if len(toRevoke) > 0 { + ctx := context.Background() + if err := s.consulACLs.RevokeTokens(ctx, toRevoke); err != nil { + return errors.Wrap(err, "failed to revoke SI tokens") + } + } + + return nil +} + // restorePeriodicDispatcher is used to restore all periodic jobs into the // periodic dispatcher. It also determines if a periodic job should have been // created during the leadership transition and force runs them. The periodic diff --git a/nomad/leader_test.go b/nomad/leader_test.go index 7cbd8278136..980a215a799 100644 --- a/nomad/leader_test.go +++ b/nomad/leader_test.go @@ -604,7 +604,7 @@ func TestLeader_ReapDuplicateEval(t *testing.T) { }) } -func TestLeader_RestoreVaultAccessors(t *testing.T) { +func TestLeader_revokeVaultAccessorsOnRestore(t *testing.T) { s1, cleanupS1 := TestServer(t, func(c *Config) { c.NumSchedulers = 0 }) @@ -612,9 +612,9 @@ func TestLeader_RestoreVaultAccessors(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) // Insert a vault accessor that should be revoked - state := s1.fsm.State() + fsmState := s1.fsm.State() va := mock.VaultAccessor() - if err := state.UpsertVaultAccessor(100, []*structs.VaultAccessor{va}); err != nil { + if err := fsmState.UpsertVaultAccessor(100, []*structs.VaultAccessor{va}); err != nil { t.Fatalf("bad: %v", err) } @@ -623,7 +623,7 @@ func TestLeader_RestoreVaultAccessors(t *testing.T) { s1.vault = tvc // Do a restore - if err := s1.restoreRevokingAccessors(); err != nil { + if err := s1.revokeVaultAccessorsOnRestore(); err != nil { t.Fatalf("Failed to restore: %v", err) } @@ -632,6 +632,35 @@ func TestLeader_RestoreVaultAccessors(t *testing.T) { } } +func TestLeader_revokeSITokenAccessorsOnRestore(t *testing.T) { + t.Parallel() + r := require.New(t) + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.NumSchedulers = 0 + }) + defer cleanupS1() + testutil.WaitForLeader(t, s1.RPC) + + // replace consul ACLs api with a mock for tracking calls + var consulACLsAPI mockConsulACLsAPI + s1.consulACLs = &consulACLsAPI + + // Insert a SI token accessor that should be revoked + fsmState := s1.fsm.State() + accessor := mock.SITokenAccessor() + err := fsmState.UpsertSITokenAccessors(100, []*structs.SITokenAccessor{accessor}) + r.NoError(err) + + // Do a restore + err = s1.revokeSITokenAccessorsOnRestore() + r.NoError(err) + + // Check the accessor was revoked + exp := []string{accessor.AccessorID} + r.ElementsMatch(exp, consulACLsAPI.revokeRequests) +} + func TestLeader_ClusterID(t *testing.T) { t.Parallel() diff --git a/nomad/mock/mock.go b/nomad/mock/mock.go index 896b3f20883..3dc86233333 100644 --- a/nomad/mock/mock.go +++ b/nomad/mock/mock.go @@ -382,6 +382,9 @@ func MaxParallelJob() *structs.Job { } // ConnectJob adds a Connect proxy sidecar group service to mock.Job. +// +// Note this does *not* include the Job.Register mutation that inserts the +// associated Sidecar Task (nor the hook that configures envoy as the default). func ConnectJob() *structs.Job { job := Job() tg := job.TaskGroups[0] @@ -399,6 +402,9 @@ func ConnectJob() *structs.Job { }, }, } + tg.Networks = structs.Networks{{ + Mode: "bridge", // always bridge ... for now? + }} return job } @@ -666,6 +672,100 @@ func ConnectAlloc() *structs.Allocation { return alloc } +func BatchConnectJob() *structs.Job { + job := &structs.Job{ + Region: "global", + ID: fmt.Sprintf("mock-connect-batch-job%s", uuid.Generate()), + Name: "mock-connect-batch-job", + Namespace: structs.DefaultNamespace, + Type: structs.JobTypeBatch, + Priority: 50, + AllAtOnce: false, + Datacenters: []string{"dc1"}, + TaskGroups: []*structs.TaskGroup{{ + Name: "mock-connect-batch-job", + Count: 1, + EphemeralDisk: &structs.EphemeralDisk{SizeMB: 150}, + Networks: []*structs.NetworkResource{{ + Mode: "bridge", + }}, + Tasks: []*structs.Task{{ + Name: "connect-proxy-testconnect", + Kind: "connect-proxy:testconnect", + Driver: "mock_driver", + Config: map[string]interface{}{ + "run_for": "500ms", + }, + LogConfig: structs.DefaultLogConfig(), + Resources: &structs.Resources{ + CPU: 500, + MemoryMB: 256, + Networks: []*structs.NetworkResource{{ + MBits: 50, + DynamicPorts: []structs.Port{{Label: "port1"}}, + }}, + }, + }}, + Services: []*structs.Service{{ + Name: "testconnect", + }}, + }}, + Meta: map[string]string{"owner": "shoenig"}, + Status: structs.JobStatusPending, + Version: 0, + CreateIndex: 42, + ModifyIndex: 99, + JobModifyIndex: 99, + } + if err := job.Canonicalize(); err != nil { + panic(err) + } + return job +} + +// BatchConnectAlloc is useful for testing task runner things. +func BatchConnectAlloc() *structs.Allocation { + alloc := &structs.Allocation{ + ID: uuid.Generate(), + EvalID: uuid.Generate(), + NodeID: "12345678-abcd-efab-cdef-123456789abc", + Namespace: structs.DefaultNamespace, + TaskGroup: "mock-connect-batch-job", + TaskResources: map[string]*structs.Resources{ + "connect-proxy-testconnect": { + CPU: 500, + MemoryMB: 256, + }, + }, + + AllocatedResources: &structs.AllocatedResources{ + Tasks: map[string]*structs.AllocatedTaskResources{ + "connect-proxy-testconnect": { + Cpu: structs.AllocatedCpuResources{CpuShares: 500}, + Memory: structs.AllocatedMemoryResources{MemoryMB: 256}, + }, + }, + Shared: structs.AllocatedSharedResources{ + Networks: []*structs.NetworkResource{{ + Mode: "bridge", + IP: "10.0.0.1", + DynamicPorts: []structs.Port{{ + Label: "connect-proxy-testconnect", + Value: 9999, + To: 9999, + }}, + }}, + DiskMB: 0, + }, + }, + Job: BatchConnectJob(), + DesiredStatus: structs.AllocDesiredStatusRun, + ClientStatus: structs.AllocClientStatusPending, + } + alloc.JobID = alloc.Job.ID + return alloc +} + func BatchAlloc() *structs.Allocation { alloc := &structs.Allocation{ ID: uuid.Generate(), @@ -824,6 +924,15 @@ func VaultAccessor() *structs.VaultAccessor { } } +func SITokenAccessor() *structs.SITokenAccessor { + return &structs.SITokenAccessor{ + NodeID: uuid.Generate(), + AllocID: uuid.Generate(), + AccessorID: uuid.Generate(), + TaskName: "foo", + } +} + func Deployment() *structs.Deployment { return &structs.Deployment{ ID: uuid.Generate(), diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index c5913c27319..4011dc421e8 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -20,6 +20,7 @@ import ( "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/raft" + "github.com/pkg/errors" ) const ( @@ -332,16 +333,25 @@ func (n *Node) deregister(args *structs.NodeBatchDeregisterRequest, } // Determine if there are any Vault accessors on the node - accessors, err := snap.VaultAccessorsByNode(ws, nodeID) - if err != nil { - n.logger.Error("looking up accessors for node failed", "node_id", nodeID, "error", err) + if accessors, err := snap.VaultAccessorsByNode(ws, nodeID); err != nil { + n.logger.Error("looking up vault accessors for node failed", "node_id", nodeID, "error", err) return err + } else if l := len(accessors); l > 0 { + n.logger.Debug("revoking vault accessors on node due to deregister", "num_accessors", l, "node_id", nodeID) + if err := n.srv.vault.RevokeTokens(context.Background(), accessors, true); err != nil { + n.logger.Error("revoking vault accessors for node failed", "node_id", nodeID, "error", err) + return err + } } - if l := len(accessors); l != 0 { - n.logger.Debug("revoking accessors on node due to deregister", "num_accessors", l, "node_id", nodeID) - if err := n.srv.vault.RevokeTokens(context.Background(), accessors, true); err != nil { - n.logger.Error("revoking accessors for node failed", "node_id", nodeID, "error", err) + // Determine if there are any SI token accessors on the node + if accessors, err := snap.SITokenAccessorsByNode(ws, nodeID); err != nil { + n.logger.Error("looking up si accessors for node failed", "node_id", nodeID, "error", err) + return err + } else if l := len(accessors); l > 0 { + n.logger.Debug("revoking si accessors on node due to deregister", "num_accessors", l, "node_id", nodeID) + if err := n.srv.consulACLs.RevokeTokens(context.Background(), accessors); err != nil { + n.logger.Error("revoking si accessors for node failed", "node_id", nodeID, "error", err) return err } } @@ -444,17 +454,26 @@ func (n *Node) UpdateStatus(args *structs.NodeUpdateStatusRequest, reply *struct // Check if we need to setup a heartbeat switch args.Status { case structs.NodeStatusDown: - // Determine if there are any Vault accessors on the node - accessors, err := n.srv.State().VaultAccessorsByNode(ws, args.NodeID) - if err != nil { - n.logger.Error("looking up accessors for node failed", "node_id", args.NodeID, "error", err) + // Determine if there are any Vault accessors on the node to cleanup + if accessors, err := n.srv.State().VaultAccessorsByNode(ws, args.NodeID); err != nil { + n.logger.Error("looking up vault accessors for node failed", "node_id", args.NodeID, "error", err) return err + } else if l := len(accessors); l > 0 { + n.logger.Debug("revoking vault accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID) + if err := n.srv.vault.RevokeTokens(context.Background(), accessors, true); err != nil { + n.logger.Error("revoking vault accessors for node failed", "node_id", args.NodeID, "error", err) + return err + } } - if l := len(accessors); l != 0 { - n.logger.Debug("revoking accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID) - if err := n.srv.vault.RevokeTokens(context.Background(), accessors, true); err != nil { - n.logger.Error("revoking accessors for node failed", "node_id", args.NodeID, "error", err) + // Determine if there are any SI token accessors on the node to cleanup + if accessors, err := n.srv.State().SITokenAccessorsByNode(ws, args.NodeID); err != nil { + n.logger.Error("looking up si accessors for node failed", "node_id", args.NodeID, "error", err) + return err + } else if l := len(accessors); l > 0 { + n.logger.Debug("revoking si accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID) + if err := n.srv.consulACLs.RevokeTokens(context.Background(), accessors); err != nil { + n.logger.Error("revoking si accessors for node failed", "node_id", args.NodeID, "error", err) return err } } @@ -1174,30 +1193,53 @@ func (n *Node) batchUpdate(future *structs.BatchFuture, updates []*structs.Alloc mErr.Errors = append(mErr.Errors, err) } - // For each allocation we are updating check if we should revoke any - // Vault Accessors - var revoke []*structs.VaultAccessor + // For each allocation we are updating, check if we should revoke any + // - Vault token accessors + // - Service Identity token accessors + var ( + revokeVault []*structs.VaultAccessor + revokeSI []*structs.SITokenAccessor + ) + for _, alloc := range updates { // Skip any allocation that isn't dead on the client if !alloc.Terminated() { continue } - // Determine if there are any Vault accessors for the allocation ws := memdb.NewWatchSet() - accessors, err := n.srv.State().VaultAccessorsByAlloc(ws, alloc.ID) - if err != nil { - n.logger.Error("looking up Vault accessors for alloc failed", "alloc_id", alloc.ID, "error", err) + + // Determine if there are any orphaned Vault accessors for the allocation + if accessors, err := n.srv.State().VaultAccessorsByAlloc(ws, alloc.ID); err != nil { + n.logger.Error("looking up vault accessors for alloc failed", "alloc_id", alloc.ID, "error", err) mErr.Errors = append(mErr.Errors, err) + } else { + revokeVault = append(revokeVault, accessors...) } - revoke = append(revoke, accessors...) + // Determine if there are any orphaned SI accessors for the allocation + if accessors, err := n.srv.State().SITokenAccessorsByAlloc(ws, alloc.ID); err != nil { + n.logger.Error("looking up si accessors for alloc failed", "alloc_id", alloc.ID, "error", err) + mErr.Errors = append(mErr.Errors, err) + } else { + revokeSI = append(revokeSI, accessors...) + } + } + + // Revoke any orphaned Vault accessors + if l := len(revokeVault); l > 0 { + n.logger.Debug("revoking vault accessors due to terminal allocations", "num_accessors", l) + if err := n.srv.vault.RevokeTokens(context.Background(), revokeVault, true); err != nil { + n.logger.Error("batched vault accessor revocation failed", "error", err) + mErr.Errors = append(mErr.Errors, err) + } } - if l := len(revoke); l != 0 { - n.logger.Debug("revoking accessors due to terminal allocations", "num_accessors", l) - if err := n.srv.vault.RevokeTokens(context.Background(), revoke, true); err != nil { - n.logger.Error("batched Vault accessor revocation failed", "error", err) + // Revoke any orphaned SI accessors + if l := len(revokeSI); l > 0 { + n.logger.Debug("revoking si accessors due to terminal allocations", "num_accessors", l) + if err := n.srv.consulACLs.RevokeTokens(context.Background(), revokeSI); err != nil { + n.logger.Error("batched si accessor revocation failed", "error", err) mErr.Errors = append(mErr.Errors, err) } } @@ -1369,102 +1411,94 @@ func (n *Node) createNodeEvals(nodeID string, nodeIndex uint64) ([]string, uint6 // DeriveVaultToken is used by the clients to request wrapped Vault tokens for // tasks -func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, - reply *structs.DeriveVaultTokenResponse) error { - - // setErr is a helper for setting the recoverable error on the reply and - // logging it - setErr := func(e error, recoverable bool) { - if e == nil { - return - } - re, ok := e.(*structs.RecoverableError) - if ok { - // No need to wrap if error is already a RecoverableError - reply.Error = re - } else { - reply.Error = structs.NewRecoverableError(e, recoverable).(*structs.RecoverableError) +func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, reply *structs.DeriveVaultTokenResponse) error { + setError := func(e error, recoverable bool) { + if e != nil { + if re, ok := e.(*structs.RecoverableError); ok { + reply.Error = re // No need to wrap if error is already a RecoverableError + } else { + reply.Error = structs.NewRecoverableError(e, recoverable).(*structs.RecoverableError) + } + n.logger.Error("DeriveVaultToken failed", "recoverable", recoverable, "error", e) } - - n.logger.Error("DeriveVaultToken failed", "recoverable", recoverable, "error", e) } if done, err := n.srv.forward("Node.DeriveVaultToken", args, args, reply); done { - setErr(err, structs.IsRecoverable(err) || err == structs.ErrNoLeader) + setError(err, structs.IsRecoverable(err) || err == structs.ErrNoLeader) return nil } defer metrics.MeasureSince([]string{"nomad", "client", "derive_vault_token"}, time.Now()) // Verify the arguments if args.NodeID == "" { - setErr(fmt.Errorf("missing node ID"), false) + setError(fmt.Errorf("missing node ID"), false) return nil } if args.SecretID == "" { - setErr(fmt.Errorf("missing node SecretID"), false) + setError(fmt.Errorf("missing node SecretID"), false) return nil } if args.AllocID == "" { - setErr(fmt.Errorf("missing allocation ID"), false) + setError(fmt.Errorf("missing allocation ID"), false) return nil } if len(args.Tasks) == 0 { - setErr(fmt.Errorf("no tasks specified"), false) + setError(fmt.Errorf("no tasks specified"), false) return nil } // Verify the following: // * The Node exists and has the correct SecretID - // * The Allocation exists on the specified node - // * The allocation contains the given tasks and they each require Vault + // * The Allocation exists on the specified Node + // * The Allocation contains the given tasks and they each require Vault // tokens snap, err := n.srv.fsm.State().Snapshot() if err != nil { - setErr(err, false) + setError(err, false) return nil } ws := memdb.NewWatchSet() node, err := snap.NodeByID(ws, args.NodeID) if err != nil { - setErr(err, false) + setError(err, false) return nil } if node == nil { - setErr(fmt.Errorf("Node %q does not exist", args.NodeID), false) + setError(fmt.Errorf("Node %q does not exist", args.NodeID), false) return nil } if node.SecretID != args.SecretID { - setErr(fmt.Errorf("SecretID mismatch"), false) + setError(fmt.Errorf("SecretID mismatch"), false) return nil } alloc, err := snap.AllocByID(ws, args.AllocID) if err != nil { - setErr(err, false) + setError(err, false) return nil } if alloc == nil { - setErr(fmt.Errorf("Allocation %q does not exist", args.AllocID), false) + setError(fmt.Errorf("Allocation %q does not exist", args.AllocID), false) return nil } if alloc.NodeID != args.NodeID { - setErr(fmt.Errorf("Allocation %q not running on Node %q", args.AllocID, args.NodeID), false) + setError(fmt.Errorf("Allocation %q not running on Node %q", args.AllocID, args.NodeID), false) return nil } if alloc.TerminalStatus() { - setErr(fmt.Errorf("Can't request Vault token for terminal allocation"), false) + setError(fmt.Errorf("Can't request Vault token for terminal allocation"), false) return nil } // Check the policies policies := alloc.Job.VaultPolicies() if policies == nil { - setErr(fmt.Errorf("Job doesn't require Vault policies"), false) + setError(fmt.Errorf("Job doesn't require Vault policies"), false) return nil } tg, ok := policies[alloc.TaskGroup] if !ok { - setErr(fmt.Errorf("Task group does not require Vault policies"), false) + setError(fmt.Errorf("Task group does not require Vault policies"), false) return nil } @@ -1479,7 +1513,7 @@ func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, if len(unneeded) != 0 { e := fmt.Errorf("Requested Vault tokens for tasks without defined Vault policies: %s", strings.Join(unneeded, ", ")) - setErr(e, false) + setError(e, false) return nil } @@ -1532,7 +1566,6 @@ func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, case input <- task: } } - }() // Wait for everything to complete or for an error @@ -1585,7 +1618,7 @@ func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, retry = true } - setErr(err, retry) + setError(err, retry) return nil } @@ -1595,6 +1628,244 @@ func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, return nil } +func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs.DeriveSITokenResponse) error { + setError := func(e error, recoverable bool) { + if e != nil { + if re, ok := e.(*structs.RecoverableError); ok { + reply.Error = re // No need to wrap if error is already a RecoverableError + } else { + reply.Error = structs.NewRecoverableError(e, recoverable).(*structs.RecoverableError) + } + n.logger.Error("DeriveSIToken failed", "recoverable", recoverable, "error", e) + } + } + + if done, err := n.srv.forward("Node.DeriveSIToken", args, args, reply); done { + setError(err, structs.IsRecoverable(err) || err == structs.ErrNoLeader) + return nil + } + defer metrics.MeasureSince([]string{"nomad", "client", "derive_si_token"}, time.Now()) + + // Verify the arguments + if err := args.Validate(); err != nil { + setError(err, false) + return nil + } + + // Get the ClusterID + clusterID, err := n.srv.ClusterID() + if err != nil { + setError(err, false) + return nil + } + + // Verify the following: + // * The Node exists and has the correct SecretID. + // * The Allocation exists on the specified Node. + // * The Allocation contains the given tasks, and each task requires a + // SI token. + + snap, err := n.srv.fsm.State().Snapshot() + if err != nil { + setError(err, false) + return nil + } + node, err := snap.NodeByID(nil, args.NodeID) + if err != nil { + setError(err, false) + return nil + } + if node == nil { + setError(errors.Errorf("Node %q does not exist", args.NodeID), false) + return nil + } + if node.SecretID != args.SecretID { + setError(errors.Errorf("SecretID mismatch"), false) + return nil + } + + alloc, err := snap.AllocByID(nil, args.AllocID) + if err != nil { + setError(err, false) + return nil + } + if alloc == nil { + setError(errors.Errorf("Allocation %q does not exist", args.AllocID), false) + return nil + } + if alloc.NodeID != args.NodeID { + setError(errors.Errorf("Allocation %q not running on node %q", args.AllocID, args.NodeID), false) + return nil + } + if alloc.TerminalStatus() { + setError(errors.Errorf("Cannot request SI token for terminal allocation"), false) + return nil + } + + // make sure task group contains at least one connect enabled service + tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) + if tg == nil { + setError(errors.Errorf("Allocation %q does not contain TaskGroup %q", args.AllocID, alloc.TaskGroup), false) + return nil + } + if !tgUsesConnect(tg) { + setError(errors.Errorf("TaskGroup %q does not use Connect", tg.Name), false) + return nil + } + + // make sure each task in args.Tasks is a connect-enabled task + unneeded := tasksNotUsingConnect(tg, args.Tasks) + if len(unneeded) > 0 { + setError(fmt.Errorf( + "Requested Consul Service Identity tokens for tasks that are not Connect enabled: %v", + strings.Join(unneeded, ", "), + ), false) + } + + // At this point the request is valid and we should contact Consul for tokens. + + // A lot of the following is copied from DeriveVaultToken which has been + // working fine for years. + + // Create an error group where we will spin up a fixed set of goroutines to + // handle deriving tokens but where if any fails the whole group is + // canceled. + g, ctx := errgroup.WithContext(context.Background()) + + // Cap the worker threads + numWorkers := len(args.Tasks) + if numWorkers > maxParallelRequestsPerDerive { + numWorkers = maxParallelRequestsPerDerive + } + + // would like to pull some of this out... + + // Create the SI tokens + input := make(chan string, numWorkers) + results := make(map[string]*structs.SIToken, numWorkers) + for i := 0; i < numWorkers; i++ { + g.Go(func() error { + for { + select { + case task, ok := <-input: + if !ok { + return nil + } + sii := ServiceIdentityIndex{ + ClusterID: clusterID, + AllocID: alloc.ID, + TaskName: task, + } + secret, err := n.srv.consulACLs.CreateToken(ctx, sii) + if err != nil { + return err + } + results[task] = secret + case <-ctx.Done(): + return nil + } + } + }) + } + + // Send the input + go func() { + defer close(input) + for _, task := range args.Tasks { + select { + case <-ctx.Done(): + return + case input <- task: + } + } + }() + + // Wait for everything to complete or for an error + createErr := g.Wait() + + accessors := make([]*structs.SITokenAccessor, 0, len(results)) + tokens := make(map[string]string, len(results)) + for task, secret := range results { + tokens[task] = secret.SecretID + accessor := &structs.SITokenAccessor{ + NodeID: alloc.NodeID, + AllocID: alloc.ID, + TaskName: task, + AccessorID: secret.AccessorID, + } + accessors = append(accessors, accessor) + } + + // If there was an error, revoke all created tokens. + if createErr != nil { + n.logger.Error("Consul Service Identity token creation for alloc failed", "alloc_id", alloc.ID, "error", createErr) + if revokeErr := n.srv.consulACLs.RevokeTokens(context.Background(), accessors); revokeErr != nil { + n.logger.Error("Consul Service Identity token revocation for alloc failed", "alloc_id", alloc.ID, "error", revokeErr) + } + + if recoverable, ok := createErr.(*structs.RecoverableError); ok { + reply.Error = recoverable + } else { + reply.Error = structs.NewRecoverableError(createErr, false).(*structs.RecoverableError) + } + + return nil + } + + // Commit the derived tokens to raft before returning them + requested := structs.SITokenAccessorsRequest{Accessors: accessors} + _, index, err := n.srv.raftApply(structs.ServiceIdentityAccessorRegisterRequestType, &requested) + if err != nil { + n.logger.Error("registering Service Identity token accessors for alloc failed", "alloc_id", alloc.ID, "error", err) + + // Determine if we can recover from the error + retry := false + switch err { + case raft.ErrNotLeader, raft.ErrLeadershipLost, raft.ErrRaftShutdown, raft.ErrEnqueueTimeout: + retry = true + } + setError(err, retry) + return nil + } + + // We made it! Now we can set the reply. + reply.Index = index + reply.Tokens = tokens + n.srv.setQueryMeta(&reply.QueryMeta) + return nil +} + +func tgUsesConnect(tg *structs.TaskGroup) bool { + for _, service := range tg.Services { + if service.Connect != nil { + if service.Connect.Native || service.Connect.SidecarService != nil { + return true + } + } + } + return false +} + +func tasksNotUsingConnect(tg *structs.TaskGroup, tasks []string) []string { + var unneeded []string + for _, task := range tasks { + tgTask := tg.LookupTask(task) + if !taskUsesConnect(tgTask) { + unneeded = append(unneeded, task) + } + } + return unneeded +} + +func taskUsesConnect(task *structs.Task) bool { + if task == nil { + // not even in the task group + return false + } + // todo(shoenig): TBD what Kind does a native task have? + return task.Kind.IsConnectProxy() +} + func (n *Node) EmitEvents(args *structs.EmitNodeEventsRequest, reply *structs.EmitNodeEventsResponse) error { if done, err := n.srv.forward("Node.EmitEvents", args, args, reply); done { return err diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index ed27eeed2e5..7b77ff50dd9 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -1,6 +1,7 @@ package nomad import ( + "errors" "fmt" "net" "reflect" @@ -11,6 +12,8 @@ import ( memdb "github.com/hashicorp/go-memdb" msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc" "github.com/hashicorp/nomad/acl" + "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/state" @@ -3031,6 +3034,214 @@ func TestClientEndpoint_DeriveVaultToken_VaultError(t *testing.T) { } } +func TestClientEndpoint_tgUsesConnect(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, tg *structs.TaskGroup, exp bool) { + result := tgUsesConnect(tg) + require.Equal(t, exp, result) + } + + t.Run("tg uses native", func(t *testing.T) { + try(t, &structs.TaskGroup{ + Services: []*structs.Service{ + {Connect: nil}, + {Connect: &structs.ConsulConnect{Native: true}}, + }, + }, true) + }) + + t.Run("tg uses sidecar", func(t *testing.T) { + try(t, &structs.TaskGroup{ + Services: []*structs.Service{{ + Connect: &structs.ConsulConnect{ + SidecarService: &structs.ConsulSidecarService{ + Port: "9090", + }, + }, + }}, + }, true) + }) + + t.Run("tg does not use connect", func(t *testing.T) { + try(t, &structs.TaskGroup{ + Services: []*structs.Service{ + {Connect: nil}, + }, + }, false) + }) +} + +func TestClientEndpoint_taskUsesConnect(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, task *structs.Task, exp bool) { + result := taskUsesConnect(task) + require.Equal(t, exp, result) + } + + t.Run("task uses connect", func(t *testing.T) { + try(t, &structs.Task{ + // see nomad.newConnectTask for how this works + Name: "connect-proxy-myservice", + Kind: "connect-proxy:myservice", + }, true) + }) + + t.Run("task does not use connect", func(t *testing.T) { + try(t, &structs.Task{ + Name: "mytask", + Kind: "incorrect:mytask", + }, false) + }) + + t.Run("task does not exist", func(t *testing.T) { + try(t, nil, false) + }) +} + +func TestClientEndpoint_tasksNotUsingConnect(t *testing.T) { + t.Parallel() + + taskGroup := &structs.TaskGroup{ + Name: "testgroup", + Tasks: []*structs.Task{{ + Name: "connect-proxy-service1", + Kind: "connect-proxy:service1", + }, { + Name: "incorrect-task3", + Kind: "incorrect:task3", + }, { + Name: "connect-proxy-service4", + Kind: "connect-proxy:service4", + }, { + Name: "incorrect-task5", + Kind: "incorrect:task5", + }}, + } + + requestingTasks := []string{ + "connect-proxy-service1", // yes + "task2", // does not exist + "task3", // no + "connect-proxy-service4", // yes + "task5", // no + } + + unneeded := tasksNotUsingConnect(taskGroup, requestingTasks) + exp := []string{"task2", "task3", "task5"} + require.Equal(t, exp, unneeded) +} + +func mutateConnectJob(t *testing.T, job *structs.Job) { + var jch jobConnectHook + _, warnings, err := jch.Mutate(job) + require.Empty(t, warnings) + require.NoError(t, err) +} + +func TestClientEndpoint_DeriveSIToken(t *testing.T) { + t.Parallel() + r := require.New(t) + + s1, cleanupS1 := TestServer(t, nil) // already sets consul mocks + defer cleanupS1() + + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Set allow unauthenticated (no operator token required) + s1.config.ConsulConfig.AllowUnauthenticated = helper.BoolToPtr(true) + + // Create the node + node := mock.Node() + err := state.UpsertNode(2, node) + r.NoError(err) + + // Create an alloc with a typical connect service (sidecar) defined + alloc := mock.ConnectAlloc() + alloc.NodeID = node.ID + mutateConnectJob(t, alloc.Job) // appends sidecar task + sidecarTask := alloc.Job.TaskGroups[0].Tasks[1] + + err = state.UpsertAllocs(3, []*structs.Allocation{alloc}) + r.NoError(err) + + request := &structs.DeriveSITokenRequest{ + NodeID: node.ID, + SecretID: node.SecretID, + AllocID: alloc.ID, + Tasks: []string{sidecarTask.Name}, + QueryOptions: structs.QueryOptions{Region: "global"}, + } + + var response structs.DeriveSITokenResponse + err = msgpackrpc.CallWithCodec(codec, "Node.DeriveSIToken", request, &response) + r.NoError(err) + r.Nil(response.Error) + + // Check the state store and ensure we created a Consul SI Token Accessor + ws := memdb.NewWatchSet() + accessors, err := state.SITokenAccessorsByNode(ws, node.ID) + r.NoError(err) + r.Equal(1, len(accessors)) // only asked for one + r.Equal("connect-proxy-testconnect", accessors[0].TaskName) // set by the mock + r.Equal(node.ID, accessors[0].NodeID) // should match + r.Equal(alloc.ID, accessors[0].AllocID) // should match + r.True(helper.IsUUID(accessors[0].AccessorID)) // should be set + r.Greater(accessors[0].CreateIndex, uint64(3)) // more than 3rd +} + +func TestClientEndpoint_DeriveSIToken_ConsulError(t *testing.T) { + t.Parallel() + r := require.New(t) + + s1, cleanupS1 := TestServer(t, nil) + defer cleanupS1() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Set allow unauthenticated (no operator token required) + s1.config.ConsulConfig.AllowUnauthenticated = helper.BoolToPtr(true) + + // Create the node + node := mock.Node() + err := state.UpsertNode(2, node) + r.NoError(err) + + // Create an alloc with a typical connect service (sidecar) defined + alloc := mock.ConnectAlloc() + alloc.NodeID = node.ID + mutateConnectJob(t, alloc.Job) // appends sidecar task + sidecarTask := alloc.Job.TaskGroups[0].Tasks[1] + + // rejigger the server to use a broken mock consul + mockACLsAPI := consul.NewMockACLsAPI(s1.logger) + mockACLsAPI.SetError(structs.NewRecoverableError(errors.New("consul recoverable error"), true)) + m, err := NewConsulACLsAPI(mockACLsAPI, s1.logger) + r.NoError(err) + s1.consulACLs = m + + err = state.UpsertAllocs(3, []*structs.Allocation{alloc}) + r.NoError(err) + + request := &structs.DeriveSITokenRequest{ + NodeID: node.ID, + SecretID: node.SecretID, + AllocID: alloc.ID, + Tasks: []string{sidecarTask.Name}, + QueryOptions: structs.QueryOptions{Region: "global"}, + } + + var response structs.DeriveSITokenResponse + err = msgpackrpc.CallWithCodec(codec, "Node.DeriveSIToken", request, &response) + r.NoError(err) + r.NotNil(response.Error) // error should be set + r.True(response.Error.IsRecoverable()) // and is recoverable +} + func TestClientEndpoint_EmitEvents(t *testing.T) { t.Parallel() require := require.New(t) diff --git a/nomad/server.go b/nomad/server.go index 9941c7861d9..2942575f340 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -203,6 +203,9 @@ type Server struct { // consulCatalog is used for discovering other Nomad Servers via Consul consulCatalog consul.CatalogAPI + // consulACLs is used for managing Consul Service Identity tokens. + consulACLs ConsulACLsAPI + // vault is the client for communicating with Vault. vault VaultClient @@ -263,7 +266,7 @@ type endpoints struct { // NewServer is used to construct a new Nomad server from the // configuration, potentially returning an error -func NewServer(config *Config, consulCatalog consul.CatalogAPI) (*Server, error) { +func NewServer(config *Config, consulCatalog consul.CatalogAPI, consulACLs consul.ACLsAPI) (*Server, error) { // Check the protocol version if err := config.CheckVersion(); err != nil { return nil, err @@ -340,6 +343,13 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI) (*Server, error) // Initialize the stats fetcher that autopilot will use. s.statsFetcher = NewStatsFetcher(s.logger, s.connPool, s.config.Region) + // Setup Consul (more) + if err := s.setupConsul(consulACLs); err != nil { + s.Shutdown() + s.logger.Error("failed to setup Consul ACL client", "error", err) + return nil, fmt.Errorf("Failed to setup Consul ACL client: %v", err) + } + // Setup Vault if err := s.setupVaultClient(); err != nil { s.Shutdown() @@ -951,6 +961,16 @@ func (s *Server) setupNodeDrainer() { s.nodeDrainer = drainer.NewNodeDrainer(c) } +// setupConsul is used to setup Server specific consul components. +func (s *Server) setupConsul(consulACLs consul.ACLsAPI) error { + c, err := NewConsulACLsAPI(consulACLs, s.logger) + if err != nil { + return err + } + s.consulACLs = c + return nil +} + // setupVaultClient is used to set up the Vault API client. func (s *Server) setupVaultClient() error { v, err := NewVaultClient(s.config.VaultConfig, s.logger, s.purgeVaultAccessors) diff --git a/nomad/state/schema.go b/nomad/state/schema.go index c4bdf35478d..a5ec55e71d3 100644 --- a/nomad/state/schema.go +++ b/nomad/state/schema.go @@ -41,6 +41,7 @@ func init() { evalTableSchema, allocTableSchema, vaultAccessorTableSchema, + siTokenAccessorTableSchema, aclPolicyTableSchema, aclTokenTableSchema, autopilotConfigTableSchema, @@ -67,7 +68,7 @@ func stateStoreSchema() *memdb.DBSchema { return db } -// indexTableSchema is used for +// indexTableSchema is used for tracking the most recent index used for each table. func indexTableSchema() *memdb.TableSchema { return &memdb.TableSchema{ Name: "index", @@ -550,6 +551,44 @@ func vaultAccessorTableSchema() *memdb.TableSchema { } } +// siTokenAccessorTableSchema returns the MemDB schema for the Service Identity +// token accessor table. This table tracks accessors for tokens created on behalf +// of allocations with Consul connect enabled tasks that need SI tokens. +func siTokenAccessorTableSchema() *memdb.TableSchema { + return &memdb.TableSchema{ + Name: siTokenAccessorTable, + Indexes: map[string]*memdb.IndexSchema{ + // The primary index is the accessor id + "id": { + Name: "id", + AllowMissing: false, + Unique: true, + Indexer: &memdb.StringFieldIndex{ + Field: "AccessorID", + }, + }, + + "alloc_id": { + Name: "alloc_id", + AllowMissing: false, + Unique: false, + Indexer: &memdb.StringFieldIndex{ + Field: "AllocID", + }, + }, + + "node_id": { + Name: "node_id", + AllowMissing: false, + Unique: false, + Indexer: &memdb.StringFieldIndex{ + Field: "NodeID", + }, + }, + }, + } +} + // aclPolicyTableSchema returns the MemDB schema for the policy table. // This table is used to store the policies which are referenced by tokens func aclPolicyTableSchema() *memdb.TableSchema { diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index a4733f6e294..2849584c128 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2565,6 +2565,137 @@ func (s *StateStore) VaultAccessorsByNode(ws memdb.WatchSet, nodeID string) ([]* return out, nil } +func indexEntry(table string, index uint64) *IndexEntry { + return &IndexEntry{ + Key: table, + Value: index, + } +} + +const siTokenAccessorTable = "si_token_accessors" + +// UpsertSITokenAccessors is used to register a set of Service Identity token accessors. +func (s *StateStore) UpsertSITokenAccessors(index uint64, accessors []*structs.SITokenAccessor) error { + txn := s.db.Txn(true) + defer txn.Abort() + + for _, accessor := range accessors { + // set the create index + accessor.CreateIndex = index + + // insert the accessor + if err := txn.Insert(siTokenAccessorTable, accessor); err != nil { + return errors.Wrap(err, "accessor insert failed") + } + } + + // update the index for this table + if err := txn.Insert("index", indexEntry(siTokenAccessorTable, index)); err != nil { + return errors.Wrap(err, "index update failed") + } + + txn.Commit() + return nil +} + +// DeleteSITokenAccessors is used to delete a set of Service Identity token accessors. +func (s *StateStore) DeleteSITokenAccessors(index uint64, accessors []*structs.SITokenAccessor) error { + txn := s.db.Txn(true) + defer txn.Abort() + + // Lookup each accessor + for _, accessor := range accessors { + // Delete the accessor + if err := txn.Delete(siTokenAccessorTable, accessor); err != nil { + return errors.Wrap(err, "accessor delete failed") + } + } + + // update the index for this table + if err := txn.Insert("index", indexEntry(siTokenAccessorTable, index)); err != nil { + return errors.Wrap(err, "index update failed") + } + + txn.Commit() + return nil +} + +// SITokenAccessor returns the given Service Identity token accessor. +func (s *StateStore) SITokenAccessor(ws memdb.WatchSet, accessorID string) (*structs.SITokenAccessor, error) { + txn := s.db.Txn(false) + defer txn.Abort() + + watchCh, existing, err := txn.FirstWatch(siTokenAccessorTable, "id", accessorID) + if err != nil { + return nil, errors.Wrap(err, "accessor lookup failed") + } + + ws.Add(watchCh) + + if existing != nil { + return existing.(*structs.SITokenAccessor), nil + } + + return nil, nil +} + +// SITokenAccessors returns an iterator of Service Identity token accessors. +func (s *StateStore) SITokenAccessors(ws memdb.WatchSet) (memdb.ResultIterator, error) { + txn := s.db.Txn(false) + defer txn.Abort() + + iter, err := txn.Get(siTokenAccessorTable, "id") + if err != nil { + return nil, err + } + + ws.Add(iter.WatchCh()) + + return iter, nil +} + +// SITokenAccessorsByAlloc returns all the Service Identity token accessors by alloc ID. +func (s *StateStore) SITokenAccessorsByAlloc(ws memdb.WatchSet, allocID string) ([]*structs.SITokenAccessor, error) { + txn := s.db.Txn(false) + defer txn.Abort() + + // Get an iterator over the accessors + iter, err := txn.Get(siTokenAccessorTable, "alloc_id", allocID) + if err != nil { + return nil, err + } + + ws.Add(iter.WatchCh()) + + var result []*structs.SITokenAccessor + for raw := iter.Next(); raw != nil; raw = iter.Next() { + result = append(result, raw.(*structs.SITokenAccessor)) + } + + return result, nil +} + +// SITokenAccessorsByNode returns all the Service Identity token accessors by node ID. +func (s *StateStore) SITokenAccessorsByNode(ws memdb.WatchSet, nodeID string) ([]*structs.SITokenAccessor, error) { + txn := s.db.Txn(false) + defer txn.Abort() + + // Get an iterator over the accessors + iter, err := txn.Get(siTokenAccessorTable, "node_id", nodeID) + if err != nil { + return nil, err + } + + ws.Add(iter.WatchCh()) + + var result []*structs.SITokenAccessor + for raw := iter.Next(); raw != nil; raw = iter.Next() { + result = append(result, raw.(*structs.SITokenAccessor)) + } + + return result, nil +} + // UpdateDeploymentStatus is used to make deployment status updates and // potentially make a evaluation func (s *StateStore) UpdateDeploymentStatus(index uint64, req *structs.DeploymentStatusUpdateRequest) error { @@ -4207,6 +4338,14 @@ func (r *StateRestore) VaultAccessorRestore(accessor *structs.VaultAccessor) err return nil } +// SITokenAccessorRestore is used to restore an SI token accessor +func (r *StateRestore) SITokenAccessorRestore(accessor *structs.SITokenAccessor) error { + if err := r.txn.Insert(siTokenAccessorTable, accessor); err != nil { + return errors.Wrap(err, "si token accessor insert failed") + } + return nil +} + // ACLPolicyRestore is used to restore an ACL policy func (r *StateRestore) ACLPolicyRestore(policy *structs.ACLPolicy) error { if err := r.txn.Insert("acl_policy", policy); err != nil { diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index b01018cc0c8..652458f573e 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -6793,6 +6793,201 @@ func TestStateStore_RestoreVaultAccessor(t *testing.T) { } } +func TestStateStore_UpsertSITokenAccessors(t *testing.T) { + t.Parallel() + r := require.New(t) + + state := testStateStore(t) + a1 := mock.SITokenAccessor() + a2 := mock.SITokenAccessor() + + ws := memdb.NewWatchSet() + var err error + + _, err = state.SITokenAccessor(ws, a1.AccessorID) + r.NoError(err) + + _, err = state.SITokenAccessor(ws, a2.AccessorID) + r.NoError(err) + + err = state.UpsertSITokenAccessors(1000, []*structs.SITokenAccessor{a1, a2}) + r.NoError(err) + + wsFired := watchFired(ws) + r.True(wsFired) + + noInsertWS := memdb.NewWatchSet() + result1, err := state.SITokenAccessor(noInsertWS, a1.AccessorID) + r.NoError(err) + r.Equal(a1, result1) + + result2, err := state.SITokenAccessor(noInsertWS, a2.AccessorID) + r.NoError(err) + r.Equal(a2, result2) + + iter, err := state.SITokenAccessors(noInsertWS) + r.NoError(err) + + count := 0 + for raw := iter.Next(); raw != nil; raw = iter.Next() { + count++ + accessor := raw.(*structs.SITokenAccessor) + // iterator is sorted by dynamic UUID + matches := reflect.DeepEqual(a1, accessor) || reflect.DeepEqual(a2, accessor) + r.True(matches) + } + r.Equal(2, count) + + index, err := state.Index(siTokenAccessorTable) + r.NoError(err) + r.Equal(uint64(1000), index) + + noInsertWSFired := watchFired(noInsertWS) + r.False(noInsertWSFired) +} + +func TestStateStore_DeleteSITokenAccessors(t *testing.T) { + t.Parallel() + r := require.New(t) + + state := testStateStore(t) + a1 := mock.SITokenAccessor() + a2 := mock.SITokenAccessor() + accessors := []*structs.SITokenAccessor{a1, a2} + var err error + + err = state.UpsertSITokenAccessors(1000, accessors) + r.NoError(err) + + ws := memdb.NewWatchSet() + _, err = state.SITokenAccessor(ws, a1.AccessorID) + r.NoError(err) + + err = state.DeleteSITokenAccessors(1001, accessors) + r.NoError(err) + + wsFired := watchFired(ws) + r.True(wsFired) + + wsPostDelete := memdb.NewWatchSet() + + result1, err := state.SITokenAccessor(wsPostDelete, a1.AccessorID) + r.NoError(err) + r.Nil(result1) // was deleted + + result2, err := state.SITokenAccessor(wsPostDelete, a2.AccessorID) + r.NoError(err) + r.Nil(result2) // was deleted + + index, err := state.Index(siTokenAccessorTable) + r.NoError(err) + r.Equal(uint64(1001), index) + + wsPostDeleteFired := watchFired(wsPostDelete) + r.False(wsPostDeleteFired) +} + +func TestStateStore_SITokenAccessorsByAlloc(t *testing.T) { + t.Parallel() + r := require.New(t) + + state := testStateStore(t) + alloc := mock.Alloc() + var accessors []*structs.SITokenAccessor + var expected []*structs.SITokenAccessor + + for i := 0; i < 5; i++ { + accessor := mock.SITokenAccessor() + accessor.AllocID = alloc.ID + expected = append(expected, accessor) + accessors = append(accessors, accessor) + } + + for i := 0; i < 10; i++ { + accessor := mock.SITokenAccessor() + accessor.AllocID = uuid.Generate() // does not belong to alloc + accessors = append(accessors, accessor) + } + + err := state.UpsertSITokenAccessors(1000, accessors) + r.NoError(err) + + ws := memdb.NewWatchSet() + result, err := state.SITokenAccessorsByAlloc(ws, alloc.ID) + r.NoError(err) + r.ElementsMatch(expected, result) + + index, err := state.Index(siTokenAccessorTable) + r.NoError(err) + r.Equal(uint64(1000), index) + + wsFired := watchFired(ws) + r.False(wsFired) +} + +func TestStateStore_SITokenAccessorsByNode(t *testing.T) { + t.Parallel() + r := require.New(t) + + state := testStateStore(t) + node := mock.Node() + var accessors []*structs.SITokenAccessor + var expected []*structs.SITokenAccessor + var err error + + for i := 0; i < 5; i++ { + accessor := mock.SITokenAccessor() + accessor.NodeID = node.ID + expected = append(expected, accessor) + accessors = append(accessors, accessor) + } + + for i := 0; i < 10; i++ { + accessor := mock.SITokenAccessor() + accessor.NodeID = uuid.Generate() // does not belong to node + accessors = append(accessors, accessor) + } + + err = state.UpsertSITokenAccessors(1000, accessors) + r.NoError(err) + + ws := memdb.NewWatchSet() + result, err := state.SITokenAccessorsByNode(ws, node.ID) + r.NoError(err) + r.ElementsMatch(expected, result) + + index, err := state.Index(siTokenAccessorTable) + r.NoError(err) + r.Equal(uint64(1000), index) + + wsFired := watchFired(ws) + r.False(wsFired) +} + +func TestStateStore_RestoreSITokenAccessor(t *testing.T) { + t.Parallel() + r := require.New(t) + + state := testStateStore(t) + a1 := mock.SITokenAccessor() + + restore, err := state.Restore() + r.NoError(err) + + err = restore.SITokenAccessorRestore(a1) + r.NoError(err) + + restore.Commit() + + ws := memdb.NewWatchSet() + result, err := state.SITokenAccessor(ws, a1.AccessorID) + r.NoError(err) + r.Equal(a1, result) + + wsFired := watchFired(ws) + r.False(wsFired) +} + func TestStateStore_UpsertACLPolicy(t *testing.T) { t.Parallel() diff --git a/nomad/structs/config/consul.go b/nomad/structs/config/consul.go index 8772c9bc8c4..9c324dde86b 100644 --- a/nomad/structs/config/consul.go +++ b/nomad/structs/config/consul.go @@ -143,13 +143,18 @@ func DefaultConsulConfig() *ConsulConfig { // AllowsUnauthenticated returns whether the config allows unauthenticated // creation of Consul Service Identity tokens for Consul Connect enabled Tasks. -func (a *ConsulConfig) AllowsUnauthenticated() bool { - return a.AllowUnauthenticated != nil && *a.AllowUnauthenticated +// +// If allow_unauthenticated is false, the operator must provide a token on +// job submission (i.e. -consul-token or $CONSUL_TOKEN). +// +// todo: seems like we should be using this somewhere... +func (c *ConsulConfig) AllowsUnauthenticated() bool { + return c.AllowUnauthenticated != nil && *c.AllowUnauthenticated } // Merge merges two Consul Configurations together. -func (a *ConsulConfig) Merge(b *ConsulConfig) *ConsulConfig { - result := a.Copy() +func (c *ConsulConfig) Merge(b *ConsulConfig) *ConsulConfig { + result := c.Copy() if b.ServerServiceName != "" { result.ServerServiceName = b.ServerServiceName diff --git a/nomad/structs/service_identities.go b/nomad/structs/service_identities.go new file mode 100644 index 00000000000..da964296758 --- /dev/null +++ b/nomad/structs/service_identities.go @@ -0,0 +1,64 @@ +package structs + +import "errors" + +// An SIToken is the important bits of a Service Identity token generated by Consul. +type SIToken struct { + TaskName string + AccessorID string + SecretID string +} + +// An SITokenAccessor is a reference to a created Service Identity token on +// behalf of an allocation's task. +type SITokenAccessor struct { + NodeID string + AllocID string + AccessorID string + TaskName string + + // Raft index + CreateIndex uint64 +} + +// SITokenAccessorsRequest is used to operate on a set of SITokenAccessor, like +// recording a set of accessors for an alloc into raft. +type SITokenAccessorsRequest struct { + Accessors []*SITokenAccessor +} + +// DeriveSITokenRequest is used to request Consul Service Identity tokens from +// the Nomad Server for the named tasks in the given allocation. +type DeriveSITokenRequest struct { + NodeID string + SecretID string + AllocID string + Tasks []string + QueryOptions +} + +func (r *DeriveSITokenRequest) Validate() error { + switch { + case r.NodeID == "": + return errors.New("missing node ID") + case r.SecretID == "": + return errors.New("missing node SecretID") + case r.AllocID == "": + return errors.New("missing allocation ID") + case len(r.Tasks) == 0: + return errors.New("no tasks specified") + default: + return nil + } +} + +type DeriveSITokenResponse struct { + // Tokens maps from Task Name to its associated SI token + Tokens map[string]string + + // Error stores any error that occurred. Errors are stored here so we can + // communicate whether it is retryable + Error *RecoverableError + + QueryMeta +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 5aa0a1370f3..00b51b87a00 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -84,6 +84,8 @@ const ( SchedulerConfigRequestType NodeBatchDeregisterRequestType ClusterMetadataRequestType + ServiceIdentityAccessorRegisterRequestType + ServiceIdentityAccessorDeregisterRequestType ) const ( @@ -926,27 +928,6 @@ type DeriveVaultTokenResponse struct { QueryMeta } -// DeriveSITokenRequest is used to request Consul Service Identity tokens from -// the Nomad Server for the named tasks in the given allocation. -type DeriveSITokenRequest struct { - NodeID string - SecretID string - AllocID string - Tasks []string - QueryOptions -} - -type DeriveSITokenResponse struct { - // Tokens maps from Task Name to its associated SI token - Tokens map[string]string - - // Error stores any error that occurred. Errors are stored here so we can - // communicate whether it is retryable - Error *RecoverableError - - QueryMeta -} - // GenericRequest is used to request where no // specific information is needed. type GenericRequest struct { @@ -3823,6 +3804,27 @@ func (j *Job) VaultPolicies() map[string]map[string]*Vault { return policies } +// Connect tasks returns the set of Consul Connect enabled tasks that will +// require a Service Identity token, if Consul ACLs are enabled. +// +// This method is meaningful only after the Job has passed through the job +// submission Mutator functions. +// +// task group -> []task +func (j *Job) ConnectTasks() map[string][]string { + m := make(map[string][]string) + for _, tg := range j.TaskGroups { + for _, task := range tg.Tasks { + if task.Kind.IsConnectProxy() { + // todo(shoenig): when we support native, probably need to check + // an additional TBD TaskKind as well. + m[tg.Name] = append(m[tg.Name], task.Name) + } + } + } + return m +} + // RequiredSignals returns a mapping of task groups to tasks to their required // set of signals func (j *Job) RequiredSignals() map[string]map[string][]string { @@ -5432,6 +5434,22 @@ type Task struct { Kind TaskKind } +// UsesConnect is for conveniently detecting if the Task is able to make use +// of Consul Connect features. This will be indicated in the TaskKind of the +// Task, which exports known types of Tasks. +// +// Currently only Consul Connect Proxy tasks are known. +// (Consul Connect Native tasks will be supported soon). +func (t *Task) UsesConnect() bool { + // todo(shoenig): native tasks + switch { + case t.Kind.IsConnectProxy(): + return true + default: + return false + } +} + func (t *Task) Copy() *Task { if t == nil { return nil @@ -5805,7 +5823,7 @@ func (t *Task) Warnings() error { // TaskKind identifies the special kinds of tasks using the following format: // '(:)`. The TaskKind can optionally include an identifier that -// is opague to the Task. This identier can be used to relate the task to some +// is opaque to the Task. This identifier can be used to relate the task to some // other entity based on the kind. // // For example, a task may have the TaskKind of `connect-proxy:service` where diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index dcb2dfcc82b..10edfa5b968 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -535,6 +535,47 @@ func TestJob_VaultPolicies(t *testing.T) { } } +func TestJob_ConnectTasks(t *testing.T) { + t.Parallel() + r := require.New(t) + + // todo(shoenig): this will need some updates when we support connect native + // tasks, which will have a different Kind format, probably. + + j0 := &Job{ + TaskGroups: []*TaskGroup{{ + Name: "tg1", + Tasks: []*Task{{ + Name: "connect-proxy-task1", + Kind: "connect-proxy:task1", + }, { + Name: "task2", + Kind: "task2", + }, { + Name: "connect-proxy-task3", + Kind: "connect-proxy:task3", + }}, + }, { + Name: "tg2", + Tasks: []*Task{{ + Name: "task1", + Kind: "task1", + }, { + Name: "connect-proxy-task2", + Kind: "connect-proxy:task2", + }}, + }}, + } + + connectTasks := j0.ConnectTasks() + + exp := map[string][]string{ + "tg1": {"connect-proxy-task1", "connect-proxy-task3"}, + "tg2": {"connect-proxy-task2"}, + } + r.Equal(exp, connectTasks) +} + func TestJob_RequiredSignals(t *testing.T) { j0 := &Job{} e0 := make(map[string]map[string][]string, 0) @@ -692,6 +733,27 @@ func TestJob_PartEqual(t *testing.T) { })) } +func TestTask_UsesConnect(t *testing.T) { + t.Parallel() + + t.Run("normal task", func(t *testing.T) { + task := testJob().TaskGroups[0].Tasks[0] + usesConnect := task.UsesConnect() + require.False(t, usesConnect) + }) + + t.Run("sidecar proxy", func(t *testing.T) { + task := &Task{ + Name: "connect-proxy-task1", + Kind: "connect-proxy:task1", + } + usesConnect := task.UsesConnect() + require.True(t, usesConnect) + }) + + // todo(shoenig): add native case +} + func TestTaskGroup_Validate(t *testing.T) { j := testJob() tg := &TaskGroup{ diff --git a/nomad/testing.go b/nomad/testing.go index d3e64413507..6afe2bd61b6 100644 --- a/nomad/testing.go +++ b/nomad/testing.go @@ -91,6 +91,8 @@ func TestServer(t testing.T, cb func(*Config)) (*Server, func()) { catalog := consul.NewMockCatalog(config.Logger) + acls := consul.NewMockACLsAPI(config.Logger) + for i := 10; i >= 0; i-- { // Get random ports, need to cleanup later ports := freeport.MustTake(2) @@ -102,7 +104,7 @@ func TestServer(t testing.T, cb func(*Config)) (*Server, func()) { config.SerfConfig.MemberlistConfig.BindPort = ports[1] // Create server - server, err := NewServer(config, catalog) + server, err := NewServer(config, catalog, acls) if err == nil { return server, func() { ch := make(chan error) From f70998e1d541d2ee9e59c74d7e705e38f1868ac9 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 18 Dec 2019 10:23:16 -0600 Subject: [PATCH 03/27] client: enable envoy bootstrap hook to set SI token When creating the envoy bootstrap configuration, we should append the "-token=" argument in the case where the sidsHook placed the token in the secrets directory. --- .../taskrunner/envoybootstrap_hook.go | 154 ++++++++++-- .../taskrunner/envoybootstrap_hook_test.go | 237 +++++++++++++++++- client/allocrunner/taskrunner/sids_hook.go | 34 ++- .../allocrunner/taskrunner/sids_hook_test.go | 31 ++- .../taskrunner/task_runner_hooks.go | 10 +- .../taskrunner/task_runner_test.go | 32 ++- client/allocrunner/taskrunner/tasklet.go | 2 +- client/allocrunner/taskrunner/volume_hook.go | 2 +- 8 files changed, 438 insertions(+), 64 deletions(-) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook.go b/client/allocrunner/taskrunner/envoybootstrap_hook.go index b24f1764279..5de2551a431 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook.go @@ -4,20 +4,28 @@ import ( "bytes" "context" "fmt" + "io/ioutil" "os" "os/exec" "path/filepath" "time" - log "github.com/hashicorp/go-hclog" + "github.com/hashicorp/go-hclog" "github.com/hashicorp/nomad/client/allocdir" "github.com/hashicorp/nomad/client/allocrunner/interfaces" agentconsul "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" + "github.com/pkg/errors" ) -var _ interfaces.TaskPrestartHook = &envoyBootstrapHook{} +const envoyBootstrapHookName = "envoy_bootstrap" + +type envoyBootstrapHookConfig struct { + alloc *structs.Allocation + consulHTTPAddr string + logger hclog.Logger +} const ( envoyBaseAdminPort = 19000 @@ -27,6 +35,7 @@ const ( // envoyBootstrapHook writes the bootstrap config for the Connect Envoy proxy // sidecar. type envoyBootstrapHook struct { + // alloc is the allocation with the envoy task being bootstrapped. alloc *structs.Allocation // Bootstrapping Envoy requires talking directly to Consul to generate @@ -34,20 +43,23 @@ type envoyBootstrapHook struct { // Consul's gRPC endpoint. consulHTTPAddr string - logger log.Logger + // executable is executable file that is consul + executable string + + // logger is used to log things + logger hclog.Logger } -func newEnvoyBootstrapHook(alloc *structs.Allocation, consulHTTPAddr string, logger log.Logger) *envoyBootstrapHook { - h := &envoyBootstrapHook{ - alloc: alloc, - consulHTTPAddr: consulHTTPAddr, +func newEnvoyBootstrapHook(c *envoyBootstrapHookConfig) *envoyBootstrapHook { + return &envoyBootstrapHook{ + alloc: c.alloc, + consulHTTPAddr: c.consulHTTPAddr, + logger: c.logger.Named(envoyBootstrapHookName), } - h.logger = logger.Named(h.Name()) - return h } func (envoyBootstrapHook) Name() string { - return "envoy_bootstrap" + return envoyBootstrapHookName } func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error { @@ -59,7 +71,7 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP serviceName := req.Task.Kind.Value() if serviceName == "" { - return fmt.Errorf("Connect proxy sidecar does not specify service name") + return errors.New("connect proxy sidecar does not specify service name") } tg := h.alloc.Job.LookupTaskGroup(h.alloc.TaskGroup) @@ -73,13 +85,12 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP } if service == nil { - return fmt.Errorf("Connect proxy sidecar task exists but no services configured with a sidecar") + return errors.New("connect proxy sidecar task exists but no services configured with a sidecar") } h.logger.Debug("bootstrapping Connect proxy sidecar", "task", req.Task.Name, "service", serviceName) - //TODO Should connect directly to Consul if the sidecar is running on - // the host netns. + //TODO Should connect directly to Consul if the sidecar is running on the host netns. grpcAddr := "unix://" + allocdir.AllocGRPCSocket // Envoy runs an administrative API on the loopback interface. If multiple sidecars @@ -92,24 +103,36 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP // Envoy bootstrap configuration may contain a Consul token, so write // it to the secrets directory like Vault tokens. - fn := filepath.Join(req.TaskDir.SecretsDir, "envoy_bootstrap.json") + bootstrapFilePath := filepath.Join(req.TaskDir.SecretsDir, "envoy_bootstrap.json") id := agentconsul.MakeAllocServiceID(h.alloc.ID, "group-"+tg.Name, service) - h.logger.Debug("bootstrapping envoy", "sidecar_for", service.Name, "boostrap_file", fn, "sidecar_for_id", id, "grpc_addr", grpcAddr, "admin_bind", envoyAdminBind) + h.logger.Debug("bootstrapping envoy", "sidecar_for", service.Name, "bootstrap_file", bootstrapFilePath, "sidecar_for_id", id, "grpc_addr", grpcAddr, "admin_bind", envoyAdminBind) + + siToken, err := h.maybeLoadSIToken(req.Task.Name, req.TaskDir.SecretsDir) + if err != nil { + h.logger.Error("failed to generate envoy bootstrap config", "sidecar_for", service.Name) + return errors.Wrap(err, "failed to generate envoy bootstrap config") + } + h.logger.Debug("check for SI token for task", "task", req.Task.Name, "exists", siToken != "") + + bootstrapArgs := envoyBootstrapArgs{ + sidecarFor: id, + grpcAddr: grpcAddr, + consulHTTPAddr: h.consulHTTPAddr, + envoyAdminBind: envoyAdminBind, + siToken: siToken, + }.args() + + // put old stuff in here // Since Consul services are registered asynchronously with this task // hook running, retry a small number of times with backoff. for tries := 3; ; tries-- { - cmd := exec.CommandContext(ctx, "consul", "connect", "envoy", - "-grpc-addr", grpcAddr, - "-http-addr", h.consulHTTPAddr, - "-admin-bind", envoyAdminBind, - "-bootstrap", - "-sidecar-for", id, - ) + + cmd := exec.CommandContext(ctx, "consul", bootstrapArgs...) // Redirect output to secrets/envoy_bootstrap.json - fd, err := os.Create(fn) + fd, err := os.Create(bootstrapFilePath) if err != nil { return fmt.Errorf("error creating secrets/envoy_bootstrap.json for envoy: %v", err) } @@ -122,7 +145,7 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP err = cmd.Run() // Close bootstrap.json - fd.Close() + _ = fd.Close() if err == nil { // Happy path! Bootstrap was created, exit. @@ -138,7 +161,7 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP // occurs, and (b) the file will either be rewritten on // retry or eventually garbage collected if the task // fails. - os.Remove(fn) + _ = os.Remove(bootstrapFilePath) // ExitErrors are recoverable since they indicate the // command was runnable but exited with a unsuccessful @@ -174,3 +197,82 @@ func buildEnvoyAdminBind(alloc *structs.Allocation, taskName string) string { } return fmt.Sprintf("localhost:%d", port) } + +func (h *envoyBootstrapHook) writeConfig(filename, config string) error { + if err := ioutil.WriteFile(filename, []byte(config), 0440); err != nil { + _ = os.Remove(filename) + return err + } + return nil +} + +func (_ *envoyBootstrapHook) retry(ctx context.Context) bool { + select { + case <-ctx.Done(): + return false + case <-time.After(2 * time.Second): + return true + } +} + +func (h *envoyBootstrapHook) execute(cmd *exec.Cmd) (string, error) { + var ( + stdout bytes.Buffer + stderr bytes.Buffer + ) + + cmd.Stdout = &stdout + cmd.Stderr = &stderr + + if err := cmd.Run(); err != nil { + _, recoverable := err.(*exec.ExitError) + // ExitErrors are recoverable since they indicate the + // command was runnable but exited with a unsuccessful + // error code. + return stderr.String(), structs.NewRecoverableError(err, recoverable) + } + return stdout.String(), nil +} + +type envoyBootstrapArgs struct { + sidecarFor string + grpcAddr string + envoyAdminBind string + consulHTTPAddr string + siToken string +} + +func (e envoyBootstrapArgs) args() []string { + arguments := []string{ + "connect", + "envoy", + "-grpc-addr", e.grpcAddr, + "-http-addr", e.consulHTTPAddr, + "-admin-bind", e.envoyAdminBind, + "-bootstrap", + "-sidecar-for", e.sidecarFor, + } + if e.siToken != "" { + arguments = append(arguments, []string{"-token", e.siToken}...) + } + return arguments +} + +// maybeLoadSIToken reads the SI token saved to disk in the secretes directory +// by the service identities prestart hook. This envoy bootstrap hook blocks +// until the sids hook completes, so if the SI token is required to exist (i.e. +// Consul ACLs are enabled), it will be in place by the time we try to read it. +func (h *envoyBootstrapHook) maybeLoadSIToken(task, dir string) (string, error) { + tokenPath := filepath.Join(dir, sidsTokenFile) + token, err := ioutil.ReadFile(tokenPath) + if err != nil { + if !os.IsNotExist(err) { + h.logger.Error("failed to load SI token", "task", task, "error", err) + return "", errors.Wrapf(err, "failed to load SI token for %s", task) + } + h.logger.Trace("no SI token to load", "task", task) + return "", nil // token file does not exist + } + h.logger.Trace("recovered pre-existing SI token", "task", task) + return string(token), nil +} diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go index 94e1c01622f..222fecd53b0 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go @@ -17,6 +17,7 @@ import ( agentconsul "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/args" "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" @@ -24,6 +25,213 @@ import ( var _ interfaces.TaskPrestartHook = (*envoyBootstrapHook)(nil) +func writeTmp(t *testing.T, s string, fm os.FileMode) string { + dir, err := ioutil.TempDir("", "envoy-") + require.NoError(t, err) + + fPath := filepath.Join(dir, sidsTokenFile) + err = ioutil.WriteFile(fPath, []byte(s), fm) + require.NoError(t, err) + + return dir +} + +func TestEnvoyBootstrapHook_maybeLoadSIToken(t *testing.T) { + t.Parallel() + + t.Run("file does not exist", func(t *testing.T) { + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{logger: testlog.HCLogger(t)}) + config, err := h.maybeLoadSIToken("task1", "/does/not/exist") + require.NoError(t, err) // absence of token is not an error + require.Equal(t, "", config) + }) + + t.Run("load token from file", func(t *testing.T) { + token := uuid.Generate() + f := writeTmp(t, token, 0440) + defer cleanupDir(t, f) + + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{logger: testlog.HCLogger(t)}) + config, err := h.maybeLoadSIToken("task1", f) + require.NoError(t, err) + require.Equal(t, token, config) + }) + + t.Run("file is unreadable", func(t *testing.T) { + token := uuid.Generate() + f := writeTmp(t, token, 0200) + defer cleanupDir(t, f) + + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{logger: testlog.HCLogger(t)}) + config, err := h.maybeLoadSIToken("task1", f) + require.Error(t, err) + require.False(t, os.IsNotExist(err)) + require.Equal(t, "", config) + }) +} + +func TestEnvoyBootstrapHook_envoyBootstrapArgs(t *testing.T) { + t.Parallel() + + t.Run("excluding SI token", func(t *testing.T) { + ebArgs := envoyBootstrapArgs{ + sidecarFor: "s1", + grpcAddr: "1.1.1.1", + consulHTTPAddr: "2.2.2.2", + envoyAdminBind: "localhost:3333", + } + args := ebArgs.args() + require.Equal(t, []string{"connect", "envoy", + "-grpc-addr", "1.1.1.1", + "-http-addr", "2.2.2.2", + "-admin-bind", "localhost:3333", + "-bootstrap", + "-sidecar-for", "s1", + }, args) + }) + + t.Run("including SI token", func(t *testing.T) { + token := uuid.Generate() + ebArgs := envoyBootstrapArgs{ + sidecarFor: "s1", + grpcAddr: "1.1.1.1", + consulHTTPAddr: "2.2.2.2", + envoyAdminBind: "localhost:3333", + siToken: token, + } + args := ebArgs.args() + require.Equal(t, []string{"connect", "envoy", + "-grpc-addr", "1.1.1.1", + "-http-addr", "2.2.2.2", + "-admin-bind", "localhost:3333", + "-bootstrap", + "-sidecar-for", "s1", + "-token", token, + }, args) + }) +} + +// dig through envoy config to look for consul token +type envoyConfig struct { + DynamicResources struct { + ADSConfig struct { + GRPCServices struct { + InitialMetadata []struct { + Key string `json:"key"` + Value string `json:"value"` + } `json:"initial_metadata"` + } `json:"grpc_services"` + } `json:"ads_config"` + } `json:"dynamic_resources"` +} + +func TestEnvoyBootstrapHook_with_SI_token(t *testing.T) { + t.Parallel() + testutil.RequireConsul(t) + + testconsul, err := consultest.NewTestServerConfig(func(c *consultest.TestServerConfig) { + // If -v wasn't specified squelch consul logging + if !testing.Verbose() { + c.Stdout = ioutil.Discard + c.Stderr = ioutil.Discard + } + }) + if err != nil { + t.Fatalf("error starting test consul server: %v", err) + } + defer testconsul.Stop() + + alloc := mock.Alloc() + alloc.AllocatedResources.Shared.Networks = []*structs.NetworkResource{ + { + Mode: "bridge", + IP: "10.0.0.1", + DynamicPorts: []structs.Port{ + { + Label: "connect-proxy-foo", + Value: 9999, + To: 9999, + }, + }, + }, + } + tg := alloc.Job.TaskGroups[0] + tg.Services = []*structs.Service{ + { + Name: "foo", + PortLabel: "9999", // Just need a valid port, nothing will bind to it + Connect: &structs.ConsulConnect{ + SidecarService: &structs.ConsulSidecarService{}, + }, + }, + } + sidecarTask := &structs.Task{ + Name: "sidecar", + Kind: "connect-proxy:foo", + } + tg.Tasks = append(tg.Tasks, sidecarTask) + + logger := testlog.HCLogger(t) + + allocDir, cleanup := allocdir.TestAllocDir(t, logger, "EnvoyBootstrap") + defer cleanup() + + // Register Group Services + consulConfig := consulapi.DefaultConfig() + consulConfig.Address = testconsul.HTTPAddr + consulAPIClient, err := consulapi.NewClient(consulConfig) + require.NoError(t, err) + + consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true) + go consulClient.Run() + defer consulClient.Shutdown() + require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter()))) + + // Run Connect bootstrap Hook + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ + alloc: alloc, + consulHTTPAddr: testconsul.HTTPAddr, + logger: logger, + }) + req := &interfaces.TaskPrestartRequest{ + Task: sidecarTask, + TaskDir: allocDir.NewTaskDir(sidecarTask.Name), + } + require.NoError(t, req.TaskDir.Build(false, nil)) + + // Insert service identity token in the secrets directory + token := uuid.Generate() + siTokenFile := filepath.Join(req.TaskDir.SecretsDir, sidsTokenFile) + err = ioutil.WriteFile(siTokenFile, []byte(token), 0440) + require.NoError(t, err) + + resp := &interfaces.TaskPrestartResponse{} + + // Run the hook + require.NoError(t, h.Prestart(context.Background(), req, resp)) + + // Assert it is Done + require.True(t, resp.Done) + + // Ensure the default path matches + env := map[string]string{ + taskenv.SecretsDir: req.TaskDir.SecretsDir, + } + f, err := os.Open(args.ReplaceEnv(structs.EnvoyBootstrapPath, env)) + require.NoError(t, err) + defer f.Close() + + // Assert bootstrap configuration is valid json + var out envoyConfig + require.NoError(t, json.NewDecoder(f).Decode(&out)) + + // Assert the SI token got set + key := out.DynamicResources.ADSConfig.GRPCServices.InitialMetadata[0].Key + value := out.DynamicResources.ADSConfig.GRPCServices.InitialMetadata[0].Value + require.Equal(t, "x-consul-token", key) + require.Equal(t, token, value) +} + // TestTaskRunner_EnvoyBootstrapHook_Prestart asserts the EnvoyBootstrapHook // creates Envoy's bootstrap.json configuration based on Connect proxy sidecars // registered for the task. @@ -83,13 +291,18 @@ func TestTaskRunner_EnvoyBootstrapHook_Ok(t *testing.T) { consulConfig.Address = testconsul.HTTPAddr consulAPIClient, err := consulapi.NewClient(consulConfig) require.NoError(t, err) + consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true) go consulClient.Run() defer consulClient.Shutdown() require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter()))) // Run Connect bootstrap Hook - h := newEnvoyBootstrapHook(alloc, testconsul.HTTPAddr, logger) + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ + alloc: alloc, + consulHTTPAddr: testconsul.HTTPAddr, + logger: logger, + }) req := &interfaces.TaskPrestartRequest{ Task: sidecarTask, TaskDir: allocDir.NewTaskDir(sidecarTask.Name), @@ -116,8 +329,14 @@ func TestTaskRunner_EnvoyBootstrapHook_Ok(t *testing.T) { defer f.Close() // Assert bootstrap configuration is valid json - var out map[string]interface{} + var out envoyConfig require.NoError(t, json.NewDecoder(f).Decode(&out)) + + // Assert no SI token got set + key := out.DynamicResources.ADSConfig.GRPCServices.InitialMetadata[0].Key + value := out.DynamicResources.ADSConfig.GRPCServices.InitialMetadata[0].Value + require.Equal(t, "x-consul-token", key) + require.Equal(t, "", value) } // TestTaskRunner_EnvoyBootstrapHook_Noop asserts that the Envoy bootstrap hook @@ -134,7 +353,11 @@ func TestTaskRunner_EnvoyBootstrapHook_Noop(t *testing.T) { // Run Envoy bootstrap Hook. Use invalid Consul address as it should // not get hit. - h := newEnvoyBootstrapHook(alloc, "http://127.0.0.2:1", logger) + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ + alloc: alloc, + consulHTTPAddr: "http://127.0.0.2:1", + logger: logger, + }) req := &interfaces.TaskPrestartRequest{ Task: task, TaskDir: allocDir.NewTaskDir(task.Name), @@ -214,7 +437,11 @@ func TestTaskRunner_EnvoyBootstrapHook_RecoverableError(t *testing.T) { // not running. // Run Connect bootstrap Hook - h := newEnvoyBootstrapHook(alloc, testconsul.HTTPAddr, logger) + h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ + alloc: alloc, + consulHTTPAddr: testconsul.HTTPAddr, + logger: logger, + }) req := &interfaces.TaskPrestartRequest{ Task: sidecarTask, TaskDir: allocDir.NewTaskDir(sidecarTask.Name), @@ -225,7 +452,7 @@ func TestTaskRunner_EnvoyBootstrapHook_RecoverableError(t *testing.T) { // Run the hook err = h.Prestart(context.Background(), req, resp) - require.Error(t, err) + require.EqualError(t, err, "error creating bootstrap configuration for Connect proxy sidecar: exit status 1") require.True(t, structs.IsRecoverable(err)) // Assert it is not Done diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index aca6dbb93b0..b9768f05c96 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -8,7 +8,7 @@ import ( "sync" "time" - hclog "github.com/hashicorp/go-hclog" + "github.com/hashicorp/go-hclog" "github.com/hashicorp/nomad/client/allocrunner/interfaces" ti "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces" "github.com/hashicorp/nomad/client/consul" @@ -51,7 +51,7 @@ type sidsHook struct { alloc *structs.Allocation // taskName is the name of the task - taskName string + task *structs.Task // sidsClient is the Consul client [proxy] for requesting SI tokens sidsClient consul.ServiceIdentityAPI @@ -72,7 +72,7 @@ type sidsHook struct { func newSIDSHook(c sidsHookConfig) *sidsHook { return &sidsHook{ alloc: c.alloc, - taskName: c.task.Name, + task: c.task, sidsClient: c.sidsClient, lifecycle: c.lifecycle, logger: c.logger.Named(sidsHookName), @@ -87,13 +87,14 @@ func (h *sidsHook) Name() string { func (h *sidsHook) Prestart( ctx context.Context, req *interfaces.TaskPrestartRequest, - _ *interfaces.TaskPrestartResponse) error { + resp *interfaces.TaskPrestartResponse) error { h.lock.Lock() defer h.lock.Unlock() // do nothing if we have already done things if h.earlyExit() { + resp.Done = true return nil } @@ -113,6 +114,9 @@ func (h *sidsHook) Prestart( } } + h.logger.Info("derived SI token", "task", h.task.Name, "si_task", h.task.Kind.Value()) + + resp.Done = true return nil } @@ -145,12 +149,13 @@ func (h *sidsHook) recoverToken(dir string) (string, error) { token, err := ioutil.ReadFile(tokenPath) if err != nil { if !os.IsNotExist(err) { + h.logger.Error("failed to recover SI token", "error", err) return "", errors.Wrap(err, "failed to recover SI token") } - h.logger.Trace("no pre-existing SI token to recover", "task", h.taskName) + h.logger.Trace("no pre-existing SI token to recover", "task", h.task.Name) return "", nil // token file does not exist yet } - h.logger.Trace("recovered pre-existing SI token", "task", h.taskName) + h.logger.Trace("recovered pre-existing SI token", "task", h.task.Name) return string(token), nil } @@ -185,25 +190,30 @@ func (h *sidsHook) kill(ctx context.Context, err error) { // tryDerive loops forever until a token is created, or ctx is done. func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- string) { + // Derive the SI token using the name of the proxied / native task, not the + // name of the literal sidecar task. The virtual ACL policy of the SI token + // is oriented this way. + siTaskName := h.task.Kind.Value() + for attempt := 0; backoff(ctx, attempt); attempt++ { - tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{h.taskName}) + tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{siTaskName}) switch { case err == nil: // nothing broke and we can return the token for the task - ch <- tokens[h.taskName] + ch <- tokens[siTaskName] return case structs.IsServerSide(err): // the error is known to be a server problem, just die - h.logger.Error("failed to derive SI token", "error", err, "server_side", true) + h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "si_task", siTaskName, "server_side", true) h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) case !structs.IsRecoverable(err): // the error is known not to be recoverable, just die - h.logger.Error("failed to derive SI token", "error", err, "recoverable", false) + h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "si_task", siTaskName, "recoverable", false) h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) default: @@ -213,8 +223,8 @@ func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- string) { } } -func backoff(ctx context.Context, i int) bool { - next := computeBackoff(i) +func backoff(ctx context.Context, attempt int) bool { + next := computeBackoff(attempt) select { case <-ctx.Done(): return false diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go index 859a008b293..7a2035d30ab 100644 --- a/client/allocrunner/taskrunner/sids_hook_test.go +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -28,6 +28,12 @@ func cleanupDir(t *testing.T, dir string) { require.NoError(t, err) } +func sidecar(task string) (string, structs.TaskKind) { + name := structs.ConnectProxyPrefix + "-" + task + kind := structs.TaskKind(structs.ConnectProxyPrefix + ":" + task) + return name, kind +} + func TestSIDSHook_recoverToken(t *testing.T) { t.Parallel() @@ -35,8 +41,13 @@ func TestSIDSHook_recoverToken(t *testing.T) { secrets := tmpDir(t) defer cleanupDir(t, secrets) + taskName, taskKind := sidecar("foo") + h := newSIDSHook(sidsHookConfig{ - task: &structs.Task{Name: "task1"}, + task: &structs.Task{ + Name: taskName, + Kind: taskKind, + }, logger: testlog.HCLogger(t), }) @@ -56,8 +67,13 @@ func TestSIDSHook_recoverToken_empty(t *testing.T) { secrets := tmpDir(t) defer cleanupDir(t, secrets) + taskName, taskKind := sidecar("foo") + h := newSIDSHook(sidsHookConfig{ - task: &structs.Task{Name: "task1"}, + task: &structs.Task{ + Name: taskName, + Kind: taskKind, + }, logger: testlog.HCLogger(t), }) @@ -73,9 +89,14 @@ func TestSIDSHook_deriveSIToken(t *testing.T) { secrets := tmpDir(t) defer cleanupDir(t, secrets) + taskName, taskKind := sidecar("task1") + h := newSIDSHook(sidsHookConfig{ - alloc: &structs.Allocation{ID: "a1"}, - task: &structs.Task{Name: "task1"}, + alloc: &structs.Allocation{ID: "a1"}, + task: &structs.Task{ + Name: taskName, + Kind: taskKind, + }, logger: testlog.HCLogger(t), sidsClient: consul.NewMockServiceIdentitiesClient(), }) @@ -83,7 +104,7 @@ func TestSIDSHook_deriveSIToken(t *testing.T) { ctx := context.Background() token, err := h.deriveSIToken(ctx) r.NoError(err) - r.True(helper.IsUUID(token)) + r.True(helper.IsUUID(token), "token: %q", token) } func TestSIDSHook_computeBackoff(t *testing.T) { diff --git a/client/allocrunner/taskrunner/task_runner_hooks.go b/client/allocrunner/taskrunner/task_runner_hooks.go index 365cbd3250e..a817eb0bd89 100644 --- a/client/allocrunner/taskrunner/task_runner_hooks.go +++ b/client/allocrunner/taskrunner/task_runner_hooks.go @@ -67,7 +67,6 @@ func (tr *TaskRunner) initHooks() { newArtifactHook(tr, hookLogger), newStatsHook(tr, tr.clientConfig.StatsCollectionInterval, hookLogger), newDeviceHook(tr.devicemanager, hookLogger), - newEnvoyBootstrapHook(alloc, tr.clientConfig.ConsulConfig.Addr, hookLogger), } // If Vault is enabled, add the hook @@ -107,6 +106,8 @@ func (tr *TaskRunner) initHooks() { })) } + // If this is a Connect sidecar proxy (or a Connect Native) service, + // add the sidsHook for requesting a Service Identity token (if ACLs). if task.UsesConnect() { tr.runnerHooks = append(tr.runnerHooks, newSIDSHook(sidsHookConfig{ alloc: tr.Alloc(), @@ -117,6 +118,13 @@ func (tr *TaskRunner) initHooks() { })) } + // envoy bootstrap must execute after sidsHook maybe sets SI token + tr.runnerHooks = append(tr.runnerHooks, newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ + alloc: alloc, + consulHTTPAddr: tr.clientConfig.ConsulConfig.Addr, + logger: hookLogger, + })) + // If there are any script checks, add the hook scriptCheckHook := newScriptCheckHook(scriptCheckHookConfig{ alloc: tr.Alloc(), diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 142ca0b027b..20dc536a8ce 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1082,29 +1082,31 @@ func TestTaskRunner_CheckWatcher_Restart(t *testing.T) { actualEvents[i] = string(e.Type) } require.Equal(t, actualEvents, expectedEvents) - require.Equal(t, structs.TaskStateDead, state.State) require.True(t, state.Failed, pretty.Sprint(state)) } -type mockEnvoyBootstrapHook struct{} +type mockEnvoyBootstrapHook struct { + // nothing +} -func (mockEnvoyBootstrapHook) Name() string { +func (_ *mockEnvoyBootstrapHook) Name() string { return "mock_envoy_bootstrap" } -func (*mockEnvoyBootstrapHook) Prestart(_ context.Context, _ *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error { +func (m *mockEnvoyBootstrapHook) Prestart(_ context.Context, _ *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error { resp.Done = true return nil } // The envoy bootstrap hook tries to connect to consul and run the envoy // bootstrap command, so turn it off when testing connect jobs that are not -// using envoy (for now?). -func disableEnvoyBootstrapHook(tr *TaskRunner) { +// using envoy. +func useMockEnvoyBootstrapHook(tr *TaskRunner) { + mock := new(mockEnvoyBootstrapHook) for i, hook := range tr.runnerHooks { if _, ok := hook.(*envoyBootstrapHook); ok { - tr.runnerHooks[i] = new(mockEnvoyBootstrapHook) + tr.runnerHooks[i] = mock } } } @@ -1138,7 +1140,8 @@ func TestTaskRunner_BlockForSIDSToken(t *testing.T) { tr, err := NewTaskRunner(trConfig) r.NoError(err) defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) - disableEnvoyBootstrapHook(tr) // turn off envoy bootstrap + useMockEnvoyBootstrapHook(tr) // mock the envoy bootstrap hook + go tr.Run() // assert task runner blocks on SI token @@ -1189,10 +1192,12 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { // control when we get a Consul SI token token := "12345678-1234-1234-1234-1234567890" + siTaskName := task.Kind.Value() deriveCount := 0 deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { if deriveCount > 0 { - return map[string]string{task.Name: token}, nil + + return map[string]string{siTaskName: token}, nil } deriveCount++ return nil, structs.NewRecoverableError(errors.New("try again later"), true) @@ -1204,7 +1209,7 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { tr, err := NewTaskRunner(trConfig) r.NoError(err) defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) - disableEnvoyBootstrapHook(tr) // turn off envoy bootstrap + useMockEnvoyBootstrapHook(tr) // mock the envoy bootstrap go tr.Run() // assert task runner blocks on SI token @@ -1247,14 +1252,15 @@ func TestTaskRunner_DeriveSIToken_Unrecoverable(t *testing.T) { defer cleanup() // SI token derivation suffers a non-retryable error + siTaskName := task.Kind.Value() siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) - siClient.SetDeriveTokenError(alloc.ID, []string{task.Name}, errors.New("non-recoverable")) + siClient.SetDeriveTokenError(alloc.ID, []string{siTaskName}, errors.New("non-recoverable")) tr, err := NewTaskRunner(trConfig) r.NoError(err) defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) - disableEnvoyBootstrapHook(tr) // turn off envoy bootstrap + useMockEnvoyBootstrapHook(tr) // mock the envoy bootstrap hook go tr.Run() // Wait for the task to die @@ -1826,7 +1832,7 @@ func TestTaskRunner_RestartSignalTask_NotRunning(t *testing.T) { require.Fail(t, "timed out waiting for task to complete") } - // Assert the task ran and never restarted + // Assert the task unblocked and never restarted state := tr.TaskState() require.Equal(t, structs.TaskStateDead, state.State) require.False(t, state.Failed) diff --git a/client/allocrunner/taskrunner/tasklet.go b/client/allocrunner/taskrunner/tasklet.go index 0f6d2e578c2..63bf56c093a 100644 --- a/client/allocrunner/taskrunner/tasklet.go +++ b/client/allocrunner/taskrunner/tasklet.go @@ -148,7 +148,7 @@ func (t *tasklet) run() *taskletHandle { select { case <-t.shutdownCh: - // We've been told to exit and just ran so exit + // We've been told to exit and just unblocked so exit return default: } diff --git a/client/allocrunner/taskrunner/volume_hook.go b/client/allocrunner/taskrunner/volume_hook.go index 1e0935aeab6..fd9a5a1d191 100644 --- a/client/allocrunner/taskrunner/volume_hook.go +++ b/client/allocrunner/taskrunner/volume_hook.go @@ -96,7 +96,7 @@ func (h *volumeHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartR return err } - // Because this hook is also ran on restores, we only add mounts that do not + // Because this hook is also unblocked on restores, we only add mounts that do not // already exist. Although this loop is somewhat expensive, there are only // a small number of mounts that exist within most individual tasks. We may // want to revisit this using a `hookdata` param to be "mount only once" From 2e2811f45c5ad7050cd866adca613f26cea233cc Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 19 Dec 2019 17:40:30 -0600 Subject: [PATCH 04/27] nomad: fixup token policy validation --- client/allocrunner/taskrunner/sids_hook.go | 13 +- .../taskrunner/task_runner_test.go | 6 +- command/agent/consul/acl_testing.go | 136 +++++++++--- command/agent/consul/client.go | 5 +- nomad/consul.go | 102 +-------- nomad/consul_policy.go | 123 +++++++++++ nomad/consul_policy_test.go | 203 ++++++++++++++++++ nomad/consul_test.go | 172 +-------------- nomad/job_endpoint_test.go | 4 +- nomad/node_endpoint.go | 3 + 10 files changed, 452 insertions(+), 315 deletions(-) create mode 100644 nomad/consul_policy.go create mode 100644 nomad/consul_policy_test.go diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index b9768f05c96..e2aaa7fca95 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -190,30 +190,25 @@ func (h *sidsHook) kill(ctx context.Context, err error) { // tryDerive loops forever until a token is created, or ctx is done. func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- string) { - // Derive the SI token using the name of the proxied / native task, not the - // name of the literal sidecar task. The virtual ACL policy of the SI token - // is oriented this way. - siTaskName := h.task.Kind.Value() - for attempt := 0; backoff(ctx, attempt); attempt++ { - tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{siTaskName}) + tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{h.task.Name}) switch { case err == nil: // nothing broke and we can return the token for the task - ch <- tokens[siTaskName] + ch <- tokens[h.task.Name] return case structs.IsServerSide(err): // the error is known to be a server problem, just die - h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "si_task", siTaskName, "server_side", true) + h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "server_side", true) h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) case !structs.IsRecoverable(err): // the error is known not to be recoverable, just die - h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "si_task", siTaskName, "recoverable", false) + h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "recoverable", false) h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) default: diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 20dc536a8ce..44530a4f51a 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1192,12 +1192,11 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { // control when we get a Consul SI token token := "12345678-1234-1234-1234-1234567890" - siTaskName := task.Kind.Value() deriveCount := 0 deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { if deriveCount > 0 { - return map[string]string{siTaskName: token}, nil + return map[string]string{task.Name: token}, nil } deriveCount++ return nil, structs.NewRecoverableError(errors.New("try again later"), true) @@ -1252,9 +1251,8 @@ func TestTaskRunner_DeriveSIToken_Unrecoverable(t *testing.T) { defer cleanup() // SI token derivation suffers a non-retryable error - siTaskName := task.Kind.Value() siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) - siClient.SetDeriveTokenError(alloc.ID, []string{siTaskName}, errors.New("non-recoverable")) + siClient.SetDeriveTokenError(alloc.ID, []string{task.Name}, errors.New("non-recoverable")) tr, err := NewTaskRunner(trConfig) r.NoError(err) diff --git a/command/agent/consul/acl_testing.go b/command/agent/consul/acl_testing.go index 05045ae7aa6..29481012209 100644 --- a/command/agent/consul/acl_testing.go +++ b/command/agent/consul/acl_testing.go @@ -35,6 +35,7 @@ func NewMockACLsAPI(l hclog.Logger) *MockACLsAPI { } } +// Example Consul policies for use in tests. const ( ExamplePolicyID1 = "a7c86856-0af5-4ab5-8834-03f4517e5564" ExamplePolicyID2 = "ffa1b66c-967d-4468-8775-c687b5cfc16e" @@ -47,6 +48,7 @@ func (m *MockACLsAPI) PolicyRead(policyID string, _ *api.QueryOptions) (*api.ACL case ExamplePolicyID1: return &api.ACLPolicy{ ID: ExamplePolicyID1, + Name: "example-policy-1", Rules: `service "service1" { policy = "write" }`, }, nil, nil @@ -69,44 +71,118 @@ service "service2" { policy = "write" }`, } } +// Example Consul roles for use in tests. const ( - ExampleOperatorToken1 = "59c219c2-47e4-43f3-bb45-258fd13f59d5" - ExampleOperatorToken2 = "868cc216-e123-4c2b-b362-f4d4c087de8e" - ExampleOperatorToken3 = "6177d1b9-c0f6-4118-b891-d818a3cb80b1" + ExampleRoleID1 = "e569a3a8-7dfb-b024-e492-e790fe3c4183" + ExampleRoleID2 = "88c825f4-d0da-1c2b-0c1c-cc9fe84c4468" + ExampleRoleID3 = "b19b2058-6205-6dff-d2b0-470f29b8e627" ) -func (m *MockACLsAPI) TokenReadSelf(q *api.QueryOptions) (*api.ACLToken, *api.QueryMeta, error) { - switch q.Token { - - case ExampleOperatorToken1: - return &api.ACLToken{ - SecretID: ExampleOperatorToken1, - AccessorID: "e341bacd-535e-417c-8f45-f88d7faffcaf", - Description: "operator token 1", - Policies: []*api.ACLTokenPolicyLink{{ - ID: ExamplePolicyID1, +func (m *MockACLsAPI) RoleRead(roleID string, _ *api.QueryOptions) (*api.ACLRole, *api.QueryMeta, error) { + switch roleID { + case ExampleRoleID1: + return &api.ACLRole{ + ID: ExampleRoleID1, + Name: "example-role-1", + Policies: []*api.ACLRolePolicyLink{{ + ID: ExamplePolicyID1, + Name: "example-policy-1", }}, + ServiceIdentities: nil, // would it ever make sense ? }, nil, nil - - case ExampleOperatorToken2: - return &api.ACLToken{ - SecretID: ExampleOperatorToken2, - AccessorID: "615b4d77-5164-4ec6-b616-24c0b24ac9cb", - Description: "operator token 2", - Policies: []*api.ACLTokenPolicyLink{{ - ID: ExamplePolicyID2, + case ExampleRoleID2: + return &api.ACLRole{ + ID: ExampleRoleID2, + Name: "example-role-2", + Policies: []*api.ACLRolePolicyLink{{ + ID: ExamplePolicyID2, + Name: "example-policy-2", }}, + ServiceIdentities: nil, }, nil, nil - - case ExampleOperatorToken3: - return &api.ACLToken{ - SecretID: ExampleOperatorToken3, - AccessorID: "6b7de0d7-15f7-45b4-95eb-fb775bfe3fdc", - Description: "operator token 3", - Policies: []*api.ACLTokenPolicyLink{{ - ID: ExamplePolicyID3, - }}, + case ExampleRoleID3: + return &api.ACLRole{ + ID: ExampleRoleID3, + Name: "example-role-3", + Policies: nil, // todo + ServiceIdentities: nil, // todo + ModifyIndex: 0, }, nil, nil + default: + return nil, nil, nil + } +} + +// Example Consul "operator" tokens for use in tests. + +const ( + ExampleOperatorTokenID0 = "de591604-86eb-1e6f-8b44-d4db752921ae" + ExampleOperatorTokenID1 = "59c219c2-47e4-43f3-bb45-258fd13f59d5" + ExampleOperatorTokenID2 = "868cc216-e123-4c2b-b362-f4d4c087de8e" + ExampleOperatorTokenID3 = "6177d1b9-c0f6-4118-b891-d818a3cb80b1" + ExampleOperatorTokenID4 = "754ae26c-f3cc-e088-d486-9c0d20f5eaea" +) + +var ( + ExampleOperatorToken0 = &api.ACLToken{ + SecretID: ExampleOperatorTokenID0, + AccessorID: "228865c6-3bf6-6683-df03-06dea2779088 ", + Description: "Operator Token 0", + } + + ExampleOperatorToken1 = &api.ACLToken{ + SecretID: ExampleOperatorTokenID1, + AccessorID: "e341bacd-535e-417c-8f45-f88d7faffcaf", + Description: "Operator Token 1", + Policies: []*api.ACLTokenPolicyLink{{ + ID: ExamplePolicyID1, + }}, + } + + ExampleOperatorToken2 = &api.ACLToken{ + SecretID: ExampleOperatorTokenID2, + AccessorID: "615b4d77-5164-4ec6-b616-24c0b24ac9cb", + Description: "Operator Token 2", + Policies: []*api.ACLTokenPolicyLink{{ + ID: ExamplePolicyID2, + }}, + } + + ExampleOperatorToken3 = &api.ACLToken{ + SecretID: ExampleOperatorTokenID3, + AccessorID: "6b7de0d7-15f7-45b4-95eb-fb775bfe3fdc", + Description: "Operator Token 3", + Policies: []*api.ACLTokenPolicyLink{{ + ID: ExamplePolicyID3, + }}, + } + + ExampleOperatorToken4 = &api.ACLToken{ + SecretID: ExampleOperatorTokenID4, + AccessorID: "7b5fdb1a-71e5-f3d8-2cfe-448d973f327d", + Description: "Operator Token 4", + Policies: nil, // no direct policy, only roles + Roles: []*api.ACLTokenRoleLink{{ + ID: ExampleRoleID1, + Name: "example-role-1", + }}, + } +) + +func (m *MockACLsAPI) TokenReadSelf(q *api.QueryOptions) (*api.ACLToken, *api.QueryMeta, error) { + switch q.Token { + + case ExampleOperatorTokenID1: + return ExampleOperatorToken1, nil, nil + + case ExampleOperatorTokenID2: + return ExampleOperatorToken2, nil, nil + + case ExampleOperatorTokenID3: + return ExampleOperatorToken3, nil, nil + + case ExampleOperatorTokenID4: + return ExampleOperatorToken4, nil, nil default: return nil, nil, errors.New("no such token") diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index f20e86b8435..fd9c6b2a934 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -93,15 +93,14 @@ type AgentAPI interface { UpdateTTL(id, output, status string) error } -// ACLsAPI is the consul/api.ACL API used by Nomad Server. +// ACLsAPI is the consul/api.ACL API subset used by Nomad Server. type ACLsAPI interface { - // todo: RoleRead (...) - // We are looking up by [operator token] SecretID, which implies we need // to use this method instead of the normal TokenRead, which can only be // used to lookup tokens by their AccessorID. TokenReadSelf(q *api.QueryOptions) (*api.ACLToken, *api.QueryMeta, error) PolicyRead(policyID string, q *api.QueryOptions) (*api.ACLPolicy, *api.QueryMeta, error) + RoleRead(roleID string, q *api.QueryOptions) (*api.ACLRole, *api.QueryMeta, error) TokenCreate(partial *api.ACLToken, q *api.WriteOptions) (*api.ACLToken, *api.WriteMeta, error) TokenDelete(accessorID string, q *api.WriteOptions) (*api.WriteMeta, error) TokenList(q *api.QueryOptions) ([]*api.ACLTokenListEntry, *api.QueryMeta, error) diff --git a/nomad/consul.go b/nomad/consul.go index 9df1409e6d9..4495adfb666 100644 --- a/nomad/consul.go +++ b/nomad/consul.go @@ -9,7 +9,6 @@ import ( "github.com/armon/go-metrics" "github.com/hashicorp/consul/api" "github.com/hashicorp/go-hclog" - "github.com/hashicorp/hcl" "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/pkg/errors" @@ -49,36 +48,6 @@ const ( ConsulPolicyWrite = "write" ) -// ConsulServiceRule represents a policy for a service -type ConsulServiceRule struct { - Name string `hcl:",key"` - Policy string -} - -type ConsulPolicy struct { - Services []*ConsulServiceRule `hcl:"service,expand"` - ServicePrefixes []*ConsulServiceRule `hcl:"service_prefix,expand"` -} - -func (cp *ConsulPolicy) IsEmpty() bool { - if cp == nil { - return true - } - return len(cp.Services) == 0 && len(cp.ServicePrefixes) == 0 -} - -func ParseConsulPolicy(s string) (*ConsulPolicy, error) { - cp := new(ConsulPolicy) - if err := hcl.Decode(cp, s); err != nil { - return nil, errors.Wrap(err, "failed to parse ACL policy") - } - if cp.IsEmpty() { - // the only use case for now, may as well validate asap - return nil, errors.New("consul policy contains no service rules") - } - return cp, nil -} - type ServiceIdentityIndex struct { ClusterID string AllocID string @@ -176,72 +145,6 @@ func (c *consulACLsAPI) CheckSIPolicy(_ context.Context, task, secretID string) return nil } -func (c *consulACLsAPI) hasSufficientPolicy(task string, token *api.ACLToken) (bool, error) { - - for _, policyRef := range token.Policies { - if allowable, err := c.policyAllowsServiceWrite(task, policyRef.ID); err != nil { - return false, err - } else if allowable { - return true, nil - } - } - - // todo: probably also need to go through each role and check all those - - return false, nil -} - -// policyAllowsServiceWrite -func (c *consulACLsAPI) policyAllowsServiceWrite(task string, policyID string) (bool, error) { - policy, _, err := c.aclClient.PolicyRead(policyID, &api.QueryOptions{ - AllowStale: false, - }) - if err != nil { - return false, err - } - - // compare policy to the necessary permission for service write - // e.g. service "db" { policy = "write" } - // e.g. service_prefix "" { policy == "write" } - cp, err := ParseConsulPolicy(policy.Rules) - if err != nil { - return false, err - } - - if c.allowsServiceWrite(task, cp) { - return true, nil - } - - return false, nil -} - -const ( - serviceNameWildcard = "*" -) - -func (_ *consulACLsAPI) allowsServiceWrite(task string, cp *ConsulPolicy) bool { - for _, service := range cp.Services { - name := strings.ToLower(service.Name) - policy := strings.ToLower(service.Policy) - if policy == ConsulPolicyWrite { - if name == task || name == serviceNameWildcard { - return true - } - } - } - - for _, servicePrefix := range cp.ServicePrefixes { - prefix := strings.ToLower(servicePrefix.Name) - policy := strings.ToLower(servicePrefix.Policy) - if policy == ConsulPolicyWrite { - if strings.HasPrefix(task, prefix) { - return true - } - } - } - return false -} - func (c *consulACLsAPI) CreateToken(ctx context.Context, sii ServiceIdentityIndex) (*structs.SIToken, error) { defer metrics.MeasureSince([]string{"nomad", "consul", "create_token"}, time.Now()) @@ -254,9 +157,12 @@ func (c *consulACLsAPI) CreateToken(ctx context.Context, sii ServiceIdentityInde // todo: rate limiting + // the token created must be for the service, not the sidecar of the service + // https://www.consul.io/docs/acl/acl-system.html#acl-service-identities + serviceName := strings.TrimPrefix(sii.TaskName, structs.ConnectProxyPrefix+"-") partial := &api.ACLToken{ Description: sii.Description(), - ServiceIdentities: []*api.ACLServiceIdentity{{ServiceName: sii.TaskName}}, + ServiceIdentities: []*api.ACLServiceIdentity{{ServiceName: serviceName}}, } token, _, err := c.aclClient.TokenCreate(partial, nil) diff --git a/nomad/consul_policy.go b/nomad/consul_policy.go new file mode 100644 index 00000000000..4f8a62379f0 --- /dev/null +++ b/nomad/consul_policy.go @@ -0,0 +1,123 @@ +package nomad + +import ( + "strings" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/hcl" + "github.com/pkg/errors" +) + +// ConsulServiceRule represents a policy for a service +type ConsulServiceRule struct { + Name string `hcl:",key"` + Policy string +} + +type ConsulPolicy struct { + Services []*ConsulServiceRule `hcl:"service,expand"` + ServicePrefixes []*ConsulServiceRule `hcl:"service_prefix,expand"` +} + +func (cp *ConsulPolicy) IsEmpty() bool { + if cp == nil { + return true + } + return len(cp.Services) == 0 && len(cp.ServicePrefixes) == 0 +} + +func ParseConsulPolicy(s string) (*ConsulPolicy, error) { + cp := new(ConsulPolicy) + if err := hcl.Decode(cp, s); err != nil { + return nil, errors.Wrap(err, "failed to parse ACL policy") + } + if cp.IsEmpty() { + // the only use case for now, may as well validate asap + return nil, errors.New("consul policy contains no service rules") + } + return cp, nil +} + +func (c *consulACLsAPI) hasSufficientPolicy(task string, token *api.ACLToken) (bool, error) { + // check each policy directly attached to the token + for _, policyRef := range token.Policies { + if allowable, err := c.policyAllowsServiceWrite(task, policyRef.ID); err != nil { + return false, err + } else if allowable { + return true, nil + } + } + + // check each policy on each role attached to the token + for _, roleLink := range token.Roles { + role, _, err := c.aclClient.RoleRead(roleLink.ID, &api.QueryOptions{ + AllowStale: false, + }) + if err != nil { + return false, err + } + + for _, policyLink := range role.Policies { + allowable, err := c.policyAllowsServiceWrite(task, policyLink.ID) + if err != nil { + return false, err + } + if allowable { + return true, nil + } + } + } + + return false, nil +} + +// policyAllowsServiceWrite +func (c *consulACLsAPI) policyAllowsServiceWrite(task string, policyID string) (bool, error) { + policy, _, err := c.aclClient.PolicyRead(policyID, &api.QueryOptions{ + AllowStale: false, + }) + if err != nil { + return false, err + } + + // compare policy to the necessary permission for service write + // e.g. service "db" { policy = "write" } + // e.g. service_prefix "" { policy == "write" } + cp, err := ParseConsulPolicy(policy.Rules) + if err != nil { + return false, err + } + + if c.allowsServiceWrite(task, cp) { + return true, nil + } + + return false, nil +} + +const ( + serviceNameWildcard = "*" +) + +func (_ *consulACLsAPI) allowsServiceWrite(task string, cp *ConsulPolicy) bool { + for _, service := range cp.Services { + name := strings.ToLower(service.Name) + policy := strings.ToLower(service.Policy) + if policy == ConsulPolicyWrite { + if name == task || name == serviceNameWildcard { + return true + } + } + } + + for _, servicePrefix := range cp.ServicePrefixes { + prefix := strings.ToLower(servicePrefix.Name) + policy := strings.ToLower(servicePrefix.Policy) + if policy == ConsulPolicyWrite { + if strings.HasPrefix(task, prefix) { + return true + } + } + } + return false +} diff --git a/nomad/consul_policy_test.go b/nomad/consul_policy_test.go new file mode 100644 index 00000000000..35f5d407bc6 --- /dev/null +++ b/nomad/consul_policy_test.go @@ -0,0 +1,203 @@ +package nomad + +import ( + "testing" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/helper/testlog" + "github.com/stretchr/testify/require" +) + +func TestConsulPolicy_ParseConsulPolicy(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, text string, expPolicy *ConsulPolicy, expErr string) { + policy, err := ParseConsulPolicy(text) + if expErr != "" { + require.EqualError(t, err, expErr) + require.True(t, policy.IsEmpty()) + } else { + require.NoError(t, err) + require.Equal(t, expPolicy, policy) + } + } + + t.Run("service", func(t *testing.T) { + text := `service "web" { policy = "read" }` + exp := &ConsulPolicy{ + Services: []*ConsulServiceRule{{Name: "web", Policy: "read"}}, + ServicePrefixes: []*ConsulServiceRule(nil), + } + try(t, text, exp, "") + }) + + t.Run("service_prefix", func(t *testing.T) { + text := `service_prefix "data" { policy = "write" }` + exp := &ConsulPolicy{ + Services: []*ConsulServiceRule(nil), + ServicePrefixes: []*ConsulServiceRule{{Name: "data", Policy: "write"}}, + } + try(t, text, exp, "") + }) + + t.Run("empty", func(t *testing.T) { + text := `` + expErr := "consul policy contains no service rules" + try(t, text, nil, expErr) + }) + + t.Run("malformed", func(t *testing.T) { + text := `this is not valid HCL!` + expErr := "failed to parse ACL policy: At 1:22: illegal char" + try(t, text, nil, expErr) + }) +} + +func TestConsulPolicy_IsEmpty(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, cp *ConsulPolicy, exp bool) { + result := cp.IsEmpty() + require.Equal(t, exp, result) + } + + t.Run("nil", func(t *testing.T) { + cp := (*ConsulPolicy)(nil) + try(t, cp, true) + }) + + t.Run("empty slices", func(t *testing.T) { + cp := &ConsulPolicy{ + Services: []*ConsulServiceRule(nil), + ServicePrefixes: []*ConsulServiceRule(nil), + } + try(t, cp, true) + }) + + t.Run("services nonempty", func(t *testing.T) { + cp := &ConsulPolicy{ + Services: []*ConsulServiceRule{{Name: "example", Policy: "write"}}, + } + try(t, cp, false) + }) + + t.Run("service_prefixes nonempty", func(t *testing.T) { + cp := &ConsulPolicy{ + ServicePrefixes: []*ConsulServiceRule{{Name: "pre", Policy: "read"}}, + } + try(t, cp, false) + }) +} + +func TestConsulACLsAPI_allowsServiceWrite(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, task string, cp *ConsulPolicy, exp bool) { + cAPI := new(consulACLsAPI) + result := cAPI.allowsServiceWrite(task, cp) + require.Equal(t, exp, result) + } + + makeCP := func(services [][2]string, prefixes [][2]string) *ConsulPolicy { + serviceRules := make([]*ConsulServiceRule, 0, len(services)) + for _, service := range services { + serviceRules = append(serviceRules, &ConsulServiceRule{Name: service[0], Policy: service[1]}) + } + prefixRules := make([]*ConsulServiceRule, 0, len(prefixes)) + for _, prefix := range prefixes { + prefixRules = append(prefixRules, &ConsulServiceRule{Name: prefix[0], Policy: prefix[1]}) + } + return &ConsulPolicy{Services: serviceRules, ServicePrefixes: prefixRules} + } + + t.Run("matching service policy write", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"task1", "write"}}, + nil, + ), true) + }) + + t.Run("matching service policy read", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"task1", "read"}}, + nil, + ), false) + }) + + t.Run("wildcard service policy write", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"*", "write"}}, + nil, + ), true) + }) + + t.Run("wrong service policy write", func(t *testing.T) { + try(t, "other1", makeCP( + [][2]string{{"task1", "write"}}, + nil, + ), false) + }) + + t.Run("matching prefix policy write", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"task-", "write"}}, + ), true) + }) + + t.Run("matching prefix policy read", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"task-", "read"}}, + ), false) + }) + + t.Run("empty prefix policy write", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"", "write"}}, + ), true) + }) + + t.Run("late matching service", func(t *testing.T) { + try(t, "task1", makeCP( + [][2]string{{"task0", "write"}, {"task1", "write"}}, + nil, + ), true) + }) + + t.Run("late matching prefix", func(t *testing.T) { + try(t, "task-one", makeCP( + nil, + [][2]string{{"foo-", "write"}, {"task-", "write"}}, + ), true) + }) +} + +func TestConsulACLsAPI_hasSufficientPolicy(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, task string, token *api.ACLToken, exp bool) { + logger := testlog.HCLogger(t) + cAPI := &consulACLsAPI{ + aclClient: consul.NewMockACLsAPI(logger), + logger: logger, + } + result, err := cAPI.hasSufficientPolicy(task, token) + require.NoError(t, err) + require.Equal(t, exp, result) + } + + t.Run("no useful policy or role", func(t *testing.T) { + try(t, "service1", consul.ExampleOperatorToken0, false) + }) + + t.Run("working policy only", func(t *testing.T) { + try(t, "service1", consul.ExampleOperatorToken1, true) + }) + + t.Run("working role only", func(t *testing.T) { + try(t, "service1", consul.ExampleOperatorToken4, true) + }) +} diff --git a/nomad/consul_test.go b/nomad/consul_test.go index 8a5d844e4d5..2f9d0f48fcf 100644 --- a/nomad/consul_test.go +++ b/nomad/consul_test.go @@ -13,87 +13,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestConsulPolicy_ParseConsulPolicy(t *testing.T) { - t.Parallel() - - try := func(t *testing.T, text string, expPolicy *ConsulPolicy, expErr string) { - policy, err := ParseConsulPolicy(text) - if expErr != "" { - require.EqualError(t, err, expErr) - require.True(t, policy.IsEmpty()) - } else { - require.NoError(t, err) - require.Equal(t, expPolicy, policy) - } - } - - t.Run("service", func(t *testing.T) { - text := `service "web" { policy = "read" }` - exp := &ConsulPolicy{ - Services: []*ConsulServiceRule{{Name: "web", Policy: "read"}}, - ServicePrefixes: []*ConsulServiceRule(nil), - } - try(t, text, exp, "") - }) - - t.Run("service_prefix", func(t *testing.T) { - text := `service_prefix "data" { policy = "write" }` - exp := &ConsulPolicy{ - Services: []*ConsulServiceRule(nil), - ServicePrefixes: []*ConsulServiceRule{{Name: "data", Policy: "write"}}, - } - try(t, text, exp, "") - }) - - t.Run("empty", func(t *testing.T) { - text := `` - expErr := "consul policy contains no service rules" - try(t, text, nil, expErr) - }) - - t.Run("malformed", func(t *testing.T) { - text := `this is not valid HCL!` - expErr := "failed to parse ACL policy: At 1:22: illegal char" - try(t, text, nil, expErr) - }) -} - -func TestConsulPolicy_IsEmpty(t *testing.T) { - t.Parallel() - - try := func(t *testing.T, cp *ConsulPolicy, exp bool) { - result := cp.IsEmpty() - require.Equal(t, exp, result) - } - - t.Run("nil", func(t *testing.T) { - cp := (*ConsulPolicy)(nil) - try(t, cp, true) - }) - - t.Run("empty slices", func(t *testing.T) { - cp := &ConsulPolicy{ - Services: []*ConsulServiceRule(nil), - ServicePrefixes: []*ConsulServiceRule(nil), - } - try(t, cp, true) - }) - - t.Run("services nonempty", func(t *testing.T) { - cp := &ConsulPolicy{ - Services: []*ConsulServiceRule{{Name: "example", Policy: "write"}}, - } - try(t, cp, false) - }) - - t.Run("service_prefixes nonempty", func(t *testing.T) { - cp := &ConsulPolicy{ - ServicePrefixes: []*ConsulServiceRule{{Name: "pre", Policy: "read"}}, - } - try(t, cp, false) - }) -} - var _ ConsulACLsAPI = (*consulACLsAPI)(nil) var _ ConsulACLsAPI = (*mockConsulACLsAPI)(nil) @@ -229,15 +148,15 @@ func TestConsulACLsAPI_CheckSIPolicy(t *testing.T) { } t.Run("operator has service write", func(t *testing.T) { - try(t, "service1", consul.ExampleOperatorToken1, "") + try(t, "service1", consul.ExampleOperatorTokenID1, "") }) t.Run("operator has service_prefix write", func(t *testing.T) { - try(t, "foo-service1", consul.ExampleOperatorToken2, "") + try(t, "foo-service1", consul.ExampleOperatorTokenID2, "") }) t.Run("operator permissions insufficient", func(t *testing.T) { - try(t, "service1", consul.ExampleOperatorToken3, + try(t, "service1", consul.ExampleOperatorTokenID3, "permission denied for \"service1\"", ) }) @@ -252,88 +171,3 @@ func TestConsulACLsAPI_CheckSIPolicy(t *testing.T) { ) }) } - -func TestConsulACLsAPI_allowsServiceWrite(t *testing.T) { - t.Parallel() - - try := func(t *testing.T, task string, cp *ConsulPolicy, exp bool) { - cAPI := new(consulACLsAPI) - result := cAPI.allowsServiceWrite(task, cp) - require.Equal(t, exp, result) - } - - makeCP := func(services [][2]string, prefixes [][2]string) *ConsulPolicy { - serviceRules := make([]*ConsulServiceRule, 0, len(services)) - for _, service := range services { - serviceRules = append(serviceRules, &ConsulServiceRule{Name: service[0], Policy: service[1]}) - } - prefixRules := make([]*ConsulServiceRule, 0, len(prefixes)) - for _, prefix := range prefixes { - prefixRules = append(prefixRules, &ConsulServiceRule{Name: prefix[0], Policy: prefix[1]}) - } - return &ConsulPolicy{Services: serviceRules, ServicePrefixes: prefixRules} - } - - t.Run("matching service policy write", func(t *testing.T) { - try(t, "task1", makeCP( - [][2]string{{"task1", "write"}}, - nil, - ), true) - }) - - t.Run("matching service policy read", func(t *testing.T) { - try(t, "task1", makeCP( - [][2]string{{"task1", "read"}}, - nil, - ), false) - }) - - t.Run("wildcard service policy write", func(t *testing.T) { - try(t, "task1", makeCP( - [][2]string{{"*", "write"}}, - nil, - ), true) - }) - - t.Run("wrong service policy write", func(t *testing.T) { - try(t, "other1", makeCP( - [][2]string{{"task1", "write"}}, - nil, - ), false) - }) - - t.Run("matching prefix policy write", func(t *testing.T) { - try(t, "task-one", makeCP( - nil, - [][2]string{{"task-", "write"}}, - ), true) - }) - - t.Run("matching prefix policy read", func(t *testing.T) { - try(t, "task-one", makeCP( - nil, - [][2]string{{"task-", "read"}}, - ), false) - }) - - t.Run("empty prefix policy write", func(t *testing.T) { - try(t, "task-one", makeCP( - nil, - [][2]string{{"", "write"}}, - ), true) - }) - - t.Run("late matching service", func(t *testing.T) { - try(t, "task1", makeCP( - [][2]string{{"task0", "write"}, {"task1", "write"}}, - nil, - ), true) - }) - - t.Run("late matching prefix", func(t *testing.T) { - try(t, "task-one", makeCP( - nil, - [][2]string{{"foo-", "write"}, {"task-", "write"}}, - ), true) - }) -} diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 257961fca32..9889b2f324d 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -320,8 +320,8 @@ func TestJobEndpoint_Register_Connect_AllowUnauthenticatedFalse(t *testing.T) { // Each variation of the provided Consul operator token noOpToken := "" unrecognizedOpToken := uuid.Generate() - unauthorizedOpToken := consul.ExampleOperatorToken3 - authorizedOpToken := consul.ExampleOperatorToken1 + unauthorizedOpToken := consul.ExampleOperatorTokenID3 + authorizedOpToken := consul.ExampleOperatorTokenID1 t.Run("no token provided", func(t *testing.T) { request := newRequest(job) diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 4011dc421e8..c92d1b6f53f 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -1714,6 +1714,8 @@ func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs. } // make sure each task in args.Tasks is a connect-enabled task + // note: the tasks at this point should be the "connect-sidecar-" name + // unneeded := tasksNotUsingConnect(tg, args.Tasks) if len(unneeded) > 0 { setError(fmt.Errorf( @@ -1751,6 +1753,7 @@ func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs. if !ok { return nil } + sii := ServiceIdentityIndex{ ClusterID: clusterID, AllocID: alloc.ID, From 6a0e31d42b85a903d6a4e5ecca6d5e61d892c0b2 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 2 Jan 2020 09:03:05 -0600 Subject: [PATCH 05/27] nomad: handle SI token revocations concurrently Be able to revoke SI token accessors concurrently, and also ratelimit the requests being made to Consul for the various ACL API uses. --- command/agent/consul/acl_testing.go | 2 +- nomad/consul.go | 277 ++++++++++++++++++++++++---- nomad/consul_test.go | 144 +++++++++++++-- nomad/job_endpoint.go | 6 - nomad/job_endpoint_test.go | 2 +- nomad/leader.go | 4 +- nomad/leader_test.go | 5 +- nomad/node_endpoint.go | 26 +-- nomad/node_endpoint_test.go | 3 +- nomad/server.go | 20 +- nomad/vault.go | 6 +- 11 files changed, 395 insertions(+), 100 deletions(-) diff --git a/command/agent/consul/acl_testing.go b/command/agent/consul/acl_testing.go index 29481012209..00979543f17 100644 --- a/command/agent/consul/acl_testing.go +++ b/command/agent/consul/acl_testing.go @@ -284,7 +284,7 @@ func (m *MockACLsAPI) tokenDelete(tokenID string, _ *api.WriteOptions) (*api.Wri } if _, exists := m.state.tokens[tokenID]; !exists { - return nil, errors.New("token does not exist") + return nil, nil // consul no-ops delete of non-existent token } delete(m.state.tokens, tokenID) diff --git a/nomad/consul.go b/nomad/consul.go index 4495adfb666..8c7846f2484 100644 --- a/nomad/consul.go +++ b/nomad/consul.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "strings" + "sync" "time" "github.com/armon/go-metrics" @@ -12,6 +13,7 @@ import ( "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/pkg/errors" + "golang.org/x/sync/errgroup" "golang.org/x/time/rate" ) @@ -28,7 +30,9 @@ const ( // revocation requests Nomad will make against Consul. siTokenMaxParallelRevokes = 64 - // todo: more revocation things + // siTokenRevocationIterval is the interval at which SI tokens that failed + // initial revocation are retried. + siTokenRevocationIterval = 5 * time.Minute ) const ( @@ -83,48 +87,92 @@ type ConsulACLsAPI interface { CreateToken(context.Context, ServiceIdentityIndex) (*structs.SIToken, error) // RevokeTokens instructs Consul to revoke the given token accessors. - RevokeTokens(context.Context, []*structs.SITokenAccessor) error + RevokeTokens(context.Context, []*structs.SITokenAccessor, bool) bool + // Stop is used to stop background token revocations. Intended to be used + // on Nomad Server shutdown. + Stop() + + // todo(shoenig): use list endpoint for finding orphaned tokens // ListTokens lists every token in Consul. - // - // To be used for reconciliation (later). - ListTokens() ([]string, error) + // ListTokens() ([]string, error) +} + +// PurgeSITokenAccessorFunc is called to remove SI Token accessors from the +// system (i.e. raft). If the function returns an error, the token will still +// be tracked and revocation attempts will retry in the background until there +// is a success. +type PurgeSITokenAccessorFunc func([]*structs.SITokenAccessor) error + +type SITokenStats struct { + TrackedForRevoke int } type consulACLsAPI struct { // aclClient is the API subset of the real consul client we need for - // managing Service Identity tokens. + // managing Service Identity tokens aclClient consul.ACLsAPI // limiter is used to rate limit requests to consul limiter *rate.Limiter + bgRevokeLock sync.Mutex + // Track accessors that must have their revocation retried in the background. + bgRetryRevocation []*structs.SITokenAccessor + // Track whether the background revocations have been stopped, to avoid + // creating tokens we would no longer be able to revoke. Expected to be used + // on a Server shutdown. + bgRevokeStopped bool + + // purgeFunc is the Nomad Server function that removes the reference to the + // SI token accessor from the persistent raft store + purgeFunc PurgeSITokenAccessorFunc + + // stopC is used to signal the client is shutting down and token revocation + // background goroutine should stop + stopC chan struct{} + // logger is used to log messages logger hclog.Logger } -func NewConsulACLsAPI(aclClient consul.ACLsAPI, logger hclog.Logger) (ConsulACLsAPI, error) { +func NewConsulACLsAPI(aclClient consul.ACLsAPI, logger hclog.Logger, purgeFunc PurgeSITokenAccessorFunc) *consulACLsAPI { c := &consulACLsAPI{ aclClient: aclClient, + limiter: rate.NewLimiter(siTokenRequestRateLimit, int(siTokenRequestRateLimit)), + stopC: make(chan struct{}), + purgeFunc: purgeFunc, logger: logger.Named("consul_acl"), - limiter: rate.NewLimiter(requestRateLimit, int(requestRateLimit)), } - return c, nil + + go c.bgRetryRevokeDaemon() + + return c +} + +// Stop stops background token revocations from happening. Once stopped, tokens +// may no longer be created. +func (c *consulACLsAPI) Stop() { + c.bgRevokeLock.Lock() + defer c.bgRevokeLock.Unlock() + + c.stopC <- struct{}{} + c.bgRevokeStopped = true } -func (c *consulACLsAPI) CheckSIPolicy(_ context.Context, task, secretID string) error { +func (c *consulACLsAPI) CheckSIPolicy(ctx context.Context, task, secretID string) error { + defer metrics.MeasureSince([]string{"nomad", "consul", "check_si_policy"}, time.Now()) + if id := strings.TrimSpace(secretID); id == "" { - // todo: check in tests return errors.New("missing consul token") } - // todo: log request time, result, etc. - - // todo: use ctx - - // todo: use rate limiting + // Ensure we are under our rate limit. + if err := c.limiter.Wait(ctx); err != nil { + return err + } - opToken, meta, err := c.aclClient.TokenReadSelf(&api.QueryOptions{ + opToken, _, err := c.aclClient.TokenReadSelf(&api.QueryOptions{ AllowStale: false, Token: secretID, }) @@ -132,8 +180,6 @@ func (c *consulACLsAPI) CheckSIPolicy(_ context.Context, task, secretID string) return errors.Wrap(err, "unable to validate operator consul token") } - _ = meta - allowable, err := c.hasSufficientPolicy(task, opToken) if err != nil { return errors.Wrap(err, "unable to validate operator consul token") @@ -148,16 +194,21 @@ func (c *consulACLsAPI) CheckSIPolicy(_ context.Context, task, secretID string) func (c *consulACLsAPI) CreateToken(ctx context.Context, sii ServiceIdentityIndex) (*structs.SIToken, error) { defer metrics.MeasureSince([]string{"nomad", "consul", "create_token"}, time.Now()) + // make sure the background token revocations have not been stopped + c.bgRevokeLock.Lock() + stopped := c.bgRevokeStopped + c.bgRevokeLock.Unlock() + + if stopped { + return nil, errors.New("client stopped and may no longer create tokens") + } + // sanity check the metadata for the token we want if err := sii.Validate(); err != nil { return nil, err } - // todo: use ctx - - // todo: rate limiting - - // the token created must be for the service, not the sidecar of the service + // the SI token created must be for the service, not the sidecar of the service // https://www.consul.io/docs/acl/acl-system.html#acl-service-identities serviceName := strings.TrimPrefix(sii.TaskName, structs.ConnectProxyPrefix+"-") partial := &api.ACLToken{ @@ -165,6 +216,11 @@ func (c *consulACLsAPI) CreateToken(ctx context.Context, sii ServiceIdentityInde ServiceIdentities: []*api.ACLServiceIdentity{{ServiceName: serviceName}}, } + // Ensure we are under our rate limit. + if err := c.limiter.Wait(ctx); err != nil { + return nil, err + } + token, _, err := c.aclClient.TokenCreate(partial, nil) if err != nil { return nil, err @@ -177,31 +233,180 @@ func (c *consulACLsAPI) CreateToken(ctx context.Context, sii ServiceIdentityInde }, nil } -func (c *consulACLsAPI) RevokeTokens(ctx context.Context, accessors []*structs.SITokenAccessor) error { +// RevokeTokens revokes the passed set of SI token accessors. If committed is set, +// the client's purge function is called (which purges the tokens from the Server's +// persistent store). If there is an error purging either because of Consul failures +// or because of the purge function, the revocation is retried in the background. +// +// The revocation of an SI token accessor is idempotent. +// +// A return value of true indicates one or more accessors were stored for +// a revocation retry attempt in the background (intended for tests). +func (c *consulACLsAPI) RevokeTokens(ctx context.Context, accessors []*structs.SITokenAccessor, committed bool) bool { defer metrics.MeasureSince([]string{"nomad", "consul", "revoke_tokens"}, time.Now()) - // todo: use ctx + nTokens := float32(len(accessors)) - // todo: rate limiting - - for _, accessor := range accessors { - if err := c.revokeToken(ctx, accessor); err != nil { - // todo: accumulate errors and IDs that are going to need another attempt - return err + if err := c.parallelRevoke(ctx, accessors); err != nil { + // If these tokens were uncommitted into raft, it is a best effort to + // revoke them now. If this immediate revocation does not work, Nomad loses + // track of them and will need to do a brute reconciliation later. This + // should happen rarely, and will be implemented soon. + if !committed { + metrics.IncrCounter([]string{"nomad", "consul", "undistributed_si_tokens_abandoned"}, nTokens) } + + c.logger.Warn("failed to revoke tokens, will reattempt later", "error", err) + c.storeForRevocation(accessors) + return true } - return nil + if !committed { + // Un-committed tokens were revoked without incident (nothing to purge) + metrics.IncrCounter([]string{"nomad", "consul", "undistributed_si_tokens_revoked"}, nTokens) + return false + } + + // Committed tokens were revoked without incident, now purge them + if err := c.purgeFunc(accessors); err != nil { + c.logger.Error("failed to purge SI token accessors", "error", err) + c.storeForRevocation(accessors) + return true + } + + // Track that the SI tokens were revoked and purged successfully + metrics.IncrCounter([]string{"nomad", "consul", "distributed_si_tokens_revoked"}, nTokens) + return false +} + +func (c *consulACLsAPI) storeForRevocation(accessors []*structs.SITokenAccessor) { + c.bgRevokeLock.Lock() + defer c.bgRevokeLock.Unlock() + + // copy / append the set of accessors we must track for revocation in the + // background + c.bgRetryRevocation = append(c.bgRetryRevocation, accessors...) } -func (c *consulACLsAPI) revokeToken(_ context.Context, accessor *structs.SITokenAccessor) error { +func (c *consulACLsAPI) parallelRevoke(ctx context.Context, accessors []*structs.SITokenAccessor) error { + g, pCtx := errgroup.WithContext(ctx) + + // Cap the handlers + handlers := len(accessors) + if handlers > siTokenMaxParallelRevokes { + handlers = siTokenMaxParallelRevokes + } + + // Revoke the SI Token Accessors + input := make(chan *structs.SITokenAccessor, handlers) + for i := 0; i < handlers; i++ { + g.Go(func() error { + for { + select { + case accessor, ok := <-input: + if !ok { + return nil + } + if err := c.singleRevoke(ctx, accessor); err != nil { + return errors.Wrapf(err, + "failed to revoke SI token accessor (alloc %q, node %q, task %q)", + accessor.AllocID, accessor.NodeID, accessor.TaskName, + ) + } + case <-pCtx.Done(): + return nil + } + } + }) + } + + // Send the input + go func() { + defer close(input) + for _, accessor := range accessors { + select { + case <-pCtx.Done(): + return + case input <- accessor: + } + } + }() + + // Wait for everything to complete + return g.Wait() +} + +func (c *consulACLsAPI) singleRevoke(ctx context.Context, accessor *structs.SITokenAccessor) error { c.logger.Trace("revoke SI token", "task", accessor.TaskName, "alloc_id", accessor.AllocID, "node_id", accessor.NodeID) + + // Ensure we are under our rate limit. + if err := c.limiter.Wait(ctx); err != nil { + return err + } + + // Consul will no-op the deletion of a non-existent token (no error) _, err := c.aclClient.TokenDelete(accessor.AccessorID, nil) return err } -func (c *consulACLsAPI) ListTokens() ([]string, error) { - defer metrics.MeasureSince([]string{"nomad", "consul", "list_tokens"}, time.Now()) +func (c *consulACLsAPI) bgRetryRevokeDaemon() { + ticker := time.NewTicker(siTokenRevocationIterval) + defer ticker.Stop() + + for { + select { + case <-c.stopC: + return + case <-ticker.C: + c.bgRetryRevoke() + } + } +} + +func (c *consulACLsAPI) bgRetryRevoke() { + c.bgRevokeLock.Lock() + defer c.bgRevokeLock.Unlock() + + // fast path, nothing to do + if len(c.bgRetryRevocation) == 0 { + return + } + // unlike vault tokens, SI tokens do not have a TTL, and so we must try to + // remove all SI token accessors, every time, until they're gone + toPurge := make([]*structs.SITokenAccessor, len(c.bgRetryRevocation), len(c.bgRetryRevocation)) + copy(toPurge, c.bgRetryRevocation) + + if err := c.parallelRevoke(context.Background(), toPurge); err != nil { + c.logger.Warn("background token revocation failed", "error", err) + return + } + + // Call the revocation function + if err := c.purgeFunc(toPurge); err != nil { + // Just try again later (revocation is idempotent) + c.logger.Error("token revocation failed", "error", err) + return + } + + // Track that the SI tokens were revoked successfully + nTokens := float32(len(toPurge)) + metrics.IncrCounter([]string{"nomad", "consul", "distributed_tokens_revoked"}, nTokens) + + // Reset the list of accessors to retry, since we just removed them all. + c.bgRetryRevocation = nil +} + +func (c *consulACLsAPI) ListTokens() ([]string, error) { + // defer metrics.MeasureSince([]string{"nomad", "consul", "list_tokens"}, time.Now()) return nil, errors.New("not yet implemented") } + +// purgeSITokenAccessors is the Nomad Server method which will remove the set +// of SI token accessors from the persistent raft store. +func (s *Server) purgeSITokenAccessors(accessors []*structs.SITokenAccessor) error { + // Commit this update via Raft + request := structs.SITokenAccessorsRequest{Accessors: accessors} + _, _, err := s.raftApply(structs.ServiceIdentityAccessorDeregisterRequestType, request) + return err +} diff --git a/nomad/consul_test.go b/nomad/consul_test.go index 2f9d0f48fcf..9bd2d09d7e7 100644 --- a/nomad/consul_test.go +++ b/nomad/consul_test.go @@ -3,7 +3,9 @@ package nomad import ( "context" "errors" + "sync" "testing" + "time" "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper" @@ -11,13 +13,21 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" + "golang.org/x/time/rate" ) var _ ConsulACLsAPI = (*consulACLsAPI)(nil) var _ ConsulACLsAPI = (*mockConsulACLsAPI)(nil) +type revokeRequest struct { + accessorID string + committed bool +} + type mockConsulACLsAPI struct { - revokeRequests []string + lock sync.Mutex + revokeRequests []revokeRequest + stopped bool } func (m *mockConsulACLsAPI) CheckSIPolicy(_ context.Context, _, _ string) error { @@ -32,13 +42,41 @@ func (m *mockConsulACLsAPI) ListTokens() ([]string, error) { panic("not implemented yet") } -func (m *mockConsulACLsAPI) RevokeTokens(_ context.Context, accessors []*structs.SITokenAccessor) error { +func (m *mockConsulACLsAPI) Stop() { + m.lock.Lock() + defer m.lock.Unlock() + m.stopped = true +} + +type mockPurgingServer struct { + purgedAccessorIDs []string + failure error +} + +func (mps *mockPurgingServer) purgeFunc(accessors []*structs.SITokenAccessor) error { + if mps.failure != nil { + return mps.failure + } + for _, accessor := range accessors { - m.revokeRequests = append(m.revokeRequests, accessor.AccessorID) + mps.purgedAccessorIDs = append(mps.purgedAccessorIDs, accessor.AccessorID) } return nil } +func (m *mockConsulACLsAPI) RevokeTokens(_ context.Context, accessors []*structs.SITokenAccessor, committed bool) bool { + m.lock.Lock() + defer m.lock.Unlock() + + for _, accessor := range accessors { + m.revokeRequests = append(m.revokeRequests, revokeRequest{ + accessorID: accessor.AccessorID, + committed: committed, + }) + } + return false +} + func TestConsulACLsAPI_CreateToken(t *testing.T) { t.Parallel() @@ -47,8 +85,7 @@ func TestConsulACLsAPI_CreateToken(t *testing.T) { aclAPI := consul.NewMockACLsAPI(logger) aclAPI.SetError(expErr) - c, err := NewConsulACLsAPI(aclAPI, logger) - require.NoError(t, err) + c := NewConsulACLsAPI(aclAPI, logger, nil) ctx := context.Background() sii := ServiceIdentityIndex{ @@ -86,8 +123,7 @@ func TestConsulACLsAPI_RevokeTokens(t *testing.T) { logger := testlog.HCLogger(t) aclAPI := consul.NewMockACLsAPI(logger) - c, err := NewConsulACLsAPI(aclAPI, logger) - require.NoError(t, err) + c := NewConsulACLsAPI(aclAPI, logger, nil) ctx := context.Background() generated, err := c.CreateToken(ctx, ServiceIdentityIndex{ @@ -112,34 +148,110 @@ func TestConsulACLsAPI_RevokeTokens(t *testing.T) { t.Run("revoke token success", func(t *testing.T) { ctx, c, token := setup(t, nil) - err := c.RevokeTokens(ctx, accessors(token.AccessorID)) - require.NoError(t, err) + retryLater := c.RevokeTokens(ctx, accessors(token.AccessorID), false) + require.False(t, retryLater) }) t.Run("revoke token non-existent", func(t *testing.T) { ctx, c, _ := setup(t, nil) - err := c.RevokeTokens(ctx, accessors(uuid.Generate())) - require.EqualError(t, err, "token does not exist") + retryLater := c.RevokeTokens(ctx, accessors(uuid.Generate()), false) + require.False(t, retryLater) }) t.Run("revoke token error", func(t *testing.T) { exp := errors.New("consul broke") ctx, c, token := setup(t, exp) - err := c.RevokeTokens(ctx, accessors(token.AccessorID)) - require.EqualError(t, err, exp.Error()) + retryLater := c.RevokeTokens(ctx, accessors(token.AccessorID), false) + require.True(t, retryLater) }) } +func TestConsulACLsAPI_bgRetryRevoke(t *testing.T) { + t.Parallel() + + // manually create so the bg daemon does not run, letting us explicitly + // call and test bgRetryRevoke + setup := func(t *testing.T) (*consulACLsAPI, *mockPurgingServer) { + logger := testlog.HCLogger(t) + aclAPI := consul.NewMockACLsAPI(logger) + server := new(mockPurgingServer) + shortWait := rate.Limit(1 * time.Millisecond) + + return &consulACLsAPI{ + aclClient: aclAPI, + purgeFunc: server.purgeFunc, + limiter: rate.NewLimiter(shortWait, int(shortWait)), + stopC: make(chan struct{}), + logger: logger, + }, server + } + + t.Run("retry revoke no items", func(t *testing.T) { + c, server := setup(t) + c.bgRetryRevoke() + require.Empty(t, server) + }) + + t.Run("retry revoke success", func(t *testing.T) { + c, server := setup(t) + accessorID := uuid.Generate() + c.bgRetryRevocation = append(c.bgRetryRevocation, &structs.SITokenAccessor{ + NodeID: uuid.Generate(), + AllocID: uuid.Generate(), + AccessorID: accessorID, + TaskName: "task1", + }) + require.Empty(t, server.purgedAccessorIDs) + c.bgRetryRevoke() + require.Equal(t, 1, len(server.purgedAccessorIDs)) + require.Equal(t, accessorID, server.purgedAccessorIDs[0]) + require.Empty(t, c.bgRetryRevocation) // should be empty now + }) + + t.Run("retry revoke failure", func(t *testing.T) { + c, server := setup(t) + server.failure = errors.New("revocation fail") + accessorID := uuid.Generate() + c.bgRetryRevocation = append(c.bgRetryRevocation, &structs.SITokenAccessor{ + NodeID: uuid.Generate(), + AllocID: uuid.Generate(), + AccessorID: accessorID, + TaskName: "task1", + }) + require.Empty(t, server.purgedAccessorIDs) + c.bgRetryRevoke() + require.Equal(t, 1, len(c.bgRetryRevocation)) // non-empty because purge failed + require.Equal(t, accessorID, c.bgRetryRevocation[0].AccessorID) + }) +} + +func TestConsulACLsAPI_Stop(t *testing.T) { + t.Parallel() + + setup := func(t *testing.T) *consulACLsAPI { + logger := testlog.HCLogger(t) + return NewConsulACLsAPI(nil, logger, nil) + } + + c := setup(t) + c.Stop() + _, err := c.CreateToken(context.Background(), ServiceIdentityIndex{ + ClusterID: "", + AllocID: "", + TaskName: "", + }) + require.Error(t, err) +} + func TestConsulACLsAPI_CheckSIPolicy(t *testing.T) { t.Parallel() try := func(t *testing.T, service, token string, expErr string) { logger := testlog.HCLogger(t) aclAPI := consul.NewMockACLsAPI(logger) - cAPI, err := NewConsulACLsAPI(aclAPI, logger) - require.NoError(t, err) + cAPI := NewConsulACLsAPI(aclAPI, logger, nil) - err = cAPI.CheckSIPolicy(context.Background(), service, token) + err := cAPI.CheckSIPolicy(context.Background(), service, token) if expErr != "" { require.EqualError(t, err, expErr) } else { diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 215bb54ef58..e7f74b27b3e 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -218,24 +218,18 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis // helper function that checks if the "operator token" supplied with the // job has sufficient ACL permissions for establishing consul connect services checkOperatorToken := func(task string) error { - fmt.Println("@@ check op token for:", task, "au:", j.srv.config.ConsulConfig.AllowsUnauthenticated()) - if j.srv.config.ConsulConfig.AllowsUnauthenticated() { // if consul.allow_unauthenticated is enabled (which is the default) // just let the Job through without checking anything. - fmt.Println("@@ allows unauthenticated = true, not checking") return nil } proxiedTask := strings.TrimPrefix(task, structs.ConnectProxyPrefix+"-") - fmt.Println("@@ proxiedTask:", proxiedTask) ctx := context.Background() if err := j.srv.consulACLs.CheckSIPolicy(ctx, proxiedTask, args.Job.ConsulToken); err != nil { // not much in the way of exported error types, we could parse // the content, but all errors are going to be failures anyway - fmt.Println("stopped in check for", proxiedTask) return errors.Wrap(err, "operator token denied") } - fmt.Println("got passed check for", proxiedTask) return nil } diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 9889b2f324d..aba840ce32f 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -328,7 +328,7 @@ func TestJobEndpoint_Register_Connect_AllowUnauthenticatedFalse(t *testing.T) { request.Job.ConsulToken = noOpToken var response structs.JobRegisterResponse err := msgpackrpc.CallWithCodec(codec, "Job.Register", request, &response) - require.EqualError(t, err, "operator token denied: unable to validate operator consul token: no such token") + require.EqualError(t, err, "operator token denied: missing consul token") }) t.Run("unknown token provided", func(t *testing.T) { diff --git a/nomad/leader.go b/nomad/leader.go index 8f02c179a26..a2b886d9ed4 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -419,9 +419,7 @@ func (s *Server) revokeSITokenAccessorsOnRestore() error { if len(toRevoke) > 0 { ctx := context.Background() - if err := s.consulACLs.RevokeTokens(ctx, toRevoke); err != nil { - return errors.Wrap(err, "failed to revoke SI tokens") - } + _ = s.consulACLs.RevokeTokens(ctx, toRevoke, true) } return nil diff --git a/nomad/leader_test.go b/nomad/leader_test.go index 980a215a799..a1abce5d27d 100644 --- a/nomad/leader_test.go +++ b/nomad/leader_test.go @@ -657,7 +657,10 @@ func TestLeader_revokeSITokenAccessorsOnRestore(t *testing.T) { r.NoError(err) // Check the accessor was revoked - exp := []string{accessor.AccessorID} + exp := []revokeRequest{{ + accessorID: accessor.AccessorID, + committed: true, + }} r.ElementsMatch(exp, consulACLsAPI.revokeRequests) } diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index c92d1b6f53f..d9ee28ef6ef 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -350,10 +350,7 @@ func (n *Node) deregister(args *structs.NodeBatchDeregisterRequest, return err } else if l := len(accessors); l > 0 { n.logger.Debug("revoking si accessors on node due to deregister", "num_accessors", l, "node_id", nodeID) - if err := n.srv.consulACLs.RevokeTokens(context.Background(), accessors); err != nil { - n.logger.Error("revoking si accessors for node failed", "node_id", nodeID, "error", err) - return err - } + _ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, true) } reply.EvalIDs = append(reply.EvalIDs, evalIDs...) @@ -472,10 +469,7 @@ func (n *Node) UpdateStatus(args *structs.NodeUpdateStatusRequest, reply *struct return err } else if l := len(accessors); l > 0 { n.logger.Debug("revoking si accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID) - if err := n.srv.consulACLs.RevokeTokens(context.Background(), accessors); err != nil { - n.logger.Error("revoking si accessors for node failed", "node_id", args.NodeID, "error", err) - return err - } + _ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, true) } default: ttl, err := n.srv.resetHeartbeatTimer(args.NodeID) @@ -1226,7 +1220,7 @@ func (n *Node) batchUpdate(future *structs.BatchFuture, updates []*structs.Alloc } } - // Revoke any orphaned Vault accessors + // Revoke any orphaned Vault token accessors if l := len(revokeVault); l > 0 { n.logger.Debug("revoking vault accessors due to terminal allocations", "num_accessors", l) if err := n.srv.vault.RevokeTokens(context.Background(), revokeVault, true); err != nil { @@ -1235,13 +1229,10 @@ func (n *Node) batchUpdate(future *structs.BatchFuture, updates []*structs.Alloc } } - // Revoke any orphaned SI accessors + // Revoke any orphaned SI token accessors if l := len(revokeSI); l > 0 { n.logger.Debug("revoking si accessors due to terminal allocations", "num_accessors", l) - if err := n.srv.consulACLs.RevokeTokens(context.Background(), revokeSI); err != nil { - n.logger.Error("batched si accessor revocation failed", "error", err) - mErr.Errors = append(mErr.Errors, err) - } + _ = n.srv.consulACLs.RevokeTokens(context.Background(), revokeSI, true) } // Respond to the future @@ -1799,12 +1790,11 @@ func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs. accessors = append(accessors, accessor) } - // If there was an error, revoke all created tokens. + // If there was an error, revoke all created tokens. These tokens have not + // yet been committed to the persistent store. if createErr != nil { n.logger.Error("Consul Service Identity token creation for alloc failed", "alloc_id", alloc.ID, "error", createErr) - if revokeErr := n.srv.consulACLs.RevokeTokens(context.Background(), accessors); revokeErr != nil { - n.logger.Error("Consul Service Identity token revocation for alloc failed", "alloc_id", alloc.ID, "error", revokeErr) - } + _ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, false) if recoverable, ok := createErr.(*structs.RecoverableError); ok { reply.Error = recoverable diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 7b77ff50dd9..5ee1e3a44b1 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -3220,8 +3220,7 @@ func TestClientEndpoint_DeriveSIToken_ConsulError(t *testing.T) { // rejigger the server to use a broken mock consul mockACLsAPI := consul.NewMockACLsAPI(s1.logger) mockACLsAPI.SetError(structs.NewRecoverableError(errors.New("consul recoverable error"), true)) - m, err := NewConsulACLsAPI(mockACLsAPI, s1.logger) - r.NoError(err) + m := NewConsulACLsAPI(mockACLsAPI, s1.logger, nil) s1.consulACLs = m err = state.UpsertAllocs(3, []*structs.Allocation{alloc}) diff --git a/nomad/server.go b/nomad/server.go index 2942575f340..6af90d623b8 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -344,11 +344,7 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI, consulACLs consu s.statsFetcher = NewStatsFetcher(s.logger, s.connPool, s.config.Region) // Setup Consul (more) - if err := s.setupConsul(consulACLs); err != nil { - s.Shutdown() - s.logger.Error("failed to setup Consul ACL client", "error", err) - return nil, fmt.Errorf("Failed to setup Consul ACL client: %v", err) - } + s.setupConsul(consulACLs) // Setup Vault if err := s.setupVaultClient(); err != nil { @@ -589,11 +585,14 @@ func (s *Server) Shutdown() error { s.fsm.Close() } - // Stop Vault token renewal + // Stop Vault token renewal and revocations if s.vault != nil { s.vault.Stop() } + // Stop the Consul ACLs token revocations + s.consulACLs.Stop() + return nil } @@ -962,13 +961,8 @@ func (s *Server) setupNodeDrainer() { } // setupConsul is used to setup Server specific consul components. -func (s *Server) setupConsul(consulACLs consul.ACLsAPI) error { - c, err := NewConsulACLsAPI(consulACLs, s.logger) - if err != nil { - return err - } - s.consulACLs = c - return nil +func (s *Server) setupConsul(consulACLs consul.ACLsAPI) { + s.consulACLs = NewConsulACLsAPI(consulACLs, s.logger, s.purgeSITokenAccessors) } // setupVaultClient is used to set up the Vault API client. diff --git a/nomad/vault.go b/nomad/vault.go index 7126b933820..4a36471810a 100644 --- a/nomad/vault.go +++ b/nomad/vault.go @@ -148,7 +148,7 @@ type VaultStats struct { TokenExpiry time.Time } -// PurgeVaultAccessor is called to remove VaultAccessors from the system. If +// PurgeVaultAccessorFn is called to remove VaultAccessors from the system. If // the function returns an error, the token will still be tracked and revocation // will retry till there is a success type PurgeVaultAccessorFn func(accessors []*structs.VaultAccessor) error @@ -1165,7 +1165,7 @@ func (v *vaultClient) parallelRevoke(ctx context.Context, accessors []*structs.V handlers = maxParallelRevokes } - // Create the Vault Tokens + // Revoke the Vault Token Accessors input := make(chan *structs.VaultAccessor, handlers) for i := 0; i < handlers; i++ { g.Go(func() error { @@ -1226,7 +1226,7 @@ func (v *vaultClient) revokeDaemon() { continue } - // Build the list of allocations that need to revoked while pruning any TTL'd checks + // Build the list of accessors that need to be revoked while pruning any TTL'd checks revoking := make([]*structs.VaultAccessor, 0, len(v.revoking)) for va, ttl := range v.revoking { if now.After(ttl) { From 369520ec8c59ab367d2e0e62df91716dde53cfcf Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Tue, 7 Jan 2020 08:14:24 -0600 Subject: [PATCH 06/27] agent: re-enable the server in dev mode --- command/agent/command.go | 8 +++----- command/agent/config.go | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/command/agent/command.go b/command/agent/command.go index 32c46862f3b..1fd42e6b012 100644 --- a/command/agent/command.go +++ b/command/agent/command.go @@ -252,11 +252,9 @@ func (c *Command) readConfig() *Config { config.Client = &ClientConfig{} } - // todo(shoenig): disable the server in dev mode just for my own demo stuff - // this should not be merged! - //if config.Server == nil { - // config.Server = &ServerConfig{} - //} + if config.Server == nil { + config.Server = &ServerConfig{} + } // Merge any CLI options over config file options config = config.Merge(cmdConfig) diff --git a/command/agent/config.go b/command/agent/config.go index b195fc83ac9..39259abb2c4 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -769,7 +769,7 @@ func DevConfig(mode *devModeConfig) *Config { conf.BindAddr = mode.bindAddr conf.LogLevel = "DEBUG" conf.Client.Enabled = true - conf.Server.Enabled = false + conf.Server.Enabled = true conf.DevMode = mode != nil conf.EnableDebug = true conf.DisableAnonymousSignature = true From a26477900a19f53e6eda34585bf04f87e155a82a Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Tue, 7 Jan 2020 08:48:11 -0600 Subject: [PATCH 07/27] client: remove unused indirection for referencing consul executable Was thinking about using the testing pattern where you create executable shell scripts as test resources which "mock" the process a bit of code is meant to fork+exec. Turns out that wasn't really necessary in this case. --- client/allocrunner/taskrunner/envoybootstrap_hook.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook.go b/client/allocrunner/taskrunner/envoybootstrap_hook.go index 5de2551a431..29e53fb49a4 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook.go @@ -43,9 +43,6 @@ type envoyBootstrapHook struct { // Consul's gRPC endpoint. consulHTTPAddr string - // executable is executable file that is consul - executable string - // logger is used to log things logger hclog.Logger } From 869942c6bca1576aea96ca4271d1e509581f881f Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Tue, 7 Jan 2020 09:11:33 -0600 Subject: [PATCH 08/27] client: skip task SI token file load failure if testing as root The TestEnvoyBootstrapHook_maybeLoadSIToken test case only works when running as a non-priveleged user, since it deliberately tries to read an un-readable file to simulate a failure loading the SI token file. --- client/allocrunner/taskrunner/envoybootstrap_hook_test.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go index 222fecd53b0..1c1ca38f7c8 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go @@ -21,6 +21,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" + "golang.org/x/sys/unix" ) var _ interfaces.TaskPrestartHook = (*envoyBootstrapHook)(nil) @@ -39,6 +40,10 @@ func writeTmp(t *testing.T, s string, fm os.FileMode) string { func TestEnvoyBootstrapHook_maybeLoadSIToken(t *testing.T) { t.Parallel() + if unix.Geteuid() == 0 { + t.Skip("test only works as non-root") + } + t.Run("file does not exist", func(t *testing.T) { h := newEnvoyBootstrapHook(&envoyBootstrapHookConfig{logger: testlog.HCLogger(t)}) config, err := h.maybeLoadSIToken("task1", "/does/not/exist") From 53e3d8e5a7563622b87c56963f9b9cf14e7ac689 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Tue, 7 Jan 2020 11:58:29 -0600 Subject: [PATCH 09/27] comments: cleanup some leftover debug comments and such --- client/allocrunner/taskrunner/envoybootstrap_hook.go | 1 - client/allocrunner/taskrunner/task_runner_test.go | 2 +- client/allocrunner/taskrunner/tasklet.go | 2 +- client/allocrunner/taskrunner/volume_hook.go | 2 +- client/consul/identities_test.go | 4 ++-- command/agent/consul/acl_testing.go | 7 +++---- nomad/consul.go | 4 ++-- nomad/consul_policy.go | 10 ++++++++-- nomad/node_endpoint.go | 7 +++++-- nomad/structs/config/consul.go | 2 -- 10 files changed, 23 insertions(+), 18 deletions(-) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook.go b/client/allocrunner/taskrunner/envoybootstrap_hook.go index 29e53fb49a4..93e725be0ff 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook.go @@ -121,7 +121,6 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP siToken: siToken, }.args() - // put old stuff in here // Since Consul services are registered asynchronously with this task // hook running, retry a small number of times with backoff. for tries := 3; ; tries-- { diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 44530a4f51a..b361723026d 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1830,7 +1830,7 @@ func TestTaskRunner_RestartSignalTask_NotRunning(t *testing.T) { require.Fail(t, "timed out waiting for task to complete") } - // Assert the task unblocked and never restarted + // Assert the task ran and never restarted state := tr.TaskState() require.Equal(t, structs.TaskStateDead, state.State) require.False(t, state.Failed) diff --git a/client/allocrunner/taskrunner/tasklet.go b/client/allocrunner/taskrunner/tasklet.go index 63bf56c093a..0f6d2e578c2 100644 --- a/client/allocrunner/taskrunner/tasklet.go +++ b/client/allocrunner/taskrunner/tasklet.go @@ -148,7 +148,7 @@ func (t *tasklet) run() *taskletHandle { select { case <-t.shutdownCh: - // We've been told to exit and just unblocked so exit + // We've been told to exit and just ran so exit return default: } diff --git a/client/allocrunner/taskrunner/volume_hook.go b/client/allocrunner/taskrunner/volume_hook.go index fd9a5a1d191..1e0935aeab6 100644 --- a/client/allocrunner/taskrunner/volume_hook.go +++ b/client/allocrunner/taskrunner/volume_hook.go @@ -96,7 +96,7 @@ func (h *volumeHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartR return err } - // Because this hook is also unblocked on restores, we only add mounts that do not + // Because this hook is also ran on restores, we only add mounts that do not // already exist. Although this loop is somewhat expensive, there are only // a small number of mounts that exist within most individual tasks. We may // want to revisit this using a `hookdata` param to be "mount only once" diff --git a/client/consul/identities_test.go b/client/consul/identities_test.go index e56000d4a71..0ac7ac275e2 100644 --- a/client/consul/identities_test.go +++ b/client/consul/identities_test.go @@ -9,7 +9,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestCSI_DeriveTokens(t *testing.T) { +func TestSI_DeriveTokens(t *testing.T) { logger := testlog.HCLogger(t) dFunc := func(alloc *structs.Allocation, taskNames []string) (map[string]string, error) { return map[string]string{"a": "b"}, nil @@ -20,7 +20,7 @@ func TestCSI_DeriveTokens(t *testing.T) { require.Equal(t, map[string]string{"a": "b"}, tokens) } -func TestCSI_DeriveTokens_error(t *testing.T) { +func TestSI_DeriveTokens_error(t *testing.T) { logger := testlog.HCLogger(t) dFunc := func(alloc *structs.Allocation, taskNames []string) (map[string]string, error) { return nil, errors.New("some failure") diff --git a/command/agent/consul/acl_testing.go b/command/agent/consul/acl_testing.go index 00979543f17..0d4c2af074a 100644 --- a/command/agent/consul/acl_testing.go +++ b/command/agent/consul/acl_testing.go @@ -88,7 +88,7 @@ func (m *MockACLsAPI) RoleRead(roleID string, _ *api.QueryOptions) (*api.ACLRole ID: ExamplePolicyID1, Name: "example-policy-1", }}, - ServiceIdentities: nil, // would it ever make sense ? + ServiceIdentities: nil, }, nil, nil case ExampleRoleID2: return &api.ACLRole{ @@ -104,9 +104,8 @@ func (m *MockACLsAPI) RoleRead(roleID string, _ *api.QueryOptions) (*api.ACLRole return &api.ACLRole{ ID: ExampleRoleID3, Name: "example-role-3", - Policies: nil, // todo - ServiceIdentities: nil, // todo - ModifyIndex: 0, + Policies: nil, // todo add more if needed + ServiceIdentities: nil, // todo add more if needed }, nil, nil default: return nil, nil, nil diff --git a/nomad/consul.go b/nomad/consul.go index 8c7846f2484..9d1f862891f 100644 --- a/nomad/consul.go +++ b/nomad/consul.go @@ -378,14 +378,14 @@ func (c *consulACLsAPI) bgRetryRevoke() { copy(toPurge, c.bgRetryRevocation) if err := c.parallelRevoke(context.Background(), toPurge); err != nil { - c.logger.Warn("background token revocation failed", "error", err) + c.logger.Warn("background SI token revocation failed", "error", err) return } // Call the revocation function if err := c.purgeFunc(toPurge); err != nil { // Just try again later (revocation is idempotent) - c.logger.Error("token revocation failed", "error", err) + c.logger.Error("background SI token purge failed", "error", err) return } diff --git a/nomad/consul_policy.go b/nomad/consul_policy.go index 4f8a62379f0..51cb2bc323c 100644 --- a/nomad/consul_policy.go +++ b/nomad/consul_policy.go @@ -8,17 +8,21 @@ import ( "github.com/pkg/errors" ) -// ConsulServiceRule represents a policy for a service +// ConsulServiceRule represents a policy for a service. type ConsulServiceRule struct { Name string `hcl:",key"` Policy string } +// ConsulPolicy represents the parts of a ConsulServiceRule Policy that are +// relevant to Service Identity authorizations. type ConsulPolicy struct { Services []*ConsulServiceRule `hcl:"service,expand"` ServicePrefixes []*ConsulServiceRule `hcl:"service_prefix,expand"` } +// IsEmpty returns true if there are no Services or ServicePrefixes defined for +// the ConsulPolicy. func (cp *ConsulPolicy) IsEmpty() bool { if cp == nil { return true @@ -26,6 +30,9 @@ func (cp *ConsulPolicy) IsEmpty() bool { return len(cp.Services) == 0 && len(cp.ServicePrefixes) == 0 } +// ParseConsulPolicy parses raw string s into a ConsulPolicy. An error is +// returned if decoding the policy fails, or if the decoded policy has no +// Services or ServicePrefixes defined. func ParseConsulPolicy(s string) (*ConsulPolicy, error) { cp := new(ConsulPolicy) if err := hcl.Decode(cp, s); err != nil { @@ -71,7 +78,6 @@ func (c *consulACLsAPI) hasSufficientPolicy(task string, token *api.ACLToken) (b return false, nil } -// policyAllowsServiceWrite func (c *consulACLsAPI) policyAllowsServiceWrite(task string, policyID string) (bool, error) { policy, _, err := c.aclClient.PolicyRead(policyID, &api.QueryOptions{ AllowStale: false, diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index d9ee28ef6ef..671e5778cee 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -350,6 +350,9 @@ func (n *Node) deregister(args *structs.NodeBatchDeregisterRequest, return err } else if l := len(accessors); l > 0 { n.logger.Debug("revoking si accessors on node due to deregister", "num_accessors", l, "node_id", nodeID) + // Unlike with the Vault integration, there's no error returned here, since + // bootstrapping the Consul client is elsewhere. Errors in revocation trigger + // background retry attempts rather than inline error handling. _ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, true) } @@ -465,10 +468,10 @@ func (n *Node) UpdateStatus(args *structs.NodeUpdateStatusRequest, reply *struct // Determine if there are any SI token accessors on the node to cleanup if accessors, err := n.srv.State().SITokenAccessorsByNode(ws, args.NodeID); err != nil { - n.logger.Error("looking up si accessors for node failed", "node_id", args.NodeID, "error", err) + n.logger.Error("looking up SI accessors for node failed", "node_id", args.NodeID, "error", err) return err } else if l := len(accessors); l > 0 { - n.logger.Debug("revoking si accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID) + n.logger.Debug("revoking SI accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID) _ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, true) } default: diff --git a/nomad/structs/config/consul.go b/nomad/structs/config/consul.go index 9c324dde86b..a0326ae132b 100644 --- a/nomad/structs/config/consul.go +++ b/nomad/structs/config/consul.go @@ -146,8 +146,6 @@ func DefaultConsulConfig() *ConsulConfig { // // If allow_unauthenticated is false, the operator must provide a token on // job submission (i.e. -consul-token or $CONSUL_TOKEN). -// -// todo: seems like we should be using this somewhere... func (c *ConsulConfig) AllowsUnauthenticated() bool { return c.AllowUnauthenticated != nil && *c.AllowUnauthenticated } From 7416a4fe3e5888d0783386b5dc0d47b6fbc76bdc Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 15 Jan 2020 08:41:23 -0600 Subject: [PATCH 10/27] nomad,client: apply smaller PR suggestions Apply smaller suggestions like doc strings, variable names, etc. Co-Authored-By: Nick Ethier Co-Authored-By: Michael Schurter --- client/allocrunner/taskrunner/envoybootstrap_hook.go | 8 ++++---- client/allocrunner/taskrunner/envoybootstrap_hook_test.go | 2 ++ client/allocrunner/taskrunner/sids_hook.go | 2 +- nomad/consul_policy.go | 2 +- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook.go b/client/allocrunner/taskrunner/envoybootstrap_hook.go index 93e725be0ff..7382c24b1ca 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook.go @@ -141,7 +141,7 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP err = cmd.Run() // Close bootstrap.json - _ = fd.Close() + fd.Close() if err == nil { // Happy path! Bootstrap was created, exit. @@ -157,7 +157,7 @@ func (h *envoyBootstrapHook) Prestart(ctx context.Context, req *interfaces.TaskP // occurs, and (b) the file will either be rewritten on // retry or eventually garbage collected if the task // fails. - _ = os.Remove(bootstrapFilePath) + os.Remove(bootstrapFilePath) // ExitErrors are recoverable since they indicate the // command was runnable but exited with a unsuccessful @@ -249,12 +249,12 @@ func (e envoyBootstrapArgs) args() []string { "-sidecar-for", e.sidecarFor, } if e.siToken != "" { - arguments = append(arguments, []string{"-token", e.siToken}...) + arguments = append(arguments, "-token", e.siToken) } return arguments } -// maybeLoadSIToken reads the SI token saved to disk in the secretes directory +// maybeLoadSIToken reads the SI token saved to disk in the secrets directory // by the service identities prestart hook. This envoy bootstrap hook blocks // until the sids hook completes, so if the SI token is required to exist (i.e. // Consul ACLs are enabled), it will be in place by the time we try to read it. diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go index 1c1ca38f7c8..f4be8d80a76 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go @@ -130,6 +130,8 @@ type envoyConfig struct { } `json:"dynamic_resources"` } +// TestEnvoyBootstrapHook_with_SI_token asserts the bootstrap file written for +// Envoy contains a Consul SI token. func TestEnvoyBootstrapHook_with_SI_token(t *testing.T) { t.Parallel() testutil.RequireConsul(t) diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index e2aaa7fca95..1db3713c055 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -18,7 +18,7 @@ import ( const ( // the name of this hook, used in logs - sidsHookName = "consul_sids" + sidsHookName = "consul_si_token" // sidsBackoffBaseline is the baseline time for exponential backoff when // attempting to retrieve a Consul SI token diff --git a/nomad/consul_policy.go b/nomad/consul_policy.go index 51cb2bc323c..c860f0871d1 100644 --- a/nomad/consul_policy.go +++ b/nomad/consul_policy.go @@ -105,7 +105,7 @@ const ( serviceNameWildcard = "*" ) -func (_ *consulACLsAPI) allowsServiceWrite(task string, cp *ConsulPolicy) bool { +func (cp *ConsulPolicy) allowsServiceWrite(task string) bool { for _, service := range cp.Services { name := strings.ToLower(service.Name) policy := strings.ToLower(service.Policy) From 00bfeddc5e247f44eae284f6642b227f9712dee3 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 15 Jan 2020 09:29:47 -0600 Subject: [PATCH 11/27] nomad,client: apply more comment/style PR tweaks --- .../allocrunner/taskrunner/envoybootstrap_hook.go | 5 +++++ .../taskrunner/envoybootstrap_hook_test.go | 7 +++++++ client/allocrunner/taskrunner/sids_hook_test.go | 3 ++- client/allocrunner/taskrunner/task_runner_hooks.go | 13 ++++++------- client/allocrunner/taskrunner/task_runner_test.go | 5 +++-- nomad/job_endpoint.go | 11 ++++------- nomad/leader.go | 2 +- 7 files changed, 28 insertions(+), 18 deletions(-) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook.go b/client/allocrunner/taskrunner/envoybootstrap_hook.go index 7382c24b1ca..d945a2912b8 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook.go @@ -230,6 +230,9 @@ func (h *envoyBootstrapHook) execute(cmd *exec.Cmd) (string, error) { return stdout.String(), nil } +// envoyBootstrapArgs is used to accumulate CLI arguments that will be passed +// along to the exec invocation of consul which will then generate the bootstrap +// configuration file for envoy. type envoyBootstrapArgs struct { sidecarFor string grpcAddr string @@ -238,6 +241,8 @@ type envoyBootstrapArgs struct { siToken string } +// args returns the CLI arguments consul needs in the correct order, with the +// -token argument present or not present depending on whether it is set. func (e envoyBootstrapArgs) args() []string { arguments := []string{ "connect", diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go index f4be8d80a76..276a9bd1b40 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook_test.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook_test.go @@ -1,3 +1,7 @@ +// +build !windows +// todo(shoenig): Once Connect is supported on Windows, we'll need to make this +// set of tests work there too. + package taskrunner import ( @@ -40,6 +44,9 @@ func writeTmp(t *testing.T, s string, fm os.FileMode) string { func TestEnvoyBootstrapHook_maybeLoadSIToken(t *testing.T) { t.Parallel() + // This test fails when running as root because the test case for checking + // the error condition when the file is unreadable fails (root can read the + // file even though the permissions are set to 0200). if unix.Geteuid() == 0 { t.Skip("test only works as non-root") } diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go index 7a2035d30ab..415d1f98007 100644 --- a/client/allocrunner/taskrunner/sids_hook_test.go +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -11,6 +11,7 @@ import ( "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" ) @@ -51,7 +52,7 @@ func TestSIDSHook_recoverToken(t *testing.T) { logger: testlog.HCLogger(t), }) - expected := "12345678-1234-1234-1234-1234567890" + expected := uuid.Generate() err := h.writeToken(secrets, expected) r.NoError(err) diff --git a/client/allocrunner/taskrunner/task_runner_hooks.go b/client/allocrunner/taskrunner/task_runner_hooks.go index a817eb0bd89..2da0c9fe528 100644 --- a/client/allocrunner/taskrunner/task_runner_hooks.go +++ b/client/allocrunner/taskrunner/task_runner_hooks.go @@ -116,15 +116,14 @@ func (tr *TaskRunner) initHooks() { lifecycle: tr, logger: hookLogger, })) + // envoy bootstrap must execute after sidsHook maybe sets SI token + tr.runnerHooks = append(tr.runnerHooks, newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ + alloc: alloc, + consulHTTPAddr: tr.clientConfig.ConsulConfig.Addr, + logger: hookLogger, + })) } - // envoy bootstrap must execute after sidsHook maybe sets SI token - tr.runnerHooks = append(tr.runnerHooks, newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ - alloc: alloc, - consulHTTPAddr: tr.clientConfig.ConsulConfig.Addr, - logger: hookLogger, - })) - // If there are any script checks, add the hook scriptCheckHook := newScriptCheckHook(scriptCheckHookConfig{ alloc: tr.Alloc(), diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index b361723026d..7b59d626a6b 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -28,6 +28,7 @@ import ( mockdriver "github.com/hashicorp/nomad/drivers/mock" "github.com/hashicorp/nomad/drivers/rawexec" "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/plugins/device" @@ -1127,7 +1128,7 @@ func TestTaskRunner_BlockForSIDSToken(t *testing.T) { defer cleanup() // control when we get a Consul SI token - token := "12345678-1234-1234-1234-1234567890" + token := uuid.Generate() waitCh := make(chan struct{}) deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { <-waitCh @@ -1191,7 +1192,7 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { defer cleanup() // control when we get a Consul SI token - token := "12345678-1234-1234-1234-1234567890" + token := uuid.Generate() deriveCount := 0 deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { if deriveCount > 0 { diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index e7f74b27b3e..d4c77030d80 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -234,13 +234,10 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis } // Enforce that the operator has necessary Consul ACL permissions - connectTasks := args.Job.ConnectTasks() - if len(connectTasks) > 0 { - for _, tg := range connectTasks { - for _, task := range tg { - if err := checkOperatorToken(task); err != nil { - return err - } + for _, tg := range args.Job.ConnectTasks() { + for _, task := range tg { + if err := checkOperatorToken(task); err != nil { + return err } } } diff --git a/nomad/leader.go b/nomad/leader.go index a2b886d9ed4..6d5cc585620 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -419,7 +419,7 @@ func (s *Server) revokeSITokenAccessorsOnRestore() error { if len(toRevoke) > 0 { ctx := context.Background() - _ = s.consulACLs.RevokeTokens(ctx, toRevoke, true) + s.consulACLs.RevokeTokens(ctx, toRevoke, true) } return nil From 2116d35b1a85e81d0d3b52dfa6801a4437b29907 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 15 Jan 2020 09:56:48 -0600 Subject: [PATCH 12/27] client: set context timeout around SI token derivation The derivation of an SI token needs to be safegaurded by a context timeout, otherwise an unresponsive Consul could cause the siHook to block forever on Prestart. --- client/allocrunner/taskrunner/sids_hook.go | 32 ++++++++++----- .../allocrunner/taskrunner/sids_hook_test.go | 40 +++++++++++++++++-- 2 files changed, 60 insertions(+), 12 deletions(-) diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index 1db3713c055..77dd46ead53 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -28,6 +28,11 @@ const ( // to retrieve a Consul SI token sidsBackoffLimit = 3 * time.Minute + // sidsDerivationTimeout limits the amount of time we may spend trying to + // derive a SI token. If the hook does not get a token within this amount of + // time, the result is a failure. + sidsDerivationTimeout = 5 * time.Minute + // sidsTokenFile is the name of the file holding the Consul SI token inside // the task's secret directory sidsTokenFile = "si_token" @@ -59,6 +64,11 @@ type sidsHook struct { // lifecycle is used to signal, restart, and kill a task lifecycle ti.TaskLifecycle + // derivationTimeout is the amount of time we may wait for Consul to successfully + // provide a SI token. Making this configurable for testing, otherwise + // default to sidsDerivationTimeout + derivationTimeout time.Duration + // logger is used to log logger hclog.Logger @@ -71,12 +81,13 @@ type sidsHook struct { func newSIDSHook(c sidsHookConfig) *sidsHook { return &sidsHook{ - alloc: c.alloc, - task: c.task, - sidsClient: c.sidsClient, - lifecycle: c.lifecycle, - logger: c.logger.Named(sidsHookName), - firstRun: true, + alloc: c.alloc, + task: c.task, + sidsClient: c.sidsClient, + lifecycle: c.lifecycle, + derivationTimeout: sidsDerivationTimeout, + logger: c.logger.Named(sidsHookName), + firstRun: true, } } @@ -163,18 +174,21 @@ func (h *sidsHook) recoverToken(dir string) (string, error) { // derive an SI token until a token is successfully created, or ctx is signaled // done. func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { + ctx2, cancel := context.WithTimeout(ctx, h.derivationTimeout) + defer cancel() + tokenCh := make(chan string) // keep trying to get the token in the background - go h.tryDerive(ctx, tokenCh) + go h.tryDerive(ctx2, tokenCh) // wait until we get a token, or we get a signal to quit for { select { case token := <-tokenCh: return token, nil - case <-ctx.Done(): - return "", ctx.Err() + case <-ctx2.Done(): + return "", ctx2.Err() } } } diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go index 415d1f98007..5052fe9e254 100644 --- a/client/allocrunner/taskrunner/sids_hook_test.go +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -37,8 +37,8 @@ func sidecar(task string) (string, structs.TaskKind) { func TestSIDSHook_recoverToken(t *testing.T) { t.Parallel() - r := require.New(t) + secrets := tmpDir(t) defer cleanupDir(t, secrets) @@ -63,8 +63,8 @@ func TestSIDSHook_recoverToken(t *testing.T) { func TestSIDSHook_recoverToken_empty(t *testing.T) { t.Parallel() - r := require.New(t) + secrets := tmpDir(t) defer cleanupDir(t, secrets) @@ -85,8 +85,8 @@ func TestSIDSHook_recoverToken_empty(t *testing.T) { func TestSIDSHook_deriveSIToken(t *testing.T) { t.Parallel() - r := require.New(t) + secrets := tmpDir(t) defer cleanupDir(t, secrets) @@ -108,6 +108,40 @@ func TestSIDSHook_deriveSIToken(t *testing.T) { r.True(helper.IsUUID(token), "token: %q", token) } +func TestSIDSHook_deriveSIToken_timeout(t *testing.T) { + t.Parallel() + r := require.New(t) + + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + + taskName, taskKind := sidecar("task1") + + siClient := consul.NewMockServiceIdentitiesClient() + siClient.DeriveTokenFn = func(allocation *structs.Allocation, strings []string) (m map[string]string, err error) { + select { + // block forever, hopefully triggering a timeout in the caller + } + } + + h := newSIDSHook(sidsHookConfig{ + alloc: &structs.Allocation{ID: "a1"}, + task: &structs.Task{ + Name: taskName, + Kind: taskKind, + }, + logger: testlog.HCLogger(t), + sidsClient: siClient, + }) + + // set the timeout to a really small value for testing + h.derivationTimeout = time.Duration(1 * time.Millisecond) + + ctx := context.Background() + _, err := h.deriveSIToken(ctx) + r.EqualError(err, "context deadline exceeded") +} + func TestSIDSHook_computeBackoff(t *testing.T) { t.Parallel() From 28e8963c09a4531b2ca942f1fa2e6101d744b33c Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 15 Jan 2020 10:41:52 -0600 Subject: [PATCH 13/27] client: manage TR kill from parent on SI token derivation failure Re-orient the management of the tr.kill to happen in the parent of the spawned goroutine that is doing the actual token derivation. This makes the code a little more straightforward, making it easier to reason about not leaking the worker goroutine. --- client/allocrunner/taskrunner/sids_hook.go | 42 +++++++++++++------ .../taskrunner/task_runner_test.go | 2 +- 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index 77dd46ead53..9b074a226a9 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -170,6 +170,13 @@ func (h *sidsHook) recoverToken(dir string) (string, error) { return string(token), nil } +// siDerivationResult is used to pass along the result of attempting to derive +// an SI token between the goroutine doing the derivation and its caller +type siDerivationResult struct { + token string + err error +} + // deriveSIToken spawns and waits on a goroutine which will make attempts to // derive an SI token until a token is successfully created, or ctx is signaled // done. @@ -177,16 +184,20 @@ func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { ctx2, cancel := context.WithTimeout(ctx, h.derivationTimeout) defer cancel() - tokenCh := make(chan string) + resultCh := make(chan siDerivationResult) // keep trying to get the token in the background - go h.tryDerive(ctx2, tokenCh) + go h.tryDerive(ctx2, resultCh) // wait until we get a token, or we get a signal to quit for { select { - case token := <-tokenCh: - return token, nil + case result := <-resultCh: + if result.err != nil { + h.kill(ctx, errors.Wrap(result.err, "consul: failed to derive SI token")) + return "", result.err + } + return result.token, nil case <-ctx2.Done(): return "", ctx2.Err() } @@ -203,31 +214,36 @@ func (h *sidsHook) kill(ctx context.Context, err error) { } // tryDerive loops forever until a token is created, or ctx is done. -func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- string) { +func (h *sidsHook) tryDerive(ctx context.Context, ch chan<- siDerivationResult) { for attempt := 0; backoff(ctx, attempt); attempt++ { tokens, err := h.sidsClient.DeriveSITokens(h.alloc, []string{h.task.Name}) switch { - case err == nil: - // nothing broke and we can return the token for the task - ch <- tokens[h.task.Name] + token, exists := tokens[h.task.Name] + if !exists { + err := errors.New("response does not include token for task") + h.logger.Error("derive SI token is missing token for task", "error", err, "task", h.task.Name) + ch <- siDerivationResult{token: "", err: err} + return + } + ch <- siDerivationResult{token: token, err: nil} return - case structs.IsServerSide(err): // the error is known to be a server problem, just die h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "server_side", true) - h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) - + ch <- siDerivationResult{token: "", err: err} + return case !structs.IsRecoverable(err): // the error is known not to be recoverable, just die h.logger.Error("failed to derive SI token", "error", err, "task", h.task.Name, "recoverable", false) - h.kill(ctx, errors.Wrap(err, "consul: failed to derive SI token")) + ch <- siDerivationResult{token: "", err: err} + return default: // the error is marked recoverable, retry after some backoff - h.logger.Error("failed to derive SI token", "error", err, "recoverable", true) + h.logger.Error("failed attempt to derive SI token", "error", err, "recoverable", true) } } } diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 7b59d626a6b..15b0aee310f 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1191,7 +1191,7 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) defer cleanup() - // control when we get a Consul SI token + // control when we get a Consul SI token (recoverable failure on first call) token := uuid.Generate() deriveCount := 0 deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { From d88f25fd4800245d25e6785561dac504a0306fbf Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Wed, 15 Jan 2020 13:08:10 -0600 Subject: [PATCH 14/27] nomad: fix leftover missed refactoring in consul policy checking --- nomad/consul_policy.go | 2 +- nomad/consul_policy_test.go | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/nomad/consul_policy.go b/nomad/consul_policy.go index c860f0871d1..3630a4df7fa 100644 --- a/nomad/consul_policy.go +++ b/nomad/consul_policy.go @@ -94,7 +94,7 @@ func (c *consulACLsAPI) policyAllowsServiceWrite(task string, policyID string) ( return false, err } - if c.allowsServiceWrite(task, cp) { + if cp.allowsServiceWrite(task) { return true, nil } diff --git a/nomad/consul_policy_test.go b/nomad/consul_policy_test.go index 35f5d407bc6..f7cde067f25 100644 --- a/nomad/consul_policy_test.go +++ b/nomad/consul_policy_test.go @@ -94,8 +94,7 @@ func TestConsulACLsAPI_allowsServiceWrite(t *testing.T) { t.Parallel() try := func(t *testing.T, task string, cp *ConsulPolicy, exp bool) { - cAPI := new(consulACLsAPI) - result := cAPI.allowsServiceWrite(task, cp) + result := cp.allowsServiceWrite(task) require.Equal(t, exp, result) } From ee47a27c561a9f7329899e8645c4bb4203ca6972 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 30 Jan 2020 10:49:07 -0600 Subject: [PATCH 15/27] nomad: make TaskGroup.UsesConnect helper a public helper --- .../taskrunner/envoybootstrap_hook.go | 9 ----- nomad/node_endpoint.go | 13 +------ nomad/node_endpoint_test.go | 38 ------------------- nomad/structs/structs.go | 11 ++++++ nomad/structs/structs_test.go | 38 +++++++++++++++++++ 5 files changed, 50 insertions(+), 59 deletions(-) diff --git a/client/allocrunner/taskrunner/envoybootstrap_hook.go b/client/allocrunner/taskrunner/envoybootstrap_hook.go index d945a2912b8..09efccb798c 100644 --- a/client/allocrunner/taskrunner/envoybootstrap_hook.go +++ b/client/allocrunner/taskrunner/envoybootstrap_hook.go @@ -202,15 +202,6 @@ func (h *envoyBootstrapHook) writeConfig(filename, config string) error { return nil } -func (_ *envoyBootstrapHook) retry(ctx context.Context) bool { - select { - case <-ctx.Done(): - return false - case <-time.After(2 * time.Second): - return true - } -} - func (h *envoyBootstrapHook) execute(cmd *exec.Cmd) (string, error) { var ( stdout bytes.Buffer diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 671e5778cee..d4753027f27 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -1702,7 +1702,7 @@ func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs. setError(errors.Errorf("Allocation %q does not contain TaskGroup %q", args.AllocID, alloc.TaskGroup), false) return nil } - if !tgUsesConnect(tg) { + if !tg.UsesConnect() { setError(errors.Errorf("TaskGroup %q does not use Connect", tg.Name), false) return nil } @@ -1831,17 +1831,6 @@ func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs. return nil } -func tgUsesConnect(tg *structs.TaskGroup) bool { - for _, service := range tg.Services { - if service.Connect != nil { - if service.Connect.Native || service.Connect.SidecarService != nil { - return true - } - } - } - return false -} - func tasksNotUsingConnect(tg *structs.TaskGroup, tasks []string) []string { var unneeded []string for _, task := range tasks { diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 5ee1e3a44b1..a3fb9d5b60d 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -3034,44 +3034,6 @@ func TestClientEndpoint_DeriveVaultToken_VaultError(t *testing.T) { } } -func TestClientEndpoint_tgUsesConnect(t *testing.T) { - t.Parallel() - - try := func(t *testing.T, tg *structs.TaskGroup, exp bool) { - result := tgUsesConnect(tg) - require.Equal(t, exp, result) - } - - t.Run("tg uses native", func(t *testing.T) { - try(t, &structs.TaskGroup{ - Services: []*structs.Service{ - {Connect: nil}, - {Connect: &structs.ConsulConnect{Native: true}}, - }, - }, true) - }) - - t.Run("tg uses sidecar", func(t *testing.T) { - try(t, &structs.TaskGroup{ - Services: []*structs.Service{{ - Connect: &structs.ConsulConnect{ - SidecarService: &structs.ConsulSidecarService{ - Port: "9090", - }, - }, - }}, - }, true) - }) - - t.Run("tg does not use connect", func(t *testing.T) { - try(t, &structs.TaskGroup{ - Services: []*structs.Service{ - {Connect: nil}, - }, - }, false) - }) -} - func TestClientEndpoint_taskUsesConnect(t *testing.T) { t.Parallel() diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 00b51b87a00..517bd25865d 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -5252,6 +5252,17 @@ func (tg *TaskGroup) LookupTask(name string) *Task { return nil } +func (tg *TaskGroup) UsesConnect() bool { + for _, service := range tg.Services { + if service.Connect != nil { + if service.Connect.Native || service.Connect.SidecarService != nil { + return true + } + } + } + return false +} + func (tg *TaskGroup) GoString() string { return fmt.Sprintf("*%#v", *tg) } diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index 10edfa5b968..2f9b52fa469 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -754,6 +754,44 @@ func TestTask_UsesConnect(t *testing.T) { // todo(shoenig): add native case } +func TestTaskGroup_UsesConnect(t *testing.T) { + t.Parallel() + + try := func(t *testing.T, tg *TaskGroup, exp bool) { + result := tg.UsesConnect() + require.Equal(t, exp, result) + } + + t.Run("tg uses native", func(t *testing.T) { + try(t, &TaskGroup{ + Services: []*Service{ + {Connect: nil}, + {Connect: &ConsulConnect{Native: true}}, + }, + }, true) + }) + + t.Run("tg uses sidecar", func(t *testing.T) { + try(t, &TaskGroup{ + Services: []*Service{{ + Connect: &ConsulConnect{ + SidecarService: &ConsulSidecarService{ + Port: "9090", + }, + }, + }}, + }, true) + }) + + t.Run("tg does not use connect", func(t *testing.T) { + try(t, &TaskGroup{ + Services: []*Service{ + {Connect: nil}, + }, + }, false) + }) +} + func TestTaskGroup_Validate(t *testing.T) { j := testJob() tg := &TaskGroup{ From 6f1503acf771cc7d48d6d2b1c0be985ea4acd607 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 30 Jan 2020 12:59:04 -0600 Subject: [PATCH 16/27] client: PR cleanup - shadow context variable --- client/allocrunner/taskrunner/sids_hook.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index 9b074a226a9..bf92d25acb0 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -181,13 +181,13 @@ type siDerivationResult struct { // derive an SI token until a token is successfully created, or ctx is signaled // done. func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { - ctx2, cancel := context.WithTimeout(ctx, h.derivationTimeout) + ctx, cancel := context.WithTimeout(ctx, h.derivationTimeout) defer cancel() resultCh := make(chan siDerivationResult) // keep trying to get the token in the background - go h.tryDerive(ctx2, resultCh) + go h.tryDerive(ctx, resultCh) // wait until we get a token, or we get a signal to quit for { @@ -198,8 +198,8 @@ func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { return "", result.err } return result.token, nil - case <-ctx2.Done(): - return "", ctx2.Err() + case <-ctx.Done(): + return "", ctx.Err() } } } From d3ba869d626960666ba9c50baad654d4e72732a5 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 30 Jan 2020 13:40:56 -0600 Subject: [PATCH 17/27] client: PR cleanup - improved logging around kill task in SIDS hook --- client/allocrunner/taskrunner/sids_hook.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/client/allocrunner/taskrunner/sids_hook.go b/client/allocrunner/taskrunner/sids_hook.go index bf92d25acb0..b22a31d050b 100644 --- a/client/allocrunner/taskrunner/sids_hook.go +++ b/client/allocrunner/taskrunner/sids_hook.go @@ -194,7 +194,8 @@ func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { select { case result := <-resultCh: if result.err != nil { - h.kill(ctx, errors.Wrap(result.err, "consul: failed to derive SI token")) + h.logger.Error("failed to derive SI token", "error", result.err) + h.kill(ctx, errors.Wrap(result.err, "failed to derive SI token")) return "", result.err } return result.token, nil @@ -204,13 +205,14 @@ func (h *sidsHook) deriveSIToken(ctx context.Context) (string, error) { } } -func (h *sidsHook) kill(ctx context.Context, err error) { - _ = h.lifecycle.Kill( - ctx, +func (h *sidsHook) kill(ctx context.Context, reason error) { + if err := h.lifecycle.Kill(ctx, structs.NewTaskEvent(structs.TaskKilling). SetFailsTask(). - SetDisplayMessage(err.Error()), - ) + SetDisplayMessage(reason.Error()), + ); err != nil { + h.logger.Error("failed to kill task", "kill_reason", reason, "error", err) + } } // tryDerive loops forever until a token is created, or ctx is done. From 50cc5653aeadcca447d15f69f2c262857de988d2 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 30 Jan 2020 15:33:20 -0600 Subject: [PATCH 18/27] client: additional test cases around failures in SIDS hook --- .../allocrunner/taskrunner/sids_hook_test.go | 79 ++++++++++++++++--- .../taskrunner/task_runner_test.go | 56 +++++++++++++ 2 files changed, 126 insertions(+), 9 deletions(-) diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go index 5052fe9e254..5499df7d503 100644 --- a/client/allocrunner/taskrunner/sids_hook_test.go +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -4,6 +4,7 @@ import ( "context" "io/ioutil" "os" + "path/filepath" "testing" "time" @@ -43,7 +44,6 @@ func TestSIDSHook_recoverToken(t *testing.T) { defer cleanupDir(t, secrets) taskName, taskKind := sidecar("foo") - h := newSIDSHook(sidsHookConfig{ task: &structs.Task{ Name: taskName, @@ -69,7 +69,6 @@ func TestSIDSHook_recoverToken_empty(t *testing.T) { defer cleanupDir(t, secrets) taskName, taskKind := sidecar("foo") - h := newSIDSHook(sidsHookConfig{ task: &structs.Task{ Name: taskName, @@ -83,15 +82,81 @@ func TestSIDSHook_recoverToken_empty(t *testing.T) { r.Empty(token) } -func TestSIDSHook_deriveSIToken(t *testing.T) { +func TestSIDSHook_recoverToken_unReadable(t *testing.T) { t.Parallel() r := require.New(t) secrets := tmpDir(t) defer cleanupDir(t, secrets) - taskName, taskKind := sidecar("task1") + err := os.Chmod(secrets, 0000) + r.NoError(err) + + taskName, taskKind := sidecar("foo") + h := newSIDSHook(sidsHookConfig{ + task: &structs.Task{ + Name: taskName, + Kind: taskKind, + }, + logger: testlog.HCLogger(t), + }) + + _, err = h.recoverToken(secrets) + r.Error(err) +} + +func TestSIDSHook_writeToken(t *testing.T) { + t.Parallel() + r := require.New(t) + + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + + id := uuid.Generate() + h := new(sidsHook) + err := h.writeToken(secrets, id) + r.NoError(err) + + content, err := ioutil.ReadFile(filepath.Join(secrets, sidsTokenFile)) + r.NoError(err) + r.Equal(id, string(content)) +} +func TestSIDSHook_writeToken_unWritable(t *testing.T) { + t.Parallel() + r := require.New(t) + + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + + err := os.Chmod(secrets, 0000) + r.NoError(err) + + id := uuid.Generate() + h := new(sidsHook) + err = h.writeToken(secrets, id) + r.Error(err) +} + +func Test_SIDSHook_writeToken_nonExistent(t *testing.T) { + t.Parallel() + r := require.New(t) + + base := tmpDir(t) + defer cleanupDir(t, base) + secrets := filepath.Join(base, "does/not/exist") + + id := uuid.Generate() + h := new(sidsHook) + err := h.writeToken(secrets, id) + r.Error(err) +} + +func TestSIDSHook_deriveSIToken(t *testing.T) { + t.Parallel() + r := require.New(t) + + taskName, taskKind := sidecar("task1") h := newSIDSHook(sidsHookConfig{ alloc: &structs.Allocation{ID: "a1"}, task: &structs.Task{ @@ -112,11 +177,6 @@ func TestSIDSHook_deriveSIToken_timeout(t *testing.T) { t.Parallel() r := require.New(t) - secrets := tmpDir(t) - defer cleanupDir(t, secrets) - - taskName, taskKind := sidecar("task1") - siClient := consul.NewMockServiceIdentitiesClient() siClient.DeriveTokenFn = func(allocation *structs.Allocation, strings []string) (m map[string]string, err error) { select { @@ -124,6 +184,7 @@ func TestSIDSHook_deriveSIToken_timeout(t *testing.T) { } } + taskName, taskKind := sidecar("task1") h := newSIDSHook(sidsHookConfig{ alloc: &structs.Allocation{ID: "a1"}, task: &structs.Task{ diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 15b0aee310f..3dbafab54d7 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1231,6 +1231,62 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { r.Equal(token, string(data)) } +func TestTaskRunner_DeriveSIToken_UnWritableTokenFile(t *testing.T) { + t.Parallel() + r := require.New(t) + + alloc := mock.BatchConnectAlloc() + task := alloc.Job.TaskGroups[0].Tasks[0] + task.Config = map[string]interface{}{ + "run_for": "0s", + } + + trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) + defer cleanup() + + // make the si_token file un-writable, triggering a failure after a + // successful token derivation + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + trConfig.TaskDir.SecretsDir = secrets + err := ioutil.WriteFile(filepath.Join(secrets, sidsTokenFile), nil, 0400) + r.NoError(err) + + // derive token works just fine + deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { + return map[string]string{task.Name: uuid.Generate()}, nil + } + siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) + siClient.DeriveTokenFn = deriveFn + + // start the task runner + tr, err := NewTaskRunner(trConfig) + r.NoError(err) + defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) + useMockEnvoyBootstrapHook(tr) // mock the envoy bootstrap + + go tr.Run() + + // wait for task runner to finish running + select { + case <-tr.WaitCh(): + case <-time.After(time.Duration(testutil.TestMultiplier()*15) * time.Second): + r.Fail("timed out waiting for task runner") + } + + // assert task exited un-successfully + finalState := tr.TaskState() + r.Equal(structs.TaskStateDead, finalState.State) + r.True(finalState.Failed) // should have failed to write SI token + r.Contains(finalState.Events[2].DisplayMessage, "failed to write SI token") + + // assert the token is *not* on disk, as secrets dir was un-writable + tokenPath := filepath.Join(trConfig.TaskDir.SecretsDir, sidsTokenFile) + token, err := ioutil.ReadFile(tokenPath) + r.NoError(err) + r.Empty(token) +} + // TestTaskRunner_DeriveSIToken_Unrecoverable asserts that an unrecoverable error // from deriving a service identity token will fail a task. func TestTaskRunner_DeriveSIToken_Unrecoverable(t *testing.T) { From 86219d17a706c0f1bde1e677bbe1cdbb084dc4e7 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Thu, 30 Jan 2020 16:16:45 -0600 Subject: [PATCH 19/27] tests: skip some SIDS hook tests if running tests as root --- .../allocrunner/taskrunner/sids_hook_test.go | 89 +++++++++++++++++++ .../taskrunner/task_runner_test.go | 56 ------------ 2 files changed, 89 insertions(+), 56 deletions(-) diff --git a/client/allocrunner/taskrunner/sids_hook_test.go b/client/allocrunner/taskrunner/sids_hook_test.go index 5499df7d503..c4911c11174 100644 --- a/client/allocrunner/taskrunner/sids_hook_test.go +++ b/client/allocrunner/taskrunner/sids_hook_test.go @@ -1,3 +1,7 @@ +// +build !windows +// todo(shoenig): Once Connect is supported on Windows, we'll need to make this +// set of tests work there too. + package taskrunner import ( @@ -10,11 +14,15 @@ import ( "github.com/hashicorp/nomad/client/allocrunner/interfaces" "github.com/hashicorp/nomad/client/consul" + consulapi "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/testlog" "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/testutil" "github.com/stretchr/testify/require" + "golang.org/x/sys/unix" ) var _ interfaces.TaskPrestartHook = (*sidsHook)(nil) @@ -83,6 +91,13 @@ func TestSIDSHook_recoverToken_empty(t *testing.T) { } func TestSIDSHook_recoverToken_unReadable(t *testing.T) { + // This test fails when running as root because the test case for checking + // the error condition when the file is unreadable fails (root can read the + // file even though the permissions are set to 0200). + if unix.Geteuid() == 0 { + t.Skip("test only works as non-root") + } + t.Parallel() r := require.New(t) @@ -123,6 +138,13 @@ func TestSIDSHook_writeToken(t *testing.T) { } func TestSIDSHook_writeToken_unWritable(t *testing.T) { + // This test fails when running as root because the test case for checking + // the error condition when the file is unreadable fails (root can read the + // file even though the permissions are set to 0200). + if unix.Geteuid() == 0 { + t.Skip("test only works as non-root") + } + t.Parallel() r := require.New(t) @@ -238,3 +260,70 @@ func TestSIDSHook_backoffKilled(t *testing.T) { stop := !backoff(ctx, 1000) r.True(stop) } + +func TestTaskRunner_DeriveSIToken_UnWritableTokenFile(t *testing.T) { + // Normally this test would live in test_runner_test.go, but since it requires + // root and the check for root doesn't like Windows, we put this file in here + // for now. + + // This test fails when running as root because the test case for checking + // the error condition when the file is unreadable fails (root can read the + // file even though the permissions are set to 0200). + if unix.Geteuid() == 0 { + t.Skip("test only works as non-root") + } + + t.Parallel() + r := require.New(t) + + alloc := mock.BatchConnectAlloc() + task := alloc.Job.TaskGroups[0].Tasks[0] + task.Config = map[string]interface{}{ + "run_for": "0s", + } + + trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) + defer cleanup() + + // make the si_token file un-writable, triggering a failure after a + // successful token derivation + secrets := tmpDir(t) + defer cleanupDir(t, secrets) + trConfig.TaskDir.SecretsDir = secrets + err := ioutil.WriteFile(filepath.Join(secrets, sidsTokenFile), nil, 0400) + r.NoError(err) + + // derive token works just fine + deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { + return map[string]string{task.Name: uuid.Generate()}, nil + } + siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) + siClient.DeriveTokenFn = deriveFn + + // start the task runner + tr, err := NewTaskRunner(trConfig) + r.NoError(err) + defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) + useMockEnvoyBootstrapHook(tr) // mock the envoy bootstrap + + go tr.Run() + + // wait for task runner to finish running + select { + case <-tr.WaitCh(): + case <-time.After(time.Duration(testutil.TestMultiplier()*15) * time.Second): + r.Fail("timed out waiting for task runner") + } + + // assert task exited un-successfully + finalState := tr.TaskState() + r.Equal(structs.TaskStateDead, finalState.State) + r.True(finalState.Failed) // should have failed to write SI token + r.Contains(finalState.Events[2].DisplayMessage, "failed to write SI token") + + // assert the token is *not* on disk, as secrets dir was un-writable + tokenPath := filepath.Join(trConfig.TaskDir.SecretsDir, sidsTokenFile) + token, err := ioutil.ReadFile(tokenPath) + r.NoError(err) + r.Empty(token) +} diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 3dbafab54d7..15b0aee310f 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1231,62 +1231,6 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { r.Equal(token, string(data)) } -func TestTaskRunner_DeriveSIToken_UnWritableTokenFile(t *testing.T) { - t.Parallel() - r := require.New(t) - - alloc := mock.BatchConnectAlloc() - task := alloc.Job.TaskGroups[0].Tasks[0] - task.Config = map[string]interface{}{ - "run_for": "0s", - } - - trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) - defer cleanup() - - // make the si_token file un-writable, triggering a failure after a - // successful token derivation - secrets := tmpDir(t) - defer cleanupDir(t, secrets) - trConfig.TaskDir.SecretsDir = secrets - err := ioutil.WriteFile(filepath.Join(secrets, sidsTokenFile), nil, 0400) - r.NoError(err) - - // derive token works just fine - deriveFn := func(*structs.Allocation, []string) (map[string]string, error) { - return map[string]string{task.Name: uuid.Generate()}, nil - } - siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) - siClient.DeriveTokenFn = deriveFn - - // start the task runner - tr, err := NewTaskRunner(trConfig) - r.NoError(err) - defer tr.Kill(context.Background(), structs.NewTaskEvent("cleanup")) - useMockEnvoyBootstrapHook(tr) // mock the envoy bootstrap - - go tr.Run() - - // wait for task runner to finish running - select { - case <-tr.WaitCh(): - case <-time.After(time.Duration(testutil.TestMultiplier()*15) * time.Second): - r.Fail("timed out waiting for task runner") - } - - // assert task exited un-successfully - finalState := tr.TaskState() - r.Equal(structs.TaskStateDead, finalState.State) - r.True(finalState.Failed) // should have failed to write SI token - r.Contains(finalState.Events[2].DisplayMessage, "failed to write SI token") - - // assert the token is *not* on disk, as secrets dir was un-writable - tokenPath := filepath.Join(trConfig.TaskDir.SecretsDir, sidsTokenFile) - token, err := ioutil.ReadFile(tokenPath) - r.NoError(err) - r.Empty(token) -} - // TestTaskRunner_DeriveSIToken_Unrecoverable asserts that an unrecoverable error // from deriving a service identity token will fail a task. func TestTaskRunner_DeriveSIToken_Unrecoverable(t *testing.T) { From 5b2bb63fec038f9d2c73fbd849a7b1a952fc7662 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Mon, 13 Jan 2020 10:13:07 -0600 Subject: [PATCH 20/27] e2e: e2e test for connect with consul acls Provide script for managing Consul ACLs on a TF provisioned cluster for e2e testing. Script can be used to 'enable' or 'disable' Consul ACLs, and automatically takes care of the bootstrapping process if necessary. The bootstrapping process takes a long time, so we may need to extend the overall e2e timeout (20 minutes seems fine). Introduces basic tests for Consul Connect with ACLs. --- e2e/connect/acls.go | 410 +++++++++++++++++++++ e2e/connect/connect.go | 11 + e2e/consulacls/README.md | 45 +++ e2e/consulacls/acl-disable.hcl | 6 + e2e/consulacls/acl-enable.hcl | 8 + e2e/consulacls/consul-acls-manage.sh | 375 +++++++++++++++++++ e2e/consulacls/consul-client-policy.hcl | 25 ++ e2e/consulacls/consul-server-policy.hcl | 26 ++ e2e/consulacls/manage.go | 125 +++++++ e2e/consulacls/nomad-client-consul.hcl | 4 + e2e/consulacls/nomad-client-policy.hcl | 6 + e2e/consulacls/nomad-server-consul.hcl | 8 + e2e/consulacls/nomad-server-policy.hcl | 6 + e2e/framework/framework.go | 1 - e2e/framework/provisioning/runner_linux.go | 61 ++- 15 files changed, 1111 insertions(+), 6 deletions(-) create mode 100644 e2e/connect/acls.go create mode 100644 e2e/consulacls/README.md create mode 100644 e2e/consulacls/acl-disable.hcl create mode 100644 e2e/consulacls/acl-enable.hcl create mode 100755 e2e/consulacls/consul-acls-manage.sh create mode 100644 e2e/consulacls/consul-client-policy.hcl create mode 100644 e2e/consulacls/consul-server-policy.hcl create mode 100644 e2e/consulacls/manage.go create mode 100644 e2e/consulacls/nomad-client-consul.hcl create mode 100644 e2e/consulacls/nomad-client-policy.hcl create mode 100644 e2e/consulacls/nomad-server-consul.hcl create mode 100644 e2e/consulacls/nomad-server-policy.hcl diff --git a/e2e/connect/acls.go b/e2e/connect/acls.go new file mode 100644 index 00000000000..a283d165df3 --- /dev/null +++ b/e2e/connect/acls.go @@ -0,0 +1,410 @@ +package connect + +import ( + "os" + "strings" + "testing" + "time" + + capi "github.com/hashicorp/consul/api" + consulapi "github.com/hashicorp/consul/api" + napi "github.com/hashicorp/nomad/api" + "github.com/hashicorp/nomad/e2e/consulacls" + "github.com/hashicorp/nomad/e2e/e2eutil" + "github.com/hashicorp/nomad/e2e/framework" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/jobspec" + "github.com/kr/pretty" + "github.com/stretchr/testify/require" +) + +const ( + // envConsulToken is the consul http token environment variable + envConsulToken = "CONSUL_HTTP_TOKEN" + + // demoConnectJob is the example connect enabled job useful for testing + demoConnectJob = "connect/input/demo.nomad" +) + +type ConnectACLsE2ETest struct { + framework.TC + + // manageConsulACLs is used to 'enable' and 'disable' Consul ACLs in the + // Consul Cluster that has been setup for e2e testing. + manageConsulACLs consulacls.Manager + // consulMasterToken is set to the generated Consul ACL token after using + // the consul-acls-manage.sh script to enable ACLs. + consulMasterToken string + + // things to cleanup after each test case + jobIDs []string + consulPolicyIDs []string + consulTokenIDs []string +} + +func (tc *ConnectACLsE2ETest) BeforeAll(f *framework.F) { + // Wait for Nomad to be ready before doing anything. + e2eutil.WaitForLeader(f.T(), tc.Nomad()) + e2eutil.WaitForNodesReady(f.T(), tc.Nomad(), 2) + + // Now enable Consul ACLs, the bootstrapping process for which will be + // managed automatically if needed. + var err error + tc.manageConsulACLs, err = consulacls.New(consulacls.DefaultTFStateFile) + require.NoError(f.T(), err) + tc.enableConsulACLs(f) + + // Sanity check the consul master token exists, otherwise tests are just + // going to be a train wreck. + tokenLength := len(tc.consulMasterToken) + require.Equal(f.T(), 36, tokenLength, "consul master token wrong length") + + // Sanity check the CONSUL_HTTP_TOKEN is NOT set, because that will cause + // the agent checks to fail (which do not allow having a token set (!)). + consulTokenEnv := os.Getenv(envConsulToken) + require.Empty(f.T(), consulTokenEnv) + + // Wait for Nomad to be ready _again_, since everything was restarted during + // the bootstrap process. + e2eutil.WaitForLeader(f.T(), tc.Nomad()) + e2eutil.WaitForNodesReady(f.T(), tc.Nomad(), 2) +} + +// enableConsulACLs effectively executes `consul-acls-manage.sh enable`, which +// will activate Consul ACLs, going through the bootstrap process if necessary. +func (tc *ConnectACLsE2ETest) enableConsulACLs(f *framework.F) { + tc.consulMasterToken = tc.manageConsulACLs.Enable(f.T()) +} + +// AfterAll runs after all tests are complete. +// +// We disable ConsulACLs in here to isolate the use of Consul ACLs only to +// test suites that explicitly want to test with them enabled. +func (tc *ConnectACLsE2ETest) AfterAll(f *framework.F) { + tc.disableConsulACLs(f) +} + +// disableConsulACLs effectively executes `consul-acls-manage.sh disable`, which +// will de-activate Consul ACLs. +func (tc *ConnectACLsE2ETest) disableConsulACLs(f *framework.F) { + tc.manageConsulACLs.Disable(f.T()) +} + +// AfterEach does cleanup of Consul ACL objects that were created during each +// test case. Each test case may assume it is starting from a "fresh" state - +// as if the consul ACL bootstrap process had just taken place. +func (tc *ConnectACLsE2ETest) AfterEach(f *framework.F) { + if os.Getenv("NOMAD_TEST_SKIPCLEANUP") == "1" { + return + } + + t := f.T() + r := require.New(t) + + // cleanup jobs + for _, id := range tc.jobIDs { + t.Log("cleanup: deregister nomad job id:", id) + _, _, err := tc.Nomad().Jobs().Deregister(id, true, nil) + r.NoError(err) + } + + // cleanup consul tokens + for _, id := range tc.consulTokenIDs { + t.Log("cleanup: delete consul token id:", id) + _, err := tc.Consul().ACL().TokenDelete(id, &capi.WriteOptions{Token: tc.consulMasterToken}) + r.NoError(err) + } + + // cleanup consul policies + for _, id := range tc.consulPolicyIDs { + t.Log("cleanup: delete consul policy id:", id) + _, err := tc.Consul().ACL().PolicyDelete(id, &capi.WriteOptions{Token: tc.consulMasterToken}) + r.NoError(err) + } + + // do garbage collection + err := tc.Nomad().System().GarbageCollect() + r.NoError(err) + + tc.jobIDs = []string{} + tc.consulTokenIDs = []string{} + tc.consulPolicyIDs = []string{} +} + +type consulPolicy struct { + Name string // e.g. nomad-operator + Rules string // e.g. service "" { policy="write" } +} + +func (tc *ConnectACLsE2ETest) createConsulPolicy(p consulPolicy, f *framework.F) string { + r := require.New(f.T()) + result, _, err := tc.Consul().ACL().PolicyCreate(&capi.ACLPolicy{ + Name: p.Name, + Description: "test policy " + p.Name, + Rules: p.Rules, + }, &capi.WriteOptions{Token: tc.consulMasterToken}) + r.NoError(err, "failed to create consul policy") + tc.consulPolicyIDs = append(tc.consulPolicyIDs, result.ID) + return result.ID +} + +func (tc *ConnectACLsE2ETest) createOperatorToken(policyID string, f *framework.F) string { + r := require.New(f.T()) + token, _, err := tc.Consul().ACL().TokenCreate(&capi.ACLToken{ + Description: "operator token", + Policies: []*capi.ACLTokenPolicyLink{{ID: policyID}}, + }, &capi.WriteOptions{Token: tc.consulMasterToken}) + r.NoError(err, "failed to create operator token") + tc.consulTokenIDs = append(tc.consulTokenIDs, token.AccessorID) + return token.SecretID +} + +// TODO: This is test is broken and requires an actual fix. +// We currently do not check if the provided operator token is a master token, +// and we need to do that to be consistent with the semantics of the Consul ACL +// system. Fix will be covered in a separate issue. +// +//func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MasterToken(f *framework.F) { +// t := f.T() +// r := require.New(t) +// +// t.Log("test register Connect job w/ ACLs enabled w/ master token") +// +// jobID := "connect" + uuid.Generate()[0:8] +// tc.jobIDs = append(tc.jobIDs, jobID) +// +// jobAPI := tc.Nomad().Jobs() +// +// job, err := jobspec.ParseFile(demoConnectJob) +// r.NoError(err) +// +// // Set the job file to use the consul master token. +// // One should never do this in practice, but, it should work. +// // https://www.consul.io/docs/acl/acl-system.html#builtin-tokens +// // +// // note: We cannot just set the environment variable when using the API +// // directly - that only works when using the nomad CLI command which does +// // the step of converting the environment variable into a set option. +// job.ConsulToken = &tc.consulMasterToken +// +// resp, _, err := jobAPI.Register(job, nil) +// r.NoError(err) +// r.NotNil(resp) +// r.Zero(resp.Warnings) +//} +// +func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MissingOperatorToken(f *framework.F) { + t := f.T() + r := require.New(t) + + t.Log("test register Connect job w/ ACLs enabled w/o operator token") + + job, err := jobspec.ParseFile(demoConnectJob) + r.NoError(err) + + jobAPI := tc.Nomad().Jobs() + + // Explicitly show the ConsulToken is not set + job.ConsulToken = nil + + _, _, err = jobAPI.Register(job, nil) + r.Error(err) + + t.Log("job correctly rejected, with error:", err) +} + +func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_FakeOperatorToken(f *framework.F) { + t := f.T() + r := require.New(t) + + t.Log("test register Connect job w/ ACLs enabled w/ operator token") + + policyID := tc.createConsulPolicy(consulPolicy{ + Name: "nomad-operator-policy", + Rules: `service "count-api" { policy = "write" } service "count-dashboard" { policy = "write" }`, + }, f) + t.Log("created operator policy:", policyID) + + // generate a fake consul token token + fakeToken := uuid.Generate() + job := tc.parseJobSpecFile(t, demoConnectJob) + + jobAPI := tc.Nomad().Jobs() + + // deliberately set the fake Consul token + job.ConsulToken = &fakeToken + + // should fail, because the token is fake + _, _, err := jobAPI.Register(job, nil) + r.Error(err) + t.Log("job correctly rejected, with error:", err) +} + +func (tc *ConnectACLsE2ETest) TestConnectACLs_ConnectDemo(f *framework.F) { + t := f.T() + r := require.New(t) + + t.Log("test register Connect job w/ ACLs enabled w/ operator token") + + // === Setup ACL policy and token === + + // create a policy allowing writes of services "count-api" and "count-dashboard" + policyID := tc.createConsulPolicy(consulPolicy{ + Name: "nomad-operator-policy", + Rules: `service "count-api" { policy = "write" } service "count-dashboard" { policy = "write" }`, + }, f) + t.Log("created operator policy:", policyID) + + // create a Consul "operator token" blessed with the above policy + operatorToken := tc.createOperatorToken(policyID, f) + t.Log("created operator token:", operatorToken) + + // === Register the Nomad job === + + // parse the example connect jobspec file + jobID := "connect" + uuid.Generate()[0:8] + tc.jobIDs = append(tc.jobIDs, jobID) + job := tc.parseJobSpecFile(t, demoConnectJob) + job.ID = &jobID + jobAPI := tc.Nomad().Jobs() + + // set the valid consul operator token + job.ConsulToken = &operatorToken + + // registering the job should succeed + resp, _, err := jobAPI.Register(job, nil) + r.NoError(err) + r.NotNil(resp) + r.Empty(resp.Warnings) + t.Log("job has been registered with evalID:", resp.EvalID) + + // === Make sure the evaluation actually succeeds === +EVAL: + qOpts := &napi.QueryOptions{WaitIndex: resp.EvalCreateIndex} + evalAPI := tc.Nomad().Evaluations() + eval, qMeta, err := evalAPI.Info(resp.EvalID, qOpts) + r.NoError(err) + qOpts.WaitIndex = qMeta.LastIndex + + switch eval.Status { + case "pending": + goto EVAL + case "complete": + // ok! + case "failed", "canceled", "blocked": + r.Failf("eval %s\n%s\n", eval.Status, pretty.Sprint(eval)) + default: + r.Failf("unknown eval status: %s\n%s\n", eval.Status, pretty.Sprint(eval)) + } + + // assert there were no placement failures + r.Zero(eval.FailedTGAllocs, pretty.Sprint(eval.FailedTGAllocs)) + r.Len(eval.QueuedAllocations, 2, pretty.Sprint(eval.QueuedAllocations)) + + // === Assert allocs are running === + for i := 0; i < 20; i++ { + allocs, qMeta, err := evalAPI.Allocations(eval.ID, qOpts) + r.NoError(err) + r.Len(allocs, 2) + qOpts.WaitIndex = qMeta.LastIndex + + running := 0 + for _, alloc := range allocs { + switch alloc.ClientStatus { + case "running": + running++ + case "pending": + // keep trying + default: + r.Failf("alloc failed", "alloc: %s", pretty.Sprint(alloc)) + } + } + + if running == len(allocs) { + break + } + + time.Sleep(500 * time.Millisecond) + } + + allocs, _, err := evalAPI.Allocations(eval.ID, qOpts) + r.NoError(err) + allocIDs := make(map[string]bool, 2) + for _, a := range allocs { + if a.ClientStatus != "running" || a.DesiredStatus != "run" { + r.Failf("terminal alloc", "alloc %s (%s) terminal; client=%s desired=%s", a.TaskGroup, a.ID, a.ClientStatus, a.DesiredStatus) + } + allocIDs[a.ID] = true + } + + // === Check Consul service health === + agentAPI := tc.Consul().Agent() + + failing := map[string]*capi.AgentCheck{} + for i := 0; i < 60; i++ { + checks, err := agentAPI.Checks() + require.NoError(t, err) + + // filter out checks for other services + for cid, check := range checks { + found := false + for allocID := range allocIDs { + if strings.Contains(check.ServiceID, allocID) { + found = true + break + } + } + + if !found { + delete(checks, cid) + } + } + + // ensure checks are all passing + failing = map[string]*consulapi.AgentCheck{} + for _, check := range checks { + if check.Status != "passing" { + failing[check.CheckID] = check + break + } + } + + if len(failing) == 0 { + break + } + + t.Logf("still %d checks not passing", len(failing)) + + time.Sleep(time.Second) + } + + require.Len(t, failing, 0, pretty.Sprint(failing)) + + // === Check Consul SI tokens were generated for sidecars === + aclAPI := tc.Consul().ACL() + + entries, _, err := aclAPI.TokenList(&capi.QueryOptions{ + Token: tc.consulMasterToken, + }) + r.NoError(err) + + foundSITokenForCountDash := false + foundSITokenForCountAPI := false + for _, entry := range entries { + if strings.Contains(entry.Description, "[connect-proxy-count-dashboard]") { + foundSITokenForCountDash = true + } else if strings.Contains(entry.Description, "[connect-proxy-count-api]") { + foundSITokenForCountAPI = true + } + } + r.True(foundSITokenForCountDash, "no SI token found for count-dash") + r.True(foundSITokenForCountAPI, "no SI token found for count-api") + + t.Log("connect job with ACLs enable finished") +} + +func (tc *ConnectACLsE2ETest) parseJobSpecFile(t *testing.T, filename string) *napi.Job { + job, err := jobspec.ParseFile(filename) + require.NoError(t, err) + return job +} diff --git a/e2e/connect/connect.go b/e2e/connect/connect.go index 59f5c2b5f05..f2a522c968c 100644 --- a/e2e/connect/connect.go +++ b/e2e/connect/connect.go @@ -21,6 +21,7 @@ type ConnectE2ETest struct { } func init() { + // connect tests without Consul ACLs enabled framework.AddSuites(&framework.TestSuite{ Component: "Connect", CanRunLocal: true, @@ -30,6 +31,16 @@ func init() { new(ConnectClientStateE2ETest), }, }) + + // connect tests with Consul ACLs enabled + framework.AddSuites(&framework.TestSuite{ + Component: "ConnectACLs", + CanRunLocal: false, + Consul: true, + Cases: []framework.TestCase{ + new(ConnectACLsE2ETest), + }, + }) } func (tc *ConnectE2ETest) BeforeAll(f *framework.F) { diff --git a/e2e/consulacls/README.md b/e2e/consulacls/README.md new file mode 100644 index 00000000000..10394525a68 --- /dev/null +++ b/e2e/consulacls/README.md @@ -0,0 +1,45 @@ +# Configure Consul ACLs + +This directory contains a set of scripts for re-configuring Consul in the TF +provisioned e2e environment to enable Consul ACLs. + +## Usage + +The `consul-acls-manage.sh` script can be used to manipulate the Consul cluster +to activate or de-activate Consul ACLs. There are 3 targets into the script, only +2 of which should be used from e2e framework tests. + +### bootstrap + +The command `consul-acls-manage.sh bootstrap` should *NOT* be used from e2e +framework tests. It's merely a convenience entry-point for doing development / +debugging on the script itself. + +The bootstrap process will upload "reasonable" ACL policy files to Consul Servers, +Consul Clients, Nomad Servers, and Nomad Clients. + +The bootstrap process creates a file on local disk which contains the generated +Consul ACL master token. The file is named based on the current TF state file +serial number. `/tmp/e2e-consul-bootstrap-.token` + +### enable + +The command `consul-acls-manage.sh enable` will enable Consul ACLs, going through +the bootstrap process only if necessary. Whether the bootstrap process is necessary +depends on the existence of a token file that matches the current TF state serial +number. If no associated token file exists for the current TF state, the bootstrap +process is required. Otherwise, the bootstrap process is skipped. + +If the bootstrap process was not required (i.e. it already occurred and a +Consul master token already exists for the current TF state), the script will +activate ACLs in the Consul Server configurations and restart those agents. After +using `enable`, the `disable` command can be used to turn Consul ACLs back off, +without destroying any of the existing ACL configuration. + +### disable + +The command `consul-acls-manage.sh disable` will disable Consul ACLs. This does +not "cleanup" the policy files for Consul / Nomad agents, it merely deactivates +ACLs in the Consul Server configurations and restarts those agents. After using +`disable`, the `enable` command can be used to turn Consul ACLs back on, using +the same ACL token(s) generated before. diff --git a/e2e/consulacls/acl-disable.hcl b/e2e/consulacls/acl-disable.hcl new file mode 100644 index 00000000000..9dac26de2ac --- /dev/null +++ b/e2e/consulacls/acl-disable.hcl @@ -0,0 +1,6 @@ +# This partial consul configuration file will disable Consul ACLs. The +# consul-acls-manage.sh script uploads this file as "acl.hcl" to Consul Server +# configuration directories, and restarts those agents. +acl = { + enabled = false +} diff --git a/e2e/consulacls/acl-enable.hcl b/e2e/consulacls/acl-enable.hcl new file mode 100644 index 00000000000..ec1361ebee8 --- /dev/null +++ b/e2e/consulacls/acl-enable.hcl @@ -0,0 +1,8 @@ +# This partial consul configuration file will enable Consul ACLs. The +# consul-acls-manage.sh script uploads this file as "acl.hcl" to Consul Server +# configuration directories, and restarts those agents. +acl = { + enabled = true + default_policy = "deny" + enable_token_persistence = true +} diff --git a/e2e/consulacls/consul-acls-manage.sh b/e2e/consulacls/consul-acls-manage.sh new file mode 100755 index 00000000000..d2a24491d1e --- /dev/null +++ b/e2e/consulacls/consul-acls-manage.sh @@ -0,0 +1,375 @@ +#!/usr/bin/env bash + +# must be run from e2e directory + +set -o errexit +set -o nounset +set -o pipefail + +tfstatefile="terraform/terraform.tfstate" + +# Make sure we are running from the e2e/ directory +[ "$(basename "$(pwd)")" == "e2e" ] || (echo "must be run from nomad/e2e directory" && exit 1) + +# Make sure one argument was provided (subcommand) +[ ${#} -eq 1 ] || (echo "expect one argument (subcommand)" && exit 1) + +# Make sure terraform state file exists +[ -f "${tfstatefile}" ] || (echo "file ${tfstatefile} must exist (run terraform?)" && exit 1) + +# Load Linux Client Node IPs from terraform state file +linux_clients=$(jq -r .outputs.linux_clients.value[] <"${tfstatefile}" | xargs) + +# Load Windows Client Node IPs from terraform state file +windows_clients=$(jq -r .outputs.windows_clients.value[] <"${tfstatefile}" | xargs) + +# Combine all the clients together +clients="${linux_clients} ${windows_clients}" + +# Load Server Node IPs from terraform/terraform.tfstate +servers=$(jq -r .outputs.servers.value[] <"${tfstatefile}" | xargs) + +# Use the 0th server as the ACL bootstrap server +server0=$(echo "${servers}" | cut -d' ' -f1) + +# Find the .pem file to use +pemfile="terraform/$(jq -r '.resources[] | select(.name=="private_key_pem") | .instances[0].attributes.filename' <"terraform/terraform.tfstate")" + +# See AWS service file +consul_configs="/etc/consul.d" +nomad_configs="/etc/nomad.d" + +# Not really present in the config +user=ubuntu + +# Create a filename based on the TF state file (.serial), where we will store and/or +# lookup the consul master token. The presense of this file is what determines +# whether a full ACL bootstrap must occur, or if we only need to activate ACLs +# whenever the "enable" sub-command is chosen. +token_file="/tmp/e2e-consul-bootstrap-$(jq .serial <${tfstatefile}).token" + +# One argument - the subcommand to run which may be: bootstrap, enable, or disable +subcommand="${1}" + +echo "==== SETUP configuration =====" +echo "SETUP command is: ${subcommand}" +echo "SETUP token file: ${token_file}" +echo "SETUP servers: ${servers}" +echo "SETUP linux clients: ${linux_clients}" +echo "SETUP windows clients: ${windows_clients}" +echo "SETUP pem file: ${pemfile}" +echo "SETUP consul configs: ${consul_configs}" +echo "SETUP nomad configs: ${nomad_configs}" +echo "SETUP aws user: ${user}" +echo "SETUP bootstrap server: ${server0}" + +function doSSH() { + hostname="$1" + command="$2" + echo "-----> will ssh command '${command}' on ${hostname}" + ssh \ + -o StrictHostKeyChecking=no \ + -o UserKnownHostsFile=/dev/null \ + -i "${pemfile}" \ + "${user}@${hostname}" "${command}" +} + +function doSCP() { + original="$1" + username="$2" + hostname="$3" + destination="$4" + echo "------> will scp ${original} to ${hostname}" + scp \ + -o StrictHostKeyChecking=no \ + -o UserKnownHostsFile=/dev/null \ + -i "${pemfile}" \ + "${original}" "${username}@${hostname}:${destination}" +} + +function doBootstrap() { + echo "=== Bootstrap: Consul Configs ===" + + # Stop all Nomad agents. + stopNomad + + # Stop all Consul agents. + stopConsul + + # Run the activation step, which uploads the ACLs-enabled acl.hcl file + # to each Consul Server's configuration directory, then (re)starts each + # Consul Server agent. + doActivate + + echo "=== Bootstrap: Consul ACL Bootstrap ===" + + # Bootstrap Consul ACLs on server[0] + echo "-> bootstrap ACL using ${server0}" + consul_http_token=$(doSSH "${server0}" "/usr/local/bin/consul acl bootstrap" | grep SecretID | awk '{print $2}') + consul_http_addr="http://${server0}:8500" + export CONSUL_HTTP_TOKEN=${consul_http_token} + export CONSUL_HTTP_ADDR=${consul_http_addr} + echo " consul http: ${CONSUL_HTTP_ADDR}" + echo " consul root: ${CONSUL_HTTP_TOKEN}" + echo "${CONSUL_HTTP_TOKEN}" > "${token_file}" + + # Create Consul Server Policy & Consul Server agent tokens + echo "-> configure consul server policy" + consul acl policy create -name server-policy -rules @consulacls/consul-server-policy.hcl + + # Create & Set agent token for each Consul Server + for server in ${servers}; do + echo "---> will create agent token for server ${server}" + server_agent_token=$(consul acl token create -description "consul server agent token" -policy-name server-policy | grep SecretID | awk '{print $2}') + echo "---> setting token for server agent: ${server} -> ${server_agent_token}" + consul acl set-agent-token agent "${server_agent_token}" + echo "---> done setting agent token for server ${server}" + done + + # Wait 10s before continuing with configuring consul clients. + echo "-> sleep 10s" + sleep 10 + + # Start the Consul Clients back up so we can set their tokens now + startConsulClients + + # Create Consul Client Policy & Client agent tokens + echo "-> configure consul client policy" + consul acl policy create -name client-policy -rules @consulacls/consul-client-policy.hcl + + # Create & Set agent token for each Consul Client (including windows) + for client in ${clients}; do + echo "---> will create consul agent token for client ${client}" + client_agent_token=$(consul acl token create -description "consul client agent token" -policy-name client-policy | grep SecretID | awk '{print $2}') + echo "---> setting consul token for consul client ${client} -> ${client_agent_token}" + consul acl set-agent-token agent "${client_agent_token}" + echo "---> done setting consul agent token for client ${client}" + done + + echo "=== Bootstrap: Nomad Configs ===" + + # Create Nomad Server consul Policy and Nomad Server consul tokens + echo "-> configure nomad server policy & consul token" + consul acl policy create -name nomad-server-policy -rules @consulacls/nomad-server-policy.hcl + nomad_server_consul_token=$(consul acl token create -description "nomad server consul token" -policy-name nomad-server-policy | grep SecretID | awk '{print $2}') + nomad_server_consul_token_tmp=$(mktemp) + cp consulacls/nomad-server-consul.hcl "${nomad_server_consul_token_tmp}" + sed -i "s/CONSUL_TOKEN/${nomad_server_consul_token}/g" "${nomad_server_consul_token_tmp}" + for server in ${servers}; do + echo "---> upload nomad-server-consul.hcl to ${server}" + doSCP "${nomad_server_consul_token_tmp}" "${user}" "${server}" "/tmp/nomad-server-consul.hcl" + doSSH "${server}" "sudo mv /tmp/nomad-server-consul.hcl ${nomad_configs}/nomad-server-consul.hcl" + done + + # Create Nomad Client consul Policy and Nomad Client consul token + echo "-> configure nomad client policy & consul token" + consul acl policy create -name nomad-client-policy -rules @consulacls/nomad-client-policy.hcl + nomad_client_consul_token=$(consul acl token create -description "nomad client consul token" -policy-name nomad-client-policy | grep SecretID | awk '{print $2}') + nomad_client_consul_token_tmp=$(mktemp) + cp consulacls/nomad-client-consul.hcl "${nomad_client_consul_token_tmp}" + sed -i "s/CONSUL_TOKEN/${nomad_client_consul_token}/g" "${nomad_client_consul_token_tmp}" + for linux_client in ${linux_clients}; do + echo "---> upload nomad-client-token.hcl to ${linux_client}" + doSCP "${nomad_client_consul_token_tmp}" "${user}" "${linux_client}" "/tmp/nomad-client-consul.hcl" + doSSH "${linux_client}" "sudo mv /tmp/nomad-client-consul.hcl ${nomad_configs}/nomad-client-consul.hcl" + done + + startNomad + + export NOMAD_ADDR="http://${server0}:4646" + + echo "=== Activate: DONE ===" +} + +function doEnable() { + if [ ! -f "${token_file}" ]; then + echo "ENABLE: token file does not exist, doing a full ACL bootstrap" + doBootstrap + else + echo "ENABLE: token file already exists, will activate ACLs" + doActivate + fi + + echo "=== Enable: DONE ===" + + # show the status of all the agents + echo "---> token file is ${token_file}" + consul_http_token=$(cat "${token_file}") + export CONSUL_HTTP_TOKEN="${consul_http_token}" + echo "export CONSUL_HTTP_TOKEN=${CONSUL_HTTP_TOKEN}" + doStatus +} + +function doDisable() { + if [ ! -f "${token_file}" ]; then + echo "DISABLE: token file does not exist, did bootstrap ever happen?" + exit 1 + else + echo "DISABLE: token file exists, will deactivate ACLs" + doDeactivate + fi + + echo "=== Disable: DONE ===" + + # show the status of all the agents + unset CONSUL_HTTP_TOKEN + doStatus +} + +function doActivate() { + echo "=== Activate ===" + + stopConsul + + # Upload acl-enable.hcl to each Consul Server agent's configuration directory. + for server in ${servers}; do + echo " activate: upload acl-enable.hcl to ${server}::acl.hcl" + doSCP "consulacls/acl-enable.hcl" "${user}" "${server}" "/tmp/acl.hcl" + doSSH "${server}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" + done + + # Restart each Consul Server agent to pickup the new config. + for server in ${servers}; do + echo " activate: restart Consul Server on ${server} ..." + doSSH "${server}" "sudo systemctl start consul" + sleep 1 + done + + sleep 10 + + startConsulClients + + sleep 10 + + echo "=== Activate: DONE ===" +} + +function stopNomad { + echo "=== Stop Nomad agents ===" + # Stop every Nomad agent (clients and servers) in preperation for Consul ACL + # bootstrapping. + for server in ${servers}; do + echo " stop Nomad Server on ${server}" + doSSH "${server}" "sudo systemctl stop nomad" + sleep 1 + done + + for linux_client in ${linux_clients}; do + echo " stop Nomad Client on ${linux_client}" + doSSH "${linux_client}" "sudo systemctl stop nomad" + sleep 1 + done + + echo "... all nomad agents stopped" +} + +function startNomad { + echo "=== Start Nomad agents ===" + # Start every Nomad agent (clients and servers) after having Consul ACL + # bootstrapped and configurations set for Nomad. + for server in ${servers}; do + echo " start Nomad Server on ${server}" + doSSH "${server}" "sudo systemctl start nomad" + sleep 1 + done + + # give the servers a chance to settle + sleep 10 + + for linux_client in ${linux_clients}; do + echo " start Nomad Client on ${linux_client}" + doSSH "${linux_client}" "sudo systemctl start nomad" + sleep 3 + done + + # give the clients a long time to settle + sleep 30 + + echo "... all nomad agents started" +} + +function stopConsul { + echo "=== Stop Consul agents ===" + # Stop every Nonsul agent (clients and servers) in preperation for Consul ACL + # bootstrapping. + for server in ${servers}; do + echo " stop Consul Server on ${server}" + doSSH "${server}" "sudo systemctl stop consul" + sleep 1 + done + + for linux_client in ${linux_clients}; do + echo " stop Consul Client on ${linux_client}" + doSSH "${linux_client}" "sudo systemctl stop consul" + sleep 1 + done + + echo "... all consul agents stopped" +} + +function startConsulClients { + echo "=== Start Consul Clients ===" + # Start Consul Clients + for linux_client in ${linux_clients}; do + echo " start Consul Client on ${linux_client}" + doSSH "${linux_client}" "sudo systemctl start consul" + sleep 2 + done + + sleep 5 # let them settle + echo "... all consul clients started" +} + +function doDeactivate { + echo "=== Deactivate ===" + # Upload acl-disable.hcl to each Consul Server agent's configuration directory. + for server in ${servers}; do + echo " deactivate: upload acl-disable.hcl to ${server}::acl.hcl" + doSCP "consulacls/acl-disable.hcl" "${user}" "${server}" "/tmp/acl.hcl" + doSSH "${server}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" + done + + # Restart each Consul server agent to pickup the new config. + for server in ${servers}; do + echo " deactivate: restart Consul Server on ${server} ..." + doSSH "${server}" "sudo systemctl restart consul" + sleep 3 # let the agent settle + done + + # Wait 10s before moving on, Consul needs a second to calm down. + echo " deactivate: sleep 10s ..." + sleep 10 +} + +function doStatus { + # assumes CONSUL_HTTP_TOKEN is set (or not) + echo "consul members" + consul members + echo "" + echo "nomad server members" + nomad server members + echo "" + echo "nomad node status" + nomad node status + echo "" +} + +# It's the entrypoint to our script! +case "${subcommand}" in + bootstrap) + # The bootstrap target exists to make some local development easier. Test + # cases running from the e2e framework should always use "enable" which aims + # to be idempotent. + doBootstrap + ;; + enable) + doEnable + ;; + disable) + doDisable + ;; + *) + echo "incorrect subcommand ${subcommand}" + exit 1 + ;; +esac diff --git a/e2e/consulacls/consul-client-policy.hcl b/e2e/consulacls/consul-client-policy.hcl new file mode 100644 index 00000000000..14d3d458eaf --- /dev/null +++ b/e2e/consulacls/consul-client-policy.hcl @@ -0,0 +1,25 @@ +acl = "write" + +agent "" { + policy = "write" +} + +event "" { + policy = "write" +} + +key "" { + policy = "write" +} + +node "" { + policy = "write" +} + +query "" { + policy = "write" +} + +service "" { + policy = "write" +} diff --git a/e2e/consulacls/consul-server-policy.hcl b/e2e/consulacls/consul-server-policy.hcl new file mode 100644 index 00000000000..0e7cea6a87d --- /dev/null +++ b/e2e/consulacls/consul-server-policy.hcl @@ -0,0 +1,26 @@ +acl = "write" + +agent "" { + policy = "write" +} + +event "" { + policy = "write" +} + +key "" { + policy = "write" +} + +node "" { + policy = "write" +} + +query "" { + policy = "write" +} + +service "" { + policy = "write" +} + diff --git a/e2e/consulacls/manage.go b/e2e/consulacls/manage.go new file mode 100644 index 00000000000..fadf477bd6a --- /dev/null +++ b/e2e/consulacls/manage.go @@ -0,0 +1,125 @@ +package consulacls + +import ( + "encoding/json" + "fmt" + "io/ioutil" + "strings" + "testing" + + "github.com/hashicorp/nomad/e2e/framework/provisioning" + "github.com/pkg/errors" + "github.com/stretchr/testify/require" +) + +// DefaultTFStateFile is the location of the TF state file, as created for the +// e2e test framework. This file is used to extract the TF serial number, which +// is used to determine whether the consul bootstrap process is necessary or has +// already taken place. +const DefaultTFStateFile = "terraform/terraform.tfstate" + +// A Manager is used to manipulate whether Consul ACLs are enabled or disabled. +// Only works with TF provisioned clusters. +type Manager interface { + // Enable Consul ACLs in the Consul cluster. The Consul ACL master token + // associated with the Consul cluster is returned. + // + // A complete bootstrap process will take place if necessary. + // + // Once enabled, Consul ACLs can be disabled with Disable. + Enable(t *testing.T) string + + // Disable Consul ACLs in the Consul Cluster. + // + // Once disabled, Consul ACLs can be re-enabled with Enable. + Disable(t *testing.T) +} + +type tfManager struct { + serial int + token string +} + +func New(tfStateFile string) (*tfManager, error) { + serial, err := extractSerial(tfStateFile) + if err != nil { + return nil, err + } + return &tfManager{ + serial: serial, + }, nil +} + +func (m *tfManager) Enable(t *testing.T) string { + // Create the local script runner that will be used to run the ACL management + // script, this time with the "enable" sub-command. + var runner provisioning.LinuxRunner + err := runner.Open(t) + require.NoError(t, err) + + // Run the consul ACL bootstrap script, which will store the master token + // in the deterministic path based on the TF state serial number. If the + // bootstrap process had already taken place, ACLs will be activated but + // without going through the bootstrap process again, re-using the already + // existing Consul ACL master token. + err = runner.Run(strings.Join([]string{ + "consulacls/consul-acls-manage.sh", "enable", + }, " ")) + require.NoError(t, err) + + // Read the Consul ACL master token that was generated (or if the token + // already existed because the bootstrap process had already taken place, + // that one). + token, err := m.readToken() + require.NoError(t, err) + return token +} + +type tfState struct { + Serial int `json:"serial"` +} + +// extractSerial will parse the TF state file looking for the serial number. +func extractSerial(filename string) (int, error) { + if filename == "" { + filename = DefaultTFStateFile + } + b, err := ioutil.ReadFile(filename) + if err != nil { + return 0, errors.Wrap(err, "failed to extract TF serial") + } + var state tfState + if err := json.Unmarshal(b, &state); err != nil { + return 0, errors.Wrap(err, "failed to extract TF serial") + } + return state.Serial, nil +} + +// tokenPath returns the expected path for the Consul ACL master token generated +// by the consul-acls-manage.sh bootstrap script for the current TF serial version. +func (m *tfManager) tokenPath() string { + return fmt.Sprintf("/tmp/e2e-consul-bootstrap-%d.token", m.serial) +} + +func (m *tfManager) readToken() (string, error) { + b, err := ioutil.ReadFile(m.tokenPath()) + if err != nil { + return "", err + } + return strings.TrimSpace(string(b)), nil +} + +func (m *tfManager) Disable(t *testing.T) { + // Create the local script runner that will be used to run the ACL management + // script, this time with the "disable" sub-command. + var runner provisioning.LinuxRunner + err := runner.Open(t) + require.NoError(t, err) + + // Run the consul ACL bootstrap script, which will modify the Consul Server + // ACL policies to disable ACLs, and then restart those agents. + err = runner.Run(strings.Join([]string{ + "consulacls/consul-acls-manage.sh", "disable", + }, " ")) + require.NoError(t, err) +} diff --git a/e2e/consulacls/nomad-client-consul.hcl b/e2e/consulacls/nomad-client-consul.hcl new file mode 100644 index 00000000000..ef89f47ac36 --- /dev/null +++ b/e2e/consulacls/nomad-client-consul.hcl @@ -0,0 +1,4 @@ +// The provided consul.token value must be blessed with service=write ACLs. +consul { + token="CONSUL_TOKEN" +} diff --git a/e2e/consulacls/nomad-client-policy.hcl b/e2e/consulacls/nomad-client-policy.hcl new file mode 100644 index 00000000000..059c59a099b --- /dev/null +++ b/e2e/consulacls/nomad-client-policy.hcl @@ -0,0 +1,6 @@ +// The Nomad Client will be registering things into its buddy Consul Client. + +service "" { + policy = "write" +} + diff --git a/e2e/consulacls/nomad-server-consul.hcl b/e2e/consulacls/nomad-server-consul.hcl new file mode 100644 index 00000000000..8a7985f2c10 --- /dev/null +++ b/e2e/consulacls/nomad-server-consul.hcl @@ -0,0 +1,8 @@ +// Nomad Server needs to set allow_unauthenticated=false to enforce the use +// of a Consul Operator Token on job submission for Connect enabled jobs. +// +// The provided consul.token value must be blessed with acl=write ACLs. +consul { + allow_unauthenticated = false + token="CONSUL_TOKEN" +} diff --git a/e2e/consulacls/nomad-server-policy.hcl b/e2e/consulacls/nomad-server-policy.hcl new file mode 100644 index 00000000000..4cee1a41ddc --- /dev/null +++ b/e2e/consulacls/nomad-server-policy.hcl @@ -0,0 +1,6 @@ +// The Nomad Server requires total access to Consul ACLs, because the Server +// will be requesting new SI tokens from Consul. + +acl = "write" + + diff --git a/e2e/framework/framework.go b/e2e/framework/framework.go index 73cd8adb5b3..558488b53ca 100644 --- a/e2e/framework/framework.go +++ b/e2e/framework/framework.go @@ -183,7 +183,6 @@ func (f *Framework) Run(t *testing.T) { } }) } - } // Run starts the package scoped Framework, running each TestSuite diff --git a/e2e/framework/provisioning/runner_linux.go b/e2e/framework/provisioning/runner_linux.go index 2f087a38d7e..868021330d1 100644 --- a/e2e/framework/provisioning/runner_linux.go +++ b/e2e/framework/provisioning/runner_linux.go @@ -1,25 +1,57 @@ package provisioning import ( + "context" + "fmt" + "log" "os/exec" "strings" "testing" + "time" + + "github.com/pkg/errors" ) // LinuxRunner is a ProvisioningRunner that runs on the executing host only. // The Nomad configurations used with this runner will need to avoid port // conflicts! -type LinuxRunner struct{} +// +// Must call Open before other methods. +type LinuxRunner struct { + // populated on Open. + t *testing.T +} -func (runner *LinuxRunner) Open(_ *testing.T) error { return nil } +// Open sets up the LinuxRunner to run using t as a logging mechanism. +func (runner *LinuxRunner) Open(t *testing.T) error { + runner.t = t + return nil +} +func parseCommand(command string) (string, []string) { + fields := strings.Fields(strings.TrimSpace(command)) + if len(fields) == 1 { + return fields[0], nil + } + return fields[0], fields[1:] +} + +// Run the script (including any arguments) func (runner *LinuxRunner) Run(script string) error { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Minute) + defer cancel() + commands := strings.Split(script, "\n") for _, command := range commands { - cmd := exec.Command(strings.TrimSpace(command)) - err := cmd.Run() + executable, args := parseCommand(command) + response, err := exec.CommandContext(ctx, executable, args...).CombinedOutput() + + // Nothing fancy around separating stdin from stdout, or failed vs + // successful commands for now. + runner.LogOutput(string(response)) + if err != nil { - return err + return errors.Wrapf(err, "failed to execute command %q", command) } } return nil @@ -31,3 +63,22 @@ func (runner *LinuxRunner) Copy(local, remote string) error { } func (runner *LinuxRunner) Close() {} + +func (runner *LinuxRunner) Logf(format string, args ...interface{}) { + if runner.t == nil { + log.Fatal("no t.Testing configured for LinuxRunner") + } + if testing.Verbose() { + fmt.Printf("[local] "+format+"\n", args...) + } else { + runner.t.Logf("[local] "+format, args...) + } +} + +func (runner *LinuxRunner) LogOutput(output string) { + if testing.Verbose() { + fmt.Println("\033[32m" + output + "\033[0m") + } else { + runner.t.Log(output) + } +} From 8062cde01845634930d6e00f8733c86c1c556443 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Fri, 24 Jan 2020 09:32:23 -0600 Subject: [PATCH 21/27] e2e: remove forgotten unused field from new struct --- e2e/consulacls/manage.go | 1 - 1 file changed, 1 deletion(-) diff --git a/e2e/consulacls/manage.go b/e2e/consulacls/manage.go index fadf477bd6a..e9e21aa353e 100644 --- a/e2e/consulacls/manage.go +++ b/e2e/consulacls/manage.go @@ -37,7 +37,6 @@ type Manager interface { type tfManager struct { serial int - token string } func New(tfStateFile string) (*tfManager, error) { From 814b50e5403a4e1b6337f318c8835b958a406b1f Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Fri, 24 Jan 2020 10:19:30 -0600 Subject: [PATCH 22/27] e2e: do not use eventually when waiting for allocs This test is causing panics. Unlike the other similar tests, this one is using require.Eventually which is doing something bad, and this change replaces it with a for-loop like the other tests. Failure: === RUN TestE2E/Connect === RUN TestE2E/Connect/*connect.ConnectE2ETest === RUN TestE2E/Connect/*connect.ConnectE2ETest/TestConnectDemo === RUN TestE2E/Connect/*connect.ConnectE2ETest/TestMultiServiceConnect === RUN TestE2E/Connect/*connect.ConnectClientStateE2ETest panic: Fail in goroutine after TestE2E/Connect/*connect.ConnectE2ETest has completed goroutine 38 [running]: testing.(*common).Fail(0xc000656500) /opt/google/go/src/testing/testing.go:565 +0x11e testing.(*common).Fail(0xc000656100) /opt/google/go/src/testing/testing.go:559 +0x96 testing.(*common).FailNow(0xc000656100) /opt/google/go/src/testing/testing.go:587 +0x2b testing.(*common).Fatalf(0xc000656100, 0x1512f90, 0x10, 0xc000675f88, 0x1, 0x1) /opt/google/go/src/testing/testing.go:672 +0x91 github.com/hashicorp/nomad/e2e/connect.(*ConnectE2ETest).TestMultiServiceConnect.func1(0x0) /home/shoenig/go/src/github.com/hashicorp/nomad/e2e/connect/multi_service.go:72 +0x296 github.com/hashicorp/nomad/vendor/github.com/stretchr/testify/assert.Eventually.func1(0xc0004962a0, 0xc0002338f0) /home/shoenig/go/src/github.com/hashicorp/nomad/vendor/github.com/stretchr/testify/assert/assertions.go:1494 +0x27 created by github.com/hashicorp/nomad/vendor/github.com/stretchr/testify/assert.Eventually /home/shoenig/go/src/github.com/hashicorp/nomad/vendor/github.com/stretchr/testify/assert/assertions.go:1493 +0x272 FAIL github.com/hashicorp/nomad/e2e 21.427s --- e2e/connect/multi_service.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/e2e/connect/multi_service.go b/e2e/connect/multi_service.go index 0aa4eb8c0ed..3a93ebedba2 100644 --- a/e2e/connect/multi_service.go +++ b/e2e/connect/multi_service.go @@ -55,7 +55,7 @@ EVAL: require.Len(t, eval.QueuedAllocations, 1, pretty.Sprint(eval.QueuedAllocations)) // Assert allocs are running - require.Eventually(t, func() bool { + for i := 0; i < 20; i++ { allocs, qmeta, err := evalapi.Allocations(eval.ID, qopts) require.NoError(t, err) require.Len(t, allocs, 1) @@ -69,15 +69,16 @@ EVAL: case "pending": // keep trying default: - t.Fatalf("alloc failed: %s", pretty.Sprint(alloc)) + require.Failf(t, "alloc failed", "alloc: %s", pretty.Sprint(alloc)) } } if running == len(allocs) { - return true + break } - return false - }, 10*time.Second, 500*time.Millisecond) + + time.Sleep(500 * time.Millisecond) + } allocs, _, err := evalapi.Allocations(eval.ID, qopts) require.NoError(t, err) From 968f0408f61a4d2a9a4e4e4a4ce903b9fa00d350 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Fri, 24 Jan 2020 12:03:08 -0600 Subject: [PATCH 23/27] e2e: uncomment test case that is not broken --- e2e/connect/acls.go | 59 +++++++++++++++++++-------------------------- 1 file changed, 25 insertions(+), 34 deletions(-) diff --git a/e2e/connect/acls.go b/e2e/connect/acls.go index a283d165df3..c98bf2176ae 100644 --- a/e2e/connect/acls.go +++ b/e2e/connect/acls.go @@ -159,40 +159,31 @@ func (tc *ConnectACLsE2ETest) createOperatorToken(policyID string, f *framework. return token.SecretID } -// TODO: This is test is broken and requires an actual fix. -// We currently do not check if the provided operator token is a master token, -// and we need to do that to be consistent with the semantics of the Consul ACL -// system. Fix will be covered in a separate issue. -// -//func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MasterToken(f *framework.F) { -// t := f.T() -// r := require.New(t) -// -// t.Log("test register Connect job w/ ACLs enabled w/ master token") -// -// jobID := "connect" + uuid.Generate()[0:8] -// tc.jobIDs = append(tc.jobIDs, jobID) -// -// jobAPI := tc.Nomad().Jobs() -// -// job, err := jobspec.ParseFile(demoConnectJob) -// r.NoError(err) -// -// // Set the job file to use the consul master token. -// // One should never do this in practice, but, it should work. -// // https://www.consul.io/docs/acl/acl-system.html#builtin-tokens -// // -// // note: We cannot just set the environment variable when using the API -// // directly - that only works when using the nomad CLI command which does -// // the step of converting the environment variable into a set option. -// job.ConsulToken = &tc.consulMasterToken -// -// resp, _, err := jobAPI.Register(job, nil) -// r.NoError(err) -// r.NotNil(resp) -// r.Zero(resp.Warnings) -//} -// +func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MasterToken(f *framework.F) { + t := f.T() + r := require.New(t) + + t.Log("test register Connect job w/ ACLs enabled w/ master token") + + jobID := "connect" + uuid.Generate()[0:8] + tc.jobIDs = append(tc.jobIDs, jobID) + + jobAPI := tc.Nomad().Jobs() + + job, err := jobspec.ParseFile(demoConnectJob) + r.NoError(err) + + // Set the job file to use the consul master token. + // One should never do this in practice, but, it should work. + // https://www.consul.io/docs/acl/acl-system.html#builtin-tokens + job.ConsulToken = &tc.consulMasterToken + + resp, _, err := jobAPI.Register(job, nil) + r.NoError(err) + r.NotNil(resp) + r.Zero(resp.Warnings) +} + func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MissingOperatorToken(f *framework.F) { t := f.T() r := require.New(t) From 676af6a3d298a05fd90996750fe045b4bbdc1d8c Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Mon, 27 Jan 2020 09:52:44 -0600 Subject: [PATCH 24/27] e2e: use hclfmt on consul acls policy config files --- e2e/consulacls/README.md | 3 ++- e2e/consulacls/acl-enable.hcl | 4 ++-- e2e/consulacls/consul-server-policy.hcl | 1 - e2e/consulacls/nomad-client-consul.hcl | 2 +- e2e/consulacls/nomad-client-policy.hcl | 1 - e2e/consulacls/nomad-server-consul.hcl | 2 +- e2e/consulacls/nomad-server-policy.hcl | 2 -- 7 files changed, 6 insertions(+), 9 deletions(-) diff --git a/e2e/consulacls/README.md b/e2e/consulacls/README.md index 10394525a68..772261b85a4 100644 --- a/e2e/consulacls/README.md +++ b/e2e/consulacls/README.md @@ -7,7 +7,8 @@ provisioned e2e environment to enable Consul ACLs. The `consul-acls-manage.sh` script can be used to manipulate the Consul cluster to activate or de-activate Consul ACLs. There are 3 targets into the script, only -2 of which should be used from e2e framework tests. +2 of which should be used from e2e framework tests. The script should be run from +the e2e directory (i.e. the directory from wich the e2e framework also runs). ### bootstrap diff --git a/e2e/consulacls/acl-enable.hcl b/e2e/consulacls/acl-enable.hcl index ec1361ebee8..f4204befd44 100644 --- a/e2e/consulacls/acl-enable.hcl +++ b/e2e/consulacls/acl-enable.hcl @@ -2,7 +2,7 @@ # consul-acls-manage.sh script uploads this file as "acl.hcl" to Consul Server # configuration directories, and restarts those agents. acl = { - enabled = true - default_policy = "deny" + enabled = true + default_policy = "deny" enable_token_persistence = true } diff --git a/e2e/consulacls/consul-server-policy.hcl b/e2e/consulacls/consul-server-policy.hcl index 0e7cea6a87d..14d3d458eaf 100644 --- a/e2e/consulacls/consul-server-policy.hcl +++ b/e2e/consulacls/consul-server-policy.hcl @@ -23,4 +23,3 @@ query "" { service "" { policy = "write" } - diff --git a/e2e/consulacls/nomad-client-consul.hcl b/e2e/consulacls/nomad-client-consul.hcl index ef89f47ac36..2cd6ecc8471 100644 --- a/e2e/consulacls/nomad-client-consul.hcl +++ b/e2e/consulacls/nomad-client-consul.hcl @@ -1,4 +1,4 @@ // The provided consul.token value must be blessed with service=write ACLs. consul { - token="CONSUL_TOKEN" + token = "CONSUL_TOKEN" } diff --git a/e2e/consulacls/nomad-client-policy.hcl b/e2e/consulacls/nomad-client-policy.hcl index 059c59a099b..8d3206a137b 100644 --- a/e2e/consulacls/nomad-client-policy.hcl +++ b/e2e/consulacls/nomad-client-policy.hcl @@ -3,4 +3,3 @@ service "" { policy = "write" } - diff --git a/e2e/consulacls/nomad-server-consul.hcl b/e2e/consulacls/nomad-server-consul.hcl index 8a7985f2c10..ac42b5bab13 100644 --- a/e2e/consulacls/nomad-server-consul.hcl +++ b/e2e/consulacls/nomad-server-consul.hcl @@ -4,5 +4,5 @@ // The provided consul.token value must be blessed with acl=write ACLs. consul { allow_unauthenticated = false - token="CONSUL_TOKEN" + token = "CONSUL_TOKEN" } diff --git a/e2e/consulacls/nomad-server-policy.hcl b/e2e/consulacls/nomad-server-policy.hcl index 4cee1a41ddc..185c6e0a44e 100644 --- a/e2e/consulacls/nomad-server-policy.hcl +++ b/e2e/consulacls/nomad-server-policy.hcl @@ -2,5 +2,3 @@ // will be requesting new SI tokens from Consul. acl = "write" - - From 02aeb1770aecbd1aa0f1684cc7a952074457a468 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Mon, 27 Jan 2020 13:39:09 -0600 Subject: [PATCH 25/27] e2e: agent token was only being set for server0 --- e2e/consulacls/consul-acls-manage.sh | 46 ++++++++++++---------------- 1 file changed, 19 insertions(+), 27 deletions(-) diff --git a/e2e/consulacls/consul-acls-manage.sh b/e2e/consulacls/consul-acls-manage.sh index d2a24491d1e..aa595a452f7 100755 --- a/e2e/consulacls/consul-acls-manage.sh +++ b/e2e/consulacls/consul-acls-manage.sh @@ -97,8 +97,8 @@ function doBootstrap() { stopConsul # Run the activation step, which uploads the ACLs-enabled acl.hcl file - # to each Consul Server's configuration directory, then (re)starts each - # Consul Server agent. + # to each Consul configuration directory, then (re)starts each + # Consul agent. doActivate echo "=== Bootstrap: Consul ACL Bootstrap ===" @@ -122,28 +122,25 @@ function doBootstrap() { echo "---> will create agent token for server ${server}" server_agent_token=$(consul acl token create -description "consul server agent token" -policy-name server-policy | grep SecretID | awk '{print $2}') echo "---> setting token for server agent: ${server} -> ${server_agent_token}" - consul acl set-agent-token agent "${server_agent_token}" + (export CONSUL_HTTP_ADDR="${server}:8500"; consul acl set-agent-token agent "${server_agent_token}") echo "---> done setting agent token for server ${server}" done # Wait 10s before continuing with configuring consul clients. - echo "-> sleep 10s" + echo "-> sleep 10s before continuing with clients" sleep 10 - # Start the Consul Clients back up so we can set their tokens now - startConsulClients - # Create Consul Client Policy & Client agent tokens echo "-> configure consul client policy" consul acl policy create -name client-policy -rules @consulacls/consul-client-policy.hcl - # Create & Set agent token for each Consul Client (including windows) - for client in ${clients}; do - echo "---> will create consul agent token for client ${client}" + # Create & Set agent token for each Consul Client (excluding Windows) + for linux_client in ${linux_clients}; do + echo "---> will create consul agent token for client ${linux_client}" client_agent_token=$(consul acl token create -description "consul client agent token" -policy-name client-policy | grep SecretID | awk '{print $2}') - echo "---> setting consul token for consul client ${client} -> ${client_agent_token}" - consul acl set-agent-token agent "${client_agent_token}" - echo "---> done setting consul agent token for client ${client}" + echo "---> setting consul token for consul client ${linux_client} -> ${client_agent_token}" + (export CONSUL_HTTP_ADDR="${linux_client}:8500"; consul acl set-agent-token agent "${client_agent_token}") + echo "---> done setting consul agent token for client ${linux_client}" done echo "=== Bootstrap: Nomad Configs ===" @@ -221,26 +218,21 @@ function doActivate() { stopConsul - # Upload acl-enable.hcl to each Consul Server agent's configuration directory. - for server in ${servers}; do - echo " activate: upload acl-enable.hcl to ${server}::acl.hcl" - doSCP "consulacls/acl-enable.hcl" "${user}" "${server}" "/tmp/acl.hcl" - doSSH "${server}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" + # Upload acl-enable.hcl to each Consul agent's configuration directory. + for agent in ${servers} ${linux_clients}; do + echo " activate: upload acl-enable.hcl to ${agent}::acl.hcl" + doSCP "consulacls/acl-enable.hcl" "${user}" "${agent}" "/tmp/acl.hcl" + doSSH "${agent}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" done - # Restart each Consul Server agent to pickup the new config. - for server in ${servers}; do - echo " activate: restart Consul Server on ${server} ..." - doSSH "${server}" "sudo systemctl start consul" + # Restart each Consul agent to pickup the new config. + for agent in ${servers} ${linux_clients}; do + echo " activate: restart Consul agent on ${agent} ..." + doSSH "${agent}" "sudo systemctl start consul" sleep 1 done sleep 10 - - startConsulClients - - sleep 10 - echo "=== Activate: DONE ===" } From e4d7b5f423eac83cc258a77eeb800f0fcf9ea640 Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Mon, 27 Jan 2020 13:52:53 -0600 Subject: [PATCH 26/27] e2e: remove redundant extra API call for getting allocs --- e2e/connect/acls.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/e2e/connect/acls.go b/e2e/connect/acls.go index c98bf2176ae..f31e82dca5d 100644 --- a/e2e/connect/acls.go +++ b/e2e/connect/acls.go @@ -293,8 +293,10 @@ EVAL: r.Len(eval.QueuedAllocations, 2, pretty.Sprint(eval.QueuedAllocations)) // === Assert allocs are running === + var allocs []*napi.AllocationListStub + for i := 0; i < 20; i++ { - allocs, qMeta, err := evalAPI.Allocations(eval.ID, qOpts) + allocs, qMeta, err = evalAPI.Allocations(eval.ID, qOpts) r.NoError(err) r.Len(allocs, 2) qOpts.WaitIndex = qMeta.LastIndex @@ -318,8 +320,6 @@ EVAL: time.Sleep(500 * time.Millisecond) } - allocs, _, err := evalAPI.Allocations(eval.ID, qOpts) - r.NoError(err) allocIDs := make(map[string]bool, 2) for _, a := range allocs { if a.ClientStatus != "running" || a.DesiredStatus != "run" { From 45920e9d197ae02633522abdcd3f0b9002f59c8b Mon Sep 17 00:00:00 2001 From: Seth Hoenig Date: Tue, 28 Jan 2020 16:33:59 -0600 Subject: [PATCH 27/27] e2e: setup consul ACLs a little more correctly --- .../taskrunner/task_runner_hooks.go | 19 +- .../taskrunner/task_runner_test.go | 14 +- e2e/affinities/affinities.go | 6 +- e2e/clientstate/clientstate.go | 6 +- e2e/connect/acls.go | 212 +++++++++++------- e2e/connect/acls_test.go | 19 ++ e2e/connect/client.go | 2 +- e2e/connect/connect.go | 19 +- e2e/consul/consul.go | 4 +- e2e/consul/script_checks.go | 12 +- e2e/consulacls/acl-pre-enable.hcl | 13 ++ e2e/consulacls/consul-acls-manage.sh | 117 +++++++--- .../consul-client-default-token.hcl | 7 + e2e/consulacls/consul-client-policy.hcl | 12 +- e2e/consulacls/consul-server-policy.hcl | 12 +- e2e/consulacls/nomad-client-policy.hcl | 10 +- e2e/consulacls/nomad-server-policy.hcl | 12 + e2e/deployment/deployment.go | 4 +- e2e/e2eutil/utils.go | 47 +++- e2e/framework/framework.go | 1 + e2e/hostvolumes/host_volumes.go | 2 +- e2e/metrics/metrics.go | 2 +- e2e/metrics/prometheus.go | 4 +- e2e/nomad09upgrade/upgrade.go | 2 +- e2e/nomadexec/exec.go | 2 +- e2e/spread/spread.go | 4 +- e2e/taskevents/taskevents.go | 2 +- 27 files changed, 380 insertions(+), 186 deletions(-) create mode 100644 e2e/connect/acls_test.go create mode 100644 e2e/consulacls/acl-pre-enable.hcl create mode 100644 e2e/consulacls/consul-client-default-token.hcl diff --git a/client/allocrunner/taskrunner/task_runner_hooks.go b/client/allocrunner/taskrunner/task_runner_hooks.go index 2da0c9fe528..549b8316ebf 100644 --- a/client/allocrunner/taskrunner/task_runner_hooks.go +++ b/client/allocrunner/taskrunner/task_runner_hooks.go @@ -109,13 +109,18 @@ func (tr *TaskRunner) initHooks() { // If this is a Connect sidecar proxy (or a Connect Native) service, // add the sidsHook for requesting a Service Identity token (if ACLs). if task.UsesConnect() { - tr.runnerHooks = append(tr.runnerHooks, newSIDSHook(sidsHookConfig{ - alloc: tr.Alloc(), - task: tr.Task(), - sidsClient: tr.siClient, - lifecycle: tr, - logger: hookLogger, - })) + // Enable the Service Identity hook only if the Nomad client is configured + // with a consul token, indicating that Consul ACLs are enabled + if tr.clientConfig.ConsulConfig.Token != "" { + tr.runnerHooks = append(tr.runnerHooks, newSIDSHook(sidsHookConfig{ + alloc: tr.Alloc(), + task: tr.Task(), + sidsClient: tr.siClient, + lifecycle: tr, + logger: hookLogger, + })) + } + // envoy bootstrap must execute after sidsHook maybe sets SI token tr.runnerHooks = append(tr.runnerHooks, newEnvoyBootstrapHook(&envoyBootstrapHookConfig{ alloc: alloc, diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 15b0aee310f..9d1fc71acc2 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -1095,7 +1095,7 @@ func (_ *mockEnvoyBootstrapHook) Name() string { return "mock_envoy_bootstrap" } -func (m *mockEnvoyBootstrapHook) Prestart(_ context.Context, _ *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error { +func (_ *mockEnvoyBootstrapHook) Prestart(_ context.Context, _ *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error { resp.Done = true return nil } @@ -1127,6 +1127,10 @@ func TestTaskRunner_BlockForSIDSToken(t *testing.T) { trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) defer cleanup() + // set a consul token on the Nomad client's consul config, because that is + // what gates the action of requesting SI token(s) + trConfig.ClientConfig.ConsulConfig.Token = uuid.Generate() + // control when we get a Consul SI token token := uuid.Generate() waitCh := make(chan struct{}) @@ -1191,6 +1195,10 @@ func TestTaskRunner_DeriveSIToken_Retry(t *testing.T) { trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) defer cleanup() + // set a consul token on the Nomad client's consul config, because that is + // what gates the action of requesting SI token(s) + trConfig.ClientConfig.ConsulConfig.Token = uuid.Generate() + // control when we get a Consul SI token (recoverable failure on first call) token := uuid.Generate() deriveCount := 0 @@ -1251,6 +1259,10 @@ func TestTaskRunner_DeriveSIToken_Unrecoverable(t *testing.T) { trConfig, cleanup := testTaskRunnerConfig(t, alloc, task.Name) defer cleanup() + // set a consul token on the Nomad client's consul config, because that is + // what gates the action of requesting SI token(s) + trConfig.ClientConfig.ConsulConfig.Token = uuid.Generate() + // SI token derivation suffers a non-retryable error siClient := trConfig.ConsulSI.(*consulapi.MockServiceIdentitiesClient) siClient.SetDeriveTokenError(alloc.ID, []string{task.Name}, errors.New("non-recoverable")) diff --git a/e2e/affinities/affinities.go b/e2e/affinities/affinities.go index 94e305a6eae..8c3e0202ce3 100644 --- a/e2e/affinities/affinities.go +++ b/e2e/affinities/affinities.go @@ -35,7 +35,7 @@ func (tc *BasicAffinityTest) TestSingleAffinities(f *framework.F) { uuid := uuid.Generate() jobId := "aff" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "affinities/input/single_affinity.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "affinities/input/single_affinity.nomad", jobId, "") jobAllocs := nomadClient.Allocations() require := require.New(f.T()) @@ -59,7 +59,7 @@ func (tc *BasicAffinityTest) TestMultipleAffinities(f *framework.F) { uuid := uuid.Generate() jobId := "multiaff" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "affinities/input/multiple_affinities.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "affinities/input/multiple_affinities.nomad", jobId, "") jobAllocs := nomadClient.Allocations() require := require.New(f.T()) @@ -101,7 +101,7 @@ func (tc *BasicAffinityTest) TestAntiAffinities(f *framework.F) { uuid := uuid.Generate() jobId := "antiaff" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "affinities/input/anti_affinities.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "affinities/input/anti_affinities.nomad", jobId, "") jobAllocs := nomadClient.Allocations() require := require.New(f.T()) diff --git a/e2e/clientstate/clientstate.go b/e2e/clientstate/clientstate.go index 0883d536787..71ac22a2888 100644 --- a/e2e/clientstate/clientstate.go +++ b/e2e/clientstate/clientstate.go @@ -100,7 +100,7 @@ func (tc *ClientStateTC) TestClientState_Kill(f *framework.F) { f.NoError(err) jobID := "sleeper-" + uuid.Generate()[:8] - allocs := e2eutil.RegisterAndWaitForAllocs(t, client, "clientstate/sleeper.nomad", jobID) + allocs := e2eutil.RegisterAndWaitForAllocs(t, client, "clientstate/sleeper.nomad", jobID, "") f.Len(allocs, 1) alloc, _, err := client.Allocations().Info(allocs[0].ID, nil) @@ -242,7 +242,7 @@ func (tc *ClientStateTC) TestClientState_KillDuringRestart(f *framework.F) { f.NoError(err) jobID := "restarter-" + uuid.Generate()[:8] - allocs := e2eutil.RegisterAndWaitForAllocs(t, client, "clientstate/restarter.nomad", jobID) + allocs := e2eutil.RegisterAndWaitForAllocs(t, client, "clientstate/restarter.nomad", jobID, "") f.Len(allocs, 1) alloc, _, err := client.Allocations().Info(allocs[0].ID, nil) @@ -363,7 +363,7 @@ func (tc *ClientStateTC) TestClientState_Corrupt(f *framework.F) { f.NoError(err) jobID := "sleeper-" + uuid.Generate()[:8] - allocs := e2eutil.RegisterAndWaitForAllocs(t, client, "clientstate/sleeper.nomad", jobID) + allocs := e2eutil.RegisterAndWaitForAllocs(t, client, "clientstate/sleeper.nomad", jobID, "") f.Len(allocs, 1) alloc, _, err := client.Allocations().Info(allocs[0].ID, nil) diff --git a/e2e/connect/acls.go b/e2e/connect/acls.go index f31e82dca5d..7be3128fddf 100644 --- a/e2e/connect/acls.go +++ b/e2e/connect/acls.go @@ -1,13 +1,14 @@ package connect import ( + "fmt" "os" + "regexp" "strings" "testing" "time" capi "github.com/hashicorp/consul/api" - consulapi "github.com/hashicorp/consul/api" napi "github.com/hashicorp/nomad/api" "github.com/hashicorp/nomad/e2e/consulacls" "github.com/hashicorp/nomad/e2e/e2eutil" @@ -126,6 +127,14 @@ func (tc *ConnectACLsE2ETest) AfterEach(f *framework.F) { err := tc.Nomad().System().GarbageCollect() r.NoError(err) + // assert there are no leftover SI tokens, which may take a minute to be + // cleaned up + r.Eventually(func() bool { + siTokens := tc.countSITokens(t) + t.Log("cleanup: checking for remaining SI tokens:", siTokens) + return len(siTokens) == 0 + }, 2*time.Minute, 2*time.Second, "SI tokens did not get removed") + tc.jobIDs = []string{} tc.consulTokenIDs = []string{} tc.consulPolicyIDs = []string{} @@ -159,7 +168,7 @@ func (tc *ConnectACLsE2ETest) createOperatorToken(policyID string, f *framework. return token.SecretID } -func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MasterToken(f *framework.F) { +func (tc *ConnectACLsE2ETest) TestConnectACLsRegisterMasterToken(f *framework.F) { t := f.T() r := require.New(t) @@ -178,13 +187,17 @@ func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MasterToken(f *framework.F // https://www.consul.io/docs/acl/acl-system.html#builtin-tokens job.ConsulToken = &tc.consulMasterToken - resp, _, err := jobAPI.Register(job, nil) + // Avoid using Register here, because that would actually create and run the + // Job which runs the task, creates the SI token, which all needs to be + // given time to settle and cleaned up. That is all covered in the big slow + // test at the bottom. + resp, _, err := jobAPI.Plan(job, false, nil) r.NoError(err) r.NotNil(resp) - r.Zero(resp.Warnings) + fmt.Println("resp:", resp) } -func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MissingOperatorToken(f *framework.F) { +func (tc *ConnectACLsE2ETest) TestConnectACLsRegisterMissingOperatorToken(f *framework.F) { t := f.T() r := require.New(t) @@ -204,7 +217,7 @@ func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_MissingOperatorToken(f *fr t.Log("job correctly rejected, with error:", err) } -func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_FakeOperatorToken(f *framework.F) { +func (tc *ConnectACLsE2ETest) TestConnectACLsRegisterFakeOperatorToken(f *framework.F) { t := f.T() r := require.New(t) @@ -231,7 +244,7 @@ func (tc *ConnectACLsE2ETest) TestConnectACLsRegister_FakeOperatorToken(f *frame t.Log("job correctly rejected, with error:", err) } -func (tc *ConnectACLsE2ETest) TestConnectACLs_ConnectDemo(f *framework.F) { +func (tc *ConnectACLsE2ETest) TestConnectACLsConnectDemo(f *framework.F) { t := f.T() r := require.New(t) @@ -251,81 +264,94 @@ func (tc *ConnectACLsE2ETest) TestConnectACLs_ConnectDemo(f *framework.F) { t.Log("created operator token:", operatorToken) // === Register the Nomad job === + jobID := "connectACL_connect_demo" + //{ + // + // nomadClient := tc.Nomad() + // allocs := e2eutil.RegisterAndWaitForAllocs(t, nomadClient, demoConnectJob, jobID, operatorToken) + // allocIDs := e2eutil.AllocIDsFromAllocationListStubs(allocs) + // e2eutil.WaitForAllocsRunning(t, nomadClient, allocIDs) + //} - // parse the example connect jobspec file - jobID := "connect" + uuid.Generate()[0:8] - tc.jobIDs = append(tc.jobIDs, jobID) - job := tc.parseJobSpecFile(t, demoConnectJob) - job.ID = &jobID - jobAPI := tc.Nomad().Jobs() - - // set the valid consul operator token - job.ConsulToken = &operatorToken - - // registering the job should succeed - resp, _, err := jobAPI.Register(job, nil) - r.NoError(err) - r.NotNil(resp) - r.Empty(resp.Warnings) - t.Log("job has been registered with evalID:", resp.EvalID) - - // === Make sure the evaluation actually succeeds === -EVAL: - qOpts := &napi.QueryOptions{WaitIndex: resp.EvalCreateIndex} - evalAPI := tc.Nomad().Evaluations() - eval, qMeta, err := evalAPI.Info(resp.EvalID, qOpts) - r.NoError(err) - qOpts.WaitIndex = qMeta.LastIndex - - switch eval.Status { - case "pending": - goto EVAL - case "complete": - // ok! - case "failed", "canceled", "blocked": - r.Failf("eval %s\n%s\n", eval.Status, pretty.Sprint(eval)) - default: - r.Failf("unknown eval status: %s\n%s\n", eval.Status, pretty.Sprint(eval)) - } + var allocs []*napi.AllocationListStub + allocIDs := make(map[string]bool, 2) + { + // jobID := "connect" + uuid.Generate()[0:8] (nicer name now) - // assert there were no placement failures - r.Zero(eval.FailedTGAllocs, pretty.Sprint(eval.FailedTGAllocs)) - r.Len(eval.QueuedAllocations, 2, pretty.Sprint(eval.QueuedAllocations)) + // parse the example connect jobspec file + tc.jobIDs = append(tc.jobIDs, jobID) + job := tc.parseJobSpecFile(t, demoConnectJob) + job.ID = &jobID + jobAPI := tc.Nomad().Jobs() - // === Assert allocs are running === - var allocs []*napi.AllocationListStub + // set the valid consul operator token + job.ConsulToken = &operatorToken - for i := 0; i < 20; i++ { - allocs, qMeta, err = evalAPI.Allocations(eval.ID, qOpts) + // registering the job should succeed + resp, _, err := jobAPI.Register(job, nil) + r.NoError(err) + r.NotNil(resp) + r.Empty(resp.Warnings) + t.Log("job has been registered with evalID:", resp.EvalID) + + // === Make sure the evaluation actually succeeds === + EVAL: + qOpts := &napi.QueryOptions{WaitIndex: resp.EvalCreateIndex} + evalAPI := tc.Nomad().Evaluations() + eval, qMeta, err := evalAPI.Info(resp.EvalID, qOpts) r.NoError(err) - r.Len(allocs, 2) qOpts.WaitIndex = qMeta.LastIndex - running := 0 - for _, alloc := range allocs { - switch alloc.ClientStatus { - case "running": - running++ - case "pending": - // keep trying - default: - r.Failf("alloc failed", "alloc: %s", pretty.Sprint(alloc)) - } + switch eval.Status { + case "pending": + goto EVAL + case "complete": + // ok! + case "failed", "canceled", "blocked": + r.Failf("eval %s\n%s\n", eval.Status, pretty.Sprint(eval)) + default: + r.Failf("unknown eval status: %s\n%s\n", eval.Status, pretty.Sprint(eval)) } - if running == len(allocs) { - break - } + // assert there were no placement failures + r.Zero(eval.FailedTGAllocs, pretty.Sprint(eval.FailedTGAllocs)) + r.Len(eval.QueuedAllocations, 2, pretty.Sprint(eval.QueuedAllocations)) + + // === Assert allocs are running === + // var allocs []*napi.AllocationListStub // move scope + + for i := 0; i < 20; i++ { + allocs, qMeta, err = evalAPI.Allocations(eval.ID, qOpts) + r.NoError(err) + r.Len(allocs, 2) + qOpts.WaitIndex = qMeta.LastIndex + + running := 0 + for _, alloc := range allocs { + switch alloc.ClientStatus { + case "running": + running++ + case "pending": + // keep trying + default: + r.Failf("alloc failed", "alloc: %s", pretty.Sprint(alloc)) + } + } - time.Sleep(500 * time.Millisecond) - } + if running == len(allocs) { + t.Log("running:", running, "allocs:", allocs) + break + } - allocIDs := make(map[string]bool, 2) - for _, a := range allocs { - if a.ClientStatus != "running" || a.DesiredStatus != "run" { - r.Failf("terminal alloc", "alloc %s (%s) terminal; client=%s desired=%s", a.TaskGroup, a.ID, a.ClientStatus, a.DesiredStatus) + time.Sleep(500 * time.Millisecond) + } + + for _, a := range allocs { + if a.ClientStatus != "running" || a.DesiredStatus != "run" { + r.Failf("terminal alloc", "alloc %s (%s) terminal; client=%s desired=%s", a.TaskGroup, a.ID, a.ClientStatus, a.DesiredStatus) + } + allocIDs[a.ID] = true } - allocIDs[a.ID] = true } // === Check Consul service health === @@ -339,7 +365,8 @@ EVAL: // filter out checks for other services for cid, check := range checks { found := false - for allocID := range allocIDs { + // for _, allocID := range allocIDs { // list + for allocID := range allocIDs { // map if strings.Contains(check.ServiceID, allocID) { found = true break @@ -352,7 +379,7 @@ EVAL: } // ensure checks are all passing - failing = map[string]*consulapi.AgentCheck{} + failing = map[string]*capi.AgentCheck{} for _, check := range checks { if check.Status != "passing" { failing[check.CheckID] = check @@ -372,26 +399,41 @@ EVAL: require.Len(t, failing, 0, pretty.Sprint(failing)) // === Check Consul SI tokens were generated for sidecars === - aclAPI := tc.Consul().ACL() + foundSITokens := tc.countSITokens(t) + r.Equal(2, len(foundSITokens), "expected 2 SI tokens total: %v", foundSITokens) + r.Equal(1, foundSITokens["connect-proxy-count-api"], "expected 1 SI token for connect-proxy-count-api: %v", foundSITokens) + r.Equal(1, foundSITokens["connect-proxy-count-dashboard"], "expected 1 SI token for connect-proxy-count-dashboard: %v", foundSITokens) + + t.Log("connect job with ACLs enable finished") +} + +var ( + siTokenRe = regexp.MustCompile(`_nomad_si \[[\w-]{36}] \[[\w-]{36}] \[([\S]+)]`) +) + +func (tc *ConnectACLsE2ETest) serviceofSIToken(description string) string { + if m := siTokenRe.FindStringSubmatch(description); len(m) == 2 { + return m[1] + } + return "" +} - entries, _, err := aclAPI.TokenList(&capi.QueryOptions{ +func (tc *ConnectACLsE2ETest) countSITokens(t *testing.T) map[string]int { + aclAPI := tc.Consul().ACL() + tokens, _, err := aclAPI.TokenList(&capi.QueryOptions{ Token: tc.consulMasterToken, }) - r.NoError(err) + require.NoError(t, err) - foundSITokenForCountDash := false - foundSITokenForCountAPI := false - for _, entry := range entries { - if strings.Contains(entry.Description, "[connect-proxy-count-dashboard]") { - foundSITokenForCountDash = true - } else if strings.Contains(entry.Description, "[connect-proxy-count-api]") { - foundSITokenForCountAPI = true + // count the number of SI tokens matching each service name + foundSITokens := make(map[string]int) + for _, token := range tokens { + if service := tc.serviceofSIToken(token.Description); service != "" { + foundSITokens[service]++ } } - r.True(foundSITokenForCountDash, "no SI token found for count-dash") - r.True(foundSITokenForCountAPI, "no SI token found for count-api") - t.Log("connect job with ACLs enable finished") + return foundSITokens } func (tc *ConnectACLsE2ETest) parseJobSpecFile(t *testing.T, filename string) *napi.Job { diff --git a/e2e/connect/acls_test.go b/e2e/connect/acls_test.go new file mode 100644 index 00000000000..37e2d02bc26 --- /dev/null +++ b/e2e/connect/acls_test.go @@ -0,0 +1,19 @@ +package connect + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func Test_serviceOfSIToken(t *testing.T) { + try := func(description, exp string) { + tc := new(ConnectACLsE2ETest) + result := tc.serviceofSIToken(description) + require.Equal(t, exp, result) + } + + try("", "") + try("foobarbaz", "") + try("_nomad_si [8b1a5d3f-7e61-4a5a-8a57-7e7ad91e63b6] [8b1a5d3f-7e61-4a5a-8a57-7e7ad91e63b6] [foo-service]", "foo-service") +} diff --git a/e2e/connect/client.go b/e2e/connect/client.go index 8cea468e5fe..2d9a2016c7f 100644 --- a/e2e/connect/client.go +++ b/e2e/connect/client.go @@ -41,7 +41,7 @@ func (tc *ConnectClientStateE2ETest) TestClientRestart(f *framework.F) { consulClient := tc.Consul() allocs := e2eutil.RegisterAndWaitForAllocs(t, client, - "connect/input/demo.nomad", jobID) + "connect/input/demo.nomad", jobID, "") require.Equal(2, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, diff --git a/e2e/connect/connect.go b/e2e/connect/connect.go index f2a522c968c..44e54bbbfb2 100644 --- a/e2e/connect/connect.go +++ b/e2e/connect/connect.go @@ -22,21 +22,22 @@ type ConnectE2ETest struct { func init() { // connect tests without Consul ACLs enabled - framework.AddSuites(&framework.TestSuite{ - Component: "Connect", - CanRunLocal: true, - Consul: true, - Cases: []framework.TestCase{ - new(ConnectE2ETest), - new(ConnectClientStateE2ETest), - }, - }) + //framework.AddSuites(&framework.TestSuite{ + // Component: "Connect", + // CanRunLocal: true, + // Consul: true, + // Cases: []framework.TestCase{ + // new(ConnectE2ETest), + // new(ConnectClientStateE2ETest), + // }, + //}) // connect tests with Consul ACLs enabled framework.AddSuites(&framework.TestSuite{ Component: "ConnectACLs", CanRunLocal: false, Consul: true, + Parallel: false, Cases: []framework.TestCase{ new(ConnectACLsE2ETest), }, diff --git a/e2e/consul/consul.go b/e2e/consul/consul.go index e0b85c0e9e2..0ae9c04d23f 100644 --- a/e2e/consul/consul.go +++ b/e2e/consul/consul.go @@ -47,7 +47,7 @@ func (tc *ConsulE2ETest) TestConsulRegistration(f *framework.F) { jobId := "consul" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "consul/input/consul_example.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "consul/input/consul_example.nomad", jobId, "") consulClient := tc.Consul() require := require.New(f.T()) require.Equal(3, len(allocs)) @@ -105,7 +105,7 @@ func (tc *ConsulE2ETest) TestCanaryInplaceUpgrades(f *framework.F) { jobId := "consul" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "consul/input/canary_tags.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "consul/input/canary_tags.nomad", jobId, "") consulClient := tc.Consul() require := require.New(f.T()) require.Equal(2, len(allocs)) diff --git a/e2e/consul/script_checks.go b/e2e/consul/script_checks.go index 7e980eaed40..837e151d8e9 100644 --- a/e2e/consul/script_checks.go +++ b/e2e/consul/script_checks.go @@ -42,7 +42,7 @@ func (tc *ScriptChecksE2ETest) TestGroupScriptCheck(f *framework.F) { // Job run: verify that checks were registered in Consul allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), - nomadClient, "consul/input/checks_group.nomad", jobId) + nomadClient, "consul/input/checks_group.nomad", jobId, "") require.Equal(1, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, "group-service-1", capi.HealthPassing) e2eutil.RequireConsulStatus(require, consulClient, "group-service-2", capi.HealthWarning) @@ -56,7 +56,7 @@ func (tc *ScriptChecksE2ETest) TestGroupScriptCheck(f *framework.F) { // Job update: verify checks are re-registered in Consul allocs = e2eutil.RegisterAndWaitForAllocs(f.T(), - nomadClient, "consul/input/checks_group_update.nomad", jobId) + nomadClient, "consul/input/checks_group_update.nomad", jobId, "") require.Equal(1, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, "group-service-1", capi.HealthPassing) e2eutil.RequireConsulStatus(require, consulClient, "group-service-2", capi.HealthPassing) @@ -76,7 +76,7 @@ func (tc *ScriptChecksE2ETest) TestGroupScriptCheck(f *framework.F) { // Restore for next test allocs = e2eutil.RegisterAndWaitForAllocs(f.T(), - nomadClient, "consul/input/checks_group.nomad", jobId) + nomadClient, "consul/input/checks_group.nomad", jobId, "") require.Equal(2, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, "group-service-1", capi.HealthPassing) e2eutil.RequireConsulStatus(require, consulClient, "group-service-2", capi.HealthWarning) @@ -109,7 +109,7 @@ func (tc *ScriptChecksE2ETest) TestTaskScriptCheck(f *framework.F) { // Job run: verify that checks were registered in Consul allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), - nomadClient, "consul/input/checks_task.nomad", jobId) + nomadClient, "consul/input/checks_task.nomad", jobId, "") require.Equal(1, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, "task-service-1", capi.HealthPassing) e2eutil.RequireConsulStatus(require, consulClient, "task-service-2", capi.HealthWarning) @@ -123,7 +123,7 @@ func (tc *ScriptChecksE2ETest) TestTaskScriptCheck(f *framework.F) { // Job update: verify checks are re-registered in Consul allocs = e2eutil.RegisterAndWaitForAllocs(f.T(), - nomadClient, "consul/input/checks_task_update.nomad", jobId) + nomadClient, "consul/input/checks_task_update.nomad", jobId, "") require.Equal(1, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, "task-service-1", capi.HealthPassing) e2eutil.RequireConsulStatus(require, consulClient, "task-service-2", capi.HealthPassing) @@ -143,7 +143,7 @@ func (tc *ScriptChecksE2ETest) TestTaskScriptCheck(f *framework.F) { // Restore for next test allocs = e2eutil.RegisterAndWaitForAllocs(f.T(), - nomadClient, "consul/input/checks_task.nomad", jobId) + nomadClient, "consul/input/checks_task.nomad", jobId, "") require.Equal(2, len(allocs)) e2eutil.RequireConsulStatus(require, consulClient, "task-service-1", capi.HealthPassing) e2eutil.RequireConsulStatus(require, consulClient, "task-service-2", capi.HealthWarning) diff --git a/e2e/consulacls/acl-pre-enable.hcl b/e2e/consulacls/acl-pre-enable.hcl new file mode 100644 index 00000000000..a91916d7f5e --- /dev/null +++ b/e2e/consulacls/acl-pre-enable.hcl @@ -0,0 +1,13 @@ +# This partial consul configuration file will enable Consul ACLs in the default:allow +# mode, which is nessessary for the ACL bootstrapping process of a pre-existing cluster. +# +# The consul-acls-manage.sh script uploads this file as "acl.hcl" to Consul Server +# configuration directories, and restarts those agents. +# +# Later the consul-acls-manage.sh script will replace this configuration with the +# one found in acl-enable.sh so as to enforce ACLs. +acl = { + enabled = true + default_policy = "allow" + enable_token_persistence = true +} diff --git a/e2e/consulacls/consul-acls-manage.sh b/e2e/consulacls/consul-acls-manage.sh index aa595a452f7..367381937cf 100755 --- a/e2e/consulacls/consul-acls-manage.sh +++ b/e2e/consulacls/consul-acls-manage.sh @@ -24,7 +24,7 @@ linux_clients=$(jq -r .outputs.linux_clients.value[] <"${tfstatefile}" | xargs) windows_clients=$(jq -r .outputs.windows_clients.value[] <"${tfstatefile}" | xargs) # Combine all the clients together -clients="${linux_clients} ${windows_clients}" +# clients="${linux_clients} ${windows_clients}" # Load Server Node IPs from terraform/terraform.tfstate servers=$(jq -r .outputs.servers.value[] <"${tfstatefile}" | xargs) @@ -93,15 +93,14 @@ function doBootstrap() { # Stop all Nomad agents. stopNomad - # Stop all Consul agents. - stopConsul - - # Run the activation step, which uploads the ACLs-enabled acl.hcl file + # Run the pre-activation step, which uploads an acl.hcl file (with default:allow) # to each Consul configuration directory, then (re)starts each # Consul agent. - doActivate + doPreActivateACLs echo "=== Bootstrap: Consul ACL Bootstrap ===" + echo "sleeping 2 minutes to let Consul agents settle (avoid Legacy mode error)..." + sleep 120 # Bootstrap Consul ACLs on server[0] echo "-> bootstrap ACL using ${server0}" @@ -126,9 +125,9 @@ function doBootstrap() { echo "---> done setting agent token for server ${server}" done - # Wait 10s before continuing with configuring consul clients. - echo "-> sleep 10s before continuing with clients" - sleep 10 + # Wait 30s before continuing with configuring consul clients. + echo "-> sleep 3s before continuing with clients" + sleep 3 # Create Consul Client Policy & Client agent tokens echo "-> configure consul client policy" @@ -140,9 +139,13 @@ function doBootstrap() { client_agent_token=$(consul acl token create -description "consul client agent token" -policy-name client-policy | grep SecretID | awk '{print $2}') echo "---> setting consul token for consul client ${linux_client} -> ${client_agent_token}" (export CONSUL_HTTP_ADDR="${linux_client}:8500"; consul acl set-agent-token agent "${client_agent_token}") - echo "---> done setting consul agent token for client ${linux_client}" + echo "---> done setting agent token for client ${linux_client}" done + # Now, upload the ACL policy file with default:deny so that ACL are actually + # enforced. + doActivateACLs + echo "=== Bootstrap: Nomad Configs ===" # Create Nomad Server consul Policy and Nomad Server consul tokens @@ -178,13 +181,40 @@ function doBootstrap() { echo "=== Activate: DONE ===" } -function doEnable() { +function doSetAllowUnauthenticated { + value="${1}" + [ "${value}" == "true" ] || [ "${value}" == "false" ] || ( echo "allow_unauthenticated must be 'true' or 'false'" && exit 1) + for server in ${servers}; do + if [ "${value}" == "true" ]; then + echo "---> setting consul.allow_unauthenticated=true on ${server}" + doSSH "${server}" "sudo sed -i 's/allow_unauthenticated = false/allow_unauthenticated = true/g' ${nomad_configs}/nomad-server-consul.hcl" + else + echo "---> setting consul.allow_unauthenticated=false on ${server}" + doSSH "${server}" "sudo sed -i 's/allow_unauthenticated = true/allow_unauthenticated = false/g' ${nomad_configs}/nomad-server-consul.hcl" + fi + doSSH "${server}" "sudo systemctl restart nomad" + done + + for linux_client in ${linux_clients}; do + if [ "${value}" == "true" ]; then + echo "---> comment out consul token for Nomad client ${linux_client}" + doSSH "${linux_client}" "sudo sed -i 's!token =!// token =!g' ${nomad_configs}/nomad-client-consul.hcl" + else + echo "---> un-comment consul token for Nomad client ${linux_client}" + doSSH "${linux_client}" "sudo sed -i 's!// token =!token =!g' ${nomad_configs}/nomad-client-consul.hcl" + fi + doSSH "${linux_client}" "sudo systemctl restart nomad" + done +} + +function doEnable { if [ ! -f "${token_file}" ]; then echo "ENABLE: token file does not exist, doing a full ACL bootstrap" doBootstrap else echo "ENABLE: token file already exists, will activate ACLs" - doActivate + doSetAllowUnauthenticated "false" + doActivateACLs fi echo "=== Enable: DONE ===" @@ -197,13 +227,14 @@ function doEnable() { doStatus } -function doDisable() { +function doDisable { if [ ! -f "${token_file}" ]; then echo "DISABLE: token file does not exist, did bootstrap ever happen?" exit 1 else echo "DISABLE: token file exists, will deactivate ACLs" - doDeactivate + doSetAllowUnauthenticated "true" + doDeactivateACLs fi echo "=== Disable: DONE ===" @@ -213,8 +244,29 @@ function doDisable() { doStatus } -function doActivate() { - echo "=== Activate ===" +function doPreActivateACLs { + echo "=== PreActivate (set default:allow) ===" + + stopConsul + + # Upload acl-pre-enable.hcl to each Consul agent's configuration directory. + for agent in ${servers} ${linux_clients}; do + echo " pre-activate: upload acl-pre-enable.hcl to ${agent}::acl.hcl" + doSCP "consulacls/acl-pre-enable.hcl" "${user}" "${agent}" "/tmp/acl.hcl" + doSSH "${agent}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" + done + + # Start each Consul agent to pickup the new config. + for agent in ${servers} ${linux_clients}; do + echo " pre-activate: start Consul agent on ${agent}" + doSSH "${agent}" "sudo systemctl start consul" + done + + echo "=== PreActivate: DONE ===" +} + +function doActivateACLs { + echo "=== Activate (set default:deny) ===" stopConsul @@ -225,14 +277,14 @@ function doActivate() { doSSH "${agent}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" done - # Restart each Consul agent to pickup the new config. + # Start each Consul agent to pickup the new config. for agent in ${servers} ${linux_clients}; do echo " activate: restart Consul agent on ${agent} ..." doSSH "${agent}" "sudo systemctl start consul" - sleep 1 done - sleep 10 + echo "--> activate ACLs sleep for 2 minutes to let Consul figure things out" + sleep 120 echo "=== Activate: DONE ===" } @@ -312,20 +364,19 @@ function startConsulClients { echo "... all consul clients started" } -function doDeactivate { +function doDeactivateACLs { echo "=== Deactivate ===" - # Upload acl-disable.hcl to each Consul Server agent's configuration directory. - for server in ${servers}; do - echo " deactivate: upload acl-disable.hcl to ${server}::acl.hcl" - doSCP "consulacls/acl-disable.hcl" "${user}" "${server}" "/tmp/acl.hcl" - doSSH "${server}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" + # Upload acl-disable.hcl to each Consul agent's configuration directory. + for agent in ${servers} ${linux_clients}; do + echo " deactivate: upload acl-disable.hcl to ${agent}::acl.hcl" + doSCP "consulacls/acl-disable.hcl" "${user}" "${agent}" "/tmp/acl.hcl" + doSSH "${agent}" "sudo mv /tmp/acl.hcl ${consul_configs}/acl.hcl" done - # Restart each Consul server agent to pickup the new config. - for server in ${servers}; do - echo " deactivate: restart Consul Server on ${server} ..." - doSSH "${server}" "sudo systemctl restart consul" - sleep 3 # let the agent settle + # Restart each Consul agent to pickup the new config. + for agent in ${servers} ${linux_clients}; do + echo " deactivate: restart Consul on ${agent} ..." + doSSH "${agent}" "sudo systemctl restart consul" done # Wait 10s before moving on, Consul needs a second to calm down. @@ -348,12 +399,6 @@ function doStatus { # It's the entrypoint to our script! case "${subcommand}" in - bootstrap) - # The bootstrap target exists to make some local development easier. Test - # cases running from the e2e framework should always use "enable" which aims - # to be idempotent. - doBootstrap - ;; enable) doEnable ;; diff --git a/e2e/consulacls/consul-client-default-token.hcl b/e2e/consulacls/consul-client-default-token.hcl new file mode 100644 index 00000000000..00fe300c2c7 --- /dev/null +++ b/e2e/consulacls/consul-client-default-token.hcl @@ -0,0 +1,7 @@ +acl { + tokens { + agent = "CONSUL_TOKEN" + agent_master = "CONSUL_TOKEN" + default = "CONSUL_TOKEN" + } +} diff --git a/e2e/consulacls/consul-client-policy.hcl b/e2e/consulacls/consul-client-policy.hcl index 14d3d458eaf..6eef6296c68 100644 --- a/e2e/consulacls/consul-client-policy.hcl +++ b/e2e/consulacls/consul-client-policy.hcl @@ -1,25 +1,25 @@ acl = "write" -agent "" { +agent_prefix "" { policy = "write" } -event "" { +event_prefix "" { policy = "write" } -key "" { +key_prefix "" { policy = "write" } -node "" { +node_prefix "" { policy = "write" } -query "" { +query_prefix "" { policy = "write" } -service "" { +service_prefix "" { policy = "write" } diff --git a/e2e/consulacls/consul-server-policy.hcl b/e2e/consulacls/consul-server-policy.hcl index 14d3d458eaf..6eef6296c68 100644 --- a/e2e/consulacls/consul-server-policy.hcl +++ b/e2e/consulacls/consul-server-policy.hcl @@ -1,25 +1,25 @@ acl = "write" -agent "" { +agent_prefix "" { policy = "write" } -event "" { +event_prefix "" { policy = "write" } -key "" { +key_prefix "" { policy = "write" } -node "" { +node_prefix "" { policy = "write" } -query "" { +query_prefix "" { policy = "write" } -service "" { +service_prefix "" { policy = "write" } diff --git a/e2e/consulacls/nomad-client-policy.hcl b/e2e/consulacls/nomad-client-policy.hcl index 8d3206a137b..464d22a5e84 100644 --- a/e2e/consulacls/nomad-client-policy.hcl +++ b/e2e/consulacls/nomad-client-policy.hcl @@ -1,5 +1,13 @@ // The Nomad Client will be registering things into its buddy Consul Client. -service "" { +service_prefix "" { policy = "write" } + +node_prefix "" { + policy = "write" +} + +agent_prefix "" { + policy = "read" +} diff --git a/e2e/consulacls/nomad-server-policy.hcl b/e2e/consulacls/nomad-server-policy.hcl index 185c6e0a44e..fd47ae56bff 100644 --- a/e2e/consulacls/nomad-server-policy.hcl +++ b/e2e/consulacls/nomad-server-policy.hcl @@ -2,3 +2,15 @@ // will be requesting new SI tokens from Consul. acl = "write" + +service_prefix "" { + policy = "write" +} + +node_prefix "" { + policy = "write" +} + +agent_prefix "" { + policy = "read" +} diff --git a/e2e/deployment/deployment.go b/e2e/deployment/deployment.go index 04f818f543f..62761a8f9cd 100644 --- a/e2e/deployment/deployment.go +++ b/e2e/deployment/deployment.go @@ -41,13 +41,13 @@ func (tc *DeploymentTest) TestDeploymentAutoPromote(f *framework.F) { // unique each run, cluster could have previous jobs jobId := "deployment" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - e2eutil.RegisterAndWaitForAllocs(t, nomadClient, "deployment/input/deployment_auto0.nomad", jobId) + e2eutil.RegisterAndWaitForAllocs(t, nomadClient, "deployment/input/deployment_auto0.nomad", jobId, "") ds := e2eutil.DeploymentsForJob(t, nomadClient, jobId) require.Equal(t, 1, len(ds)) deploy := ds[0] // Upgrade - e2eutil.RegisterAllocs(t, nomadClient, "deployment/input/deployment_auto1.nomad", jobId) + e2eutil.RegisterAllocs(t, nomadClient, "deployment/input/deployment_auto1.nomad", jobId, "") // Find the deployment we don't already have testutil.WaitForResult(func() (bool, error) { diff --git a/e2e/e2eutil/utils.go b/e2e/e2eutil/utils.go index 676f71a5df7..effa1bd0fb4 100644 --- a/e2e/e2eutil/utils.go +++ b/e2e/e2eutil/utils.go @@ -54,13 +54,26 @@ func WaitForNodesReady(t *testing.T, nomadClient *api.Client, nodes int) { }) } -func RegisterAllocs(t *testing.T, nomadClient *api.Client, jobFile string, jobID string) []*api.AllocationListStub { +func stringToPtrOrNil(s string) *string { + if s == "" { + return nil + } + return helper.StringToPtr(s) +} + +func RegisterAllocs(t *testing.T, nomadClient *api.Client, jobFile, jobID, cToken string) []*api.AllocationListStub { + r := require.New(t) + // Parse job job, err := jobspec.ParseFile(jobFile) - require := require.New(t) - require.Nil(err) + r.Nil(err) + + // Set custom job ID (distinguish among tests) job.ID = helper.StringToPtr(jobID) + // Set a Consul "operator" token for the job, if provided. + job.ConsulToken = stringToPtrOrNil(cToken) + // Register job var idx uint64 jobs := nomadClient.Jobs() @@ -72,30 +85,32 @@ func RegisterAllocs(t *testing.T, nomadClient *api.Client, jobFile string, jobID idx = meta.LastIndex return resp.EvalID != "", fmt.Errorf("expected EvalID:%s", pretty.Sprint(resp)) }, func(err error) { - require.NoError(err) + r.NoError(err) }) - allocs, _, _ := jobs.Allocations(jobID, false, &api.QueryOptions{WaitIndex: idx}) + allocs, _, err := jobs.Allocations(jobID, false, &api.QueryOptions{WaitIndex: idx}) + require.NoError(t, err) return allocs } -func RegisterAndWaitForAllocs(t *testing.T, nomadClient *api.Client, jobFile string, jobID string) []*api.AllocationListStub { - require := require.New(t) +func RegisterAndWaitForAllocs(t *testing.T, nomadClient *api.Client, jobFile, jobID, cToken string) []*api.AllocationListStub { + r := require.New(t) g := NewGomegaWithT(t) jobs := nomadClient.Jobs() // Start allocations - RegisterAllocs(t, nomadClient, jobFile, jobID) + RegisterAllocs(t, nomadClient, jobFile, jobID, cToken) // Wrap in retry to wait until placement g.Eventually(func() []*api.AllocationListStub { // Look for allocations allocs, _, _ := jobs.Allocations(jobID, false, nil) + fmt.Println("!! Eventually Allocations:", allocs) return allocs }, 30*time.Second, time.Second).ShouldNot(BeEmpty()) allocs, _, err := jobs.Allocations(jobID, false, nil) - require.NoError(err) + r.NoError(err) return allocs } @@ -113,6 +128,20 @@ func WaitForAllocRunning(t *testing.T, nomadClient *api.Client, allocID string) }) } +func WaitForAllocsRunning(t *testing.T, nomadClient *api.Client, allocIDs []string) { + for _, allocID := range allocIDs { + WaitForAllocRunning(t, nomadClient, allocID) + } +} + +func AllocIDsFromAllocationListStubs(allocs []*api.AllocationListStub) []string { + allocIDs := make([]string, 0, len(allocs)) + for _, alloc := range allocs { + allocIDs = append(allocIDs, alloc.ID) + } + return allocIDs +} + func DeploymentsForJob(t *testing.T, nomadClient *api.Client, jobID string) []*api.Deployment { ds, _, err := nomadClient.Deployments().List(nil) require.NoError(t, err) diff --git a/e2e/framework/framework.go b/e2e/framework/framework.go index 558488b53ca..5fe2e39d7a5 100644 --- a/e2e/framework/framework.go +++ b/e2e/framework/framework.go @@ -303,6 +303,7 @@ func isTestMethod(m string) bool { if !strings.HasPrefix(m, "Test") { return false } + // THINKING: adding flag to target a specific test or test regex? return true } diff --git a/e2e/hostvolumes/host_volumes.go b/e2e/hostvolumes/host_volumes.go index 17188af5a5a..1f7a30af0e0 100644 --- a/e2e/hostvolumes/host_volumes.go +++ b/e2e/hostvolumes/host_volumes.go @@ -39,7 +39,7 @@ func (tc *BasicHostVolumeTest) TestSingleHostVolume(f *framework.F) { uuid := uuid.Generate() jobID := "hostvol" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobID) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "hostvolumes/input/single_mount.nomad", jobID) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "hostvolumes/input/single_mount.nomad", jobID, "") waitForTaskState := func(desiredState string) { require.Eventually(func() bool { diff --git a/e2e/metrics/metrics.go b/e2e/metrics/metrics.go index 7d44e9f2870..083120b283d 100644 --- a/e2e/metrics/metrics.go +++ b/e2e/metrics/metrics.go @@ -117,7 +117,7 @@ func (tc *MetricsTest) runWorkloads(t *testing.T, workloads map[string]string) { jobID := "metrics-" + jobName + "-" + uuid[0:8] tc.jobIDs = append(tc.jobIDs, jobID) file := "metrics/input/" + jobName + ".nomad" - allocs := e2eutil.RegisterAndWaitForAllocs(t, tc.Nomad(), file, jobID) + allocs := e2eutil.RegisterAndWaitForAllocs(t, tc.Nomad(), file, jobID, "") if len(allocs) == 0 { t.Fatalf("failed to register %s", jobID) } diff --git a/e2e/metrics/prometheus.go b/e2e/metrics/prometheus.go index cefa6a5e5cd..d6e25d8ea94 100644 --- a/e2e/metrics/prometheus.go +++ b/e2e/metrics/prometheus.go @@ -17,7 +17,7 @@ func (tc *MetricsTest) setUpPrometheus(f *framework.F) error { uuid := uuid.Generate() fabioID := "fabio" + uuid[0:8] fabioAllocs := e2eutil.RegisterAndWaitForAllocs(f.T(), tc.Nomad(), - "fabio/fabio.nomad", fabioID) + "fabio/fabio.nomad", fabioID, "") if len(fabioAllocs) < 1 { return fmt.Errorf("fabio failed to start") } @@ -36,7 +36,7 @@ func (tc *MetricsTest) setUpPrometheus(f *framework.F) error { tc.fabioAddress = fmt.Sprintf("http://%s:9999", publicIP) prometheusID := "prometheus" + uuid[0:8] prometheusAllocs := e2eutil.RegisterAndWaitForAllocs(f.T(), tc.Nomad(), - "prometheus/prometheus.nomad", prometheusID) + "prometheus/prometheus.nomad", prometheusID, "") if len(prometheusAllocs) < 1 { return fmt.Errorf("prometheus failed to start") } diff --git a/e2e/nomad09upgrade/upgrade.go b/e2e/nomad09upgrade/upgrade.go index e48e84eb702..bf0ed6eacfd 100644 --- a/e2e/nomad09upgrade/upgrade.go +++ b/e2e/nomad09upgrade/upgrade.go @@ -160,7 +160,7 @@ func (tc *UpgradePathTC) testUpgradeForJob(t *testing.T, ver string, jobfile str // Register a sleep job jobID := "sleep-" + uuid.Generate()[:8] t.Logf("registering exec job with id %s", jobID) - e2eutil.RegisterAndWaitForAllocs(t, client, jobfile, jobID) + e2eutil.RegisterAndWaitForAllocs(t, client, jobfile, jobID, "") allocs, _, err := client.Jobs().Allocations(jobID, false, nil) require.NoError(err) require.Len(allocs, 1) diff --git a/e2e/nomadexec/exec.go b/e2e/nomadexec/exec.go index 6877f1b3aa0..341ee83c754 100644 --- a/e2e/nomadexec/exec.go +++ b/e2e/nomadexec/exec.go @@ -57,7 +57,7 @@ func (tc *NomadExecE2ETest) BeforeAll(f *framework.F) { // register a job for execing into tc.jobID = "nomad-exec" + uuid.Generate()[:8] - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), tc.Nomad(), tc.jobFilePath, tc.jobID) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), tc.Nomad(), tc.jobFilePath, tc.jobID, "") f.Len(allocs, 1) e2eutil.WaitForAllocRunning(f.T(), tc.Nomad(), allocs[0].ID) diff --git a/e2e/spread/spread.go b/e2e/spread/spread.go index 628807ff2cb..e765a6c5c3c 100644 --- a/e2e/spread/spread.go +++ b/e2e/spread/spread.go @@ -34,7 +34,7 @@ func (tc *SpreadTest) TestEvenSpread(f *framework.F) { uuid := uuid.Generate() jobId := "spread" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "spread/input/even_spread.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "spread/input/even_spread.nomad", jobId, "") jobAllocs := nomadClient.Allocations() dcToAllocs := make(map[string]int) @@ -61,7 +61,7 @@ func (tc *SpreadTest) TestMultipleSpreads(f *framework.F) { uuid := uuid.Generate() jobId := "spread" + uuid[0:8] tc.jobIds = append(tc.jobIds, jobId) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "spread/input/multiple_spread.nomad", jobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, "spread/input/multiple_spread.nomad", jobId, "") jobAllocs := nomadClient.Allocations() dcToAllocs := make(map[string]int) diff --git a/e2e/taskevents/taskevents.go b/e2e/taskevents/taskevents.go index 65f3698b52e..7abb77420ca 100644 --- a/e2e/taskevents/taskevents.go +++ b/e2e/taskevents/taskevents.go @@ -67,7 +67,7 @@ func (tc *TaskEventsTest) waitUntilEvents(f *framework.F, jobName string, numEve tc.jobIds = append(tc.jobIds, uniqJobId) jobFile := fmt.Sprintf("taskevents/input/%s.nomad", jobName) - allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, jobFile, uniqJobId) + allocs := e2eutil.RegisterAndWaitForAllocs(f.T(), nomadClient, jobFile, uniqJobId, "") require.Len(t, allocs, 1) allocID := allocs[0].ID