Merge pull request #2467 from hashicorp/f-consul-refactor

Refactor Consul Syncer into new ServiceClient
This commit is contained in:
Michael Schurter 2017-04-19 16:55:49 -07:00 committed by GitHub
commit 033fe839b0
54 changed files with 3228 additions and 2567 deletions

View File

@ -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

View File

@ -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()

View File

@ -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

View File

@ -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 {

View File

@ -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

View File

@ -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)
}

14
client/consul.go Normal file
View File

@ -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
}

76
client/consul_test.go Normal file
View File

@ -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))
}

View File

@ -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)

View File

@ -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

View File

@ -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)

View File

@ -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)
}
}

View File

@ -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
}

View File

@ -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)
}
}

View File

@ -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})
}

View File

@ -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 = "/"
}

View File

@ -1,8 +0,0 @@
// +build windows
package executor
import "os/exec"
func (e *ExecScriptCheck) setChroot(cmd *exec.Cmd) {
}

View File

@ -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() {

View File

@ -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)

View File

@ -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 {

View File

@ -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()

View File

@ -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 {

View File

@ -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

View File

@ -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()

View File

@ -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 {

View File

@ -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)
}
}

View File

@ -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 {

View File

@ -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)
}
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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) {
}

View File

@ -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 {

View File

@ -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)

View File

@ -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
}

View File

@ -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"
}}`)
}

View File

@ -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
}

View File

@ -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)
}
}
}

View File

@ -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
}

View File

@ -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)
}

View File

@ -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)
}
}

View File

@ -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}
}

View File

@ -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))
}

File diff suppressed because it is too large Load Diff

View File

@ -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)
}
}

View File

@ -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!")
}
}
}

View File

@ -552,6 +552,7 @@ func ApiTaskToStructsTask(apiTask *api.Task, structsTask *structs.Task) {
Interval: check.Interval,
Timeout: check.Timeout,
InitialStatus: check.InitialStatus,
TLSSkipVerify: check.TLSSkipVerify,
}
}
}

View File

@ -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 ->")

View File

@ -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
}

View File

@ -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)
}

View File

@ -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)
}
})
}
}

View File

@ -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))
}

View File

@ -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.

View File

@ -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: <enum>)` - Specifies the originating status of the
@ -143,6 +143,9 @@ does not automatically enable service discovery.
- `type` `(string: <required>)` - 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

View File

@ -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.