diff --git a/api/tasks.go b/api/tasks.go index fc490b37fb9..f5f837a0eaa 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/client/alloc_runner.go b/client/alloc_runner.go index 719a84d4a67..8ca6f671eae 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 6bac5e4051b..f1bc828b807 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 dd665926112..749f5da1793 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 7183c939f1d..89e14823b85 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 0e79c77e788..abf857da4fc 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 00000000000..043a17bdb16 --- /dev/null +++ b/client/consul.go @@ -0,0 +1,14 @@ +package client + +import ( + "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 driver.ScriptExecutor) error + RemoveTask(allocID string, task *structs.Task) + UpdateTask(allocID string, existing, newTask *structs.Task, exec driver.ScriptExecutor) error +} diff --git a/client/consul_test.go b/client/consul_test.go new file mode 100644 index 00000000000..4c34fd9a3e0 --- /dev/null +++ b/client/consul_test.go @@ -0,0 +1,76 @@ +package client + +import ( + "fmt" + "io/ioutil" + "log" + "os" + "sync" + "testing" + + "github.com/hashicorp/nomad/client/driver" + "github.com/hashicorp/nomad/nomad/structs" +) + +// 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 { + ops []mockConsulOp + mu sync.Mutex + + logger *log.Logger +} + +func newMockConsulServiceClient() *mockConsulServiceClient { + m := mockConsulServiceClient{ + ops: make([]mockConsulOp, 0, 20), + logger: log.New(ioutil.Discard, "", 0), + } + if testing.Verbose() { + m.logger = log.New(os.Stderr, "", log.LstdFlags) + } + return &m +} + +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, %v, %v, %T)", allocID, old, new, exec) + m.ops = append(m.ops, newMockConsulOp("update", allocID, old, 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.ops = append(m.ops, newMockConsulOp("add", allocID, task, exec)) + return nil +} + +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.ops = append(m.ops, newMockConsulOp("remove", allocID, task, nil)) +} diff --git a/client/driver/docker.go b/client/driver/docker.go index b045e3f3bba..bea4a350134 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" @@ -427,6 +429,7 @@ func (d *DockerDriver) Validate(config map[string]interface{}) error { func (d *DockerDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } @@ -564,9 +567,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 +1227,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 +1269,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 +1364,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.go b/client/driver/driver.go index a89db81679f..ee28888cfad 100644 --- a/client/driver/driver.go +++ b/client/driver/driver.go @@ -1,6 +1,7 @@ package driver import ( + "context" "errors" "fmt" "log" @@ -191,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. @@ -254,6 +259,16 @@ 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 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) } // ExecContext is a task's execution context diff --git a/client/driver/exec.go b/client/driver/exec.go index d94c82443ff..bc6ee3aee25 100644 --- a/client/driver/exec.go +++ b/client/driver/exec.go @@ -1,6 +1,7 @@ package driver import ( + "context" "encoding/json" "fmt" "log" @@ -85,6 +86,7 @@ func (d *ExecDriver) Validate(config map[string]interface{}) error { func (d *ExecDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } @@ -163,9 +165,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 +222,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 +258,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 +309,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 85976df1aa7..6a580f24352 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" @@ -280,3 +282,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.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.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 de93146ccdf..00000000000 --- 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 3affd0e08f3..00000000000 --- 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 9533026fd71..00000000000 --- 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 b18812dd8d0..00000000000 --- 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 a35c2722d60..00000000000 --- 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 53b9c37f581..90797fbefa2 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 8508233d6c8..51325e09a9e 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 17f40ac25d4..7c4074feffc 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 4a90b4efad6..c215e6882d5 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 @@ -106,6 +108,7 @@ func (d *JavaDriver) Validate(config map[string]interface{}) error { func (d *JavaDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } @@ -284,6 +287,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 +295,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 +307,7 @@ type javaId struct { MaxKillTimeout time.Duration PluginConfig *PluginReattachConfig IsolationConfig *dstructs.IsolationConfig + TaskDir string UserPid int } @@ -352,10 +354,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 +366,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 +389,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 +439,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/lxc.go b/client/driver/lxc.go index 0d369d34cd0..ac7d3c298a3 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 8f34297e513..bdaff3f703c 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" @@ -75,6 +76,7 @@ func NewMockDriver(ctx *DriverContext) Driver { func (d *MockDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: false, + Exec: true, } } @@ -234,6 +236,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 ad90c80383c..856b2c02316 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, } } @@ -273,10 +275,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 +320,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 } @@ -360,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") } @@ -402,11 +401,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 1500cf00f01..5e602f47bb1 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" @@ -79,6 +80,7 @@ func (d *RawExecDriver) Validate(config map[string]interface{}) error { func (d *RawExecDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: true, + Exec: true, } } @@ -164,9 +166,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 +213,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 +246,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 +289,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 7b96c2218cd..e1bd4d36261 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" @@ -298,3 +300,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.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.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 55e2a026c9d..a8cff3c0249 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 @@ -158,6 +165,7 @@ func (d *RktDriver) Validate(config map[string]interface{}) error { func (d *RktDriver) Abilities() DriverAbilities { return DriverAbilities{ SendSignals: false, + Exec: true, } } @@ -229,7 +237,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 +261,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 +452,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 +483,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 +513,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 +523,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 +530,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 +556,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 +597,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 b54160becdc..b61ba04af79 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" @@ -489,3 +491,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.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.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 7e1c79890bb..def668548ff 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 474cdcf17f1..397641e3e79 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/driver/utils_windows.go b/client/driver/utils_windows.go index 5b2b7d8423b..ff56089d104 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) { +} diff --git a/client/task_runner.go b/client/task_runner.go index 55176c0e705..c578ca5ea0b 100644 --- a/client/task_runner.go +++ b/client/task_runner.go @@ -61,6 +61,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 +174,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 +196,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 +291,15 @@ func (r *TaskRunner) RestoreState() error { r.task.Name, r.alloc.ID, err) return nil } + + 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) + } + r.handleLock.Lock() r.handle = handle r.handleLock.Unlock() @@ -1045,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() @@ -1061,8 +1073,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) @@ -1121,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): @@ -1167,8 +1186,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) @@ -1217,12 +1240,57 @@ 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() + return nil } +// 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 + } + 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 +// 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 +1403,24 @@ 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 r.handleLock.Lock() if r.handle != nil { + 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)) } + + 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)) + } } r.handleLock.Unlock() @@ -1356,14 +1435,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 { diff --git a/client/task_runner_test.go b/client/task_runner_test.go index 6baa14f9734..ede8cb1647c 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) @@ -465,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, @@ -525,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) diff --git a/command/agent/agent.go b/command/agent/agent.go index 142dfe2e34e..8080591e694 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -8,17 +8,19 @@ 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" "github.com/hashicorp/nomad/nomad" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/structs/config" ) const ( @@ -30,6 +32,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 +48,16 @@ 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 + + // consulSupportsTLSSkipVerify flags whether or not Nomad can register + // checks with TLSSkipVerify + consulSupportsTLSSkipVerify bool client *client.Client @@ -63,8 +77,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 +90,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 +344,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) } @@ -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,15 +419,14 @@ 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, + httpServ, } - if !conf.TLSConfig.EnableHTTP { - consulServices[consul.GenerateServiceKey(httpServ)] = httpServ + if err := a.consulService.RegisterAgent(consulRoleServer, consulServices); err != nil { + return err } - a.consulSyncer.SetServices(consul.ServerDomain, consulServices) } return nil @@ -462,7 +476,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) } @@ -475,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, @@ -494,10 +506,18 @@ func (a *Agent) setupClient() error { }, }, } - if !conf.TLSConfig.EnableHTTP { - a.consulSyncer.SetServices(consul.ClientDomain, map[consul.ServiceKey]*structs.Service{ - consul.GenerateServiceKey(httpServ): httpServ, - }) + 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 } } @@ -612,8 +632,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 +679,65 @@ 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 - } + // Determine version for TLSSkipVerify + if self, err := client.Agent().Self(); err != nil { + a.consulSupportsTLSSkipVerify = consulSupportsTLSSkipVerify(self) + } - // 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.consulSupportsTLSSkipVerify, a.logger) + go a.consulService.Run() 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"] + 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) != 2 { + return false + } + v, err := version.NewVersion(parts[0]) + if err != nil { + return false + } + if v.LessThan(consulTLSSkipVerifyMinVersion) { + return false + } + return true +} diff --git a/command/agent/agent_test.go b/command/agent/agent_test.go index 89db490d7a7..24d45ee8d1e 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/catalog_testing.go b/command/agent/consul/catalog_testing.go new file mode 100644 index 00000000000..f0dd0326ce0 --- /dev/null +++ b/command/agent/consul/catalog_testing.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/chaos_test.go b/command/agent/consul/chaos_test.go deleted file mode 100644 index 89b69ea2cb6..00000000000 --- 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 551f94b6fc3..00000000000 --- 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 00000000000..2fd2542e422 --- /dev/null +++ b/command/agent/consul/client.go @@ -0,0 +1,706 @@ +package consul + +import ( + "fmt" + "log" + "net" + "net/url" + "strconv" + "strings" + "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" +) + +const ( + // nomadServicePrefix is the first prefix that scopes all Nomad registered + // services + nomadServicePrefix = "_nomad" + + // 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 + 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" +) + +// 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 { + 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 + ServiceDeregister(serviceID string) error + 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 + 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{} + + // 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 + + opCh chan *operations + + services map[string]*api.AgentServiceRegistration + checks map[string]*api.AgentCheckRegistration + scripts map[string]*scriptCheck + runningScripts map[string]*scriptHandle + + // 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 sync.Mutex +} + +// NewServiceClient creates a new Consul ServiceClient from an existing Consul API +// Client and logger. +func NewServiceClient(consulClient AgentAPI, skipVerifySupport bool, logger *log.Logger) *ServiceClient { + return &ServiceClient{ + 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{}), + } +} + +// Run the Consul main loop which retries operations against Consul. It should +// be called exactly once. +func (c *ServiceClient) Run() { + 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) + } + + if err := c.sync(); err != nil { + if failures == 0 { + c.logger.Printf("[WARN] consul.sync: failed to update services in Consul: %v", err) + } + 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: + } + } + 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.sync: successfully updated services in Consul") + failures = 0 + } + } + + select { + 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 + } + return + default: + } + + } +} + +// commit operations unless already shutting down. +func (c *ServiceClient) commit(ops *operations) { + select { + case c.opCh <- ops: + case <-c.shutdownCh: + } +} + +// 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) + } + 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. +func (c *ServiceClient) sync() error { + sreg, creg, sdereg, cdereg := 0, 0, 0, 0 + + 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) + } + + // Remove Nomad services in Consul but unknown locally + for id := range consulServices { + if _, ok := c.services[id]; ok { + // Known service, skip + continue + } + if !isNomadService(id) { + // Not managed by Nomad, skip + continue + } + // 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 + // need updating + portsChanged := make(map[string]struct{}, len(c.services)) + + // Add Nomad services missing from Consul + 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(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 + for id, check := range consulChecks { + if _, ok := c.checks[id]; ok { + // Known check, leave it + continue + } + if !isNomadService(check.ServiceID) { + // Not managed by Nomad, skip + continue + } + // 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 + for id, check := range c.checks { + 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 { + 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 { + // If it's already running, cancel and replace + if oldScript, running := c.runningScripts[id]; running { + oldScript.cancel() + } + // Start and store the handle + c.runningScripts[id] = script.run() + } + } + + c.logger.Printf("[DEBUG] consul.sync: registered %d services, %d checks; deregistered %d services, %d checks", + sreg, creg, sdereg, cdereg) + return nil +} + +// 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 { + ops := operations{} + + 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) + } + 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, + } + ops.regServices = append(ops.regServices, 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 != "" { + // 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) + } + 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) + } + ops.regChecks = append(ops.regChecks, checkReg) + } + } + + // 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 + for _, id := range ops.regServices { + c.agentServices[id.ID] = struct{}{} + } + for _, id := range ops.regChecks { + c.agentChecks[id.ID] = struct{}{} + } + 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, + exec driver.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 = 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 { + 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 fmt.Errorf("failed to add check %q: %v", check.Name, err) + } + ops.regChecks = append(ops.regChecks, checkReg) + } + 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 driver.ScriptExecutor) error { + ops := &operations{} + for _, service := range task.Services { + if err := c.serviceRegs(ops, allocID, service, exec, task); err != nil { + return err + } + } + 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 driver.ScriptExecutor) error { + ops := &operations{} + + existingIDs := make(map[string]*structs.Service, len(existing.Services)) + for _, s := range existing.Services { + existingIDs[makeTaskServiceID(allocID, existing.Name, s)] = s + } + newIDs := make(map[string]*structs.Service, len(newTask.Services)) + for _, s := range newTask.Services { + newIDs[makeTaskServiceID(allocID, newTask.Name, s)] = s + } + + // 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 = append(ops.deregServices, existingID) + for _, check := range existingSvc.Checks { + ops.deregChecks = append(ops.deregChecks, createCheckID(existingID, check)) + } + continue + } + + 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)) + for _, check := range existingSvc.Checks { + existingChecks[createCheckID(existingID, check)] = struct{}{} + } + + // Register new checks + for _, check := range newSvc.Checks { + checkID := createCheckID(existingID, check) + if _, exists := existingChecks[checkID]; exists { + // Check exists, so don't remove it + delete(existingChecks, checkID) + } + } + + // 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 + for _, newSvc := range newIDs { + err := c.serviceRegs(ops, allocID, newSvc, exec, newTask) + if err != nil { + return err + } + } + + c.commit(ops) + 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) { + ops := operations{} + + for _, service := range task.Services { + id := makeTaskServiceID(allocID, task.Name, service) + ops.deregServices = append(ops.deregServices, id) + + for _, check := range service.Checks { + ops.deregChecks = append(ops.deregChecks, createCheckID(id, check)) + } + } + + // Now add them to the deregistration fields; main Run loop will update + c.commit(&ops) +} + +// Shutdown the Consul client. Update running task registations and deregister +// 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: + } + + // Deregister Nomad agent Consul entries before closing shutdown. + ops := operations{} + for id := range c.agentServices { + ops.deregServices = append(ops.deregServices, id) + } + for id := range c.agentChecks { + ops.deregChecks = append(ops.deregChecks, id) + } + 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) + + // Wait for Run to finish any outstanding operations and exit + select { + case <-c.exitCh: + case <-deadline: + // Don't wait forever though + 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) + for _, h := range c.runningScripts { + select { + case <-h.wait(): + case <-deadline: + return fmt.Errorf("timed out waiting for script checks to run") + } + } + return nil +} + +// 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. +// +// 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, + 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" + } + if check.TLSSkipVerify { + chkReg.TLSSkipVerify = true + } + 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 +} + +// 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/int_test.go b/command/agent/consul/int_test.go new file mode 100644 index 00000000000..9243b5822e1 --- /dev/null +++ b/command/agent/consul/int_test.go @@ -0,0 +1,230 @@ +package consul_test + +import ( + "io/ioutil" + "log" + "os" + "os/user" + "path/filepath" + "runtime" + "testing" + "time" + + 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 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) { + // 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(), true, 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() { + // Make sure we always shutdown task runner when the test exits + 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 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 { + 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 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) + } + } + 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/script.go b/command/agent/consul/script.go new file mode 100644 index 00000000000..379025f76f7 --- /dev/null +++ b/command/agent/consul/script.go @@ -0,0 +1,169 @@ +package consul + +import ( + "context" + "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" +) + +// heartbeater is the subset of consul agent functionality needed by script +// checks to heartbeat +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() + exitCh chan struct{} +} + +// wait returns a chan that's closed when the script exits +func (s *scriptHandle) wait() <-chan struct{} { + return s.exitCh +} + +// 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 + agent heartbeater + + // lastCheckOk is true if the last check was ok; otherwise false + lastCheckOk bool + + logger *log.Logger + shutdownCh <-chan struct{} +} + +// newScriptCheck creates a new scriptCheck. run() should be called once the +// initial check is registered with Consul. +func newScriptCheck(allocID, taskName, checkID string, check *structs.ServiceCheck, + exec driver.ScriptExecutor, agent heartbeater, logger *log.Logger, + shutdownCh <-chan struct{}) *scriptCheck { + + return &scriptCheck{ + allocID: allocID, + taskName: taskName, + id: checkID, + 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()) + exitCh := make(chan struct{}) + go func() { + defer close(exitCh) + 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) + } + metrics.IncrCounter([]string{"client", "consul", "script_runs"}, 1) + + // 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 + cancel() + 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 + } + + // 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) + } + + // cleanup context + cancel() + + state := api.HealthCritical + switch code { + case 0: + state = api.HealthPassing + case 1: + state = api.HealthWarning + } + + var outputMsg string + if err != nil { + state = api.HealthCritical + outputMsg = err.Error() + } else { + outputMsg = string(output) + } + + // Actually heartbeat the check + err = s.agent.UpdateTTL(s.id, outputMsg, state) + select { + case <-ctx.Done(): + // check has been removed; don't report errors + return + default: + } + + if err != nil { + if s.lastCheckOk { + s.lastCheckOk = false + 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 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 task %q alloc %q check %q succeeded", + s.taskName, s.allocID, s.check.Name) + } + + select { + case <-s.shutdownCh: + // We've been told to exit and just heartbeated so exit + return + default: + } + } + }() + return &scriptHandle{cancel: cancel, exitCh: exitCh} +} diff --git a/command/agent/consul/script_test.go b/command/agent/consul/script_test.go new file mode 100644 index 00000000000..53aed1f8d2e --- /dev/null +++ b/command/agent/consul/script_test.go @@ -0,0 +1,279 @@ +package consul + +import ( + "context" + "fmt" + "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 { + // 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 <- struct{}{} + cmd := exec.CommandContext(ctx, testtask.Path(), "sleep", "9000h") + 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("allocid", "testtask", "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 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 execStatus +} + +func (f *fakeHeartbeater) UpdateTTL(checkID, output, status string) error { + f.updates <- execStatus{checkID: checkID, output: output, status: status} + return nil +} + +func newFakeHeartbeater() *fakeHeartbeater { + return &fakeHeartbeater{updates: make(chan execStatus)} +} + +// 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("allocid", "testtask", "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.status != api.HealthCritical { + 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") + } + 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") + } +} + +// 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.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) + } + 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 + err error +} + +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 +// 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{}) + exec := newSimpleExec(0, nil) + check := newScriptCheck("allocid", "testtask", "checkid", &serviceCheck, exec, 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.status != api.HealthPassing { + 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") + } + + select { + case <-handle.wait(): + // ok! + case <-time.After(3 * time.Second): + 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) func(t *testing.T) { + return func(t *testing.T) { + t.Parallel() + 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 + 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") + 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)) +} diff --git a/command/agent/consul/syncer.go b/command/agent/consul/syncer.go deleted file mode 100644 index c111f3aafcb..00000000000 --- 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 42879ca7783..00000000000 --- 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 00000000000..96794e0c1b8 --- /dev/null +++ b/command/agent/consul/unit_test.go @@ -0,0 +1,769 @@ +package consul + +import ( + "context" + "fmt" + "io/ioutil" + "log" + "os" + "reflect" + "sync" + "testing" + "time" + + "github.com/hashicorp/consul/api" + "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) + } + 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: xPort}, + {Label: "y", Value: yPort}, + }, + }, + }, + }, + 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 + + // 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 + } + 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 := newFakeConsul() + return &testFakeCtx{ + ServiceClient: NewServiceClient(fc, true, testLogger()), + FakeConsul: fc, + Task: testTask(), + execs: make(chan int, 100), + } +} + +// 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 + + // 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 { + 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) + } + + 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) + } + + 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) + } + } + + origTask := ctx.Task + ctx.Task = testTask() + ctx.Task.Services[0].Tags[0] = "newtag" + if err := ctx.ServiceClient.UpdateTask("allocid", origTask, 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 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_ChangePorts asserts that changing the ports on a service updates +// 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() + 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() + + 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 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 != xPort { + t.Errorf("expected Port=%d != %d", xPort, 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.syncOnce(); 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.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 _, 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", + }, + } + + 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) + } + + 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 scripts finish + if err := ctx.ServiceClient.Shutdown(); err != nil { + t.Errorf("unexpected error shutting down client: %v", err) + } + + // 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 + 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 + } + + // Make shutdown wait time just a bit longer than ctx.Exec takes + ctx.ServiceClient.shutdownWait = 3 * time.Second + + 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) + } + + // 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 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 + } + + // Use a short shutdown deadline since we're intentionally blocking forever + ctx.ServiceClient.shutdownWait = time.Second + + 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() + if err == nil { + t.Errorf("expected a timed out error from shutdown") + } + + // 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) + } + + // 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) + } + } +} + +// 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!") + } + } +} diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index e807ee96347..58fa02b1aa0 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 8a208f9864d..469ae9a5894 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 ->") diff --git a/nomad/server.go b/nomad/server.go index efed68cebdf..14355a871ce 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 = 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 = 9 * 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 902498a1d4a..719bfbf6259 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/nomad/structs/diff_test.go b/nomad/structs/diff_test.go index 729f0e1a151..53a2868dcc6 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) + } + }) } } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index f04a525f071..49196b41bc5 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)) } diff --git a/website/source/docs/http/json-jobs.html.md b/website/source/docs/http/json-jobs.html.md index f5f1db24dc9..69c8f34ee41 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 ee6a1b6b449..f213ae35742 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 @@ -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 ce07cc8d35e..a95cd09dca2 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.