From 10cb924b2cbf82d4f7947aafe99866355e0b9a14 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 31 Jan 2017 16:43:57 -0800 Subject: [PATCH 01/38] Refactor Consul Syncer into new ServiceClient Fixes #2478 #2474 #1995 #2294 The new client only handles agent and task service advertisement. Server discovery is mostly unchanged. The Nomad client agent now handles all Consul operations instead of the executor handling task related operations. When upgrading from an earlier version of Nomad existing executors will be told to deregister from Consul so that the Nomad agent can re-register the task's services and checks. Drivers - other than qemu - now support an Exec method for executing abritrary commands in a task's environment. This is used to implement script checks. Interfaces are used extensively to avoid interacting with Consul in tests that don't assert any Consul related behavior. --- client/alloc_runner.go | 36 +- client/alloc_runner_test.go | 10 +- client/allocdir/alloc_dir.go | 8 +- client/client.go | 77 +- client/client_test.go | 30 +- client/consul.go | 13 + client/consul_test.go | 56 + client/driver/docker.go | 50 +- client/driver/driver_test.go | 13 + client/driver/exec.go | 18 +- client/driver/exec_test.go | 64 ++ client/driver/executor/checks.go | 205 ---- client/driver/executor/checks_linux_test.go | 56 - client/driver/executor/checks_test.go | 96 -- client/driver/executor/checks_unix.go | 18 - client/driver/executor/checks_windows.go | 8 - client/driver/executor/executor.go | 148 +-- client/driver/executor/executor_test.go | 26 - client/driver/executor_plugin.go | 16 +- client/driver/java.go | 21 +- client/driver/mock_driver.go | 6 + client/driver/qemu.go | 12 - client/driver/raw_exec.go | 15 +- client/driver/raw_exec_test.go | 62 + client/driver/rkt.go | 60 +- client/driver/rkt_test.go | 74 ++ client/driver/utils.go | 57 +- client/driver/utils_unix.go | 8 + client/task_runner.go | 72 +- client/task_runner_test.go | 5 +- command/agent/agent.go | 102 +- command/agent/consul/chaos_test.go | 193 ---- command/agent/consul/check.go | 91 -- command/agent/consul/client.go | 636 +++++++++++ command/agent/consul/int_test.go | 228 ++++ command/agent/consul/mock.go | 27 + command/agent/consul/script.go | 136 +++ command/agent/consul/script_test.go | 165 +++ command/agent/consul/syncer.go | 1016 ----------------- command/agent/consul/syncer_test.go | 358 ------ command/agent/consul/unit_test.go | 497 ++++++++ nomad/server.go | 70 +- nomad/server_test.go | 8 +- .../docs/job-specification/service.html.md | 2 +- 44 files changed, 2360 insertions(+), 2509 deletions(-) create mode 100644 client/consul.go create mode 100644 client/consul_test.go delete mode 100644 client/driver/executor/checks.go delete mode 100644 client/driver/executor/checks_linux_test.go delete mode 100644 client/driver/executor/checks_test.go delete mode 100644 client/driver/executor/checks_unix.go delete mode 100644 client/driver/executor/checks_windows.go delete mode 100644 command/agent/consul/chaos_test.go delete mode 100644 command/agent/consul/check.go create mode 100644 command/agent/consul/client.go create mode 100644 command/agent/consul/int_test.go create mode 100644 command/agent/consul/mock.go create mode 100644 command/agent/consul/script.go create mode 100644 command/agent/consul/script_test.go delete mode 100644 command/agent/consul/syncer.go delete mode 100644 command/agent/consul/syncer_test.go create mode 100644 command/agent/consul/unit_test.go diff --git a/client/alloc_runner.go b/client/alloc_runner.go index 719a84d4a671..8ca6f671eae1 100644 --- a/client/alloc_runner.go +++ b/client/alloc_runner.go @@ -59,7 +59,8 @@ type AllocRunner struct { updateCh chan *structs.Allocation - vaultClient vaultclient.VaultClient + vaultClient vaultclient.VaultClient + consulClient ConsulServiceAPI otherAllocDir *allocdir.AllocDir @@ -96,20 +97,23 @@ type allocRunnerState struct { // NewAllocRunner is used to create a new allocation context func NewAllocRunner(logger *log.Logger, config *config.Config, updater AllocStateUpdater, - alloc *structs.Allocation, vaultClient vaultclient.VaultClient) *AllocRunner { + alloc *structs.Allocation, vaultClient vaultclient.VaultClient, + consulClient ConsulServiceAPI) *AllocRunner { + ar := &AllocRunner{ - config: config, - updater: updater, - logger: logger, - alloc: alloc, - dirtyCh: make(chan struct{}, 1), - tasks: make(map[string]*TaskRunner), - taskStates: copyTaskStates(alloc.TaskStates), - restored: make(map[string]struct{}), - updateCh: make(chan *structs.Allocation, 64), - destroyCh: make(chan struct{}), - waitCh: make(chan struct{}), - vaultClient: vaultClient, + config: config, + updater: updater, + logger: logger, + alloc: alloc, + dirtyCh: make(chan struct{}, 1), + tasks: make(map[string]*TaskRunner), + taskStates: copyTaskStates(alloc.TaskStates), + restored: make(map[string]struct{}), + updateCh: make(chan *structs.Allocation, 64), + destroyCh: make(chan struct{}), + waitCh: make(chan struct{}), + vaultClient: vaultClient, + consulClient: consulClient, } return ar } @@ -174,7 +178,7 @@ func (r *AllocRunner) RestoreState() error { } task := &structs.Task{Name: name} - tr := NewTaskRunner(r.logger, r.config, r.setTaskState, td, r.Alloc(), task, r.vaultClient) + tr := NewTaskRunner(r.logger, r.config, r.setTaskState, td, r.Alloc(), task, r.vaultClient, r.consulClient) r.tasks[name] = tr // Skip tasks in terminal states. @@ -512,7 +516,7 @@ func (r *AllocRunner) Run() { taskdir := r.allocDir.NewTaskDir(task.Name) r.allocDirLock.Unlock() - tr := NewTaskRunner(r.logger, r.config, r.setTaskState, taskdir, r.Alloc(), task.Copy(), r.vaultClient) + tr := NewTaskRunner(r.logger, r.config, r.setTaskState, taskdir, r.Alloc(), task.Copy(), r.vaultClient, r.consulClient) r.tasks[task.Name] = tr tr.MarkReceived() diff --git a/client/alloc_runner_test.go b/client/alloc_runner_test.go index 6bac5e4051b9..f1bc828b8072 100644 --- a/client/alloc_runner_test.go +++ b/client/alloc_runner_test.go @@ -40,7 +40,7 @@ func testAllocRunnerFromAlloc(alloc *structs.Allocation, restarts bool) (*MockAl alloc.Job.Type = structs.JobTypeBatch } vclient := vaultclient.NewMockVaultClient() - ar := NewAllocRunner(logger, conf, upd.Update, alloc, vclient) + ar := NewAllocRunner(logger, conf, upd.Update, alloc, vclient, newMockConsulServiceClient()) return upd, ar } @@ -323,7 +323,8 @@ func TestAllocRunner_SaveRestoreState(t *testing.T) { // Create a new alloc runner l2 := prefixedTestLogger("----- ar2: ") ar2 := NewAllocRunner(l2, ar.config, upd.Update, - &structs.Allocation{ID: ar.alloc.ID}, ar.vaultClient) + &structs.Allocation{ID: ar.alloc.ID}, ar.vaultClient, + ar.consulClient) err = ar2.RestoreState() if err != nil { t.Fatalf("err: %v", err) @@ -415,7 +416,7 @@ func TestAllocRunner_SaveRestoreState_TerminalAlloc(t *testing.T) { // Create a new alloc runner ar2 := NewAllocRunner(ar.logger, ar.config, upd.Update, - &structs.Allocation{ID: ar.alloc.ID}, ar.vaultClient) + &structs.Allocation{ID: ar.alloc.ID}, ar.vaultClient, ar.consulClient) ar2.logger = prefixedTestLogger("ar2: ") err = ar2.RestoreState() if err != nil { @@ -573,7 +574,8 @@ func TestAllocRunner_RestoreOldState(t *testing.T) { *alloc.Job.LookupTaskGroup(alloc.TaskGroup).RestartPolicy = structs.RestartPolicy{Attempts: 0} alloc.Job.Type = structs.JobTypeBatch vclient := vaultclient.NewMockVaultClient() - ar := NewAllocRunner(logger, conf, upd.Update, alloc, vclient) + cclient := newMockConsulServiceClient() + ar := NewAllocRunner(logger, conf, upd.Update, alloc, vclient, cclient) defer ar.Destroy() // RestoreState should fail on the task state since we only test the diff --git a/client/allocdir/alloc_dir.go b/client/allocdir/alloc_dir.go index dd665926112a..749f5da17932 100644 --- a/client/allocdir/alloc_dir.go +++ b/client/allocdir/alloc_dir.go @@ -34,8 +34,12 @@ var ( // included in snapshots. SharedDataDir = "data" + // TmpDirName is the name of the temporary directory in each alloc and + // task. + TmpDirName = "tmp" + // The set of directories that exist inside eache shared alloc directory. - SharedAllocDirs = []string{LogDirName, "tmp", SharedDataDir} + SharedAllocDirs = []string{LogDirName, TmpDirName, SharedDataDir} // The name of the directory that exists inside each task directory // regardless of driver. @@ -46,7 +50,7 @@ var ( TaskSecrets = "secrets" // TaskDirs is the set of directories created in each tasks directory. - TaskDirs = map[string]os.FileMode{"tmp": os.ModeSticky | 0777} + TaskDirs = map[string]os.FileMode{TmpDirName: os.ModeSticky | 0777} ) type AllocDir struct { diff --git a/client/client.go b/client/client.go index 7183c939f1df..89e14823b85f 100644 --- a/client/client.go +++ b/client/client.go @@ -49,10 +49,6 @@ const ( // datacenters looking for the Nomad server service. datacenterQueryLimit = 9 - // consulReaperIntv is the interval at which the Consul reaper will - // run. - consulReaperIntv = 5 * time.Second - // registerRetryIntv is minimum interval on which we retry // registration. We pick a value between this and 2x this. registerRetryIntv = 15 * time.Second @@ -142,8 +138,12 @@ type Client struct { // allocUpdates stores allocations that need to be synced to the server. allocUpdates chan *structs.Allocation - // consulSyncer advertises this Nomad Agent with Consul - consulSyncer *consul.Syncer + // consulService is Nomad's custom Consul client for managing services + // and checks. + consulService ConsulServiceAPI + + // consulCatalog is the subset of Consul's Catalog API Nomad uses. + consulCatalog consul.CatalogAPI // HostStatsCollector collects host resource usage stats hostStatsCollector *stats.HostStatsCollector @@ -196,7 +196,7 @@ var ( ) // NewClient is used to create a new client from the given configuration -func NewClient(cfg *config.Config, consulSyncer *consul.Syncer, logger *log.Logger) (*Client, error) { +func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulService ConsulServiceAPI, logger *log.Logger) (*Client, error) { // Create the tls wrapper var tlsWrap tlsutil.RegionWrapper if cfg.TLSConfig.EnableRPC { @@ -210,7 +210,8 @@ func NewClient(cfg *config.Config, consulSyncer *consul.Syncer, logger *log.Logg // Create the client c := &Client{ config: cfg, - consulSyncer: consulSyncer, + consulCatalog: consulCatalog, + consulService: consulService, start: time.Now(), connPool: nomad.NewPool(cfg.LogOutput, clientRPCCache, clientMaxStreams, tlsWrap), logger: logger, @@ -285,9 +286,6 @@ func NewClient(cfg *config.Config, consulSyncer *consul.Syncer, logger *log.Logg } } - // Start Consul reaper - go c.consulReaper() - // 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) @@ -606,7 +604,7 @@ func (c *Client) restoreState() error { id := entry.Name() alloc := &structs.Allocation{ID: id} c.configLock.RLock() - ar := NewAllocRunner(c.logger, c.configCopy, c.updateAllocStatus, alloc, c.vaultClient) + ar := NewAllocRunner(c.logger, c.configCopy, c.updateAllocStatus, alloc, c.vaultClient, c.consulService) c.configLock.RUnlock() c.allocLock.Lock() c.allocs[id] = ar @@ -1894,7 +1892,7 @@ func (c *Client) addAlloc(alloc *structs.Allocation, prevAllocDir *allocdir.Allo defer c.allocLock.Unlock() c.configLock.RLock() - ar := NewAllocRunner(c.logger, c.configCopy, c.updateAllocStatus, alloc, c.vaultClient) + ar := NewAllocRunner(c.logger, c.configCopy, c.updateAllocStatus, alloc, c.vaultClient, c.consulService) ar.SetPreviousAllocDir(prevAllocDir) c.configLock.RUnlock() go ar.Run() @@ -2047,8 +2045,7 @@ func (c *Client) consulDiscoveryImpl() error { c.heartbeatLock.Lock() defer c.heartbeatLock.Unlock() - consulCatalog := c.consulSyncer.ConsulClient().Catalog() - dcs, err := consulCatalog.Datacenters() + dcs, err := c.consulCatalog.Datacenters() if err != nil { return fmt.Errorf("client.consul: unable to query Consul datacenters: %v", err) } @@ -2084,7 +2081,7 @@ DISCOLOOP: Near: "_agent", WaitTime: consul.DefaultQueryWaitDuration, } - consulServices, _, err := consulCatalog.Service(serviceName, consul.ServiceTagRPC, consulOpts) + consulServices, _, err := c.consulCatalog.Service(serviceName, consul.ServiceTagRPC, consulOpts) if err != nil { mErr.Errors = append(mErr.Errors, fmt.Errorf("unable to query service %+q from Consul datacenter %+q: %v", serviceName, dc, err)) continue @@ -2143,54 +2140,6 @@ DISCOLOOP: } } -// consulReaper periodically reaps unmatched domains from Consul. Intended to -// be called in its own goroutine. See consulReaperIntv for interval. -func (c *Client) consulReaper() { - ticker := time.NewTicker(consulReaperIntv) - defer ticker.Stop() - lastok := true - for { - select { - case <-ticker.C: - if err := c.consulReaperImpl(); err != nil { - if lastok { - c.logger.Printf("[ERR] client.consul: error reaping services in consul: %v", err) - lastok = false - } - } else { - lastok = true - } - case <-c.shutdownCh: - return - } - } -} - -// consulReaperImpl reaps unmatched domains from Consul. -func (c *Client) consulReaperImpl() error { - const estInitialExecutorDomains = 8 - - // Create the domains to keep and add the server and client - domains := make([]consul.ServiceDomain, 2, estInitialExecutorDomains) - domains[0] = consul.ServerDomain - domains[1] = consul.ClientDomain - - for allocID, ar := range c.getAllocRunners() { - ar.taskStatusLock.RLock() - taskStates := copyTaskStates(ar.taskStates) - ar.taskStatusLock.RUnlock() - for taskName, taskState := range taskStates { - // Only keep running tasks - if taskState.State == structs.TaskStateRunning { - d := consul.NewExecutorDomain(allocID, taskName) - domains = append(domains, d) - } - } - } - - return c.consulSyncer.ReapUnmatched(domains) -} - // emitStats collects host resource usage stats periodically func (c *Client) emitStats() { // Start collecting host stats right away and then keep collecting every diff --git a/client/client_test.go b/client/client_test.go index 0e79c77e788c..abf857da4fc7 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -75,15 +75,11 @@ func testServer(t *testing.T, cb func(*nomad.Config)) (*nomad.Server, string) { cb(config) } - shutdownCh := make(chan struct{}) logger := log.New(config.LogOutput, "", log.LstdFlags) - consulSyncer, err := consul.NewSyncer(config.ConsulConfig, shutdownCh, logger) - if err != nil { - t.Fatalf("err: %v", err) - } + catalog := consul.NewMockCatalog(logger) // Create server - server, err := nomad.NewServer(config, consulSyncer, logger) + server, err := nomad.NewServer(config, catalog, logger) if err != nil { t.Fatalf("err: %v", err) } @@ -105,14 +101,11 @@ func testClient(t *testing.T, cb func(c *config.Config)) *Client { cb(conf) } - shutdownCh := make(chan struct{}) - consulSyncer, err := consul.NewSyncer(conf.ConsulConfig, shutdownCh, log.New(os.Stderr, "", log.LstdFlags)) - if err != nil { - t.Fatalf("err: %v", err) - } - logger := log.New(conf.LogOutput, "", log.LstdFlags) - client, err := NewClient(conf, consulSyncer, logger) + catalog := consul.NewMockCatalog(logger) + mockService := newMockConsulServiceClient() + mockService.logger = logger + client, err := NewClient(conf, catalog, mockService, logger) if err != nil { t.Fatalf("err: %v", err) } @@ -754,14 +747,11 @@ func TestClient_SaveRestoreState(t *testing.T) { } // Create a new client - shutdownCh := make(chan struct{}) logger := log.New(c1.config.LogOutput, "", log.LstdFlags) - consulSyncer, err := consul.NewSyncer(c1.config.ConsulConfig, shutdownCh, logger) - if err != nil { - t.Fatalf("err: %v", err) - } - - c2, err := NewClient(c1.config, consulSyncer, logger) + catalog := consul.NewMockCatalog(logger) + mockService := newMockConsulServiceClient() + mockService.logger = logger + c2, err := NewClient(c1.config, catalog, mockService, logger) if err != nil { t.Fatalf("err: %v", err) } diff --git a/client/consul.go b/client/consul.go new file mode 100644 index 000000000000..41da0abebfb9 --- /dev/null +++ b/client/consul.go @@ -0,0 +1,13 @@ +package client + +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 { + RegisterTask(allocID string, task *structs.Task, exec consul.ScriptExecutor) error + RemoveTask(allocID string, task *structs.Task) +} diff --git a/client/consul_test.go b/client/consul_test.go new file mode 100644 index 000000000000..f37aec3f9adc --- /dev/null +++ b/client/consul_test.go @@ -0,0 +1,56 @@ +package client + +import ( + "io/ioutil" + "log" + "os" + "sync" + "testing" + + "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/nomad/structs" +) + +// mockConsulOp represents the register/deregister operations. +type mockConsulOp struct { + allocID string + task *structs.Task + exec consul.ScriptExecutor +} + +// mockConsulServiceClient implements the ConsulServiceAPI interface to record +// and log task registration/deregistration. +type mockConsulServiceClient struct { + registers []mockConsulOp + removes []mockConsulOp + mu sync.Mutex + + logger *log.Logger +} + +func newMockConsulServiceClient() *mockConsulServiceClient { + m := mockConsulServiceClient{ + registers: make([]mockConsulOp, 0, 10), + removes: make([]mockConsulOp, 0, 10), + logger: log.New(ioutil.Discard, "", 0), + } + if testing.Verbose() { + m.logger = log.New(os.Stderr, "", log.LstdFlags) + } + return &m +} + +func (m *mockConsulServiceClient) RegisterTask(allocID string, task *structs.Task, exec consul.ScriptExecutor) error { + m.logger.Printf("[TEST] mock_consul: RegisterTask(%q, %q, %T)", allocID, task.Name, exec) + m.mu.Lock() + defer m.mu.Unlock() + m.registers = append(m.registers, mockConsulOp{allocID, task, exec}) + return nil +} + +func (m *mockConsulServiceClient) RemoveTask(allocID string, task *structs.Task) { + m.logger.Printf("[TEST] mock_consul: RemoveTask(%q, %q)", allocID, task.Name) + m.mu.Lock() + defer m.mu.Unlock() + m.removes = append(m.removes, mockConsulOp{allocID, task, nil}) +} diff --git a/client/driver/docker.go b/client/driver/docker.go index b045e3f3bbab..171c1ed6df64 100644 --- a/client/driver/docker.go +++ b/client/driver/docker.go @@ -1,6 +1,7 @@ package driver import ( + "context" "encoding/json" "fmt" "log" @@ -14,6 +15,7 @@ import ( "syscall" "time" + "github.com/armon/circbuf" docker "github.com/fsouza/go-dockerclient" "github.com/docker/docker/cli/config/configfile" @@ -564,9 +566,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle doneCh: make(chan bool), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := exec.SyncServices(consulContext(d.config, container.ID)); err != nil { - d.logger.Printf("[ERR] driver.docker: error registering services with consul for task: %q: %v", task.Name, err) - } go h.collectStats() go h.run() return h, nil @@ -1227,10 +1226,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er doneCh: make(chan bool), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := exec.SyncServices(consulContext(d.config, pid.ContainerID)); err != nil { - h.logger.Printf("[ERR] driver.docker: error registering services with consul: %v", err) - } - go h.collectStats() go h.run() return h, nil @@ -1273,6 +1268,42 @@ func (h *DockerHandle) Update(task *structs.Task) error { return nil } +func (h *DockerHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + fullCmd := make([]string, len(args)+1) + fullCmd[0] = cmd + copy(fullCmd[1:], args) + createExecOpts := docker.CreateExecOptions{ + AttachStdin: false, + AttachStdout: true, + AttachStderr: true, + Tty: false, + Cmd: fullCmd, + Container: h.containerID, + Context: ctx, + } + exec, err := h.client.CreateExec(createExecOpts) + if err != nil { + return nil, 0, err + } + + output, _ := circbuf.NewBuffer(int64(dstructs.CheckBufSize)) + startOpts := docker.StartExecOptions{ + Detach: false, + Tty: false, + OutputStream: output, + ErrorStream: output, + Context: ctx, + } + if err := client.StartExec(exec.ID, startOpts); err != nil { + return nil, 0, err + } + res, err := client.InspectExec(exec.ID) + if err != nil { + return output.Bytes(), 0, err + } + return output.Bytes(), res.ExitCode, nil +} + func (h *DockerHandle) Signal(s os.Signal) error { // Convert types sysSig, ok := s.(syscall.Signal) @@ -1332,11 +1363,6 @@ func (h *DockerHandle) run() { close(h.doneCh) - // Remove services - if err := h.executor.DeregisterServices(); err != nil { - h.logger.Printf("[ERR] driver.docker: error deregistering services: %v", err) - } - // Shutdown the syslog collector if err := h.executor.Exit(); err != nil { h.logger.Printf("[ERR] driver.docker: failed to kill the syslog collector: %v", err) diff --git a/client/driver/driver_test.go b/client/driver/driver_test.go index 8cd44b331ae9..b186efac8c16 100644 --- a/client/driver/driver_test.go +++ b/client/driver/driver_test.go @@ -13,6 +13,7 @@ import ( "github.com/hashicorp/nomad/client/allocdir" "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/driver/env" + "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/testtask" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" @@ -421,3 +422,15 @@ func TestCreatedResources_CopyRemove(t *testing.T) { t.Fatalf("res1 should not equal res2: #%v", res1) } } + +// TestHandleExec statically asserts the drivers we expect to implement the +// consul.Executor interface do. +func TestHandleScriptExecutor(t *testing.T) { + _ = []consul.ScriptExecutor{ + &DockerHandle{}, + &execHandle{}, + &javaHandle{}, + &rawExecHandle{}, + &rktHandle{}, + } +} diff --git a/client/driver/exec.go b/client/driver/exec.go index d94c82443fff..7da657e1ed3e 100644 --- a/client/driver/exec.go +++ b/client/driver/exec.go @@ -1,6 +1,7 @@ package driver import ( + "context" "encoding/json" "fmt" "log" @@ -163,9 +164,7 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, version: d.config.Version, doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), - } - if err := exec.SyncServices(consulContext(d.config, "")); err != nil { - d.logger.Printf("[ERR] driver.exec: error registering services with consul for task: %q: %v", task.Name, err) + taskDir: ctx.TaskDir, } go h.run() return h, nil @@ -222,9 +221,7 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro maxKillTimeout: id.MaxKillTimeout, doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), - } - if err := exec.SyncServices(consulContext(d.config, "")); err != nil { - d.logger.Printf("[ERR] driver.exec: error registering services with consul: %v", err) + taskDir: ctx.TaskDir, } go h.run() return h, nil @@ -260,6 +257,10 @@ func (h *execHandle) Update(task *structs.Task) error { return nil } +func (h *execHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + return execChroot(ctx, h.taskDir.Dir, cmd, args) +} + func (h *execHandle) Signal(s os.Signal) error { return h.executor.Signal(s) } @@ -307,11 +308,6 @@ func (h *execHandle) run() { } } - // Remove services - if err := h.executor.DeregisterServices(); err != nil { - h.logger.Printf("[ERR] driver.exec: failed to deregister services: %v", err) - } - // Exit the executor if err := h.executor.Exit(); err != nil { h.logger.Printf("[ERR] driver.exec: error destroying executor: %v", err) diff --git a/client/driver/exec_test.go b/client/driver/exec_test.go index 85976df1aa78..2f35efa8dd6d 100644 --- a/client/driver/exec_test.go +++ b/client/driver/exec_test.go @@ -1,6 +1,8 @@ package driver import ( + "bytes" + "context" "fmt" "io/ioutil" "path/filepath" @@ -11,6 +13,7 @@ import ( "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/driver/env" + "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -280,3 +283,64 @@ func TestExecDriverUser(t *testing.T) { t.Fatalf("Expecting '%v' in '%v'", msg, err) } } + +// TestExecDriver_HandlerExec ensures the exec driver's handle properly executes commands inside the chroot. +func TestExecDriver_HandlerExec(t *testing.T) { + ctestutils.ExecCompatible(t) + task := &structs.Task{ + Name: "sleep", + Driver: "exec", + Config: map[string]interface{}{ + "command": "/bin/sleep", + "args": []string{"9000"}, + }, + LogConfig: &structs.LogConfig{ + MaxFiles: 10, + MaxFileSizeMB: 10, + }, + Resources: basicResources, + } + + ctx := testDriverContexts(t, task) + defer ctx.AllocDir.Destroy() + d := NewExecDriver(ctx.DriverCtx) + + if _, err := d.Prestart(ctx.ExecCtx, task); err != nil { + t.Fatalf("prestart err: %v", err) + } + handle, err := d.Start(ctx.ExecCtx, task) + if err != nil { + t.Fatalf("err: %v", err) + } + if handle == nil { + t.Fatalf("missing handle") + } + + // Exec a command that should work + out, code, err := handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"/alloc"}) + if err != nil { + t.Fatalf("error exec'ing stat: %v", err) + } + if code != 0 { + t.Fatalf("expected `stat /alloc` to succeed but exit code was: %d", code) + } + if expected := 100; len(out) < expected { + t.Fatalf("expected at least %d bytes of output but found %d:\n%s", expected, len(out), out) + } + + // Exec a command that should fail + out, code, err = handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"lkjhdsaflkjshowaisxmcvnlia"}) + if err != nil { + t.Fatalf("error exec'ing stat: %v", err) + } + if code == 0 { + t.Fatalf("expected `stat` to fail but exit code was: %d", code) + } + if expected := "No such file or directory"; !bytes.Contains(out, []byte(expected)) { + t.Fatalf("expected output to contain %q but found: %q", expected, out) + } + + if err := handle.Kill(); err != nil { + t.Fatalf("error killing exec handle: %v", err) + } +} diff --git a/client/driver/executor/checks.go b/client/driver/executor/checks.go deleted file mode 100644 index de93146ccdf4..000000000000 --- a/client/driver/executor/checks.go +++ /dev/null @@ -1,205 +0,0 @@ -package executor - -import ( - "fmt" - "log" - "os/exec" - "sync" - "syscall" - "time" - - "github.com/armon/circbuf" - docker "github.com/fsouza/go-dockerclient" - cstructs "github.com/hashicorp/nomad/client/driver/structs" -) - -var ( - // We store the client globally to cache the connection to the docker daemon. - createClient sync.Once - client *docker.Client -) - -const ( - // The default check timeout - defaultCheckTimeout = 30 * time.Second -) - -// DockerScriptCheck runs nagios compatible scripts in a docker container and -// provides the check result -type DockerScriptCheck struct { - id string // id of the check - interval time.Duration // interval of the check - timeout time.Duration // timeout of the check - containerID string // container id in which the check will be invoked - logger *log.Logger - cmd string // check command - args []string // check command arguments - - dockerEndpoint string // docker endpoint - tlsCert string // path to tls certificate - tlsCa string // path to tls ca - tlsKey string // path to tls key -} - -// dockerClient creates the client to interact with the docker daemon -func (d *DockerScriptCheck) dockerClient() (*docker.Client, error) { - if client != nil { - return client, nil - } - - var err error - createClient.Do(func() { - if d.dockerEndpoint != "" { - if d.tlsCert+d.tlsKey+d.tlsCa != "" { - d.logger.Printf("[DEBUG] executor.checks: using TLS client connection to %s", d.dockerEndpoint) - client, err = docker.NewTLSClient(d.dockerEndpoint, d.tlsCert, d.tlsKey, d.tlsCa) - } else { - d.logger.Printf("[DEBUG] executor.checks: using standard client connection to %s", d.dockerEndpoint) - client, err = docker.NewClient(d.dockerEndpoint) - } - return - } - - d.logger.Println("[DEBUG] executor.checks: using client connection initialized from environment") - client, err = docker.NewClientFromEnv() - }) - return client, err -} - -// Run runs a script check inside a docker container -func (d *DockerScriptCheck) Run() *cstructs.CheckResult { - var ( - exec *docker.Exec - err error - execRes *docker.ExecInspect - time = time.Now() - ) - - if client, err = d.dockerClient(); err != nil { - return &cstructs.CheckResult{Err: err} - } - execOpts := docker.CreateExecOptions{ - AttachStdin: false, - AttachStdout: true, - AttachStderr: true, - Tty: false, - Cmd: append([]string{d.cmd}, d.args...), - Container: d.containerID, - } - if exec, err = client.CreateExec(execOpts); err != nil { - return &cstructs.CheckResult{Err: err} - } - - output, _ := circbuf.NewBuffer(int64(cstructs.CheckBufSize)) - startOpts := docker.StartExecOptions{ - Detach: false, - Tty: false, - OutputStream: output, - ErrorStream: output, - } - - if err = client.StartExec(exec.ID, startOpts); err != nil { - return &cstructs.CheckResult{Err: err} - } - if execRes, err = client.InspectExec(exec.ID); err != nil { - return &cstructs.CheckResult{Err: err} - } - return &cstructs.CheckResult{ - ExitCode: execRes.ExitCode, - Output: string(output.Bytes()), - Timestamp: time, - } -} - -// ID returns the check id -func (d *DockerScriptCheck) ID() string { - return d.id -} - -// Interval returns the interval at which the check has to run -func (d *DockerScriptCheck) Interval() time.Duration { - return d.interval -} - -// Timeout returns the duration after which a check is timed out. -func (d *DockerScriptCheck) Timeout() time.Duration { - if d.timeout == 0 { - return defaultCheckTimeout - } - return d.timeout -} - -// ExecScriptCheck runs a nagios compatible script and returns the check result -type ExecScriptCheck struct { - id string // id of the script check - interval time.Duration // interval at which the check is invoked - timeout time.Duration // timeout duration of the check - cmd string // command of the check - args []string // args passed to the check - taskDir string // the root directory of the check - - FSIsolation bool // indicates whether the check has to be run within a chroot -} - -// Run runs an exec script check -func (e *ExecScriptCheck) Run() *cstructs.CheckResult { - buf, _ := circbuf.NewBuffer(int64(cstructs.CheckBufSize)) - cmd := exec.Command(e.cmd, e.args...) - cmd.Stdout = buf - cmd.Stderr = buf - e.setChroot(cmd) - ts := time.Now() - if err := cmd.Start(); err != nil { - return &cstructs.CheckResult{Err: err} - } - errCh := make(chan error, 2) - go func() { - errCh <- cmd.Wait() - }() - - select { - case err := <-errCh: - endTime := time.Now() - if err == nil { - return &cstructs.CheckResult{ - ExitCode: 0, - Output: string(buf.Bytes()), - Timestamp: ts, - } - } - exitCode := 1 - if exitErr, ok := err.(*exec.ExitError); ok { - if status, ok := exitErr.Sys().(syscall.WaitStatus); ok { - exitCode = status.ExitStatus() - } - } - return &cstructs.CheckResult{ - ExitCode: exitCode, - Output: string(buf.Bytes()), - Timestamp: ts, - Duration: endTime.Sub(ts), - } - case <-time.After(e.Timeout()): - errCh <- fmt.Errorf("timed out after waiting 30s") - } - - return nil -} - -// ID returns the check id -func (e *ExecScriptCheck) ID() string { - return e.id -} - -// Interval returns the interval at which the check has to run -func (e *ExecScriptCheck) Interval() time.Duration { - return e.interval -} - -// Timeout returns the duration after which a check is timed out. -func (e *ExecScriptCheck) Timeout() time.Duration { - if e.timeout == 0 { - return defaultCheckTimeout - } - return e.timeout -} diff --git a/client/driver/executor/checks_linux_test.go b/client/driver/executor/checks_linux_test.go deleted file mode 100644 index 3affd0e08f37..000000000000 --- a/client/driver/executor/checks_linux_test.go +++ /dev/null @@ -1,56 +0,0 @@ -package executor - -import ( - "log" - "os" - "strings" - "testing" - - dstructs "github.com/hashicorp/nomad/client/driver/structs" - "github.com/hashicorp/nomad/client/testutil" -) - -func TestExecScriptCheckWithIsolation(t *testing.T) { - testutil.ExecCompatible(t) - - execCmd := ExecCommand{Cmd: "/bin/echo", Args: []string{"hello world"}} - ctx, allocDir := testExecutorContextWithChroot(t) - defer allocDir.Destroy() - - execCmd.FSIsolation = true - execCmd.ResourceLimits = true - execCmd.User = dstructs.DefaultUnpriviledgedUser - - executor := NewExecutor(log.New(os.Stdout, "", log.LstdFlags)) - - if err := executor.SetContext(ctx); err != nil { - t.Fatalf("Unexpected error") - } - - _, err := executor.LaunchCmd(&execCmd) - if err != nil { - t.Fatalf("error in launching command: %v", err) - } - - check := &ExecScriptCheck{ - id: "foo", - cmd: "/bin/echo", - args: []string{"hello", "world"}, - taskDir: ctx.TaskDir, - FSIsolation: true, - } - - res := check.Run() - expectedOutput := "hello world" - expectedExitCode := 0 - if res.Err != nil { - t.Fatalf("err: %v", res.Err) - } - if strings.TrimSpace(res.Output) != expectedOutput { - t.Fatalf("output expected: %v, actual: %v", expectedOutput, res.Output) - } - - if res.ExitCode != expectedExitCode { - t.Fatalf("exitcode expected: %v, actual: %v", expectedExitCode, res.ExitCode) - } -} diff --git a/client/driver/executor/checks_test.go b/client/driver/executor/checks_test.go deleted file mode 100644 index 9533026fd713..000000000000 --- a/client/driver/executor/checks_test.go +++ /dev/null @@ -1,96 +0,0 @@ -package executor - -import ( - "log" - "os" - "strings" - "testing" - "time" - - docker "github.com/fsouza/go-dockerclient" - - "github.com/hashicorp/nomad/client/testutil" -) - -func TestExecScriptCheckNoIsolation(t *testing.T) { - check := &ExecScriptCheck{ - id: "foo", - cmd: "/bin/echo", - args: []string{"hello", "world"}, - taskDir: "/tmp", - FSIsolation: false, - } - - res := check.Run() - expectedOutput := "hello world" - expectedExitCode := 0 - if res.Err != nil { - t.Fatalf("err: %v", res.Err) - } - if strings.TrimSpace(res.Output) != expectedOutput { - t.Fatalf("output expected: %v, actual: %v", expectedOutput, res.Output) - } - - if res.ExitCode != expectedExitCode { - t.Fatalf("exitcode expected: %v, actual: %v", expectedExitCode, res.ExitCode) - } -} - -func TestDockerScriptCheck(t *testing.T) { - if !testutil.DockerIsConnected(t) { - return - } - client, err := docker.NewClientFromEnv() - if err != nil { - t.Fatalf("error creating docker client: %v", err) - } - - if err := client.PullImage(docker.PullImageOptions{Repository: "busybox", Tag: "latest"}, - docker.AuthConfiguration{}); err != nil { - t.Fatalf("error pulling redis: %v", err) - } - - container, err := client.CreateContainer(docker.CreateContainerOptions{ - Config: &docker.Config{ - Image: "busybox", - Cmd: []string{"/bin/sleep", "1000"}, - }, - }) - if err != nil { - t.Fatalf("error creating container: %v", err) - } - defer removeContainer(client, container.ID) - - if err := client.StartContainer(container.ID, container.HostConfig); err != nil { - t.Fatalf("error starting container: %v", err) - } - - check := &DockerScriptCheck{ - id: "1", - interval: 5 * time.Second, - containerID: container.ID, - logger: log.New(os.Stdout, "", log.LstdFlags), - cmd: "/bin/echo", - args: []string{"hello", "world"}, - } - - res := check.Run() - expectedOutput := "hello world" - expectedExitCode := 0 - if res.Err != nil { - t.Fatalf("err: %v", res.Err) - } - if strings.TrimSpace(res.Output) != expectedOutput { - t.Fatalf("output expected: %v, actual: %v", expectedOutput, res.Output) - } - - if res.ExitCode != expectedExitCode { - t.Fatalf("exitcode expected: %v, actual: %v", expectedExitCode, res.ExitCode) - } -} - -// removeContainer kills and removes a container -func removeContainer(client *docker.Client, containerID string) { - client.KillContainer(docker.KillContainerOptions{ID: containerID}) - client.RemoveContainer(docker.RemoveContainerOptions{ID: containerID, RemoveVolumes: true, Force: true}) -} diff --git a/client/driver/executor/checks_unix.go b/client/driver/executor/checks_unix.go deleted file mode 100644 index b18812dd8d04..000000000000 --- a/client/driver/executor/checks_unix.go +++ /dev/null @@ -1,18 +0,0 @@ -// +build darwin dragonfly freebsd linux netbsd openbsd solaris - -package executor - -import ( - "os/exec" - "syscall" -) - -func (e *ExecScriptCheck) setChroot(cmd *exec.Cmd) { - if e.FSIsolation { - if cmd.SysProcAttr == nil { - cmd.SysProcAttr = &syscall.SysProcAttr{} - } - cmd.SysProcAttr.Chroot = e.taskDir - } - cmd.Dir = "/" -} diff --git a/client/driver/executor/checks_windows.go b/client/driver/executor/checks_windows.go deleted file mode 100644 index a35c2722d60e..000000000000 --- a/client/driver/executor/checks_windows.go +++ /dev/null @@ -1,8 +0,0 @@ -// +build windows - -package executor - -import "os/exec" - -func (e *ExecScriptCheck) setChroot(cmd *exec.Cmd) { -} diff --git a/client/driver/executor/executor.go b/client/driver/executor/executor.go index 53b9c37f581b..90797fbefa26 100644 --- a/client/driver/executor/executor.go +++ b/client/driver/executor/executor.go @@ -23,10 +23,8 @@ import ( "github.com/hashicorp/nomad/client/driver/env" "github.com/hashicorp/nomad/client/driver/logging" "github.com/hashicorp/nomad/client/stats" - "github.com/hashicorp/nomad/command/agent/consul" shelpers "github.com/hashicorp/nomad/helper/stats" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/nomad/structs/config" dstructs "github.com/hashicorp/nomad/client/driver/structs" cstructs "github.com/hashicorp/nomad/client/structs" @@ -56,38 +54,11 @@ type Executor interface { Exit() error UpdateLogConfig(logConfig *structs.LogConfig) error UpdateTask(task *structs.Task) error - SyncServices(ctx *ConsulContext) error - DeregisterServices() error Version() (*ExecutorVersion, error) Stats() (*cstructs.TaskResourceUsage, error) Signal(s os.Signal) error } -// ConsulContext holds context to configure the Consul client and run checks -type ConsulContext struct { - // ConsulConfig contains the configuration information for talking - // with this Nomad Agent's Consul Agent. - ConsulConfig *config.ConsulConfig - - // ContainerID is the ID of the container - ContainerID string - - // TLSCert is the cert which docker client uses while interactng with the docker - // daemon over TLS - TLSCert string - - // TLSCa is the CA which the docker client uses while interacting with the docker - // daeemon over TLS - TLSCa string - - // TLSKey is the TLS key which the docker client uses while interacting with - // the docker daemon - TLSKey string - - // DockerEndpoint is the endpoint of the docker daemon - DockerEndpoint string -} - // ExecutorContext holds context to configure the command user // wants to run and isolate it type ExecutorContext struct { @@ -196,8 +167,6 @@ type UniversalExecutor struct { resConCtx resourceContainerContext - consulSyncer *consul.Syncer - consulCtx *ConsulContext totalCpuStats *stats.CpuStats userCpuStats *stats.CpuStats systemCpuStats *stats.CpuStats @@ -224,7 +193,7 @@ func NewExecutor(logger *log.Logger) Executor { // Version returns the api version of the executor func (e *UniversalExecutor) Version() (*ExecutorVersion, error) { - return &ExecutorVersion{Version: "1.0.0"}, nil + return &ExecutorVersion{Version: "1.1.0"}, nil } // SetContext is used to set the executors context and should be the first call @@ -377,28 +346,9 @@ func (e *UniversalExecutor) UpdateTask(task *structs.Task) error { e.lre.FileSize = fileSize } e.rotatorLock.Unlock() - - // Re-syncing task with Consul agent - if e.consulSyncer != nil { - e.interpolateServices(e.ctx.Task) - domain := consul.NewExecutorDomain(e.ctx.AllocID, task.Name) - serviceMap := generateServiceKeys(e.ctx.AllocID, task.Services) - e.consulSyncer.SetServices(domain, serviceMap) - } return nil } -// generateServiceKeys takes a list of interpolated Nomad Services and returns a map -// of ServiceKeys to Nomad Services. -func generateServiceKeys(allocID string, services []*structs.Service) map[consul.ServiceKey]*structs.Service { - keys := make(map[consul.ServiceKey]*structs.Service, len(services)) - for _, service := range services { - key := consul.GenerateServiceKey(service) - keys[key] = service - } - return keys -} - func (e *UniversalExecutor) wait() { defer close(e.processExited) err := e.cmd.Wait() @@ -464,10 +414,6 @@ func (e *UniversalExecutor) Exit() error { e.lro.Close() } - if e.consulSyncer != nil { - e.consulSyncer.Shutdown() - } - // If the executor did not launch a process, return. if e.command == nil { return nil @@ -514,38 +460,6 @@ func (e *UniversalExecutor) ShutDown() error { return nil } -// SyncServices syncs the services of the task that the executor is running with -// Consul -func (e *UniversalExecutor) SyncServices(ctx *ConsulContext) error { - e.logger.Printf("[INFO] executor: registering services") - e.consulCtx = ctx - if e.consulSyncer == nil { - cs, err := consul.NewSyncer(ctx.ConsulConfig, e.shutdownCh, e.logger) - if err != nil { - return err - } - e.consulSyncer = cs - go e.consulSyncer.Run() - } - e.interpolateServices(e.ctx.Task) - e.consulSyncer.SetDelegatedChecks(e.createCheckMap(), e.createCheck) - e.consulSyncer.SetAddrFinder(e.ctx.Task.FindHostAndPortFor) - domain := consul.NewExecutorDomain(e.ctx.AllocID, e.ctx.Task.Name) - serviceMap := generateServiceKeys(e.ctx.AllocID, e.ctx.Task.Services) - e.consulSyncer.SetServices(domain, serviceMap) - return nil -} - -// DeregisterServices removes the services of the task that the executor is -// running from Consul -func (e *UniversalExecutor) DeregisterServices() error { - e.logger.Printf("[INFO] executor: de-registering services and shutting down consul service") - if e.consulSyncer != nil { - return e.consulSyncer.Shutdown() - } - return nil -} - // pidStats returns the resource usage stats per pid func (e *UniversalExecutor) pidStats() (map[string]*cstructs.ResourceUsage, error) { stats := make(map[string]*cstructs.ResourceUsage) @@ -677,66 +591,6 @@ func (e *UniversalExecutor) listenerUnix() (net.Listener, error) { return net.Listen("unix", path) } -// createCheckMap creates a map of checks that the executor will handle on it's -// own -func (e *UniversalExecutor) createCheckMap() map[string]struct{} { - checks := map[string]struct{}{ - "script": struct{}{}, - } - return checks -} - -// createCheck creates NomadCheck from a ServiceCheck -func (e *UniversalExecutor) createCheck(check *structs.ServiceCheck, checkID string) (consul.Check, error) { - if check.Type == structs.ServiceCheckScript && e.ctx.Driver == "docker" { - return &DockerScriptCheck{ - id: checkID, - interval: check.Interval, - timeout: check.Timeout, - containerID: e.consulCtx.ContainerID, - logger: e.logger, - cmd: check.Command, - args: check.Args, - }, nil - } - - if check.Type == structs.ServiceCheckScript && (e.ctx.Driver == "exec" || - e.ctx.Driver == "raw_exec" || e.ctx.Driver == "java") { - return &ExecScriptCheck{ - id: checkID, - interval: check.Interval, - timeout: check.Timeout, - cmd: check.Command, - args: check.Args, - taskDir: e.ctx.TaskDir, - FSIsolation: e.command.FSIsolation, - }, nil - - } - return nil, fmt.Errorf("couldn't create check for %v", check.Name) -} - -// interpolateServices interpolates tags in a service and checks with values from the -// task's environment. -func (e *UniversalExecutor) interpolateServices(task *structs.Task) { - e.ctx.TaskEnv.Build() - for _, service := range task.Services { - for _, check := range service.Checks { - check.Name = e.ctx.TaskEnv.ReplaceEnv(check.Name) - check.Type = e.ctx.TaskEnv.ReplaceEnv(check.Type) - check.Command = e.ctx.TaskEnv.ReplaceEnv(check.Command) - check.Args = e.ctx.TaskEnv.ParseAndReplace(check.Args) - check.Path = e.ctx.TaskEnv.ReplaceEnv(check.Path) - check.Protocol = e.ctx.TaskEnv.ReplaceEnv(check.Protocol) - check.PortLabel = e.ctx.TaskEnv.ReplaceEnv(check.PortLabel) - check.InitialStatus = e.ctx.TaskEnv.ReplaceEnv(check.InitialStatus) - } - service.Name = e.ctx.TaskEnv.ReplaceEnv(service.Name) - service.PortLabel = e.ctx.TaskEnv.ReplaceEnv(service.PortLabel) - service.Tags = e.ctx.TaskEnv.ParseAndReplace(service.Tags) - } -} - // collectPids collects the pids of the child processes that the executor is // running every 5 seconds func (e *UniversalExecutor) collectPids() { diff --git a/client/driver/executor/executor_test.go b/client/driver/executor/executor_test.go index 8508233d6c8b..51325e09a9ec 100644 --- a/client/driver/executor/executor_test.go +++ b/client/driver/executor/executor_test.go @@ -5,7 +5,6 @@ import ( "log" "os" "path/filepath" - "reflect" "strings" "syscall" "testing" @@ -259,31 +258,6 @@ func TestExecutor_MakeExecutable(t *testing.T) { } } -func TestExecutorInterpolateServices(t *testing.T) { - task := mock.Job().TaskGroups[0].Tasks[0] - // Make a fake exececutor - ctx, allocDir := testExecutorContext(t) - defer allocDir.Destroy() - executor := NewExecutor(log.New(os.Stdout, "", log.LstdFlags)) - - executor.(*UniversalExecutor).ctx = ctx - executor.(*UniversalExecutor).interpolateServices(task) - expectedTags := []string{"pci:true", "datacenter:dc1"} - if !reflect.DeepEqual(task.Services[0].Tags, expectedTags) { - t.Fatalf("expected: %v, actual: %v", expectedTags, task.Services[0].Tags) - } - - expectedCheckCmd := "/usr/local/check-table-mysql" - expectedCheckArgs := []string{"5.6"} - if !reflect.DeepEqual(task.Services[0].Checks[0].Command, expectedCheckCmd) { - t.Fatalf("expected: %v, actual: %v", expectedCheckCmd, task.Services[0].Checks[0].Command) - } - - if !reflect.DeepEqual(task.Services[0].Checks[0].Args, expectedCheckArgs) { - t.Fatalf("expected: %v, actual: %v", expectedCheckArgs, task.Services[0].Checks[0].Args) - } -} - func TestScanPids(t *testing.T) { p1 := NewFakeProcess(2, 5) p2 := NewFakeProcess(10, 2) diff --git a/client/driver/executor_plugin.go b/client/driver/executor_plugin.go index 17f40ac25d42..7c4074feffc8 100644 --- a/client/driver/executor_plugin.go +++ b/client/driver/executor_plugin.go @@ -33,11 +33,6 @@ type LaunchCmdArgs struct { Cmd *executor.ExecCommand } -// SyncServicesArgs wraps the consul context for the purposes of RPC -type SyncServicesArgs struct { - Ctx *executor.ConsulContext -} - func (e *ExecutorRPC) LaunchCmd(cmd *executor.ExecCommand) (*executor.ProcessState, error) { var ps *executor.ProcessState err := e.client.Call("Plugin.LaunchCmd", LaunchCmdArgs{Cmd: cmd}, &ps) @@ -76,10 +71,6 @@ func (e *ExecutorRPC) UpdateTask(task *structs.Task) error { return e.client.Call("Plugin.UpdateTask", task, new(interface{})) } -func (e *ExecutorRPC) SyncServices(ctx *executor.ConsulContext) error { - return e.client.Call("Plugin.SyncServices", SyncServicesArgs{Ctx: ctx}, new(interface{})) -} - func (e *ExecutorRPC) DeregisterServices() error { return e.client.Call("Plugin.DeregisterServices", new(interface{}), new(interface{})) } @@ -149,12 +140,9 @@ func (e *ExecutorRPCServer) UpdateTask(args *structs.Task, resp *interface{}) er return e.Impl.UpdateTask(args) } -func (e *ExecutorRPCServer) SyncServices(args SyncServicesArgs, resp *interface{}) error { - return e.Impl.SyncServices(args.Ctx) -} - func (e *ExecutorRPCServer) DeregisterServices(args interface{}, resp *interface{}) error { - return e.Impl.DeregisterServices() + // In 0.6 this is a noop. Goes away in 0.7. + return nil } func (e *ExecutorRPCServer) Version(args interface{}, version *executor.ExecutorVersion) error { diff --git a/client/driver/java.go b/client/driver/java.go index 4a90b4efad65..c684e85bf562 100644 --- a/client/driver/java.go +++ b/client/driver/java.go @@ -2,6 +2,7 @@ package driver import ( "bytes" + "context" "encoding/json" "fmt" "log" @@ -59,6 +60,7 @@ type javaHandle struct { userPid int executor executor.Executor isolationConfig *dstructs.IsolationConfig + taskDir string killTimeout time.Duration maxKillTimeout time.Duration @@ -284,6 +286,7 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, executor: execIntf, userPid: ps.Pid, isolationConfig: ps.IsolationConfig, + taskDir: ctx.TaskDir.Dir, killTimeout: GetKillTimeout(task.KillTimeout, maxKill), maxKillTimeout: maxKill, version: d.config.Version, @@ -291,9 +294,6 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - d.logger.Printf("[ERR] driver.java: error registering services with consul for task: %q: %v", task.Name, err) - } go h.run() return h, nil } @@ -306,6 +306,7 @@ type javaId struct { MaxKillTimeout time.Duration PluginConfig *PluginReattachConfig IsolationConfig *dstructs.IsolationConfig + TaskDir string UserPid int } @@ -352,10 +353,6 @@ func (d *JavaDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - d.logger.Printf("[ERR] driver.java: error registering services with consul: %v", err) - } - go h.run() return h, nil } @@ -368,6 +365,7 @@ func (h *javaHandle) ID() string { PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()), UserPid: h.userPid, IsolationConfig: h.isolationConfig, + TaskDir: h.taskDir, } data, err := json.Marshal(id) @@ -390,6 +388,10 @@ func (h *javaHandle) Update(task *structs.Task) error { return nil } +func (h *javaHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + return execChroot(ctx, h.taskDir, cmd, args) +} + func (h *javaHandle) Signal(s os.Signal) error { return h.executor.Signal(s) } @@ -436,11 +438,6 @@ func (h *javaHandle) run() { } } - // Remove services - if err := h.executor.DeregisterServices(); err != nil { - h.logger.Printf("[ERR] driver.java: failed to kill the deregister services: %v", err) - } - // Exit the executor h.executor.Exit() h.pluginClient.Kill() diff --git a/client/driver/mock_driver.go b/client/driver/mock_driver.go index 8f34297e5139..518e79cea171 100644 --- a/client/driver/mock_driver.go +++ b/client/driver/mock_driver.go @@ -3,6 +3,7 @@ package driver import ( + "context" "encoding/json" "errors" "fmt" @@ -234,6 +235,11 @@ func (h *mockDriverHandle) WaitCh() chan *dstructs.WaitResult { return h.waitCh } +func (h *mockDriverHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + h.logger.Printf("[DEBUG] driver.mock: Exec(%q, %q)", cmd, args) + return []byte(fmt.Sprintf("Exec(%q, %q)", cmd, args)), 0, nil +} + // TODO Implement when we need it. func (h *mockDriverHandle) Update(task *structs.Task) error { return nil diff --git a/client/driver/qemu.go b/client/driver/qemu.go index ad90c80383c2..4e04f95229b3 100644 --- a/client/driver/qemu.go +++ b/client/driver/qemu.go @@ -273,10 +273,6 @@ func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - h.logger.Printf("[ERR] driver.qemu: error registering services for task: %q: %v", task.Name, err) - } go h.run() return h, nil } @@ -322,9 +318,6 @@ func (d *QemuDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - h.logger.Printf("[ERR] driver.qemu: error registering services: %v", err) - } go h.run() return h, nil } @@ -402,11 +395,6 @@ func (h *qemuHandle) run() { } close(h.doneCh) - // Remove services - if err := h.executor.DeregisterServices(); err != nil { - h.logger.Printf("[ERR] driver.qemu: failed to deregister services: %v", err) - } - // Exit the executor h.executor.Exit() h.pluginClient.Kill() diff --git a/client/driver/raw_exec.go b/client/driver/raw_exec.go index 1500cf00f013..e0e86c20bf64 100644 --- a/client/driver/raw_exec.go +++ b/client/driver/raw_exec.go @@ -1,6 +1,7 @@ package driver import ( + "context" "encoding/json" "fmt" "log" @@ -164,9 +165,6 @@ func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandl doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - h.logger.Printf("[ERR] driver.raw_exec: error registering services with consul for task: %q: %v", task.Name, err) - } go h.run() return h, nil } @@ -214,9 +212,6 @@ func (d *RawExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, e doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - h.logger.Printf("[ERR] driver.raw_exec: error registering services with consul: %v", err) - } go h.run() return h, nil } @@ -250,6 +245,10 @@ func (h *rawExecHandle) Update(task *structs.Task) error { return nil } +func (h *rawExecHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + return execChroot(ctx, "", cmd, args) +} + func (h *rawExecHandle) Signal(s os.Signal) error { return h.executor.Signal(s) } @@ -289,10 +288,6 @@ func (h *rawExecHandle) run() { h.logger.Printf("[ERR] driver.raw_exec: error killing user process: %v", e) } } - // Remove services - if err := h.executor.DeregisterServices(); err != nil { - h.logger.Printf("[ERR] driver.raw_exec: failed to deregister services: %v", err) - } // Exit the executor if err := h.executor.Exit(); err != nil { diff --git a/client/driver/raw_exec_test.go b/client/driver/raw_exec_test.go index 7b96c2218cde..cb098f764884 100644 --- a/client/driver/raw_exec_test.go +++ b/client/driver/raw_exec_test.go @@ -1,6 +1,8 @@ package driver import ( + "bytes" + "context" "fmt" "io/ioutil" "path/filepath" @@ -11,6 +13,7 @@ import ( "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/driver/env" + "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/testtask" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -298,3 +301,62 @@ func TestRawExecDriverUser(t *testing.T) { t.Fatalf("Expecting '%v' in '%v'", msg, err) } } + +func TestRawExecDriver_HandlerExec(t *testing.T) { + task := &structs.Task{ + Name: "sleep", + Driver: "raw_exec", + Config: map[string]interface{}{ + "command": testtask.Path(), + "args": []string{"sleep", "9000"}, + }, + LogConfig: &structs.LogConfig{ + MaxFiles: 10, + MaxFileSizeMB: 10, + }, + Resources: basicResources, + } + testtask.SetTaskEnv(task) + ctx := testDriverContexts(t, task) + defer ctx.AllocDir.Destroy() + d := NewRawExecDriver(ctx.DriverCtx) + + if _, err := d.Prestart(ctx.ExecCtx, task); err != nil { + t.Fatalf("prestart err: %v", err) + } + handle, err := d.Start(ctx.ExecCtx, task) + if err != nil { + t.Fatalf("err: %v", err) + } + if handle == nil { + t.Fatalf("missing handle") + } + + // Exec a command that should work + out, code, err := handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"/tmp"}) + if err != nil { + t.Fatalf("error exec'ing stat: %v", err) + } + if code != 0 { + t.Fatalf("expected `stat /alloc` to succeed but exit code was: %d", code) + } + if expected := 100; len(out) < expected { + t.Fatalf("expected at least %d bytes of output but found %d:\n%s", expected, len(out), out) + } + + // Exec a command that should fail + out, code, err = handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"lkjhdsaflkjshowaisxmcvnlia"}) + if err != nil { + t.Fatalf("error exec'ing stat: %v", err) + } + if code == 0 { + t.Fatalf("expected `stat` to fail but exit code was: %d", code) + } + if expected := "No such file or directory"; !bytes.Contains(out, []byte(expected)) { + t.Fatalf("expected output to contain %q but found: %q", expected, out) + } + + if err := handle.Kill(); err != nil { + t.Fatalf("error killing exec handle: %v", err) + } +} diff --git a/client/driver/rkt.go b/client/driver/rkt.go index 55e2a026c9de..d1e039dd60cf 100644 --- a/client/driver/rkt.go +++ b/client/driver/rkt.go @@ -2,8 +2,10 @@ package driver import ( "bytes" + "context" "encoding/json" "fmt" + "io/ioutil" "log" "net" "os" @@ -51,6 +53,9 @@ const ( // rktCmd is the command rkt is installed as. rktCmd = "rkt" + + // rktUuidDeadline is how long to wait for the uuid file to be written + rktUuidDeadline = 5 * time.Second ) // RktDriver is a driver for running images via Rkt @@ -81,6 +86,7 @@ type RktDriverConfig struct { // rktHandle is returned from Start/Open as a handle to the PID type rktHandle struct { + uuid string pluginClient *plugin.Client executorPid int executor executor.Executor @@ -94,6 +100,7 @@ type rktHandle struct { // rktPID is a struct to map the pid running the process to the vm image on // disk type rktPID struct { + UUID string PluginConfig *PluginReattachConfig ExecutorPid int KillTimeout time.Duration @@ -229,7 +236,7 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e img := driverConfig.ImageName // Build the command. - var cmdArgs []string + cmdArgs := make([]string, 0, 32) // Add debug option to rkt command. debug := driverConfig.Debug @@ -253,6 +260,11 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e } cmdArgs = append(cmdArgs, "run") + // Write the UUID out to a file in the state dir so we can read it back + // in and access the pod by UUID from other commands + uuidPath := filepath.Join(ctx.TaskDir.Dir, "rkt.uuid") + cmdArgs = append(cmdArgs, fmt.Sprintf("--uuid-file-save=%s", uuidPath)) + // Convert underscores to dashes in task names for use in volume names #2358 sanitizedName := strings.Replace(task.Name, "_", "-", -1) @@ -439,9 +451,28 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e return nil, err } - d.logger.Printf("[DEBUG] driver.rkt: started ACI %q with: %v", img, cmdArgs) + // Wait for UUID file to get written + uuid := "" + deadline := time.Now().Add(rktUuidDeadline) + var lastErr error + for time.Now().Before(deadline) { + if uuidBytes, err := ioutil.ReadFile(uuidPath); err != nil { + lastErr = err + } else { + uuid = string(uuidBytes) + break + } + time.Sleep(400 * time.Millisecond) + } + if uuid == "" { + d.logger.Printf("[WARN] driver.rkt: reading uuid from %q failed; unable to run script checks for task %q. Last error: %v", + uuidPath, d.taskName, lastErr) + } + + d.logger.Printf("[DEBUG] driver.rkt: started ACI %q (UUID: %s) for task %q with: %v", img, uuid, d.taskName, cmdArgs) maxKill := d.DriverContext.config.MaxKillTimeout h := &rktHandle{ + uuid: uuid, pluginClient: pluginClient, executor: execIntf, executorPid: ps.Pid, @@ -451,9 +482,6 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - h.logger.Printf("[ERR] driver.rkt: error registering services for task: %q: %v", task.Name, err) - } go h.run() return h, nil } @@ -484,6 +512,7 @@ func (d *RktDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error d.logger.Printf("[DEBUG] driver.rkt: version of executor: %v", ver.Version) // Return a driver handle h := &rktHandle{ + uuid: id.UUID, pluginClient: pluginClient, executorPid: id.ExecutorPid, executor: exec, @@ -493,9 +522,6 @@ func (d *RktDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error doneCh: make(chan struct{}), waitCh: make(chan *dstructs.WaitResult, 1), } - if err := h.executor.SyncServices(consulContext(d.config, "")); err != nil { - h.logger.Printf("[ERR] driver.rkt: error registering services: %v", err) - } go h.run() return h, nil } @@ -503,6 +529,7 @@ func (d *RktDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error func (h *rktHandle) ID() string { // Return a handle to the PID pid := &rktPID{ + UUID: h.uuid, PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()), KillTimeout: h.killTimeout, MaxKillTimeout: h.maxKillTimeout, @@ -528,6 +555,19 @@ func (h *rktHandle) Update(task *structs.Task) error { return nil } +func (h *rktHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + if h.uuid == "" { + return nil, 0, fmt.Errorf("unable to find rkt pod UUID") + } + // enter + UUID + cmd + args... + enterArgs := make([]string, 3+len(args)) + enterArgs[0] = "enter" + enterArgs[1] = h.uuid + enterArgs[2] = cmd + copy(enterArgs[3:], args) + return execChroot(ctx, "", rktCmd, enterArgs) +} + func (h *rktHandle) Signal(s os.Signal) error { return fmt.Errorf("Rkt does not support signals") } @@ -556,10 +596,6 @@ func (h *rktHandle) run() { h.logger.Printf("[ERROR] driver.rkt: error killing user process: %v", e) } } - // Remove services - if err := h.executor.DeregisterServices(); err != nil { - h.logger.Printf("[ERR] driver.rkt: failed to deregister services: %v", err) - } // Exit the executor if err := h.executor.Exit(); err != nil { diff --git a/client/driver/rkt_test.go b/client/driver/rkt_test.go index b54160becdcb..34403cb1086e 100644 --- a/client/driver/rkt_test.go +++ b/client/driver/rkt_test.go @@ -1,6 +1,8 @@ package driver import ( + "bytes" + "context" "fmt" "io/ioutil" "os" @@ -12,6 +14,7 @@ import ( "time" "github.com/hashicorp/nomad/client/config" + "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -489,3 +492,74 @@ func TestRktDriver_PortsMapping(t *testing.T) { t.Fatalf("timeout") } } + +func TestRktDriver_HandlerExec(t *testing.T) { + if os.Getenv("NOMAD_TEST_RKT") == "" { + t.Skip("skipping rkt tests") + } + + ctestutils.RktCompatible(t) + task := &structs.Task{ + Name: "etcd", + Driver: "rkt", + Config: map[string]interface{}{ + "trust_prefix": "coreos.com/etcd", + "image": "coreos.com/etcd:v2.0.4", + "command": "/etcd", + }, + LogConfig: &structs.LogConfig{ + MaxFiles: 10, + MaxFileSizeMB: 10, + }, + Resources: &structs.Resources{ + MemoryMB: 128, + CPU: 100, + }, + } + + ctx := testDriverContexts(t, task) + defer ctx.AllocDir.Destroy() + d := NewRktDriver(ctx.DriverCtx) + + if _, err := d.Prestart(ctx.ExecCtx, task); err != nil { + t.Fatalf("error in prestart: %v", err) + } + handle, err := d.Start(ctx.ExecCtx, task) + if err != nil { + t.Fatalf("err: %v", err) + } + if handle == nil { + t.Fatalf("missing handle") + } + + // Give the pod a second to start + time.Sleep(time.Second) + + // Exec a command that should work + out, code, err := handle.(consul.ScriptExecutor).Exec(context.TODO(), "/etcd", []string{"--version"}) + if err != nil { + t.Fatalf("error exec'ing etcd --version: %v", err) + } + if code != 0 { + t.Fatalf("expected `etcd --version` to succeed but exit code was: %d\n%s", code, string(out)) + } + if expected := []byte("etcd version "); !bytes.HasPrefix(out, expected) { + t.Fatalf("expected output to start with %q but found:\n%q", expected, out) + } + + // Exec a command that should fail + out, code, err = handle.(consul.ScriptExecutor).Exec(context.TODO(), "/etcd", []string{"--kaljdshf"}) + if err != nil { + t.Fatalf("error exec'ing bad command: %v", err) + } + if code == 0 { + t.Fatalf("expected `stat` to fail but exit code was: %d", code) + } + if expected := "flag provided but not defined"; !bytes.Contains(out, []byte(expected)) { + t.Fatalf("expected output to contain %q but found: %q", expected, out) + } + + if err := handle.Kill(); err != nil { + t.Fatalf("error killing handle: %v", err) + } +} diff --git a/client/driver/utils.go b/client/driver/utils.go index 7e1c79890bb7..def668548ff7 100644 --- a/client/driver/utils.go +++ b/client/driver/utils.go @@ -1,6 +1,7 @@ package driver import ( + "context" "encoding/json" "fmt" "io" @@ -8,8 +9,10 @@ import ( "os/exec" "path/filepath" "strings" + "syscall" "time" + "github.com/armon/circbuf" "github.com/hashicorp/go-multierror" "github.com/hashicorp/go-plugin" "github.com/hashicorp/nomad/client/config" @@ -85,19 +88,16 @@ func createExecutorWithConfig(config *plugin.ClientConfig, w io.Writer) (executo if err != nil { return nil, nil, fmt.Errorf("unable to dispense the executor plugin: %v", err) } - executorPlugin := raw.(executor.Executor) - return executorPlugin, executorClient, nil -} - -func consulContext(clientConfig *config.Config, containerID string) *executor.ConsulContext { - return &executor.ConsulContext{ - ConsulConfig: clientConfig.ConsulConfig, - ContainerID: containerID, - DockerEndpoint: clientConfig.Read("docker.endpoint"), - TLSCa: clientConfig.Read("docker.tls.ca"), - TLSCert: clientConfig.Read("docker.tls.cert"), - TLSKey: clientConfig.Read("docker.tls.key"), + executorPlugin, ok := raw.(*ExecutorRPC) + if !ok { + return nil, nil, fmt.Errorf("unexpected executor rpc type: %T", raw) } + // 0.6 Upgrade path: Deregister services from the executor as the Nomad + // client agent now handles all Consul interactions. + if err := executorPlugin.DeregisterServices(); err != nil { + return nil, nil, err + } + return executorPlugin, executorClient, nil } // killProcess kills a process with the given pid @@ -181,3 +181,36 @@ func getExecutorUser(task *structs.Task) string { } return task.User } + +// execChroot executes cmd with args inside chroot if set and returns the +// output, exit code, and error. If chroot is an empty string the command is +// executed on the host. +func execChroot(ctx context.Context, chroot, name string, args []string) ([]byte, int, error) { + buf, _ := circbuf.NewBuffer(int64(cstructs.CheckBufSize)) + cmd := exec.CommandContext(ctx, name, args...) + cmd.Dir = "/" + cmd.Stdout = buf + cmd.Stderr = buf + if chroot != "" { + setChroot(cmd, chroot) + } + if err := cmd.Run(); err != nil { + exitErr, ok := err.(*exec.ExitError) + if !ok { + // Non-exit error, return it and let the caller treat + // it as a critical failure + return nil, 0, err + } + + // Some kind of error happened; default to critical + exitCode := 2 + if status, ok := exitErr.Sys().(syscall.WaitStatus); ok { + exitCode = status.ExitStatus() + } + + // Don't return the exitError as the caller only needs the + // output and code. + return buf.Bytes(), exitCode, nil + } + return buf.Bytes(), 0, nil +} diff --git a/client/driver/utils_unix.go b/client/driver/utils_unix.go index 474cdcf17f1f..397641e3e796 100644 --- a/client/driver/utils_unix.go +++ b/client/driver/utils_unix.go @@ -16,3 +16,11 @@ func isolateCommand(cmd *exec.Cmd) { } cmd.SysProcAttr.Setsid = true } + +// setChroot on a command +func setChroot(cmd *exec.Cmd, chroot string) { + if cmd.SysProcAttr == nil { + cmd.SysProcAttr = &syscall.SysProcAttr{} + } + cmd.SysProcAttr.Chroot = chroot +} diff --git a/client/task_runner.go b/client/task_runner.go index 55176c0e705a..64eb4570f1f0 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -21,6 +21,7 @@ import ( "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/client/getter" "github.com/hashicorp/nomad/client/vaultclient" + "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/client/driver/env" @@ -61,6 +62,7 @@ type TaskRunner struct { logger *log.Logger alloc *structs.Allocation restartTracker *RestartTracker + consul ConsulServiceAPI // running marks whether the task is running running bool @@ -173,7 +175,7 @@ type SignalEvent struct { func NewTaskRunner(logger *log.Logger, config *config.Config, updater TaskStateUpdater, taskDir *allocdir.TaskDir, alloc *structs.Allocation, task *structs.Task, - vaultClient vaultclient.VaultClient) *TaskRunner { + vaultClient vaultclient.VaultClient, consulClient ConsulServiceAPI) *TaskRunner { // Merge in the task resources task.Resources = alloc.TaskResources[task.Name] @@ -195,6 +197,7 @@ func NewTaskRunner(logger *log.Logger, config *config.Config, task: task, taskDir: taskDir, createdResources: driver.NewCreatedResources(), + consul: consulClient, vaultClient: vaultClient, vaultFuture: NewTokenFuture().Set(""), updateCh: make(chan *structs.Allocation, 64), @@ -289,6 +292,19 @@ func (r *TaskRunner) RestoreState() error { r.task.Name, r.alloc.ID, err) return nil } + + //FIXME is there a better place to do this? used to be in executor + // Prepare services + interpolateServices(r.getTaskEnv(), r.task) + + // Ensure the service is registered + scriptExec, _ := handle.(consul.ScriptExecutor) + if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { + //FIXME What to do if this fails? + r.logger.Printf("[WARN] client: failed to register services and checks for task %q alloc %q: %v", + r.task.Name, r.alloc.ID, err) + } + r.handleLock.Lock() r.handle = handle r.handleLock.Unlock() @@ -1220,9 +1236,43 @@ func (r *TaskRunner) startTask() error { r.handleLock.Lock() r.handle = handle r.handleLock.Unlock() + + //FIXME is there a better place to do this? used to be in executor + // Prepare services + interpolateServices(r.getTaskEnv(), r.task) + + // RegisterTask properly handles scriptExec being nil, so it just + // ignore the ok value. + scriptExec, _ := handle.(consul.ScriptExecutor) + if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { + //FIXME handle errors?! + //FIXME could break into prepare & submit steps as only preperation can error... + r.logger.Printf("[ERR] client: failed to register services and checks for task %q alloc %q: %v", r.task.Name, r.alloc.ID, err) + } + return nil } +// interpolateServices interpolates tags in a service and checks with values from the +// task's environment. +func interpolateServices(taskEnv *env.TaskEnvironment, task *structs.Task) { + for _, service := range task.Services { + for _, check := range service.Checks { + check.Name = taskEnv.ReplaceEnv(check.Name) + check.Type = taskEnv.ReplaceEnv(check.Type) + check.Command = taskEnv.ReplaceEnv(check.Command) + check.Args = taskEnv.ParseAndReplace(check.Args) + check.Path = taskEnv.ReplaceEnv(check.Path) + check.Protocol = taskEnv.ReplaceEnv(check.Protocol) + check.PortLabel = taskEnv.ReplaceEnv(check.PortLabel) + check.InitialStatus = taskEnv.ReplaceEnv(check.InitialStatus) + } + service.Name = taskEnv.ReplaceEnv(service.Name) + service.PortLabel = taskEnv.ReplaceEnv(service.PortLabel) + service.Tags = taskEnv.ParseAndReplace(service.Tags) + } +} + // buildTaskDir creates the task directory before driver.Prestart. It is safe // to call multiple times as its state is persisted. func (r *TaskRunner) buildTaskDir(fsi cstructs.FSIsolation) error { @@ -1335,13 +1385,16 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { // Merge in the task resources updatedTask.Resources = update.TaskResources[updatedTask.Name] - // Update will update resources and store the new kill timeout. var mErr multierror.Error + var scriptExec consul.ScriptExecutor r.handleLock.Lock() if r.handle != nil { + // Update will update resources and store the new kill timeout. if err := r.handle.Update(updatedTask); err != nil { mErr.Errors = append(mErr.Errors, fmt.Errorf("updating task resources failed: %v", err)) } + // Not all drivers support Exec (eg QEMU) + scriptExec, _ = r.handle.(consul.ScriptExecutor) } r.handleLock.Unlock() @@ -1350,9 +1403,21 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { r.restartTracker.SetPolicy(tg.RestartPolicy) } + // Deregister the old service+checks + r.consul.RemoveTask(r.alloc.ID, r.task) + // Store the updated alloc. r.alloc = update r.task = updatedTask + + //FIXME is there a better place to do this? used to be in executor + // Prepare services + interpolateServices(r.getTaskEnv(), r.task) + + // Register the new service+checks + if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { + mErr.Errors = append(mErr.Errors, fmt.Errorf("error registering updated task with consul: %v", err)) + } return mErr.ErrorOrNil() } @@ -1361,6 +1426,9 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { // given limit. It returns whether the task was destroyed and the error // associated with the last kill attempt. func (r *TaskRunner) handleDestroy() (destroyed bool, err error) { + // Remove from Consul + r.consul.RemoveTask(r.alloc.ID, r.task) + // Cap the number of times we attempt to kill the task. for i := 0; i < killFailureLimit; i++ { if err = r.handle.Kill(); err != nil { diff --git a/client/task_runner_test.go b/client/task_runner_test.go index 6baa14f97344..ec511e93dfbc 100644 --- a/client/task_runner_test.go +++ b/client/task_runner_test.go @@ -104,7 +104,8 @@ func testTaskRunnerFromAlloc(t *testing.T, restarts bool, alloc *structs.Allocat } vclient := vaultclient.NewMockVaultClient() - tr := NewTaskRunner(logger, conf, upd.Update, taskDir, alloc, task, vclient) + cclient := newMockConsulServiceClient() + tr := NewTaskRunner(logger, conf, upd.Update, taskDir, alloc, task, vclient, cclient) if !restarts { tr.restartTracker = noRestartsTracker() } @@ -366,7 +367,7 @@ func TestTaskRunner_SaveRestoreState(t *testing.T) { // Create a new task runner task2 := &structs.Task{Name: ctx.tr.task.Name, Driver: ctx.tr.task.Driver} tr2 := NewTaskRunner(ctx.tr.logger, ctx.tr.config, ctx.upd.Update, - ctx.tr.taskDir, ctx.tr.alloc, task2, ctx.tr.vaultClient) + ctx.tr.taskDir, ctx.tr.alloc, task2, ctx.tr.vaultClient, ctx.tr.consul) tr2.restartTracker = noRestartsTracker() if err := tr2.RestoreState(); err != nil { t.Fatalf("err: %v", err) diff --git a/command/agent/agent.go b/command/agent/agent.go index 142dfe2e34ef..8472a98f1da6 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -8,17 +8,18 @@ import ( "os" "path/filepath" "runtime" - "strconv" "strings" "sync" "sync/atomic" "time" + "github.com/hashicorp/consul/api" "github.com/hashicorp/nomad/client" clientconfig "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/structs/config" ) const ( @@ -30,6 +31,10 @@ const ( serverRpcCheckTimeout = 3 * time.Second serverSerfCheckInterval = 10 * time.Second serverSerfCheckTimeout = 3 * time.Second + + // roles used in identifying Consul entries for Nomad agents + consulRoleServer = "server" + consulRoleClient = "client" ) // Agent is a long running daemon that is used to run both @@ -42,8 +47,12 @@ type Agent struct { logger *log.Logger logOutput io.Writer - // consulSyncer registers the Nomad agent with the Consul Agent - consulSyncer *consul.Syncer + // consulService is Nomad's custom Consul client for managing services + // and checks. + consulService *consul.ServiceClient + + // consulCatalog is the subset of Consul's Catalog API Nomad uses. + consulCatalog consul.CatalogAPI client *client.Client @@ -63,8 +72,8 @@ func NewAgent(config *Config, logOutput io.Writer) (*Agent, error) { shutdownCh: make(chan struct{}), } - if err := a.setupConsulSyncer(); err != nil { - return nil, fmt.Errorf("Failed to initialize Consul syncer task: %v", err) + if err := a.setupConsul(config.Consul); err != nil { + return nil, fmt.Errorf("Failed to initialize Consul client: %v", err) } if err := a.setupServer(); err != nil { return nil, err @@ -76,15 +85,6 @@ func NewAgent(config *Config, logOutput io.Writer) (*Agent, error) { return nil, fmt.Errorf("must have at least client or server mode enabled") } - // The Nomad Agent runs the consul.Syncer regardless of whether or not the - // Agent is running in Client or Server mode (or both), and regardless of - // the consul.auto_advertise parameter. The Client and Server both reuse the - // same consul.Syncer instance. This Syncer task periodically executes - // callbacks that update Consul. The reason the Syncer is always running is - // because one of the callbacks is attempts to self-bootstrap Nomad using - // information found in Consul. - go a.consulSyncer.Run() - return a, nil } @@ -339,7 +339,7 @@ func (a *Agent) setupServer() error { } // Create the server - server, err := nomad.NewServer(conf, a.consulSyncer, a.logger) + server, err := nomad.NewServer(conf, a.consulCatalog, a.logger) if err != nil { return fmt.Errorf("server setup failed: %v", err) } @@ -405,14 +405,16 @@ func (a *Agent) setupServer() error { // Add the http port check if TLS isn't enabled // TODO Add TLS check when Consul 0.7.1 comes out. - consulServices := map[consul.ServiceKey]*structs.Service{ - consul.GenerateServiceKey(rpcServ): rpcServ, - consul.GenerateServiceKey(serfServ): serfServ, + consulServices := []*structs.Service{ + rpcServ, + serfServ, } if !conf.TLSConfig.EnableHTTP { - consulServices[consul.GenerateServiceKey(httpServ)] = httpServ + consulServices = append(consulServices, httpServ) + } + if err := a.consulService.RegisterAgent(consulRoleServer, consulServices); err != nil { + return err } - a.consulSyncer.SetServices(consul.ServerDomain, consulServices) } return nil @@ -462,7 +464,7 @@ func (a *Agent) setupClient() error { } // Create the client - client, err := client.NewClient(conf, a.consulSyncer, a.logger) + client, err := client.NewClient(conf, a.consulCatalog, a.consulService, a.logger) if err != nil { return fmt.Errorf("client setup failed: %v", err) } @@ -495,9 +497,9 @@ func (a *Agent) setupClient() error { }, } if !conf.TLSConfig.EnableHTTP { - a.consulSyncer.SetServices(consul.ClientDomain, map[consul.ServiceKey]*structs.Service{ - consul.GenerateServiceKey(httpServ): httpServ, - }) + if err := a.consulService.RegisterAgent(consulRoleClient, []*structs.Service{httpServ}); err != nil { + return err + } } } @@ -612,8 +614,8 @@ func (a *Agent) Shutdown() error { } } - if err := a.consulSyncer.Shutdown(); err != nil { - a.logger.Printf("[ERR] agent: shutting down consul service failed: %v", err) + if err := a.consulService.Shutdown(); err != nil { + a.logger.Printf("[ERR] agent: shutting down Consul client failed: %v", err) } a.logger.Println("[INFO] agent: shutdown complete") @@ -659,46 +661,22 @@ func (a *Agent) Stats() map[string]map[string]string { return stats } -// setupConsulSyncer creates the Consul tasks used by this Nomad Agent -// (either Client or Server mode). -func (a *Agent) setupConsulSyncer() error { - var err error - a.consulSyncer, err = consul.NewSyncer(a.config.Consul, a.shutdownCh, a.logger) +// setupConsul creates the Consul client and starts its main Run loop. +func (a *Agent) setupConsul(consulConfig *config.ConsulConfig) error { + apiConf, err := consulConfig.ApiConfig() + if err != nil { + return err + } + client, err := api.NewClient(apiConf) if err != nil { return err } - a.consulSyncer.SetAddrFinder(func(portLabel string) (string, int) { - host, port, err := net.SplitHostPort(portLabel) - if err != nil { - p, err := strconv.Atoi(port) - if err != nil { - return "", 0 - } - return "", p - } - - // If the addr for the service is ":port", then we fall back - // to Nomad's default address resolution protocol. - // - // TODO(sean@): This should poll Consul to figure out what - // its advertise address is and use that in order to handle - // the case where there is something funky like NAT on this - // host. For now we just use the BindAddr if set, otherwise - // we fall back to a loopback addr. - if host == "" { - if a.config.BindAddr != "" { - host = a.config.BindAddr - } else { - host = "127.0.0.1" - } - } - p, err := strconv.Atoi(port) - if err != nil { - return host, 0 - } - return host, p - }) + // Create Consul Catalog client for service discovery. + a.consulCatalog = client.Catalog() + // Create Consul Service client for service advertisement and checks. + a.consulService = consul.NewServiceClient(client.Agent(), a.logger) + go a.consulService.Run() return nil } diff --git a/command/agent/consul/chaos_test.go b/command/agent/consul/chaos_test.go deleted file mode 100644 index 89b69ea2cb67..000000000000 --- a/command/agent/consul/chaos_test.go +++ /dev/null @@ -1,193 +0,0 @@ -// +build chaos - -package consul - -import ( - "fmt" - "io/ioutil" - "sort" - "strings" - "sync" - "testing" - "time" - - "github.com/hashicorp/consul/testutil" - "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/nomad/structs/config" -) - -func TestSyncerChaos(t *testing.T) { - // Create an embedded Consul server - testconsul := testutil.NewTestServerConfig(t, func(c *testutil.TestServerConfig) { - // If -v wasn't specified squelch consul logging - if !testing.Verbose() { - c.Stdout = ioutil.Discard - c.Stderr = ioutil.Discard - } - }) - defer testconsul.Stop() - - // Configure Syncer to talk to the test server - cconf := config.DefaultConsulConfig() - cconf.Addr = testconsul.HTTPAddr - - clientSyncer, err := NewSyncer(cconf, nil, logger) - if err != nil { - t.Fatalf("Error creating Syncer: %v", err) - } - defer clientSyncer.Shutdown() - - execSyncer, err := NewSyncer(cconf, nil, logger) - if err != nil { - t.Fatalf("Error creating Syncer: %v", err) - } - defer execSyncer.Shutdown() - - clientService := &structs.Service{Name: "nomad-client"} - services := map[ServiceKey]*structs.Service{ - GenerateServiceKey(clientService): clientService, - } - if err := clientSyncer.SetServices("client", services); err != nil { - t.Fatalf("error setting client service: %v", err) - } - - const execn = 100 - const reapern = 2 - errors := make(chan error, 100) - wg := sync.WaitGroup{} - - // Start goroutines to concurrently SetServices - for i := 0; i < execn; i++ { - wg.Add(1) - go func(i int) { - defer wg.Done() - domain := ServiceDomain(fmt.Sprintf("exec-%d", i)) - services := map[ServiceKey]*structs.Service{} - for ii := 0; ii < 10; ii++ { - s := &structs.Service{Name: fmt.Sprintf("exec-%d-%d", i, ii)} - services[GenerateServiceKey(s)] = s - if err := execSyncer.SetServices(domain, services); err != nil { - select { - case errors <- err: - default: - } - return - } - time.Sleep(1) - } - }(i) - } - - // SyncServices runs a timer started by Syncer.Run which we don't use - // in this test, so run SyncServices concurrently - wg.Add(1) - go func() { - defer wg.Done() - for i := 0; i < execn; i++ { - if err := execSyncer.SyncServices(); err != nil { - select { - case errors <- err: - default: - } - return - } - time.Sleep(100) - } - }() - - wg.Add(1) - go func() { - defer wg.Done() - if err := clientSyncer.ReapUnmatched([]ServiceDomain{"nomad-client"}); err != nil { - select { - case errors <- err: - default: - } - return - } - }() - - // Reap all but exec-0-* - wg.Add(1) - go func() { - defer wg.Done() - for i := 0; i < execn; i++ { - if err := execSyncer.ReapUnmatched([]ServiceDomain{"exec-0", ServiceDomain(fmt.Sprintf("exec-%d", i))}); err != nil { - select { - case errors <- err: - default: - } - } - time.Sleep(100) - } - }() - - go func() { - wg.Wait() - close(errors) - }() - - for err := range errors { - if err != nil { - t.Errorf("error setting service from executor goroutine: %v", err) - } - } - - // Do a final ReapUnmatched to get consul back into a deterministic state - if err := execSyncer.ReapUnmatched([]ServiceDomain{"exec-0"}); err != nil { - t.Fatalf("error doing final reap: %v", err) - } - - // flattenedServices should be fully populated as ReapUnmatched doesn't - // touch Syncer's internal state - expected := map[string]struct{}{} - for i := 0; i < execn; i++ { - for ii := 0; ii < 10; ii++ { - expected[fmt.Sprintf("exec-%d-%d", i, ii)] = struct{}{} - } - } - - for _, s := range execSyncer.flattenedServices() { - _, ok := expected[s.Name] - if !ok { - t.Errorf("%s unexpected", s.Name) - } - delete(expected, s.Name) - } - if len(expected) > 0 { - left := []string{} - for s := range expected { - left = append(left, s) - } - sort.Strings(left) - t.Errorf("Couldn't find %d names in flattened services:\n%s", len(expected), strings.Join(left, "\n")) - } - - // All but exec-0 and possibly some of exec-99 should have been reaped - { - services, err := execSyncer.client.Agent().Services() - if err != nil { - t.Fatalf("Error getting services: %v", err) - } - expected := []int{} - for k, service := range services { - if service.Service == "consul" { - continue - } - i := -1 - ii := -1 - fmt.Sscanf(service.Service, "exec-%d-%d", &i, &ii) - switch { - case i == -1 || ii == -1: - t.Errorf("invalid service: %s -> %s", k, service.Service) - case i != 0 || ii > 9: - t.Errorf("unexpected service: %s -> %s", k, service.Service) - default: - expected = append(expected, ii) - } - } - if len(expected) != 10 { - t.Errorf("expected 0-9 but found: %#q", expected) - } - } -} diff --git a/command/agent/consul/check.go b/command/agent/consul/check.go deleted file mode 100644 index 551f94b6fc39..000000000000 --- a/command/agent/consul/check.go +++ /dev/null @@ -1,91 +0,0 @@ -package consul - -import ( - "log" - "sync" - "time" - - "github.com/hashicorp/consul/lib" - cstructs "github.com/hashicorp/nomad/client/driver/structs" -) - -// CheckRunner runs a given check in a specific interval and update a -// corresponding Consul TTL check -type CheckRunner struct { - check Check - runCheck func(Check) - logger *log.Logger - stop bool - stopCh chan struct{} - stopLock sync.Mutex - - started bool - startedLock sync.Mutex -} - -// NewCheckRunner configures and returns a CheckRunner -func NewCheckRunner(check Check, runCheck func(Check), logger *log.Logger) *CheckRunner { - cr := CheckRunner{ - check: check, - runCheck: runCheck, - logger: logger, - stopCh: make(chan struct{}), - } - return &cr -} - -// Start is used to start the check. The check runs until stop is called -func (r *CheckRunner) Start() { - r.startedLock.Lock() - defer r.startedLock.Unlock() - if r.started { - return - } - r.stopLock.Lock() - defer r.stopLock.Unlock() - go r.run() - r.started = true -} - -// Started returns if the check runner has started running -func (r *CheckRunner) Started() bool { - r.startedLock.Lock() - defer r.startedLock.Unlock() - return r.started -} - -// Stop is used to stop the check. -func (r *CheckRunner) Stop() { - r.stopLock.Lock() - defer r.stopLock.Unlock() - if !r.stop { - r.stop = true - close(r.stopCh) - } -} - -// run is invoked by a goroutine to run until Stop() is called -func (r *CheckRunner) run() { - // Get the randomized initial pause time - initialPauseTime := lib.RandomStagger(r.check.Interval()) - r.logger.Printf("[DEBUG] agent: pausing %v before first invocation of %s", initialPauseTime, r.check.ID()) - next := time.NewTimer(initialPauseTime) - for { - select { - case <-next.C: - r.runCheck(r.check) - next.Reset(r.check.Interval()) - case <-r.stopCh: - next.Stop() - return - } - } -} - -// Check is an interface which check providers can implement for Nomad to run -type Check interface { - Run() *cstructs.CheckResult - ID() string - Interval() time.Duration - Timeout() time.Duration -} diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go new file mode 100644 index 000000000000..6365eb0b8316 --- /dev/null +++ b/command/agent/consul/client.go @@ -0,0 +1,636 @@ +package consul + +import ( + "context" + "fmt" + "log" + "net" + "net/url" + "strconv" + "strings" + "sync" + "time" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/go-multierror" + "github.com/hashicorp/nomad/nomad/structs" +) + +var mark = struct{}{} + +const ( + // nomadServicePrefix is the first prefix that scopes all Nomad registered + // services + nomadServicePrefix = "_nomad" + + // The periodic time interval for syncing services and checks with Consul + defaultSyncInterval = 6 * time.Second + + // ttlCheckBuffer is the time interval that Nomad can take to report Consul + // the check result + ttlCheckBuffer = 31 * time.Second + + // defaultShutdownWait is how long Shutdown() should block waiting for + // enqueued operations to sync to Consul by default. + defaultShutdownWait = time.Minute + + // DefaultQueryWaitDuration is the max duration the Consul Agent will + // spend waiting for a response from a Consul Query. + DefaultQueryWaitDuration = 2 * time.Second + + // ServiceTagHTTP is the tag assigned to HTTP services + ServiceTagHTTP = "http" + + // ServiceTagRPC is the tag assigned to RPC services + ServiceTagRPC = "rpc" + + // ServiceTagSerf is the tag assigned to Serf services + ServiceTagSerf = "serf" +) + +// ScriptExecutor is the interface the ServiceClient uses to execute script +// checks inside a container. +type ScriptExecutor interface { + Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) +} + +// CatalogAPI is the consul/api.Catalog API used by Nomad. +type CatalogAPI interface { + Datacenters() ([]string, error) + Service(service, tag string, q *api.QueryOptions) ([]*api.CatalogService, *api.QueryMeta, error) +} + +// AgentAPI is the consul/api.Agent API used by Nomad. +type AgentAPI interface { + CheckRegister(check *api.AgentCheckRegistration) error + CheckDeregister(checkID string) error + ServiceRegister(service *api.AgentServiceRegistration) error + ServiceDeregister(serviceID string) error + UpdateTTL(id, output, status string) error +} + +// ServiceClient handles task and agent service registration with Consul. +type ServiceClient struct { + client AgentAPI + logger *log.Logger + retryInterval time.Duration + syncInterval time.Duration + + // runningCh is closed when the main Run loop exits + runningCh chan struct{} + + // shutdownCh is closed when the client should shutdown + shutdownCh chan struct{} + + // shutdownWait is how long Shutdown() blocks waiting for the final + // sync() to finish. Defaults to defaultShutdownWait + shutdownWait time.Duration + + // syncCh triggers a sync in the main Run loop + syncCh chan struct{} + + // services and checks to be registered + regServices map[string]*api.AgentServiceRegistration + regChecks map[string]*api.AgentCheckRegistration + + // services and checks to be unregisterd + deregServices map[string]struct{} + deregChecks map[string]struct{} + + // script checks to be run() after their corresponding check is + // registered + regScripts map[string]*scriptCheck + + // script check cancel funcs to be called before their corresponding + // check is removed. Only accessed in sync() so not covered by regLock + runningScripts map[string]*scriptHandle + + // regLock must be held while accessing reg and dereg maps + regLock sync.Mutex + + // Registered agent services and checks + agentServices map[string]struct{} + agentChecks map[string]struct{} + + // agentLock must be held while accessing agent maps + agentLock sync.Mutex +} + +// NewServiceClient creates a new Consul ServiceClient from an existing Consul API +// Client and logger. +func NewServiceClient(consulClient AgentAPI, logger *log.Logger) *ServiceClient { + return &ServiceClient{ + client: consulClient, + logger: logger, + retryInterval: defaultSyncInterval, //TODO what should this default to?! + syncInterval: defaultSyncInterval, + runningCh: make(chan struct{}), + shutdownCh: make(chan struct{}), + shutdownWait: defaultShutdownWait, + syncCh: make(chan struct{}, 1), + regServices: make(map[string]*api.AgentServiceRegistration), + regChecks: make(map[string]*api.AgentCheckRegistration), + deregServices: make(map[string]struct{}), + deregChecks: make(map[string]struct{}), + regScripts: make(map[string]*scriptCheck), + runningScripts: make(map[string]*scriptHandle), + agentServices: make(map[string]struct{}, 8), + agentChecks: make(map[string]struct{}, 8), + } +} + +// Run the Consul main loop which retries operations against Consul. It should +// be called exactly once. +func (c *ServiceClient) Run() { + defer close(c.runningCh) + timer := time.NewTimer(0) + defer timer.Stop() + + // Drain the initial tick so we don't sync until instructed + <-timer.C + + lastOk := true + for { + select { + case <-c.syncCh: + timer.Reset(0) + case <-timer.C: + if err := c.sync(); err != nil { + if lastOk { + lastOk = false + c.logger.Printf("[WARN] consul: failed to update services in Consul: %v", err) + } + //TODO Log? and jitter/backoff + timer.Reset(c.retryInterval) + } else { + if !lastOk { + c.logger.Printf("[INFO] consul: successfully updated services in Consul") + lastOk = true + } + } + case <-c.shutdownCh: + return + } + } +} + +// forceSync asynchronously causes a sync to happen. Any operations enqueued +// prior to calling forceSync will be synced. +func (c *ServiceClient) forceSync() { + select { + case c.syncCh <- mark: + default: + } +} + +// sync enqueued operations. +func (c *ServiceClient) sync() error { + // Shallow copy and reset the pending operations fields + c.regLock.Lock() + regServices := make(map[string]*api.AgentServiceRegistration, len(c.regServices)) + for k, v := range c.regServices { + regServices[k] = v + } + c.regServices = map[string]*api.AgentServiceRegistration{} + + regChecks := make(map[string]*api.AgentCheckRegistration, len(c.regChecks)) + for k, v := range c.regChecks { + regChecks[k] = v + } + c.regChecks = map[string]*api.AgentCheckRegistration{} + + regScripts := make(map[string]*scriptCheck, len(c.regScripts)) + for k, v := range c.regScripts { + regScripts[k] = v + } + c.regScripts = map[string]*scriptCheck{} + + deregServices := make(map[string]struct{}, len(c.deregServices)) + for k := range c.deregServices { + deregServices[k] = mark + } + c.deregServices = map[string]struct{}{} + + deregChecks := make(map[string]struct{}, len(c.deregChecks)) + for k := range c.deregChecks { + deregChecks[k] = mark + } + c.deregChecks = map[string]struct{}{} + c.regLock.Unlock() + + var err error + + regServiceN, regCheckN, deregServiceN, deregCheckN := len(regServices), len(regChecks), len(deregServices), len(deregChecks) + + // Register Services + for id, service := range regServices { + if err = c.client.ServiceRegister(service); err != nil { + goto ERROR + } + delete(regServices, id) + } + + // Register Checks + for id, check := range regChecks { + if err = c.client.CheckRegister(check); err != nil { + goto ERROR + } + delete(regChecks, id) + + // Run the script for this check if one exists + if script, ok := regScripts[id]; ok { + // This check is a script check; run it + c.runningScripts[id] = script.run() + } + } + + // Deregister Checks + for id := range deregChecks { + if h, ok := c.runningScripts[id]; ok { + // This check is a script check; stop it + h.cancel() + delete(c.runningScripts, id) + } + + if err = c.client.CheckDeregister(id); err != nil { + goto ERROR + } + delete(deregChecks, id) + } + + // Deregister Services + for id := range deregServices { + if err = c.client.ServiceDeregister(id); err != nil { + goto ERROR + } + delete(deregServices, id) + } + + c.logger.Printf("[DEBUG] consul: registered %d services / %d checks; deregisterd %d services / %d checks", regServiceN, regCheckN, deregServiceN, deregCheckN) + return nil + + //TODO Labels and gotos are nasty; move to a function? +ERROR: + // An error occurred, repopulate the operation maps omitting any keys + // that have been updated while sync() ran. + c.regLock.Lock() + for id, service := range regServices { + if _, ok := c.regServices[id]; ok { + continue + } + if _, ok := c.deregServices[id]; ok { + continue + } + c.regServices[id] = service + } + for id, check := range regChecks { + if _, ok := c.regChecks[id]; ok { + continue + } + if _, ok := c.deregChecks[id]; ok { + continue + } + c.regChecks[id] = check + } + for id, script := range regScripts { + if _, ok := c.regScripts[id]; ok { + // a new version of this script was added, drop this one + continue + } + c.regScripts[id] = script + } + for id, _ := range deregServices { + if _, ok := c.regServices[id]; ok { + continue + } + c.deregServices[id] = mark + } + for id, _ := range deregChecks { + if _, ok := c.regChecks[id]; ok { + continue + } + c.deregChecks[id] = mark + } + c.regLock.Unlock() + return err +} + +// RegisterAgent registers Nomad agents (client or server). Script checks are +// not supported and will return an error. Registration is asynchronous. +// +// Agents will be deregistered when Shutdown is called. +func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) error { + regs := make([]*api.AgentServiceRegistration, len(services)) + checks := make([]*api.AgentCheckRegistration, 0, len(services)) + + for i, service := range services { + id := makeAgentServiceID(role, service) + host, rawport, err := net.SplitHostPort(service.PortLabel) + if err != nil { + return fmt.Errorf("error parsing port label %q from service %q: %v", service.PortLabel, service.Name, err) + } + port, err := strconv.Atoi(rawport) + if err != nil { + return fmt.Errorf("error parsing port %q from service %q: %v", rawport, service.Name, err) + } + serviceReg := &api.AgentServiceRegistration{ + ID: id, + Name: service.Name, + Tags: service.Tags, + Address: host, + Port: port, + } + regs[i] = serviceReg + + for _, check := range service.Checks { + checkID := createCheckID(id, check) + if check.Type == structs.ServiceCheckScript { + return fmt.Errorf("service %q contains invalid check: agent checks do not support scripts", service.Name) + } + checkHost, checkPort := serviceReg.Address, serviceReg.Port + if check.PortLabel != "" { + host, rawport, err := net.SplitHostPort(check.PortLabel) + if err != nil { + return fmt.Errorf("error parsing port label %q from check %q: %v", service.PortLabel, check.Name, err) + } + port, err := strconv.Atoi(rawport) + if err != nil { + return fmt.Errorf("error parsing port %q from check %q: %v", rawport, check.Name, err) + } + checkHost, checkPort = host, port + } + checkReg, err := createCheckReg(id, checkID, check, checkHost, checkPort) + if err != nil { + return fmt.Errorf("failed to add check %q: %v", check.Name, err) + } + checks = append(checks, checkReg) + } + } + + // Now add them to the registration queue + c.enqueueRegs(regs, checks, nil) + + // Record IDs for deregistering on shutdown + c.agentLock.Lock() + for _, s := range regs { + c.agentServices[s.ID] = mark + } + for _, ch := range checks { + c.agentChecks[ch.ID] = mark + } + c.agentLock.Unlock() + return nil +} + +// RegisterTask with Consul. Adds all sevice entries and checks to Consul. If +// exec is nil and a script check exists an error is returned. +// +// Actual communication with Consul is done asynchrously (see Run). +func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec ScriptExecutor) error { + regs := make([]*api.AgentServiceRegistration, len(task.Services)) + checks := make([]*api.AgentCheckRegistration, 0, len(task.Services)*2) // just guess at size + var scriptChecks []*scriptCheck + + for i, service := range task.Services { + id := makeTaskServiceID(allocID, task.Name, service) + host, port := task.FindHostAndPortFor(service.PortLabel) + serviceReg := &api.AgentServiceRegistration{ + ID: id, + Name: service.Name, + Tags: make([]string, len(service.Tags)), + Address: host, + Port: port, + } + // copy isn't strictly necessary but can avoid bugs especially + // with tests that may reuse Tasks + copy(serviceReg.Tags, service.Tags) + regs[i] = serviceReg + + for _, check := range service.Checks { + checkID := createCheckID(id, check) + if check.Type == structs.ServiceCheckScript { + if exec == nil { + return fmt.Errorf("driver %q doesn't support script checks", task.Driver) + } + scriptChecks = append(scriptChecks, newScriptCheck(checkID, check, exec, c.client, c.logger, c.shutdownCh)) + } + host, port := serviceReg.Address, serviceReg.Port + if check.PortLabel != "" { + host, port = task.FindHostAndPortFor(check.PortLabel) + } + checkReg, err := createCheckReg(id, checkID, check, host, port) + if err != nil { + return fmt.Errorf("failed to add check %q: %v", check.Name, err) + } + checks = append(checks, checkReg) + } + + } + + // Now add them to the registration queue + c.enqueueRegs(regs, checks, scriptChecks) + return nil +} + +// RemoveTask from Consul. Removes all service entries and checks. +// +// Actual communication with Consul is done asynchrously (see Run). +func (c *ServiceClient) RemoveTask(allocID string, task *structs.Task) { + deregs := make([]string, len(task.Services)) + checks := make([]string, 0, len(task.Services)*2) // just guess at size + + for i, service := range task.Services { + id := makeTaskServiceID(allocID, task.Name, service) + deregs[i] = id + + for _, check := range service.Checks { + checks = append(checks, createCheckID(id, check)) + } + } + + // Now add them to the deregistration fields; main Run loop will update + c.enqueueDeregs(deregs, checks) +} + +// enqueueRegs enqueues service and check registrations for the next time +// operations are sync'd to Consul. +func (c *ServiceClient) enqueueRegs(regs []*api.AgentServiceRegistration, checks []*api.AgentCheckRegistration, scriptChecks []*scriptCheck) { + c.regLock.Lock() + for _, reg := range regs { + // Add reg + c.regServices[reg.ID] = reg + // Make sure it's not being removed + delete(c.deregServices, reg.ID) + } + for _, check := range checks { + // Add check + c.regChecks[check.ID] = check + // Make sure it's not being removed + delete(c.deregChecks, check.ID) + } + for _, script := range scriptChecks { + c.regScripts[script.id] = script + } + c.regLock.Unlock() + + c.forceSync() +} + +// enqueueDeregs enqueues service and check removals for the next time +// operations are sync'd to Consul. +func (c *ServiceClient) enqueueDeregs(deregs []string, checks []string) { + c.regLock.Lock() + for _, dereg := range deregs { + // Add dereg + c.deregServices[dereg] = mark + // Make sure it's not being added + delete(c.regServices, dereg) + } + for _, check := range checks { + // Add check for removal + c.deregChecks[check] = mark + // Make sure it's not being added + delete(c.regChecks, check) + } + c.regLock.Unlock() + + c.forceSync() +} + +// Shutdown the Consul client. Update running task registations and deregister +// agent from Consul. Blocks up to shutdownWait before giving up on syncing +// operations. +func (c *ServiceClient) Shutdown() error { + select { + case <-c.shutdownCh: + return nil + default: + close(c.shutdownCh) + } + + var mErr multierror.Error + + // Don't let Shutdown block indefinitely + deadline := time.After(c.shutdownWait) + + // Deregister agent services and checks + c.agentLock.Lock() + for id := range c.agentServices { + if err := c.client.ServiceDeregister(id); err != nil { + mErr.Errors = append(mErr.Errors, err) + } + } + + // Deregister Checks + for id := range c.agentChecks { + if err := c.client.CheckDeregister(id); err != nil { + mErr.Errors = append(mErr.Errors, err) + } + } + c.agentLock.Unlock() + + // Wait for Run to finish any outstanding sync() calls and exit + select { + case <-c.runningCh: + // sync one last time to ensure all enqueued operations are applied + if err := c.sync(); err != nil { + mErr.Errors = append(mErr.Errors, err) + } + case <-deadline: + // Don't wait forever though + mErr.Errors = append(mErr.Errors, fmt.Errorf("timed out waiting for Consul operations to complete")) + return mErr.ErrorOrNil() + } + + // Give script checks time to exit (no need to lock as Run() has exited) + for _, h := range c.runningScripts { + select { + case <-h.wait(): + case <-deadline: + mErr.Errors = append(mErr.Errors, fmt.Errorf("timed out waiting for script checks to run")) + return mErr.ErrorOrNil() + } + } + return mErr.ErrorOrNil() +} + +// makeAgentServiceID creates a unique ID for identifying an agent service in +// Consul. +// +// Agent service IDs are of the form: +// +// {nomadServicePrefix}-{ROLE}-{Service.Name}-{Service.Tags...} +// Example Server ID: _nomad-server-nomad-serf +// Example Client ID: _nomad-client-nomad-client-http +// +func makeAgentServiceID(role string, service *structs.Service) string { + parts := make([]string, len(service.Tags)+3) + parts[0] = nomadServicePrefix + parts[1] = role + parts[2] = service.Name + copy(parts[3:], service.Tags) + return strings.Join(parts, "-") +} + +// makeTaskServiceID creates a unique ID for identifying a task service in +// Consul. +// +// Task service IDs are of the form: +// +// {nomadServicePrefix}-executor-{ALLOC_ID}-{Service.Name}-{Service.Tags...} +// Example Service ID: _nomad-executor-1234-echo-http-tag1-tag2-tag3 +// +func makeTaskServiceID(allocID, taskName string, service *structs.Service) string { + parts := make([]string, len(service.Tags)+5) + parts[0] = nomadServicePrefix + parts[1] = "executor" + parts[2] = allocID + parts[3] = taskName + parts[4] = service.Name + copy(parts[5:], service.Tags) + return strings.Join(parts, "-") +} + +// createCheckID creates a unique ID for a check. +func createCheckID(serviceID string, check *structs.ServiceCheck) string { + return check.Hash(serviceID) +} + +// createCheckReg creates a Check that can be registered with Consul. +// +// Only supports HTTP(S) and TCP checks. Script checks must be handled +// externally. +func createCheckReg(serviceID, checkID string, check *structs.ServiceCheck, host string, port int) (*api.AgentCheckRegistration, error) { + chkReg := api.AgentCheckRegistration{ + ID: checkID, + Name: check.Name, + ServiceID: serviceID, + } + chkReg.Status = check.InitialStatus + chkReg.Timeout = check.Timeout.String() + chkReg.Interval = check.Interval.String() + + switch check.Type { + case structs.ServiceCheckHTTP: + if check.Protocol == "" { + check.Protocol = "http" + } + base := url.URL{ + Scheme: check.Protocol, + Host: net.JoinHostPort(host, strconv.Itoa(port)), + } + relative, err := url.Parse(check.Path) + if err != nil { + return nil, err + } + url := base.ResolveReference(relative) + chkReg.HTTP = url.String() + case structs.ServiceCheckTCP: + chkReg.TCP = net.JoinHostPort(host, strconv.Itoa(port)) + case structs.ServiceCheckScript: + chkReg.TTL = (check.Interval + ttlCheckBuffer).String() + default: + return nil, fmt.Errorf("check type %+q not valid", check.Type) + } + return &chkReg, nil +} diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go new file mode 100644 index 000000000000..2f0239d5f780 --- /dev/null +++ b/command/agent/consul/int_test.go @@ -0,0 +1,228 @@ +package consul_test + +import ( + "io/ioutil" + "log" + "os" + "path/filepath" + "testing" + "time" + + "golang.org/x/sys/unix" + + consulapi "github.com/hashicorp/consul/api" + "github.com/hashicorp/consul/testutil" + "github.com/hashicorp/nomad/client" + "github.com/hashicorp/nomad/client/allocdir" + "github.com/hashicorp/nomad/client/config" + "github.com/hashicorp/nomad/client/driver" + "github.com/hashicorp/nomad/client/vaultclient" + "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" +) + +func testLogger() *log.Logger { + if testing.Verbose() { + return log.New(os.Stderr, "", log.LstdFlags) + } + return log.New(ioutil.Discard, "", 0) +} + +// TestConsul_Integration asserts TaskRunner properly registers and deregisters +// services and checks with Consul using an embedded Consul agent. +func TestConsul_Integration(t *testing.T) { + if _, ok := driver.BuiltinDrivers["mock_driver"]; !ok { + t.Skip(`test requires mock_driver; run with "-tags nomad_test"`) + } + if testing.Short() { + t.Skip("-short set; skipping") + } + if unix.Geteuid() != 0 { + t.Skip("Must be run as root") + } + // Create an embedded Consul server + testconsul := testutil.NewTestServerConfig(t, func(c *testutil.TestServerConfig) { + // If -v wasn't specified squelch consul logging + if !testing.Verbose() { + c.Stdout = ioutil.Discard + c.Stderr = ioutil.Discard + } + }) + defer testconsul.Stop() + + conf := config.DefaultConfig() + conf.ConsulConfig.Addr = testconsul.HTTPAddr + consulConfig, err := conf.ConsulConfig.ApiConfig() + if err != nil { + t.Fatalf("error generating consul config: %v", err) + } + + conf.StateDir, err = ioutil.TempDir("", "nomadtest-consulstate") + if err != nil { + t.Fatalf("error creating temp dir: %v", err) + } + defer os.RemoveAll(conf.StateDir) + conf.AllocDir, err = ioutil.TempDir("", "nomdtest-consulalloc") + if err != nil { + t.Fatalf("error creating temp dir: %v", err) + } + defer os.RemoveAll(conf.AllocDir) + + alloc := mock.Alloc() + task := alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config = map[string]interface{}{ + "run_for": "1h", + } + // Choose a port that shouldn't be in use + task.Resources.Networks[0].ReservedPorts = []structs.Port{{Label: "http", Value: 3}} + task.Services = []*structs.Service{ + { + Name: "httpd", + PortLabel: "http", + Tags: []string{"nomad", "test", "http"}, + Checks: []*structs.ServiceCheck{ + { + Name: "httpd-http-check", + Type: "http", + Path: "/", + Protocol: "http", + PortLabel: "http", + Interval: 9000 * time.Hour, + Timeout: 1, // fail as fast as possible + }, + { + Name: "httpd-script-check", + Type: "script", + Command: "/bin/true", + Interval: 10 * time.Second, + Timeout: 10 * time.Second, + }, + }, + }, + { + Name: "httpd2", + PortLabel: "http", + Tags: []string{"test", "http2"}, + }, + } + + logger := testLogger() + logUpdate := func(name, state string, event *structs.TaskEvent) { + logger.Printf("[TEST] test.updater: name=%q state=%q event=%v", name, state, event) + } + allocDir := allocdir.NewAllocDir(logger, filepath.Join(conf.AllocDir, alloc.ID)) + if err := allocDir.Build(); err != nil { + t.Fatalf("error building alloc dir: %v", err) + } + taskDir := allocDir.NewTaskDir(task.Name) + vclient := vaultclient.NewMockVaultClient() + consulClient, err := consulapi.NewClient(consulConfig) + if err != nil { + t.Fatalf("error creating consul client: %v", err) + } + serviceClient := consul.NewServiceClient(consulClient.Agent(), logger) + defer serviceClient.Shutdown() // just-in-case cleanup + consulRan := make(chan struct{}) + go func() { + serviceClient.Run() + close(consulRan) + }() + tr := client.NewTaskRunner(logger, conf, logUpdate, taskDir, alloc, task, vclient, serviceClient) + tr.MarkReceived() + go tr.Run() + defer func() { + // Just in case cleanup + select { + case <-tr.WaitCh(): + // Exited cleanly, no need to kill + default: + tr.Kill("", "", true) // just in case + } + }() + + // Block waiting for the service to appear + catalog := consulClient.Catalog() + res, meta, err := catalog.Service("httpd2", "test", nil) + for len(res) == 0 { + //Expected initial request to fail, do a blocking query + res, meta, err = catalog.Service("httpd2", "test", &consulapi.QueryOptions{WaitIndex: meta.LastIndex + 1, WaitTime: 3 * time.Second}) + if err != nil { + t.Fatalf("error querying for service: %v", err) + } + } + if len(res) != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", len(res), res) + } + res = res[:] + + // Assert the service with the checks exists + for len(res) == 0 { + res, meta, err = catalog.Service("httpd", "http", &consulapi.QueryOptions{WaitIndex: meta.LastIndex + 1, WaitTime: 3 * time.Second}) + if err != nil { + t.Fatalf("error querying for service: %v", err) + } + } + if len(res) != 1 { + t.Fatalf("exepcted 1 service but found %d:\n%#v", len(res), res) + } + + // Assert the script check passes (mock_driver script checks always + // pass) after having time to run once + time.Sleep(2 * time.Second) + checks, _, err := consulClient.Health().Checks("httpd", nil) + if err != nil { + t.Fatalf("error querying checks: %v", err) + } + if expected := 2; len(checks) != expected { + t.Fatalf("expected %d checks but found %d:\n%#v", expected, len(checks), checks) + } + for _, check := range checks { + if expected := "httpd"; check.ServiceName != expected { + t.Fatalf("expected checks to be for %q but found service name = %q", expected, check.ServiceName) + } + switch check.Name { + case "httpd-http-check": + // Port check should fail + if expected := consulapi.HealthCritical; check.Status != expected { + t.Errorf("expected %q status to be %q but found %q", check.Name, expected, check.Status) + } + case "httpd-script-check": + // mock_driver script checks always succeed + if expected := consulapi.HealthPassing; check.Status != expected { + t.Errorf("expected %q status to be %q but found %q", check.Name, expected, check.Status) + } + default: + t.Errorf("unexpected check %q with status %q", check.Name, check.Status) + } + } + + logger.Printf("[TEST] consul.test: killing task") + + // Kill the task + tr.Kill("", "", false) + + select { + case <-tr.WaitCh(): + case <-time.After(10 * time.Second): + t.Fatalf("timed out waiting for Run() to exit") + } + + // Shutdown Consul ServiceClient to ensure all pending operations complete + if err := serviceClient.Shutdown(); err != nil { + t.Errorf("error shutting down Consul ServiceClient: %v", err) + } + + // Ensure Consul is clean + services, _, err := catalog.Services(nil) + if err != nil { + t.Fatalf("error query services: %v", err) + } + if len(services) != 1 { + t.Fatalf("expected only 1 service in Consul but found %d:\n%#v", len(services), services) + } + if _, ok := services["consul"]; !ok { + t.Fatalf(`expected only the "consul" key in Consul but found: %#v`, services) + } +} diff --git a/command/agent/consul/mock.go b/command/agent/consul/mock.go new file mode 100644 index 000000000000..f0dd0326ce0f --- /dev/null +++ b/command/agent/consul/mock.go @@ -0,0 +1,27 @@ +package consul + +import ( + "log" + + "github.com/hashicorp/consul/api" +) + +// MockCatalog can be used for testing where the CatalogAPI is needed. +type MockCatalog struct { + logger *log.Logger +} + +func NewMockCatalog(l *log.Logger) *MockCatalog { + return &MockCatalog{logger: l} +} + +func (m *MockCatalog) Datacenters() ([]string, error) { + dcs := []string{"dc1"} + m.logger.Printf("[DEBUG] mock_consul: Datacenters() -> (%q, nil)", dcs) + return dcs, nil +} + +func (m *MockCatalog) Service(service, tag string, q *api.QueryOptions) ([]*api.CatalogService, *api.QueryMeta, error) { + m.logger.Printf("[DEBUG] mock_consul: Service(%q, %q, %#v) -> (nil, nil, nil)", service, tag, q) + return nil, nil, nil +} diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go new file mode 100644 index 000000000000..1c63877a3a7a --- /dev/null +++ b/command/agent/consul/script.go @@ -0,0 +1,136 @@ +package consul + +import ( + "context" + "log" + "time" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/nomad/structs" +) + +// heartbeater is the subset of consul agent functionality needed by script +// checks to heartbeat +type heartbeater interface { + UpdateTTL(id, output, status string) error +} + +type scriptHandle struct { + // cancel the script + cancel func() + done chan struct{} +} + +// wait returns a chan that's closed when the script exits +func (s *scriptHandle) wait() <-chan struct{} { + return s.done +} + +type scriptCheck struct { + id string + check *structs.ServiceCheck + exec ScriptExecutor + agent heartbeater + + // lastCheckOk is true if the last check was ok; otherwise false + lastCheckOk bool + + logger *log.Logger + shutdownCh <-chan struct{} +} + +func newScriptCheck(id string, check *structs.ServiceCheck, exec ScriptExecutor, agent heartbeater, logger *log.Logger, shutdownCh <-chan struct{}) *scriptCheck { + return &scriptCheck{ + id: id, + check: check, + exec: exec, + agent: agent, + lastCheckOk: true, // start logging on first failure + logger: logger, + shutdownCh: shutdownCh, + } +} + +// run this script check and return its cancel func. If the shutdownCh is +// closed the check will be run once more before exiting. +func (s *scriptCheck) run() *scriptHandle { + ctx, cancel := context.WithCancel(context.Background()) + done := make(chan struct{}) + go func() { + defer close(done) + timer := time.NewTimer(0) + defer timer.Stop() + for { + // Block until check is removed, Nomad is shutting + // down, or the check interval is up + select { + case <-ctx.Done(): + // check has been removed + return + case <-s.shutdownCh: + // unblock but don't exit until after we heartbeat once more + case <-timer.C: + timer.Reset(s.check.Interval) + } + + // Execute check script with timeout + execctx, cancel := context.WithTimeout(ctx, s.check.Timeout) + output, code, err := s.exec.Exec(execctx, s.check.Command, s.check.Args) + switch execctx.Err() { + case context.Canceled: + // check removed during execution; exit + return + case context.DeadlineExceeded: + // Log deadline exceeded every time, but flip last check to false + s.lastCheckOk = false + s.logger.Printf("[WARN] consul.checks: check %q timed out (%s)", s.check.Name, s.check.Timeout) + } + // cleanup context + cancel() + + state := api.HealthCritical + switch code { + case 0: + state = api.HealthPassing + case 1: + state = api.HealthWarning + } + if err != nil { + state = api.HealthCritical + output = []byte(err.Error()) + } + + // Actually heartbeat the check + err = s.agent.UpdateTTL(s.id, string(output), state) + select { + case <-ctx.Done(): + // check has been removed; don't report errors + return + default: + } + + if err != nil { + //FIXME Backoff? Retry faster? + if s.lastCheckOk { + s.lastCheckOk = false + s.logger.Printf("[WARN] consul.checks: update for check %q failed: %v", s.check.Name, err) + } else { + s.logger.Printf("[DEBUG] consul.checks: update for check %q still failing: %v", s.check.Name, err) + } + + } else if !s.lastCheckOk { + // Succeeded for the first time or after failing; log + s.lastCheckOk = true + s.logger.Printf("[INFO] consul.checks: update for check %q succeeded", s.check.Name) + } + + select { + case <-s.shutdownCh: + // We've been told to exit + return + default: + } + } + }() + return &scriptHandle{cancel: cancel, done: done} +} diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go new file mode 100644 index 000000000000..a9bda3e44499 --- /dev/null +++ b/command/agent/consul/script_test.go @@ -0,0 +1,165 @@ +package consul + +import ( + "context" + "os/exec" + "testing" + "time" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/nomad/structs" +) + +// blockingScriptExec implements ScriptExec by running a subcommand that never +// exits. +type blockingScriptExec struct { + // running is ticked before blocking to allow synchronizing operations + running chan struct{} + + // set to true if Exec is called and has exited + exited bool +} + +func newBlockingScriptExec() *blockingScriptExec { + return &blockingScriptExec{running: make(chan struct{})} +} + +func (b *blockingScriptExec) Exec(ctx context.Context, _ string, _ []string) ([]byte, int, error) { + b.running <- mark + cmd := exec.CommandContext(ctx, "/bin/sleep", "9000") + err := cmd.Run() + code := 0 + if exitErr, ok := err.(*exec.ExitError); ok { + if !exitErr.Success() { + code = 1 + } + } + b.exited = true + return []byte{}, code, err +} + +// TestConsulScript_Exec_Cancel asserts cancelling a script check shortcircuits +// any running scripts. +func TestConsulScript_Exec_Cancel(t *testing.T) { + serviceCheck := structs.ServiceCheck{ + Name: "sleeper", + Interval: time.Hour, + Timeout: time.Hour, + } + exec := newBlockingScriptExec() + + // pass nil for heartbeater as it shouldn't be called + check := newScriptCheck("checkid", &serviceCheck, exec, nil, testLogger(), nil) + handle := check.run() + + // wait until Exec is called + <-exec.running + + // cancel now that we're blocked in exec + handle.cancel() + + select { + case <-handle.wait(): + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exit") + } + if !exec.exited { + t.Errorf("expected script executor to run and exit but it has not") + } +} + +type fakeHeartbeater struct { + updates chan string +} + +func (f *fakeHeartbeater) UpdateTTL(checkID, output, status string) error { + f.updates <- status + return nil +} + +func newFakeHeartbeater() *fakeHeartbeater { + return &fakeHeartbeater{updates: make(chan string)} +} + +// TestConsulScript_Exec_Timeout asserts a script will be killed when the +// timeout is reached. +func TestConsulScript_Exec_Timeout(t *testing.T) { + t.Parallel() // run the slow tests in parallel + serviceCheck := structs.ServiceCheck{ + Name: "sleeper", + Interval: time.Hour, + Timeout: time.Second, + } + exec := newBlockingScriptExec() + + hb := newFakeHeartbeater() + check := newScriptCheck("checkid", &serviceCheck, exec, hb, testLogger(), nil) + handle := check.run() + defer handle.cancel() // just-in-case cleanup + <-exec.running + + // Check for UpdateTTL call + select { + case update := <-hb.updates: + if update != api.HealthCritical { + t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) + } + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exit") + } + if !exec.exited { + t.Errorf("expected script executor to run and exit but it has not") + } + + // Cancel and watch for exit + handle.cancel() + select { + case <-handle.wait(): + // ok! + case update := <-hb.updates: + t.Errorf("unexpected UpdateTTL call on exit with status=%q", update) + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exit") + } +} + +type noopExec struct{} + +func (noopExec) Exec(context.Context, string, []string) ([]byte, int, error) { + return []byte{}, 0, nil +} + +// TestConsulScript_Exec_Shutdown asserts a script will be executed once more +// when told to shutdown. +func TestConsulScript_Exec_Shutdown(t *testing.T) { + serviceCheck := structs.ServiceCheck{ + Name: "sleeper", + Interval: time.Hour, + Timeout: 3 * time.Second, + } + + hb := newFakeHeartbeater() + shutdown := make(chan struct{}) + check := newScriptCheck("checkid", &serviceCheck, noopExec{}, hb, testLogger(), shutdown) + handle := check.run() + defer handle.cancel() // just-in-case cleanup + + // Tell scriptCheck to exit + close(shutdown) + + select { + case update := <-hb.updates: + if update != api.HealthPassing { + t.Error("expected %q due to timeout but received %q", api.HealthPassing, update) + } + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exit") + } + + select { + case <-handle.wait(): + // ok! + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exit") + } +} diff --git a/command/agent/consul/syncer.go b/command/agent/consul/syncer.go deleted file mode 100644 index c111f3aafcb3..000000000000 --- a/command/agent/consul/syncer.go +++ /dev/null @@ -1,1016 +0,0 @@ -// Package consul is used by Nomad to register all services both static services -// and dynamic via allocations. -// -// Consul Service IDs have the following format: ${nomadServicePrefix}-${groupName}-${serviceKey} -// groupName takes on one of the following values: -// - server -// - client -// - executor-${alloc-id}-${task-name} -// -// serviceKey should be generated by service registrators. -// If the serviceKey is being generated by the executor for a Nomad Task.Services -// the following helper should be used: -// NOTE: Executor should interpolate the service prior to calling -// func GenerateTaskServiceKey(service *structs.Service) string -// -// The Nomad Client reaps services registered from dead allocations that were -// not properly cleaned up by the executor (this is not the expected case). -// -// TODO fix this comment -// The Consul ServiceIDs generated by the executor will contain the allocation -// ID. Thus the client can generate the list of Consul ServiceIDs to keep by -// calling the following method on all running allocations the client is aware -// of: -// func GenerateExecutorServiceKeyPrefixFromAlloc(allocID string) string -package consul - -import ( - "fmt" - "log" - "net" - "net/url" - "strconv" - "strings" - "sync" - "time" - - consul "github.com/hashicorp/consul/api" - "github.com/hashicorp/go-multierror" - - "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/nomad/structs/config" - "github.com/hashicorp/nomad/nomad/types" -) - -const ( - // initialSyncBuffer is the max time an initial sync will sleep - // before syncing. - initialSyncBuffer = 30 * time.Second - - // initialSyncDelay is the delay before an initial sync. - initialSyncDelay = 5 * time.Second - - // nomadServicePrefix is the first prefix that scopes all Nomad registered - // services - nomadServicePrefix = "_nomad" - - // The periodic time interval for syncing services and checks with Consul - defaultSyncInterval = 6 * time.Second - - // defaultSyncJitter provides a little variance in the frequency at which - // Syncer polls Consul. - defaultSyncJitter = time.Second - - // ttlCheckBuffer is the time interval that Nomad can take to report Consul - // the check result - ttlCheckBuffer = 31 * time.Second - - // DefaultQueryWaitDuration is the max duration the Consul Agent will - // spend waiting for a response from a Consul Query. - DefaultQueryWaitDuration = 2 * time.Second - - // ServiceTagHTTP is the tag assigned to HTTP services - ServiceTagHTTP = "http" - - // ServiceTagRPC is the tag assigned to RPC services - ServiceTagRPC = "rpc" - - // ServiceTagSerf is the tag assigned to Serf services - ServiceTagSerf = "serf" -) - -// consulServiceID and consulCheckID are the IDs registered with Consul -type consulServiceID string -type consulCheckID string - -// ServiceKey is the generated service key that is used to build the Consul -// ServiceID -type ServiceKey string - -// ServiceDomain is the domain of services registered by Nomad -type ServiceDomain string - -const ( - ClientDomain ServiceDomain = "client" - ServerDomain ServiceDomain = "server" -) - -// NewExecutorDomain returns a domain specific to the alloc ID and task -func NewExecutorDomain(allocID, task string) ServiceDomain { - return ServiceDomain(fmt.Sprintf("executor-%s-%s", allocID, task)) -} - -// Syncer allows syncing of services and checks with Consul -type Syncer struct { - client *consul.Client - consulAvailable bool - - // servicesGroups and checkGroups are named groups of services and checks - // respectively that will be flattened and reconciled with Consul when - // SyncServices() is called. The key to the servicesGroups map is unique - // per handler and is used to allow the Agent's services to be maintained - // independently of the Client or Server's services. - servicesGroups map[ServiceDomain]map[ServiceKey]*consul.AgentServiceRegistration - checkGroups map[ServiceDomain]map[ServiceKey][]*consul.AgentCheckRegistration - groupsLock sync.RWMutex - - // The "Consul Registry" is a collection of Consul Services and - // Checks all guarded by the registryLock. - registryLock sync.RWMutex - - // checkRunners are delegated Consul checks being ran by the Syncer - checkRunners map[consulCheckID]*CheckRunner - - addrFinder func(portLabel string) (string, int) - createDelegatedCheck func(*structs.ServiceCheck, string) (Check, error) - delegateChecks map[string]struct{} // delegateChecks are the checks that the Nomad client runs and reports to Consul - // End registryLock guarded attributes. - - logger *log.Logger - - shutdownCh chan struct{} - shutdown bool - shutdownLock sync.Mutex - - // notifyShutdownCh is used to notify a Syncer it needs to shutdown. - // This can happen because there was an explicit call to the Syncer's - // Shutdown() method, or because the calling task signaled the - // program is going to exit by closing its shutdownCh. - notifyShutdownCh chan struct{} - - // periodicCallbacks is walked sequentially when the timer in Run - // fires. - periodicCallbacks map[string]types.PeriodicCallback - notifySyncCh chan struct{} - periodicLock sync.RWMutex - - // The periodic time interval for syncing services and checks with Consul - syncInterval time.Duration - - // syncJitter provides a little variance in the frequency at which - // Syncer polls Consul. - syncJitter time.Duration -} - -// NewSyncer returns a new consul.Syncer -func NewSyncer(consulConfig *config.ConsulConfig, shutdownCh chan struct{}, logger *log.Logger) (*Syncer, error) { - var consulClientConfig *consul.Config - var err error - consulClientConfig, err = consulConfig.ApiConfig() - if err != nil { - return nil, err - } - - var consulClient *consul.Client - if consulClient, err = consul.NewClient(consulClientConfig); err != nil { - return nil, err - } - consulSyncer := Syncer{ - client: consulClient, - logger: logger, - consulAvailable: true, - shutdownCh: shutdownCh, - servicesGroups: make(map[ServiceDomain]map[ServiceKey]*consul.AgentServiceRegistration), - checkGroups: make(map[ServiceDomain]map[ServiceKey][]*consul.AgentCheckRegistration), - checkRunners: make(map[consulCheckID]*CheckRunner), - periodicCallbacks: make(map[string]types.PeriodicCallback), - notifySyncCh: make(chan struct{}, 1), - // default noop implementation of addrFinder - addrFinder: func(string) (string, int) { return "", 0 }, - syncInterval: defaultSyncInterval, - syncJitter: defaultSyncJitter, - } - - return &consulSyncer, nil -} - -// SetDelegatedChecks sets the checks that nomad is going to run and report the -// result back to consul -func (c *Syncer) SetDelegatedChecks(delegateChecks map[string]struct{}, createDelegatedCheckFn func(*structs.ServiceCheck, string) (Check, error)) *Syncer { - c.delegateChecks = delegateChecks - c.createDelegatedCheck = createDelegatedCheckFn - return c -} - -// SetAddrFinder sets a function to find the host and port for a Service given its port label -func (c *Syncer) SetAddrFinder(addrFinder func(string) (string, int)) *Syncer { - c.addrFinder = addrFinder - return c -} - -// GenerateServiceKey should be called to generate a serviceKey based on the -// Service. -func GenerateServiceKey(service *structs.Service) ServiceKey { - var key string - numTags := len(service.Tags) - switch numTags { - case 0: - key = fmt.Sprintf("%s", service.Name) - default: - tags := strings.Join(service.Tags, "-") - key = fmt.Sprintf("%s-%s", service.Name, tags) - } - return ServiceKey(key) -} - -// SetServices stores the map of Nomad Services to the provided service -// domain name. -func (c *Syncer) SetServices(domain ServiceDomain, services map[ServiceKey]*structs.Service) error { - var mErr multierror.Error - numServ := len(services) - registeredServices := make(map[ServiceKey]*consul.AgentServiceRegistration, numServ) - registeredChecks := make(map[ServiceKey][]*consul.AgentCheckRegistration, numServ) - for serviceKey, service := range services { - serviceReg, err := c.createService(service, domain, serviceKey) - if err != nil { - mErr.Errors = append(mErr.Errors, err) - continue - } - registeredServices[serviceKey] = serviceReg - - // Register the check(s) for this service - for _, chk := range service.Checks { - // Create a Consul check registration - chkReg, err := c.createCheckReg(chk, serviceReg) - if err != nil { - mErr.Errors = append(mErr.Errors, err) - continue - } - - // creating a nomad check if we have to handle this particular check type - c.registryLock.RLock() - if _, ok := c.delegateChecks[chk.Type]; ok { - _, ok := c.checkRunners[consulCheckID(chkReg.ID)] - c.registryLock.RUnlock() - if ok { - continue - } - - nc, err := c.createDelegatedCheck(chk, chkReg.ID) - if err != nil { - mErr.Errors = append(mErr.Errors, err) - continue - } - - cr := NewCheckRunner(nc, c.runCheck, c.logger) - c.registryLock.Lock() - // TODO type the CheckRunner - c.checkRunners[consulCheckID(nc.ID())] = cr - c.registryLock.Unlock() - } else { - c.registryLock.RUnlock() - } - - registeredChecks[serviceKey] = append(registeredChecks[serviceKey], chkReg) - } - } - - if len(mErr.Errors) > 0 { - return mErr.ErrorOrNil() - } - - // Update the services and checks groups for this domain - c.groupsLock.Lock() - - // Create map for service group if it doesn't exist - serviceKeys, ok := c.servicesGroups[domain] - if !ok { - serviceKeys = make(map[ServiceKey]*consul.AgentServiceRegistration, len(registeredServices)) - c.servicesGroups[domain] = serviceKeys - } - - // Remove stale services - for existingServiceKey := range serviceKeys { - if _, ok := registeredServices[existingServiceKey]; !ok { - // Exisitng service needs to be removed - delete(serviceKeys, existingServiceKey) - } - } - - // Add registered services - for serviceKey, service := range registeredServices { - serviceKeys[serviceKey] = service - } - - // Create map for check group if it doesn't exist - checkKeys, ok := c.checkGroups[domain] - if !ok { - checkKeys = make(map[ServiceKey][]*consul.AgentCheckRegistration, len(registeredChecks)) - c.checkGroups[domain] = checkKeys - } - - // Remove stale checks - for existingCheckKey := range checkKeys { - if _, ok := registeredChecks[existingCheckKey]; !ok { - // Exisitng check needs to be removed - delete(checkKeys, existingCheckKey) - } - } - - // Add registered checks - for checkKey, checks := range registeredChecks { - checkKeys[checkKey] = checks - } - c.groupsLock.Unlock() - - // Sync immediately - c.SyncNow() - - return nil -} - -// SyncNow expires the current timer forcing the list of periodic callbacks -// to be synced immediately. -func (c *Syncer) SyncNow() { - select { - case c.notifySyncCh <- struct{}{}: - default: - } -} - -// flattenedServices returns a flattened list of services that are registered -// locally -func (c *Syncer) flattenedServices() []*consul.AgentServiceRegistration { - const initialNumServices = 8 - services := make([]*consul.AgentServiceRegistration, 0, initialNumServices) - c.groupsLock.RLock() - defer c.groupsLock.RUnlock() - for _, servicesGroup := range c.servicesGroups { - for _, service := range servicesGroup { - services = append(services, service) - } - } - return services -} - -// flattenedChecks returns a flattened list of checks that are registered -// locally -func (c *Syncer) flattenedChecks() []*consul.AgentCheckRegistration { - const initialNumChecks = 8 - checks := make([]*consul.AgentCheckRegistration, 0, initialNumChecks) - c.groupsLock.RLock() - for _, checkGroup := range c.checkGroups { - for _, check := range checkGroup { - checks = append(checks, check...) - } - } - c.groupsLock.RUnlock() - return checks -} - -func (c *Syncer) signalShutdown() { - select { - case c.notifyShutdownCh <- struct{}{}: - default: - } -} - -// Shutdown de-registers the services and checks and shuts down periodic syncing -func (c *Syncer) Shutdown() error { - var mErr multierror.Error - - c.shutdownLock.Lock() - if !c.shutdown { - c.shutdown = true - } - c.shutdownLock.Unlock() - - c.signalShutdown() - - // Stop all the checks that nomad is running - c.registryLock.RLock() - defer c.registryLock.RUnlock() - for _, cr := range c.checkRunners { - cr.Stop() - } - - // De-register all the services registered by this syncer from Consul - services, err := c.queryAgentServices() - if err != nil { - mErr.Errors = append(mErr.Errors, err) - } - for serviceID := range services { - convertedID := string(serviceID) - if err := c.client.Agent().ServiceDeregister(convertedID); err != nil { - c.logger.Printf("[WARN] consul.syncer: failed to deregister service ID %+q: %v", convertedID, err) - mErr.Errors = append(mErr.Errors, err) - } - } - return mErr.ErrorOrNil() -} - -// queryChecks queries the Consul Agent for a list of Consul checks that -// have been registered with this Consul Syncer. -func (c *Syncer) queryChecks() (map[consulCheckID]*consul.AgentCheck, error) { - checks, err := c.client.Agent().Checks() - if err != nil { - return nil, err - } - return c.filterConsulChecks(checks), nil -} - -// queryAgentServices queries the Consul Agent for a list of Consul services that -// have been registered with this Consul Syncer. -func (c *Syncer) queryAgentServices() (map[consulServiceID]*consul.AgentService, error) { - services, err := c.client.Agent().Services() - if err != nil { - return nil, err - } - return c.filterConsulServices(services), nil -} - -// syncChecks synchronizes this Syncer's Consul Checks with the Consul Agent. -func (c *Syncer) syncChecks() error { - var mErr multierror.Error - consulChecks, err := c.queryChecks() - if err != nil { - return err - } - - // Synchronize checks with Consul - missingChecks, existingChecks, changedChecks, staleChecks := c.calcChecksDiff(consulChecks) - for _, check := range missingChecks { - if err := c.registerCheck(check); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - for _, check := range existingChecks { - c.ensureCheckRunning(check) - } - for _, check := range changedChecks { - // NOTE(sean@): Do we need to deregister the check before - // re-registering it? Not deregistering to avoid missing the - // TTL but doesn't correct reconcile any possible drift with - // the check. - // - // if err := c.deregisterCheck(check.ID); err != nil { - // mErr.Errors = append(mErr.Errors, err) - // } - if err := c.registerCheck(check); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - for _, check := range staleChecks { - if err := c.deregisterCheck(consulCheckID(check.ID)); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - return mErr.ErrorOrNil() -} - -// compareConsulCheck takes a consul.AgentCheckRegistration instance and -// compares it with a consul.AgentCheck. Returns true if they are equal -// according to consul.AgentCheck, otherwise false. -func compareConsulCheck(localCheck *consul.AgentCheckRegistration, consulCheck *consul.AgentCheck) bool { - if consulCheck.CheckID != localCheck.ID || - consulCheck.Name != localCheck.Name || - consulCheck.Notes != localCheck.Notes || - consulCheck.ServiceID != localCheck.ServiceID { - return false - } - return true -} - -// calcChecksDiff takes the argument (consulChecks) and calculates the delta -// between the consul.Syncer's list of known checks (c.flattenedChecks()). -// Four arrays are returned: -// -// 1) a slice of checks that exist only locally in the Syncer and are missing -// from the Consul Agent (consulChecks) and therefore need to be registered. -// -// 2) a slice of checks that exist in both the local consul.Syncer's -// tracked list and Consul Agent (consulChecks). -// -// 3) a slice of checks that exist in both the local consul.Syncer's -// tracked list and Consul Agent (consulServices) but have diverged state. -// -// 4) a slice of checks that exist only in the Consul Agent (consulChecks) -// and should be removed because the Consul Agent has drifted from the -// Syncer. -func (c *Syncer) calcChecksDiff(consulChecks map[consulCheckID]*consul.AgentCheck) ( - missingChecks []*consul.AgentCheckRegistration, - equalChecks []*consul.AgentCheckRegistration, - changedChecks []*consul.AgentCheckRegistration, - staleChecks []*consul.AgentCheckRegistration) { - - type mergedCheck struct { - check *consul.AgentCheckRegistration - // 'l' == Nomad local only - // 'e' == equal - // 'c' == changed - // 'a' == Consul agent only - state byte - } - var ( - localChecksCount = 0 - equalChecksCount = 0 - changedChecksCount = 0 - agentChecks = 0 - ) - flattenedChecks := c.flattenedChecks() - localChecks := make(map[string]*mergedCheck, len(flattenedChecks)+len(consulChecks)) - for _, localCheck := range flattenedChecks { - localChecksCount++ - localChecks[localCheck.ID] = &mergedCheck{localCheck, 'l'} - } - for _, consulCheck := range consulChecks { - if localCheck, found := localChecks[consulCheck.CheckID]; found { - localChecksCount-- - if compareConsulCheck(localCheck.check, consulCheck) { - equalChecksCount++ - localChecks[consulCheck.CheckID].state = 'e' - } else { - changedChecksCount++ - localChecks[consulCheck.CheckID].state = 'c' - } - } else { - agentChecks++ - agentCheckReg := &consul.AgentCheckRegistration{ - ID: consulCheck.CheckID, - Name: consulCheck.Name, - Notes: consulCheck.Notes, - ServiceID: consulCheck.ServiceID, - } - localChecks[consulCheck.CheckID] = &mergedCheck{agentCheckReg, 'a'} - } - } - - missingChecks = make([]*consul.AgentCheckRegistration, 0, localChecksCount) - equalChecks = make([]*consul.AgentCheckRegistration, 0, equalChecksCount) - changedChecks = make([]*consul.AgentCheckRegistration, 0, changedChecksCount) - staleChecks = make([]*consul.AgentCheckRegistration, 0, agentChecks) - for _, check := range localChecks { - switch check.state { - case 'l': - missingChecks = append(missingChecks, check.check) - case 'e': - equalChecks = append(equalChecks, check.check) - case 'c': - changedChecks = append(changedChecks, check.check) - case 'a': - staleChecks = append(staleChecks, check.check) - } - } - - return missingChecks, equalChecks, changedChecks, staleChecks -} - -// compareConsulService takes a consul.AgentServiceRegistration instance and -// compares it with a consul.AgentService. Returns true if they are equal -// according to consul.AgentService, otherwise false. -func compareConsulService(localService *consul.AgentServiceRegistration, consulService *consul.AgentService) bool { - if consulService.ID != localService.ID || - consulService.Service != localService.Name || - consulService.Port != localService.Port || - consulService.Address != localService.Address || - consulService.EnableTagOverride != localService.EnableTagOverride { - return false - } - - serviceTags := make(map[string]byte, len(localService.Tags)) - for _, tag := range localService.Tags { - serviceTags[tag] = 'l' - } - for _, tag := range consulService.Tags { - if _, found := serviceTags[tag]; !found { - return false - } - serviceTags[tag] = 'b' - } - for _, state := range serviceTags { - if state == 'l' { - return false - } - } - - return true -} - -// calcServicesDiff takes the argument (consulServices) and calculates the -// delta between the consul.Syncer's list of known services -// (c.flattenedServices()). Four arrays are returned: -// -// 1) a slice of services that exist only locally in the Syncer and are -// missing from the Consul Agent (consulServices) and therefore need to be -// registered. -// -// 2) a slice of services that exist in both the local consul.Syncer's -// tracked list and Consul Agent (consulServices) *AND* are identical. -// -// 3) a slice of services that exist in both the local consul.Syncer's -// tracked list and Consul Agent (consulServices) but have diverged state. -// -// 4) a slice of services that exist only in the Consul Agent -// (consulServices) and should be removed because the Consul Agent has -// drifted from the Syncer. -func (c *Syncer) calcServicesDiff(consulServices map[consulServiceID]*consul.AgentService) (missingServices []*consul.AgentServiceRegistration, equalServices []*consul.AgentServiceRegistration, changedServices []*consul.AgentServiceRegistration, staleServices []*consul.AgentServiceRegistration) { - type mergedService struct { - service *consul.AgentServiceRegistration - // 'l' == Nomad local only - // 'e' == equal - // 'c' == changed - // 'a' == Consul agent only - state byte - } - var ( - localServicesCount = 0 - equalServicesCount = 0 - changedServicesCount = 0 - agentServices = 0 - ) - flattenedServices := c.flattenedServices() - localServices := make(map[string]*mergedService, len(flattenedServices)+len(consulServices)) - for _, localService := range flattenedServices { - localServicesCount++ - localServices[localService.ID] = &mergedService{localService, 'l'} - } - for _, consulService := range consulServices { - if localService, found := localServices[consulService.ID]; found { - localServicesCount-- - if compareConsulService(localService.service, consulService) { - equalServicesCount++ - localServices[consulService.ID].state = 'e' - } else { - changedServicesCount++ - localServices[consulService.ID].state = 'c' - } - } else { - agentServices++ - agentServiceReg := &consul.AgentServiceRegistration{ - ID: consulService.ID, - Name: consulService.Service, - Tags: consulService.Tags, - Port: consulService.Port, - Address: consulService.Address, - } - localServices[consulService.ID] = &mergedService{agentServiceReg, 'a'} - } - } - - missingServices = make([]*consul.AgentServiceRegistration, 0, localServicesCount) - equalServices = make([]*consul.AgentServiceRegistration, 0, equalServicesCount) - changedServices = make([]*consul.AgentServiceRegistration, 0, changedServicesCount) - staleServices = make([]*consul.AgentServiceRegistration, 0, agentServices) - for _, service := range localServices { - switch service.state { - case 'l': - missingServices = append(missingServices, service.service) - case 'e': - equalServices = append(equalServices, service.service) - case 'c': - changedServices = append(changedServices, service.service) - case 'a': - staleServices = append(staleServices, service.service) - } - } - - return missingServices, equalServices, changedServices, staleServices -} - -// syncServices synchronizes this Syncer's Consul Services with the Consul -// Agent. -func (c *Syncer) syncServices() error { - consulServices, err := c.queryAgentServices() - if err != nil { - return err - } - - // Synchronize services with Consul - var mErr multierror.Error - missingServices, _, changedServices, removedServices := c.calcServicesDiff(consulServices) - for _, service := range missingServices { - if err := c.client.Agent().ServiceRegister(service); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - for _, service := range changedServices { - // Re-register the local service - if err := c.client.Agent().ServiceRegister(service); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - for _, service := range removedServices { - if err := c.deregisterService(service.ID); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - return mErr.ErrorOrNil() -} - -// registerCheck registers a check definition with Consul -func (c *Syncer) registerCheck(chkReg *consul.AgentCheckRegistration) error { - c.registryLock.RLock() - if cr, ok := c.checkRunners[consulCheckID(chkReg.ID)]; ok { - cr.Start() - } - c.registryLock.RUnlock() - return c.client.Agent().CheckRegister(chkReg) -} - -// ensureCheckRunning starts the check runner for a check if it's not already running -func (c *Syncer) ensureCheckRunning(chk *consul.AgentCheckRegistration) { - c.registryLock.RLock() - defer c.registryLock.RUnlock() - if cr, ok := c.checkRunners[consulCheckID(chk.ID)]; ok && !cr.Started() { - c.logger.Printf("[DEBUG] consul.syncer: starting runner for existing check. %v", chk.ID) - cr.Start() - } -} - -// createCheckReg creates a Check that can be registered with Nomad. It also -// creates a Nomad check for the check types that it can handle. -func (c *Syncer) createCheckReg(check *structs.ServiceCheck, serviceReg *consul.AgentServiceRegistration) (*consul.AgentCheckRegistration, error) { - chkReg := consul.AgentCheckRegistration{ - ID: check.Hash(serviceReg.ID), - Name: check.Name, - ServiceID: serviceReg.ID, - } - chkReg.Timeout = check.Timeout.String() - chkReg.Interval = check.Interval.String() - host, port := serviceReg.Address, serviceReg.Port - if check.PortLabel != "" { - host, port = c.addrFinder(check.PortLabel) - } - switch check.Type { - case structs.ServiceCheckHTTP: - if check.Protocol == "" { - check.Protocol = "http" - } - base := url.URL{ - Scheme: check.Protocol, - Host: net.JoinHostPort(host, strconv.Itoa(port)), - } - relative, err := url.Parse(check.Path) - if err != nil { - return nil, err - } - url := base.ResolveReference(relative) - chkReg.HTTP = url.String() - case structs.ServiceCheckTCP: - chkReg.TCP = net.JoinHostPort(host, strconv.Itoa(port)) - case structs.ServiceCheckScript: - chkReg.TTL = (check.Interval + ttlCheckBuffer).String() - default: - return nil, fmt.Errorf("check type %+q not valid", check.Type) - } - chkReg.Status = check.InitialStatus - return &chkReg, nil -} - -// generateConsulServiceID takes the domain and service key and returns a Consul -// ServiceID -func generateConsulServiceID(domain ServiceDomain, key ServiceKey) consulServiceID { - return consulServiceID(fmt.Sprintf("%s-%s-%s", nomadServicePrefix, domain, key)) -} - -// createService creates a Consul AgentService from a Nomad ConsulService. -func (c *Syncer) createService(service *structs.Service, domain ServiceDomain, key ServiceKey) (*consul.AgentServiceRegistration, error) { - c.registryLock.RLock() - defer c.registryLock.RUnlock() - - srv := consul.AgentServiceRegistration{ - ID: string(generateConsulServiceID(domain, key)), - Name: service.Name, - Tags: service.Tags, - } - host, port := c.addrFinder(service.PortLabel) - if host != "" { - srv.Address = host - } - - if port != 0 { - srv.Port = port - } - - return &srv, nil -} - -// deregisterService de-registers a service with the given ID from consul -func (c *Syncer) deregisterService(serviceID string) error { - return c.client.Agent().ServiceDeregister(serviceID) -} - -// deregisterCheck de-registers a check from Consul -func (c *Syncer) deregisterCheck(id consulCheckID) error { - c.registryLock.Lock() - defer c.registryLock.Unlock() - - // Deleting from Consul Agent - if err := c.client.Agent().CheckDeregister(string(id)); err != nil { - // CheckDeregister() will be reattempted again in a future - // sync. - return err - } - - // Remove the check from the local registry - if cr, ok := c.checkRunners[id]; ok { - cr.Stop() - delete(c.checkRunners, id) - } - - return nil -} - -// Run triggers periodic syncing of services and checks with Consul. This is -// a long lived go-routine which is stopped during shutdown. -func (c *Syncer) Run() { - sync := time.NewTimer(0) - for { - select { - case <-sync.C: - d := c.syncInterval - c.syncJitter - sync.Reset(d) - - if err := c.SyncServices(); err != nil { - if c.consulAvailable { - c.logger.Printf("[DEBUG] consul.syncer: error in syncing: %v", err) - } - c.consulAvailable = false - } else { - if !c.consulAvailable { - c.logger.Printf("[DEBUG] consul.syncer: syncs succesful") - } - c.consulAvailable = true - } - case <-c.notifySyncCh: - sync.Reset(0) - case <-c.shutdownCh: - c.Shutdown() - case <-c.notifyShutdownCh: - sync.Stop() - c.logger.Printf("[INFO] consul.syncer: shutting down syncer ") - return - } - } -} - -// RunHandlers executes each handler (randomly) -func (c *Syncer) RunHandlers() error { - c.periodicLock.RLock() - handlers := make(map[string]types.PeriodicCallback, len(c.periodicCallbacks)) - for name, fn := range c.periodicCallbacks { - handlers[name] = fn - } - c.periodicLock.RUnlock() - - var mErr multierror.Error - for _, fn := range handlers { - if err := fn(); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - return mErr.ErrorOrNil() -} - -// SyncServices sync the services with the Consul Agent -func (c *Syncer) SyncServices() error { - var mErr multierror.Error - if err := c.syncServices(); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - if err := c.syncChecks(); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - if err := c.RunHandlers(); err != nil { - return err - } - return mErr.ErrorOrNil() -} - -// filterConsulServices prunes out all the service who were not registered with -// the syncer -func (c *Syncer) filterConsulServices(consulServices map[string]*consul.AgentService) map[consulServiceID]*consul.AgentService { - localServices := make(map[consulServiceID]*consul.AgentService, len(consulServices)) - c.groupsLock.RLock() - defer c.groupsLock.RUnlock() - for serviceID, service := range consulServices { - for domain := range c.servicesGroups { - if strings.HasPrefix(service.ID, fmt.Sprintf("%s-%s", nomadServicePrefix, domain)) { - localServices[consulServiceID(serviceID)] = service - break - } - } - } - return localServices -} - -// filterConsulChecks prunes out all the consul checks which do not have -// services with Syncer's idPrefix. -func (c *Syncer) filterConsulChecks(consulChecks map[string]*consul.AgentCheck) map[consulCheckID]*consul.AgentCheck { - localChecks := make(map[consulCheckID]*consul.AgentCheck, len(consulChecks)) - c.groupsLock.RLock() - defer c.groupsLock.RUnlock() - for checkID, check := range consulChecks { - for domain := range c.checkGroups { - if strings.HasPrefix(check.ServiceID, fmt.Sprintf("%s-%s", nomadServicePrefix, domain)) { - localChecks[consulCheckID(checkID)] = check - break - } - } - } - return localChecks -} - -// consulPresent indicates whether the Consul Agent is responding -func (c *Syncer) consulPresent() bool { - _, err := c.client.Agent().Self() - return err == nil -} - -// runCheck runs a check and updates the corresponding ttl check in consul -func (c *Syncer) runCheck(check Check) { - res := check.Run() - if res.Duration >= check.Timeout() { - c.logger.Printf("[DEBUG] consul.syncer: check took time: %v, timeout: %v", res.Duration, check.Timeout()) - } - state := consul.HealthCritical - output := res.Output - switch res.ExitCode { - case 0: - state = consul.HealthPassing - case 1: - state = consul.HealthWarning - default: - state = consul.HealthCritical - } - if res.Err != nil { - state = consul.HealthCritical - output = res.Err.Error() - } - if err := c.client.Agent().UpdateTTL(check.ID(), output, state); err != nil { - if c.consulAvailable { - c.logger.Printf("[DEBUG] consul.syncer: check %+q failed, disabling Consul checks until until next successful sync: %v", check.ID(), err) - c.consulAvailable = false - } else { - c.consulAvailable = true - } - } -} - -// ReapUnmatched prunes all services that do not exist in the passed domains -func (c *Syncer) ReapUnmatched(domains []ServiceDomain) error { - servicesInConsul, err := c.ConsulClient().Agent().Services() - if err != nil { - return err - } - - var mErr multierror.Error - for serviceID := range servicesInConsul { - // Skip any service that was not registered by Nomad - if !strings.HasPrefix(serviceID, nomadServicePrefix) { - continue - } - - // Filter services that do not exist in the desired domains - match := false - for _, domain := range domains { - // Include the hyphen so it is explicit to that domain otherwise it - // maybe a subset match - desired := fmt.Sprintf("%s-%s-", nomadServicePrefix, domain) - if strings.HasPrefix(serviceID, desired) { - match = true - break - } - } - - if !match { - if err := c.deregisterService(serviceID); err != nil { - mErr.Errors = append(mErr.Errors, err) - } - } - } - - return mErr.ErrorOrNil() -} - -// AddPeriodicHandler adds a uniquely named callback. Returns true if -// successful, false if a handler with the same name already exists. -func (c *Syncer) AddPeriodicHandler(name string, fn types.PeriodicCallback) bool { - c.periodicLock.Lock() - defer c.periodicLock.Unlock() - if _, found := c.periodicCallbacks[name]; found { - c.logger.Printf("[ERROR] consul.syncer: failed adding handler %+q", name) - return false - } - c.periodicCallbacks[name] = fn - return true -} - -// NumHandlers returns the number of callbacks registered with the syncer -func (c *Syncer) NumHandlers() int { - c.periodicLock.RLock() - defer c.periodicLock.RUnlock() - return len(c.periodicCallbacks) -} - -// RemovePeriodicHandler removes a handler with a given name. -func (c *Syncer) RemovePeriodicHandler(name string) { - c.periodicLock.Lock() - defer c.periodicLock.Unlock() - delete(c.periodicCallbacks, name) -} - -// ConsulClient returns the Consul client used by the Syncer. -func (c *Syncer) ConsulClient() *consul.Client { - return c.client -} diff --git a/command/agent/consul/syncer_test.go b/command/agent/consul/syncer_test.go deleted file mode 100644 index 42879ca77839..000000000000 --- a/command/agent/consul/syncer_test.go +++ /dev/null @@ -1,358 +0,0 @@ -package consul - -import ( - "io/ioutil" - "log" - "net" - "os" - "reflect" - "testing" - "time" - - "github.com/hashicorp/consul/api" - "github.com/hashicorp/consul/testutil" - "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/nomad/structs/config" -) - -const ( - allocID = "12" - serviceRegPrefix = "test" - serviceGroupName = "executor" -) - -var logger = log.New(os.Stdout, "", log.LstdFlags) - -func TestSyncNow(t *testing.T) { - cs, testconsul := testConsul(t) - defer cs.Shutdown() - defer testconsul.Stop() - - cs.SetAddrFinder(func(h string) (string, int) { - a, pstr, _ := net.SplitHostPort(h) - p, _ := net.LookupPort("tcp", pstr) - return a, p - }) - cs.syncInterval = 9000 * time.Hour - - service := &structs.Service{Name: "foo1", Tags: []string{"a", "b"}} - services := map[ServiceKey]*structs.Service{ - GenerateServiceKey(service): service, - } - - // Run syncs once on startup and then blocks forever - go cs.Run() - - if err := cs.SetServices(serviceGroupName, services); err != nil { - t.Fatalf("error setting services: %v", err) - } - - synced := false - for i := 0; !synced && i < 10; i++ { - time.Sleep(250 * time.Millisecond) - agentServices, err := cs.queryAgentServices() - if err != nil { - t.Fatalf("error querying consul services: %v", err) - } - synced = len(agentServices) == 1 - } - if !synced { - t.Fatalf("initial sync never occurred") - } - - // SetServices again should cause another sync - service1 := &structs.Service{Name: "foo1", Tags: []string{"Y", "Z"}} - service2 := &structs.Service{Name: "bar"} - services = map[ServiceKey]*structs.Service{ - GenerateServiceKey(service1): service1, - GenerateServiceKey(service2): service2, - } - - if err := cs.SetServices(serviceGroupName, services); err != nil { - t.Fatalf("error setting services: %v", err) - } - - synced = false - for i := 0; !synced && i < 10; i++ { - time.Sleep(250 * time.Millisecond) - agentServices, err := cs.queryAgentServices() - if err != nil { - t.Fatalf("error querying consul services: %v", err) - } - synced = len(agentServices) == 2 - } - if !synced { - t.Fatalf("SetServices didn't sync immediately") - } -} - -func TestCheckRegistration(t *testing.T) { - cs, err := NewSyncer(config.DefaultConsulConfig(), make(chan struct{}), logger) - if err != nil { - t.Fatalf("Err: %v", err) - } - - check1 := structs.ServiceCheck{ - Name: "check-foo-1", - Type: structs.ServiceCheckTCP, - Interval: 30 * time.Second, - Timeout: 5 * time.Second, - InitialStatus: api.HealthPassing, - } - check2 := structs.ServiceCheck{ - Name: "check1", - Type: "tcp", - PortLabel: "port2", - Interval: 3 * time.Second, - Timeout: 1 * time.Second, - } - check3 := structs.ServiceCheck{ - Name: "check3", - Type: "http", - PortLabel: "port3", - Path: "/health?p1=1&p2=2", - Interval: 3 * time.Second, - Timeout: 1 * time.Second, - } - service1 := structs.Service{ - Name: "foo-1", - Tags: []string{"tag1", "tag2"}, - PortLabel: "port1", - Checks: []*structs.ServiceCheck{ - &check1, &check2, - }, - } - task := structs.Task{ - Name: "foo", - Services: []*structs.Service{&service1}, - Resources: &structs.Resources{ - Networks: []*structs.NetworkResource{ - &structs.NetworkResource{ - IP: "10.10.11.5", - DynamicPorts: []structs.Port{ - structs.Port{ - Label: "port1", - Value: 20002, - }, - structs.Port{ - Label: "port2", - Value: 20003, - }, - structs.Port{ - Label: "port3", - Value: 20004, - }, - }, - }, - }, - }, - } - cs.SetAddrFinder(task.FindHostAndPortFor) - srvReg, _ := cs.createService(&service1, "domain", "key") - check1Reg, _ := cs.createCheckReg(&check1, srvReg) - check2Reg, _ := cs.createCheckReg(&check2, srvReg) - check3Reg, _ := cs.createCheckReg(&check3, srvReg) - - expected := "10.10.11.5:20002" - if check1Reg.TCP != expected { - t.Fatalf("expected: %v, actual: %v", expected, check1Reg.TCP) - } - - expected = "10.10.11.5:20003" - if check2Reg.TCP != expected { - t.Fatalf("expected: %v, actual: %v", expected, check2Reg.TCP) - } - - expected = "http://10.10.11.5:20004/health?p1=1&p2=2" - if check3Reg.HTTP != expected { - t.Fatalf("expected: %v, actual: %v", expected, check3Reg.HTTP) - } - - expected = api.HealthPassing - if check1Reg.Status != expected { - t.Fatalf("expected: %v, actual: %v", expected, check1Reg.Status) - } -} - -// testConsul returns a Syncer configured with an embedded Consul server. -// -// Callers must defer Syncer.Shutdown() and TestServer.Stop() -// -func testConsul(t *testing.T) (*Syncer, *testutil.TestServer) { - // Create an embedded Consul server - testconsul := testutil.NewTestServerConfig(t, func(c *testutil.TestServerConfig) { - // If -v wasn't specified squelch consul logging - if !testing.Verbose() { - c.Stdout = ioutil.Discard - c.Stderr = ioutil.Discard - } - }) - - // Configure Syncer to talk to the test server - cconf := config.DefaultConsulConfig() - cconf.Addr = testconsul.HTTPAddr - - cs, err := NewSyncer(cconf, nil, logger) - if err != nil { - t.Fatalf("Error creating Syncer: %v", err) - } - return cs, testconsul -} - -func TestConsulServiceRegisterServices(t *testing.T) { - cs, testconsul := testConsul(t) - defer cs.Shutdown() - defer testconsul.Stop() - - service1 := &structs.Service{Name: "foo", Tags: []string{"a", "b"}} - service2 := &structs.Service{Name: "foo"} - services := map[ServiceKey]*structs.Service{ - GenerateServiceKey(service1): service1, - GenerateServiceKey(service2): service2, - } - - // Call SetServices to update services in consul - if err := cs.SetServices(serviceGroupName, services); err != nil { - t.Fatalf("error setting services: %v", err) - } - - // Manually call SyncServers to cause a synchronous consul update - if err := cs.SyncServices(); err != nil { - t.Fatalf("error syncing services: %v", err) - } - - numservices := len(cs.flattenedServices()) - if numservices != 2 { - t.Fatalf("expected 2 services but found %d", numservices) - } - - numchecks := len(cs.flattenedChecks()) - if numchecks != 0 { - t.Fatalf("expected 0 checks but found %d", numchecks) - } - - // Assert services are in consul - agentServices, err := cs.client.Agent().Services() - if err != nil { - t.Fatalf("error querying consul services: %v", err) - } - found := 0 - for id, as := range agentServices { - if id == "consul" { - found++ - continue - } - if _, ok := services[ServiceKey(as.Service)]; ok { - found++ - continue - } - t.Errorf("unexpected service in consul: %s", id) - } - if found != 3 { - t.Fatalf("expected 3 services in consul but found %d:\nconsul: %#v", len(agentServices), agentServices) - } - - agentChecks, err := cs.queryChecks() - if err != nil { - t.Fatalf("error querying consul checks: %v", err) - } - if len(agentChecks) != numchecks { - t.Fatalf("expected %d checks in consul but found %d:\n%#v", numservices, len(agentChecks), agentChecks) - } -} - -func TestConsulServiceUpdateService(t *testing.T) { - cs, testconsul := testConsul(t) - defer cs.Shutdown() - defer testconsul.Stop() - - cs.SetAddrFinder(func(h string) (string, int) { - a, pstr, _ := net.SplitHostPort(h) - p, _ := net.LookupPort("tcp", pstr) - return a, p - }) - - service1 := &structs.Service{Name: "foo1", Tags: []string{"a", "b"}} - service2 := &structs.Service{Name: "foo2"} - services := map[ServiceKey]*structs.Service{ - GenerateServiceKey(service1): service1, - GenerateServiceKey(service2): service2, - } - if err := cs.SetServices(serviceGroupName, services); err != nil { - t.Fatalf("error setting services: %v", err) - } - if err := cs.SyncServices(); err != nil { - t.Fatalf("error syncing services: %v", err) - } - - // Now update both services - service1 = &structs.Service{Name: "foo1", Tags: []string{"a", "z"}} - service2 = &structs.Service{Name: "foo2", PortLabel: ":8899"} - service3 := &structs.Service{Name: "foo3"} - services = map[ServiceKey]*structs.Service{ - GenerateServiceKey(service1): service1, - GenerateServiceKey(service2): service2, - GenerateServiceKey(service3): service3, - } - if err := cs.SetServices(serviceGroupName, services); err != nil { - t.Fatalf("error setting services: %v", err) - } - if err := cs.SyncServices(); err != nil { - t.Fatalf("error syncing services: %v", err) - } - - agentServices, err := cs.queryAgentServices() - if err != nil { - t.Fatalf("error querying consul services: %v", err) - } - if len(agentServices) != 3 { - t.Fatalf("expected 3 services in consul but found %d:\n%#v", len(agentServices), agentServices) - } - consulServices := make(map[string]*api.AgentService, 3) - for _, as := range agentServices { - consulServices[as.ID] = as - } - - found := 0 - for _, s := range cs.flattenedServices() { - // Assert sure changes were applied to internal state - switch s.Name { - case "foo1": - found++ - if !reflect.DeepEqual(service1.Tags, s.Tags) { - t.Errorf("incorrect tags on foo1:\n expected: %v\n found: %v", service1.Tags, s.Tags) - } - case "foo2": - found++ - if s.Address != "" { - t.Errorf("expected empty host on foo2 but found %q", s.Address) - } - if s.Port != 8899 { - t.Errorf("expected port 8899 on foo2 but found %d", s.Port) - } - case "foo3": - found++ - default: - t.Errorf("unexpected service: %s", s.Name) - } - - // Assert internal state equals consul's state - cs, ok := consulServices[s.ID] - if !ok { - t.Errorf("service not in consul: %s id: %s", s.Name, s.ID) - continue - } - if !reflect.DeepEqual(s.Tags, cs.Tags) { - t.Errorf("mismatched tags in syncer state and consul for %s:\nsyncer: %v\nconsul: %v", s.Name, s.Tags, cs.Tags) - } - if cs.Port != s.Port { - t.Errorf("mismatched port in syncer state and consul for %s\nsyncer: %v\nconsul: %v", s.Name, s.Port, cs.Port) - } - if cs.Address != s.Address { - t.Errorf("mismatched address in syncer state and consul for %s\nsyncer: %v\nconsul: %v", s.Name, s.Address, cs.Address) - } - } - if found != 3 { - t.Fatalf("expected 3 services locally but found %d", found) - } -} diff --git a/command/agent/consul/unit_test.go b/command/agent/consul/unit_test.go new file mode 100644 index 000000000000..59ea83d7b12f --- /dev/null +++ b/command/agent/consul/unit_test.go @@ -0,0 +1,497 @@ +package consul + +import ( + "context" + "fmt" + "io/ioutil" + "log" + "os" + "reflect" + "sync" + "testing" + "time" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/nomad/structs" +) + +func testLogger() *log.Logger { + if testing.Verbose() { + return log.New(os.Stderr, "", log.LstdFlags) + } + return log.New(ioutil.Discard, "", 0) +} + +func testTask() *structs.Task { + return &structs.Task{ + Name: "taskname", + Resources: &structs.Resources{ + Networks: []*structs.NetworkResource{ + { + DynamicPorts: []structs.Port{{Label: "x", Value: 1234}}, + }, + }, + }, + Services: []*structs.Service{ + { + Name: "taskname-service", + PortLabel: "x", + Tags: []string{"tag1", "tag2"}, + }, + }, + } +} + +// testFakeCtx contains a fake Consul AgentAPI and implements the Exec +// interface to allow testing without running Consul. +type testFakeCtx struct { + ServiceClient *ServiceClient + FakeConsul *fakeConsul + Task *structs.Task + + ExecFunc func(ctx context.Context, cmd string, args []string) ([]byte, int, error) +} + +// Exec implements the ScriptExecutor interface and will use an alternate +// implementation t.ExecFunc if non-nil. +func (t *testFakeCtx) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + if t.ExecFunc == nil { + // Default impl is just "ok" + return []byte("ok"), 0, nil + } + return t.ExecFunc(ctx, cmd, args) +} + +// setupFake creates a testFakeCtx with a ServiceClient backed by a fakeConsul. +// A test Task is also provided. +func setupFake() *testFakeCtx { + fc := &fakeConsul{ + services: make(map[string]*api.AgentServiceRegistration), + checks: make(map[string]*api.AgentCheckRegistration), + checkTTLs: make(map[string]int), + } + return &testFakeCtx{ + ServiceClient: NewServiceClient(fc, testLogger()), + FakeConsul: fc, + Task: testTask(), + } +} + +// fakeConsul is a fake in-memory Consul backend for ServiceClient. +type fakeConsul struct { + // maps of what services and checks have been registered + services map[string]*api.AgentServiceRegistration + checks map[string]*api.AgentCheckRegistration + mu sync.Mutex + + // when UpdateTTL is called the check ID will have its counter inc'd + checkTTLs map[string]int +} + +func (c *fakeConsul) CheckRegister(check *api.AgentCheckRegistration) error { + c.mu.Lock() + defer c.mu.Unlock() + c.checks[check.ID] = check + return nil +} + +func (c *fakeConsul) CheckDeregister(checkID string) error { + c.mu.Lock() + defer c.mu.Unlock() + delete(c.checks, checkID) + delete(c.checkTTLs, checkID) + return nil +} + +func (c *fakeConsul) ServiceRegister(service *api.AgentServiceRegistration) error { + c.mu.Lock() + defer c.mu.Unlock() + c.services[service.ID] = service + return nil +} + +func (c *fakeConsul) ServiceDeregister(serviceID string) error { + c.mu.Lock() + defer c.mu.Unlock() + delete(c.services, serviceID) + return nil +} + +func (c *fakeConsul) UpdateTTL(id string, output string, status string) error { + c.mu.Lock() + defer c.mu.Unlock() + check, ok := c.checks[id] + if !ok { + return fmt.Errorf("unknown check id: %q", id) + } + // Flip initial status to passing + check.Status = "passing" + c.checkTTLs[id]++ + return nil +} + +func TestConsul_ChangeTags(t *testing.T) { + ctx := setupFake() + + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + // Manually call sync() since Run() isn't running + if err := ctx.ServiceClient.sync(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + + origKey := "" + for k, v := range ctx.FakeConsul.services { + origKey = k + if v.Name != ctx.Task.Services[0].Name { + t.Errorf("expected Name=%q != %q", ctx.Task.Services[0].Name, v.Name) + } + if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) + } + } + + // Changing a tag removes old entry before adding new one + ctx.ServiceClient.RemoveTask("allocid", ctx.Task) + ctx.Task.Services[0].Tags[0] = "newtag" + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + if err := ctx.ServiceClient.sync(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + + for k, v := range ctx.FakeConsul.services { + if k == origKey { + t.Errorf("expected key to change but found %q", k) + } + if v.Name != ctx.Task.Services[0].Name { + t.Errorf("expected Name=%q != %q", ctx.Task.Services[0].Name, v.Name) + } + if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) + } + } +} + +// TestConsul_RegServices tests basic service registration. +func TestConsul_RegServices(t *testing.T) { + ctx := setupFake() + port := ctx.Task.Resources.Networks[0].DynamicPorts[0].Value + + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + // Manually call sync() since Run() isn't running + if err := ctx.ServiceClient.sync(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + for _, v := range ctx.FakeConsul.services { + if v.Name != ctx.Task.Services[0].Name { + t.Errorf("expected Name=%q != %q", ctx.Task.Services[0].Name, v.Name) + } + if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) + } + if v.Port != port { + t.Errorf("expected Port=%d != %d", port, v.Port) + } + } + + // Make a change which will register a new service + ctx.Task.Services[0].Name = "taskname-service2" + ctx.Task.Services[0].Tags[0] = "tag3" + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { + t.Fatalf("unpexpected error registering task: %v", err) + } + + // Make sure changes don't take affect until sync() is called (since + // Run() isn't running) + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + for _, v := range ctx.FakeConsul.services { + if reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags to differ, changes applied before sync()") + } + } + + // Now sync() and re-check for the applied updates + if err := ctx.ServiceClient.sync(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + if n := len(ctx.FakeConsul.services); n != 2 { + t.Fatalf("expected 2 services but found %d:\n%#v", n, ctx.FakeConsul.services) + } + found := false + for _, v := range ctx.FakeConsul.services { + if v.Name == ctx.Task.Services[0].Name { + if found { + t.Fatalf("found new service name %q twice", v.Name) + } + found = true + if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) + } + } + } + if !found { + t.Fatalf("did not find new service %q", ctx.Task.Services[0].Name) + } + + // Remove the new task + ctx.ServiceClient.RemoveTask("allocid", ctx.Task) + if err := ctx.ServiceClient.sync(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + for _, v := range ctx.FakeConsul.services { + if v.Name != "taskname-service" { + t.Errorf("expected original task to survive not %q", v.Name) + } + } +} + +// TestConsul_ShutdownOK tests the ok path for the shutdown logic in +// ServiceClient. +func TestConsul_ShutdownOK(t *testing.T) { + ctx := setupFake() + + // Add a script check to make sure its TTL gets updated + ctx.Task.Services[0].Checks = []*structs.ServiceCheck{ + { + Name: "scriptcheck", + Type: "script", + Command: "true", + // Make check block until shutdown + Interval: 9000 * time.Hour, + Timeout: 10 * time.Second, + InitialStatus: "warning", + }, + } + + hasShutdown := make(chan struct{}) + go func() { + ctx.ServiceClient.Run() + close(hasShutdown) + }() + + // Register a task and agent + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + agentServices := []*structs.Service{ + { + Name: "http", + Tags: []string{"nomad"}, + PortLabel: "localhost:2345", + }, + } + if err := ctx.ServiceClient.RegisterAgent("client", agentServices); err != nil { + t.Fatalf("unexpected error registering agent: %v", err) + } + + // Shutdown should block until all enqueued operations finish. + if err := ctx.ServiceClient.Shutdown(); err != nil { + t.Errorf("unexpected error shutting down client: %v", err) + } + + // assert Run() exits in a timely fashion after Shutdown() exits + select { + case <-hasShutdown: + // ok! Run() exited as expected + case <-time.After(10 * time.Second): + t.Fatalf("expected Run() to exit, but it did not") + } + + // Nothing should be enqueued anymore + enqueued := (len(ctx.ServiceClient.regServices) + len(ctx.ServiceClient.deregServices) + + len(ctx.ServiceClient.regChecks) + len(ctx.ServiceClient.deregChecks)) + if enqueued > 0 { + t.Errorf("%d operations still enqueued", enqueued) + } + + // UpdateTTL should have been called once for the script check + if n := len(ctx.FakeConsul.checkTTLs); n != 1 { + t.Fatalf("expected 1 checkTTL entry but found: %d", n) + } + for _, v := range ctx.FakeConsul.checkTTLs { + if v != 1 { + t.Fatalf("expected script check to be updated once but found %d", v) + } + } + for _, v := range ctx.FakeConsul.checks { + if v.Status != "passing" { + t.Fatalf("expected check to be passing but found %q", v.Status) + } + } +} + +// TestConsul_ShutdownSlow tests the slow but ok path for the shutdown logic in +// ServiceClient. +func TestConsul_ShutdownSlow(t *testing.T) { + t.Parallel() // run the slow tests in parallel + ctx := setupFake() + + // Add a script check to make sure its TTL gets updated + ctx.Task.Services[0].Checks = []*structs.ServiceCheck{ + { + Name: "scriptcheck", + Type: "script", + Command: "true", + // Make check block until shutdown + Interval: 9000 * time.Hour, + Timeout: 5 * time.Second, + InitialStatus: "warning", + }, + } + + // Make Exec slow, but not too slow + ctx.ExecFunc = func(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + time.Sleep(time.Second) + return []byte{}, 0, nil + } + + // Make shutdown wait time just a bit longer than ctx.Exec takes + ctx.ServiceClient.shutdownWait = 3 * time.Second + + hasShutdown := make(chan struct{}) + go func() { + ctx.ServiceClient.Run() + close(hasShutdown) + }() + + // Register a task and agent + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + // Shutdown should block until all enqueued operations finish. + preShutdown := time.Now() + if err := ctx.ServiceClient.Shutdown(); err != nil { + t.Errorf("unexpected error shutting down client: %v", err) + } + + // assert Run() exits in a timely fashion after Shutdown() exits + select { + case <-hasShutdown: + // ok! Run() exited as expected + case <-time.After(10 * time.Second): + t.Fatalf("expected Run() to exit, but it did not") + } + + // Shutdown time should have taken: 1s <= shutdown <= 3s + shutdownTime := time.Now().Sub(preShutdown) + if shutdownTime < time.Second || shutdownTime > ctx.ServiceClient.shutdownWait { + t.Errorf("expected shutdown to take >1s and <%s but took: %s", ctx.ServiceClient.shutdownWait, shutdownTime) + } + + // UpdateTTL should have been called once for the script check + if n := len(ctx.FakeConsul.checkTTLs); n != 1 { + t.Fatalf("expected 1 checkTTL entry but found: %d", n) + } + for _, v := range ctx.FakeConsul.checkTTLs { + if v != 1 { + t.Fatalf("expected script check to be updated once but found %d", v) + } + } + for _, v := range ctx.FakeConsul.checks { + if v.Status != "passing" { + t.Fatalf("expected check to be passing but found %q", v.Status) + } + } +} + +// TestConsul_ShutdownBlocked tests the blocked past deadline path for the +// shutdown logic in ServiceClient. +func TestConsul_ShutdownBlocked(t *testing.T) { + t.Parallel() // run the slow tests in parallel + ctx := setupFake() + + // Add a script check to make sure its TTL gets updated + ctx.Task.Services[0].Checks = []*structs.ServiceCheck{ + { + Name: "scriptcheck", + Type: "script", + Command: "true", + // Make check block until shutdown + Interval: 9000 * time.Hour, + Timeout: 9000 * time.Hour, + InitialStatus: "warning", + }, + } + + block := make(chan struct{}) + defer close(block) // cleanup after test + + // Make Exec slow, but not too slow + ctx.ExecFunc = func(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + <-block + return []byte{}, 0, nil + } + + // Use a short shutdown deadline since we're intentionally blocking forever + ctx.ServiceClient.shutdownWait = time.Second + + hasShutdown := make(chan struct{}) + go func() { + ctx.ServiceClient.Run() + close(hasShutdown) + }() + + // Register a task and agent + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + // Shutdown should block until all enqueued operations finish. + preShutdown := time.Now() + err := ctx.ServiceClient.Shutdown() + if err == nil { + t.Errorf("expected a timed out error from shutdown") + } + + // assert Run() exits in a timely fashion after Shutdown() exits + maxWait := 10 * time.Second + select { + case <-hasShutdown: + // ok! Run() exited as expected + case <-time.After(maxWait): + t.Fatalf("expected Run() to exit, but it did not") + } + + // Shutdown time should have taken 1s; to avoid timing related errors + // simply test for 1s <= shutdown <= 10s + shutdownTime := time.Now().Sub(preShutdown) + if shutdownTime < ctx.ServiceClient.shutdownWait || shutdownTime > maxWait { + t.Errorf("expected shutdown to take >%s and <%s but took: %s", ctx.ServiceClient.shutdownWait, maxWait, shutdownTime) + } + + // UpdateTTL should not have been called for the script check + if n := len(ctx.FakeConsul.checkTTLs); n != 0 { + t.Fatalf("expected 0 checkTTL entry but found: %d", n) + } + for _, v := range ctx.FakeConsul.checks { + if expected := "warning"; v.Status != expected { + t.Fatalf("expected check to be %q but found %q", expected, v.Status) + } + } +} diff --git a/nomad/server.go b/nomad/server.go index efed68cebdf2..dade6cc86356 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -63,6 +63,14 @@ const ( // raftRemoveGracePeriod is how long we wait to allow a RemovePeer // to replicate to gracefully leave the cluster. raftRemoveGracePeriod = 5 * time.Second + + // defaultConsulDiscoveryInterval is how often to poll Consul for new + // servers if there is no leader. + defaultConsulDiscoveryInterval time.Duration = 9 * time.Second + + // defaultConsulDiscoveryIntervalRetry is how often to poll Consul for + // new servers if there is no leader and the last Consul query failed. + defaultConsulDiscoveryIntervalRetry time.Duration = 3 * time.Second ) // Server is Nomad server which manages the job queues, @@ -136,8 +144,8 @@ type Server struct { heartbeatTimers map[string]*time.Timer heartbeatTimersLock sync.Mutex - // consulSyncer advertises this Nomad Agent with Consul - consulSyncer *consul.Syncer + // consulCatalog is used for discovering other Nomad Servers via Consul + consulCatalog consul.CatalogAPI // vault is the client for communicating with Vault. vault VaultClient @@ -167,7 +175,7 @@ type endpoints struct { // NewServer is used to construct a new Nomad server from the // configuration, potentially returning an error -func NewServer(config *Config, consulSyncer *consul.Syncer, logger *log.Logger) (*Server, error) { +func NewServer(config *Config, consulCatalog consul.CatalogAPI, logger *log.Logger) (*Server, error) { // Check the protocol version if err := config.CheckVersion(); err != nil { return nil, err @@ -212,20 +220,20 @@ func NewServer(config *Config, consulSyncer *consul.Syncer, logger *log.Logger) // Create the server s := &Server{ - config: config, - consulSyncer: consulSyncer, - connPool: NewPool(config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap), - logger: logger, - rpcServer: rpc.NewServer(), - peers: make(map[string][]*serverParts), - localPeers: make(map[raft.ServerAddress]*serverParts), - reconcileCh: make(chan serf.Member, 32), - eventCh: make(chan serf.Event, 256), - evalBroker: evalBroker, - blockedEvals: blockedEvals, - planQueue: planQueue, - rpcTLS: incomingTLS, - shutdownCh: make(chan struct{}), + config: config, + consulCatalog: consulCatalog, + connPool: NewPool(config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap), + logger: logger, + rpcServer: rpc.NewServer(), + peers: make(map[string][]*serverParts), + localPeers: make(map[raft.ServerAddress]*serverParts), + reconcileCh: make(chan serf.Member, 32), + eventCh: make(chan serf.Event, 256), + evalBroker: evalBroker, + blockedEvals: blockedEvals, + planQueue: planQueue, + rpcTLS: incomingTLS, + shutdownCh: make(chan struct{}), } // Create the periodic dispatcher for launching periodic jobs. @@ -542,8 +550,7 @@ func (s *Server) setupBootstrapHandler() error { s.logger.Printf("[DEBUG] server.nomad: lost contact with Nomad quorum, falling back to Consul for server list") - consulCatalog := s.consulSyncer.ConsulClient().Catalog() - dcs, err := consulCatalog.Datacenters() + dcs, err := s.consulCatalog.Datacenters() if err != nil { peersTimeout.Reset(peersPollInterval + lib.RandomStagger(peersPollInterval/peersPollJitterFactor)) return fmt.Errorf("server.nomad: unable to query Consul datacenters: %v", err) @@ -570,7 +577,7 @@ func (s *Server) setupBootstrapHandler() error { Near: "_agent", WaitTime: consul.DefaultQueryWaitDuration, } - consulServices, _, err := consulCatalog.Service(nomadServerServiceName, consul.ServiceTagSerf, consulOpts) + consulServices, _, err := s.consulCatalog.Service(nomadServerServiceName, consul.ServiceTagSerf, consulOpts) if err != nil { err := fmt.Errorf("failed to query service %q in Consul datacenter %q: %v", nomadServerServiceName, dc, err) s.logger.Printf("[WARN] server.nomad: %v", err) @@ -618,7 +625,28 @@ func (s *Server) setupBootstrapHandler() error { return nil } - s.consulSyncer.AddPeriodicHandler("Nomad Server Fallback Server Handler", bootstrapFn) + // Hacky replacement for old ConsulSyncer Periodic Handler. + go func() { + lastOk := true + sync := time.NewTimer(0) + for { + select { + case <-sync.C: + d := defaultConsulDiscoveryInterval + if err := bootstrapFn(); err != nil { + // Only log if it worked last time + if lastOk { + lastOk = false + s.logger.Printf("[ERR] consul: error looking up Nomad servers: %v", err) + } + d = defaultConsulDiscoveryIntervalRetry + } + sync.Reset(d) + case <-s.shutdownCh: + return + } + } + }() return nil } diff --git a/nomad/server_test.go b/nomad/server_test.go index 902498a1d4ae..719bfbf62590 100644 --- a/nomad/server_test.go +++ b/nomad/server_test.go @@ -76,15 +76,11 @@ func testServer(t *testing.T, cb func(*Config)) *Server { // Enable raft as leader if we have bootstrap on config.RaftConfig.StartAsLeader = !config.DevDisableBootstrap - shutdownCh := make(chan struct{}) logger := log.New(config.LogOutput, fmt.Sprintf("[%s] ", config.NodeName), log.LstdFlags) - consulSyncer, err := consul.NewSyncer(config.ConsulConfig, shutdownCh, logger) - if err != nil { - t.Fatalf("err: %v", err) - } + catalog := consul.NewMockCatalog(logger) // Create server - server, err := NewServer(config, consulSyncer, logger) + server, err := NewServer(config, catalog, logger) if err != nil { t.Fatalf("err: %v", err) } diff --git a/website/source/docs/job-specification/service.html.md b/website/source/docs/job-specification/service.html.md index ee6a1b6b4499..2973142da84a 100644 --- a/website/source/docs/job-specification/service.html.md +++ b/website/source/docs/job-specification/service.html.md @@ -106,7 +106,7 @@ does not automatically enable service discovery. ~> **Caveat:** The command must be the path to the command on disk, and no shell exists by default. That means operators like `||` or `&&` are not available. Additionally, all arguments must be supplied via the `args` - parameter. The achieve the behavior of shell operators, specify the command + parameter. To achieve the behavior of shell operators, specify the command as a shell, like `/bin/bash` and then use `args` to run the check. - `initial_status` `(string: )` - Specifies the originating status of the From e13b149628555138c6b60c1ba241781f07c5b688 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 30 Mar 2017 16:40:24 -0700 Subject: [PATCH 02/38] Remove unused syncInterval --- command/agent/consul/client.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 6365eb0b8316..ea3062ff8df2 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -74,7 +74,6 @@ type ServiceClient struct { client AgentAPI logger *log.Logger retryInterval time.Duration - syncInterval time.Duration // runningCh is closed when the main Run loop exits runningCh chan struct{} @@ -123,7 +122,6 @@ func NewServiceClient(consulClient AgentAPI, logger *log.Logger) *ServiceClient client: consulClient, logger: logger, retryInterval: defaultSyncInterval, //TODO what should this default to?! - syncInterval: defaultSyncInterval, runningCh: make(chan struct{}), shutdownCh: make(chan struct{}), shutdownWait: defaultShutdownWait, From 7930d359254094c5cd75f3e26f761c3e719664e1 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 30 Mar 2017 16:59:00 -0700 Subject: [PATCH 03/38] Remove some lies --- command/agent/consul/client.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index ea3062ff8df2..3538f2f5edb6 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -121,7 +121,7 @@ func NewServiceClient(consulClient AgentAPI, logger *log.Logger) *ServiceClient return &ServiceClient{ client: consulClient, logger: logger, - retryInterval: defaultSyncInterval, //TODO what should this default to?! + retryInterval: defaultSyncInterval, runningCh: make(chan struct{}), shutdownCh: make(chan struct{}), shutdownWait: defaultShutdownWait, @@ -158,7 +158,6 @@ func (c *ServiceClient) Run() { lastOk = false c.logger.Printf("[WARN] consul: failed to update services in Consul: %v", err) } - //TODO Log? and jitter/backoff timer.Reset(c.retryInterval) } else { if !lastOk { From b6937912d8a5350ce596b9e039b2c6dee8358a0e Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Mon, 3 Apr 2017 17:08:08 -0700 Subject: [PATCH 04/38] Add UpdateTask method instead of Remove/Add --- client/consul.go | 1 + client/task_runner.go | 30 +-- command/agent/consul/client.go | 358 +++++++++++++++--------------- command/agent/consul/ops.go | 61 +++++ command/agent/consul/unit_test.go | 4 +- 5 files changed, 253 insertions(+), 201 deletions(-) create mode 100644 command/agent/consul/ops.go diff --git a/client/consul.go b/client/consul.go index 41da0abebfb9..c9fe4e107803 100644 --- a/client/consul.go +++ b/client/consul.go @@ -10,4 +10,5 @@ import ( type ConsulServiceAPI interface { RegisterTask(allocID string, task *structs.Task, exec consul.ScriptExecutor) error RemoveTask(allocID string, task *structs.Task) + UpdateTask(allocID string, existing, newTask *structs.Task, exec consul.ScriptExecutor) error } diff --git a/client/task_runner.go b/client/task_runner.go index 64eb4570f1f0..209336d700cc 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -1386,15 +1386,27 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { updatedTask.Resources = update.TaskResources[updatedTask.Name] var mErr multierror.Error - var scriptExec consul.ScriptExecutor r.handleLock.Lock() if r.handle != nil { // Update will update resources and store the new kill timeout. if err := r.handle.Update(updatedTask); err != nil { mErr.Errors = append(mErr.Errors, fmt.Errorf("updating task resources failed: %v", err)) } - // Not all drivers support Exec (eg QEMU) - scriptExec, _ = r.handle.(consul.ScriptExecutor) + + //FIXME is there a better place to do this? used to be in executor + // Prepare services + interpolateServices(r.getTaskEnv(), updatedTask) + + // Not all drivers support Exec (eg QEMU), but RegisterTask + // handles nil ScriptExecutors + scriptExec, _ := r.handle.(consul.ScriptExecutor) + + // Since the handle exists, the task is running, so we need to + // update it in Consul (if the handle doesn't exist + // registration in Consul will happen when it's created) + if err := r.consul.UpdateTask(r.alloc.ID, r.task, updatedTask, scriptExec); err != nil { + mErr.Errors = append(mErr.Errors, fmt.Errorf("error updating services and checks in Consul: %v", err)) + } } r.handleLock.Unlock() @@ -1403,21 +1415,9 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { r.restartTracker.SetPolicy(tg.RestartPolicy) } - // Deregister the old service+checks - r.consul.RemoveTask(r.alloc.ID, r.task) - // Store the updated alloc. r.alloc = update r.task = updatedTask - - //FIXME is there a better place to do this? used to be in executor - // Prepare services - interpolateServices(r.getTaskEnv(), r.task) - - // Register the new service+checks - if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { - mErr.Errors = append(mErr.Errors, fmt.Errorf("error registering updated task with consul: %v", err)) - } return mErr.ErrorOrNil() } diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 3538f2f5edb6..bc80acf709e2 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -88,17 +88,9 @@ type ServiceClient struct { // syncCh triggers a sync in the main Run loop syncCh chan struct{} - // services and checks to be registered - regServices map[string]*api.AgentServiceRegistration - regChecks map[string]*api.AgentCheckRegistration - - // services and checks to be unregisterd - deregServices map[string]struct{} - deregChecks map[string]struct{} - - // script checks to be run() after their corresponding check is - // registered - regScripts map[string]*scriptCheck + // pending service and check operations + pending *consulOps + opsLock sync.Mutex // script check cancel funcs to be called before their corresponding // check is removed. Only accessed in sync() so not covered by regLock @@ -126,11 +118,7 @@ func NewServiceClient(consulClient AgentAPI, logger *log.Logger) *ServiceClient shutdownCh: make(chan struct{}), shutdownWait: defaultShutdownWait, syncCh: make(chan struct{}, 1), - regServices: make(map[string]*api.AgentServiceRegistration), - regChecks: make(map[string]*api.AgentCheckRegistration), - deregServices: make(map[string]struct{}), - deregChecks: make(map[string]struct{}), - regScripts: make(map[string]*scriptCheck), + pending: newConsulOps(), runningScripts: make(map[string]*scriptHandle), agentServices: make(map[string]struct{}, 8), agentChecks: make(map[string]struct{}, 8), @@ -182,67 +170,39 @@ func (c *ServiceClient) forceSync() { // sync enqueued operations. func (c *ServiceClient) sync() error { - // Shallow copy and reset the pending operations fields - c.regLock.Lock() - regServices := make(map[string]*api.AgentServiceRegistration, len(c.regServices)) - for k, v := range c.regServices { - regServices[k] = v - } - c.regServices = map[string]*api.AgentServiceRegistration{} - - regChecks := make(map[string]*api.AgentCheckRegistration, len(c.regChecks)) - for k, v := range c.regChecks { - regChecks[k] = v - } - c.regChecks = map[string]*api.AgentCheckRegistration{} - - regScripts := make(map[string]*scriptCheck, len(c.regScripts)) - for k, v := range c.regScripts { - regScripts[k] = v - } - c.regScripts = map[string]*scriptCheck{} - - deregServices := make(map[string]struct{}, len(c.deregServices)) - for k := range c.deregServices { - deregServices[k] = mark - } - c.deregServices = map[string]struct{}{} - - deregChecks := make(map[string]struct{}, len(c.deregChecks)) - for k := range c.deregChecks { - deregChecks[k] = mark - } - c.deregChecks = map[string]struct{}{} - c.regLock.Unlock() + c.opsLock.Lock() + ops := c.pending + c.pending = newConsulOps() + c.opsLock.Unlock() var err error - regServiceN, regCheckN, deregServiceN, deregCheckN := len(regServices), len(regChecks), len(deregServices), len(deregChecks) + msg := ops.String() // Register Services - for id, service := range regServices { + for id, service := range ops.regServices { if err = c.client.ServiceRegister(service); err != nil { goto ERROR } - delete(regServices, id) + delete(ops.regServices, id) } // Register Checks - for id, check := range regChecks { + for id, check := range ops.regChecks { if err = c.client.CheckRegister(check); err != nil { goto ERROR } - delete(regChecks, id) + delete(ops.regChecks, id) // Run the script for this check if one exists - if script, ok := regScripts[id]; ok { + if script, ok := ops.regScripts[id]; ok { // This check is a script check; run it c.runningScripts[id] = script.run() } } // Deregister Checks - for id := range deregChecks { + for id := range ops.deregChecks { if h, ok := c.runningScripts[id]; ok { // This check is a script check; stop it h.cancel() @@ -252,63 +212,28 @@ func (c *ServiceClient) sync() error { if err = c.client.CheckDeregister(id); err != nil { goto ERROR } - delete(deregChecks, id) + delete(ops.deregChecks, id) } // Deregister Services - for id := range deregServices { + for id := range ops.deregServices { if err = c.client.ServiceDeregister(id); err != nil { goto ERROR } - delete(deregServices, id) + delete(ops.deregServices, id) } - c.logger.Printf("[DEBUG] consul: registered %d services / %d checks; deregisterd %d services / %d checks", regServiceN, regCheckN, deregServiceN, deregCheckN) + c.logger.Printf("[DEBUG] consul: %s", msg) return nil //TODO Labels and gotos are nasty; move to a function? ERROR: - // An error occurred, repopulate the operation maps omitting any keys - // that have been updated while sync() ran. - c.regLock.Lock() - for id, service := range regServices { - if _, ok := c.regServices[id]; ok { - continue - } - if _, ok := c.deregServices[id]; ok { - continue - } - c.regServices[id] = service - } - for id, check := range regChecks { - if _, ok := c.regChecks[id]; ok { - continue - } - if _, ok := c.deregChecks[id]; ok { - continue - } - c.regChecks[id] = check - } - for id, script := range regScripts { - if _, ok := c.regScripts[id]; ok { - // a new version of this script was added, drop this one - continue - } - c.regScripts[id] = script - } - for id, _ := range deregServices { - if _, ok := c.regServices[id]; ok { - continue - } - c.deregServices[id] = mark - } - for id, _ := range deregChecks { - if _, ok := c.regChecks[id]; ok { - continue - } - c.deregChecks[id] = mark - } - c.regLock.Unlock() + // An error occurred, repopulate the operation maps but give + // precendence to new ops + c.opsLock.Lock() + ops.merge(c.pending) + c.pending = ops + c.opsLock.Unlock() return err } @@ -317,10 +242,9 @@ ERROR: // // Agents will be deregistered when Shutdown is called. func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) error { - regs := make([]*api.AgentServiceRegistration, len(services)) - checks := make([]*api.AgentCheckRegistration, 0, len(services)) + ops := newConsulOps() - for i, service := range services { + for _, service := range services { id := makeAgentServiceID(role, service) host, rawport, err := net.SplitHostPort(service.PortLabel) if err != nil { @@ -337,7 +261,7 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) Address: host, Port: port, } - regs[i] = serviceReg + ops.regServices[id] = serviceReg for _, check := range service.Checks { checkID := createCheckID(id, check) @@ -360,22 +284,79 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) if err != nil { return fmt.Errorf("failed to add check %q: %v", check.Name, err) } - checks = append(checks, checkReg) + ops.regChecks[checkID] = checkReg } } // Now add them to the registration queue - c.enqueueRegs(regs, checks, nil) + c.opsLock.Lock() + c.pending.merge(ops) + c.opsLock.Unlock() // Record IDs for deregistering on shutdown c.agentLock.Lock() - for _, s := range regs { - c.agentServices[s.ID] = mark + for id := range ops.regServices { + c.agentServices[id] = mark } - for _, ch := range checks { - c.agentChecks[ch.ID] = mark + for id := range ops.regChecks { + c.agentChecks[id] = mark } c.agentLock.Unlock() + c.forceSync() + return nil +} + +type addrParser func(portLabel string) (string, int) + +// makeCheckReg adds a check reg to operations. +func (c *ServiceClient) makeCheckReg(ops *consulOps, check *structs.ServiceCheck, + service *api.AgentServiceRegistration, exec ScriptExecutor, parseAddr addrParser) error { + + checkID := createCheckID(service.ID, check) + if check.Type == structs.ServiceCheckScript { + if exec == nil { + return fmt.Errorf("driver doesn't support script checks") + } + ops.regScripts[checkID] = newScriptCheck( + checkID, check, exec, c.client, c.logger, c.shutdownCh) + } + host, port := service.Address, service.Port + if check.PortLabel != "" { + host, port = parseAddr(check.PortLabel) + } + checkReg, err := createCheckReg(service.ID, checkID, check, host, port) + if err != nil { + return fmt.Errorf("failed to add check %q: %v", check.Name, err) + } + ops.regChecks[checkID] = checkReg + return nil +} + +// serviceRegs creates service registrations, check registrations, and script +// checks from a service. +func (c *ServiceClient) serviceRegs(ops *consulOps, allocID string, service *structs.Service, + exec ScriptExecutor, task *structs.Task) error { + + id := makeTaskServiceID(allocID, task.Name, service) + host, port := task.FindHostAndPortFor(service.PortLabel) + serviceReg := &api.AgentServiceRegistration{ + ID: id, + Name: service.Name, + Tags: make([]string, len(service.Tags)), + Address: host, + Port: port, + } + // copy isn't strictly necessary but can avoid bugs especially + // with tests that may reuse Tasks + copy(serviceReg.Tags, service.Tags) + ops.regServices[id] = serviceReg + + for _, check := range service.Checks { + err := c.makeCheckReg(ops, check, serviceReg, exec, task.FindHostAndPortFor) + if err != nil { + return err + } + } return nil } @@ -384,48 +365,99 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) // // Actual communication with Consul is done asynchrously (see Run). func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec ScriptExecutor) error { - regs := make([]*api.AgentServiceRegistration, len(task.Services)) - checks := make([]*api.AgentCheckRegistration, 0, len(task.Services)*2) // just guess at size - var scriptChecks []*scriptCheck + ops := newConsulOps() + for _, service := range task.Services { + if err := c.serviceRegs(ops, allocID, service, exec, task); err != nil { + return err + } + } - for i, service := range task.Services { - id := makeTaskServiceID(allocID, task.Name, service) - host, port := task.FindHostAndPortFor(service.PortLabel) - serviceReg := &api.AgentServiceRegistration{ - ID: id, - Name: service.Name, - Tags: make([]string, len(service.Tags)), - Address: host, - Port: port, + // Now add them to the registration queue + c.opsLock.Lock() + c.pending.merge(ops) + c.opsLock.Unlock() + c.forceSync() + return nil +} + +// UpdateTask in Consul. Does not alter the service if only checks have +// changed. +func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Task, exec ScriptExecutor) error { + ops := newConsulOps() + + existingIDs := make(map[string]*structs.Service, len(existing.Services)) + for _, s := range existing.Services { + existingIDs[makeTaskServiceID(allocID, existing.Name, s)] = s + c.logger.Printf("[XXX] EXISTING: %s", makeTaskServiceID(allocID, existing.Name, s)) + } + newIDs := make(map[string]*structs.Service, len(newTask.Services)) + for _, s := range newTask.Services { + newIDs[makeTaskServiceID(allocID, newTask.Name, s)] = s + c.logger.Printf("[XXX] UPDATED : %s", makeTaskServiceID(allocID, newTask.Name, s)) + } + + parseAddr := newTask.FindHostAndPortFor + + // Loop over existing Service IDs to see if they have been removed or + // updated. + for existingID, existingSvc := range existingIDs { + newSvc, ok := newIDs[existingID] + if !ok { + // Existing sevice entry removed + ops.deregServices[existingID] = mark + for _, check := range existingSvc.Checks { + ops.deregChecks[createCheckID(existingID, check)] = mark + } + continue } - // copy isn't strictly necessary but can avoid bugs especially - // with tests that may reuse Tasks - copy(serviceReg.Tags, service.Tags) - regs[i] = serviceReg - for _, check := range service.Checks { - checkID := createCheckID(id, check) + // Manipulating checks is cheap and easy, so just remove old and add new + for _, check := range existingSvc.Checks { + ops.deregChecks[createCheckID(existingID, check)] = mark + } + + // Register new checks + for _, check := range newSvc.Checks { + checkID := createCheckID(existingID, check) + // Don't deregister this check if it hasn't changed + delete(ops.deregChecks, checkID) if check.Type == structs.ServiceCheckScript { if exec == nil { - return fmt.Errorf("driver %q doesn't support script checks", task.Driver) + return fmt.Errorf("driver doesn't support script checks") } - scriptChecks = append(scriptChecks, newScriptCheck(checkID, check, exec, c.client, c.logger, c.shutdownCh)) + ops.regScripts[checkID] = newScriptCheck( + checkID, check, exec, c.client, c.logger, c.shutdownCh) } - host, port := serviceReg.Address, serviceReg.Port + host, port := parseAddr(existingSvc.PortLabel) if check.PortLabel != "" { - host, port = task.FindHostAndPortFor(check.PortLabel) + host, port = parseAddr(check.PortLabel) } - checkReg, err := createCheckReg(id, checkID, check, host, port) + checkReg, err := createCheckReg(existingID, checkID, check, host, port) if err != nil { - return fmt.Errorf("failed to add check %q: %v", check.Name, err) + return err } - checks = append(checks, checkReg) + ops.regChecks[checkID] = checkReg } + // Service hasn't changed and checks are updated so don't + // process this service again later + delete(newIDs, existingID) } - // Now add them to the registration queue - c.enqueueRegs(regs, checks, scriptChecks) + // Any remaining services should just be enqueued directly + for _, newSvc := range newIDs { + err := c.serviceRegs(ops, allocID, newSvc, exec, newTask) + if err != nil { + return err + } + } + + // Finally enqueue the updates and force sync + c.opsLock.Lock() + c.pending.merge(ops) + c.opsLock.Unlock() + + c.forceSync() return nil } @@ -433,62 +465,20 @@ func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec Sc // // Actual communication with Consul is done asynchrously (see Run). func (c *ServiceClient) RemoveTask(allocID string, task *structs.Task) { - deregs := make([]string, len(task.Services)) - checks := make([]string, 0, len(task.Services)*2) // just guess at size + ops := newConsulOps() - for i, service := range task.Services { + for _, service := range task.Services { id := makeTaskServiceID(allocID, task.Name, service) - deregs[i] = id + ops.deregServices[id] = mark for _, check := range service.Checks { - checks = append(checks, createCheckID(id, check)) + ops.deregChecks[createCheckID(id, check)] = mark } } // Now add them to the deregistration fields; main Run loop will update - c.enqueueDeregs(deregs, checks) -} - -// enqueueRegs enqueues service and check registrations for the next time -// operations are sync'd to Consul. -func (c *ServiceClient) enqueueRegs(regs []*api.AgentServiceRegistration, checks []*api.AgentCheckRegistration, scriptChecks []*scriptCheck) { c.regLock.Lock() - for _, reg := range regs { - // Add reg - c.regServices[reg.ID] = reg - // Make sure it's not being removed - delete(c.deregServices, reg.ID) - } - for _, check := range checks { - // Add check - c.regChecks[check.ID] = check - // Make sure it's not being removed - delete(c.deregChecks, check.ID) - } - for _, script := range scriptChecks { - c.regScripts[script.id] = script - } - c.regLock.Unlock() - - c.forceSync() -} - -// enqueueDeregs enqueues service and check removals for the next time -// operations are sync'd to Consul. -func (c *ServiceClient) enqueueDeregs(deregs []string, checks []string) { - c.regLock.Lock() - for _, dereg := range deregs { - // Add dereg - c.deregServices[dereg] = mark - // Make sure it's not being added - delete(c.regServices, dereg) - } - for _, check := range checks { - // Add check for removal - c.deregChecks[check] = mark - // Make sure it's not being added - delete(c.regChecks, check) - } + c.pending.merge(ops) c.regLock.Unlock() c.forceSync() diff --git a/command/agent/consul/ops.go b/command/agent/consul/ops.go new file mode 100644 index 000000000000..0592b9a48a0e --- /dev/null +++ b/command/agent/consul/ops.go @@ -0,0 +1,61 @@ +package consul + +import ( + "fmt" + + "github.com/hashicorp/consul/api" +) + +type consulOps struct { + // services and checks to be registered + regServices map[string]*api.AgentServiceRegistration + regChecks map[string]*api.AgentCheckRegistration + + // services and checks to be unregisterd + deregServices map[string]struct{} + deregChecks map[string]struct{} + + // script checks to be run() after their corresponding check is + // registered + regScripts map[string]*scriptCheck +} + +func newConsulOps() *consulOps { + return &consulOps{ + regServices: make(map[string]*api.AgentServiceRegistration), + regChecks: make(map[string]*api.AgentCheckRegistration), + deregServices: make(map[string]struct{}), + deregChecks: make(map[string]struct{}), + regScripts: make(map[string]*scriptCheck), + } +} + +// merge newer operations. New operations registrations override existing +// deregistrations. +func (c *consulOps) merge(newer *consulOps) { + for id, service := range newer.regServices { + delete(c.deregServices, id) + c.regServices[id] = service + } + for id, check := range newer.regChecks { + delete(c.deregChecks, id) + c.regChecks[id] = check + } + for id, script := range newer.regScripts { + c.regScripts[id] = script + } + for id, _ := range newer.deregServices { + delete(c.regServices, id) + c.deregServices[id] = mark + } + for id, _ := range newer.deregChecks { + delete(c.regChecks, id) + delete(c.regScripts, id) + c.deregChecks[id] = mark + } +} + +func (c *consulOps) String() string { + return fmt.Sprintf("registered %d services / %d checks; deregisterd %d services / %d checks", + len(c.regServices), len(c.regChecks), len(c.deregServices), len(c.deregChecks)) +} diff --git a/command/agent/consul/unit_test.go b/command/agent/consul/unit_test.go index 59ea83d7b12f..16de58449576 100644 --- a/command/agent/consul/unit_test.go +++ b/command/agent/consul/unit_test.go @@ -323,8 +323,8 @@ func TestConsul_ShutdownOK(t *testing.T) { } // Nothing should be enqueued anymore - enqueued := (len(ctx.ServiceClient.regServices) + len(ctx.ServiceClient.deregServices) + - len(ctx.ServiceClient.regChecks) + len(ctx.ServiceClient.deregChecks)) + enqueued := (len(ctx.ServiceClient.pending.regServices) + len(ctx.ServiceClient.pending.deregServices) + + len(ctx.ServiceClient.pending.regChecks) + len(ctx.ServiceClient.pending.deregChecks)) if enqueued > 0 { t.Errorf("%d operations still enqueued", enqueued) } From 80882f3e0e6030685b4370f08622980d4db4f40d Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Fri, 7 Apr 2017 17:10:26 -0700 Subject: [PATCH 05/38] Switch ServiceClient to synchronizing state Previously it applied a stream of operations. Reconciling state is less complex and error prone at the cost of slightly higher CPU/memory usage. --- command/agent/consul/client.go | 405 ++++++++++++++++++------------ command/agent/consul/ops.go | 61 ----- command/agent/consul/script.go | 10 +- command/agent/consul/unit_test.go | 160 +++++++----- 4 files changed, 338 insertions(+), 298 deletions(-) delete mode 100644 command/agent/consul/ops.go diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index bc80acf709e2..93f3721f598f 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -23,8 +23,12 @@ const ( // services nomadServicePrefix = "_nomad" - // The periodic time interval for syncing services and checks with Consul - defaultSyncInterval = 6 * time.Second + // defaultRetryInterval is how quickly to retry syncing services and + // checks to Consul when an error occurs. Will backoff up to a max. + defaultRetryInterval = time.Second + + // defaultMaxRetryInterval is the default max retry interval. + defaultMaxRetryInterval = 30 * time.Second // ttlCheckBuffer is the time interval that Nomad can take to report Consul // the check result @@ -62,6 +66,8 @@ type CatalogAPI interface { // AgentAPI is the consul/api.Agent API used by Nomad. type AgentAPI interface { + Services() (map[string]*api.AgentService, error) + Checks() (map[string]*api.AgentCheck, error) CheckRegister(check *api.AgentCheckRegistration) error CheckDeregister(checkID string) error ServiceRegister(service *api.AgentServiceRegistration) error @@ -69,14 +75,30 @@ type AgentAPI interface { UpdateTTL(id, output, status string) error } +// addrParser is usually the Task.FindHostAndPortFor method for turning a +// portLabel into an address and port. +type addrParser func(portLabel string) (string, int) + +// operations are submitted to the main loop via commit() for synchronizing +// with Consul. +type operations struct { + regServices []*api.AgentServiceRegistration + regChecks []*api.AgentCheckRegistration + scripts []*scriptCheck + + deregServices []string + deregChecks []string +} + // ServiceClient handles task and agent service registration with Consul. type ServiceClient struct { - client AgentAPI - logger *log.Logger - retryInterval time.Duration + client AgentAPI + logger *log.Logger + retryInterval time.Duration + maxRetryInterval time.Duration - // runningCh is closed when the main Run loop exits - runningCh chan struct{} + // exitCh is closed when the main Run loop exits + exitCh chan struct{} // shutdownCh is closed when the client should shutdown shutdownCh chan struct{} @@ -85,156 +107,210 @@ type ServiceClient struct { // sync() to finish. Defaults to defaultShutdownWait shutdownWait time.Duration - // syncCh triggers a sync in the main Run loop - syncCh chan struct{} + opCh chan *operations - // pending service and check operations - pending *consulOps - opsLock sync.Mutex - - // script check cancel funcs to be called before their corresponding - // check is removed. Only accessed in sync() so not covered by regLock + services map[string]*api.AgentServiceRegistration + checks map[string]*api.AgentCheckRegistration + scripts map[string]*scriptCheck runningScripts map[string]*scriptHandle - // regLock must be held while accessing reg and dereg maps - regLock sync.Mutex - - // Registered agent services and checks + // agent services and checks record entries for the agent itself which + // should be removed on shutdown agentServices map[string]struct{} agentChecks map[string]struct{} - - // agentLock must be held while accessing agent maps - agentLock sync.Mutex + agentLock sync.Mutex } // NewServiceClient creates a new Consul ServiceClient from an existing Consul API // Client and logger. func NewServiceClient(consulClient AgentAPI, logger *log.Logger) *ServiceClient { return &ServiceClient{ - client: consulClient, - logger: logger, - retryInterval: defaultSyncInterval, - runningCh: make(chan struct{}), - shutdownCh: make(chan struct{}), - shutdownWait: defaultShutdownWait, - syncCh: make(chan struct{}, 1), - pending: newConsulOps(), - runningScripts: make(map[string]*scriptHandle), - agentServices: make(map[string]struct{}, 8), - agentChecks: make(map[string]struct{}, 8), + client: consulClient, + logger: logger, + retryInterval: defaultRetryInterval, + maxRetryInterval: defaultMaxRetryInterval, + exitCh: make(chan struct{}), + shutdownCh: make(chan struct{}), + shutdownWait: defaultShutdownWait, + opCh: make(chan *operations, 8), + services: make(map[string]*api.AgentServiceRegistration), + checks: make(map[string]*api.AgentCheckRegistration), + scripts: make(map[string]*scriptCheck), + runningScripts: make(map[string]*scriptHandle), + agentServices: make(map[string]struct{}), + agentChecks: make(map[string]struct{}), } } // Run the Consul main loop which retries operations against Consul. It should // be called exactly once. func (c *ServiceClient) Run() { - defer close(c.runningCh) - timer := time.NewTimer(0) - defer timer.Stop() + defer close(c.exitCh) + retryTimer := time.NewTimer(0) + <-retryTimer.C // disabled by default + failures := 0 + for { + select { + case <-retryTimer.C: + case <-c.shutdownCh: + case ops := <-c.opCh: + c.merge(ops) + } - // Drain the initial tick so we don't sync until instructed - <-timer.C + if err := c.sync(); err != nil { + if failures == 0 { + c.logger.Printf("[WARN] consul: failed to update services in Consul: %v", err) + } + failures++ + if !retryTimer.Stop() { + <-retryTimer.C + } + backoff := c.retryInterval * time.Duration(failures) + if backoff > c.maxRetryInterval { + backoff = c.maxRetryInterval + } + retryTimer.Reset(backoff) + } else { + if failures > 0 { + c.logger.Printf("[INFO] consul: successfully updated services in Consul") + failures = 0 + } + } - lastOk := true - for { select { - case <-c.syncCh: - timer.Reset(0) - case <-timer.C: - if err := c.sync(); err != nil { - if lastOk { - lastOk = false - c.logger.Printf("[WARN] consul: failed to update services in Consul: %v", err) - } - timer.Reset(c.retryInterval) - } else { - if !lastOk { - c.logger.Printf("[INFO] consul: successfully updated services in Consul") - lastOk = true + case <-c.shutdownCh: + // Exit only after sync'ing all outstanding operations + if len(c.opCh) > 0 { + for len(c.opCh) > 0 { + c.merge(<-c.opCh) } + continue } - case <-c.shutdownCh: return + default: } + } } -// forceSync asynchronously causes a sync to happen. Any operations enqueued -// prior to calling forceSync will be synced. -func (c *ServiceClient) forceSync() { +// commit operations and returns false if shutdown signalled before committing. +func (c *ServiceClient) commit(ops *operations) bool { select { - case c.syncCh <- mark: - default: + case c.opCh <- ops: + return true + case <-c.shutdownCh: + return false + } +} + +// merge registrations into state map prior to sync'ing with Consul +func (c *ServiceClient) merge(ops *operations) { + for _, s := range ops.regServices { + c.services[s.ID] = s + } + for _, check := range ops.regChecks { + c.checks[check.ID] = check + } + for _, s := range ops.scripts { + c.scripts[s.id] = s + } + for _, sid := range ops.deregServices { + delete(c.services, sid) + } + for _, cid := range ops.deregChecks { + if script, ok := c.runningScripts[cid]; ok { + script.cancel() + delete(c.scripts, cid) + } + delete(c.checks, cid) } } // sync enqueued operations. func (c *ServiceClient) sync() error { - c.opsLock.Lock() - ops := c.pending - c.pending = newConsulOps() - c.opsLock.Unlock() + sreg, creg, sdereg, cdereg := 0, 0, 0, 0 - var err error + consulServices, err := c.client.Services() + if err != nil { + return fmt.Errorf("error querying Consul services: %v", err) + } - msg := ops.String() + consulChecks, err := c.client.Checks() + if err != nil { + return fmt.Errorf("error querying Consul checks: %v", err) + } - // Register Services - for id, service := range ops.regServices { - if err = c.client.ServiceRegister(service); err != nil { - goto ERROR + // Remove Nomad services in Consul but unknown locally + for id := range consulServices { + if _, ok := c.services[id]; ok { + // Known service, skip + continue } - delete(ops.regServices, id) + if !isNomadService(id) { + // Not managed by Nomad, skip + continue + } + // Unknown Nomad managed service; kill + if err := c.client.ServiceDeregister(id); err != nil { + return err + } + sdereg++ } - // Register Checks - for id, check := range ops.regChecks { - if err = c.client.CheckRegister(check); err != nil { - goto ERROR + // Add Nomad services missing from Consul + for id, service := range c.services { + if _, ok := consulServices[id]; ok { + // Already in Consul; skipping + continue } - delete(ops.regChecks, id) - - // Run the script for this check if one exists - if script, ok := ops.regScripts[id]; ok { - // This check is a script check; run it - c.runningScripts[id] = script.run() + if err = c.client.ServiceRegister(service); err != nil { + return err } + sreg++ } - // Deregister Checks - for id := range ops.deregChecks { - if h, ok := c.runningScripts[id]; ok { - // This check is a script check; stop it - h.cancel() - delete(c.runningScripts, id) + // Remove Nomad checks in Consul but unknown locally + for id, check := range consulChecks { + if _, ok := c.checks[id]; ok { + // Known check, skip + continue } - - if err = c.client.CheckDeregister(id); err != nil { - goto ERROR + if !isNomadService(check.ServiceID) { + // Not managed by Nomad, skip + continue } - delete(ops.deregChecks, id) + // Unknown Nomad managed check; kill + if err := c.client.CheckDeregister(id); err != nil { + return err + } + cdereg++ } - // Deregister Services - for id := range ops.deregServices { - if err = c.client.ServiceDeregister(id); err != nil { - goto ERROR + // Add Nomad checks missing from Consul + for id, check := range c.checks { + if _, ok := consulChecks[id]; ok { + // Already in Consul; skipping + continue + } + if err := c.client.CheckRegister(check); err != nil { + return err + } + creg++ + + // Handle starting scripts + if script, ok := c.scripts[id]; ok { + // If it's already running, don't run it again + if _, running := c.runningScripts[id]; running { + continue + } + // Not running, start and store the handle + c.runningScripts[id] = script.run() } - delete(ops.deregServices, id) } - c.logger.Printf("[DEBUG] consul: %s", msg) + c.logger.Printf("[DEBUG] consul.sync: registered %d services, %d checks; deregistered %d services, %d checks", + sreg, creg, sdereg, cdereg) return nil - - //TODO Labels and gotos are nasty; move to a function? -ERROR: - // An error occurred, repopulate the operation maps but give - // precendence to new ops - c.opsLock.Lock() - ops.merge(c.pending) - c.pending = ops - c.opsLock.Unlock() - return err } // RegisterAgent registers Nomad agents (client or server). Script checks are @@ -242,7 +318,7 @@ ERROR: // // Agents will be deregistered when Shutdown is called. func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) error { - ops := newConsulOps() + ops := operations{} for _, service := range services { id := makeAgentServiceID(role, service) @@ -261,7 +337,7 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) Address: host, Port: port, } - ops.regServices[id] = serviceReg + ops.regServices = append(ops.regServices, serviceReg) for _, check := range service.Checks { checkID := createCheckID(id, check) @@ -284,32 +360,30 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) if err != nil { return fmt.Errorf("failed to add check %q: %v", check.Name, err) } - ops.regChecks[checkID] = checkReg + ops.regChecks = append(ops.regChecks, checkReg) } } // Now add them to the registration queue - c.opsLock.Lock() - c.pending.merge(ops) - c.opsLock.Unlock() + if ok := c.commit(&ops); !ok { + // shutting down, exit + return nil + } // Record IDs for deregistering on shutdown c.agentLock.Lock() - for id := range ops.regServices { - c.agentServices[id] = mark + for _, id := range ops.regServices { + c.agentServices[id.ID] = mark } - for id := range ops.regChecks { - c.agentChecks[id] = mark + for _, id := range ops.regChecks { + c.agentChecks[id.ID] = mark } c.agentLock.Unlock() - c.forceSync() return nil } -type addrParser func(portLabel string) (string, int) - // makeCheckReg adds a check reg to operations. -func (c *ServiceClient) makeCheckReg(ops *consulOps, check *structs.ServiceCheck, +func (c *ServiceClient) makeCheckReg(ops *operations, check *structs.ServiceCheck, service *api.AgentServiceRegistration, exec ScriptExecutor, parseAddr addrParser) error { checkID := createCheckID(service.ID, check) @@ -317,8 +391,9 @@ func (c *ServiceClient) makeCheckReg(ops *consulOps, check *structs.ServiceCheck if exec == nil { return fmt.Errorf("driver doesn't support script checks") } - ops.regScripts[checkID] = newScriptCheck( - checkID, check, exec, c.client, c.logger, c.shutdownCh) + ops.scripts = append(ops.scripts, newScriptCheck( + checkID, check, exec, c.client, c.logger, c.shutdownCh)) + } host, port := service.Address, service.Port if check.PortLabel != "" { @@ -328,13 +403,13 @@ func (c *ServiceClient) makeCheckReg(ops *consulOps, check *structs.ServiceCheck if err != nil { return fmt.Errorf("failed to add check %q: %v", check.Name, err) } - ops.regChecks[checkID] = checkReg + ops.regChecks = append(ops.regChecks, checkReg) return nil } // serviceRegs creates service registrations, check registrations, and script // checks from a service. -func (c *ServiceClient) serviceRegs(ops *consulOps, allocID string, service *structs.Service, +func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *structs.Service, exec ScriptExecutor, task *structs.Task) error { id := makeTaskServiceID(allocID, task.Name, service) @@ -349,7 +424,7 @@ func (c *ServiceClient) serviceRegs(ops *consulOps, allocID string, service *str // copy isn't strictly necessary but can avoid bugs especially // with tests that may reuse Tasks copy(serviceReg.Tags, service.Tags) - ops.regServices[id] = serviceReg + ops.regServices = append(ops.regServices, serviceReg) for _, check := range service.Checks { err := c.makeCheckReg(ops, check, serviceReg, exec, task.FindHostAndPortFor) @@ -365,35 +440,28 @@ func (c *ServiceClient) serviceRegs(ops *consulOps, allocID string, service *str // // Actual communication with Consul is done asynchrously (see Run). func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec ScriptExecutor) error { - ops := newConsulOps() + ops := &operations{} for _, service := range task.Services { if err := c.serviceRegs(ops, allocID, service, exec, task); err != nil { return err } } - - // Now add them to the registration queue - c.opsLock.Lock() - c.pending.merge(ops) - c.opsLock.Unlock() - c.forceSync() + c.commit(ops) return nil } // UpdateTask in Consul. Does not alter the service if only checks have // changed. func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Task, exec ScriptExecutor) error { - ops := newConsulOps() + ops := &operations{} existingIDs := make(map[string]*structs.Service, len(existing.Services)) for _, s := range existing.Services { existingIDs[makeTaskServiceID(allocID, existing.Name, s)] = s - c.logger.Printf("[XXX] EXISTING: %s", makeTaskServiceID(allocID, existing.Name, s)) } newIDs := make(map[string]*structs.Service, len(newTask.Services)) for _, s := range newTask.Services { newIDs[makeTaskServiceID(allocID, newTask.Name, s)] = s - c.logger.Printf("[XXX] UPDATED : %s", makeTaskServiceID(allocID, newTask.Name, s)) } parseAddr := newTask.FindHostAndPortFor @@ -404,29 +472,38 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta newSvc, ok := newIDs[existingID] if !ok { // Existing sevice entry removed - ops.deregServices[existingID] = mark + ops.deregServices = append(ops.deregServices, existingID) for _, check := range existingSvc.Checks { - ops.deregChecks[createCheckID(existingID, check)] = mark + ops.deregChecks = append(ops.deregChecks, createCheckID(existingID, check)) } continue } - // Manipulating checks is cheap and easy, so just remove old and add new + // Service exists and wasn't updated, don't add it later + delete(newIDs, existingID) + + // Check to see what checks were updated + existingChecks := make(map[string]struct{}, len(existingSvc.Checks)) for _, check := range existingSvc.Checks { - ops.deregChecks[createCheckID(existingID, check)] = mark + existingChecks[createCheckID(existingID, check)] = mark } // Register new checks for _, check := range newSvc.Checks { checkID := createCheckID(existingID, check) - // Don't deregister this check if it hasn't changed - delete(ops.deregChecks, checkID) + if _, exists := existingChecks[checkID]; exists { + // Check already exists; skip it + delete(existingChecks, checkID) + continue + } + + // New check, register it if check.Type == structs.ServiceCheckScript { if exec == nil { return fmt.Errorf("driver doesn't support script checks") } - ops.regScripts[checkID] = newScriptCheck( - checkID, check, exec, c.client, c.logger, c.shutdownCh) + ops.scripts = append(ops.scripts, newScriptCheck( + checkID, check, exec, c.client, c.logger, c.shutdownCh)) } host, port := parseAddr(existingSvc.PortLabel) if check.PortLabel != "" { @@ -436,12 +513,13 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta if err != nil { return err } - ops.regChecks[checkID] = checkReg + ops.regChecks = append(ops.regChecks, checkReg) } - // Service hasn't changed and checks are updated so don't - // process this service again later - delete(newIDs, existingID) + // Remove existing checks not in updated service + for cid := range existingChecks { + ops.deregChecks = append(ops.deregChecks, cid) + } } // Any remaining services should just be enqueued directly @@ -452,12 +530,7 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta } } - // Finally enqueue the updates and force sync - c.opsLock.Lock() - c.pending.merge(ops) - c.opsLock.Unlock() - - c.forceSync() + c.commit(ops) return nil } @@ -465,23 +538,19 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta // // Actual communication with Consul is done asynchrously (see Run). func (c *ServiceClient) RemoveTask(allocID string, task *structs.Task) { - ops := newConsulOps() + ops := operations{} for _, service := range task.Services { id := makeTaskServiceID(allocID, task.Name, service) - ops.deregServices[id] = mark + ops.deregServices = append(ops.deregServices, id) for _, check := range service.Checks { - ops.deregChecks[createCheckID(id, check)] = mark + ops.deregChecks = append(ops.deregChecks, createCheckID(id, check)) } } // Now add them to the deregistration fields; main Run loop will update - c.regLock.Lock() - c.pending.merge(ops) - c.regLock.Unlock() - - c.forceSync() + c.commit(&ops) } // Shutdown the Consul client. Update running task registations and deregister @@ -516,13 +585,9 @@ func (c *ServiceClient) Shutdown() error { } c.agentLock.Unlock() - // Wait for Run to finish any outstanding sync() calls and exit + // Wait for Run to finish any outstanding operations and exit select { - case <-c.runningCh: - // sync one last time to ensure all enqueued operations are applied - if err := c.sync(); err != nil { - mErr.Errors = append(mErr.Errors, err) - } + case <-c.exitCh: case <-deadline: // Don't wait forever though mErr.Errors = append(mErr.Errors, fmt.Errorf("timed out waiting for Consul operations to complete")) @@ -621,3 +686,9 @@ func createCheckReg(serviceID, checkID string, check *structs.ServiceCheck, host } return &chkReg, nil } + +// isNomadService returns true if the ID matches the pattern of a Nomad managed +// service. +func isNomadService(id string) bool { + return strings.HasPrefix(id, nomadServicePrefix) +} diff --git a/command/agent/consul/ops.go b/command/agent/consul/ops.go deleted file mode 100644 index 0592b9a48a0e..000000000000 --- a/command/agent/consul/ops.go +++ /dev/null @@ -1,61 +0,0 @@ -package consul - -import ( - "fmt" - - "github.com/hashicorp/consul/api" -) - -type consulOps struct { - // services and checks to be registered - regServices map[string]*api.AgentServiceRegistration - regChecks map[string]*api.AgentCheckRegistration - - // services and checks to be unregisterd - deregServices map[string]struct{} - deregChecks map[string]struct{} - - // script checks to be run() after their corresponding check is - // registered - regScripts map[string]*scriptCheck -} - -func newConsulOps() *consulOps { - return &consulOps{ - regServices: make(map[string]*api.AgentServiceRegistration), - regChecks: make(map[string]*api.AgentCheckRegistration), - deregServices: make(map[string]struct{}), - deregChecks: make(map[string]struct{}), - regScripts: make(map[string]*scriptCheck), - } -} - -// merge newer operations. New operations registrations override existing -// deregistrations. -func (c *consulOps) merge(newer *consulOps) { - for id, service := range newer.regServices { - delete(c.deregServices, id) - c.regServices[id] = service - } - for id, check := range newer.regChecks { - delete(c.deregChecks, id) - c.regChecks[id] = check - } - for id, script := range newer.regScripts { - c.regScripts[id] = script - } - for id, _ := range newer.deregServices { - delete(c.regServices, id) - c.deregServices[id] = mark - } - for id, _ := range newer.deregChecks { - delete(c.regChecks, id) - delete(c.regScripts, id) - c.deregChecks[id] = mark - } -} - -func (c *consulOps) String() string { - return fmt.Sprintf("registered %d services / %d checks; deregisterd %d services / %d checks", - len(c.regServices), len(c.regChecks), len(c.deregServices), len(c.deregChecks)) -} diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 1c63877a3a7a..96c52e741069 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -27,10 +27,11 @@ func (s *scriptHandle) wait() <-chan struct{} { } type scriptCheck struct { - id string - check *structs.ServiceCheck - exec ScriptExecutor - agent heartbeater + id string + check *structs.ServiceCheck + exec ScriptExecutor + agent heartbeater + running bool // lastCheckOk is true if the last check was ok; otherwise false lastCheckOk bool @@ -132,5 +133,6 @@ func (s *scriptCheck) run() *scriptHandle { } } }() + s.running = true return &scriptHandle{cancel: cancel, done: done} } diff --git a/command/agent/consul/unit_test.go b/command/agent/consul/unit_test.go index 16de58449576..66934663b941 100644 --- a/command/agent/consul/unit_test.go +++ b/command/agent/consul/unit_test.go @@ -62,14 +62,24 @@ func (t *testFakeCtx) Exec(ctx context.Context, cmd string, args []string) ([]by return t.ExecFunc(ctx, cmd, args) } +var errNoOps = fmt.Errorf("testing error: no pending operations") + +// syncOps simulates one iteration of the ServiceClient.Run loop and returns +// any errors returned by sync() or errNoOps if no pending operations. +func (t *testFakeCtx) syncOnce() error { + select { + case ops := <-t.ServiceClient.opCh: + t.ServiceClient.merge(ops) + return t.ServiceClient.sync() + default: + return errNoOps + } +} + // setupFake creates a testFakeCtx with a ServiceClient backed by a fakeConsul. // A test Task is also provided. func setupFake() *testFakeCtx { - fc := &fakeConsul{ - services: make(map[string]*api.AgentServiceRegistration), - checks: make(map[string]*api.AgentCheckRegistration), - checkTTLs: make(map[string]int), - } + fc := newFakeConsul() return &testFakeCtx{ ServiceClient: NewServiceClient(fc, testLogger()), FakeConsul: fc, @@ -86,6 +96,55 @@ type fakeConsul struct { // when UpdateTTL is called the check ID will have its counter inc'd checkTTLs map[string]int + + // What check status to return from Checks() + checkStatus string +} + +func newFakeConsul() *fakeConsul { + return &fakeConsul{ + services: make(map[string]*api.AgentServiceRegistration), + checks: make(map[string]*api.AgentCheckRegistration), + checkTTLs: make(map[string]int), + checkStatus: api.HealthPassing, + } +} + +func (c *fakeConsul) Services() (map[string]*api.AgentService, error) { + c.mu.Lock() + defer c.mu.Unlock() + + r := make(map[string]*api.AgentService, len(c.services)) + for k, v := range c.services { + r[k] = &api.AgentService{ + ID: v.ID, + Service: v.Name, + Tags: make([]string, len(v.Tags)), + Port: v.Port, + Address: v.Address, + EnableTagOverride: v.EnableTagOverride, + } + copy(r[k].Tags, v.Tags) + } + return r, nil +} + +func (c *fakeConsul) Checks() (map[string]*api.AgentCheck, error) { + c.mu.Lock() + defer c.mu.Unlock() + + r := make(map[string]*api.AgentCheck, len(c.checks)) + for k, v := range c.checks { + r[k] = &api.AgentCheck{ + CheckID: v.ID, + Name: v.Name, + Status: c.checkStatus, + Notes: v.Notes, + ServiceID: v.ServiceID, + ServiceName: c.services[v.ServiceID].Name, + } + } + return r, nil } func (c *fakeConsul) CheckRegister(check *api.AgentCheckRegistration) error { @@ -137,8 +196,7 @@ func TestConsul_ChangeTags(t *testing.T) { t.Fatalf("unexpected error registering task: %v", err) } - // Manually call sync() since Run() isn't running - if err := ctx.ServiceClient.sync(); err != nil { + if err := ctx.syncOnce(); err != nil { t.Fatalf("unexpected error syncing task: %v", err) } @@ -157,13 +215,13 @@ func TestConsul_ChangeTags(t *testing.T) { } } - // Changing a tag removes old entry before adding new one - ctx.ServiceClient.RemoveTask("allocid", ctx.Task) + origTask := ctx.Task + ctx.Task = testTask() ctx.Task.Services[0].Tags[0] = "newtag" - if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { + if err := ctx.ServiceClient.UpdateTask("allocid", origTask, ctx.Task, nil); err != nil { t.Fatalf("unexpected error registering task: %v", err) } - if err := ctx.ServiceClient.sync(); err != nil { + if err := ctx.syncOnce(); err != nil { t.Fatalf("unexpected error syncing task: %v", err) } @@ -193,8 +251,7 @@ func TestConsul_RegServices(t *testing.T) { t.Fatalf("unexpected error registering task: %v", err) } - // Manually call sync() since Run() isn't running - if err := ctx.ServiceClient.sync(); err != nil { + if err := ctx.syncOnce(); err != nil { t.Fatalf("unexpected error syncing task: %v", err) } @@ -232,7 +289,7 @@ func TestConsul_RegServices(t *testing.T) { } // Now sync() and re-check for the applied updates - if err := ctx.ServiceClient.sync(); err != nil { + if err := ctx.syncOnce(); err != nil { t.Fatalf("unexpected error syncing task: %v", err) } if n := len(ctx.FakeConsul.services); n != 2 { @@ -256,7 +313,7 @@ func TestConsul_RegServices(t *testing.T) { // Remove the new task ctx.ServiceClient.RemoveTask("allocid", ctx.Task) - if err := ctx.ServiceClient.sync(); err != nil { + if err := ctx.syncOnce(); err != nil { t.Fatalf("unexpected error syncing task: %v", err) } if n := len(ctx.FakeConsul.services); n != 1 { @@ -287,11 +344,7 @@ func TestConsul_ShutdownOK(t *testing.T) { }, } - hasShutdown := make(chan struct{}) - go func() { - ctx.ServiceClient.Run() - close(hasShutdown) - }() + go ctx.ServiceClient.Run() // Register a task and agent if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { @@ -309,26 +362,11 @@ func TestConsul_ShutdownOK(t *testing.T) { t.Fatalf("unexpected error registering agent: %v", err) } - // Shutdown should block until all enqueued operations finish. + // Shutdown should block until scripts finish if err := ctx.ServiceClient.Shutdown(); err != nil { t.Errorf("unexpected error shutting down client: %v", err) } - // assert Run() exits in a timely fashion after Shutdown() exits - select { - case <-hasShutdown: - // ok! Run() exited as expected - case <-time.After(10 * time.Second): - t.Fatalf("expected Run() to exit, but it did not") - } - - // Nothing should be enqueued anymore - enqueued := (len(ctx.ServiceClient.pending.regServices) + len(ctx.ServiceClient.pending.deregServices) + - len(ctx.ServiceClient.pending.regChecks) + len(ctx.ServiceClient.pending.deregChecks)) - if enqueued > 0 { - t.Errorf("%d operations still enqueued", enqueued) - } - // UpdateTTL should have been called once for the script check if n := len(ctx.FakeConsul.checkTTLs); n != 1 { t.Fatalf("expected 1 checkTTL entry but found: %d", n) @@ -365,7 +403,13 @@ func TestConsul_ShutdownSlow(t *testing.T) { } // Make Exec slow, but not too slow + waiter := make(chan struct{}) ctx.ExecFunc = func(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + select { + case <-waiter: + default: + close(waiter) + } time.Sleep(time.Second) return []byte{}, 0, nil } @@ -373,31 +417,22 @@ func TestConsul_ShutdownSlow(t *testing.T) { // Make shutdown wait time just a bit longer than ctx.Exec takes ctx.ServiceClient.shutdownWait = 3 * time.Second - hasShutdown := make(chan struct{}) - go func() { - ctx.ServiceClient.Run() - close(hasShutdown) - }() + go ctx.ServiceClient.Run() // Register a task and agent if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { t.Fatalf("unexpected error registering task: %v", err) } + // wait for Exec to get called before shutting down + <-waiter + // Shutdown should block until all enqueued operations finish. preShutdown := time.Now() if err := ctx.ServiceClient.Shutdown(); err != nil { t.Errorf("unexpected error shutting down client: %v", err) } - // assert Run() exits in a timely fashion after Shutdown() exits - select { - case <-hasShutdown: - // ok! Run() exited as expected - case <-time.After(10 * time.Second): - t.Fatalf("expected Run() to exit, but it did not") - } - // Shutdown time should have taken: 1s <= shutdown <= 3s shutdownTime := time.Now().Sub(preShutdown) if shutdownTime < time.Second || shutdownTime > ctx.ServiceClient.shutdownWait { @@ -442,8 +477,10 @@ func TestConsul_ShutdownBlocked(t *testing.T) { block := make(chan struct{}) defer close(block) // cleanup after test - // Make Exec slow, but not too slow + // Make Exec block forever + waiter := make(chan struct{}) ctx.ExecFunc = func(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + close(waiter) <-block return []byte{}, 0, nil } @@ -451,17 +488,16 @@ func TestConsul_ShutdownBlocked(t *testing.T) { // Use a short shutdown deadline since we're intentionally blocking forever ctx.ServiceClient.shutdownWait = time.Second - hasShutdown := make(chan struct{}) - go func() { - ctx.ServiceClient.Run() - close(hasShutdown) - }() + go ctx.ServiceClient.Run() // Register a task and agent if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { t.Fatalf("unexpected error registering task: %v", err) } + // Wait for exec to be called + <-waiter + // Shutdown should block until all enqueued operations finish. preShutdown := time.Now() err := ctx.ServiceClient.Shutdown() @@ -469,18 +505,10 @@ func TestConsul_ShutdownBlocked(t *testing.T) { t.Errorf("expected a timed out error from shutdown") } - // assert Run() exits in a timely fashion after Shutdown() exits - maxWait := 10 * time.Second - select { - case <-hasShutdown: - // ok! Run() exited as expected - case <-time.After(maxWait): - t.Fatalf("expected Run() to exit, but it did not") - } - - // Shutdown time should have taken 1s; to avoid timing related errors - // simply test for 1s <= shutdown <= 10s + // Shutdown time should have taken shutdownWait; to avoid timing + // related errors simply test for wait <= shutdown <= wait+3s shutdownTime := time.Now().Sub(preShutdown) + maxWait := ctx.ServiceClient.shutdownWait + (3 * time.Second) if shutdownTime < ctx.ServiceClient.shutdownWait || shutdownTime > maxWait { t.Errorf("expected shutdown to take >%s and <%s but took: %s", ctx.ServiceClient.shutdownWait, maxWait, shutdownTime) } From b4e40efa1546559528b8e4a3a9444419949df3ce Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 12 Apr 2017 12:07:10 -0700 Subject: [PATCH 06/38] Fix shutdown when consul is down --- command/agent/consul/client.go | 44 ++++++++++++++++------------------ 1 file changed, 20 insertions(+), 24 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 93f3721f598f..6a5bbe49b06b 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -12,7 +12,6 @@ import ( "time" "github.com/hashicorp/consul/api" - "github.com/hashicorp/go-multierror" "github.com/hashicorp/nomad/nomad/structs" ) @@ -159,11 +158,14 @@ func (c *ServiceClient) Run() { if err := c.sync(); err != nil { if failures == 0 { - c.logger.Printf("[WARN] consul: failed to update services in Consul: %v", err) + c.logger.Printf("[WARN] consul.sync: failed to update services in Consul: %v", err) } failures++ if !retryTimer.Stop() { - <-retryTimer.C + select { + case <-retryTimer.C: + default: + } } backoff := c.retryInterval * time.Duration(failures) if backoff > c.maxRetryInterval { @@ -172,7 +174,7 @@ func (c *ServiceClient) Run() { retryTimer.Reset(backoff) } else { if failures > 0 { - c.logger.Printf("[INFO] consul: successfully updated services in Consul") + c.logger.Printf("[INFO] consul.sync: successfully updated services in Consul") failures = 0 } } @@ -561,37 +563,32 @@ func (c *ServiceClient) Shutdown() error { case <-c.shutdownCh: return nil default: - close(c.shutdownCh) } - var mErr multierror.Error - - // Don't let Shutdown block indefinitely - deadline := time.After(c.shutdownWait) - - // Deregister agent services and checks + // First deregister Nomad agent Consul entries + ops := operations{} c.agentLock.Lock() for id := range c.agentServices { - if err := c.client.ServiceDeregister(id); err != nil { - mErr.Errors = append(mErr.Errors, err) - } + ops.deregServices = append(ops.deregServices, id) } - - // Deregister Checks for id := range c.agentChecks { - if err := c.client.CheckDeregister(id); err != nil { - mErr.Errors = append(mErr.Errors, err) - } + ops.deregChecks = append(ops.deregChecks, id) } c.agentLock.Unlock() + c.commit(&ops) + + // Then signal shutdown + close(c.shutdownCh) + + // Give run loop time to sync, but don't block indefinitely + deadline := time.After(c.shutdownWait) // Wait for Run to finish any outstanding operations and exit select { case <-c.exitCh: case <-deadline: // Don't wait forever though - mErr.Errors = append(mErr.Errors, fmt.Errorf("timed out waiting for Consul operations to complete")) - return mErr.ErrorOrNil() + return fmt.Errorf("timed out waiting for Consul operations to complete") } // Give script checks time to exit (no need to lock as Run() has exited) @@ -599,11 +596,10 @@ func (c *ServiceClient) Shutdown() error { select { case <-h.wait(): case <-deadline: - mErr.Errors = append(mErr.Errors, fmt.Errorf("timed out waiting for script checks to run")) - return mErr.ErrorOrNil() + return fmt.Errorf("timed out waiting for script checks to run") } } - return mErr.ErrorOrNil() + return nil } // makeAgentServiceID creates a unique ID for identifying an agent service in From 63a83072559a4f92e21918d2b49e074c205ddfcb Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 12 Apr 2017 13:26:55 -0700 Subject: [PATCH 07/38] Move ScriptExecutor to driver --- client/consul.go | 6 +++--- client/driver/driver.go | 7 +++++++ client/task_runner.go | 7 +++---- command/agent/consul/client.go | 16 +++++----------- command/agent/consul/script.go | 7 +++++-- 5 files changed, 23 insertions(+), 20 deletions(-) diff --git a/client/consul.go b/client/consul.go index c9fe4e107803..043a17bdb16f 100644 --- a/client/consul.go +++ b/client/consul.go @@ -1,14 +1,14 @@ package client import ( - "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/client/driver" "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 { - RegisterTask(allocID string, task *structs.Task, exec consul.ScriptExecutor) error + RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor) error RemoveTask(allocID string, task *structs.Task) - UpdateTask(allocID string, existing, newTask *structs.Task, exec consul.ScriptExecutor) error + UpdateTask(allocID string, existing, newTask *structs.Task, exec driver.ScriptExecutor) error } diff --git a/client/driver/driver.go b/client/driver/driver.go index a89db81679f6..74c8980311c7 100644 --- a/client/driver/driver.go +++ b/client/driver/driver.go @@ -1,6 +1,7 @@ package driver import ( + "context" "errors" "fmt" "log" @@ -256,6 +257,12 @@ type DriverHandle interface { Signal(s os.Signal) error } +// ScriptExecutor is a DriverHandle that supports Exec()ing commands in the +// driver's context. +type ScriptExecutor interface { + Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) +} + // ExecContext is a task's execution context type ExecContext struct { // TaskDir contains information about the task directory structure. diff --git a/client/task_runner.go b/client/task_runner.go index 209336d700cc..e4e4c15e868e 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -21,7 +21,6 @@ import ( "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/client/getter" "github.com/hashicorp/nomad/client/vaultclient" - "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/client/driver/env" @@ -298,7 +297,7 @@ func (r *TaskRunner) RestoreState() error { interpolateServices(r.getTaskEnv(), r.task) // Ensure the service is registered - scriptExec, _ := handle.(consul.ScriptExecutor) + scriptExec, _ := handle.(driver.ScriptExecutor) if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { //FIXME What to do if this fails? r.logger.Printf("[WARN] client: failed to register services and checks for task %q alloc %q: %v", @@ -1243,7 +1242,7 @@ func (r *TaskRunner) startTask() error { // RegisterTask properly handles scriptExec being nil, so it just // ignore the ok value. - scriptExec, _ := handle.(consul.ScriptExecutor) + scriptExec, _ := handle.(driver.ScriptExecutor) if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { //FIXME handle errors?! //FIXME could break into prepare & submit steps as only preperation can error... @@ -1399,7 +1398,7 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { // Not all drivers support Exec (eg QEMU), but RegisterTask // handles nil ScriptExecutors - scriptExec, _ := r.handle.(consul.ScriptExecutor) + scriptExec, _ := r.handle.(driver.ScriptExecutor) // Since the handle exists, the task is running, so we need to // update it in Consul (if the handle doesn't exist diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 6a5bbe49b06b..449184e2deb1 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -1,7 +1,6 @@ package consul import ( - "context" "fmt" "log" "net" @@ -12,6 +11,7 @@ import ( "time" "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/nomad/structs" ) @@ -51,12 +51,6 @@ const ( ServiceTagSerf = "serf" ) -// ScriptExecutor is the interface the ServiceClient uses to execute script -// checks inside a container. -type ScriptExecutor interface { - Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) -} - // CatalogAPI is the consul/api.Catalog API used by Nomad. type CatalogAPI interface { Datacenters() ([]string, error) @@ -386,7 +380,7 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) // makeCheckReg adds a check reg to operations. func (c *ServiceClient) makeCheckReg(ops *operations, check *structs.ServiceCheck, - service *api.AgentServiceRegistration, exec ScriptExecutor, parseAddr addrParser) error { + service *api.AgentServiceRegistration, exec driver.ScriptExecutor, parseAddr addrParser) error { checkID := createCheckID(service.ID, check) if check.Type == structs.ServiceCheckScript { @@ -412,7 +406,7 @@ func (c *ServiceClient) makeCheckReg(ops *operations, check *structs.ServiceChec // serviceRegs creates service registrations, check registrations, and script // checks from a service. func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *structs.Service, - exec ScriptExecutor, task *structs.Task) error { + exec driver.ScriptExecutor, task *structs.Task) error { id := makeTaskServiceID(allocID, task.Name, service) host, port := task.FindHostAndPortFor(service.PortLabel) @@ -441,7 +435,7 @@ func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *st // exec is nil and a script check exists an error is returned. // // Actual communication with Consul is done asynchrously (see Run). -func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec ScriptExecutor) error { +func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor) error { ops := &operations{} for _, service := range task.Services { if err := c.serviceRegs(ops, allocID, service, exec, task); err != nil { @@ -454,7 +448,7 @@ func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec Sc // UpdateTask in Consul. Does not alter the service if only checks have // changed. -func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Task, exec ScriptExecutor) error { +func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Task, exec driver.ScriptExecutor) error { ops := &operations{} existingIDs := make(map[string]*structs.Service, len(existing.Services)) diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 96c52e741069..608de8db1ca4 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -6,6 +6,7 @@ import ( "time" "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/nomad/structs" ) @@ -29,7 +30,7 @@ func (s *scriptHandle) wait() <-chan struct{} { type scriptCheck struct { id string check *structs.ServiceCheck - exec ScriptExecutor + exec driver.ScriptExecutor agent heartbeater running bool @@ -40,7 +41,9 @@ type scriptCheck struct { shutdownCh <-chan struct{} } -func newScriptCheck(id string, check *structs.ServiceCheck, exec ScriptExecutor, agent heartbeater, logger *log.Logger, shutdownCh <-chan struct{}) *scriptCheck { +func newScriptCheck(id string, check *structs.ServiceCheck, exec driver.ScriptExecutor, agent heartbeater, + logger *log.Logger, shutdownCh <-chan struct{}) *scriptCheck { + return &scriptCheck{ id: id, check: check, From a3fc76b63a31fbd911544dd3a02d085e6b051807 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 12 Apr 2017 13:27:56 -0700 Subject: [PATCH 08/38] Fix comment to reflect reality --- command/agent/consul/client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 449184e2deb1..be1d0a819c98 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -640,8 +640,8 @@ func createCheckID(serviceID string, check *structs.ServiceCheck) string { // createCheckReg creates a Check that can be registered with Consul. // -// Only supports HTTP(S) and TCP checks. Script checks must be handled -// externally. +// Script checks simply have a TTL set and the caller is responsible for +// running the script and heartbeating. func createCheckReg(serviceID, checkID string, check *structs.ServiceCheck, host string, port int) (*api.AgentCheckRegistration, error) { chkReg := api.AgentCheckRegistration{ ID: checkID, From fd69d48244fe4d67939418e63b6beaadbc161550 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 12 Apr 2017 13:33:33 -0700 Subject: [PATCH 09/38] Move removal from Consul into TaskRunner cleanup --- client/task_runner.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/client/task_runner.go b/client/task_runner.go index e4e4c15e868e..c4baef9f958f 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -1076,8 +1076,12 @@ func (r *TaskRunner) run() { } } -// cleanup calls Driver.Cleanup when a task is stopping. Errors are logged. +// cleanup removes Consul entries and calls Driver.Cleanup when a task is +// stopping. Errors are logged. func (r *TaskRunner) cleanup() { + // Remove from Consul + r.consul.RemoveTask(r.alloc.ID, r.task) + drv, err := r.createDriver() if err != nil { r.logger.Printf("[ERR] client: error creating driver to cleanup resources: %v", err) @@ -1425,9 +1429,6 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { // given limit. It returns whether the task was destroyed and the error // associated with the last kill attempt. func (r *TaskRunner) handleDestroy() (destroyed bool, err error) { - // Remove from Consul - r.consul.RemoveTask(r.alloc.ID, r.task) - // Cap the number of times we attempt to kill the task. for i := 0; i < killFailureLimit; i++ { if err = r.handle.Kill(); err != nil { From b4c7d92936064788e6340223c5dcf8cdc1d16bd3 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 12 Apr 2017 13:35:14 -0700 Subject: [PATCH 10/38] Explain cleanup defer in test --- command/agent/consul/int_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go index 2f0239d5f780..122414243fd6 100644 --- a/command/agent/consul/int_test.go +++ b/command/agent/consul/int_test.go @@ -133,7 +133,7 @@ func TestConsul_Integration(t *testing.T) { tr.MarkReceived() go tr.Run() defer func() { - // Just in case cleanup + // Make sure we always shutdown task runner when the test exits select { case <-tr.WaitCh(): // Exited cleanly, no need to kill From 4002d87a6bd5dd383a8a32c665c066fe29fcadcd Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 12 Apr 2017 13:47:38 -0700 Subject: [PATCH 11/38] Add comments, clarify names, fix PR comments --- command/agent/consul/script.go | 38 ++++++++++++++++++++-------------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 608de8db1ca4..4d0a66b011d9 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -16,23 +16,26 @@ type heartbeater interface { UpdateTTL(id, output, status string) error } +// scriptHandle is returned by scriptCheck.run by cancelling a scriptCheck and +// waiting for it to shutdown. type scriptHandle struct { // cancel the script cancel func() - done chan struct{} + exitCh chan struct{} } // wait returns a chan that's closed when the script exits func (s *scriptHandle) wait() <-chan struct{} { - return s.done + return s.exitCh } +// scriptCheck runs script checks via a ScriptExecutor and updates the +// appropriate check's TTL when the script succeeds. type scriptCheck struct { - id string - check *structs.ServiceCheck - exec driver.ScriptExecutor - agent heartbeater - running bool + id string + check *structs.ServiceCheck + exec driver.ScriptExecutor + agent heartbeater // lastCheckOk is true if the last check was ok; otherwise false lastCheckOk bool @@ -41,6 +44,8 @@ type scriptCheck struct { shutdownCh <-chan struct{} } +// newScriptCheck creates a new scriptCheck. run() should be called once the +// initial check is registered with Consul. func newScriptCheck(id string, check *structs.ServiceCheck, exec driver.ScriptExecutor, agent heartbeater, logger *log.Logger, shutdownCh <-chan struct{}) *scriptCheck { @@ -59,9 +64,9 @@ func newScriptCheck(id string, check *structs.ServiceCheck, exec driver.ScriptEx // closed the check will be run once more before exiting. func (s *scriptCheck) run() *scriptHandle { ctx, cancel := context.WithCancel(context.Background()) - done := make(chan struct{}) + exitCh := make(chan struct{}) go func() { - defer close(done) + defer close(exitCh) timer := time.NewTimer(0) defer timer.Stop() for { @@ -89,6 +94,7 @@ func (s *scriptCheck) run() *scriptHandle { s.lastCheckOk = false s.logger.Printf("[WARN] consul.checks: check %q timed out (%s)", s.check.Name, s.check.Timeout) } + // cleanup context cancel() @@ -99,13 +105,17 @@ func (s *scriptCheck) run() *scriptHandle { case 1: state = api.HealthWarning } + + var outputMsg string if err != nil { state = api.HealthCritical - output = []byte(err.Error()) + outputMsg = err.Error() + } else { + outputMsg = string(output) } // Actually heartbeat the check - err = s.agent.UpdateTTL(s.id, string(output), state) + err = s.agent.UpdateTTL(s.id, outputMsg, state) select { case <-ctx.Done(): // check has been removed; don't report errors @@ -114,7 +124,6 @@ func (s *scriptCheck) run() *scriptHandle { } if err != nil { - //FIXME Backoff? Retry faster? if s.lastCheckOk { s.lastCheckOk = false s.logger.Printf("[WARN] consul.checks: update for check %q failed: %v", s.check.Name, err) @@ -130,12 +139,11 @@ func (s *scriptCheck) run() *scriptHandle { select { case <-s.shutdownCh: - // We've been told to exit + // We've been told to exit and just heartbeated so exit return default: } } }() - s.running = true - return &scriptHandle{cancel: cancel, done: done} + return &scriptHandle{cancel: cancel, exitCh: exitCh} } From 0e0845e94b99054fb725c44c7a9fd8bdcf1cac40 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 13 Apr 2017 09:52:16 -0700 Subject: [PATCH 12/38] Use a DriverAbility to expose Exec functionality --- client/driver/docker.go | 1 + client/driver/driver.go | 12 ++++- client/driver/exec.go | 1 + client/driver/java.go | 1 + client/driver/lxc.go | 6 +++ client/driver/mock_driver.go | 1 + client/driver/qemu.go | 6 +++ client/driver/raw_exec.go | 1 + client/driver/rkt.go | 1 + client/task_runner.go | 91 ++++++++++++++++++++++-------------- 10 files changed, 83 insertions(+), 38 deletions(-) diff --git a/client/driver/docker.go b/client/driver/docker.go index 171c1ed6df64..bea4a3501342 100644 --- a/client/driver/docker.go +++ b/client/driver/docker.go @@ -429,6 +429,7 @@ func (d *DockerDriver) Validate(config map[string]interface{}) error { func (d *DockerDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } diff --git a/client/driver/driver.go b/client/driver/driver.go index 74c8980311c7..ee28888cfad0 100644 --- a/client/driver/driver.go +++ b/client/driver/driver.go @@ -192,6 +192,10 @@ type Driver interface { type DriverAbilities struct { // SendSignals marks the driver as being able to send signals SendSignals bool + + // Exec marks the driver as being able to execute arbitrary commands + // such as health checks. Used by the ScriptExecutor interface. + Exec bool } // LogEventFn is a callback which allows Drivers to emit task events. @@ -255,10 +259,14 @@ type DriverHandle interface { // Signal is used to send a signal to the task Signal(s os.Signal) error + + // ScriptExecutor is an interface used to execute commands such as + // health check scripts in the a DriverHandle's context. + ScriptExecutor } -// ScriptExecutor is a DriverHandle that supports Exec()ing commands in the -// driver's context. +// ScriptExecutor is an interface that supports Exec()ing commands in the +// driver's context. Split out of DriverHandle to ease testing. type ScriptExecutor interface { Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) } diff --git a/client/driver/exec.go b/client/driver/exec.go index 7da657e1ed3e..bc6ee3aee258 100644 --- a/client/driver/exec.go +++ b/client/driver/exec.go @@ -86,6 +86,7 @@ func (d *ExecDriver) Validate(config map[string]interface{}) error { func (d *ExecDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } diff --git a/client/driver/java.go b/client/driver/java.go index c684e85bf562..c215e6882d52 100644 --- a/client/driver/java.go +++ b/client/driver/java.go @@ -108,6 +108,7 @@ func (d *JavaDriver) Validate(config map[string]interface{}) error { func (d *JavaDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } diff --git a/client/driver/lxc.go b/client/driver/lxc.go index 0d369d34cd0a..ac7d3c298a33 100644 --- a/client/driver/lxc.go +++ b/client/driver/lxc.go @@ -3,6 +3,7 @@ package driver import ( + "context" "encoding/json" "fmt" "log" @@ -149,6 +150,7 @@ func (d *LxcDriver) Validate(config map[string]interface{}) error { func (d *LxcDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: false, + Exec: false, } } @@ -375,6 +377,10 @@ func (h *lxcDriverHandle) Update(task *structs.Task) error { return nil } +func (h *lxcDriverHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + return nil, 0, fmt.Errorf("lxc driver cannot execute commands") +} + func (h *lxcDriverHandle) Kill() error { h.logger.Printf("[INFO] driver.lxc: shutting down container %q", h.container.Name()) if err := h.container.Shutdown(h.killTimeout); err != nil { diff --git a/client/driver/mock_driver.go b/client/driver/mock_driver.go index 518e79cea171..bdaff3f703c4 100644 --- a/client/driver/mock_driver.go +++ b/client/driver/mock_driver.go @@ -76,6 +76,7 @@ func NewMockDriver(ctx *DriverContext) Driver { func (d *MockDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: false, + Exec: true, } } diff --git a/client/driver/qemu.go b/client/driver/qemu.go index 4e04f95229b3..856b2c023166 100644 --- a/client/driver/qemu.go +++ b/client/driver/qemu.go @@ -1,6 +1,7 @@ package driver import ( + "context" "encoding/json" "fmt" "log" @@ -97,6 +98,7 @@ func (d *QemuDriver) Validate(config map[string]interface{}) error { func (d *QemuDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: false, + Exec: false, } } @@ -353,6 +355,10 @@ func (h *qemuHandle) Update(task *structs.Task) error { return nil } +func (h *qemuHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + return nil, 0, fmt.Errorf("Qemu driver can't execute commands") +} + func (h *qemuHandle) Signal(s os.Signal) error { return fmt.Errorf("Qemu driver can't send signals") } diff --git a/client/driver/raw_exec.go b/client/driver/raw_exec.go index e0e86c20bf64..5e602f47bb1e 100644 --- a/client/driver/raw_exec.go +++ b/client/driver/raw_exec.go @@ -80,6 +80,7 @@ func (d *RawExecDriver) Validate(config map[string]interface{}) error { func (d *RawExecDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } diff --git a/client/driver/rkt.go b/client/driver/rkt.go index d1e039dd60cf..a8cff3c02490 100644 --- a/client/driver/rkt.go +++ b/client/driver/rkt.go @@ -165,6 +165,7 @@ func (d *RktDriver) Validate(config map[string]interface{}) error { func (d *RktDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: false, + Exec: true, } } diff --git a/client/task_runner.go b/client/task_runner.go index c4baef9f958f..88ce02017ef6 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -292,15 +292,11 @@ func (r *TaskRunner) RestoreState() error { return nil } - //FIXME is there a better place to do this? used to be in executor - // Prepare services - interpolateServices(r.getTaskEnv(), r.task) - - // Ensure the service is registered - scriptExec, _ := handle.(driver.ScriptExecutor) - if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { - //FIXME What to do if this fails? - r.logger.Printf("[WARN] client: failed to register services and checks for task %q alloc %q: %v", + if err := r.registerServices(d, handle); err != nil { + // Don't hard fail here as there's a chance this task + // registered with Consul properly when it initial + // started. + r.logger.Printf("[WARN] client: failed to register services and checks with consul for task %q in alloc %q: %v", r.task.Name, r.alloc.ID, err) } @@ -1186,8 +1182,12 @@ func (r *TaskRunner) killTask(killingEvent *structs.TaskEvent) { // Mark that we received the kill event r.setState(structs.TaskStateRunning, event) + r.handleLock.Lock() + handle := r.handle + r.handleLock.Unlock() + // Kill the task using an exponential backoff in-case of failures. - destroySuccess, err := r.handleDestroy() + destroySuccess, err := r.handleDestroy(handle) if !destroySuccess { // We couldn't successfully destroy the resource created. r.logger.Printf("[ERR] client: failed to kill task %q. Resources may have been leaked: %v", r.task.Name, err) @@ -1236,24 +1236,35 @@ func (r *TaskRunner) startTask() error { } + if err := r.registerServices(drv, handle); err != nil { + // All IO is done asynchronously, so errors from registering + // services are hard failures. + r.logger.Printf("[ERR] client: failed to register services and checks for task %q alloc %q: %v", r.task.Name, r.alloc.ID, err) + + // Kill the started task + if destroyed, err := r.handleDestroy(handle); !destroyed { + r.logger.Printf("[ERR] client: failed to kill task %q alloc %q. Resources may be leaked: %v", + r.task.Name, r.alloc.ID, err) + } + return structs.NewRecoverableError(err, false) + } + r.handleLock.Lock() r.handle = handle r.handleLock.Unlock() - //FIXME is there a better place to do this? used to be in executor - // Prepare services - interpolateServices(r.getTaskEnv(), r.task) + return nil +} - // RegisterTask properly handles scriptExec being nil, so it just - // ignore the ok value. - scriptExec, _ := handle.(driver.ScriptExecutor) - if err := r.consul.RegisterTask(r.alloc.ID, r.task, scriptExec); err != nil { - //FIXME handle errors?! - //FIXME could break into prepare & submit steps as only preperation can error... - r.logger.Printf("[ERR] client: failed to register services and checks for task %q alloc %q: %v", r.task.Name, r.alloc.ID, err) +// registerServices and checks with Consul. +func (r *TaskRunner) registerServices(d driver.Driver, h driver.ScriptExecutor) error { + var exec driver.ScriptExecutor + if d.Abilities().Exec { + // Allow set the script executor if the driver supports it + exec = h } - - return nil + interpolateServices(r.getTaskEnv(), r.task) + return r.consul.RegisterTask(r.alloc.ID, r.task, exec) } // interpolateServices interpolates tags in a service and checks with values from the @@ -1391,23 +1402,20 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { var mErr multierror.Error r.handleLock.Lock() if r.handle != nil { + // Need to check driver abilities for updating services + drv, err := r.createDriver() + if err != nil { + // Something has really gone wrong; don't continue + r.handleLock.Unlock() + return fmt.Errorf("error accessing driver when updating task %q: %v", r.task.Name, err) + } + // Update will update resources and store the new kill timeout. if err := r.handle.Update(updatedTask); err != nil { mErr.Errors = append(mErr.Errors, fmt.Errorf("updating task resources failed: %v", err)) } - //FIXME is there a better place to do this? used to be in executor - // Prepare services - interpolateServices(r.getTaskEnv(), updatedTask) - - // Not all drivers support Exec (eg QEMU), but RegisterTask - // handles nil ScriptExecutors - scriptExec, _ := r.handle.(driver.ScriptExecutor) - - // Since the handle exists, the task is running, so we need to - // update it in Consul (if the handle doesn't exist - // registration in Consul will happen when it's created) - if err := r.consul.UpdateTask(r.alloc.ID, r.task, updatedTask, scriptExec); err != nil { + if err := r.updateServices(drv, r.handle, r.task, updatedTask); err != nil { mErr.Errors = append(mErr.Errors, fmt.Errorf("error updating services and checks in Consul: %v", err)) } } @@ -1424,14 +1432,25 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { return mErr.ErrorOrNil() } +// updateServices and checks with Consul. +func (r *TaskRunner) updateServices(d driver.Driver, h driver.ScriptExecutor, old, new *structs.Task) error { + var exec driver.ScriptExecutor + if d.Abilities().Exec { + // Allow set the script executor if the driver supports it + exec = h + } + interpolateServices(r.getTaskEnv(), r.task) + return r.consul.UpdateTask(r.alloc.ID, old, new, exec) +} + // handleDestroy kills the task handle. In the case that killing fails, // handleDestroy will retry with an exponential backoff and will give up at a // given limit. It returns whether the task was destroyed and the error // associated with the last kill attempt. -func (r *TaskRunner) handleDestroy() (destroyed bool, err error) { +func (r *TaskRunner) handleDestroy(handle driver.DriverHandle) (destroyed bool, err error) { // Cap the number of times we attempt to kill the task. for i := 0; i < killFailureLimit; i++ { - if err = r.handle.Kill(); err != nil { + if err = handle.Kill(); err != nil { // Calculate the new backoff backoff := (1 << (2 * uint64(i))) * killBackoffBaseline if backoff > killBackoffLimit { From 7ac9215fb787bb0f6ac4441c349fa502ddeb3e33 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 13 Apr 2017 13:47:05 -0700 Subject: [PATCH 13/38] Use nifty testtask sleep command for xplat compat --- command/agent/consul/int_test.go | 4 ++-- command/agent/consul/script_test.go | 10 +++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go index 122414243fd6..8a5635e179e6 100644 --- a/command/agent/consul/int_test.go +++ b/command/agent/consul/int_test.go @@ -145,7 +145,7 @@ func TestConsul_Integration(t *testing.T) { // Block waiting for the service to appear catalog := consulClient.Catalog() res, meta, err := catalog.Service("httpd2", "test", nil) - for len(res) == 0 { + for i := 0; len(res) == 0 && i < 10; i++ { //Expected initial request to fail, do a blocking query res, meta, err = catalog.Service("httpd2", "test", &consulapi.QueryOptions{WaitIndex: meta.LastIndex + 1, WaitTime: 3 * time.Second}) if err != nil { @@ -158,7 +158,7 @@ func TestConsul_Integration(t *testing.T) { res = res[:] // Assert the service with the checks exists - for len(res) == 0 { + for i := 0; len(res) == 0 && i < 10; i++ { res, meta, err = catalog.Service("httpd", "http", &consulapi.QueryOptions{WaitIndex: meta.LastIndex + 1, WaitTime: 3 * time.Second}) if err != nil { t.Fatalf("error querying for service: %v", err) diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index a9bda3e44499..896c516c3004 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -2,14 +2,22 @@ package consul import ( "context" + "os" "os/exec" "testing" "time" "github.com/hashicorp/consul/api" + "github.com/hashicorp/nomad/helper/testtask" "github.com/hashicorp/nomad/nomad/structs" ) +func TestMain(m *testing.M) { + if !testtask.Run() { + os.Exit(m.Run()) + } +} + // blockingScriptExec implements ScriptExec by running a subcommand that never // exits. type blockingScriptExec struct { @@ -26,7 +34,7 @@ func newBlockingScriptExec() *blockingScriptExec { func (b *blockingScriptExec) Exec(ctx context.Context, _ string, _ []string) ([]byte, int, error) { b.running <- mark - cmd := exec.CommandContext(ctx, "/bin/sleep", "9000") + cmd := exec.CommandContext(ctx, testtask.Path(), "sleep", "9000h") err := cmd.Run() code := 0 if exitErr, ok := err.(*exec.ExitError); ok { From 80617a9c1adb7615fd56c0f37635c8a88a5beb5c Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 13 Apr 2017 13:49:23 -0700 Subject: [PATCH 14/38] Stop being lazy and just type out struct{}{} --- command/agent/consul/client.go | 8 +++----- command/agent/consul/script_test.go | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index be1d0a819c98..d6c125a5b54a 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -15,8 +15,6 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -var mark = struct{}{} - const ( // nomadServicePrefix is the first prefix that scopes all Nomad registered // services @@ -369,10 +367,10 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) // Record IDs for deregistering on shutdown c.agentLock.Lock() for _, id := range ops.regServices { - c.agentServices[id.ID] = mark + c.agentServices[id.ID] = struct{}{} } for _, id := range ops.regChecks { - c.agentChecks[id.ID] = mark + c.agentChecks[id.ID] = struct{}{} } c.agentLock.Unlock() return nil @@ -481,7 +479,7 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta // Check to see what checks were updated existingChecks := make(map[string]struct{}, len(existingSvc.Checks)) for _, check := range existingSvc.Checks { - existingChecks[createCheckID(existingID, check)] = mark + existingChecks[createCheckID(existingID, check)] = struct{}{} } // Register new checks diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index 896c516c3004..fdb2e241d484 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -33,7 +33,7 @@ func newBlockingScriptExec() *blockingScriptExec { } func (b *blockingScriptExec) Exec(ctx context.Context, _ string, _ []string) ([]byte, int, error) { - b.running <- mark + b.running <- struct{}{} cmd := exec.CommandContext(ctx, testtask.Path(), "sleep", "9000h") err := cmd.Run() code := 0 From 48269c36b02d1054941d17bfcfd184f8ef7e2188 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 13 Apr 2017 16:43:38 -0700 Subject: [PATCH 15/38] Plumb alloc id + task name into script check logs --- command/agent/consul/client.go | 45 +++++++++++------------------ command/agent/consul/script.go | 24 ++++++++++----- command/agent/consul/script_test.go | 6 ++-- 3 files changed, 37 insertions(+), 38 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index d6c125a5b54a..3e73b75caf10 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -376,31 +376,6 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) return nil } -// makeCheckReg adds a check reg to operations. -func (c *ServiceClient) makeCheckReg(ops *operations, check *structs.ServiceCheck, - service *api.AgentServiceRegistration, exec driver.ScriptExecutor, parseAddr addrParser) error { - - checkID := createCheckID(service.ID, check) - if check.Type == structs.ServiceCheckScript { - if exec == nil { - return fmt.Errorf("driver doesn't support script checks") - } - ops.scripts = append(ops.scripts, newScriptCheck( - checkID, check, exec, c.client, c.logger, c.shutdownCh)) - - } - host, port := service.Address, service.Port - if check.PortLabel != "" { - host, port = parseAddr(check.PortLabel) - } - checkReg, err := createCheckReg(service.ID, checkID, check, host, port) - if err != nil { - return fmt.Errorf("failed to add check %q: %v", check.Name, err) - } - ops.regChecks = append(ops.regChecks, checkReg) - return nil -} - // serviceRegs creates service registrations, check registrations, and script // checks from a service. func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *structs.Service, @@ -421,10 +396,24 @@ func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *st ops.regServices = append(ops.regServices, serviceReg) for _, check := range service.Checks { - err := c.makeCheckReg(ops, check, serviceReg, exec, task.FindHostAndPortFor) + checkID := createCheckID(id, check) + if check.Type == structs.ServiceCheckScript { + if exec == nil { + return fmt.Errorf("driver doesn't support script checks") + } + ops.scripts = append(ops.scripts, newScriptCheck( + allocID, task.Name, checkID, check, exec, c.client, c.logger, c.shutdownCh)) + + } + host, port := serviceReg.Address, serviceReg.Port + if check.PortLabel != "" { + host, port = task.FindHostAndPortFor(check.PortLabel) + } + checkReg, err := createCheckReg(id, checkID, check, host, port) if err != nil { - return err + return fmt.Errorf("failed to add check %q: %v", check.Name, err) } + ops.regChecks = append(ops.regChecks, checkReg) } return nil } @@ -497,7 +486,7 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta return fmt.Errorf("driver doesn't support script checks") } ops.scripts = append(ops.scripts, newScriptCheck( - checkID, check, exec, c.client, c.logger, c.shutdownCh)) + existingID, newTask.Name, checkID, check, exec, c.client, c.logger, c.shutdownCh)) } host, port := parseAddr(existingSvc.PortLabel) if check.PortLabel != "" { diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 4d0a66b011d9..3e96df00d421 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -32,6 +32,9 @@ func (s *scriptHandle) wait() <-chan struct{} { // scriptCheck runs script checks via a ScriptExecutor and updates the // appropriate check's TTL when the script succeeds. type scriptCheck struct { + allocID string + taskName string + id string check *structs.ServiceCheck exec driver.ScriptExecutor @@ -46,11 +49,14 @@ type scriptCheck struct { // newScriptCheck creates a new scriptCheck. run() should be called once the // initial check is registered with Consul. -func newScriptCheck(id string, check *structs.ServiceCheck, exec driver.ScriptExecutor, agent heartbeater, - logger *log.Logger, shutdownCh <-chan struct{}) *scriptCheck { +func newScriptCheck(allocID, taskName, checkID string, check *structs.ServiceCheck, + exec driver.ScriptExecutor, agent heartbeater, logger *log.Logger, + shutdownCh <-chan struct{}) *scriptCheck { return &scriptCheck{ - id: id, + allocID: allocID, + taskName: taskName, + id: checkID, check: check, exec: exec, agent: agent, @@ -92,7 +98,8 @@ func (s *scriptCheck) run() *scriptHandle { case context.DeadlineExceeded: // Log deadline exceeded every time, but flip last check to false s.lastCheckOk = false - s.logger.Printf("[WARN] consul.checks: check %q timed out (%s)", s.check.Name, s.check.Timeout) + s.logger.Printf("[WARN] consul.checks: check %q for task %q alloc %q timed out (%s)", + s.check.Name, s.taskName, s.allocID, s.check.Timeout) } // cleanup context @@ -126,15 +133,18 @@ func (s *scriptCheck) run() *scriptHandle { if err != nil { if s.lastCheckOk { s.lastCheckOk = false - s.logger.Printf("[WARN] consul.checks: update for check %q failed: %v", s.check.Name, err) + s.logger.Printf("[WARN] consul.checks: update for task %q alloc %q check %q failed: %v", + s.taskName, s.allocID, s.check.Name, err) } else { - s.logger.Printf("[DEBUG] consul.checks: update for check %q still failing: %v", s.check.Name, err) + s.logger.Printf("[DEBUG] consul.checks: update for task %q alloc %q check %q still failing: %v", + s.taskName, s.allocID, s.check.Name, err) } } else if !s.lastCheckOk { // Succeeded for the first time or after failing; log s.lastCheckOk = true - s.logger.Printf("[INFO] consul.checks: update for check %q succeeded", s.check.Name) + s.logger.Printf("[INFO] consul.checks: update for task %q alloc %q check %q succeeded", + s.taskName, s.allocID, s.check.Name) } select { diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index fdb2e241d484..3d713f0c1aff 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -57,7 +57,7 @@ func TestConsulScript_Exec_Cancel(t *testing.T) { exec := newBlockingScriptExec() // pass nil for heartbeater as it shouldn't be called - check := newScriptCheck("checkid", &serviceCheck, exec, nil, testLogger(), nil) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, nil, testLogger(), nil) handle := check.run() // wait until Exec is called @@ -101,7 +101,7 @@ func TestConsulScript_Exec_Timeout(t *testing.T) { exec := newBlockingScriptExec() hb := newFakeHeartbeater() - check := newScriptCheck("checkid", &serviceCheck, exec, hb, testLogger(), nil) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, hb, testLogger(), nil) handle := check.run() defer handle.cancel() // just-in-case cleanup <-exec.running @@ -148,7 +148,7 @@ func TestConsulScript_Exec_Shutdown(t *testing.T) { hb := newFakeHeartbeater() shutdown := make(chan struct{}) - check := newScriptCheck("checkid", &serviceCheck, noopExec{}, hb, testLogger(), shutdown) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, noopExec{}, hb, testLogger(), shutdown) handle := check.run() defer handle.cancel() // just-in-case cleanup From 5d75efc397936f0556614d36e923eb9816157f0f Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 13 Apr 2017 16:59:27 -0700 Subject: [PATCH 16/38] Explain PortLabel handling in RegisterAgent --- command/agent/consul/client.go | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 3e73b75caf10..c37e6132298e 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -307,8 +307,10 @@ func (c *ServiceClient) sync() error { return nil } -// RegisterAgent registers Nomad agents (client or server). Script checks are -// not supported and will return an error. Registration is asynchronous. +// RegisterAgent registers Nomad agents (client or server). The +// Service.PortLabel should be a literal port to be parsed with SplitHostPort. +// Script checks are not supported and will return an error. Registration is +// asynchronous. // // Agents will be deregistered when Shutdown is called. func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) error { @@ -316,6 +318,9 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) for _, service := range services { id := makeAgentServiceID(role, service) + + // Unlike tasks, agents don't use port labels. Agent ports are + // stored directly in the PortLabel. host, rawport, err := net.SplitHostPort(service.PortLabel) if err != nil { return fmt.Errorf("error parsing port label %q from service %q: %v", service.PortLabel, service.Name, err) @@ -340,6 +345,8 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) } checkHost, checkPort := serviceReg.Address, serviceReg.Port if check.PortLabel != "" { + // Unlike tasks, agents don't use port labels. Agent ports are + // stored directly in the PortLabel. host, rawport, err := net.SplitHostPort(check.PortLabel) if err != nil { return fmt.Errorf("error parsing port label %q from check %q: %v", service.PortLabel, check.Name, err) From 6bb7d8b7366ff6a5c16686c5fcf3379591a274fd Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Thu, 13 Apr 2017 17:03:07 -0700 Subject: [PATCH 17/38] Backoff on Consul lookup failures --- nomad/server.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nomad/server.go b/nomad/server.go index dade6cc86356..14355a871ce7 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -66,11 +66,11 @@ const ( // defaultConsulDiscoveryInterval is how often to poll Consul for new // servers if there is no leader. - defaultConsulDiscoveryInterval time.Duration = 9 * time.Second + defaultConsulDiscoveryInterval time.Duration = 3 * time.Second // defaultConsulDiscoveryIntervalRetry is how often to poll Consul for // new servers if there is no leader and the last Consul query failed. - defaultConsulDiscoveryIntervalRetry time.Duration = 3 * time.Second + defaultConsulDiscoveryIntervalRetry time.Duration = 9 * time.Second ) // Server is Nomad server which manages the job queues, From db8aabe1c3067b0cfac50c924a679344c1df4062 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Fri, 14 Apr 2017 13:58:38 -0700 Subject: [PATCH 18/38] Fix circular test imports --- client/consul_test.go | 19 ++++++++++++++----- client/driver/driver_test.go | 13 ------------- client/driver/exec_test.go | 5 ++--- client/driver/raw_exec_test.go | 5 ++--- client/driver/rkt_test.go | 5 ++--- 5 files changed, 20 insertions(+), 27 deletions(-) diff --git a/client/consul_test.go b/client/consul_test.go index f37aec3f9adc..81c96d56d5fb 100644 --- a/client/consul_test.go +++ b/client/consul_test.go @@ -7,7 +7,7 @@ import ( "sync" "testing" - "github.com/hashicorp/nomad/command/agent/consul" + "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/nomad/structs" ) @@ -15,7 +15,7 @@ import ( type mockConsulOp struct { allocID string task *structs.Task - exec consul.ScriptExecutor + exec driver.ScriptExecutor } // mockConsulServiceClient implements the ConsulServiceAPI interface to record @@ -40,17 +40,26 @@ func newMockConsulServiceClient() *mockConsulServiceClient { return &m } -func (m *mockConsulServiceClient) RegisterTask(allocID string, task *structs.Task, exec consul.ScriptExecutor) error { - m.logger.Printf("[TEST] mock_consul: RegisterTask(%q, %q, %T)", allocID, task.Name, exec) +func (m *mockConsulServiceClient) UpdateTask(allocID string, old, new *structs.Task, exec driver.ScriptExecutor) error { m.mu.Lock() defer m.mu.Unlock() + m.logger.Printf("[TEST] mock_consul: UpdateTask(%q, %q, %q, %T)", allocID, old, new, exec) + m.removes = append(m.removes, mockConsulOp{allocID, old, exec}) + m.registers = append(m.registers, mockConsulOp{allocID, new, exec}) + return nil +} + +func (m *mockConsulServiceClient) RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor) error { + m.mu.Lock() + defer m.mu.Unlock() + m.logger.Printf("[TEST] mock_consul: RegisterTask(%q, %q, %T)", allocID, task.Name, exec) m.registers = append(m.registers, mockConsulOp{allocID, task, exec}) return nil } func (m *mockConsulServiceClient) RemoveTask(allocID string, task *structs.Task) { - m.logger.Printf("[TEST] mock_consul: RemoveTask(%q, %q)", allocID, task.Name) m.mu.Lock() defer m.mu.Unlock() + m.logger.Printf("[TEST] mock_consul: RemoveTask(%q, %q)", allocID, task.Name) m.removes = append(m.removes, mockConsulOp{allocID, task, nil}) } diff --git a/client/driver/driver_test.go b/client/driver/driver_test.go index b186efac8c16..8cd44b331ae9 100644 --- a/client/driver/driver_test.go +++ b/client/driver/driver_test.go @@ -13,7 +13,6 @@ import ( "github.com/hashicorp/nomad/client/allocdir" "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/driver/env" - "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/testtask" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" @@ -422,15 +421,3 @@ func TestCreatedResources_CopyRemove(t *testing.T) { t.Fatalf("res1 should not equal res2: #%v", res1) } } - -// TestHandleExec statically asserts the drivers we expect to implement the -// consul.Executor interface do. -func TestHandleScriptExecutor(t *testing.T) { - _ = []consul.ScriptExecutor{ - &DockerHandle{}, - &execHandle{}, - &javaHandle{}, - &rawExecHandle{}, - &rktHandle{}, - } -} diff --git a/client/driver/exec_test.go b/client/driver/exec_test.go index 2f35efa8dd6d..6a580f243526 100644 --- a/client/driver/exec_test.go +++ b/client/driver/exec_test.go @@ -13,7 +13,6 @@ import ( "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/driver/env" - "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -317,7 +316,7 @@ func TestExecDriver_HandlerExec(t *testing.T) { } // Exec a command that should work - out, code, err := handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"/alloc"}) + out, code, err := handle.Exec(context.TODO(), "/usr/bin/stat", []string{"/alloc"}) if err != nil { t.Fatalf("error exec'ing stat: %v", err) } @@ -329,7 +328,7 @@ func TestExecDriver_HandlerExec(t *testing.T) { } // Exec a command that should fail - out, code, err = handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"lkjhdsaflkjshowaisxmcvnlia"}) + out, code, err = handle.Exec(context.TODO(), "/usr/bin/stat", []string{"lkjhdsaflkjshowaisxmcvnlia"}) if err != nil { t.Fatalf("error exec'ing stat: %v", err) } diff --git a/client/driver/raw_exec_test.go b/client/driver/raw_exec_test.go index cb098f764884..e1bd4d362617 100644 --- a/client/driver/raw_exec_test.go +++ b/client/driver/raw_exec_test.go @@ -13,7 +13,6 @@ import ( "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/driver/env" - "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/testtask" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -333,7 +332,7 @@ func TestRawExecDriver_HandlerExec(t *testing.T) { } // Exec a command that should work - out, code, err := handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"/tmp"}) + out, code, err := handle.Exec(context.TODO(), "/usr/bin/stat", []string{"/tmp"}) if err != nil { t.Fatalf("error exec'ing stat: %v", err) } @@ -345,7 +344,7 @@ func TestRawExecDriver_HandlerExec(t *testing.T) { } // Exec a command that should fail - out, code, err = handle.(consul.ScriptExecutor).Exec(context.TODO(), "/usr/bin/stat", []string{"lkjhdsaflkjshowaisxmcvnlia"}) + out, code, err = handle.Exec(context.TODO(), "/usr/bin/stat", []string{"lkjhdsaflkjshowaisxmcvnlia"}) if err != nil { t.Fatalf("error exec'ing stat: %v", err) } diff --git a/client/driver/rkt_test.go b/client/driver/rkt_test.go index 34403cb1086e..b61ba04af798 100644 --- a/client/driver/rkt_test.go +++ b/client/driver/rkt_test.go @@ -14,7 +14,6 @@ import ( "time" "github.com/hashicorp/nomad/client/config" - "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -536,7 +535,7 @@ func TestRktDriver_HandlerExec(t *testing.T) { time.Sleep(time.Second) // Exec a command that should work - out, code, err := handle.(consul.ScriptExecutor).Exec(context.TODO(), "/etcd", []string{"--version"}) + out, code, err := handle.Exec(context.TODO(), "/etcd", []string{"--version"}) if err != nil { t.Fatalf("error exec'ing etcd --version: %v", err) } @@ -548,7 +547,7 @@ func TestRktDriver_HandlerExec(t *testing.T) { } // Exec a command that should fail - out, code, err = handle.(consul.ScriptExecutor).Exec(context.TODO(), "/etcd", []string{"--kaljdshf"}) + out, code, err = handle.Exec(context.TODO(), "/etcd", []string{"--kaljdshf"}) if err != nil { t.Fatalf("error exec'ing bad command: %v", err) } From 4f224132ad16c1fbbec2c0fc6b3eb0829a2011ed Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Mon, 17 Apr 2017 16:43:08 -0700 Subject: [PATCH 19/38] Remove stale comment --- client/task_runner.go | 1 - 1 file changed, 1 deletion(-) diff --git a/client/task_runner.go b/client/task_runner.go index 88ce02017ef6..1c0a7f4de380 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -1402,7 +1402,6 @@ func (r *TaskRunner) handleUpdate(update *structs.Allocation) error { var mErr multierror.Error r.handleLock.Lock() if r.handle != nil { - // Need to check driver abilities for updating services drv, err := r.createDriver() if err != nil { // Something has really gone wrong; don't continue From 7de3adaf87ee0b135a2c5d7113ee036de318ca92 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Mon, 17 Apr 2017 17:07:42 -0700 Subject: [PATCH 20/38] Remove commits return value ...and still protect against leaking agent entries in Consul on shutdown. --- command/agent/consul/client.go | 35 ++++++++++++++++++++-------------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index c37e6132298e..767fd4c343a3 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -187,13 +187,11 @@ func (c *ServiceClient) Run() { } } -// commit operations and returns false if shutdown signalled before committing. -func (c *ServiceClient) commit(ops *operations) bool { +// commit operations unless already shutting down. +func (c *ServiceClient) commit(ops *operations) { select { case c.opCh <- ops: - return true case <-c.shutdownCh: - return false } } @@ -365,21 +363,25 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service) } } - // Now add them to the registration queue - if ok := c.commit(&ops); !ok { - // shutting down, exit + // Don't bother committing agent checks if we're already shutting down + c.agentLock.Lock() + defer c.agentLock.Unlock() + select { + case <-c.shutdownCh: return nil + default: } + // Now add them to the registration queue + c.commit(&ops) + // Record IDs for deregistering on shutdown - c.agentLock.Lock() for _, id := range ops.regServices { c.agentServices[id.ID] = struct{}{} } for _, id := range ops.regChecks { c.agentChecks[id.ID] = struct{}{} } - c.agentLock.Unlock() return nil } @@ -544,30 +546,35 @@ func (c *ServiceClient) RemoveTask(allocID string, task *structs.Task) { } // Shutdown the Consul client. Update running task registations and deregister -// agent from Consul. Blocks up to shutdownWait before giving up on syncing -// operations. +// agent from Consul. On first call blocks up to shutdownWait before giving up +// on syncing operations. func (c *ServiceClient) Shutdown() error { + // Serialize Shutdown calls with RegisterAgent to prevent leaking agent + // entries. + c.agentLock.Lock() select { case <-c.shutdownCh: return nil default: } - // First deregister Nomad agent Consul entries + // Deregister Nomad agent Consul entries before closing shutdown. ops := operations{} - c.agentLock.Lock() for id := range c.agentServices { ops.deregServices = append(ops.deregServices, id) } for id := range c.agentChecks { ops.deregChecks = append(ops.deregChecks, id) } - c.agentLock.Unlock() c.commit(&ops) // Then signal shutdown close(c.shutdownCh) + // Safe to unlock after shutdownCh closed as RegisterAgent will check + // shutdownCh before committing. + c.agentLock.Unlock() + // Give run loop time to sync, but don't block indefinitely deadline := time.After(c.shutdownWait) From 927b265854dcfd21c03587074063b3e9a8c31fbc Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Mon, 17 Apr 2017 21:15:13 -0700 Subject: [PATCH 21/38] Rework to account for ports not being in IDs Previous implementation assumed all struct fields were included in service and check IDs. Service IDs never include port labels and check IDs *optionally* include port labels, so lots of things had to change. Added a really big test to exercise this. --- command/agent/consul/client.go | 65 +++++----- command/agent/consul/unit_test.go | 202 +++++++++++++++++++++++++++++- 2 files changed, 229 insertions(+), 38 deletions(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 767fd4c343a3..b1ad827502d8 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -249,13 +249,21 @@ func (c *ServiceClient) sync() error { sdereg++ } + // Track services whose ports have changed as their checks may also + // need updating + portsChanged := make(map[string]struct{}, len(c.services)) + // Add Nomad services missing from Consul - for id, service := range c.services { - if _, ok := consulServices[id]; ok { - // Already in Consul; skipping - continue + for id, locals := range c.services { + if remotes, ok := consulServices[id]; ok { + if locals.Port == remotes.Port { + // Already exists in Consul; skip + continue + } + // Port changed, reregister it and its checks + portsChanged[id] = struct{}{} } - if err = c.client.ServiceRegister(service); err != nil { + if err = c.client.ServiceRegister(locals); err != nil { return err } sreg++ @@ -264,7 +272,7 @@ func (c *ServiceClient) sync() error { // Remove Nomad checks in Consul but unknown locally for id, check := range consulChecks { if _, ok := c.checks[id]; ok { - // Known check, skip + // Known check, leave it continue } if !isNomadService(check.ServiceID) { @@ -280,9 +288,11 @@ func (c *ServiceClient) sync() error { // Add Nomad checks missing from Consul for id, check := range c.checks { - if _, ok := consulChecks[id]; ok { - // Already in Consul; skipping - continue + if check, ok := consulChecks[id]; ok { + if _, changed := portsChanged[check.ServiceID]; !changed { + // Already in Consul and ports didn't change; skipping + continue + } } if err := c.client.CheckRegister(check); err != nil { return err @@ -291,11 +301,11 @@ func (c *ServiceClient) sync() error { // Handle starting scripts if script, ok := c.scripts[id]; ok { - // If it's already running, don't run it again - if _, running := c.runningScripts[id]; running { - continue + // If it's already running, cancel and replace + if oldScript, running := c.runningScripts[id]; running { + oldScript.cancel() } - // Not running, start and store the handle + // Start and store the handle c.runningScripts[id] = script.run() } } @@ -456,8 +466,6 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta newIDs[makeTaskServiceID(allocID, newTask.Name, s)] = s } - parseAddr := newTask.FindHostAndPortFor - // Loop over existing Service IDs to see if they have been removed or // updated. for existingID, existingSvc := range existingIDs { @@ -471,8 +479,10 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta continue } - // Service exists and wasn't updated, don't add it later - delete(newIDs, existingID) + if newSvc.PortLabel == existingSvc.PortLabel { + // Service exists and hasn't changed, don't add it later + delete(newIDs, existingID) + } // Check to see what checks were updated existingChecks := make(map[string]struct{}, len(existingSvc.Checks)) @@ -484,28 +494,9 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta for _, check := range newSvc.Checks { checkID := createCheckID(existingID, check) if _, exists := existingChecks[checkID]; exists { - // Check already exists; skip it + // Check exists, so don't remove it delete(existingChecks, checkID) - continue - } - - // New check, register it - if check.Type == structs.ServiceCheckScript { - if exec == nil { - return fmt.Errorf("driver doesn't support script checks") - } - ops.scripts = append(ops.scripts, newScriptCheck( - existingID, newTask.Name, checkID, check, exec, c.client, c.logger, c.shutdownCh)) } - host, port := parseAddr(existingSvc.PortLabel) - if check.PortLabel != "" { - host, port = parseAddr(check.PortLabel) - } - checkReg, err := createCheckReg(existingID, checkID, check, host, port) - if err != nil { - return err - } - ops.regChecks = append(ops.regChecks, checkReg) } // Remove existing checks not in updated service diff --git a/command/agent/consul/unit_test.go b/command/agent/consul/unit_test.go index 66934663b941..4882be1cc7cb 100644 --- a/command/agent/consul/unit_test.go +++ b/command/agent/consul/unit_test.go @@ -15,6 +15,12 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +const ( + // Ports used in testTask + xPort = 1234 + yPort = 1235 +) + func testLogger() *log.Logger { if testing.Verbose() { return log.New(os.Stderr, "", log.LstdFlags) @@ -28,7 +34,10 @@ func testTask() *structs.Task { Resources: &structs.Resources{ Networks: []*structs.NetworkResource{ { - DynamicPorts: []structs.Port{{Label: "x", Value: 1234}}, + DynamicPorts: []structs.Port{ + {Label: "x", Value: xPort}, + {Label: "y", Value: yPort}, + }, }, }, }, @@ -49,12 +58,20 @@ type testFakeCtx struct { FakeConsul *fakeConsul Task *structs.Task + // Ticked whenever a script is called + execs chan int + + // If non-nil will be called by script checks ExecFunc func(ctx context.Context, cmd string, args []string) ([]byte, int, error) } // Exec implements the ScriptExecutor interface and will use an alternate // implementation t.ExecFunc if non-nil. func (t *testFakeCtx) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) { + select { + case t.execs <- 1: + default: + } if t.ExecFunc == nil { // Default impl is just "ok" return []byte("ok"), 0, nil @@ -84,6 +101,7 @@ func setupFake() *testFakeCtx { ServiceClient: NewServiceClient(fc, testLogger()), FakeConsul: fc, Task: testTask(), + execs: make(chan int, 100), } } @@ -242,6 +260,188 @@ func TestConsul_ChangeTags(t *testing.T) { } } +// TestConsul_ChangePorts asserts that changing the ports on a service updates +// it in Consul. Since ports are part of the service ID this is a slightly +// different code path than changing tags. +func TestConsul_ChangePorts(t *testing.T) { + ctx := setupFake() + ctx.Task.Services[0].Checks = []*structs.ServiceCheck{ + { + Name: "c1", + Type: "tcp", + Interval: time.Second, + Timeout: time.Second, + PortLabel: "x", + }, + { + Name: "c2", + Type: "script", + Interval: 9000 * time.Hour, + Timeout: time.Second, + }, + { + Name: "c3", + Type: "http", + Protocol: "http", + Path: "/", + Interval: time.Second, + Timeout: time.Second, + PortLabel: "y", + }, + } + + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, ctx); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + if err := ctx.syncOnce(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + + origServiceKey := "" + for k, v := range ctx.FakeConsul.services { + origServiceKey = k + if v.Name != ctx.Task.Services[0].Name { + t.Errorf("expected Name=%q != %q", ctx.Task.Services[0].Name, v.Name) + } + if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) + } + if v.Port != xPort { + t.Errorf("expected Port x=%v but found: %v", xPort, v.Port) + } + } + + if n := len(ctx.FakeConsul.checks); n != 3 { + t.Fatalf("expected 3 checks but found %d:\n%#v", n, ctx.FakeConsul.checks) + } + + origTCPKey := "" + origScriptKey := "" + origHTTPKey := "" + for k, v := range ctx.FakeConsul.checks { + switch v.Name { + case "c1": + origTCPKey = k + if expected := fmt.Sprintf(":%d", xPort); v.TCP != expected { + t.Errorf("expected Port x=%v but found: %v", expected, v.TCP) + } + case "c2": + origScriptKey = k + select { + case <-ctx.execs: + if n := len(ctx.execs); n > 0 { + t.Errorf("expected 1 exec but found: %d", n+1) + } + case <-time.After(3 * time.Second): + t.Errorf("script not called in time") + } + case "c3": + origHTTPKey = k + if expected := fmt.Sprintf("http://:%d/", yPort); v.HTTP != expected { + t.Errorf("expected Port y=%v but found: %v", expected, v.HTTP) + } + default: + t.Fatalf("unexpected check: %q", v.Name) + } + } + + // Now update the PortLabel on the Service and Check c3 + origTask := ctx.Task + ctx.Task = testTask() + ctx.Task.Services[0].PortLabel = "y" + ctx.Task.Services[0].Checks = []*structs.ServiceCheck{ + { + Name: "c1", + Type: "tcp", + Interval: time.Second, + Timeout: time.Second, + PortLabel: "x", + }, + { + Name: "c2", + Type: "script", + Interval: 9000 * time.Hour, + Timeout: time.Second, + }, + { + Name: "c3", + Type: "http", + Protocol: "http", + Path: "/", + Interval: time.Second, + Timeout: time.Second, + // Removed PortLabel + }, + } + if err := ctx.ServiceClient.UpdateTask("allocid", origTask, ctx.Task, ctx); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + if err := ctx.syncOnce(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + + if n := len(ctx.FakeConsul.services); n != 1 { + t.Fatalf("expected 1 service but found %d:\n%#v", n, ctx.FakeConsul.services) + } + + for k, v := range ctx.FakeConsul.services { + if k != origServiceKey { + t.Errorf("unexpected key change; was: %q -- but found %q", origServiceKey, k) + } + if v.Name != ctx.Task.Services[0].Name { + t.Errorf("expected Name=%q != %q", ctx.Task.Services[0].Name, v.Name) + } + if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { + t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) + } + if v.Port != yPort { + t.Errorf("expected Port y=%v but found: %v", yPort, v.Port) + } + } + + if n := len(ctx.FakeConsul.checks); n != 3 { + t.Fatalf("expected 3 check but found %d:\n%#v", n, ctx.FakeConsul.checks) + } + + for k, v := range ctx.FakeConsul.checks { + switch v.Name { + case "c1": + if k != origTCPKey { + t.Errorf("unexpected key change for %s from %q to %q", v.Name, origTCPKey, k) + } + if expected := fmt.Sprintf(":%d", xPort); v.TCP != expected { + t.Errorf("expected Port x=%v but found: %v", expected, v.TCP) + } + case "c2": + if k != origScriptKey { + t.Errorf("unexpected key change for %s from %q to %q", v.Name, origScriptKey, k) + } + select { + case <-ctx.execs: + if n := len(ctx.execs); n > 0 { + t.Errorf("expected 1 exec but found: %d", n+1) + } + case <-time.After(3 * time.Second): + t.Errorf("script not called in time") + } + case "c3": + if k == origHTTPKey { + t.Errorf("expected %s key to change from %q", v.Name, k) + } + if expected := fmt.Sprintf("http://:%d/", yPort); v.HTTP != expected { + t.Errorf("expected Port y=%v but found: %v", expected, v.HTTP) + } + default: + t.Errorf("Unkown check: %q", k) + } + } +} + // TestConsul_RegServices tests basic service registration. func TestConsul_RegServices(t *testing.T) { ctx := setupFake() From 6a56b0efc61bdd0bd302578a00f73e83ca73084d Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 13:22:40 -0700 Subject: [PATCH 22/38] Follow _testing.go convention for testing tools --- command/agent/consul/{mock.go => catalog_testing.go} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename command/agent/consul/{mock.go => catalog_testing.go} (100%) diff --git a/command/agent/consul/mock.go b/command/agent/consul/catalog_testing.go similarity index 100% rename from command/agent/consul/mock.go rename to command/agent/consul/catalog_testing.go From 465cc5157be165aaa4a11b324d05ba6f2b666fbe Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 14:52:19 -0700 Subject: [PATCH 23/38] Test script check exit codes --- command/agent/consul/script_test.go | 86 +++++++++++++++++++++++++---- 1 file changed, 76 insertions(+), 10 deletions(-) diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index 3d713f0c1aff..76192692adf4 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -2,6 +2,7 @@ package consul import ( "context" + "fmt" "os" "os/exec" "testing" @@ -76,17 +77,25 @@ func TestConsulScript_Exec_Cancel(t *testing.T) { } } +type execStatus struct { + checkID string + output string + status string +} + +// fakeHeartbeater implements the heartbeater interface to allow mocking out +// Consul in script executor tests. type fakeHeartbeater struct { - updates chan string + updates chan execStatus } func (f *fakeHeartbeater) UpdateTTL(checkID, output, status string) error { - f.updates <- status + f.updates <- execStatus{checkID: checkID, output: output, status: status} return nil } func newFakeHeartbeater() *fakeHeartbeater { - return &fakeHeartbeater{updates: make(chan string)} + return &fakeHeartbeater{updates: make(chan execStatus)} } // TestConsulScript_Exec_Timeout asserts a script will be killed when the @@ -109,7 +118,7 @@ func TestConsulScript_Exec_Timeout(t *testing.T) { // Check for UpdateTTL call select { case update := <-hb.updates: - if update != api.HealthCritical { + if update.status != api.HealthCritical { t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) } case <-time.After(3 * time.Second): @@ -131,10 +140,19 @@ func TestConsulScript_Exec_Timeout(t *testing.T) { } } -type noopExec struct{} +// simpleExec is a fake ScriptExecutor that returns whatever is specified. +type simpleExec struct { + code int + err error +} -func (noopExec) Exec(context.Context, string, []string) ([]byte, int, error) { - return []byte{}, 0, nil +func (s simpleExec) Exec(context.Context, string, []string) ([]byte, int, error) { + return []byte(fmt.Sprintf("code=%d err=%v", s.code, s.err)), s.code, s.err +} + +// newSimpleExec creates a new ScriptExecutor that returns the given code and err. +func newSimpleExec(code int, err error) simpleExec { + return simpleExec{code: code, err: err} } // TestConsulScript_Exec_Shutdown asserts a script will be executed once more @@ -148,7 +166,8 @@ func TestConsulScript_Exec_Shutdown(t *testing.T) { hb := newFakeHeartbeater() shutdown := make(chan struct{}) - check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, noopExec{}, hb, testLogger(), shutdown) + exec := newSimpleExec(0, nil) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, hb, testLogger(), shutdown) handle := check.run() defer handle.cancel() // just-in-case cleanup @@ -157,8 +176,8 @@ func TestConsulScript_Exec_Shutdown(t *testing.T) { select { case update := <-hb.updates: - if update != api.HealthPassing { - t.Error("expected %q due to timeout but received %q", api.HealthPassing, update) + if update.status != api.HealthPassing { + t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) } case <-time.After(3 * time.Second): t.Fatalf("timed out waiting for script check to exit") @@ -171,3 +190,50 @@ func TestConsulScript_Exec_Shutdown(t *testing.T) { t.Fatalf("timed out waiting for script check to exit") } } + +func TestConsulScript_Exec_Codes(t *testing.T) { + run := func(code int, err error, expected string) { + serviceCheck := structs.ServiceCheck{ + Name: "test", + Interval: time.Hour, + Timeout: 3 * time.Second, + } + + hb := newFakeHeartbeater() + shutdown := make(chan struct{}) + exec := newSimpleExec(code, err) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, hb, testLogger(), shutdown) + handle := check.run() + defer handle.cancel() + + select { + case update := <-hb.updates: + if update.status != expected { + t.Errorf("expected %q but received %q", expected, update) + } + // assert output is being reported + expectedOutput := fmt.Sprintf("code=%d err=%v", code, err) + if err != nil { + expectedOutput = err.Error() + } + if update.output != expectedOutput { + t.Errorf("expected output=%q but found: %q", expectedOutput, update.output) + } + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exec") + } + } + + // Test exit codes with errors + run(0, nil, api.HealthPassing) + run(1, nil, api.HealthWarning) + run(2, nil, api.HealthCritical) + run(9000, nil, api.HealthCritical) + + // Errors should always cause Critical status + err := fmt.Errorf("test error") + run(0, err, api.HealthCritical) + run(1, err, api.HealthCritical) + run(2, err, api.HealthCritical) + run(9000, err, api.HealthCritical) +} From d3f3af8bf933c1aed3919c817c27d35763210c47 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 15:28:44 -0700 Subject: [PATCH 24/38] Always fail script checks when deadline exceeded --- command/agent/consul/script.go | 10 ++++++-- command/agent/consul/script_test.go | 37 +++++++++++++++++++++++++++++ 2 files changed, 45 insertions(+), 2 deletions(-) diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 3e96df00d421..26228b42b247 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -96,8 +96,14 @@ func (s *scriptCheck) run() *scriptHandle { // check removed during execution; exit return case context.DeadlineExceeded: - // Log deadline exceeded every time, but flip last check to false - s.lastCheckOk = false + // If no error was returned, set one to make sure the task goes critical + if err == nil { + err = context.DeadlineExceeded + } + + // Log deadline exceeded every time as it's a + // distinct issue from checks returning + // failures s.logger.Printf("[WARN] consul.checks: check %q for task %q alloc %q timed out (%s)", s.check.Name, s.taskName, s.allocID, s.check.Timeout) } diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index 76192692adf4..1fa565d168b1 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -140,6 +140,43 @@ func TestConsulScript_Exec_Timeout(t *testing.T) { } } +// sleeperExec sleeps for 100ms but returns successfully to allow testing timeout conditions +type sleeperExec struct{} + +func (sleeperExec) Exec(context.Context, string, []string) ([]byte, int, error) { + time.Sleep(100 * time.Millisecond) + return []byte{}, 0, nil +} + +// TestConsulScript_Exec_TimeoutCritical asserts a script will be killed when +// the timeout is reached and always set a critical status regardless of what +// Exec returns. +func TestConsulScript_Exec_TimeoutCritical(t *testing.T) { + t.Parallel() // run the slow tests in parallel + serviceCheck := structs.ServiceCheck{ + Name: "sleeper", + Interval: time.Hour, + Timeout: time.Nanosecond, + } + hb := newFakeHeartbeater() + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, sleeperExec{}, hb, testLogger(), nil) + handle := check.run() + defer handle.cancel() // just-in-case cleanup + + // Check for UpdateTTL call + select { + case update := <-hb.updates: + if update.status != api.HealthCritical { + t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) + } + if update.output != context.DeadlineExceeded.Error() { + t.Errorf("expected output=%q but found: %q", context.DeadlineExceeded.Error(), update.output) + } + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to timeout") + } +} + // simpleExec is a fake ScriptExecutor that returns whatever is specified. type simpleExec struct { code int From de3d78365e3175c714bedf7fdf916a5e6301f0ae Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 16:23:39 -0700 Subject: [PATCH 25/38] Metricsify new Consul client --- command/agent/consul/client.go | 14 ++++++++++++++ command/agent/consul/script.go | 3 +++ 2 files changed, 17 insertions(+) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index b1ad827502d8..3d3b4027c33b 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -10,6 +10,7 @@ import ( "sync" "time" + metrics "github.com/armon/go-metrics" "github.com/hashicorp/consul/api" "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/nomad/structs" @@ -216,6 +217,9 @@ func (c *ServiceClient) merge(ops *operations) { } delete(c.checks, cid) } + metrics.SetGauge([]string{"client", "consul", "services"}, float32(len(c.services))) + metrics.SetGauge([]string{"client", "consul", "checks"}, float32(len(c.checks))) + metrics.SetGauge([]string{"client", "consul", "script_checks"}, float32(len(c.runningScripts))) } // sync enqueued operations. @@ -224,11 +228,13 @@ func (c *ServiceClient) sync() error { consulServices, err := c.client.Services() if err != nil { + metrics.IncrCounter([]string{"client", "consul", "sync_failure"}, 1) return fmt.Errorf("error querying Consul services: %v", err) } consulChecks, err := c.client.Checks() if err != nil { + metrics.IncrCounter([]string{"client", "consul", "sync_failure"}, 1) return fmt.Errorf("error querying Consul checks: %v", err) } @@ -244,9 +250,11 @@ func (c *ServiceClient) sync() error { } // Unknown Nomad managed service; kill if err := c.client.ServiceDeregister(id); err != nil { + metrics.IncrCounter([]string{"client", "consul", "sync_failure"}, 1) return err } sdereg++ + metrics.IncrCounter([]string{"client", "consul", "service_deregisrations"}, 1) } // Track services whose ports have changed as their checks may also @@ -264,9 +272,11 @@ func (c *ServiceClient) sync() error { portsChanged[id] = struct{}{} } if err = c.client.ServiceRegister(locals); err != nil { + metrics.IncrCounter([]string{"client", "consul", "sync_failure"}, 1) return err } sreg++ + metrics.IncrCounter([]string{"client", "consul", "service_regisrations"}, 1) } // Remove Nomad checks in Consul but unknown locally @@ -281,9 +291,11 @@ func (c *ServiceClient) sync() error { } // Unknown Nomad managed check; kill if err := c.client.CheckDeregister(id); err != nil { + metrics.IncrCounter([]string{"client", "consul", "sync_failure"}, 1) return err } cdereg++ + metrics.IncrCounter([]string{"client", "consul", "check_deregisrations"}, 1) } // Add Nomad checks missing from Consul @@ -295,9 +307,11 @@ func (c *ServiceClient) sync() error { } } if err := c.client.CheckRegister(check); err != nil { + metrics.IncrCounter([]string{"client", "consul", "sync_failure"}, 1) return err } creg++ + metrics.IncrCounter([]string{"client", "consul", "check_regisrations"}, 1) // Handle starting scripts if script, ok := c.scripts[id]; ok { diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 26228b42b247..9f540d7fa4cd 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -5,6 +5,7 @@ import ( "log" "time" + metrics "github.com/armon/go-metrics" "github.com/hashicorp/consul/api" "github.com/hashicorp/nomad/client/driver" "github.com/hashicorp/nomad/nomad/structs" @@ -87,6 +88,7 @@ func (s *scriptCheck) run() *scriptHandle { case <-timer.C: timer.Reset(s.check.Interval) } + metrics.IncrCounter([]string{"client", "consul", "script_runs"}, 1) // Execute check script with timeout execctx, cancel := context.WithTimeout(ctx, s.check.Timeout) @@ -96,6 +98,7 @@ func (s *scriptCheck) run() *scriptHandle { // check removed during execution; exit return case context.DeadlineExceeded: + metrics.IncrCounter([]string{"client", "consul", "script_timeouts"}, 1) // If no error was returned, set one to make sure the task goes critical if err == nil { err = context.DeadlineExceeded From f0bec631748c521eac26976786ce8b2a03d15aa8 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 16:36:20 -0700 Subject: [PATCH 26/38] Explain weird timer logic --- command/agent/consul/client.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 3d3b4027c33b..7f0a9a7326d3 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -155,6 +155,9 @@ func (c *ServiceClient) Run() { } failures++ if !retryTimer.Stop() { + // Timer already expired, since the timer may + // or may not have been read in the select{} + // above, conditionally receive on it select { case <-retryTimer.C: default: From 8e0c97e52f4117bec613f5ffb8a9d8becde26df8 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 17:18:23 -0700 Subject: [PATCH 27/38] Unregister from Consul when waiting for restart --- client/consul_test.go | 31 ++++++++++++++++-------- client/task_runner.go | 4 ++++ client/task_runner_test.go | 49 +++++++++++++++++++++++++++++++++++++- 3 files changed, 73 insertions(+), 11 deletions(-) diff --git a/client/consul_test.go b/client/consul_test.go index 81c96d56d5fb..7d68faae2dff 100644 --- a/client/consul_test.go +++ b/client/consul_test.go @@ -1,6 +1,7 @@ package client import ( + "fmt" "io/ioutil" "log" "os" @@ -13,26 +14,37 @@ import ( // mockConsulOp represents the register/deregister operations. type mockConsulOp struct { + op string // add, remove, or update allocID string task *structs.Task exec driver.ScriptExecutor } +func newMockConsulOp(op, allocID string, task *structs.Task, exec driver.ScriptExecutor) mockConsulOp { + if op != "add" && op != "remove" && op != "update" { + panic(fmt.Errorf("invalid consul op: %s", op)) + } + return mockConsulOp{ + op: op, + allocID: allocID, + task: task, + exec: exec, + } +} + // mockConsulServiceClient implements the ConsulServiceAPI interface to record // and log task registration/deregistration. type mockConsulServiceClient struct { - registers []mockConsulOp - removes []mockConsulOp - mu sync.Mutex + ops []mockConsulOp + mu sync.Mutex logger *log.Logger } func newMockConsulServiceClient() *mockConsulServiceClient { m := mockConsulServiceClient{ - registers: make([]mockConsulOp, 0, 10), - removes: make([]mockConsulOp, 0, 10), - logger: log.New(ioutil.Discard, "", 0), + ops: make([]mockConsulOp, 0, 20), + logger: log.New(ioutil.Discard, "", 0), } if testing.Verbose() { m.logger = log.New(os.Stderr, "", log.LstdFlags) @@ -44,8 +56,7 @@ func (m *mockConsulServiceClient) UpdateTask(allocID string, old, new *structs.T m.mu.Lock() defer m.mu.Unlock() m.logger.Printf("[TEST] mock_consul: UpdateTask(%q, %q, %q, %T)", allocID, old, new, exec) - m.removes = append(m.removes, mockConsulOp{allocID, old, exec}) - m.registers = append(m.registers, mockConsulOp{allocID, new, exec}) + m.ops = append(m.ops, newMockConsulOp("update", allocID, old, exec)) return nil } @@ -53,7 +64,7 @@ func (m *mockConsulServiceClient) RegisterTask(allocID string, task *structs.Tas m.mu.Lock() defer m.mu.Unlock() m.logger.Printf("[TEST] mock_consul: RegisterTask(%q, %q, %T)", allocID, task.Name, exec) - m.registers = append(m.registers, mockConsulOp{allocID, task, exec}) + m.ops = append(m.ops, newMockConsulOp("add", allocID, task, exec)) return nil } @@ -61,5 +72,5 @@ func (m *mockConsulServiceClient) RemoveTask(allocID string, task *structs.Task) m.mu.Lock() defer m.mu.Unlock() m.logger.Printf("[TEST] mock_consul: RemoveTask(%q, %q)", allocID, task.Name) - m.removes = append(m.removes, mockConsulOp{allocID, task, nil}) + m.ops = append(m.ops, newMockConsulOp("remove", allocID, task, nil)) } diff --git a/client/task_runner.go b/client/task_runner.go index 1c0a7f4de380..c578ca5ea0b7 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -1056,6 +1056,7 @@ func (r *TaskRunner) run() { } RESTART: + // shouldRestart will block if the task should restart after a delay. restart := r.shouldRestart() if !restart { r.cleanup() @@ -1136,6 +1137,9 @@ func (r *TaskRunner) shouldRestart() bool { return false } + // Unregister from Consul while waiting to restart. + r.consul.RemoveTask(r.alloc.ID, r.task) + // Sleep but watch for destroy events. select { case <-time.After(when): diff --git a/client/task_runner_test.go b/client/task_runner_test.go index ec511e93dfbc..ede8cb1647c9 100644 --- a/client/task_runner_test.go +++ b/client/task_runner_test.go @@ -466,7 +466,7 @@ func TestTaskRunner_Download_Retries(t *testing.T) { } task.Artifacts = []*structs.TaskArtifact{&artifact} - // Make the restart policy try one ctx.upd.te + // Make the restart policy try one ctx.update alloc.Job.TaskGroups[0].RestartPolicy = &structs.RestartPolicy{ Attempts: 1, Interval: 10 * time.Minute, @@ -526,6 +526,53 @@ func TestTaskRunner_Download_Retries(t *testing.T) { } } +// TestTaskRunner_UnregisterConsul_Retries asserts a task is unregistered from +// Consul when waiting to be retried. +func TestTaskRunner_UnregisterConsul_Retries(t *testing.T) { + ctestutil.ExecCompatible(t) + + // Create an allocation that has a task with bad artifacts. + alloc := mock.Alloc() + + // Make the restart policy try one ctx.update + alloc.Job.TaskGroups[0].RestartPolicy = &structs.RestartPolicy{ + Attempts: 1, + Interval: 10 * time.Minute, + Delay: time.Nanosecond, + Mode: structs.RestartPolicyModeFail, + } + + task := alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config = map[string]interface{}{ + "exit_code": "1", + "run_for": "1ns", + } + + ctx := testTaskRunnerFromAlloc(t, true, alloc) + ctx.tr.MarkReceived() + ctx.tr.Run() + defer ctx.Cleanup() + + // Assert it is properly registered and unregistered + consul := ctx.tr.consul.(*mockConsulServiceClient) + if expected := 4; len(consul.ops) != expected { + t.Errorf("expected %d consul ops but found: %d", expected, len(consul.ops)) + } + if consul.ops[0].op != "add" { + t.Errorf("expected first op to be add but found: %q", consul.ops[0].op) + } + if consul.ops[1].op != "remove" { + t.Errorf("expected second op to be remove but found: %q", consul.ops[1].op) + } + if consul.ops[2].op != "add" { + t.Errorf("expected third op to be add but found: %q", consul.ops[2].op) + } + if consul.ops[3].op != "remove" { + t.Errorf("expected fourth/final op to be remove but found: %q", consul.ops[3].op) + } +} + func TestTaskRunner_Validate_UserEnforcement(t *testing.T) { ctestutil.ExecCompatible(t) ctx := testTaskRunner(t, false) From 346838381b2b3425771c8428e7ebfdcaf0ff0cd7 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Tue, 18 Apr 2017 21:28:25 -0700 Subject: [PATCH 28/38] Only register HTTPS agent check when Consul>=0.7.2 Support for TLSSkipVerify in other checks coming soon! --- command/agent/agent.go | 97 +++++++++++++++++++++++++++--- command/agent/agent_test.go | 105 +++++++++++++++++++++++++++++++++ command/agent/consul/client.go | 3 + nomad/structs/structs.go | 9 +++ 4 files changed, 205 insertions(+), 9 deletions(-) diff --git a/command/agent/agent.go b/command/agent/agent.go index 8472a98f1da6..f20026eb73be 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -8,6 +8,7 @@ import ( "os" "path/filepath" "runtime" + "strconv" "strings" "sync" "sync/atomic" @@ -54,6 +55,10 @@ type Agent struct { // consulCatalog is the subset of Consul's Catalog API Nomad uses. consulCatalog consul.CatalogAPI + // consulSupportsTLSSkipVerify flags whether or not Nomad can register + // checks with TLSSkipVerify + consulSupportsTLSSkipVerify bool + client *client.Client server *nomad.Server @@ -374,6 +379,16 @@ func (a *Agent) setupServer() error { }, }, } + if conf.TLSConfig.EnableHTTP { + if a.consulSupportsTLSSkipVerify { + httpServ.Checks[0].Protocol = "https" + httpServ.Checks[0].TLSSkipVerify = true + } else { + // No TLSSkipVerify support, don't register https check + a.logger.Printf("[WARN] agent: not registering Nomad HTTPS Health Check because it requires Consul>=0.7.2") + httpServ.Checks = []*structs.ServiceCheck{} + } + } rpcServ := &structs.Service{ Name: a.config.Consul.ServerServiceName, PortLabel: a.config.AdvertiseAddrs.RPC, @@ -404,13 +419,10 @@ func (a *Agent) setupServer() error { } // Add the http port check if TLS isn't enabled - // TODO Add TLS check when Consul 0.7.1 comes out. consulServices := []*structs.Service{ rpcServ, serfServ, - } - if !conf.TLSConfig.EnableHTTP { - consulServices = append(consulServices, httpServ) + httpServ, } if err := a.consulService.RegisterAgent(consulRoleServer, consulServices); err != nil { return err @@ -477,8 +489,6 @@ func (a *Agent) setupClient() error { } // Create the Nomad Client services for Consul - // TODO think how we can re-introduce HTTP/S checks when Consul 0.7.1 comes - // out if *a.config.Consul.AutoAdvertise { httpServ := &structs.Service{ Name: a.config.Consul.ClientServiceName, @@ -496,11 +506,19 @@ func (a *Agent) setupClient() error { }, }, } - if !conf.TLSConfig.EnableHTTP { - if err := a.consulService.RegisterAgent(consulRoleClient, []*structs.Service{httpServ}); err != nil { - return err + if conf.TLSConfig.EnableHTTP { + if a.consulSupportsTLSSkipVerify { + httpServ.Checks[0].Protocol = "https" + httpServ.Checks[0].TLSSkipVerify = true + } else { + // No TLSSkipVerify support, don't register https check + a.logger.Printf("[WARN] agent: not registering Nomad HTTPS Health Check because it requires Consul>=0.7.2") + httpServ.Checks = []*structs.ServiceCheck{} } } + if err := a.consulService.RegisterAgent(consulRoleClient, []*structs.Service{httpServ}); err != nil { + return err + } } return nil @@ -672,6 +690,11 @@ func (a *Agent) setupConsul(consulConfig *config.ConsulConfig) error { return err } + // Determine version for TLSSkipVerify + if self, err := client.Agent().Self(); err != nil { + a.consulSupportsTLSSkipVerify = consulSupportsTLSSkipVerify(self) + } + // Create Consul Catalog client for service discovery. a.consulCatalog = client.Catalog() @@ -680,3 +703,59 @@ func (a *Agent) setupConsul(consulConfig *config.ConsulConfig) error { go a.consulService.Run() return nil } + +// consulSupportsTLSSkipVerify returns true if Consul supports TLSSkipVerify. +func consulSupportsTLSSkipVerify(self map[string]map[string]interface{}) bool { + member, ok := self["Member"] + if !ok { + return false + } + tagsI, ok := member["Tags"] + if !ok { + return false + } + tags, ok := tagsI.(map[string]interface{}) + if !ok { + return false + } + buildI, ok := tags["build"] + if !ok { + return false + } + build, ok := buildI.(string) + if !ok { + return false + } + parts := strings.SplitN(build, ":", 2) + if len(parts) == 0 { + return false + } + parts = strings.Split(parts[0], ".") + if len(parts) != 3 { + return false + } + major, err := strconv.Atoi(parts[0]) + if err != nil { + return false + } + minor, err := strconv.Atoi(parts[1]) + if err != nil { + return false + } + patch, err := strconv.Atoi(parts[2]) + if err != nil { + return false + } + if major > 0 || minor > 7 { + // After 0.7.2! + return true + } + if minor < 7 { + return false + } + if patch < 2 { + return false + } + // 0.7.2 or higher! + return true +} diff --git a/command/agent/agent_test.go b/command/agent/agent_test.go index 89db490d7a78..24d45ee8d1ef 100644 --- a/command/agent/agent_test.go +++ b/command/agent/agent_test.go @@ -1,6 +1,7 @@ package agent import ( + "encoding/json" "fmt" "io/ioutil" "net" @@ -358,3 +359,107 @@ func TestAgent_ClientConfig(t *testing.T) { t.Fatalf("Expected http addr: %v, got: %v", expectedHttpAddr, c.Node.HTTPAddr) } } + +func TestAgent_ConsulSupportsTLSSkipVerify(t *testing.T) { + assertSupport := func(expected bool, blob string) { + self := map[string]map[string]interface{}{} + if err := json.Unmarshal([]byte("{"+blob+"}"), &self); err != nil { + t.Fatalf("invalid json: %v", err) + } + actual := consulSupportsTLSSkipVerify(self) + if actual != expected { + t.Errorf("expected %t but got %t for:\n%s\n", expected, actual, blob) + } + } + + // 0.6.4 + assertSupport(false, `"Member": { + "Addr": "127.0.0.1", + "DelegateCur": 4, + "DelegateMax": 4, + "DelegateMin": 2, + "Name": "rusty", + "Port": 8301, + "ProtocolCur": 2, + "ProtocolMax": 3, + "ProtocolMin": 1, + "Status": 1, + "Tags": { + "build": "0.6.4:26a0ef8c", + "dc": "dc1", + "port": "8300", + "role": "consul", + "vsn": "2", + "vsn_max": "3", + "vsn_min": "1" + }}`) + + // 0.7.0 + assertSupport(false, `"Member": { + "Addr": "127.0.0.1", + "DelegateCur": 4, + "DelegateMax": 4, + "DelegateMin": 2, + "Name": "rusty", + "Port": 8301, + "ProtocolCur": 2, + "ProtocolMax": 4, + "ProtocolMin": 1, + "Status": 1, + "Tags": { + "build": "0.7.0:'a189091", + "dc": "dc1", + "port": "8300", + "role": "consul", + "vsn": "2", + "vsn_max": "3", + "vsn_min": "2" + }}`) + + // 0.7.2 + assertSupport(true, `"Member": { + "Addr": "127.0.0.1", + "DelegateCur": 4, + "DelegateMax": 4, + "DelegateMin": 2, + "Name": "rusty", + "Port": 8301, + "ProtocolCur": 2, + "ProtocolMax": 5, + "ProtocolMin": 1, + "Status": 1, + "Tags": { + "build": "0.7.2:'a9afa0c", + "dc": "dc1", + "port": "8300", + "role": "consul", + "vsn": "2", + "vsn_max": "3", + "vsn_min": "2" + }}`) + + // 0.8.1 + assertSupport(true, `"Member": { + "Addr": "127.0.0.1", + "DelegateCur": 4, + "DelegateMax": 5, + "DelegateMin": 2, + "Name": "rusty", + "Port": 8301, + "ProtocolCur": 2, + "ProtocolMax": 5, + "ProtocolMin": 1, + "Status": 1, + "Tags": { + "build": "0.8.1:'e9ca44d", + "dc": "dc1", + "id": "3ddc1b59-460e-a100-1d5c-ce3972122664", + "port": "8300", + "raft_vsn": "2", + "role": "consul", + "vsn": "2", + "vsn_max": "3", + "vsn_min": "2", + "wan_join_port": "8302" + }}`) +} diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 7f0a9a7326d3..246d4f478038 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -666,6 +666,9 @@ func createCheckReg(serviceID, checkID string, check *structs.ServiceCheck, host if check.Protocol == "" { check.Protocol = "http" } + if check.TLSSkipVerify { + chkReg.TLSSkipVerify = true + } base := url.URL{ Scheme: check.Protocol, Host: net.JoinHostPort(host, strconv.Itoa(port)), diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index f04a525f0710..49196b41bc54 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -2116,6 +2116,7 @@ type ServiceCheck struct { Interval time.Duration // Interval of the check Timeout time.Duration // Timeout of the response from the check before consul fails the check InitialStatus string // Initial status of the check + TLSSkipVerify bool // Skip TLS verification when Protocol=https } func (sc *ServiceCheck) Copy() *ServiceCheck { @@ -2199,6 +2200,10 @@ func (sc *ServiceCheck) RequiresPort() bool { } } +// Hash all ServiceCheck fields and the check's corresponding service ID to +// create an identifier. The identifier is not guaranteed to be unique as if +// the PortLabel is blank, the Service's PortLabel will be used after Hash is +// called. func (sc *ServiceCheck) Hash(serviceID string) string { h := sha1.New() io.WriteString(h, serviceID) @@ -2211,6 +2216,10 @@ func (sc *ServiceCheck) Hash(serviceID string) string { io.WriteString(h, sc.PortLabel) io.WriteString(h, sc.Interval.String()) io.WriteString(h, sc.Timeout.String()) + // Only include TLSSkipVerify if set to maintain ID stability with Nomad <0.6 + if sc.TLSSkipVerify { + io.WriteString(h, "true") + } return fmt.Sprintf("%x", h.Sum(nil)) } From 5948daf4f7907b84a6d5601600511e24e55f8a1b Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 10:33:04 -0700 Subject: [PATCH 29/38] Forgot an important word --- command/agent/consul/unit_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/command/agent/consul/unit_test.go b/command/agent/consul/unit_test.go index 4882be1cc7cb..622b4628aedb 100644 --- a/command/agent/consul/unit_test.go +++ b/command/agent/consul/unit_test.go @@ -261,7 +261,7 @@ func TestConsul_ChangeTags(t *testing.T) { } // TestConsul_ChangePorts asserts that changing the ports on a service updates -// it in Consul. Since ports are part of the service ID this is a slightly +// it in Consul. Since ports are not part of the service ID this is a slightly // different code path than changing tags. func TestConsul_ChangePorts(t *testing.T) { ctx := setupFake() From 6d9e61bb51f6ff3400139324fa7006c8e71fec32 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 10:53:22 -0700 Subject: [PATCH 30/38] Use spiffy new Go 1.8 subtest feature --- command/agent/consul/script_test.go | 73 +++++++++++++++-------------- 1 file changed, 38 insertions(+), 35 deletions(-) diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index 1fa565d168b1..35902c9f8b47 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -229,48 +229,51 @@ func TestConsulScript_Exec_Shutdown(t *testing.T) { } func TestConsulScript_Exec_Codes(t *testing.T) { - run := func(code int, err error, expected string) { - serviceCheck := structs.ServiceCheck{ - Name: "test", - Interval: time.Hour, - Timeout: 3 * time.Second, - } - - hb := newFakeHeartbeater() - shutdown := make(chan struct{}) - exec := newSimpleExec(code, err) - check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, hb, testLogger(), shutdown) - handle := check.run() - defer handle.cancel() - - select { - case update := <-hb.updates: - if update.status != expected { - t.Errorf("expected %q but received %q", expected, update) + run := func(code int, err error, expected string) func(t *testing.T) { + return func(t *testing.T) { + t.Parallel() + serviceCheck := structs.ServiceCheck{ + Name: "test", + Interval: time.Hour, + Timeout: 3 * time.Second, } - // assert output is being reported - expectedOutput := fmt.Sprintf("code=%d err=%v", code, err) - if err != nil { - expectedOutput = err.Error() - } - if update.output != expectedOutput { - t.Errorf("expected output=%q but found: %q", expectedOutput, update.output) + + hb := newFakeHeartbeater() + shutdown := make(chan struct{}) + exec := newSimpleExec(code, err) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, hb, testLogger(), shutdown) + handle := check.run() + defer handle.cancel() + + select { + case update := <-hb.updates: + if update.status != expected { + t.Errorf("expected %q but received %q", expected, update) + } + // assert output is being reported + expectedOutput := fmt.Sprintf("code=%d err=%v", code, err) + if err != nil { + expectedOutput = err.Error() + } + if update.output != expectedOutput { + t.Errorf("expected output=%q but found: %q", expectedOutput, update.output) + } + case <-time.After(3 * time.Second): + t.Fatalf("timed out waiting for script check to exec") } - case <-time.After(3 * time.Second): - t.Fatalf("timed out waiting for script check to exec") } } // Test exit codes with errors - run(0, nil, api.HealthPassing) - run(1, nil, api.HealthWarning) - run(2, nil, api.HealthCritical) - run(9000, nil, api.HealthCritical) + t.Run("Passing", run(0, nil, api.HealthPassing)) + t.Run("Warning", run(1, nil, api.HealthWarning)) + t.Run("Critical-2", run(2, nil, api.HealthCritical)) + t.Run("Critical-9000", run(9000, nil, api.HealthCritical)) // Errors should always cause Critical status err := fmt.Errorf("test error") - run(0, err, api.HealthCritical) - run(1, err, api.HealthCritical) - run(2, err, api.HealthCritical) - run(9000, err, api.HealthCritical) + t.Run("Error-0", run(0, err, api.HealthCritical)) + t.Run("Error-1", run(1, err, api.HealthCritical)) + t.Run("Error-2", run(2, err, api.HealthCritical)) + t.Run("Error-9000", run(9000, err, api.HealthCritical)) } From 64057d48fe33de79468007d2fc192e44b333ea31 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 11:10:51 -0700 Subject: [PATCH 31/38] Use go-version instead of manual version parsing --- command/agent/agent.go | 30 ++++++------------------------ 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/command/agent/agent.go b/command/agent/agent.go index f20026eb73be..62f1b47cad9a 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -8,13 +8,13 @@ import ( "os" "path/filepath" "runtime" - "strconv" "strings" "sync" "sync/atomic" "time" "github.com/hashicorp/consul/api" + version "github.com/hashicorp/go-version" "github.com/hashicorp/nomad/client" clientconfig "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/command/agent/consul" @@ -704,6 +704,8 @@ func (a *Agent) setupConsul(consulConfig *config.ConsulConfig) error { return nil } +var consulTLSSkipVerifyMinVersion = version.Must(version.NewVersion("0.7.2")) + // consulSupportsTLSSkipVerify returns true if Consul supports TLSSkipVerify. func consulSupportsTLSSkipVerify(self map[string]map[string]interface{}) bool { member, ok := self["Member"] @@ -727,35 +729,15 @@ func consulSupportsTLSSkipVerify(self map[string]map[string]interface{}) bool { return false } parts := strings.SplitN(build, ":", 2) - if len(parts) == 0 { - return false - } - parts = strings.Split(parts[0], ".") - if len(parts) != 3 { - return false - } - major, err := strconv.Atoi(parts[0]) - if err != nil { + if len(parts) != 2 { return false } - minor, err := strconv.Atoi(parts[1]) + v, err := version.NewVersion(parts[0]) if err != nil { return false } - patch, err := strconv.Atoi(parts[2]) - if err != nil { - return false - } - if major > 0 || minor > 7 { - // After 0.7.2! - return true - } - if minor < 7 { - return false - } - if patch < 2 { + if v.LessThan(consulTLSSkipVerifyMinVersion) { return false } - // 0.7.2 or higher! return true } From 86bc2fbf9cb94a454fcfd16b5cbaede79438cf51 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 11:37:57 -0700 Subject: [PATCH 32/38] Fix diff test + bonus upgrade to subtests --- nomad/structs/diff_test.go | 118 ++++++++++++++++++++++--------------- 1 file changed, 70 insertions(+), 48 deletions(-) diff --git a/nomad/structs/diff_test.go b/nomad/structs/diff_test.go index 729f0e1a1513..53a2868dcc65 100644 --- a/nomad/structs/diff_test.go +++ b/nomad/structs/diff_test.go @@ -1912,20 +1912,22 @@ func TestTaskGroupDiff(t *testing.T) { func TestTaskDiff(t *testing.T) { cases := []struct { + Name string Old, New *Task Expected *TaskDiff Error bool Contextual bool }{ { - Old: nil, - New: nil, + Name: "Empty", + Old: nil, + New: nil, Expected: &TaskDiff{ Type: DiffTypeNone, }, }, { - // Primitive only that has different names + Name: "Primitive only that has different names", Old: &Task{ Name: "foo", Meta: map[string]string{ @@ -1941,7 +1943,7 @@ func TestTaskDiff(t *testing.T) { Error: true, }, { - // Primitive only that is the same + Name: "Primitive only that is the same", Old: &Task{ Name: "foo", Driver: "exec", @@ -1974,7 +1976,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Primitive only that has diffs + Name: "Primitive only that has diffs", Old: &Task{ Name: "foo", Driver: "exec", @@ -2045,7 +2047,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Map diff + Name: "Map diff", Old: &Task{ Meta: map[string]string{ "foo": "foo", @@ -2097,7 +2099,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Constraints edited + Name: "Constraints edited", Old: &Task{ Constraints: []*Constraint{ { @@ -2185,8 +2187,8 @@ func TestTaskDiff(t *testing.T) { }, }, { - // LogConfig added - Old: &Task{}, + Name: "LogConfig added", + Old: &Task{}, New: &Task{ LogConfig: &LogConfig{ MaxFiles: 1, @@ -2218,7 +2220,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // LogConfig deleted + Name: "LogConfig deleted", Old: &Task{ LogConfig: &LogConfig{ MaxFiles: 1, @@ -2251,7 +2253,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // LogConfig edited + Name: "LogConfig edited", Old: &Task{ LogConfig: &LogConfig{ MaxFiles: 1, @@ -2289,7 +2291,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // LogConfig edited with context + Name: "LogConfig edited with context", Contextual: true, Old: &Task{ LogConfig: &LogConfig{ @@ -2328,7 +2330,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Artifacts edited + Name: "Artifacts edited", Old: &Task{ Artifacts: []*TaskArtifact{ { @@ -2420,7 +2422,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Resources edited (no networks) + Name: "Resources edited (no networks)", Old: &Task{ Resources: &Resources{ CPU: 100, @@ -2474,7 +2476,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Resources edited (no networks) with context + Name: "Resources edited (no networks) with context", Contextual: true, Old: &Task{ Resources: &Resources{ @@ -2529,7 +2531,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Network Resources edited + Name: "Network Resources edited", Old: &Task{ Resources: &Resources{ Networks: []*NetworkResource{ @@ -2677,7 +2679,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Config same + Name: "Config same", Old: &Task{ Config: map[string]interface{}{ "foo": 1, @@ -2711,7 +2713,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Config edited + Name: "Config edited", Old: &Task{ Config: map[string]interface{}{ "foo": 1, @@ -2795,7 +2797,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Config edited with context + Name: "Config edited with context", Contextual: true, Old: &Task{ Config: map[string]interface{}{ @@ -2892,7 +2894,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Services edited (no checks) + Name: "Services edited (no checks)", Old: &Task{ Services: []*Service{ { @@ -2980,7 +2982,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Services edited (no checks) with context + Name: "Services edited (no checks) with context", Contextual: true, Old: &Task{ Services: []*Service{ @@ -3023,7 +3025,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Service Checks edited + Name: "Service Checks edited", Old: &Task{ Services: []*Service{ { @@ -3155,6 +3157,12 @@ func TestTaskDiff(t *testing.T) { Old: "", New: "http", }, + { + Type: DiffTypeAdded, + Name: "TLSSkipVerify", + Old: "", + New: "false", + }, { Type: DiffTypeAdded, Name: "Timeout", @@ -3203,6 +3211,12 @@ func TestTaskDiff(t *testing.T) { Old: "http", New: "", }, + { + Type: DiffTypeDeleted, + Name: "TLSSkipVerify", + Old: "false", + New: "", + }, { Type: DiffTypeDeleted, Name: "Timeout", @@ -3223,7 +3237,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Service Checks edited with context + Name: "Service Checks edited with context", Contextual: true, Old: &Task{ Services: []*Service{ @@ -3332,6 +3346,12 @@ func TestTaskDiff(t *testing.T) { Old: "http", New: "http", }, + { + Type: DiffTypeNone, + Name: "TLSSkipVerify", + Old: "false", + New: "false", + }, { Type: DiffTypeNone, Name: "Timeout", @@ -3352,8 +3372,8 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Vault added - Old: &Task{}, + Name: "Vault added", + Old: &Task{}, New: &Task{ Vault: &Vault{ Policies: []string{"foo", "bar"}, @@ -3413,7 +3433,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Vault deleted + Name: "Vault deleted", Old: &Task{ Vault: &Vault{ Policies: []string{"foo", "bar"}, @@ -3474,7 +3494,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Vault edited + Name: "Vault edited", Old: &Task{ Vault: &Vault{ Policies: []string{"foo", "bar"}, @@ -3542,7 +3562,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Vault edited with context + Name: "Vault edited with context", Contextual: true, Old: &Task{ Vault: &Vault{ @@ -3617,7 +3637,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Template edited + Name: "Template edited", Old: &Task{ Templates: []*Template{ { @@ -3765,8 +3785,8 @@ func TestTaskDiff(t *testing.T) { }, }, { - // DispatchPayload added - Old: &Task{}, + Name: "DispatchPayload added", + Old: &Task{}, New: &Task{ DispatchPayload: &DispatchPayloadConfig{ File: "foo", @@ -3791,7 +3811,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // DispatchPayload deleted + Name: "DispatchPayload deleted", Old: &Task{ DispatchPayload: &DispatchPayloadConfig{ File: "foo", @@ -3817,7 +3837,7 @@ func TestTaskDiff(t *testing.T) { }, }, { - // Dispatch payload edited + Name: "Dispatch payload edited", Old: &Task{ DispatchPayload: &DispatchPayloadConfig{ File: "foo", @@ -3847,8 +3867,8 @@ func TestTaskDiff(t *testing.T) { }, }, { - // DispatchPayload edited with context. Place holder for if more - // fields are added + // Place holder for if more fields are added + Name: "DispatchPayload edited with context", Contextual: true, Old: &Task{ DispatchPayload: &DispatchPayloadConfig{ @@ -3881,20 +3901,22 @@ func TestTaskDiff(t *testing.T) { } for i, c := range cases { - actual, err := c.Old.Diff(c.New, c.Contextual) - if c.Error && err == nil { - t.Fatalf("case %d: expected errored", i+1) - } else if err != nil { - if !c.Error { - t.Fatalf("case %d: errored %#v", i+1, err) - } else { - continue + t.Run(c.Name, func(t *testing.T) { + actual, err := c.Old.Diff(c.New, c.Contextual) + if c.Error && err == nil { + t.Fatalf("case %d: expected errored", i+1) + } else if err != nil { + if !c.Error { + t.Fatalf("case %d: errored %#v", i+1, err) + } else { + return + } } - } - if !reflect.DeepEqual(actual, c.Expected) { - t.Errorf("case %d: got:\n%#v\n want:\n%#v\n", - i+1, actual, c.Expected) - } + if !reflect.DeepEqual(actual, c.Expected) { + t.Errorf("case %d: got:\n%#v\n want:\n%#v\n", + i+1, actual, c.Expected) + } + }) } } From 7c67166510efb85734fcd682cf174ef0ee90927d Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 11:46:30 -0700 Subject: [PATCH 33/38] Add TLSSkipVerify support to api and parser --- api/tasks.go | 1 + command/agent/job_endpoint.go | 1 + jobspec/parse.go | 1 + 3 files changed, 3 insertions(+) diff --git a/api/tasks.go b/api/tasks.go index fc490b37fb9d..f5f837a0eaa3 100644 --- a/api/tasks.go +++ b/api/tasks.go @@ -90,6 +90,7 @@ type ServiceCheck struct { Interval time.Duration Timeout time.Duration InitialStatus string `mapstructure:"initial_status"` + TLSSkipVerify bool `mapstructure:"tls_skip_verify"` } // The Service model represents a Consul service definition diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index e807ee963476..58fa02b1aa02 100644 --- a/command/agent/job_endpoint.go +++ b/command/agent/job_endpoint.go @@ -552,6 +552,7 @@ func ApiTaskToStructsTask(apiTask *api.Task, structsTask *structs.Task) { Interval: check.Interval, Timeout: check.Timeout, InitialStatus: check.InitialStatus, + TLSSkipVerify: check.TLSSkipVerify, } } } diff --git a/jobspec/parse.go b/jobspec/parse.go index 8a208f9864d8..469ae9a58943 100644 --- a/jobspec/parse.go +++ b/jobspec/parse.go @@ -947,6 +947,7 @@ func parseChecks(service *api.Service, checkObjs *ast.ObjectList) error { "command", "args", "initial_status", + "tls_skip_verify", } if err := checkHCLKeys(co.Val, valid); err != nil { return multierror.Prefix(err, "check ->") From 4cf34edb29f760eef9b944669cbf06525b669de4 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 12:18:06 -0700 Subject: [PATCH 34/38] Skip checks with TLSSkipVerify if it's unsupported Fixes #2218 --- command/agent/agent.go | 2 +- command/agent/consul/client.go | 40 +++++++++++++++--------- command/agent/consul/int_test.go | 2 +- command/agent/consul/unit_test.go | 52 ++++++++++++++++++++++++++++--- 4 files changed, 75 insertions(+), 21 deletions(-) diff --git a/command/agent/agent.go b/command/agent/agent.go index 62f1b47cad9a..8080591e6944 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -699,7 +699,7 @@ func (a *Agent) setupConsul(consulConfig *config.ConsulConfig) error { a.consulCatalog = client.Catalog() // Create Consul Service client for service advertisement and checks. - a.consulService = consul.NewServiceClient(client.Agent(), a.logger) + a.consulService = consul.NewServiceClient(client.Agent(), a.consulSupportsTLSSkipVerify, a.logger) go a.consulService.Run() return nil } diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 246d4f478038..2fd2542e422d 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -89,6 +89,9 @@ type ServiceClient struct { retryInterval time.Duration maxRetryInterval time.Duration + // skipVerifySupport is true if the local Consul agent suppots TLSSkipVerify + skipVerifySupport bool + // exitCh is closed when the main Run loop exits exitCh chan struct{} @@ -115,22 +118,23 @@ type ServiceClient struct { // NewServiceClient creates a new Consul ServiceClient from an existing Consul API // Client and logger. -func NewServiceClient(consulClient AgentAPI, logger *log.Logger) *ServiceClient { +func NewServiceClient(consulClient AgentAPI, skipVerifySupport bool, logger *log.Logger) *ServiceClient { return &ServiceClient{ - client: consulClient, - logger: logger, - retryInterval: defaultRetryInterval, - maxRetryInterval: defaultMaxRetryInterval, - exitCh: make(chan struct{}), - shutdownCh: make(chan struct{}), - shutdownWait: defaultShutdownWait, - opCh: make(chan *operations, 8), - services: make(map[string]*api.AgentServiceRegistration), - checks: make(map[string]*api.AgentCheckRegistration), - scripts: make(map[string]*scriptCheck), - runningScripts: make(map[string]*scriptHandle), - agentServices: make(map[string]struct{}), - agentChecks: make(map[string]struct{}), + client: consulClient, + skipVerifySupport: skipVerifySupport, + logger: logger, + retryInterval: defaultRetryInterval, + maxRetryInterval: defaultMaxRetryInterval, + exitCh: make(chan struct{}), + shutdownCh: make(chan struct{}), + shutdownWait: defaultShutdownWait, + opCh: make(chan *operations, 8), + services: make(map[string]*api.AgentServiceRegistration), + checks: make(map[string]*api.AgentCheckRegistration), + scripts: make(map[string]*scriptCheck), + runningScripts: make(map[string]*scriptHandle), + agentServices: make(map[string]struct{}), + agentChecks: make(map[string]struct{}), } } @@ -432,6 +436,11 @@ func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *st ops.regServices = append(ops.regServices, serviceReg) for _, check := range service.Checks { + if check.TLSSkipVerify && !c.skipVerifySupport { + c.logger.Printf("[WARN] consul.sync: skipping check %q for task %q alloc %q because Consul doesn't support tls_skip_verify. Please upgrade to Consul >= 0.7.2.", + check.Name, task.Name, allocID) + continue + } checkID := createCheckID(id, check) if check.Type == structs.ServiceCheckScript { if exec == nil { @@ -441,6 +450,7 @@ func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *st allocID, task.Name, checkID, check, exec, c.client, c.logger, c.shutdownCh)) } + host, port := serviceReg.Address, serviceReg.Port if check.PortLabel != "" { host, port = task.FindHostAndPortFor(check.PortLabel) diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go index 8a5635e179e6..eb64cf760967 100644 --- a/command/agent/consul/int_test.go +++ b/command/agent/consul/int_test.go @@ -122,7 +122,7 @@ func TestConsul_Integration(t *testing.T) { if err != nil { t.Fatalf("error creating consul client: %v", err) } - serviceClient := consul.NewServiceClient(consulClient.Agent(), logger) + serviceClient := consul.NewServiceClient(consulClient.Agent(), true, logger) defer serviceClient.Shutdown() // just-in-case cleanup consulRan := make(chan struct{}) go func() { diff --git a/command/agent/consul/unit_test.go b/command/agent/consul/unit_test.go index 622b4628aedb..96794e0c1b86 100644 --- a/command/agent/consul/unit_test.go +++ b/command/agent/consul/unit_test.go @@ -98,7 +98,7 @@ func (t *testFakeCtx) syncOnce() error { func setupFake() *testFakeCtx { fc := newFakeConsul() return &testFakeCtx{ - ServiceClient: NewServiceClient(fc, testLogger()), + ServiceClient: NewServiceClient(fc, true, testLogger()), FakeConsul: fc, Task: testTask(), execs: make(chan int, 100), @@ -445,7 +445,6 @@ func TestConsul_ChangePorts(t *testing.T) { // TestConsul_RegServices tests basic service registration. func TestConsul_RegServices(t *testing.T) { ctx := setupFake() - port := ctx.Task.Resources.Networks[0].DynamicPorts[0].Value if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { t.Fatalf("unexpected error registering task: %v", err) @@ -465,8 +464,8 @@ func TestConsul_RegServices(t *testing.T) { if !reflect.DeepEqual(v.Tags, ctx.Task.Services[0].Tags) { t.Errorf("expected Tags=%v != %v", ctx.Task.Services[0].Tags, v.Tags) } - if v.Port != port { - t.Errorf("expected Port=%d != %d", port, v.Port) + if v.Port != xPort { + t.Errorf("expected Port=%d != %d", xPort, v.Port) } } @@ -723,3 +722,48 @@ func TestConsul_ShutdownBlocked(t *testing.T) { } } } + +// TestConsul_NoTLSSkipVerifySupport asserts that checks with +// TLSSkipVerify=true are skipped when Consul doesn't support TLSSkipVerify. +func TestConsul_NoTLSSkipVerifySupport(t *testing.T) { + ctx := setupFake() + ctx.ServiceClient = NewServiceClient(ctx.FakeConsul, false, testLogger()) + ctx.Task.Services[0].Checks = []*structs.ServiceCheck{ + // This check sets TLSSkipVerify so it should get dropped + { + Name: "tls-check-skip", + Type: "http", + Protocol: "https", + Path: "/", + TLSSkipVerify: true, + }, + // This check doesn't set TLSSkipVerify so it should work fine + { + Name: "tls-check-noskip", + Type: "http", + Protocol: "https", + Path: "/", + TLSSkipVerify: false, + }, + } + + if err := ctx.ServiceClient.RegisterTask("allocid", ctx.Task, nil); err != nil { + t.Fatalf("unexpected error registering task: %v", err) + } + + if err := ctx.syncOnce(); err != nil { + t.Fatalf("unexpected error syncing task: %v", err) + } + + if len(ctx.FakeConsul.checks) != 1 { + t.Errorf("expected 1 check but found %d", len(ctx.FakeConsul.checks)) + } + for _, v := range ctx.FakeConsul.checks { + if expected := "tls-check-noskip"; v.Name != expected { + t.Errorf("only expected %q but found: %q", expected, v.Name) + } + if v.TLSSkipVerify { + t.Errorf("TLSSkipVerify=true when TLSSkipVerify not supported!") + } + } +} From a5dcf6bbd9b9e76fe7b0f58e1730885e1b002d91 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 12:27:07 -0700 Subject: [PATCH 35/38] Document `tls_skip_verify` --- website/source/docs/http/json-jobs.html.md | 5 ++++- website/source/docs/job-specification/service.html.md | 3 +++ website/source/docs/service-discovery/index.html.md | 2 ++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/website/source/docs/http/json-jobs.html.md b/website/source/docs/http/json-jobs.html.md index f5f1db24dc92..69c8f34ee412 100644 --- a/website/source/docs/http/json-jobs.html.md +++ b/website/source/docs/http/json-jobs.html.md @@ -423,7 +423,7 @@ The `Task` object supports the following keys: * `Timeout`: This indicates how long Consul will wait for a health check query to succeed. - * `Path`:The path of the http endpoint which Consul will query to query + * `Path`: The path of the http endpoint which Consul will query to query the health of a service if the type of the check is `http`. Nomad will add the IP of the service and the port, users are only required to add the relative URL of the health check endpoint. @@ -437,6 +437,9 @@ The `Task` object supports the following keys: * `Args`: Additional arguments to the `command` for script based health checks. + * `TLSSkipVerify`: If true, Consul will not attempt to verify the + certificate when performing HTTPS checks. Requires Consul >= 0.7.2. + * `Templates` - Specifies the set of [`Template`](#template) objects to render for the task. Templates can be used to inject both static and dynamic configuration with data populated from environment variables, Consul and Vault. diff --git a/website/source/docs/job-specification/service.html.md b/website/source/docs/job-specification/service.html.md index 2973142da84a..f213ae357429 100644 --- a/website/source/docs/job-specification/service.html.md +++ b/website/source/docs/job-specification/service.html.md @@ -143,6 +143,9 @@ does not automatically enable service discovery. - `type` `(string: )` - This indicates the check types supported by Nomad. Valid options are `script`, `http`, and `tcp`. +- `tls_skip_verify` `(bool: false)` - Skip verifying TLS certificates for HTTPS + checks. Requires Consul >= 0.7.2. + ## `service` Examples diff --git a/website/source/docs/service-discovery/index.html.md b/website/source/docs/service-discovery/index.html.md index ce07cc8d35ed..a95cd09dca2c 100644 --- a/website/source/docs/service-discovery/index.html.md +++ b/website/source/docs/service-discovery/index.html.md @@ -32,6 +32,8 @@ To configure a job to register with service discovery, please see the ## Assumptions +- Consul 0.7.2 or later is needed for `tls_skip_verify` in HTTP checks. + - Consul 0.6.4 or later is needed for using the Script checks. - Consul 0.6.0 or later is needed for using the TCP checks. From b9ea2765ecb0372037b1dff24f1ad89c8a75950f Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 13:05:41 -0700 Subject: [PATCH 36/38] Thanks go vet! --- client/consul_test.go | 2 +- command/agent/consul/script.go | 1 + command/agent/consul/script_test.go | 6 +++--- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/client/consul_test.go b/client/consul_test.go index 7d68faae2dff..4c34fd9a3e0f 100644 --- a/client/consul_test.go +++ b/client/consul_test.go @@ -55,7 +55,7 @@ func newMockConsulServiceClient() *mockConsulServiceClient { func (m *mockConsulServiceClient) UpdateTask(allocID string, old, new *structs.Task, exec driver.ScriptExecutor) error { m.mu.Lock() defer m.mu.Unlock() - m.logger.Printf("[TEST] mock_consul: UpdateTask(%q, %q, %q, %T)", allocID, old, new, exec) + m.logger.Printf("[TEST] mock_consul: UpdateTask(%q, %v, %v, %T)", allocID, old, new, exec) m.ops = append(m.ops, newMockConsulOp("update", allocID, old, exec)) return nil } diff --git a/command/agent/consul/script.go b/command/agent/consul/script.go index 9f540d7fa4cd..379025f76f78 100644 --- a/command/agent/consul/script.go +++ b/command/agent/consul/script.go @@ -96,6 +96,7 @@ func (s *scriptCheck) run() *scriptHandle { switch execctx.Err() { case context.Canceled: // check removed during execution; exit + cancel() return case context.DeadlineExceeded: metrics.IncrCounter([]string{"client", "consul", "script_timeouts"}, 1) diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go index 35902c9f8b47..53aed1f8d2e3 100644 --- a/command/agent/consul/script_test.go +++ b/command/agent/consul/script_test.go @@ -119,7 +119,7 @@ func TestConsulScript_Exec_Timeout(t *testing.T) { select { case update := <-hb.updates: if update.status != api.HealthCritical { - t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) + t.Errorf("expected %q due to timeout but received %q", api.HealthCritical, update) } case <-time.After(3 * time.Second): t.Fatalf("timed out waiting for script check to exit") @@ -167,7 +167,7 @@ func TestConsulScript_Exec_TimeoutCritical(t *testing.T) { select { case update := <-hb.updates: if update.status != api.HealthCritical { - t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) + t.Errorf("expected %q due to timeout but received %q", api.HealthCritical, update) } if update.output != context.DeadlineExceeded.Error() { t.Errorf("expected output=%q but found: %q", context.DeadlineExceeded.Error(), update.output) @@ -214,7 +214,7 @@ func TestConsulScript_Exec_Shutdown(t *testing.T) { select { case update := <-hb.updates: if update.status != api.HealthPassing { - t.Error("expected %q due to timeout but received %q", api.HealthCritical, update) + t.Errorf("expected %q due to timeout but received %q", api.HealthCritical, update) } case <-time.After(3 * time.Second): t.Fatalf("timed out waiting for script check to exit") From fb3b30bba98140aeb72fe7a5ff8e413a644caac1 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 13:16:48 -0700 Subject: [PATCH 37/38] Fix Windows build. --- client/driver/utils_windows.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/client/driver/utils_windows.go b/client/driver/utils_windows.go index 5b2b7d8423b4..ff56089d104d 100644 --- a/client/driver/utils_windows.go +++ b/client/driver/utils_windows.go @@ -7,3 +7,7 @@ import ( // TODO Figure out if this is needed in Wondows func isolateCommand(cmd *exec.Cmd) { } + +// setChroot is a noop on Windows +func setChroot(cmd *exec.Cmd, chroot string) { +} From 58430bf0626f5dd151b09d637b959885f8df72b6 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Wed, 19 Apr 2017 16:14:11 -0700 Subject: [PATCH 38/38] Fix consul test build on Windows --- command/agent/consul/int_test.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go index eb64cf760967..9243b5822e1f 100644 --- a/command/agent/consul/int_test.go +++ b/command/agent/consul/int_test.go @@ -4,12 +4,12 @@ import ( "io/ioutil" "log" "os" + "os/user" "path/filepath" + "runtime" "testing" "time" - "golang.org/x/sys/unix" - consulapi "github.com/hashicorp/consul/api" "github.com/hashicorp/consul/testutil" "github.com/hashicorp/nomad/client" @@ -38,8 +38,10 @@ func TestConsul_Integration(t *testing.T) { if testing.Short() { t.Skip("-short set; skipping") } - if unix.Geteuid() != 0 { - t.Skip("Must be run as root") + if runtime.GOOS != "windows" { + if u, err := user.Current(); err == nil && u.Uid != "0" { + t.Skip("Must be run as root") + } } // Create an embedded Consul server testconsul := testutil.NewTestServerConfig(t, func(c *testutil.TestServerConfig) {