Merge pull request #4792 from hashicorp/r-clientv2-rebased

AllocRunner v2 Feature Branch PR
This commit is contained in:
Michael Schurter 2018-10-16 19:06:49 -05:00 committed by GitHub
commit 2dd3c67e31
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
934 changed files with 98233 additions and 17185 deletions

View File

@ -1,11 +1,15 @@
## 0.9.0 (Unreleased)
__BACKWARDS INCOMPATIBILITIES:__
* core: Switch to structured logging using [go-hclog](https://github.com/hashicorp/go-hclog)
IMPROVEMENTS:
* core: Added advertise address to client node meta data [[GH-4390](https://github.com/hashicorp/nomad/issues/4390)]
* core: Added support for specifying node affinities. Affinities allow job operators to specify weighted placement preferences
according to different node attributes [[GH-4512](https://github.com/hashicorp/nomad/issues/4512)]
* core: Added support for spreading allocations across a specific attribute. Operators can specify spread
target percentages across failure domains such as datacenter or rack [[GH-4512](https://github.com/hashicorp/nomad/issues/4512)]
* client: Refactor client to support plugins and improve state handling [[GH-4792](https://github.com/hashicorp/nomad/pull/4792)]
* client: Extend timeout to 60 seconds for Windows CPU fingerprinting [[GH-4441](https://github.com/hashicorp/nomad/pull/4441)]
* driver/docker: Add support for specifying `cpu_cfs_period` in the Docker driver [[GH-4462](https://github.com/hashicorp/nomad/issues/4462)]
* telemetry: All client metrics include a new `node_class` tag [[GH-3882](https://github.com/hashicorp/nomad/issues/3882)]
@ -15,6 +19,7 @@ IMPROVEMENTS:
BUG FIXES:
* core: Fixed bug in reconciler where allocs already stopped were being unnecessarily updated [[GH-4764](https://github.com/hashicorp/nomad/issues/4764)]
* client: Fix an issue reloading the client config [[GH-4730](https://github.com/hashicorp/nomad/issues/4730)]
## 0.8.6 (September 26, 2018)

View File

@ -134,7 +134,7 @@ func (c *Client) resolveTokenValue(secretID string) (*structs.ACLToken, error) {
if err := c.RPC("ACL.ResolveToken", &req, &resp); err != nil {
// If we encounter an error but have a cached value, mask the error and extend the cache
if ok {
c.logger.Printf("[WARN] client: failed to resolve token, using expired cached value: %v", err)
c.logger.Warn("failed to resolve token, using expired cached value", "error", err)
cached := raw.(*cachedACLValue)
return cached.Token, nil
}
@ -198,7 +198,7 @@ func (c *Client) resolvePolicies(secretID string, policies []string) ([]*structs
if err := c.RPC("ACL.GetPolicies", &req, &resp); err != nil {
// If we encounter an error but have cached policies, mask the error and extend the cache
if len(missing) == 0 {
c.logger.Printf("[WARN] client: failed to resolve policies, using expired cached value: %v", err)
c.logger.Warn("failed to resolve policies, using expired cached value", "error", err)
out = append(out, expired...)
return out, nil
}

View File

@ -76,6 +76,7 @@ func TestAllocations_GarbageCollectAll_ACL(t *testing.T) {
}
func TestAllocations_GarbageCollect(t *testing.T) {
t.Skip("missing mock driver plugin implementation")
t.Parallel()
require := require.New(t)
client := TestClient(t, func(c *config.Config) {
@ -174,6 +175,7 @@ func TestAllocations_GarbageCollect_ACL(t *testing.T) {
}
func TestAllocations_Stats(t *testing.T) {
t.Skip("missing exec driver plugin implementation")
t.Parallel()
require := require.New(t)
client := TestClient(t, nil)

View File

@ -19,6 +19,7 @@ import (
// TestPrevAlloc_StreamAllocDir_TLS asserts ephemeral disk migrations still
// work when TLS is enabled.
func TestPrevAlloc_StreamAllocDir_TLS(t *testing.T) {
t.Skip("missing mock driver plugin implementation")
const (
caFn = "../helper/tlsutil/testdata/global-ca.pem"
serverCertFn = "../helper/tlsutil/testdata/global-server.pem"

View File

@ -6,11 +6,12 @@ import (
"fmt"
"io"
"io/ioutil"
"log"
"os"
"path/filepath"
"sync"
"time"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-multierror"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
@ -58,6 +59,8 @@ var (
TaskDirs = map[string]os.FileMode{TmpDirName: os.ModeSticky | 0777}
)
// AllocDir allows creating, destroying, and accessing an allocation's
// directory. All methods are safe for concurrent use.
type AllocDir struct {
// AllocDir is the directory used for storing any state
// of this allocation. It will be purged on alloc destroy.
@ -73,7 +76,9 @@ type AllocDir struct {
// built is true if Build has successfully run
built bool
logger *log.Logger
mu sync.RWMutex
logger hclog.Logger
}
// AllocDirFS exposes file operations on the alloc dir
@ -88,7 +93,8 @@ type AllocDirFS interface {
// NewAllocDir initializes the AllocDir struct with allocDir as base path for
// the allocation directory.
func NewAllocDir(logger *log.Logger, allocDir string) *AllocDir {
func NewAllocDir(logger hclog.Logger, allocDir string) *AllocDir {
logger = logger.Named("alloc_dir")
return &AllocDir{
AllocDir: allocDir,
SharedDir: filepath.Join(allocDir, SharedAllocName),
@ -100,6 +106,9 @@ func NewAllocDir(logger *log.Logger, allocDir string) *AllocDir {
// Copy an AllocDir and all of its TaskDirs. Returns nil if AllocDir is
// nil.
func (d *AllocDir) Copy() *AllocDir {
d.mu.RLock()
defer d.mu.RUnlock()
if d == nil {
return nil
}
@ -117,6 +126,9 @@ func (d *AllocDir) Copy() *AllocDir {
// NewTaskDir creates a new TaskDir and adds it to the AllocDirs TaskDirs map.
func (d *AllocDir) NewTaskDir(name string) *TaskDir {
d.mu.Lock()
defer d.mu.Unlock()
td := newTaskDir(d.logger, d.AllocDir, name)
d.TaskDirs[name] = td
return td
@ -129,6 +141,9 @@ func (d *AllocDir) NewTaskDir(name string) *TaskDir {
// file "NOMAD-${ALLOC_ID}-ERROR.log" will be appended to the tar with the
// error message as the contents.
func (d *AllocDir) Snapshot(w io.Writer) error {
d.mu.RLock()
defer d.mu.RUnlock()
allocDataDir := filepath.Join(d.SharedDir, SharedDataDir)
rootPaths := []string{allocDataDir}
for _, taskdir := range d.TaskDirs {
@ -195,7 +210,7 @@ func (d *AllocDir) Snapshot(w io.Writer) error {
// the snapshotting side closed the connect
// prematurely and won't try to use the tar
// anyway.
d.logger.Printf("[WARN] client: snapshotting failed and unable to write error marker: %v", writeErr)
d.logger.Warn("snapshotting failed and unable to write error marker", "error", writeErr)
}
return fmt.Errorf("failed to snapshot %s: %v", path, err)
}
@ -206,11 +221,16 @@ func (d *AllocDir) Snapshot(w io.Writer) error {
// Move other alloc directory's shared path and local dir to this alloc dir.
func (d *AllocDir) Move(other *AllocDir, tasks []*structs.Task) error {
d.mu.RLock()
if !d.built {
// Enforce the invariant that Build is called before Move
d.mu.RUnlock()
return fmt.Errorf("unable to move to %q - alloc dir is not built", d.AllocDir)
}
// Moving is slow and only reads immutable fields, so unlock during heavy IO
d.mu.RUnlock()
// Move the data directory
otherDataDir := filepath.Join(other.SharedDir, SharedDataDir)
dataDir := filepath.Join(d.SharedDir, SharedDataDir)
@ -246,7 +266,6 @@ func (d *AllocDir) Move(other *AllocDir, tasks []*structs.Task) error {
// Tears down previously build directory structure.
func (d *AllocDir) Destroy() error {
// Unmount all mounted shared alloc dirs.
var mErr multierror.Error
if err := d.UnmountAll(); err != nil {
@ -258,12 +277,17 @@ func (d *AllocDir) Destroy() error {
}
// Unset built since the alloc dir has been destroyed.
d.mu.Lock()
d.built = false
d.mu.Unlock()
return mErr.ErrorOrNil()
}
// UnmountAll linked/mounted directories in task dirs.
func (d *AllocDir) UnmountAll() error {
d.mu.RLock()
defer d.mu.RUnlock()
var mErr multierror.Error
for _, dir := range d.TaskDirs {
// Check if the directory has the shared alloc mounted.
@ -322,7 +346,9 @@ func (d *AllocDir) Build() error {
}
// Mark as built
d.mu.Lock()
d.built = true
d.mu.Unlock()
return nil
}
@ -386,11 +412,14 @@ func (d *AllocDir) ReadAt(path string, offset int64) (io.ReadCloser, error) {
p := filepath.Join(d.AllocDir, path)
// Check if it is trying to read into a secret directory
d.mu.RLock()
for _, dir := range d.TaskDirs {
if filepath.HasPrefix(p, dir.SecretsDir) {
d.mu.RUnlock()
return nil, fmt.Errorf("Reading secret file prohibited: %s", path)
}
}
d.mu.RUnlock()
f, err := os.Open(p)
if err != nil {

View File

@ -54,7 +54,7 @@ func TestAllocDir_BuildAlloc(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
d.NewTaskDir(t1.Name)
d.NewTaskDir(t2.Name)
@ -91,7 +91,7 @@ func TestAllocDir_MountSharedAlloc(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
if err := d.Build(); err != nil {
t.Fatalf("Build() failed: %v", err)
@ -136,7 +136,7 @@ func TestAllocDir_Snapshot(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
if err := d.Build(); err != nil {
t.Fatalf("Build() failed: %v", err)
@ -223,13 +223,13 @@ func TestAllocDir_Move(t *testing.T) {
defer os.RemoveAll(tmp2)
// Create two alloc dirs
d1 := NewAllocDir(testlog.Logger(t), tmp1)
d1 := NewAllocDir(testlog.HCLogger(t), tmp1)
if err := d1.Build(); err != nil {
t.Fatalf("Build() failed: %v", err)
}
defer d1.Destroy()
d2 := NewAllocDir(testlog.Logger(t), tmp2)
d2 := NewAllocDir(testlog.HCLogger(t), tmp2)
if err := d2.Build(); err != nil {
t.Fatalf("Build() failed: %v", err)
}
@ -284,7 +284,7 @@ func TestAllocDir_EscapeChecking(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
if err := d.Build(); err != nil {
t.Fatalf("Build() failed: %v", err)
}
@ -325,7 +325,7 @@ func TestAllocDir_ReadAt_SecretDir(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
if err := d.Build(); err != nil {
t.Fatalf("Build() failed: %v", err)
}
@ -410,7 +410,7 @@ func TestAllocDir_CreateDir(t *testing.T) {
// TestAllocDir_Copy asserts that AllocDir.Copy does a deep copy of itself and
// all TaskDirs.
func TestAllocDir_Copy(t *testing.T) {
a := NewAllocDir(testlog.Logger(t), "foo")
a := NewAllocDir(testlog.HCLogger(t), "foo")
a.NewTaskDir("bar")
a.NewTaskDir("baz")

View File

@ -3,16 +3,19 @@ package allocdir
import (
"fmt"
"io/ioutil"
"log"
"os"
"path/filepath"
hclog "github.com/hashicorp/go-hclog"
cstructs "github.com/hashicorp/nomad/client/structs"
)
// TaskDir contains all of the paths relevant to a task. All paths are on the
// host system so drivers should mount/link into task containers as necessary.
type TaskDir struct {
// AllocDir is the path to the alloc directory on the host
AllocDir string
// Dir is the path to Task directory on the host
Dir string
@ -37,16 +40,20 @@ type TaskDir struct {
// <task_dir>/secrets/
SecretsDir string
logger *log.Logger
logger hclog.Logger
}
// newTaskDir creates a TaskDir struct with paths set. Call Build() to
// create paths on disk.
//
// Call AllocDir.NewTaskDir to create new TaskDirs
func newTaskDir(logger *log.Logger, allocDir, taskName string) *TaskDir {
func newTaskDir(logger hclog.Logger, allocDir, taskName string) *TaskDir {
taskDir := filepath.Join(allocDir, taskName)
logger = logger.Named("task_dir").With("task_name", taskName)
return &TaskDir{
AllocDir: allocDir,
Dir: taskDir,
SharedAllocDir: filepath.Join(allocDir, SharedAllocName),
LogDir: filepath.Join(allocDir, SharedAllocName, LogDirName),

View File

@ -22,7 +22,7 @@ func TestLinuxSpecialDirs(t *testing.T) {
}
defer os.RemoveAll(allocDir)
td := newTaskDir(testlog.Logger(t), allocDir, "test")
td := newTaskDir(testlog.HCLogger(t), allocDir, "test")
// Despite the task dir not existing, unmountSpecialDirs should *not*
// return an error

View File

@ -18,7 +18,7 @@ func TestTaskDir_EmbedNonexistent(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
td := d.NewTaskDir(t1.Name)
if err := d.Build(); err != nil {
@ -40,7 +40,7 @@ func TestTaskDir_EmbedDirs(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
td := d.NewTaskDir(t1.Name)
if err := d.Build(); err != nil {
@ -97,7 +97,7 @@ func TestTaskDir_NonRoot_Image(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
td := d.NewTaskDir(t1.Name)
if err := d.Build(); err != nil {
@ -120,7 +120,7 @@ func TestTaskDir_NonRoot(t *testing.T) {
}
defer os.RemoveAll(tmp)
d := NewAllocDir(testlog.Logger(t), tmp)
d := NewAllocDir(testlog.HCLogger(t), tmp)
defer d.Destroy()
td := d.NewTaskDir(t1.Name)
if err := d.Build(); err != nil {

View File

@ -0,0 +1,472 @@
package allochealth
import (
"context"
"fmt"
"strings"
"sync"
"time"
"github.com/hashicorp/consul/api"
hclog "github.com/hashicorp/go-hclog"
cconsul "github.com/hashicorp/nomad/client/consul"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/command/agent/consul"
"github.com/hashicorp/nomad/nomad/structs"
)
const (
// allocHealthEventSource is the source used for emitting task events
allocHealthEventSource = "Alloc Unhealthy"
// consulCheckLookupInterval is the interval at which we check if the
// Consul checks are healthy or unhealthy.
consulCheckLookupInterval = 500 * time.Millisecond
)
// Tracker tracks the health of an allocation and makes health events watchable
// via channels.
type Tracker struct {
// ctx and cancelFn is used to shutdown the tracker
ctx context.Context
cancelFn context.CancelFunc
// alloc is the alloc we are tracking
alloc *structs.Allocation
// tg is the task group we are tracking
tg *structs.TaskGroup
// minHealthyTime is the duration an alloc must remain healthy to be
// considered healthy
minHealthyTime time.Duration
// useChecks specifies whether to use Consul healh checks or not
useChecks bool
// consulCheckCount is the number of checks the task group will attempt to
// register
consulCheckCount int
// allocUpdates is a listener for retrieving new alloc updates
allocUpdates *cstructs.AllocListener
// consulClient is used to look up the state of the task's checks
consulClient cconsul.ConsulServiceAPI
// healthy is used to signal whether we have determined the allocation to be
// healthy or unhealthy
healthy chan bool
// allocStopped is triggered when the allocation is stopped and tracking is
// not needed
allocStopped chan struct{}
// l is used to lock shared fields listed below
l sync.Mutex
// tasksHealthy marks whether all the tasks have met their health check
// (disregards Consul)
tasksHealthy bool
// allocFailed marks whether the allocation failed
allocFailed bool
// checksHealthy marks whether all the task's Consul checks are healthy
checksHealthy bool
// taskHealth contains the health state for each task
taskHealth map[string]*taskHealthState
logger hclog.Logger
}
// NewTracker returns a health tracker for the given allocation. An alloc
// listener and consul API object are given so that the watcher can detect
// health changes.
func NewTracker(parentCtx context.Context, logger hclog.Logger, alloc *structs.Allocation,
allocUpdates *cstructs.AllocListener, consulClient cconsul.ConsulServiceAPI,
minHealthyTime time.Duration, useChecks bool) *Tracker {
// Do not create a named sub-logger as the hook controlling
// this struct should pass in an appropriately named
// sub-logger.
t := &Tracker{
healthy: make(chan bool, 1),
allocStopped: make(chan struct{}),
alloc: alloc,
tg: alloc.Job.LookupTaskGroup(alloc.TaskGroup),
minHealthyTime: minHealthyTime,
useChecks: useChecks,
allocUpdates: allocUpdates,
consulClient: consulClient,
logger: logger,
}
t.taskHealth = make(map[string]*taskHealthState, len(t.tg.Tasks))
for _, task := range t.tg.Tasks {
t.taskHealth[task.Name] = &taskHealthState{task: task}
}
for _, task := range t.tg.Tasks {
for _, s := range task.Services {
t.consulCheckCount += len(s.Checks)
}
}
t.ctx, t.cancelFn = context.WithCancel(parentCtx)
return t
}
// Start starts the watcher.
func (t *Tracker) Start() {
go t.watchTaskEvents()
if t.useChecks {
go t.watchConsulEvents()
}
}
// HealthyCh returns a channel that will emit a boolean indicating the health of
// the allocation.
func (t *Tracker) HealthyCh() <-chan bool {
return t.healthy
}
// AllocStoppedCh returns a channel that will be fired if the allocation is
// stopped. This means that health will not be set.
func (t *Tracker) AllocStoppedCh() <-chan struct{} {
return t.allocStopped
}
// TaskEvents returns a map of events by task. This should only be called after
// health has been determined. Only tasks that have contributed to the
// allocation being unhealthy will have an event.
func (t *Tracker) TaskEvents() map[string]*structs.TaskEvent {
t.l.Lock()
defer t.l.Unlock()
// Nothing to do since the failure wasn't task related
if t.allocFailed {
return nil
}
deadline, _ := t.ctx.Deadline()
events := make(map[string]*structs.TaskEvent, len(t.tg.Tasks))
// Go through are task information and build the event map
for task, state := range t.taskHealth {
useChecks := t.tg.Update.HealthCheck == structs.UpdateStrategyHealthCheck_Checks
if e, ok := state.event(deadline, t.tg.Update.MinHealthyTime, useChecks); ok {
events[task] = structs.NewTaskEvent(allocHealthEventSource).SetMessage(e)
}
}
return events
}
// setTaskHealth is used to set the tasks health as healthy or unhealthy. If the
// allocation is terminal, health is immediately broadcasted.
func (t *Tracker) setTaskHealth(healthy, terminal bool) {
t.l.Lock()
defer t.l.Unlock()
t.tasksHealthy = healthy
// If we are marked healthy but we also require Consul to be healthy and it
// isn't yet, return, unless the task is terminal
requireConsul := t.useChecks && t.consulCheckCount > 0
if !terminal && healthy && requireConsul && !t.checksHealthy {
return
}
select {
case t.healthy <- healthy:
default:
}
// Shutdown the tracker
t.cancelFn()
}
// setCheckHealth is used to mark the checks as either healthy or unhealthy.
func (t *Tracker) setCheckHealth(healthy bool) {
t.l.Lock()
defer t.l.Unlock()
t.checksHealthy = healthy
// Only signal if we are healthy and so is the tasks
if !healthy || !t.tasksHealthy {
return
}
select {
case t.healthy <- healthy:
default:
}
// Shutdown the tracker
t.cancelFn()
}
// markAllocStopped is used to mark the allocation as having stopped.
func (t *Tracker) markAllocStopped() {
close(t.allocStopped)
t.cancelFn()
}
// watchTaskEvents is a long lived watcher that watches for the health of the
// allocation's tasks.
func (t *Tracker) watchTaskEvents() {
alloc := t.alloc
allStartedTime := time.Time{}
healthyTimer := time.NewTimer(0)
if !healthyTimer.Stop() {
select {
case <-healthyTimer.C:
default:
}
}
for {
// If the alloc is being stopped by the server just exit
switch alloc.DesiredStatus {
case structs.AllocDesiredStatusStop, structs.AllocDesiredStatusEvict:
t.logger.Trace("desired status is terminal for alloc", "alloc_id", alloc.ID, "desired_status", alloc.DesiredStatus)
t.markAllocStopped()
return
}
// Store the task states
t.l.Lock()
for task, state := range alloc.TaskStates {
t.taskHealth[task].state = state
}
t.l.Unlock()
// Detect if the alloc is unhealthy or if all tasks have started yet
latestStartTime := time.Time{}
for _, state := range alloc.TaskStates {
// One of the tasks has failed so we can exit watching
if state.Failed || !state.FinishedAt.IsZero() {
t.setTaskHealth(false, true)
return
}
if state.State != structs.TaskStateRunning {
latestStartTime = time.Time{}
break
} else if state.StartedAt.After(latestStartTime) {
latestStartTime = state.StartedAt
}
}
// If the alloc is marked as failed by the client but none of the
// individual tasks failed, that means something failed at the alloc
// level.
if alloc.ClientStatus == structs.AllocClientStatusFailed {
t.l.Lock()
t.allocFailed = true
t.l.Unlock()
t.setTaskHealth(false, true)
return
}
if !latestStartTime.Equal(allStartedTime) {
// Avoid the timer from firing at the old start time
if !healthyTimer.Stop() {
select {
case <-healthyTimer.C:
default:
}
}
// Set the timer since all tasks are started
if !latestStartTime.IsZero() {
allStartedTime = latestStartTime
healthyTimer.Reset(t.minHealthyTime)
}
}
select {
case <-t.ctx.Done():
return
case newAlloc, ok := <-t.allocUpdates.Ch:
if !ok {
return
}
alloc = newAlloc
case <-healthyTimer.C:
t.setTaskHealth(true, false)
}
}
}
// watchConsulEvents iis a long lived watcher that watches for the health of the
// allocation's Consul checks.
func (t *Tracker) watchConsulEvents() {
// checkTicker is the ticker that triggers us to look at the checks in
// Consul
checkTicker := time.NewTicker(consulCheckLookupInterval)
defer checkTicker.Stop()
// healthyTimer fires when the checks have been healthy for the
// MinHealthyTime
healthyTimer := time.NewTimer(0)
if !healthyTimer.Stop() {
select {
case <-healthyTimer.C:
default:
}
}
// primed marks whether the healthy timer has been set
primed := false
// Store whether the last Consul checks call was successful or not
consulChecksErr := false
// allocReg are the registered objects in Consul for the allocation
var allocReg *consul.AllocRegistration
OUTER:
for {
select {
case <-t.ctx.Done():
return
case <-checkTicker.C:
newAllocReg, err := t.consulClient.AllocRegistrations(t.alloc.ID)
if err != nil {
if !consulChecksErr {
consulChecksErr = true
t.logger.Warn("error looking up Consul registrations for allocation", "error", err, "alloc_id", t.alloc.ID)
}
continue OUTER
} else {
consulChecksErr = false
allocReg = newAllocReg
}
case <-healthyTimer.C:
t.setCheckHealth(true)
}
if allocReg == nil {
continue
}
// Store the task registrations
t.l.Lock()
for task, reg := range allocReg.Tasks {
t.taskHealth[task].taskRegistrations = reg
}
t.l.Unlock()
// Detect if all the checks are passing
passed := true
CHECKS:
for _, treg := range allocReg.Tasks {
for _, sreg := range treg.Services {
for _, check := range sreg.Checks {
if check.Status == api.HealthPassing {
continue
}
passed = false
t.setCheckHealth(false)
break CHECKS
}
}
}
if !passed {
// Reset the timer since we have transitioned back to unhealthy
if primed {
if !healthyTimer.Stop() {
select {
case <-healthyTimer.C:
default:
}
}
primed = false
}
} else if !primed {
// Reset the timer to fire after MinHealthyTime
if !healthyTimer.Stop() {
select {
case <-healthyTimer.C:
default:
}
}
primed = true
healthyTimer.Reset(t.minHealthyTime)
}
}
}
// taskHealthState captures all known health information about a task. It is
// largely used to determine if the task has contributed to the allocation being
// unhealthy.
type taskHealthState struct {
task *structs.Task
state *structs.TaskState
taskRegistrations *consul.TaskRegistration
}
// event takes the deadline time for the allocation to be healthy and the update
// strategy of the group. It returns true if the task has contributed to the
// allocation being unhealthy and if so, an event description of why.
func (t *taskHealthState) event(deadline time.Time, minHealthyTime time.Duration, useChecks bool) (string, bool) {
requireChecks := false
desiredChecks := 0
for _, s := range t.task.Services {
if nc := len(s.Checks); nc > 0 {
requireChecks = true
desiredChecks += nc
}
}
requireChecks = requireChecks && useChecks
if t.state != nil {
if t.state.Failed {
return "Unhealthy because of failed task", true
}
if t.state.State != structs.TaskStateRunning {
return "Task not running by deadline", true
}
// We are running so check if we have been running long enough
if t.state.StartedAt.Add(minHealthyTime).After(deadline) {
return fmt.Sprintf("Task not running for min_healthy_time of %v by deadline", minHealthyTime), true
}
}
if t.taskRegistrations != nil {
var notPassing []string
passing := 0
OUTER:
for _, sreg := range t.taskRegistrations.Services {
for _, check := range sreg.Checks {
if check.Status != api.HealthPassing {
notPassing = append(notPassing, sreg.Service.Service)
continue OUTER
} else {
passing++
}
}
}
if len(notPassing) != 0 {
return fmt.Sprintf("Services not healthy by deadline: %s", strings.Join(notPassing, ", ")), true
}
if passing != desiredChecks {
return fmt.Sprintf("Only %d out of %d checks registered and passing", passing, desiredChecks), true
}
} else if requireChecks {
return "Service checks not registered", true
}
return "", false
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,245 @@
package allocrunner
import (
"context"
"fmt"
"time"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/nomad/structs"
)
// allocHealthSetter is a shim to allow the alloc health watcher hook to set
// and clear the alloc health without full access to the alloc runner state
type allocHealthSetter struct {
ar *allocRunner
}
// ClearHealth allows the health watcher hook to clear the alloc's deployment
// health if the deployment id changes. It does not update the server as the
// status is only cleared when already receiving an update from the server.
//
// Only for use by health hook.
func (a *allocHealthSetter) ClearHealth() {
a.ar.stateLock.Lock()
a.ar.state.ClearDeploymentStatus()
a.ar.stateLock.Unlock()
}
// SetHealth allows the health watcher hook to set the alloc's
// deployment/migration health and emit task events.
//
// Only for use by health hook.
func (a *allocHealthSetter) SetHealth(healthy, isDeploy bool, trackerTaskEvents map[string]*structs.TaskEvent) {
// Updating alloc deployment state is tricky because it may be nil, but
// if it's not then we need to maintain the values of Canary and
// ModifyIndex as they're only mutated by the server.
a.ar.stateLock.Lock()
a.ar.state.SetDeploymentStatus(time.Now(), healthy)
a.ar.stateLock.Unlock()
// If deployment is unhealthy emit task events explaining why
a.ar.tasksLock.RLock()
if !healthy && isDeploy {
for task, event := range trackerTaskEvents {
if tr, ok := a.ar.tasks[task]; ok {
// Append but don't emit event since the server
// will be updated below
tr.AppendEvent(event)
}
}
}
// Gather the state of the other tasks
states := make(map[string]*structs.TaskState, len(a.ar.tasks))
for name, tr := range a.ar.tasks {
states[name] = tr.TaskState()
}
a.ar.tasksLock.RUnlock()
// Build the client allocation
calloc := a.ar.clientAlloc(states)
// Update the server
a.ar.stateUpdater.AllocStateUpdated(calloc)
// Broadcast client alloc to listeners
a.ar.allocBroadcaster.Send(calloc)
}
// initRunnerHooks intializes the runners hooks.
func (ar *allocRunner) initRunnerHooks() {
hookLogger := ar.logger.Named("runner_hook")
// create health setting shim
hs := &allocHealthSetter{ar}
// Create the alloc directory hook. This is run first to ensure the
// directory path exists for other hooks.
ar.runnerHooks = []interfaces.RunnerHook{
newAllocDirHook(hookLogger, ar.allocDir),
newDiskMigrationHook(hookLogger, ar.prevAllocWatcher, ar.allocDir),
newAllocHealthWatcherHook(hookLogger, ar.Alloc(), hs, ar.Listener(), ar.consulClient),
}
}
// prerun is used to run the runners prerun hooks.
func (ar *allocRunner) prerun() error {
if ar.logger.IsTrace() {
start := time.Now()
ar.logger.Trace("running pre-run hooks", "start", start)
defer func() {
end := time.Now()
ar.logger.Trace("finished pre-run hooks", "end", end, "duration", end.Sub(start))
}()
}
for _, hook := range ar.runnerHooks {
pre, ok := hook.(interfaces.RunnerPrerunHook)
if !ok {
continue
}
//TODO Check hook state
name := pre.Name()
var start time.Time
if ar.logger.IsTrace() {
start = time.Now()
ar.logger.Trace("running pre-run hook", "name", name, "start", start)
}
if err := pre.Prerun(context.TODO()); err != nil {
return fmt.Errorf("pre-run hook %q failed: %v", name, err)
}
//TODO Persist hook state locally
if ar.logger.IsTrace() {
end := time.Now()
ar.logger.Trace("finished pre-run hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return nil
}
// update runs the alloc runner update hooks. Update hooks are run
// asynchronously with all other alloc runner operations.
func (ar *allocRunner) update(update *structs.Allocation) error {
if ar.logger.IsTrace() {
start := time.Now()
ar.logger.Trace("running update hooks", "start", start)
defer func() {
end := time.Now()
ar.logger.Trace("finished update hooks", "end", end, "duration", end.Sub(start))
}()
}
req := &interfaces.RunnerUpdateRequest{
Alloc: update,
}
var merr multierror.Error
for _, hook := range ar.runnerHooks {
h, ok := hook.(interfaces.RunnerUpdateHook)
if !ok {
continue
}
name := h.Name()
var start time.Time
if ar.logger.IsTrace() {
start = time.Now()
ar.logger.Trace("running pre-run hook", "name", name, "start", start)
}
if err := h.Update(req); err != nil {
merr.Errors = append(merr.Errors, fmt.Errorf("update hook %q failed: %v", name, err))
}
if ar.logger.IsTrace() {
end := time.Now()
ar.logger.Trace("finished update hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return merr.ErrorOrNil()
}
// postrun is used to run the runners postrun hooks.
func (ar *allocRunner) postrun() error {
if ar.logger.IsTrace() {
start := time.Now()
ar.logger.Trace("running post-run hooks", "start", start)
defer func() {
end := time.Now()
ar.logger.Trace("finished post-run hooks", "end", end, "duration", end.Sub(start))
}()
}
for _, hook := range ar.runnerHooks {
post, ok := hook.(interfaces.RunnerPostrunHook)
if !ok {
continue
}
name := post.Name()
var start time.Time
if ar.logger.IsTrace() {
start = time.Now()
ar.logger.Trace("running post-run hook", "name", name, "start", start)
}
if err := post.Postrun(); err != nil {
return fmt.Errorf("hook %q failed: %v", name, err)
}
if ar.logger.IsTrace() {
end := time.Now()
ar.logger.Trace("finished post-run hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return nil
}
// destroy is used to run the runners destroy hooks. All hooks are run and
// errors are returned as a multierror.
func (ar *allocRunner) destroy() error {
if ar.logger.IsTrace() {
start := time.Now()
ar.logger.Trace("running destroy hooks", "start", start)
defer func() {
end := time.Now()
ar.logger.Trace("finished destroy hooks", "end", end, "duration", end.Sub(start))
}()
}
var merr multierror.Error
for _, hook := range ar.runnerHooks {
h, ok := hook.(interfaces.RunnerDestroyHook)
if !ok {
continue
}
name := h.Name()
var start time.Time
if ar.logger.IsTrace() {
start = time.Now()
ar.logger.Trace("running destroy hook", "name", name, "start", start)
}
if err := h.Destroy(); err != nil {
merr.Errors = append(merr.Errors, fmt.Errorf("destroy hook %q failed: %v", name, err))
}
if ar.logger.IsTrace() {
end := time.Now()
ar.logger.Trace("finished destroy hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return nil
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,35 @@
package allocrunner
import (
"context"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
)
// allocDirHook creates and destroys the root directory and shared directories
// for an allocation.
type allocDirHook struct {
allocDir *allocdir.AllocDir
logger log.Logger
}
func newAllocDirHook(logger log.Logger, allocDir *allocdir.AllocDir) *allocDirHook {
ad := &allocDirHook{
allocDir: allocDir,
}
ad.logger = logger.Named(ad.Name())
return ad
}
func (h *allocDirHook) Name() string {
return "alloc_dir"
}
func (h *allocDirHook) Prerun(context.Context) error {
return h.allocDir.Build()
}
func (h *allocDirHook) Destroy() error {
return h.allocDir.Destroy()
}

View File

@ -0,0 +1,48 @@
package allocrunner
import (
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocwatcher"
clientconfig "github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/consul"
"github.com/hashicorp/nomad/client/interfaces"
cstate "github.com/hashicorp/nomad/client/state"
"github.com/hashicorp/nomad/client/vaultclient"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/shared/loader"
)
// Config holds the configuration for creating an allocation runner.
type Config struct {
// Logger is the logger for the allocation runner.
Logger log.Logger
// ClientConfig is the clients configuration.
ClientConfig *clientconfig.Config
// Alloc captures the allocation that should be run.
Alloc *structs.Allocation
// StateDB is used to store and restore state.
StateDB cstate.StateDB
// Consul is the Consul client used to register task services and checks
Consul consul.ConsulServiceAPI
// Vault is the Vault client to use to retrieve Vault tokens
Vault vaultclient.VaultClient
// StateUpdater is used to emit updated task state
StateUpdater interfaces.AllocStateHandler
// PrevAllocWatcher handles waiting on previous allocations and
// migrating their ephemeral disk when necessary.
PrevAllocWatcher allocwatcher.PrevAllocWatcher
// PluginLoader is used to load plugins.
PluginLoader loader.PluginCatalog
// PluginSingletonLoader is a plugin loader that will returns singleton
// instances of the plugins.
PluginSingletonLoader loader.PluginCatalog
}

View File

@ -0,0 +1,244 @@
package allocrunner
import (
"context"
"fmt"
"sync"
"time"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allochealth"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/consul"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
// healthMutator is able to set/clear alloc health.
type healthSetter interface {
// Set health via the mutator
SetHealth(healthy, isDeploy bool, taskEvents map[string]*structs.TaskEvent)
// Clear health when the deployment ID changes
ClearHealth()
}
// allocHealthWatcherHook is responsible for watching an allocation's task
// status and (optionally) Consul health check status to determine if the
// allocation is health or unhealthy. Used by deployments and migrations.
type allocHealthWatcherHook struct {
healthSetter healthSetter
// consul client used to monitor health checks
consul consul.ConsulServiceAPI
// listener is given to trackers to listen for alloc updates and closed
// when the alloc is destroyed.
listener *cstructs.AllocListener
// hookLock is held by hook methods to prevent concurrent access by
// Update and synchronous hooks.
hookLock sync.Mutex
// watchDone is created before calling watchHealth and is closed when
// watchHealth exits. Must be passed into watchHealth to avoid races.
// Initialized already closed as Update may be called before Prerun.
watchDone chan struct{}
// ranOnce is set once Prerun or Update have run at least once. This
// prevents Prerun from running if an Update has already been
// processed. Must hold hookLock to access.
ranOnce bool
// cancelFn stops the health watching/setting goroutine. Wait on
// watchLock to block until the watcher exits.
cancelFn context.CancelFunc
// alloc set by new func or Update. Must hold hookLock to access.
alloc *structs.Allocation
// isDeploy is true if monitoring a deployment. Set in init(). Must
// hold hookLock to access.
isDeploy bool
logger log.Logger
}
func newAllocHealthWatcherHook(logger log.Logger, alloc *structs.Allocation, hs healthSetter,
listener *cstructs.AllocListener, consul consul.ConsulServiceAPI) interfaces.RunnerHook {
// Neither deployments nor migrations care about the health of
// non-service jobs so never watch their health
if alloc.Job.Type != structs.JobTypeService {
return noopAllocHealthWatcherHook{}
}
// Initialize watchDone with a closed chan in case Update runs before Prerun
closedDone := make(chan struct{})
close(closedDone)
h := &allocHealthWatcherHook{
alloc: alloc,
cancelFn: func() {}, // initialize to prevent nil func panics
watchDone: closedDone,
consul: consul,
healthSetter: hs,
listener: listener,
}
h.logger = logger.Named(h.Name())
return h
}
func (h *allocHealthWatcherHook) Name() string {
return "alloc_health_watcher"
}
// init starts the allochealth.Tracker and watchHealth goroutine on either
// Prerun or Update. Caller must set/update alloc and logger fields.
//
// Not threadsafe so the caller should lock since Updates occur concurrently.
func (h *allocHealthWatcherHook) init() error {
// No need to watch health as it's already set
if h.alloc.DeploymentStatus.HasHealth() {
return nil
}
tg := h.alloc.Job.LookupTaskGroup(h.alloc.TaskGroup)
if tg == nil {
return fmt.Errorf("task group %q does not exist in job %q", h.alloc.TaskGroup, h.alloc.Job.ID)
}
h.isDeploy = h.alloc.DeploymentID != ""
// No need to watch allocs for deployments that rely on operators
// manually setting health
if h.isDeploy && (tg.Update == nil || tg.Update.HealthCheck == structs.UpdateStrategyHealthCheck_Manual) {
return nil
}
// Define the deadline, health method, min healthy time from the
// deployment if this is a deployment; otherwise from the migration
// strategy.
deadline, useChecks, minHealthyTime := getHealthParams(time.Now(), tg, h.isDeploy)
// Create a context that is canceled when the tracker should shutdown
// or the deadline is reached.
ctx := context.Background()
ctx, h.cancelFn = context.WithDeadline(ctx, deadline)
// Create a new tracker, start it, and watch for health results.
tracker := allochealth.NewTracker(ctx, h.logger, h.alloc,
h.listener, h.consul, minHealthyTime, useChecks)
tracker.Start()
// Create a new done chan and start watching for health updates
h.watchDone = make(chan struct{})
go h.watchHealth(ctx, tracker, h.watchDone)
return nil
}
func (h *allocHealthWatcherHook) Prerun(context.Context) error {
h.hookLock.Lock()
defer h.hookLock.Unlock()
if h.ranOnce {
// An Update beat Prerun to running the watcher; noop
return nil
}
h.ranOnce = true
return h.init()
}
func (h *allocHealthWatcherHook) Update(req *interfaces.RunnerUpdateRequest) error {
h.hookLock.Lock()
defer h.hookLock.Unlock()
// Prevent Prerun from running after an Update
h.ranOnce = true
// Cancel the old watcher and create a new one
h.cancelFn()
// Wait until the watcher exits
<-h.watchDone
// Deployment has changed, reset status
if req.Alloc.DeploymentID != h.alloc.DeploymentID {
h.healthSetter.ClearHealth()
}
// Update alloc
h.alloc = req.Alloc
return h.init()
}
func (h *allocHealthWatcherHook) Destroy() error {
h.hookLock.Lock()
defer h.hookLock.Unlock()
h.cancelFn()
h.listener.Close()
// Wait until the watcher exits
<-h.watchDone
return nil
}
// watchHealth watches alloc health until it is set, the alloc is stopped, or
// the context is canceled. watchHealth will be canceled and restarted on
// Updates so calls are serialized with a lock.
func (h *allocHealthWatcherHook) watchHealth(ctx context.Context, tracker *allochealth.Tracker, done chan<- struct{}) {
defer close(done)
select {
case <-ctx.Done():
return
case <-tracker.AllocStoppedCh():
return
case healthy := <-tracker.HealthyCh():
// If this is an unhealthy deployment emit events for tasks
var taskEvents map[string]*structs.TaskEvent
if !healthy && h.isDeploy {
taskEvents = tracker.TaskEvents()
}
h.healthSetter.SetHealth(healthy, h.isDeploy, taskEvents)
}
}
// getHealthParams returns the health watcher parameters which vary based on
// whether this allocation is in a deployment or migration.
func getHealthParams(now time.Time, tg *structs.TaskGroup, isDeploy bool) (deadline time.Time, useChecks bool, minHealthyTime time.Duration) {
if isDeploy {
deadline = now.Add(tg.Update.HealthyDeadline)
minHealthyTime = tg.Update.MinHealthyTime
useChecks = tg.Update.HealthCheck == structs.UpdateStrategyHealthCheck_Checks
} else {
strategy := tg.Migrate
if strategy == nil {
// For backwards compat with pre-0.8 allocations that
// don't have a migrate strategy set.
strategy = structs.DefaultMigrateStrategy()
}
deadline = now.Add(strategy.HealthyDeadline)
minHealthyTime = strategy.MinHealthyTime
useChecks = strategy.HealthCheck == structs.MigrateStrategyHealthChecks
}
return
}
// noopAllocHealthWatcherHook is an empty hook implementation returned by
// newAllocHealthWatcherHook when an allocation will never need its health
// monitored.
type noopAllocHealthWatcherHook struct{}
func (noopAllocHealthWatcherHook) Name() string {
return "alloc_health_watcher"
}

View File

@ -0,0 +1,325 @@
package allocrunner
import (
"context"
"sync"
"testing"
"time"
consulapi "github.com/hashicorp/consul/api"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/consul"
cstructs "github.com/hashicorp/nomad/client/structs"
agentconsul "github.com/hashicorp/nomad/command/agent/consul"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
// statically assert health hook implements the expected interfaces
var _ interfaces.RunnerPrerunHook = (*allocHealthWatcherHook)(nil)
var _ interfaces.RunnerUpdateHook = (*allocHealthWatcherHook)(nil)
var _ interfaces.RunnerDestroyHook = (*allocHealthWatcherHook)(nil)
// allocHealth is emitted to a chan whenever SetHealth is called
type allocHealth struct {
healthy bool
taskEvents map[string]*structs.TaskEvent
}
// mockHealthSetter implements healthSetter that stores health internally
type mockHealthSetter struct {
setCalls int
clearCalls int
healthy *bool
isDeploy *bool
taskEvents map[string]*structs.TaskEvent
mu sync.Mutex
healthCh chan allocHealth
}
// newMockHealthSetter returns a mock HealthSetter that emits all SetHealth
// calls on a buffered chan. Callers who do need need notifications of health
// changes may just create the struct directly.
func newMockHealthSetter() *mockHealthSetter {
return &mockHealthSetter{
healthCh: make(chan allocHealth, 1),
}
}
func (m *mockHealthSetter) SetHealth(healthy, isDeploy bool, taskEvents map[string]*structs.TaskEvent) {
m.mu.Lock()
defer m.mu.Unlock()
m.setCalls++
m.healthy = &healthy
m.isDeploy = &isDeploy
m.taskEvents = taskEvents
if m.healthCh != nil {
m.healthCh <- allocHealth{healthy, taskEvents}
}
}
func (m *mockHealthSetter) ClearHealth() {
m.mu.Lock()
defer m.mu.Unlock()
m.clearCalls++
m.healthy = nil
m.isDeploy = nil
m.taskEvents = nil
}
// TestHealthHook_PrerunDestroy asserts a health hook does not error if it is run and destroyed.
func TestHealthHook_PrerunDestroy(t *testing.T) {
t.Parallel()
require := require.New(t)
b := cstructs.NewAllocBroadcaster()
defer b.Close()
logger := testlog.HCLogger(t)
consul := consul.NewMockConsulServiceClient(t, logger)
hs := &mockHealthSetter{}
h := newAllocHealthWatcherHook(logger, mock.Alloc(), hs, b.Listen(), consul)
// Assert we implemented the right interfaces
prerunh, ok := h.(interfaces.RunnerPrerunHook)
require.True(ok)
_, ok = h.(interfaces.RunnerUpdateHook)
require.True(ok)
destroyh, ok := h.(interfaces.RunnerDestroyHook)
require.True(ok)
// Prerun
require.NoError(prerunh.Prerun(context.Background()))
// Assert isDeploy is false (other tests peek at isDeploy to determine
// if an Update applied)
ahw := h.(*allocHealthWatcherHook)
ahw.hookLock.Lock()
assert.False(t, ahw.isDeploy)
ahw.hookLock.Unlock()
// Destroy
require.NoError(destroyh.Destroy())
}
// TestHealthHook_PrerunUpdateDestroy asserts Updates may be applied concurrently.
func TestHealthHook_PrerunUpdateDestroy(t *testing.T) {
t.Parallel()
require := require.New(t)
alloc := mock.Alloc()
b := cstructs.NewAllocBroadcaster()
defer b.Close()
logger := testlog.HCLogger(t)
consul := consul.NewMockConsulServiceClient(t, logger)
hs := &mockHealthSetter{}
h := newAllocHealthWatcherHook(logger, alloc.Copy(), hs, b.Listen(), consul).(*allocHealthWatcherHook)
// Prerun
require.NoError(h.Prerun(context.Background()))
// Update multiple times in a goroutine to mimic Client behavior
// (Updates are concurrent with alloc runner but are applied serially).
errs := make(chan error, 2)
go func() {
defer close(errs)
for i := 0; i < cap(errs); i++ {
alloc.AllocModifyIndex++
errs <- h.Update(&interfaces.RunnerUpdateRequest{Alloc: alloc.Copy()})
}
}()
for err := range errs {
assert.NoError(t, err)
}
// Destroy
require.NoError(h.Destroy())
}
// TestHealthHook_UpdatePrerunDestroy asserts that a hook may have Update
// called before Prerun.
func TestHealthHook_UpdatePrerunDestroy(t *testing.T) {
t.Parallel()
require := require.New(t)
alloc := mock.Alloc()
b := cstructs.NewAllocBroadcaster()
defer b.Close()
logger := testlog.HCLogger(t)
consul := consul.NewMockConsulServiceClient(t, logger)
hs := &mockHealthSetter{}
h := newAllocHealthWatcherHook(logger, alloc.Copy(), hs, b.Listen(), consul).(*allocHealthWatcherHook)
// Set a DeploymentID to cause ClearHealth to be called
alloc.DeploymentID = uuid.Generate()
// Update in a goroutine to mimic Client behavior (Updates are
// concurrent with alloc runner).
errs := make(chan error, 1)
go func(alloc *structs.Allocation) {
errs <- h.Update(&interfaces.RunnerUpdateRequest{Alloc: alloc})
close(errs)
}(alloc.Copy())
for err := range errs {
assert.NoError(t, err)
}
// Prerun should be a noop
require.NoError(h.Prerun(context.Background()))
// Assert that the Update took affect by isDeploy being true
h.hookLock.Lock()
assert.True(t, h.isDeploy)
h.hookLock.Unlock()
// Destroy
require.NoError(h.Destroy())
}
// TestHealthHook_Destroy asserts that a hook may have only Destroy called.
func TestHealthHook_Destroy(t *testing.T) {
t.Parallel()
require := require.New(t)
b := cstructs.NewAllocBroadcaster()
defer b.Close()
logger := testlog.HCLogger(t)
consul := consul.NewMockConsulServiceClient(t, logger)
hs := &mockHealthSetter{}
h := newAllocHealthWatcherHook(logger, mock.Alloc(), hs, b.Listen(), consul).(*allocHealthWatcherHook)
// Destroy
require.NoError(h.Destroy())
}
// TestHealthHook_SetHealth asserts SetHealth is called when health status is
// set. Uses task state and health checks.
func TestHealthHook_SetHealth(t *testing.T) {
t.Parallel()
require := require.New(t)
alloc := mock.Alloc()
alloc.Job.TaskGroups[0].Migrate.MinHealthyTime = 1 // let's speed things up
task := alloc.Job.TaskGroups[0].Tasks[0]
// Synthesize running alloc and tasks
alloc.ClientStatus = structs.AllocClientStatusRunning
alloc.TaskStates = map[string]*structs.TaskState{
task.Name: {
State: structs.TaskStateRunning,
StartedAt: time.Now(),
},
}
// Make Consul response
check := &consulapi.AgentCheck{
Name: task.Services[0].Checks[0].Name,
Status: consulapi.HealthPassing,
}
taskRegs := map[string]*agentconsul.TaskRegistration{
task.Name: {
Services: map[string]*agentconsul.ServiceRegistration{
task.Services[0].Name: {
Service: &consulapi.AgentService{
ID: "foo",
Service: task.Services[0].Name,
},
Checks: []*consulapi.AgentCheck{check},
},
},
},
}
b := cstructs.NewAllocBroadcaster()
defer b.Close()
logger := testlog.HCLogger(t)
// Don't reply on the first call
called := false
consul := consul.NewMockConsulServiceClient(t, logger)
consul.AllocRegistrationsFn = func(string) (*agentconsul.AllocRegistration, error) {
if !called {
called = true
return nil, nil
}
reg := &agentconsul.AllocRegistration{
Tasks: taskRegs,
}
return reg, nil
}
hs := newMockHealthSetter()
h := newAllocHealthWatcherHook(logger, alloc.Copy(), hs, b.Listen(), consul).(*allocHealthWatcherHook)
// Prerun
require.NoError(h.Prerun(context.Background()))
// Wait for health to be set (healthy)
select {
case <-time.After(5 * time.Second):
t.Fatalf("timeout waiting for health to be set")
case health := <-hs.healthCh:
require.True(health.healthy)
// Healthy allocs shouldn't emit task events
ev := health.taskEvents[task.Name]
require.Nilf(ev, "%#v", health.taskEvents)
}
// Destroy
require.NoError(h.Destroy())
}
// TestHealthHook_SystemNoop asserts that system jobs return the noop tracker.
func TestHealthHook_SystemNoop(t *testing.T) {
t.Parallel()
h := newAllocHealthWatcherHook(testlog.HCLogger(t), mock.SystemAlloc(), nil, nil, nil)
// Assert that it's the noop impl
_, ok := h.(noopAllocHealthWatcherHook)
require.True(t, ok)
// Assert the noop impl does not implement any hooks
_, ok = h.(interfaces.RunnerPrerunHook)
require.False(t, ok)
_, ok = h.(interfaces.RunnerUpdateHook)
require.False(t, ok)
_, ok = h.(interfaces.RunnerDestroyHook)
require.False(t, ok)
}
// TestHealthHook_BatchNoop asserts that batch jobs return the noop tracker.
func TestHealthHook_BatchNoop(t *testing.T) {
t.Parallel()
h := newAllocHealthWatcherHook(testlog.HCLogger(t), mock.BatchAlloc(), nil, nil, nil)
// Assert that it's the noop impl
_, ok := h.(noopAllocHealthWatcherHook)
require.True(t, ok)
}

View File

@ -0,0 +1,35 @@
package interfaces
import (
"github.com/hashicorp/nomad/client/allocrunner/state"
"github.com/hashicorp/nomad/nomad/structs"
cstructs "github.com/hashicorp/nomad/client/structs"
)
// AllocRunner is the interface for an allocation runner.
type AllocRunner interface {
// ID returns the ID of the allocation being run.
ID() string
// Run starts the runner and begins executing all the tasks as part of the
// allocation.
Run()
// State returns a copy of the runners state object
State() *state.State
TaskStateHandler
}
// TaskStateHandler exposes a handler to be called when a task's state changes
type TaskStateHandler interface {
// TaskStateUpdated is used to emit updated task state
TaskStateUpdated(task string, state *structs.TaskState)
}
// AllocStatsReporter gives acess to the latest resource usage from the
// allocation
type AllocStatsReporter interface {
LatestAllocStats(taskFilter string) (*cstructs.AllocResourceUsage, error)
}

View File

@ -0,0 +1,44 @@
package interfaces
import (
"context"
"github.com/hashicorp/nomad/client/allocrunner/state"
"github.com/hashicorp/nomad/nomad/structs"
)
// RunnnerHook is a lifecycle hook into the life cycle of an allocation runner.
type RunnerHook interface {
Name() string
}
type RunnerPrerunHook interface {
RunnerHook
Prerun(context.Context) error
}
type RunnerPostrunHook interface {
RunnerHook
Postrun() error
}
type RunnerDestroyHook interface {
RunnerHook
Destroy() error
}
type RunnerUpdateHook interface {
RunnerHook
Update(*RunnerUpdateRequest) error
}
type RunnerUpdateRequest struct {
Alloc *structs.Allocation
}
// XXX Not sure yet
type RunnerHookFactory func(target HookTarget) (RunnerHook, error)
type HookTarget interface {
// State retrieves a copy of the target alloc runners state.
State() *state.State
}

View File

@ -0,0 +1,151 @@
package interfaces
import (
"context"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/driver/env"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
/*
Restart
+--------------------------------------------------------+
| |
| *Update |
| +-------+ |
| | | |
| | | |
| +---v-------+----+ |
+----v----+ | Running | +----+-----+ +--------------+
| | *Prestart |----------------| *Exited | | *Stop | |
| Pending +-------------> *Poststart run +---^-----------> Exited +-----------> Terminal |
| | | upon entering | | | | NoRestart | |
+---------+ | running | | +----------+ +--------------+
| | |
+--------+-------+ |
| |
+-----------+
*Kill
(forces terminal)
Link: http://stable.ascii-flow.appspot.com/#Draw4489375405966393064/1824429135
*/
// TaskHook is a lifecycle hook into the life cycle of a task runner.
type TaskHook interface {
Name() string
}
type TaskPrestartRequest struct {
// HookData is previously set data by the hook
HookData map[string]string
// Task is the task to run
Task *structs.Task
// Vault token may optionally be set if a Vault token is available
VaultToken string
// TaskDir contains the task's directory tree on the host
TaskDir *allocdir.TaskDir
// TaskEnv is the task's environment
TaskEnv *env.TaskEnv
}
type TaskPrestartResponse struct {
// Env is the environment variables to set for the task
Env map[string]string
// HookData allows the hook to emit data to be passed in the next time it is
// run
HookData map[string]string
// Done lets the hook indicate that it should only be run once
Done bool
}
type TaskPrestartHook interface {
TaskHook
// Prestart is called before the task is started.
Prestart(context.Context, *TaskPrestartRequest, *TaskPrestartResponse) error
}
// DriverStats is the interface implemented by DriverHandles to return task stats.
type DriverStats interface {
Stats() (*cstructs.TaskResourceUsage, error)
}
type TaskPoststartRequest struct {
// Exec hook (may be nil)
DriverExec interfaces.ScriptExecutor
// Network info (may be nil)
DriverNetwork *cstructs.DriverNetwork
// TaskEnv is the task's environment
TaskEnv *env.TaskEnv
// Stats collector
DriverStats DriverStats
}
type TaskPoststartResponse struct{}
type TaskPoststartHook interface {
TaskHook
// Poststart is called after the task has started.
Poststart(context.Context, *TaskPoststartRequest, *TaskPoststartResponse) error
}
type TaskKillRequest struct{}
type TaskKillResponse struct{}
type TaskKillHook interface {
TaskHook
// Kill is called when a task is going to be killed.
Kill(context.Context, *TaskKillRequest, *TaskKillResponse) error
}
type TaskExitedRequest struct{}
type TaskExitedResponse struct{}
type TaskExitedHook interface {
TaskHook
// Exited is called when a task exits and may or may not be restarted.
Exited(context.Context, *TaskExitedRequest, *TaskExitedResponse) error
}
type TaskUpdateRequest struct {
VaultToken string
// Alloc is the current version of the allocation (may have been
// updated since the hook was created)
Alloc *structs.Allocation
// TaskEnv is the task's environment
TaskEnv *env.TaskEnv
}
type TaskUpdateResponse struct{}
type TaskUpdateHook interface {
TaskHook
Update(context.Context, *TaskUpdateRequest, *TaskUpdateResponse) error
}
type TaskStopRequest struct{}
type TaskStopResponse struct{}
type TaskStopHook interface {
TaskHook
// Stop is called after the task has exited and will not be started again.
Stop(context.Context, *TaskStopRequest, *TaskStopResponse) error
}

View File

@ -0,0 +1,56 @@
package allocrunner
import (
"context"
"fmt"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/allocwatcher"
)
// diskMigrationHook migrates ephemeral disk volumes. Depends on alloc dir
// being built but must be run before anything else manipulates the alloc dir.
type diskMigrationHook struct {
allocDir *allocdir.AllocDir
allocWatcher allocwatcher.PrevAllocWatcher
logger log.Logger
}
func newDiskMigrationHook(logger log.Logger, allocWatcher allocwatcher.PrevAllocWatcher, allocDir *allocdir.AllocDir) *diskMigrationHook {
h := &diskMigrationHook{
allocDir: allocDir,
allocWatcher: allocWatcher,
}
h.logger = logger.Named(h.Name())
return h
}
func (h *diskMigrationHook) Name() string {
return "migrate_disk"
}
func (h *diskMigrationHook) Prerun(ctx context.Context) error {
// Wait for a previous alloc - if any - to terminate
if err := h.allocWatcher.Wait(ctx); err != nil {
return err
}
// Wait for data to be migrated from a previous alloc if applicable
if err := h.allocWatcher.Migrate(ctx, h.allocDir); err != nil {
if err == context.Canceled {
return err
}
// Soft-fail on migration errors
h.logger.Warn("error migrating data from previous alloc", "error", err)
// Recreate alloc dir to ensure a clean slate
h.allocDir.Destroy()
if err := h.allocDir.Build(); err != nil {
return fmt.Errorf("failed to clean task directories after failed migration: %v", err)
}
}
return nil
}

View File

@ -0,0 +1,61 @@
package state
import (
"time"
"github.com/hashicorp/nomad/nomad/structs"
)
// State captures the state of the allocation runner.
type State struct {
// ClientStatus captures the overall state of the allocation
ClientStatus string
// ClientDescription is an optional human readable description of the
// allocations client state
ClientDescription string
// DeploymentStatus captures the status of the deployment
DeploymentStatus *structs.AllocDeploymentStatus
// TaskStates is a snapshot of task states.
TaskStates map[string]*structs.TaskState
}
// SetDeploymentStatus is a helper for updating the client-controlled
// DeploymentStatus fields: Healthy and Timestamp. The Canary and ModifyIndex
// fields should only be updated by the server.
func (s *State) SetDeploymentStatus(timestamp time.Time, healthy bool) {
if s.DeploymentStatus == nil {
s.DeploymentStatus = &structs.AllocDeploymentStatus{}
}
s.DeploymentStatus.Healthy = &healthy
s.DeploymentStatus.Timestamp = timestamp
}
// ClearDeploymentStatus is a helper to clear the client-controlled
// DeploymentStatus fields: Healthy and Timestamp. The Canary and ModifyIndex
// fields should only be updated by the server.
func (s *State) ClearDeploymentStatus() {
if s.DeploymentStatus == nil {
return
}
s.DeploymentStatus.Healthy = nil
s.DeploymentStatus.Timestamp = time.Time{}
}
// Copy returns a deep copy of State.
func (s *State) Copy() *State {
taskStates := make(map[string]*structs.TaskState, len(s.TaskStates))
for k, v := range s.TaskStates {
taskStates[k] = v.Copy()
}
return &State{
ClientStatus: s.ClientStatus,
ClientDescription: s.ClientDescription,
DeploymentStatus: s.DeploymentStatus.Copy(),
TaskStates: taskStates,
}
}

View File

@ -0,0 +1,52 @@
package taskrunner
import (
"context"
"fmt"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/getter"
ti "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/nomad/structs"
)
// artifactHook downloads artifacts for a task.
type artifactHook struct {
eventEmitter ti.EventEmitter
logger log.Logger
}
func newArtifactHook(e ti.EventEmitter, logger log.Logger) *artifactHook {
h := &artifactHook{
eventEmitter: e,
}
h.logger = logger.Named(h.Name())
return h
}
func (*artifactHook) Name() string {
return "artifacts"
}
func (h *artifactHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
if len(req.Task.Artifacts) == 0 {
resp.Done = true
return nil
}
h.eventEmitter.EmitEvent(structs.NewTaskEvent(structs.TaskDownloadingArtifacts))
for _, artifact := range req.Task.Artifacts {
//XXX add ctx to GetArtifact to allow cancelling long downloads
if err := getter.GetArtifact(req.TaskEnv, artifact, req.TaskDir.Dir); err != nil {
wrapped := fmt.Errorf("failed to download artifact %q: %v", artifact.GetterSource, err)
h.logger.Debug(wrapped.Error())
h.eventEmitter.EmitEvent(structs.NewTaskEvent(structs.TaskArtifactDownloadFailed).SetDownloadError(wrapped))
return wrapped
}
}
resp.Done = true
return nil
}

View File

@ -0,0 +1,71 @@
package taskrunner
import (
"context"
"io/ioutil"
"os"
"path/filepath"
"github.com/golang/snappy"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/nomad/structs"
)
// dispatchHook writes a dispatch payload to the task dir
type dispatchHook struct {
payload []byte
logger hclog.Logger
}
func newDispatchHook(alloc *structs.Allocation, logger hclog.Logger) *dispatchHook {
h := &dispatchHook{
payload: alloc.Job.Payload,
}
h.logger = logger.Named(h.Name())
return h
}
func (*dispatchHook) Name() string {
return "dispatch_payload"
}
func (h *dispatchHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
if len(h.payload) == 0 || req.Task.DispatchPayload == nil || req.Task.DispatchPayload.File == "" {
// No dispatch payload
resp.Done = true
return nil
}
err := writeDispatchPayload(req.TaskDir.LocalDir, req.Task.DispatchPayload.File, h.payload)
if err != nil {
return err
}
h.logger.Trace("dispatch payload written",
"path", req.TaskDir.LocalDir,
"filename", req.Task.DispatchPayload.File,
"bytes", len(h.payload),
)
// Dispatch payload written successfully; mark as done
resp.Done = true
return nil
}
// writeDispatchPayload writes the payload to the given file or returns an
// error.
func writeDispatchPayload(base, filename string, payload []byte) error {
renderTo := filepath.Join(base, filename)
decoded, err := snappy.Decode(nil, payload)
if err != nil {
return err
}
if err := os.MkdirAll(filepath.Dir(renderTo), 0777); err != nil {
return err
}
return ioutil.WriteFile(renderTo, decoded, 0777)
}

View File

@ -0,0 +1,144 @@
package taskrunner
import (
"context"
"io/ioutil"
"path/filepath"
"testing"
"github.com/golang/snappy"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/require"
)
// Statically assert the stats hook implements the expected interfaces
var _ interfaces.TaskPrestartHook = (*dispatchHook)(nil)
// TestTaskRunner_DispatchHook_NoPayload asserts that the hook is a noop and is
// marked as done if there is no dispatch payload.
func TestTaskRunner_DispatchHook_NoPayload(t *testing.T) {
t.Parallel()
require := require.New(t)
ctx := context.Background()
logger := testlog.HCLogger(t)
allocDir := allocdir.NewAllocDir(logger, "nomadtest_nopayload")
defer allocDir.Destroy()
// Default mock alloc/job is not a dispatch job
alloc := mock.BatchAlloc()
task := alloc.Job.TaskGroups[0].Tasks[0]
taskDir := allocDir.NewTaskDir(task.Name)
require.NoError(taskDir.Build(false, nil, cstructs.FSIsolationNone))
h := newDispatchHook(alloc, logger)
req := interfaces.TaskPrestartRequest{
Task: task,
TaskDir: taskDir,
}
resp := interfaces.TaskPrestartResponse{}
// Assert no error and Done=true as this job has no payload
require.NoError(h.Prestart(ctx, &req, &resp))
require.True(resp.Done)
// Assert payload directory is empty
files, err := ioutil.ReadDir(req.TaskDir.LocalDir)
require.NoError(err)
require.Empty(files)
}
// TestTaskRunner_DispatchHook_Ok asserts that dispatch payloads are written to
// a file in the task dir.
func TestTaskRunner_DispatchHook_Ok(t *testing.T) {
t.Parallel()
require := require.New(t)
ctx := context.Background()
logger := testlog.HCLogger(t)
allocDir := allocdir.NewAllocDir(logger, "nomadtest_dispatchok")
defer allocDir.Destroy()
// Default mock alloc/job is not a dispatch job; update it
alloc := mock.BatchAlloc()
alloc.Job.ParameterizedJob = &structs.ParameterizedJobConfig{
Payload: structs.DispatchPayloadRequired,
}
expected := []byte("hello world")
alloc.Job.Payload = snappy.Encode(nil, expected)
// Set the filename and create the task dir
task := alloc.Job.TaskGroups[0].Tasks[0]
task.DispatchPayload = &structs.DispatchPayloadConfig{
File: "out",
}
taskDir := allocDir.NewTaskDir(task.Name)
require.NoError(taskDir.Build(false, nil, cstructs.FSIsolationNone))
h := newDispatchHook(alloc, logger)
req := interfaces.TaskPrestartRequest{
Task: task,
TaskDir: taskDir,
}
resp := interfaces.TaskPrestartResponse{}
require.NoError(h.Prestart(ctx, &req, &resp))
require.True(resp.Done)
filename := filepath.Join(req.TaskDir.LocalDir, task.DispatchPayload.File)
result, err := ioutil.ReadFile(filename)
require.NoError(err)
require.Equal(expected, result)
}
// TestTaskRunner_DispatchHook_Error asserts that on an error dispatch payloads
// are not written and Done=false.
func TestTaskRunner_DispatchHook_Error(t *testing.T) {
t.Parallel()
require := require.New(t)
ctx := context.Background()
logger := testlog.HCLogger(t)
allocDir := allocdir.NewAllocDir(logger, "nomadtest_dispatcherr")
defer allocDir.Destroy()
// Default mock alloc/job is not a dispatch job; update it
alloc := mock.BatchAlloc()
alloc.Job.ParameterizedJob = &structs.ParameterizedJobConfig{
Payload: structs.DispatchPayloadRequired,
}
// Cause an error by not snappy encoding the payload
alloc.Job.Payload = []byte("hello world")
// Set the filename and create the task dir
task := alloc.Job.TaskGroups[0].Tasks[0]
task.DispatchPayload = &structs.DispatchPayloadConfig{
File: "out",
}
taskDir := allocDir.NewTaskDir(task.Name)
require.NoError(taskDir.Build(false, nil, cstructs.FSIsolationNone))
h := newDispatchHook(alloc, logger)
req := interfaces.TaskPrestartRequest{
Task: task,
TaskDir: taskDir,
}
resp := interfaces.TaskPrestartResponse{}
// Assert an error was returned and Done=false
require.Error(h.Prestart(ctx, &req, &resp))
require.False(resp.Done)
// Assert payload directory is empty
files, err := ioutil.ReadDir(req.TaskDir.LocalDir)
require.NoError(err)
require.Empty(files)
}

View File

@ -0,0 +1,66 @@
package taskrunner
import (
"context"
"time"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
// NewDriverHandle returns a handle for task operations on a specific task
func NewDriverHandle(driver drivers.DriverPlugin, taskID string, task *structs.Task, net *cstructs.DriverNetwork) *DriverHandle {
return &DriverHandle{
driver: driver,
net: net,
taskID: taskID,
task: task,
}
}
// DriverHandle encapsulates a driver plugin client and task identifier and exposes
// an api to perform driver operations on the task
type DriverHandle struct {
driver drivers.DriverPlugin
net *cstructs.DriverNetwork
task *structs.Task
taskID string
}
func (h *DriverHandle) ID() string {
return h.taskID
}
func (h *DriverHandle) WaitCh(ctx context.Context) (<-chan *drivers.ExitResult, error) {
return h.driver.WaitTask(ctx, h.taskID)
}
func (h *DriverHandle) Update(task *structs.Task) error {
return nil
}
func (h *DriverHandle) Kill() error {
return h.driver.StopTask(h.taskID, h.task.KillTimeout, h.task.KillSignal)
}
func (h *DriverHandle) Stats() (*cstructs.TaskResourceUsage, error) {
return h.driver.TaskStats(h.taskID)
}
func (h *DriverHandle) Signal(s string) error {
return h.driver.SignalTask(h.taskID, s)
}
func (h *DriverHandle) Exec(timeout time.Duration, cmd string, args []string) ([]byte, int, error) {
command := append([]string{cmd}, args...)
res, err := h.driver.ExecTask(h.taskID, command, timeout)
if err != nil {
return nil, 0, err
}
return res.Stdout, res.ExitResult.ExitCode, res.ExitResult.Err
}
func (h *DriverHandle) Network() *cstructs.DriverNetwork {
return h.net
}

View File

@ -0,0 +1,11 @@
package taskrunner
import "errors"
const (
errTaskNotRunning = "Task not running"
)
var (
ErrTaskNotRunning = errors.New(errTaskNotRunning)
)

View File

@ -0,0 +1,7 @@
package interfaces
import "github.com/hashicorp/nomad/nomad/structs"
type EventEmitter interface {
EmitEvent(event *structs.TaskEvent)
}

View File

@ -0,0 +1,11 @@
package interfaces
import (
"time"
)
// 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(timeout time.Duration, cmd string, args []string) ([]byte, int, error)
}

View File

@ -0,0 +1,13 @@
package interfaces
import (
"context"
"github.com/hashicorp/nomad/nomad/structs"
)
type TaskLifecycle interface {
Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error
Signal(event *structs.TaskEvent, signal string) error
Kill(ctx context.Context, event *structs.TaskEvent) error
}

View File

@ -0,0 +1,115 @@
package taskrunner
import (
"context"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
// Restart a task. Returns immediately if no task is running. Blocks until
// existing task exits or passed-in context is canceled.
func (tr *TaskRunner) Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error {
// Grab the handle
handle := tr.getDriverHandle()
// Check it is running
if handle == nil {
return ErrTaskNotRunning
}
// Emit the event since it may take a long time to kill
tr.EmitEvent(event)
// Tell the restart tracker that a restart triggered the exit
tr.restartTracker.SetRestartTriggered(failure)
// Kill the task using an exponential backoff in-case of failures.
destroySuccess, err := tr.handleDestroy(handle)
if !destroySuccess {
// We couldn't successfully destroy the resource created.
tr.logger.Error("failed to kill task. Resources may have been leaked", "error", err)
}
// Drain the wait channel or wait for the request context to be canceled
waitCh, err := handle.WaitCh(ctx)
if err != nil {
return err
}
<-waitCh
return nil
}
func (tr *TaskRunner) Signal(event *structs.TaskEvent, s string) error {
// Grab the handle
handle := tr.getDriverHandle()
// Check it is running
if handle == nil {
return ErrTaskNotRunning
}
// Emit the event
tr.EmitEvent(event)
// Send the signal
return handle.Signal(s)
}
// Kill a task. Blocks until task exits or context is canceled. State is set to
// dead.
func (tr *TaskRunner) Kill(ctx context.Context, event *structs.TaskEvent) error {
// Cancel the task runner to break out of restart delay or the main run
// loop.
tr.ctxCancel()
// Grab the handle
handle := tr.getDriverHandle()
// Check it is running
if handle == nil {
return ErrTaskNotRunning
}
// Emit the event since it may take a long time to kill
tr.EmitEvent(event)
// Run the hooks prior to killing the task
tr.kill()
// Tell the restart tracker that the task has been killed
tr.restartTracker.SetKilled()
// Kill the task using an exponential backoff in-case of failures.
destroySuccess, destroyErr := tr.handleDestroy(handle)
if !destroySuccess {
// We couldn't successfully destroy the resource created.
tr.logger.Error("failed to kill task. Resources may have been leaked", "error", destroyErr)
}
// Block until task has exited.
waitCh, err := handle.WaitCh(ctx)
// The error should be nil or TaskNotFound, if it's something else then a
// failure in the driver or transport layer occurred
if err != nil {
if err == drivers.ErrTaskNotFound {
return nil
}
tr.logger.Error("failed to wait on task. Resources may have been leaked", "error", err)
return err
}
<-waitCh
// Store that the task has been destroyed and any associated error.
tr.UpdateState(structs.TaskStateDead, structs.NewTaskEvent(structs.TaskKilled).SetKillError(destroyErr))
if destroyErr != nil {
return destroyErr
} else if err := ctx.Err(); err != nil {
return err
}
return nil
}

View File

@ -0,0 +1,109 @@
package taskrunner
import (
"context"
"fmt"
"path/filepath"
"runtime"
hclog "github.com/hashicorp/go-hclog"
plugin "github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/logmon"
"github.com/hashicorp/nomad/helper/uuid"
)
// logmonHook launches logmon and manages task logging
type logmonHook struct {
// logmon is the handle to the log monitor process for the task.
logmon logmon.LogMon
logmonPluginClient *plugin.Client
config *logmonHookConfig
logger hclog.Logger
}
type logmonHookConfig struct {
logDir string
stdoutFifo string
stderrFifo string
}
func newLogMonHook(cfg *logmonHookConfig, logger hclog.Logger) *logmonHook {
hook := &logmonHook{
config: cfg,
logger: logger,
}
return hook
}
func newLogMonHookConfig(taskName, logDir string) *logmonHookConfig {
cfg := &logmonHookConfig{
logDir: logDir,
}
if runtime.GOOS == "windows" {
id := uuid.Generate()[:8]
cfg.stdoutFifo = fmt.Sprintf("//./pipe/%s-%s.stdout", taskName, id)
cfg.stderrFifo = fmt.Sprintf("//./pipe/%s-%s.stderr", taskName, id)
} else {
cfg.stdoutFifo = filepath.Join(logDir, fmt.Sprintf(".%s.stdout.fifo", taskName))
cfg.stderrFifo = filepath.Join(logDir, fmt.Sprintf(".%s.stderr.fifo", taskName))
}
return cfg
}
func (*logmonHook) Name() string {
return "logmon"
}
func (h *logmonHook) launchLogMon() error {
l, c, err := logmon.LaunchLogMon(h.logger)
if err != nil {
return err
}
h.logmon = l
h.logmonPluginClient = c
return nil
}
func (h *logmonHook) Prestart(ctx context.Context,
req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
// Launch logmon instance for the task.
if err := h.launchLogMon(); err != nil {
h.logger.Error("failed to launch logmon process", "error", err)
return err
}
err := h.logmon.Start(&logmon.LogConfig{
LogDir: h.config.logDir,
StdoutLogFile: fmt.Sprintf("%s.stdout", req.Task.Name),
StderrLogFile: fmt.Sprintf("%s.stderr", req.Task.Name),
StdoutFifo: h.config.stdoutFifo,
StderrFifo: h.config.stderrFifo,
MaxFiles: req.Task.LogConfig.MaxFiles,
MaxFileSizeMB: req.Task.LogConfig.MaxFileSizeMB,
})
if err != nil {
h.logger.Error("failed to start logmon", "error", err)
return err
}
resp.Done = true
return nil
}
func (h *logmonHook) Stop(context.Context, *interfaces.TaskStopRequest, *interfaces.TaskStopResponse) error {
if h.logmon != nil {
h.logmon.Stop()
}
if h.logmonPluginClient != nil {
h.logmonPluginClient.Kill()
}
return nil
}

View File

@ -0,0 +1,7 @@
package taskrunner
import "github.com/hashicorp/nomad/client/allocrunner/interfaces"
// Statically assert the logmon hook implements the expected interfaces
var _ interfaces.TaskPrestartHook = (*logmonHook)(nil)
var _ interfaces.TaskStopHook = (*logmonHook)(nil)

View File

@ -6,8 +6,8 @@ import (
"sync"
"time"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
const (
@ -34,8 +34,9 @@ func NewRestartTracker(policy *structs.RestartPolicy, jobType string) *RestartTr
}
type RestartTracker struct {
waitRes *dstructs.WaitResult
exitRes *drivers.ExitResult
startErr error
killed bool // Whether the task has been killed
restartTriggered bool // Whether the task has been signalled to be restarted
failure bool // Whether a failure triggered the restart
count int // Current number of attempts.
@ -71,11 +72,11 @@ func (r *RestartTracker) SetStartError(err error) *RestartTracker {
return r
}
// SetWaitResult is used to mark the most recent wait result.
func (r *RestartTracker) SetWaitResult(res *dstructs.WaitResult) *RestartTracker {
// SetExitResult is used to mark the most recent wait result.
func (r *RestartTracker) SetExitResult(res *drivers.ExitResult) *RestartTracker {
r.lock.Lock()
defer r.lock.Unlock()
r.waitRes = res
r.exitRes = res
r.failure = true
return r
}
@ -95,6 +96,14 @@ func (r *RestartTracker) SetRestartTriggered(failure bool) *RestartTracker {
return r
}
// SetKilled is used to mark that the task has been killed.
func (r *RestartTracker) SetKilled() *RestartTracker {
r.lock.Lock()
defer r.lock.Unlock()
r.killed = true
return r
}
// GetReason returns a human-readable description for the last state returned by
// GetState.
func (r *RestartTracker) GetReason() string {
@ -103,6 +112,13 @@ func (r *RestartTracker) GetReason() string {
return r.reason
}
// GetCount returns the current restart count
func (r *RestartTracker) GetCount() int {
r.lock.Lock()
defer r.lock.Unlock()
return r.count
}
// GetState returns the tasks next state given the set exit code and start
// error. One of the following states are returned:
// * TaskRestarting - Task should be restarted
@ -120,11 +136,18 @@ func (r *RestartTracker) GetState() (string, time.Duration) {
// Clear out the existing state
defer func() {
r.startErr = nil
r.waitRes = nil
r.exitRes = nil
r.restartTriggered = false
r.failure = false
r.killed = false
}()
// Hot path if task was killed
if r.killed {
r.reason = ""
return structs.TaskKilled, 0
}
// Hot path if a restart was triggered
if r.restartTriggered {
r.reason = ""
@ -137,7 +160,7 @@ func (r *RestartTracker) GetState() (string, time.Duration) {
// If the task does not restart on a successful exit code and
// the exit code was successful: terminate.
if !r.onSuccess && r.waitRes != nil && r.waitRes.Successful() {
if !r.onSuccess && r.exitRes != nil && r.exitRes.Successful() {
return structs.TaskTerminated, 0
}
@ -167,10 +190,10 @@ func (r *RestartTracker) GetState() (string, time.Duration) {
r.reason = ReasonUnrecoverableErrror
return structs.TaskNotRestarting, 0
}
} else if r.waitRes != nil {
} else if r.exitRes != nil {
// If the task started successfully and restart on success isn't specified,
// don't restart but don't mark as failed.
if r.waitRes.Successful() && !r.onSuccess {
if r.exitRes.Successful() && !r.onSuccess {
r.reason = "Restart unnecessary as task terminated successfully"
return structs.TaskTerminated, 0
}

View File

@ -5,8 +5,8 @@ import (
"testing"
"time"
cstructs "github.com/hashicorp/nomad/client/driver/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
func testPolicy(success bool, mode string) *structs.RestartPolicy {
@ -25,8 +25,10 @@ func withinJitter(expected, actual time.Duration) bool {
expected.Nanoseconds()) <= jitter
}
func testWaitResult(exit int) *cstructs.WaitResult {
return cstructs.NewWaitResult(exit, 0, nil)
func testExitResult(exit int) *drivers.ExitResult {
return &drivers.ExitResult{
ExitCode: exit,
}
}
func TestClient_RestartTracker_ModeDelay(t *testing.T) {
@ -34,7 +36,7 @@ func TestClient_RestartTracker_ModeDelay(t *testing.T) {
p := testPolicy(true, structs.RestartPolicyModeDelay)
rt := NewRestartTracker(p, structs.JobTypeService)
for i := 0; i < p.Attempts; i++ {
state, when := rt.SetWaitResult(testWaitResult(127)).GetState()
state, when := rt.SetExitResult(testExitResult(127)).GetState()
if state != structs.TaskRestarting {
t.Fatalf("NextRestart() returned %v, want %v", state, structs.TaskRestarting)
}
@ -45,7 +47,7 @@ func TestClient_RestartTracker_ModeDelay(t *testing.T) {
// Follow up restarts should cause delay.
for i := 0; i < 3; i++ {
state, when := rt.SetWaitResult(testWaitResult(127)).GetState()
state, when := rt.SetExitResult(testExitResult(127)).GetState()
if state != structs.TaskRestarting {
t.Fail()
}
@ -60,7 +62,7 @@ func TestClient_RestartTracker_ModeFail(t *testing.T) {
p := testPolicy(true, structs.RestartPolicyModeFail)
rt := NewRestartTracker(p, structs.JobTypeSystem)
for i := 0; i < p.Attempts; i++ {
state, when := rt.SetWaitResult(testWaitResult(127)).GetState()
state, when := rt.SetExitResult(testExitResult(127)).GetState()
if state != structs.TaskRestarting {
t.Fatalf("NextRestart() returned %v, want %v", state, structs.TaskRestarting)
}
@ -70,7 +72,7 @@ func TestClient_RestartTracker_ModeFail(t *testing.T) {
}
// Next restart should cause fail
if state, _ := rt.SetWaitResult(testWaitResult(127)).GetState(); state != structs.TaskNotRestarting {
if state, _ := rt.SetExitResult(testExitResult(127)).GetState(); state != structs.TaskNotRestarting {
t.Fatalf("NextRestart() returned %v; want %v", state, structs.TaskNotRestarting)
}
}
@ -79,7 +81,7 @@ func TestClient_RestartTracker_NoRestartOnSuccess(t *testing.T) {
t.Parallel()
p := testPolicy(false, structs.RestartPolicyModeDelay)
rt := NewRestartTracker(p, structs.JobTypeBatch)
if state, _ := rt.SetWaitResult(testWaitResult(0)).GetState(); state != structs.TaskTerminated {
if state, _ := rt.SetExitResult(testExitResult(0)).GetState(); state != structs.TaskTerminated {
t.Fatalf("NextRestart() returned %v, expected: %v", state, structs.TaskTerminated)
}
}
@ -91,32 +93,42 @@ func TestClient_RestartTracker_ZeroAttempts(t *testing.T) {
// Test with a non-zero exit code
rt := NewRestartTracker(p, structs.JobTypeService)
if state, when := rt.SetWaitResult(testWaitResult(1)).GetState(); state != structs.TaskNotRestarting {
if state, when := rt.SetExitResult(testExitResult(1)).GetState(); state != structs.TaskNotRestarting {
t.Fatalf("expect no restart, got restart/delay: %v/%v", state, when)
}
// Even with a zero (successful) exit code non-batch jobs should exit
// with TaskNotRestarting
rt = NewRestartTracker(p, structs.JobTypeService)
if state, when := rt.SetWaitResult(testWaitResult(0)).GetState(); state != structs.TaskNotRestarting {
if state, when := rt.SetExitResult(testExitResult(0)).GetState(); state != structs.TaskNotRestarting {
t.Fatalf("expect no restart, got restart/delay: %v/%v", state, when)
}
// Batch jobs with a zero exit code and 0 attempts *do* exit cleanly
// with Terminated
rt = NewRestartTracker(p, structs.JobTypeBatch)
if state, when := rt.SetWaitResult(testWaitResult(0)).GetState(); state != structs.TaskTerminated {
if state, when := rt.SetExitResult(testExitResult(0)).GetState(); state != structs.TaskTerminated {
t.Fatalf("expect terminated, got restart/delay: %v/%v", state, when)
}
// Batch jobs with a non-zero exit code and 0 attempts exit with
// TaskNotRestarting
rt = NewRestartTracker(p, structs.JobTypeBatch)
if state, when := rt.SetWaitResult(testWaitResult(1)).GetState(); state != structs.TaskNotRestarting {
if state, when := rt.SetExitResult(testExitResult(1)).GetState(); state != structs.TaskNotRestarting {
t.Fatalf("expect no restart, got restart/delay: %v/%v", state, when)
}
}
func TestClient_RestartTracker_TaskKilled(t *testing.T) {
t.Parallel()
p := testPolicy(true, structs.RestartPolicyModeFail)
p.Attempts = 0
rt := NewRestartTracker(p, structs.JobTypeService)
if state, when := rt.SetKilled().GetState(); state != structs.TaskKilled && when != 0 {
t.Fatalf("expect no restart; got %v %v", state, when)
}
}
func TestClient_RestartTracker_RestartTriggered(t *testing.T) {
t.Parallel()
p := testPolicy(true, structs.RestartPolicyModeFail)

View File

@ -0,0 +1,199 @@
package taskrunner
import (
"context"
"fmt"
"sync"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
tinterfaces "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/consul"
"github.com/hashicorp/nomad/client/driver/env"
cstructs "github.com/hashicorp/nomad/client/structs"
agentconsul "github.com/hashicorp/nomad/command/agent/consul"
"github.com/hashicorp/nomad/nomad/structs"
)
type serviceHookConfig struct {
alloc *structs.Allocation
task *structs.Task
consul consul.ConsulServiceAPI
// Restarter is a subset of the TaskLifecycle interface
restarter agentconsul.TaskRestarter
logger log.Logger
}
type serviceHook struct {
consul consul.ConsulServiceAPI
allocID string
taskName string
restarter agentconsul.TaskRestarter
logger log.Logger
// The following fields may be updated
driverExec tinterfaces.ScriptExecutor
driverNet *cstructs.DriverNetwork
canary bool
services []*structs.Service
networks structs.Networks
taskEnv *env.TaskEnv
// Since Update() may be called concurrently with any other hook all
// hook methods must be fully serialized
mu sync.Mutex
}
func newServiceHook(c serviceHookConfig) *serviceHook {
h := &serviceHook{
consul: c.consul,
allocID: c.alloc.ID,
taskName: c.task.Name,
services: c.task.Services,
restarter: c.restarter,
}
if res := c.alloc.TaskResources[c.task.Name]; res != nil {
h.networks = res.Networks
}
if c.alloc.DeploymentStatus != nil && c.alloc.DeploymentStatus.Canary {
h.canary = true
}
h.logger = c.logger.Named(h.Name())
return h
}
func (h *serviceHook) Name() string {
return "consul_services"
}
func (h *serviceHook) Poststart(ctx context.Context, req *interfaces.TaskPoststartRequest, _ *interfaces.TaskPoststartResponse) error {
h.mu.Lock()
defer h.mu.Unlock()
// Store the TaskEnv for interpolating now and when Updating
h.driverExec = req.DriverExec
h.driverNet = req.DriverNetwork
h.taskEnv = req.TaskEnv
// Create task services struct with request's driver metadata
taskServices := h.getTaskServices()
return h.consul.RegisterTask(taskServices)
}
func (h *serviceHook) Update(ctx context.Context, req *interfaces.TaskUpdateRequest, _ *interfaces.TaskUpdateResponse) error {
h.mu.Lock()
defer h.mu.Unlock()
// Create old task services struct with request's driver metadata as it
// can't change due to Updates
oldTaskServices := h.getTaskServices()
// Store new updated values out of request
canary := false
if req.Alloc.DeploymentStatus != nil {
canary = req.Alloc.DeploymentStatus.Canary
}
var networks structs.Networks
if res := req.Alloc.TaskResources[h.taskName]; res != nil {
networks = res.Networks
}
task := req.Alloc.LookupTask(h.taskName)
if task == nil {
return fmt.Errorf("task %q not found in updated alloc", h.taskName)
}
// Update service hook fields
h.taskEnv = req.TaskEnv
h.services = task.Services
h.networks = networks
h.canary = canary
// Create new task services struct with those new values
newTaskServices := h.getTaskServices()
return h.consul.UpdateTask(oldTaskServices, newTaskServices)
}
func (h *serviceHook) Exited(context.Context, *interfaces.TaskExitedRequest, *interfaces.TaskExitedResponse) error {
h.mu.Lock()
defer h.mu.Unlock()
taskServices := h.getTaskServices()
h.consul.RemoveTask(taskServices)
// Canary flag may be getting flipped when the alloc is being
// destroyed, so remove both variations of the service
taskServices.Canary = !taskServices.Canary
h.consul.RemoveTask(taskServices)
return nil
}
func (h *serviceHook) getTaskServices() *agentconsul.TaskServices {
// Interpolate with the task's environment
interpolatedServices := interpolateServices(h.taskEnv, h.services)
// Create task services struct with request's driver metadata
return &agentconsul.TaskServices{
AllocID: h.allocID,
Name: h.taskName,
Restarter: h.restarter,
Services: interpolatedServices,
DriverExec: h.driverExec,
DriverNetwork: h.driverNet,
Networks: h.networks,
Canary: h.canary,
}
}
// interpolateServices returns an interpolated copy of services and checks with
// values from the task's environment.
func interpolateServices(taskEnv *env.TaskEnv, services []*structs.Service) []*structs.Service {
interpolated := make([]*structs.Service, len(services))
for i, origService := range services {
// Create a copy as we need to reinterpolate every time the
// environment changes
service := origService.Copy()
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)
check.Method = taskEnv.ReplaceEnv(check.Method)
check.GRPCService = taskEnv.ReplaceEnv(check.GRPCService)
if len(check.Header) > 0 {
header := make(map[string][]string, len(check.Header))
for k, vs := range check.Header {
newVals := make([]string, len(vs))
for i, v := range vs {
newVals[i] = taskEnv.ReplaceEnv(v)
}
header[taskEnv.ReplaceEnv(k)] = newVals
}
check.Header = header
}
}
service.Name = taskEnv.ReplaceEnv(service.Name)
service.PortLabel = taskEnv.ReplaceEnv(service.PortLabel)
service.Tags = taskEnv.ParseAndReplace(service.Tags)
service.CanaryTags = taskEnv.ParseAndReplace(service.CanaryTags)
interpolated[i] = service
}
return interpolated
}

View File

@ -0,0 +1,36 @@
package taskrunner
import (
"context"
"time"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
)
// shutdownDelayHook delays shutting down a task between deregistering it from
// Consul and actually killing it.
type shutdownDelayHook struct {
delay time.Duration
logger log.Logger
}
func newShutdownDelayHook(delay time.Duration, logger log.Logger) *shutdownDelayHook {
h := &shutdownDelayHook{
delay: delay,
}
h.logger = logger.Named(h.Name())
return h
}
func (*shutdownDelayHook) Name() string {
return "shutdown-delay"
}
func (h *shutdownDelayHook) Kill(ctx context.Context, req *interfaces.TaskKillRequest, resp *interfaces.TaskKillResponse) error {
select {
case <-ctx.Done():
case <-time.After(h.delay):
}
return nil
}

View File

@ -0,0 +1,69 @@
package state
import (
"github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/plugins/drivers"
)
// LocalState is Task state which is persisted for use when restarting Nomad
// agents.
type LocalState struct {
Hooks map[string]*HookState
// DriverNetwork is the network information returned by the task
// driver's Start method
DriverNetwork *structs.DriverNetwork
// TaskHandle is the handle used to reattach to the task during recovery
TaskHandle *drivers.TaskHandle
}
func NewLocalState() *LocalState {
return &LocalState{
Hooks: make(map[string]*HookState),
}
}
// Copy should be called with the lock held
func (s *LocalState) Copy() *LocalState {
// Create a copy
c := &LocalState{
Hooks: make(map[string]*HookState, len(s.Hooks)),
DriverNetwork: s.DriverNetwork.Copy(),
TaskHandle: s.TaskHandle.Copy(),
}
// Copy the hooks
for h, state := range s.Hooks {
c.Hooks[h] = state.Copy()
}
return c
}
type HookState struct {
// Prestart is true if the hook has run Prestart successfully and does
// not need to run again
PrestartDone bool
Data map[string]string
}
func (h *HookState) Copy() *HookState {
c := new(HookState)
*c = *h
c.Data = helper.CopyMapStringString(c.Data)
return c
}
func (h *HookState) Equal(o *HookState) bool {
if h == nil || o == nil {
return h == o
}
if h.PrestartDone != o.PrestartDone {
return false
}
return helper.CompareMapStringString(h.Data, o.Data)
}

View File

@ -0,0 +1,120 @@
package taskrunner
import (
"context"
"strings"
"sync"
"time"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/driver"
cstructs "github.com/hashicorp/nomad/client/structs"
)
// StatsUpdater is the interface required by the StatsHook to update stats.
// Satisfied by TaskRunner.
type StatsUpdater interface {
UpdateStats(*cstructs.TaskResourceUsage)
}
// statsHook manages the task stats collection goroutine.
type statsHook struct {
updater StatsUpdater
interval time.Duration
// stopCh is closed by Exited
stopCh chan struct{}
mu sync.Mutex
logger hclog.Logger
}
func newStatsHook(su StatsUpdater, interval time.Duration, logger hclog.Logger) *statsHook {
h := &statsHook{
updater: su,
interval: interval,
}
h.logger = logger.Named(h.Name())
return h
}
func (*statsHook) Name() string {
return "stats_hook"
}
func (h *statsHook) Poststart(ctx context.Context, req *interfaces.TaskPoststartRequest, _ *interfaces.TaskPoststartResponse) error {
h.mu.Lock()
defer h.mu.Unlock()
// This shouldn't happen, but better safe than risk leaking a goroutine
if h.stopCh != nil {
h.logger.Debug("poststart called twice without exiting between")
close(h.stopCh)
}
h.stopCh = make(chan struct{})
go h.collectResourceUsageStats(req.DriverStats, h.stopCh)
return nil
}
func (h *statsHook) Exited(context.Context, *interfaces.TaskExitedRequest, *interfaces.TaskExitedResponse) error {
h.mu.Lock()
defer h.mu.Unlock()
if h.stopCh == nil {
// No stats running
return nil
}
// Close chan to stop stats collection
close(h.stopCh)
// Clear chan so we don't double close for any reason
h.stopCh = nil
return nil
}
// collectResourceUsageStats starts collecting resource usage stats of a Task.
// Collection ends when the passed channel is closed
func (h *statsHook) collectResourceUsageStats(handle interfaces.DriverStats, stopCh <-chan struct{}) {
// start collecting the stats right away and then start collecting every
// collection interval
next := time.NewTimer(0)
defer next.Stop()
for {
select {
case <-next.C:
// Reset the timer
next.Reset(h.interval)
// Collect stats from driver
ru, err := handle.Stats()
if err != nil {
// Check if the driver doesn't implement stats
if err.Error() == driver.DriverStatsNotImplemented.Error() {
h.logger.Debug("driver does not support stats")
return
}
//XXX This is a net/rpc specific error
// We do not log when the plugin is shutdown as this is simply a
// race between the stopCollection channel being closed and calling
// Stats on the handle.
if !strings.Contains(err.Error(), "connection is shut down") {
h.logger.Debug("error fetching stats of task", "error", err)
}
continue
}
// Update stats on TaskRunner and emit them
h.updater.UpdateStats(ru)
case <-stopCh:
return
}
}
}

View File

@ -0,0 +1,180 @@
package taskrunner
import (
"context"
"testing"
"time"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/driver"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/stretchr/testify/require"
)
// Statically assert the stats hook implements the expected interfaces
var _ interfaces.TaskPoststartHook = (*statsHook)(nil)
var _ interfaces.TaskExitedHook = (*statsHook)(nil)
type mockStatsUpdater struct {
// Ch is sent task resource usage updates if not nil
Ch chan *cstructs.TaskResourceUsage
}
// newMockStatsUpdater returns a mockStatsUpdater that blocks on Ch for every
// call to UpdateStats
func newMockStatsUpdater() *mockStatsUpdater {
return &mockStatsUpdater{
Ch: make(chan *cstructs.TaskResourceUsage),
}
}
func (m *mockStatsUpdater) UpdateStats(ru *cstructs.TaskResourceUsage) {
if m.Ch != nil {
m.Ch <- ru
}
}
type mockDriverStats struct {
// err is returned by Stats if it is non-nil
err error
}
func (m *mockDriverStats) Stats() (*cstructs.TaskResourceUsage, error) {
if m.err != nil {
return nil, m.err
}
ru := &cstructs.TaskResourceUsage{
ResourceUsage: &cstructs.ResourceUsage{
MemoryStats: &cstructs.MemoryStats{
RSS: 1,
Measured: []string{"RSS"},
},
CpuStats: &cstructs.CpuStats{
SystemMode: 1,
Measured: []string{"System Mode"},
},
},
Timestamp: time.Now().UnixNano(),
Pids: map[string]*cstructs.ResourceUsage{},
}
ru.Pids["task"] = ru.ResourceUsage
return ru, nil
}
// TestTaskRunner_StatsHook_PoststartExited asserts the stats hook starts and
// stops.
func TestTaskRunner_StatsHook_PoststartExited(t *testing.T) {
t.Parallel()
require := require.New(t)
logger := testlog.HCLogger(t)
su := newMockStatsUpdater()
ds := new(mockDriverStats)
poststartReq := &interfaces.TaskPoststartRequest{DriverStats: ds}
// Create hook
h := newStatsHook(su, time.Minute, logger)
// Always call Exited to cleanup goroutines
defer h.Exited(context.Background(), nil, nil)
// Run prestart
require.NoError(h.Poststart(context.Background(), poststartReq, nil))
// An initial stats collection should run and call the updater
select {
case ru := <-su.Ch:
require.Equal(uint64(1), ru.ResourceUsage.MemoryStats.RSS)
case <-time.After(10 * time.Second):
t.Fatalf("timeout waiting for initial stats collection")
}
require.NoError(h.Exited(context.Background(), nil, nil))
}
// TestTaskRunner_StatsHook_Periodic asserts the stats hook collects stats on
// an interval.
func TestTaskRunner_StatsHook_Periodic(t *testing.T) {
t.Parallel()
require := require.New(t)
logger := testlog.HCLogger(t)
su := newMockStatsUpdater()
ds := new(mockDriverStats)
poststartReq := &interfaces.TaskPoststartRequest{DriverStats: ds}
// interval needs to be high enough that even on a slow/busy VM
// Exited() can complete within the interval.
const interval = 500 * time.Millisecond
h := newStatsHook(su, interval, logger)
defer h.Exited(context.Background(), nil, nil)
// Run prestart
require.NoError(h.Poststart(context.Background(), poststartReq, nil))
// An initial stats collection should run and call the updater
var firstrun int64
select {
case ru := <-su.Ch:
if ru.Timestamp <= 0 {
t.Fatalf("expected nonzero timestamp (%v)", ru.Timestamp)
}
firstrun = ru.Timestamp
case <-time.After(10 * time.Second):
t.Fatalf("timeout waiting for initial stats collection")
}
// Should get another update in ~500ms (see interval above)
select {
case ru := <-su.Ch:
if ru.Timestamp <= firstrun {
t.Fatalf("expected timestamp (%v) after first run (%v)", ru.Timestamp, firstrun)
}
case <-time.After(10 * time.Second):
t.Fatalf("timeout waiting for second stats collection")
}
// Exiting should prevent further updates
require.NoError(h.Exited(context.Background(), nil, nil))
// Should *not* get another update in ~500ms (see interval above)
select {
case ru := <-su.Ch:
t.Fatalf("unexpected update after exit (firstrun=%v; update=%v", firstrun, ru.Timestamp)
case <-time.After(2 * interval):
// Ok! No update after exit as expected.
}
}
// TestTaskRunner_StatsHook_NotImplemented asserts the stats hook stops if the
// driver returns NotImplemented.
func TestTaskRunner_StatsHook_NotImplemented(t *testing.T) {
t.Parallel()
require := require.New(t)
logger := testlog.HCLogger(t)
su := newMockStatsUpdater()
ds := &mockDriverStats{
err: driver.DriverStatsNotImplemented,
}
poststartReq := &interfaces.TaskPoststartRequest{DriverStats: ds}
h := newStatsHook(su, 1, logger)
defer h.Exited(context.Background(), nil, nil)
// Run prestart
require.NoError(h.Poststart(context.Background(), poststartReq, nil))
// An initial stats collection should run and *not* call the updater
select {
case ru := <-su.Ch:
t.Fatalf("unexpected resource update (timestamp=%v)", ru.Timestamp)
case <-time.After(500 * time.Millisecond):
// Ok! No update received because error was returned
}
}

View File

@ -0,0 +1,51 @@
package taskrunner
import (
"context"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
cconfig "github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/nomad/structs"
)
type taskDirHook struct {
runner *TaskRunner
logger log.Logger
}
func newTaskDirHook(runner *TaskRunner, logger log.Logger) *taskDirHook {
td := &taskDirHook{
runner: runner,
}
td.logger = logger.Named(td.Name())
return td
}
func (h *taskDirHook) Name() string {
return "task_dir"
}
func (h *taskDirHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
cc := h.runner.clientConfig
chroot := cconfig.DefaultChrootEnv
if len(cc.ChrootEnv) > 0 {
chroot = cc.ChrootEnv
}
// Emit the event that we are going to be building the task directory
h.runner.EmitEvent(structs.NewTaskEvent(structs.TaskSetup).SetMessage(structs.TaskBuildingTaskDir))
// Build the task directory structure
fsi := h.runner.driverCapabilities.FSIsolation
err := h.runner.taskDir.Build(false, chroot, fsi)
if err != nil {
return err
}
// Update the environment variables based on the built task directory
driver.SetEnvvars(h.runner.envBuilder, fsi, h.runner.taskDir, h.runner.clientConfig)
resp.Done = true
return nil
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,73 @@
package taskrunner
import (
"github.com/hashicorp/nomad/nomad/structs"
)
func (tr *TaskRunner) Alloc() *structs.Allocation {
tr.allocLock.Lock()
defer tr.allocLock.Unlock()
return tr.alloc
}
func (tr *TaskRunner) setAlloc(updated *structs.Allocation) {
tr.allocLock.Lock()
tr.alloc = updated
tr.allocLock.Unlock()
}
func (tr *TaskRunner) Task() *structs.Task {
tr.taskLock.RLock()
defer tr.taskLock.RUnlock()
return tr.task
}
func (tr *TaskRunner) TaskState() *structs.TaskState {
tr.stateLock.Lock()
defer tr.stateLock.Unlock()
return tr.state.Copy()
}
func (tr *TaskRunner) getVaultToken() string {
tr.vaultTokenLock.Lock()
defer tr.vaultTokenLock.Unlock()
return tr.vaultToken
}
// setVaultToken updates the vault token on the task runner as well as in the
// task's environment. These two places must be set atomically to avoid a task
// seeing a different token on the task runner and in its environment.
func (tr *TaskRunner) setVaultToken(token string) {
tr.vaultTokenLock.Lock()
defer tr.vaultTokenLock.Unlock()
// Update the Vault token on the runner
tr.vaultToken = token
// Update the task's environment
tr.envBuilder.SetVaultToken(token, tr.task.Vault.Env)
}
// getDriverHandle returns a driver handle and its result proxy. Use the
// result proxy instead of the handle's WaitCh.
func (tr *TaskRunner) getDriverHandle() *DriverHandle {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
return tr.handle
}
// setDriverHanlde sets the driver handle and creates a new result proxy.
func (tr *TaskRunner) setDriverHandle(handle *DriverHandle) {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
tr.handle = handle
}
func (tr *TaskRunner) clearDriverHandle() {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
if tr.handle != nil {
tr.driver.DestroyTask(tr.handle.ID(), true)
}
tr.handle = nil
}

View File

@ -0,0 +1,392 @@
package taskrunner
import (
"context"
"fmt"
"time"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/state"
"github.com/hashicorp/nomad/nomad/structs"
)
// initHooks intializes the tasks hooks.
func (tr *TaskRunner) initHooks() {
hookLogger := tr.logger.Named("task_hook")
task := tr.Task()
tr.logmonHookConfig = newLogMonHookConfig(task.Name, tr.taskDir.LogDir)
// Create the task directory hook. This is run first to ensure the
// directory path exists for other hooks.
tr.runnerHooks = []interfaces.TaskHook{
newValidateHook(tr.clientConfig, hookLogger),
newTaskDirHook(tr, hookLogger),
newLogMonHook(tr.logmonHookConfig, hookLogger),
newDispatchHook(tr.Alloc(), hookLogger),
newArtifactHook(tr, hookLogger),
newShutdownDelayHook(task.ShutdownDelay, hookLogger),
newStatsHook(tr, tr.clientConfig.StatsCollectionInterval, hookLogger),
}
// If Vault is enabled, add the hook
if task.Vault != nil {
tr.runnerHooks = append(tr.runnerHooks, newVaultHook(&vaultHookConfig{
vaultStanza: task.Vault,
client: tr.vaultClient,
events: tr,
lifecycle: tr,
updater: tr,
logger: hookLogger,
alloc: tr.Alloc(),
task: tr.taskName,
}))
}
// If there are templates is enabled, add the hook
if len(task.Templates) != 0 {
tr.runnerHooks = append(tr.runnerHooks, newTemplateHook(&templateHookConfig{
logger: hookLogger,
lifecycle: tr,
events: tr,
templates: task.Templates,
clientConfig: tr.clientConfig,
envBuilder: tr.envBuilder,
}))
}
// If there are any services, add the hook
if len(task.Services) != 0 {
tr.runnerHooks = append(tr.runnerHooks, newServiceHook(serviceHookConfig{
alloc: tr.Alloc(),
task: tr.Task(),
consul: tr.consulClient,
restarter: tr,
logger: hookLogger,
}))
}
}
// prestart is used to run the runners prestart hooks.
func (tr *TaskRunner) prestart() error {
// Determine if the allocation is terminaland we should avoid running
// prestart hooks.
alloc := tr.Alloc()
if alloc.TerminalStatus() {
tr.logger.Trace("skipping prestart hooks since allocation is terminal")
return nil
}
if tr.logger.IsTrace() {
start := time.Now()
tr.logger.Trace("running prestart hooks", "start", start)
defer func() {
end := time.Now()
tr.logger.Trace("finished prestart hooks", "end", end, "duration", end.Sub(start))
}()
}
for _, hook := range tr.runnerHooks {
pre, ok := hook.(interfaces.TaskPrestartHook)
if !ok {
tr.logger.Trace("skipping non-prestart hook", "name", hook.Name())
continue
}
name := pre.Name()
// Build the request
req := interfaces.TaskPrestartRequest{
Task: tr.Task(),
TaskDir: tr.taskDir,
TaskEnv: tr.envBuilder.Build(),
}
tr.localStateLock.RLock()
origHookState := tr.localState.Hooks[name]
tr.localStateLock.RUnlock()
if origHookState != nil && origHookState.PrestartDone {
tr.logger.Trace("skipping done prestart hook", "name", pre.Name())
continue
}
req.VaultToken = tr.getVaultToken()
// Time the prestart hook
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running prestart hook", "name", name, "start", start)
}
// Run the prestart hook
var resp interfaces.TaskPrestartResponse
if err := pre.Prestart(tr.ctx, &req, &resp); err != nil {
return structs.WrapRecoverable(fmt.Sprintf("prestart hook %q failed: %v", name, err), err)
}
// Store the hook state
{
hookState := &state.HookState{
Data: resp.HookData,
PrestartDone: resp.Done,
}
// Store and persist local state if the hook state has changed
if !hookState.Equal(origHookState) {
tr.localStateLock.Lock()
tr.localState.Hooks[name] = hookState
tr.localStateLock.Unlock()
if err := tr.persistLocalState(); err != nil {
return err
}
}
}
// Store the environment variables returned by the hook
if len(resp.Env) != 0 {
tr.envBuilder.SetGenericEnv(resp.Env)
}
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished prestart hook", "name", name, "end", end, "duration", end.Sub(start))
}
}
return nil
}
// poststart is used to run the runners poststart hooks.
func (tr *TaskRunner) poststart() error {
if tr.logger.IsTrace() {
start := time.Now()
tr.logger.Trace("running poststart hooks", "start", start)
defer func() {
end := time.Now()
tr.logger.Trace("finished poststart hooks", "end", end, "duration", end.Sub(start))
}()
}
handle := tr.getDriverHandle()
net := handle.Network()
var merr multierror.Error
for _, hook := range tr.runnerHooks {
post, ok := hook.(interfaces.TaskPoststartHook)
if !ok {
continue
}
name := post.Name()
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running poststart hook", "name", name, "start", start)
}
req := interfaces.TaskPoststartRequest{
DriverExec: handle,
DriverNetwork: net,
DriverStats: handle,
TaskEnv: tr.envBuilder.Build(),
}
var resp interfaces.TaskPoststartResponse
if err := post.Poststart(tr.ctx, &req, &resp); err != nil {
merr.Errors = append(merr.Errors, fmt.Errorf("poststart hook %q failed: %v", name, err))
}
// No need to persist as PoststartResponse is currently empty
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished poststart hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return merr.ErrorOrNil()
}
// exited is used to run the exited hooks before a task is stopped.
func (tr *TaskRunner) exited() error {
if tr.logger.IsTrace() {
start := time.Now()
tr.logger.Trace("running exited hooks", "start", start)
defer func() {
end := time.Now()
tr.logger.Trace("finished exited hooks", "end", end, "duration", end.Sub(start))
}()
}
var merr multierror.Error
for _, hook := range tr.runnerHooks {
post, ok := hook.(interfaces.TaskExitedHook)
if !ok {
continue
}
name := post.Name()
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running exited hook", "name", name, "start", start)
}
req := interfaces.TaskExitedRequest{}
var resp interfaces.TaskExitedResponse
if err := post.Exited(tr.ctx, &req, &resp); err != nil {
merr.Errors = append(merr.Errors, fmt.Errorf("exited hook %q failed: %v", name, err))
}
// No need to persist as TaskExitedResponse is currently empty
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished exited hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return merr.ErrorOrNil()
}
// stop is used to run the stop hooks.
func (tr *TaskRunner) stop() error {
if tr.logger.IsTrace() {
start := time.Now()
tr.logger.Trace("running stop hooks", "start", start)
defer func() {
end := time.Now()
tr.logger.Trace("finished stop hooks", "end", end, "duration", end.Sub(start))
}()
}
var merr multierror.Error
for _, hook := range tr.runnerHooks {
post, ok := hook.(interfaces.TaskStopHook)
if !ok {
continue
}
name := post.Name()
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running stop hook", "name", name, "start", start)
}
req := interfaces.TaskStopRequest{}
var resp interfaces.TaskStopResponse
if err := post.Stop(tr.ctx, &req, &resp); err != nil {
merr.Errors = append(merr.Errors, fmt.Errorf("stop hook %q failed: %v", name, err))
}
// No need to persist as TaskStopResponse is currently empty
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished stop hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
return merr.ErrorOrNil()
}
// update is used to run the runners update hooks. Should only be called from
// Run(). To trigger an update, update state on the TaskRunner and call
// triggerUpdateHooks.
func (tr *TaskRunner) updateHooks() {
if tr.logger.IsTrace() {
start := time.Now()
tr.logger.Trace("running update hooks", "start", start)
defer func() {
end := time.Now()
tr.logger.Trace("finished update hooks", "end", end, "duration", end.Sub(start))
}()
}
// Prepare state needed by Update hooks
alloc := tr.Alloc()
// Execute Update hooks
for _, hook := range tr.runnerHooks {
upd, ok := hook.(interfaces.TaskUpdateHook)
if !ok {
tr.logger.Trace("skipping non-update hook", "name", hook.Name())
continue
}
name := upd.Name()
// Build the request
req := interfaces.TaskUpdateRequest{
VaultToken: tr.getVaultToken(),
Alloc: alloc,
TaskEnv: tr.envBuilder.Build(),
}
// Time the update hook
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running update hook", "name", name, "start", start)
}
// Run the update hook
var resp interfaces.TaskUpdateResponse
if err := upd.Update(tr.ctx, &req, &resp); err != nil {
tr.logger.Error("update hook failed", "name", name, "error", err)
}
// No need to persist as TaskUpdateResponse is currently empty
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished update hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
}
// kill is used to run the runners kill hooks.
func (tr *TaskRunner) kill() {
if tr.logger.IsTrace() {
start := time.Now()
tr.logger.Trace("running kill hooks", "start", start)
defer func() {
end := time.Now()
tr.logger.Trace("finished kill hooks", "end", end, "duration", end.Sub(start))
}()
}
for _, hook := range tr.runnerHooks {
upd, ok := hook.(interfaces.TaskKillHook)
if !ok {
tr.logger.Trace("skipping non-kill hook", "name", hook.Name())
continue
}
name := upd.Name()
// Time the update hook
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running kill hook", "name", name, "start", start)
}
// Run the update hook
req := interfaces.TaskKillRequest{}
var resp interfaces.TaskKillResponse
if err := upd.Kill(context.Background(), &req, &resp); err != nil {
tr.logger.Error("kill hook failed", "name", name, "error", err)
}
// No need to persist as TaskKillResponse is currently empty
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished kill hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
}

View File

@ -0,0 +1,701 @@
package template
import (
"context"
"fmt"
"math/rand"
"os"
"path/filepath"
"sort"
"strconv"
"strings"
"sync"
"time"
ctconf "github.com/hashicorp/consul-template/config"
"github.com/hashicorp/consul-template/manager"
"github.com/hashicorp/consul-template/signals"
envparse "github.com/hashicorp/go-envparse"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/structs"
)
const (
// consulTemplateSourceName is the source name when using the TaskHooks.
consulTemplateSourceName = "Template"
// hostSrcOption is the Client option that determines whether the template
// source may be from the host
hostSrcOption = "template.allow_host_source"
// missingDepEventLimit is the number of missing dependencies that will be
// logged before we switch to showing just the number of missing
// dependencies.
missingDepEventLimit = 3
// DefaultMaxTemplateEventRate is the default maximum rate at which a
// template event should be fired.
DefaultMaxTemplateEventRate = 3 * time.Second
)
// TaskTemplateManager is used to run a set of templates for a given task
type TaskTemplateManager struct {
// config holds the template managers configuration
config *TaskTemplateManagerConfig
// lookup allows looking up the set of Nomad templates by their consul-template ID
lookup map[string][]*structs.Template
// runner is the consul-template runner
runner *manager.Runner
// signals is a lookup map from the string representation of a signal to its
// actual signal
signals map[string]os.Signal
// shutdownCh is used to signal and started goroutine to shutdown
shutdownCh chan struct{}
// shutdown marks whether the manager has been shutdown
shutdown bool
shutdownLock sync.Mutex
}
// TaskTemplateManagerConfig is used to configure an instance of the
// TaskTemplateManager
type TaskTemplateManagerConfig struct {
// UnblockCh is closed when the template has been rendered
UnblockCh chan struct{}
// Lifecycle is used to interact with the task the template manager is being
// run for
Lifecycle interfaces.TaskLifecycle
// Events is used to emit events for the task
Events interfaces.EventEmitter
// Templates is the set of templates we are managing
Templates []*structs.Template
// ClientConfig is the Nomad Client configuration
ClientConfig *config.Config
// VaultToken is the Vault token for the task.
VaultToken string
// TaskDir is the task's directory
TaskDir string
// EnvBuilder is the environment variable builder for the task.
EnvBuilder *env.Builder
// MaxTemplateEventRate is the maximum rate at which we should emit events.
MaxTemplateEventRate time.Duration
// retryRate is only used for testing and is used to increase the retry rate
retryRate time.Duration
}
// Validate validates the configuration.
func (c *TaskTemplateManagerConfig) Validate() error {
if c == nil {
return fmt.Errorf("Nil config passed")
} else if c.UnblockCh == nil {
return fmt.Errorf("Invalid unblock channel given")
} else if c.Lifecycle == nil {
return fmt.Errorf("Invalid lifecycle hooks given")
} else if c.Events == nil {
return fmt.Errorf("Invalid event hook given")
} else if c.ClientConfig == nil {
return fmt.Errorf("Invalid client config given")
} else if c.TaskDir == "" {
return fmt.Errorf("Invalid task directory given: %q", c.TaskDir)
} else if c.EnvBuilder == nil {
return fmt.Errorf("Invalid task environment given")
} else if c.MaxTemplateEventRate == 0 {
return fmt.Errorf("Invalid max template event rate given")
}
return nil
}
func NewTaskTemplateManager(config *TaskTemplateManagerConfig) (*TaskTemplateManager, error) {
// Check pre-conditions
if err := config.Validate(); err != nil {
return nil, err
}
tm := &TaskTemplateManager{
config: config,
shutdownCh: make(chan struct{}),
}
// Parse the signals that we need
for _, tmpl := range config.Templates {
if tmpl.ChangeSignal == "" {
continue
}
sig, err := signals.Parse(tmpl.ChangeSignal)
if err != nil {
return nil, fmt.Errorf("Failed to parse signal %q", tmpl.ChangeSignal)
}
if tm.signals == nil {
tm.signals = make(map[string]os.Signal)
}
tm.signals[tmpl.ChangeSignal] = sig
}
// Build the consul-template runner
runner, lookup, err := templateRunner(config)
if err != nil {
return nil, err
}
tm.runner = runner
tm.lookup = lookup
go tm.run()
return tm, nil
}
// Stop is used to stop the consul-template runner
func (tm *TaskTemplateManager) Stop() {
tm.shutdownLock.Lock()
defer tm.shutdownLock.Unlock()
if tm.shutdown {
return
}
close(tm.shutdownCh)
tm.shutdown = true
// Stop the consul-template runner
if tm.runner != nil {
tm.runner.Stop()
}
}
// run is the long lived loop that handles errors and templates being rendered
func (tm *TaskTemplateManager) run() {
// Runner is nil if there is no templates
if tm.runner == nil {
// Unblock the start if there is nothing to do
close(tm.config.UnblockCh)
return
}
// Start the runner
go tm.runner.Start()
// Block till all the templates have been rendered
tm.handleFirstRender()
// Detect if there was a shutdown.
select {
case <-tm.shutdownCh:
return
default:
}
// Read environment variables from env templates before we unblock
envMap, err := loadTemplateEnv(tm.config.Templates, tm.config.TaskDir)
if err != nil {
tm.config.Lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template failed to read environment variables: %v", err)))
return
}
tm.config.EnvBuilder.SetTemplateEnv(envMap)
// Unblock the task
close(tm.config.UnblockCh)
// If all our templates are change mode no-op, then we can exit here
if tm.allTemplatesNoop() {
return
}
// handle all subsequent render events.
tm.handleTemplateRerenders(time.Now())
}
// handleFirstRender blocks till all templates have been rendered
func (tm *TaskTemplateManager) handleFirstRender() {
// missingDependencies is the set of missing dependencies.
var missingDependencies map[string]struct{}
// eventTimer is used to trigger the firing of an event showing the missing
// dependencies.
eventTimer := time.NewTimer(tm.config.MaxTemplateEventRate)
if !eventTimer.Stop() {
<-eventTimer.C
}
// outstandingEvent tracks whether there is an outstanding event that should
// be fired.
outstandingEvent := false
// Wait till all the templates have been rendered
WAIT:
for {
select {
case <-tm.shutdownCh:
return
case err, ok := <-tm.runner.ErrCh:
if !ok {
continue
}
tm.config.Lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template failed: %v", err)))
case <-tm.runner.TemplateRenderedCh():
// A template has been rendered, figure out what to do
events := tm.runner.RenderEvents()
// Not all templates have been rendered yet
if len(events) < len(tm.lookup) {
continue
}
for _, event := range events {
// This template hasn't been rendered
if event.LastWouldRender.IsZero() {
continue WAIT
}
}
break WAIT
case <-tm.runner.RenderEventCh():
events := tm.runner.RenderEvents()
joinedSet := make(map[string]struct{})
for _, event := range events {
missing := event.MissingDeps
if missing == nil {
continue
}
for _, dep := range missing.List() {
joinedSet[dep.String()] = struct{}{}
}
}
// Check to see if the new joined set is the same as the old
different := len(joinedSet) != len(missingDependencies)
if !different {
for k := range joinedSet {
if _, ok := missingDependencies[k]; !ok {
different = true
break
}
}
}
// Nothing to do
if !different {
continue
}
// Update the missing set
missingDependencies = joinedSet
// Update the event timer channel
if !outstandingEvent {
// We got new data so reset
outstandingEvent = true
eventTimer.Reset(tm.config.MaxTemplateEventRate)
}
case <-eventTimer.C:
if missingDependencies == nil {
continue
}
// Clear the outstanding event
outstandingEvent = false
// Build the missing set
missingSlice := make([]string, 0, len(missingDependencies))
for k := range missingDependencies {
missingSlice = append(missingSlice, k)
}
sort.Strings(missingSlice)
if l := len(missingSlice); l > missingDepEventLimit {
missingSlice[missingDepEventLimit] = fmt.Sprintf("and %d more", l-missingDepEventLimit)
missingSlice = missingSlice[:missingDepEventLimit+1]
}
missingStr := strings.Join(missingSlice, ", ")
tm.config.Events.EmitEvent(structs.NewTaskEvent(consulTemplateSourceName).SetDisplayMessage(fmt.Sprintf("Missing: %s", missingStr)))
}
}
}
// handleTemplateRerenders is used to handle template render events after they
// have all rendered. It takes action based on which set of templates re-render.
// The passed allRenderedTime is the time at which all templates have rendered.
// This is used to avoid signaling the task for any render event before hand.
func (tm *TaskTemplateManager) handleTemplateRerenders(allRenderedTime time.Time) {
// A lookup for the last time the template was handled
handledRenders := make(map[string]time.Time, len(tm.config.Templates))
for {
select {
case <-tm.shutdownCh:
return
case err, ok := <-tm.runner.ErrCh:
if !ok {
continue
}
tm.config.Lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template failed: %v", err)))
case <-tm.runner.TemplateRenderedCh():
// A template has been rendered, figure out what to do
var handling []string
signals := make(map[string]struct{})
restart := false
var splay time.Duration
events := tm.runner.RenderEvents()
for id, event := range events {
// First time through
if allRenderedTime.After(event.LastDidRender) || allRenderedTime.Equal(event.LastDidRender) {
handledRenders[id] = allRenderedTime
continue
}
// We have already handled this one
if htime := handledRenders[id]; htime.After(event.LastDidRender) || htime.Equal(event.LastDidRender) {
continue
}
// Lookup the template and determine what to do
tmpls, ok := tm.lookup[id]
if !ok {
tm.config.Lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template runner returned unknown template id %q", id)))
return
}
// Read environment variables from templates
envMap, err := loadTemplateEnv(tm.config.Templates, tm.config.TaskDir)
if err != nil {
tm.config.Lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template failed to read environment variables: %v", err)))
return
}
tm.config.EnvBuilder.SetTemplateEnv(envMap)
for _, tmpl := range tmpls {
switch tmpl.ChangeMode {
case structs.TemplateChangeModeSignal:
signals[tmpl.ChangeSignal] = struct{}{}
case structs.TemplateChangeModeRestart:
restart = true
case structs.TemplateChangeModeNoop:
continue
}
if tmpl.Splay > splay {
splay = tmpl.Splay
}
}
handling = append(handling, id)
}
if restart || len(signals) != 0 {
if splay != 0 {
ns := splay.Nanoseconds()
offset := rand.Int63n(ns)
t := time.Duration(offset)
select {
case <-time.After(t):
case <-tm.shutdownCh:
return
}
}
// Update handle time
for _, id := range handling {
handledRenders[id] = events[id].LastDidRender
}
if restart {
tm.config.Lifecycle.Restart(context.Background(),
structs.NewTaskEvent(structs.TaskRestarting).
SetDisplayMessage("Template with change_mode restart re-rendered"), false)
} else if len(signals) != 0 {
var mErr multierror.Error
for signal := range signals {
s := tm.signals[signal]
event := structs.NewTaskEvent(structs.TaskSignaling).SetTaskSignal(s).SetDisplayMessage("Template re-rendered")
if err := tm.config.Lifecycle.Signal(event, signal); err != nil {
multierror.Append(&mErr, err)
}
}
if err := mErr.ErrorOrNil(); err != nil {
flat := make([]os.Signal, 0, len(signals))
for signal := range signals {
flat = append(flat, tm.signals[signal])
}
tm.config.Lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template failed to send signals %v: %v", flat, err)))
}
}
}
}
}
}
// allTemplatesNoop returns whether all the managed templates have change mode noop.
func (tm *TaskTemplateManager) allTemplatesNoop() bool {
for _, tmpl := range tm.config.Templates {
if tmpl.ChangeMode != structs.TemplateChangeModeNoop {
return false
}
}
return true
}
// templateRunner returns a consul-template runner for the given templates and a
// lookup by destination to the template. If no templates are in the config, a
// nil template runner and lookup is returned.
func templateRunner(config *TaskTemplateManagerConfig) (
*manager.Runner, map[string][]*structs.Template, error) {
if len(config.Templates) == 0 {
return nil, nil, nil
}
// Parse the templates
ctmplMapping, err := parseTemplateConfigs(config)
if err != nil {
return nil, nil, err
}
// Create the runner configuration.
runnerConfig, err := newRunnerConfig(config, ctmplMapping)
if err != nil {
return nil, nil, err
}
runner, err := manager.NewRunner(runnerConfig, false, false)
if err != nil {
return nil, nil, err
}
// Set Nomad's environment variables
runner.Env = config.EnvBuilder.Build().All()
// Build the lookup
idMap := runner.TemplateConfigMapping()
lookup := make(map[string][]*structs.Template, len(idMap))
for id, ctmpls := range idMap {
for _, ctmpl := range ctmpls {
templates := lookup[id]
templates = append(templates, ctmplMapping[ctmpl])
lookup[id] = templates
}
}
return runner, lookup, nil
}
// parseTemplateConfigs converts the tasks templates in the config into
// consul-templates
func parseTemplateConfigs(config *TaskTemplateManagerConfig) (map[ctconf.TemplateConfig]*structs.Template, error) {
allowAbs := config.ClientConfig.ReadBoolDefault(hostSrcOption, true)
taskEnv := config.EnvBuilder.Build()
ctmpls := make(map[ctconf.TemplateConfig]*structs.Template, len(config.Templates))
for _, tmpl := range config.Templates {
var src, dest string
if tmpl.SourcePath != "" {
if filepath.IsAbs(tmpl.SourcePath) {
if !allowAbs {
return nil, fmt.Errorf("Specifying absolute template paths disallowed by client config: %q", tmpl.SourcePath)
}
src = tmpl.SourcePath
} else {
src = filepath.Join(config.TaskDir, taskEnv.ReplaceEnv(tmpl.SourcePath))
}
}
if tmpl.DestPath != "" {
dest = filepath.Join(config.TaskDir, taskEnv.ReplaceEnv(tmpl.DestPath))
}
ct := ctconf.DefaultTemplateConfig()
ct.Source = &src
ct.Destination = &dest
ct.Contents = &tmpl.EmbeddedTmpl
ct.LeftDelim = &tmpl.LeftDelim
ct.RightDelim = &tmpl.RightDelim
// Set the permissions
if tmpl.Perms != "" {
v, err := strconv.ParseUint(tmpl.Perms, 8, 12)
if err != nil {
return nil, fmt.Errorf("Failed to parse %q as octal: %v", tmpl.Perms, err)
}
m := os.FileMode(v)
ct.Perms = &m
}
ct.Finalize()
ctmpls[*ct] = tmpl
}
return ctmpls, nil
}
// newRunnerConfig returns a consul-template runner configuration, setting the
// Vault and Consul configurations based on the clients configs.
func newRunnerConfig(config *TaskTemplateManagerConfig,
templateMapping map[ctconf.TemplateConfig]*structs.Template) (*ctconf.Config, error) {
cc := config.ClientConfig
conf := ctconf.DefaultConfig()
// Gather the consul-template templates
flat := ctconf.TemplateConfigs(make([]*ctconf.TemplateConfig, 0, len(templateMapping)))
for ctmpl := range templateMapping {
local := ctmpl
flat = append(flat, &local)
}
conf.Templates = &flat
// Go through the templates and determine the minimum Vault grace
vaultGrace := time.Duration(-1)
for _, tmpl := range templateMapping {
// Initial condition
if vaultGrace < 0 {
vaultGrace = tmpl.VaultGrace
} else if tmpl.VaultGrace < vaultGrace {
vaultGrace = tmpl.VaultGrace
}
}
// Force faster retries
if config.retryRate != 0 {
rate := config.retryRate
conf.Consul.Retry.Backoff = &rate
}
// Setup the Consul config
if cc.ConsulConfig != nil {
conf.Consul.Address = &cc.ConsulConfig.Addr
conf.Consul.Token = &cc.ConsulConfig.Token
if cc.ConsulConfig.EnableSSL != nil && *cc.ConsulConfig.EnableSSL {
verify := cc.ConsulConfig.VerifySSL != nil && *cc.ConsulConfig.VerifySSL
conf.Consul.SSL = &ctconf.SSLConfig{
Enabled: helper.BoolToPtr(true),
Verify: &verify,
Cert: &cc.ConsulConfig.CertFile,
Key: &cc.ConsulConfig.KeyFile,
CaCert: &cc.ConsulConfig.CAFile,
}
}
if cc.ConsulConfig.Auth != "" {
parts := strings.SplitN(cc.ConsulConfig.Auth, ":", 2)
if len(parts) != 2 {
return nil, fmt.Errorf("Failed to parse Consul Auth config")
}
conf.Consul.Auth = &ctconf.AuthConfig{
Enabled: helper.BoolToPtr(true),
Username: &parts[0],
Password: &parts[1],
}
}
}
// Setup the Vault config
// Always set these to ensure nothing is picked up from the environment
emptyStr := ""
conf.Vault.RenewToken = helper.BoolToPtr(false)
conf.Vault.Token = &emptyStr
if cc.VaultConfig != nil && cc.VaultConfig.IsEnabled() {
conf.Vault.Address = &cc.VaultConfig.Addr
conf.Vault.Token = &config.VaultToken
conf.Vault.Grace = helper.TimeToPtr(vaultGrace)
if strings.HasPrefix(cc.VaultConfig.Addr, "https") || cc.VaultConfig.TLSCertFile != "" {
skipVerify := cc.VaultConfig.TLSSkipVerify != nil && *cc.VaultConfig.TLSSkipVerify
verify := !skipVerify
conf.Vault.SSL = &ctconf.SSLConfig{
Enabled: helper.BoolToPtr(true),
Verify: &verify,
Cert: &cc.VaultConfig.TLSCertFile,
Key: &cc.VaultConfig.TLSKeyFile,
CaCert: &cc.VaultConfig.TLSCaFile,
CaPath: &cc.VaultConfig.TLSCaPath,
ServerName: &cc.VaultConfig.TLSServerName,
}
} else {
conf.Vault.SSL = &ctconf.SSLConfig{
Enabled: helper.BoolToPtr(false),
Verify: helper.BoolToPtr(false),
Cert: &emptyStr,
Key: &emptyStr,
CaCert: &emptyStr,
CaPath: &emptyStr,
ServerName: &emptyStr,
}
}
}
conf.Finalize()
return conf, nil
}
// loadTemplateEnv loads task environment variables from all templates.
func loadTemplateEnv(tmpls []*structs.Template, taskDir string) (map[string]string, error) {
all := make(map[string]string, 50)
for _, t := range tmpls {
if !t.Envvars {
continue
}
f, err := os.Open(filepath.Join(taskDir, t.DestPath))
if err != nil {
return nil, fmt.Errorf("error opening env template: %v", err)
}
defer f.Close()
// Parse environment fil
vars, err := envparse.Parse(f)
if err != nil {
return nil, fmt.Errorf("error parsing env template %q: %v", t.DestPath, err)
}
for k, v := range vars {
all[k] = v
}
}
return all, nil
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,158 @@
package taskrunner
import (
"context"
"fmt"
"sync"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
ti "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/template"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/nomad/structs"
)
type templateHookConfig struct {
// logger is used to log
logger log.Logger
// lifecycle is used to interact with the task's lifecycle
lifecycle ti.TaskLifecycle
// events is used to emit events
events ti.EventEmitter
// templates is the set of templates we are managing
templates []*structs.Template
// clientConfig is the Nomad Client configuration
clientConfig *config.Config
// envBuilder is the environment variable builder for the task.
envBuilder *env.Builder
}
type templateHook struct {
config *templateHookConfig
// logger is used to log
logger log.Logger
// templateManager is used to manage any consul-templates this task may have
templateManager *template.TaskTemplateManager
managerLock sync.Mutex
// vaultToken is the current Vault token
vaultToken string
// taskDir is the task directory
taskDir string
}
func newTemplateHook(config *templateHookConfig) *templateHook {
h := &templateHook{
config: config,
}
h.logger = config.logger.Named(h.Name())
return h
}
func (*templateHook) Name() string {
return "template"
}
func (h *templateHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
h.managerLock.Lock()
defer h.managerLock.Unlock()
// If we have already run prerun before exit early.
if h.templateManager != nil {
return nil
}
// Store the current Vault token and the task directory
h.taskDir = req.TaskDir.Dir
h.vaultToken = req.VaultToken
unblockCh, err := h.newManager()
if err != nil {
return err
}
// Wait for the template to render
select {
case <-ctx.Done():
case <-unblockCh:
}
return nil
}
func (h *templateHook) newManager() (unblock chan struct{}, err error) {
unblock = make(chan struct{})
m, err := template.NewTaskTemplateManager(&template.TaskTemplateManagerConfig{
UnblockCh: unblock,
Lifecycle: h.config.lifecycle,
Events: h.config.events,
Templates: h.config.templates,
ClientConfig: h.config.clientConfig,
VaultToken: h.vaultToken,
TaskDir: h.taskDir,
EnvBuilder: h.config.envBuilder,
MaxTemplateEventRate: template.DefaultMaxTemplateEventRate,
})
if err != nil {
h.logger.Error("failed to create template manager", "error", err)
return nil, err
}
h.templateManager = m
return unblock, nil
}
func (h *templateHook) Stop(ctx context.Context, req *interfaces.TaskStopRequest, resp *interfaces.TaskStopResponse) error {
h.managerLock.Lock()
defer h.managerLock.Unlock()
// Shutdown any created template
if h.templateManager != nil {
h.templateManager.Stop()
}
return nil
}
// Handle new Vault token
func (h *templateHook) Update(ctx context.Context, req *interfaces.TaskUpdateRequest, resp *interfaces.TaskUpdateResponse) error {
h.managerLock.Lock()
defer h.managerLock.Unlock()
// Nothing to do
if h.templateManager == nil {
return nil
}
// Check if the Vault token has changed
if req.VaultToken == h.vaultToken {
return nil
} else {
h.vaultToken = req.VaultToken
}
// Shutdown the old template
h.templateManager.Stop()
h.templateManager = nil
// Create the new template
if _, err := h.newManager(); err != nil {
err := fmt.Errorf("failed to build template manager: %v", err)
h.logger.Error("failed to build template manager", "error", err)
h.config.lifecycle.Kill(context.Background(),
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Template update %v", err)))
}
return nil
}

View File

@ -0,0 +1,66 @@
package taskrunner
import (
"context"
"fmt"
log "github.com/hashicorp/go-hclog"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/nomad/structs"
)
// validateHook validates the task is able to be run.
type validateHook struct {
config *config.Config
logger log.Logger
}
func newValidateHook(config *config.Config, logger log.Logger) *validateHook {
h := &validateHook{
config: config,
}
h.logger = logger.Named(h.Name())
return h
}
func (*validateHook) Name() string {
return "validate"
}
func (h *validateHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
if err := validateTask(req.Task, req.TaskEnv, h.config); err != nil {
return err
}
resp.Done = true
return nil
}
func validateTask(task *structs.Task, taskEnv *env.TaskEnv, conf *config.Config) error {
var mErr multierror.Error
// Validate the user
unallowedUsers := conf.ReadStringListToMapDefault("user.blacklist", config.DefaultUserBlacklist)
checkDrivers := conf.ReadStringListToMapDefault("user.checked_drivers", config.DefaultUserCheckedDrivers)
if _, driverMatch := checkDrivers[task.Driver]; driverMatch {
if _, unallowed := unallowedUsers[task.User]; unallowed {
mErr.Errors = append(mErr.Errors, fmt.Errorf("running as user %q is disallowed", task.User))
}
}
// Validate the Service names once they're interpolated
for i, service := range task.Services {
name := taskEnv.ReplaceEnv(service.Name)
if err := service.ValidateName(name); err != nil {
mErr.Errors = append(mErr.Errors, fmt.Errorf("service (%d) failed validation: %v", i, err))
}
}
if len(mErr.Errors) == 1 {
return mErr.Errors[0]
}
return mErr.ErrorOrNil()
}

View File

@ -0,0 +1,63 @@
package taskrunner
import (
"testing"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/require"
)
func TestTaskRunner_Validate_UserEnforcement(t *testing.T) {
t.Parallel()
taskEnv := env.NewEmptyBuilder().Build()
conf := config.DefaultConfig()
// Try to run as root with exec.
task := &structs.Task{
Driver: "exec",
User: "root",
}
if err := validateTask(task, taskEnv, conf); err == nil {
t.Fatalf("expected error running as root with exec")
}
// Try to run a non-blacklisted user with exec.
task.User = "foobar"
require.NoError(t, validateTask(task, taskEnv, conf))
// Try to run as root with docker.
task.Driver = "docker"
task.User = "root"
require.NoError(t, validateTask(task, taskEnv, conf))
}
func TestTaskRunner_Validate_ServiceName(t *testing.T) {
t.Parallel()
builder := env.NewEmptyBuilder()
conf := config.DefaultConfig()
// Create a task with a service for validation
task := &structs.Task{
Services: []*structs.Service{
{
Name: "ok",
},
},
}
require.NoError(t, validateTask(task, builder.Build(), conf))
// Add an env var that should validate
builder.SetGenericEnv(map[string]string{"FOO": "bar"})
task.Services[0].Name = "${FOO}"
require.NoError(t, validateTask(task, builder.Build(), conf))
// Add an env var that should *not* validate
builder.SetGenericEnv(map[string]string{"BAD": "invalid/in/consul"})
task.Services[0].Name = "${BAD}"
require.Error(t, validateTask(task, builder.Build(), conf))
}

View File

@ -0,0 +1,403 @@
package taskrunner
import (
"context"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"sync"
"time"
"github.com/hashicorp/consul-template/signals"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
ti "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/vaultclient"
"github.com/hashicorp/nomad/nomad/structs"
)
const (
// vaultBackoffBaseline is the baseline time for exponential backoff when
// attempting to retrieve a Vault token
vaultBackoffBaseline = 5 * time.Second
// vaultBackoffLimit is the limit of the exponential backoff when attempting
// to retrieve a Vault token
vaultBackoffLimit = 3 * time.Minute
// vaultTokenFile is the name of the file holding the Vault token inside the
// task's secret directory
vaultTokenFile = "vault_token"
)
type vaultTokenUpdateHandler interface {
updatedVaultToken(token string)
}
func (tr *TaskRunner) updatedVaultToken(token string) {
// Update the task runner and environment
tr.setVaultToken(token)
// Trigger update hooks with the new Vault token
tr.triggerUpdateHooks()
}
type vaultHookConfig struct {
vaultStanza *structs.Vault
client vaultclient.VaultClient
events ti.EventEmitter
lifecycle ti.TaskLifecycle
updater vaultTokenUpdateHandler
logger log.Logger
alloc *structs.Allocation
task string
}
type vaultHook struct {
// vaultStanza is the vault stanza for the task
vaultStanza *structs.Vault
// eventEmitter is used to emit events to the task
eventEmitter ti.EventEmitter
// lifecycle is used to signal, restart and kill a task
lifecycle ti.TaskLifecycle
// updater is used to update the Vault token
updater vaultTokenUpdateHandler
// client is the Vault client to retrieve and renew the Vault token
client vaultclient.VaultClient
// logger is used to log
logger log.Logger
// ctx and cancel are used to kill the long running token manager
ctx context.Context
cancel context.CancelFunc
// tokenPath is the path in which to read and write the token
tokenPath string
// alloc is the allocation
alloc *structs.Allocation
// taskName is the name of the task
taskName string
// firstRun stores whether it is the first run for the hook
firstRun bool
// future is used to wait on retrieving a Vault token
future *tokenFuture
}
func newVaultHook(config *vaultHookConfig) *vaultHook {
ctx, cancel := context.WithCancel(context.Background())
h := &vaultHook{
vaultStanza: config.vaultStanza,
client: config.client,
eventEmitter: config.events,
lifecycle: config.lifecycle,
updater: config.updater,
alloc: config.alloc,
taskName: config.task,
firstRun: true,
ctx: ctx,
cancel: cancel,
future: newTokenFuture(),
}
h.logger = config.logger.Named(h.Name())
return h
}
func (*vaultHook) Name() string {
return "vault"
}
func (h *vaultHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
// If we have already run prestart before exit early. We do not use the
// PrestartDone value because we want to recover the token on restoration.
first := h.firstRun
h.firstRun = false
if !first {
return nil
}
// Try to recover a token if it was previously written in the secrets
// directory
recoveredToken := ""
h.tokenPath = filepath.Join(req.TaskDir.SecretsDir, vaultTokenFile)
data, err := ioutil.ReadFile(h.tokenPath)
if err != nil {
if !os.IsNotExist(err) {
return fmt.Errorf("failed to recover vault token: %v", err)
}
// Token file doesn't exist
} else {
// Store the recovered token
recoveredToken = string(data)
}
// Launch the token manager
go h.run(recoveredToken)
// Block until we get a token
select {
case <-h.future.Wait():
case <-ctx.Done():
return nil
}
h.updater.updatedVaultToken(h.future.Get())
return nil
}
func (h *vaultHook) Stop(ctx context.Context, req *interfaces.TaskStopRequest, resp *interfaces.TaskStopResponse) error {
// Shutdown any created manager
h.cancel()
return nil
}
// run should be called in a go-routine and manages the derivation, renewal and
// handling of errors with the Vault token. The optional parameter allows
// setting the initial Vault token. This is useful when the Vault token is
// recovered off disk.
func (h *vaultHook) run(token string) {
// Helper for stopping token renewal
stopRenewal := func() {
if err := h.client.StopRenewToken(h.future.Get()); err != nil {
h.logger.Warn("failed to stop token renewal", "error", err)
}
}
// updatedToken lets us store state between loops. If true, a new token
// has been retrieved and we need to apply the Vault change mode
var updatedToken bool
OUTER:
for {
// Check if we should exit
if h.ctx.Err() != nil {
stopRenewal()
return
}
// Clear the token
h.future.Clear()
// Check if there already is a token which can be the case for
// restoring the TaskRunner
if token == "" {
// Get a token
var exit bool
token, exit = h.deriveVaultToken()
if exit {
// Exit the manager
return
}
// Write the token to disk
if err := h.writeToken(token); err != nil {
errorString := "failed to write Vault token to disk"
h.logger.Error(errorString, "error", err)
h.lifecycle.Kill(h.ctx,
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Vault %v", errorString)))
return
}
}
// Start the renewal process
renewCh, err := h.client.RenewToken(token, 30)
// An error returned means the token is not being renewed
if err != nil {
h.logger.Error("failed to start renewal of Vault token", "error", err)
token = ""
goto OUTER
}
// The Vault token is valid now, so set it
h.future.Set(token)
if updatedToken {
switch h.vaultStanza.ChangeMode {
case structs.VaultChangeModeSignal:
s, err := signals.Parse(h.vaultStanza.ChangeSignal)
if err != nil {
h.logger.Error("failed to parse signal", "error", err)
h.lifecycle.Kill(h.ctx,
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Vault: failed to parse signal: %v", err)))
return
}
event := structs.NewTaskEvent(structs.TaskSignaling).SetTaskSignal(s).SetDisplayMessage("Vault: new Vault token acquired")
if err := h.lifecycle.Signal(event, h.vaultStanza.ChangeSignal); err != nil {
h.logger.Error("failed to send signal", "error", err)
h.lifecycle.Kill(h.ctx,
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Vault: failed to send signal: %v", err)))
return
}
case structs.VaultChangeModeRestart:
const noFailure = false
h.lifecycle.Restart(h.ctx,
structs.NewTaskEvent(structs.TaskRestarting).
SetDisplayMessage("Vault: new Vault token acquired"), false)
case structs.VaultChangeModeNoop:
fallthrough
default:
h.logger.Error("invalid Vault change mode", "mode", h.vaultStanza.ChangeMode)
}
// We have handled it
updatedToken = false
// Call the handler
h.updater.updatedVaultToken(token)
}
// Start watching for renewal errors
select {
case err := <-renewCh:
// Clear the token
token = ""
h.logger.Error("failed to renew Vault token", "error", err)
stopRenewal()
// Check if we have to do anything
if h.vaultStanza.ChangeMode != structs.VaultChangeModeNoop {
updatedToken = true
}
case <-h.ctx.Done():
stopRenewal()
return
}
}
}
// deriveVaultToken derives the Vault token using exponential backoffs. It
// returns the Vault token and whether the manager should exit.
func (h *vaultHook) deriveVaultToken() (token string, exit bool) {
attempts := 0
for {
tokens, err := h.client.DeriveToken(h.alloc, []string{h.taskName})
if err == nil {
return tokens[h.taskName], false
}
// Check if this is a server side error
if structs.IsServerSide(err) {
h.logger.Error("failed to derive Vault token", "error", err, "server_side", true)
h.lifecycle.Kill(h.ctx,
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Vault: server failed to derive vault token: %v", err)))
return "", true
}
// Check if we can't recover from the error
if !structs.IsRecoverable(err) {
h.logger.Error("failed to derive Vault token", "error", err, "recoverable", false)
h.lifecycle.Kill(h.ctx,
structs.NewTaskEvent(structs.TaskKilling).
SetFailsTask().
SetDisplayMessage(fmt.Sprintf("Vault: failed to derive vault token: %v", err)))
return "", true
}
// Handle the retry case
backoff := (1 << (2 * uint64(attempts))) * vaultBackoffBaseline
if backoff > vaultBackoffLimit {
backoff = vaultBackoffLimit
}
h.logger.Error("failed to derive Vault token", "error", err, "recoverable", true, "backoff", backoff)
attempts++
// Wait till retrying
select {
case <-h.ctx.Done():
return "", true
case <-time.After(backoff):
}
}
}
// writeToken writes the given token to disk
func (h *vaultHook) writeToken(token string) error {
if err := ioutil.WriteFile(h.tokenPath, []byte(token), 0666); err != nil {
return fmt.Errorf("failed to write vault token: %v", err)
}
return nil
}
// tokenFuture stores the Vault token and allows consumers to block till a valid
// token exists
type tokenFuture struct {
waiting []chan struct{}
token string
set bool
m sync.Mutex
}
// newTokenFuture returns a new token future without any token set
func newTokenFuture() *tokenFuture {
return &tokenFuture{}
}
// Wait returns a channel that can be waited on. When this channel unblocks, a
// valid token will be available via the Get method
func (f *tokenFuture) Wait() <-chan struct{} {
f.m.Lock()
defer f.m.Unlock()
c := make(chan struct{})
if f.set {
close(c)
return c
}
f.waiting = append(f.waiting, c)
return c
}
// Set sets the token value and unblocks any caller of Wait
func (f *tokenFuture) Set(token string) *tokenFuture {
f.m.Lock()
defer f.m.Unlock()
f.set = true
f.token = token
for _, w := range f.waiting {
close(w)
}
f.waiting = nil
return f
}
// Clear clears the set vault token.
func (f *tokenFuture) Clear() *tokenFuture {
f.m.Lock()
defer f.m.Unlock()
f.token = ""
f.set = false
return f
}
// Get returns the set Vault token
func (f *tokenFuture) Get() string {
f.m.Lock()
defer f.m.Unlock()
return f.token
}

View File

@ -0,0 +1 @@
package allocrunner

File diff suppressed because it is too large Load Diff

View File

@ -1,3 +1,5 @@
// +build deprecated
package allocrunner
import (

File diff suppressed because it is too large Load Diff

View File

@ -1,3 +1,5 @@
// +build deprecated
package taskrunner
import (

View File

@ -1,3 +1,5 @@
// +build deprecated
package taskrunner
// Name returns the name of the task

File diff suppressed because it is too large Load Diff

View File

@ -1,3 +1,5 @@
// +build deprecated
package taskrunner
import (
@ -664,6 +666,7 @@ func TestTaskRunner_UnregisterConsul_Retries(t *testing.T) {
}
}
//XXX Ported to allocrunner/task_runner/validate_hook_test.go
func TestTaskRunner_Validate_UserEnforcement(t *testing.T) {
t.Parallel()
ctx := testTaskRunner(t, false)

View File

@ -1,4 +1,4 @@
// +build !windows
// +build deprecated,!windows
package taskrunner

View File

@ -1,3 +1,5 @@
// +build deprecated
package allocrunner
import (
@ -7,6 +9,7 @@ import (
"testing"
"github.com/boltdb/bolt"
"github.com/hashicorp/nomad/client/allocwatcher"
"github.com/hashicorp/nomad/client/config"
consulApi "github.com/hashicorp/nomad/client/consul"
"github.com/hashicorp/nomad/client/vaultclient"
@ -51,7 +54,7 @@ func TestAllocRunnerFromAlloc(t *testing.T, alloc *structs.Allocation, restarts
alloc.Job.Type = structs.JobTypeBatch
}
vclient := vaultclient.NewMockVaultClient()
ar := NewAllocRunner(testlog.Logger(t), conf, db, upd.Update, alloc, vclient, consulApi.NewMockConsulServiceClient(t, testlog.HCLogger(t)), NoopPrevAlloc{})
ar := NewAllocRunner(testlog.Logger(t), conf, db, upd.Update, alloc, vclient, consulApi.NewMockConsulServiceClient(t, testlog.HCLogger(t)), allocwatcher.NoopPrevAlloc{})
return upd, ar
}

View File

@ -1,11 +1,10 @@
package allocrunner
package allocwatcher
import (
"archive/tar"
"context"
"fmt"
"io"
"log"
"os"
"path/filepath"
"sync"
@ -13,6 +12,7 @@ import (
"time"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/go-hclog"
nomadapi "github.com/hashicorp/nomad/api"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/config"
@ -26,8 +26,8 @@ const (
getRemoteRetryIntv = 30 * time.Second
)
// rpcer is the interface needed by a prevAllocWatcher to make RPC calls.
type rpcer interface {
// RPCer is the interface needed by a prevAllocWatcher to make RPC calls.
type RPCer interface {
// RPC allows retrieving remote allocs.
RPC(method string, args interface{}, reply interface{}) error
}
@ -38,10 +38,18 @@ type terminated interface {
Terminated() bool
}
// prevAllocWatcher allows AllocRunners to wait for a previous allocation to
// AllocRunnerMeta provides metadata about an AllocRunner such as its alloc and
// alloc dir.
type AllocRunnerMeta interface {
GetAllocDir() *allocdir.AllocDir
Listener() *cstructs.AllocListener
Alloc() *structs.Allocation
}
// PrevAllocWatcher allows AllocRunners to wait for a previous allocation to
// terminate and migrate its data whether or not the previous allocation is
// local or remote.
type prevAllocWatcher interface {
type PrevAllocWatcher interface {
// Wait for previous alloc to terminate
Wait(context.Context) error
@ -55,41 +63,66 @@ type prevAllocWatcher interface {
IsMigrating() bool
}
// NewAllocWatcher creates a prevAllocWatcher appropriate for whether this
type Config struct {
// Alloc is the current allocation which may need to block on its
// previous allocation stopping.
Alloc *structs.Allocation
// PreviousRunner is non-nil iff All has a PreviousAllocation and it is
// running locally.
PreviousRunner AllocRunnerMeta
// RPC allows the alloc watcher to monitor remote allocations.
RPC RPCer
// Config is necessary for using the RPC.
Config *config.Config
// MigrateToken is used to migrate remote alloc dirs when ACLs are
// enabled.
MigrateToken string
Logger hclog.Logger
}
// NewAllocWatcher creates a PrevAllocWatcher appropriate for whether this
// alloc's previous allocation was local or remote. If this alloc has no
// previous alloc then a noop implementation is returned.
func NewAllocWatcher(alloc *structs.Allocation, prevAR *AllocRunner, rpc rpcer, config *config.Config, l *log.Logger, migrateToken string) prevAllocWatcher {
if alloc.PreviousAllocation == "" {
func NewAllocWatcher(c Config) PrevAllocWatcher {
if c.Alloc.PreviousAllocation == "" {
// No previous allocation, use noop transitioner
return NoopPrevAlloc{}
}
tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup)
logger := c.Logger.Named("alloc_watcher")
logger = logger.With("alloc_id", c.Alloc.ID)
logger = logger.With("previous_alloc", c.Alloc.PreviousAllocation)
if prevAR != nil {
tg := c.Alloc.Job.LookupTaskGroup(c.Alloc.TaskGroup)
if c.PreviousRunner != nil {
// Previous allocation is local, use local transitioner
return &localPrevAlloc{
allocID: alloc.ID,
prevAllocID: alloc.PreviousAllocation,
allocID: c.Alloc.ID,
prevAllocID: c.Alloc.PreviousAllocation,
tasks: tg.Tasks,
sticky: tg.EphemeralDisk != nil && tg.EphemeralDisk.Sticky,
prevAllocDir: prevAR.GetAllocDir(),
prevListener: prevAR.GetListener(),
prevWaitCh: prevAR.WaitCh(),
prevStatus: prevAR.Alloc(),
logger: l,
prevAllocDir: c.PreviousRunner.GetAllocDir(),
prevListener: c.PreviousRunner.Listener(),
prevStatus: c.PreviousRunner.Alloc(),
logger: logger,
}
}
return &remotePrevAlloc{
allocID: alloc.ID,
prevAllocID: alloc.PreviousAllocation,
allocID: c.Alloc.ID,
prevAllocID: c.Alloc.PreviousAllocation,
tasks: tg.Tasks,
config: config,
config: c.Config,
migrate: tg.EphemeralDisk != nil && tg.EphemeralDisk.Migrate,
rpc: rpc,
logger: l,
migrateToken: migrateToken,
rpc: c.RPC,
migrateToken: c.MigrateToken,
logger: logger,
}
}
@ -118,10 +151,6 @@ type localPrevAlloc struct {
// terminated (and therefore won't send updates to the listener)
prevStatus terminated
// prevWaitCh is closed when the previous alloc is garbage collected
// which is a failsafe against blocking the new alloc forever
prevWaitCh <-chan struct{}
// waiting and migrating are true when alloc runner is waiting on the
// prevAllocWatcher. Writers must acquire the waitingLock and readers
// should use the helper methods IsWaiting and IsMigrating.
@ -129,7 +158,7 @@ type localPrevAlloc struct {
migrating bool
waitingLock sync.RWMutex
logger *log.Logger
logger hclog.Logger
}
// IsWaiting returns true if there's a concurrent call inside Wait
@ -161,21 +190,14 @@ func (p *localPrevAlloc) Wait(ctx context.Context) error {
defer p.prevListener.Close()
if p.prevStatus.Terminated() {
// Fast path - previous alloc already terminated!
return nil
}
// Block until previous alloc exits
p.logger.Printf("[DEBUG] client: alloc %q waiting for previous alloc %q to terminate", p.allocID, p.prevAllocID)
p.logger.Debug("waiting for previous alloc to terminate")
for {
select {
case prevAlloc, ok := <-p.prevListener.Ch:
if !ok || prevAlloc.Terminated() {
return nil
}
case <-p.prevWaitCh:
return nil
case <-ctx.Done():
return ctx.Err()
}
@ -198,13 +220,14 @@ func (p *localPrevAlloc) Migrate(ctx context.Context, dest *allocdir.AllocDir) e
p.waitingLock.Unlock()
}()
p.logger.Printf("[DEBUG] client: alloc %q copying previous alloc %q", p.allocID, p.prevAllocID)
p.logger.Debug("copying previous alloc")
moveErr := dest.Move(p.prevAllocDir, p.tasks)
// Always cleanup previous alloc
if err := p.prevAllocDir.Destroy(); err != nil {
p.logger.Printf("[ERR] client: error destroying allocdir %v: %v", p.prevAllocDir.AllocDir, err)
p.logger.Error("error destroying alloc dir",
"error", err, "previous_alloc_dir", p.prevAllocDir.AllocDir)
}
return moveErr
@ -230,7 +253,7 @@ type remotePrevAlloc struct {
// rpc provides an RPC method for watching for updates to the previous
// alloc and determining what node it was on.
rpc rpcer
rpc RPCer
// nodeID is the node the previous alloc. Set by Wait() for use in
// Migrate() iff the previous alloc has not already been GC'd.
@ -243,7 +266,7 @@ type remotePrevAlloc struct {
migrating bool
waitingLock sync.RWMutex
logger *log.Logger
logger hclog.Logger
// migrateToken allows a client to migrate data in an ACL-protected remote
// volume
@ -277,7 +300,7 @@ func (p *remotePrevAlloc) Wait(ctx context.Context) error {
p.waitingLock.Unlock()
}()
p.logger.Printf("[DEBUG] client: alloc %q waiting for remote previous alloc %q to terminate", p.allocID, p.prevAllocID)
p.logger.Debug("waiting for remote previous alloc to terminate")
req := structs.AllocSpecificRequest{
AllocID: p.prevAllocID,
QueryOptions: structs.QueryOptions{
@ -300,7 +323,7 @@ func (p *remotePrevAlloc) Wait(ctx context.Context) error {
resp := structs.SingleAllocResponse{}
err := p.rpc.RPC("Alloc.GetAlloc", &req, &resp)
if err != nil {
p.logger.Printf("[ERR] client: failed to query previous alloc %q: %v", p.prevAllocID, err)
p.logger.Error("error querying previous alloc", "error", err)
retry := getRemoteRetryIntv + lib.RandomStagger(getRemoteRetryIntv)
select {
case <-time.After(retry):
@ -310,7 +333,7 @@ func (p *remotePrevAlloc) Wait(ctx context.Context) error {
}
}
if resp.Alloc == nil {
p.logger.Printf("[DEBUG] client: blocking alloc %q has been GC'd", p.prevAllocID)
p.logger.Debug("blocking alloc was GC'd")
return nil
}
if resp.Alloc.Terminated() {
@ -345,12 +368,11 @@ func (p *remotePrevAlloc) Migrate(ctx context.Context, dest *allocdir.AllocDir)
p.waitingLock.Unlock()
}()
p.logger.Printf("[DEBUG] client: alloc %q copying from remote previous alloc %q", p.allocID, p.prevAllocID)
p.logger.Debug("copying from remote previous alloc")
if p.nodeID == "" {
// NodeID couldn't be found; likely alloc was GC'd
p.logger.Printf("[WARN] client: alloc %q couldn't migrate data from previous alloc %q; previous alloc may have been GC'd",
p.allocID, p.prevAllocID)
p.logger.Warn("unable to migrate data from previous alloc; previous alloc may have been GC'd")
return nil
}
@ -371,7 +393,8 @@ func (p *remotePrevAlloc) Migrate(ctx context.Context, dest *allocdir.AllocDir)
}
if err := prevAllocDir.Destroy(); err != nil {
p.logger.Printf("[ERR] client: error destroying allocdir %q: %v", prevAllocDir.AllocDir, err)
p.logger.Error("error destroying alloc dir",
"error", err, "previous_alloc_dir", prevAllocDir.AllocDir)
}
return nil
}
@ -391,7 +414,7 @@ func (p *remotePrevAlloc) getNodeAddr(ctx context.Context, nodeID string) (strin
for {
err := p.rpc.RPC("Node.GetNode", &req, &resp)
if err != nil {
p.logger.Printf("[ERR] client: failed to query node info %q: %v", nodeID, err)
p.logger.Error("failed to query node", "error", err, "node", nodeID)
retry := getRemoteRetryIntv + lib.RandomStagger(getRemoteRetryIntv)
select {
case <-time.After(retry):
@ -456,7 +479,7 @@ func (p *remotePrevAlloc) migrateAllocDir(ctx context.Context, nodeAddr string)
// stream remote alloc to dir to a local path. Caller should cleanup dest on
// error.
func (p *remotePrevAlloc) streamAllocDir(ctx context.Context, resp io.ReadCloser, dest string) error {
p.logger.Printf("[DEBUG] client: alloc %q streaming snapshot of previous alloc %q to %q", p.allocID, p.prevAllocID, dest)
p.logger.Debug("streaming snapshot of previous alloc", "destination", dest)
tr := tar.NewReader(resp)
defer resp.Close()
@ -466,8 +489,7 @@ func (p *remotePrevAlloc) streamAllocDir(ctx context.Context, resp io.ReadCloser
canceled := func() bool {
select {
case <-ctx.Done():
p.logger.Printf("[INFO] client: stopping migration of previous alloc %q for new alloc: %v",
p.prevAllocID, p.allocID)
p.logger.Info("migration of previous alloc canceled")
return true
default:
return false

View File

@ -1,4 +1,4 @@
package allocrunner
package allocwatcher
import (
"archive/tar"
@ -14,26 +14,118 @@ import (
"testing"
"time"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/testutil"
cstructs "github.com/hashicorp/nomad/client/structs"
ctestutil "github.com/hashicorp/nomad/client/testutil"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/testutil"
"github.com/stretchr/testify/require"
)
// fakeAllocRunner implements AllocRunnerMeta
type fakeAllocRunner struct {
alloc *structs.Allocation
AllocDir *allocdir.AllocDir
Broadcaster *cstructs.AllocBroadcaster
}
// newFakeAllocRunner creates a new AllocRunnerMeta. Callers must call
// AllocDir.Destroy() when finished.
func newFakeAllocRunner(t *testing.T, logger hclog.Logger) *fakeAllocRunner {
alloc := mock.Alloc()
alloc.Job.TaskGroups[0].EphemeralDisk.Sticky = true
alloc.Job.TaskGroups[0].EphemeralDisk.Migrate = true
path, err := ioutil.TempDir("", "nomad_test_watcher")
require.NoError(t, err)
return &fakeAllocRunner{
alloc: alloc,
AllocDir: allocdir.NewAllocDir(logger, path),
Broadcaster: cstructs.NewAllocBroadcaster(),
}
}
func (f *fakeAllocRunner) GetAllocDir() *allocdir.AllocDir {
return f.AllocDir
}
func (f *fakeAllocRunner) Listener() *cstructs.AllocListener {
return f.Broadcaster.Listen()
}
func (f *fakeAllocRunner) Alloc() *structs.Allocation {
return f.alloc
}
// newConfig returns a new Config and cleanup func
func newConfig(t *testing.T) (Config, func()) {
logger := testlog.HCLogger(t)
prevAR := newFakeAllocRunner(t, logger)
alloc := mock.Alloc()
alloc.PreviousAllocation = prevAR.Alloc().ID
alloc.Job.TaskGroups[0].EphemeralDisk.Sticky = true
alloc.Job.TaskGroups[0].EphemeralDisk.Migrate = true
config := Config{
Alloc: alloc,
PreviousRunner: prevAR,
RPC: nil,
Config: nil,
MigrateToken: "fake_token",
Logger: logger,
}
cleanup := func() {
prevAR.AllocDir.Destroy()
}
return config, cleanup
}
// TestPrevAlloc_Noop asserts that when no previous allocation is set the noop
// implementation is returned that does not block or perform migrations.
func TestPrevAlloc_Noop(t *testing.T) {
conf, cleanup := newConfig(t)
defer cleanup()
conf.Alloc.PreviousAllocation = ""
watcher := NewAllocWatcher(conf)
require.NotNil(t, watcher)
_, ok := watcher.(NoopPrevAlloc)
require.True(t, ok, "expected watcher to be NoopPrevAlloc")
done := make(chan int, 2)
go func() {
watcher.Wait(context.Background())
done <- 1
watcher.Migrate(context.Background(), nil)
done <- 1
}()
require.False(t, watcher.IsWaiting())
require.False(t, watcher.IsMigrating())
<-done
<-done
}
// TestPrevAlloc_LocalPrevAlloc asserts that when a previous alloc runner is
// set a localPrevAlloc will block on it.
func TestPrevAlloc_LocalPrevAlloc(t *testing.T) {
_, prevAR := TestAllocRunner(t, false)
prevAR.alloc.Job.TaskGroups[0].Tasks[0].Config["run_for"] = "10s"
t.Parallel()
conf, cleanup := newConfig(t)
newAlloc := mock.Alloc()
newAlloc.PreviousAllocation = prevAR.Alloc().ID
newAlloc.Job.TaskGroups[0].EphemeralDisk.Sticky = false
task := newAlloc.Job.TaskGroups[0].Tasks[0]
task.Driver = "mock_driver"
task.Config["run_for"] = "500ms"
defer cleanup()
waiter := NewAllocWatcher(newAlloc, prevAR, nil, nil, testlog.Logger(t), "")
conf.Alloc.Job.TaskGroups[0].Tasks[0].Driver = "mock_driver"
conf.Alloc.Job.TaskGroups[0].Tasks[0].Config["run_for"] = "500ms"
waiter := NewAllocWatcher(conf)
// Wait in a goroutine with a context to make sure it exits at the right time
ctx, cancel := context.WithCancel(context.Background())
@ -43,39 +135,55 @@ func TestPrevAlloc_LocalPrevAlloc(t *testing.T) {
waiter.Wait(ctx)
}()
select {
case <-ctx.Done():
t.Fatalf("Wait exited too early")
case <-time.After(33 * time.Millisecond):
// Good! It's blocking
}
// Assert watcher is waiting
testutil.WaitForResult(func() (bool, error) {
return waiter.IsWaiting(), fmt.Errorf("expected watcher to be waiting")
}, func(err error) {
t.Fatalf("error: %v", err)
})
// Start the previous allocs to cause it to update but not terminate
go prevAR.Run()
defer prevAR.Destroy()
// Broadcast a non-terminal alloc update to assert only terminal
// updates break out of waiting.
update := conf.PreviousRunner.Alloc().Copy()
update.DesiredStatus = structs.AllocDesiredStatusStop
update.ModifyIndex++
update.AllocModifyIndex++
select {
case <-ctx.Done():
t.Fatalf("Wait exited too early")
case <-time.After(33 * time.Millisecond):
// Good! It's still blocking
}
broadcaster := conf.PreviousRunner.(*fakeAllocRunner).Broadcaster
err := broadcaster.Send(update)
require.NoError(t, err)
// Stop the previous alloc
prevAR.Destroy()
// Assert watcher is still waiting because alloc isn't terminal
testutil.WaitForResult(func() (bool, error) {
return waiter.IsWaiting(), fmt.Errorf("expected watcher to be waiting")
}, func(err error) {
t.Fatalf("error: %v", err)
})
select {
case <-ctx.Done():
// Good! We unblocked when the previous alloc stopped
case <-time.After(time.Second):
t.Fatalf("Wait exited too early")
}
// Stop the previous alloc and assert watcher stops blocking
update = update.Copy()
update.DesiredStatus = structs.AllocDesiredStatusStop
update.ClientStatus = structs.AllocClientStatusComplete
update.ModifyIndex++
update.AllocModifyIndex++
err = broadcaster.Send(update)
require.NoError(t, err)
testutil.WaitForResult(func() (bool, error) {
if waiter.IsWaiting() {
return false, fmt.Errorf("did not expect watcher to be waiting")
}
return !waiter.IsMigrating(), fmt.Errorf("did not expect watcher to be migrating")
}, func(err error) {
t.Fatalf("error: %v", err)
})
}
// TestPrevAlloc_StreamAllocDir_Ok asserts that streaming a tar to an alloc dir
// works.
func TestPrevAlloc_StreamAllocDir_Ok(t *testing.T) {
testutil.RequireRoot(t)
ctestutil.RequireRoot(t)
t.Parallel()
dir, err := ioutil.TempDir("", "")
if err != nil {
@ -178,7 +286,7 @@ func TestPrevAlloc_StreamAllocDir_Ok(t *testing.T) {
defer os.RemoveAll(dir1)
rc := ioutil.NopCloser(buf)
prevAlloc := &remotePrevAlloc{logger: testlog.Logger(t)}
prevAlloc := &remotePrevAlloc{logger: testlog.HCLogger(t)}
if err := prevAlloc.streamAllocDir(context.Background(), rc, dir1); err != nil {
t.Fatalf("err: %v", err)
}
@ -228,7 +336,7 @@ func TestPrevAlloc_StreamAllocDir_Error(t *testing.T) {
// This test only unit tests streamAllocDir so we only need a partially
// complete remotePrevAlloc
prevAlloc := &remotePrevAlloc{
logger: testlog.Logger(t),
logger: testlog.HCLogger(t),
allocID: "123",
prevAllocID: "abc",
migrate: true,

View File

@ -0,0 +1,4 @@
// Package allocwatcher allows blocking until another allocation - whether
// running locally or remotely - completes and migrates the allocation
// directory if necessary.
package allocwatcher

File diff suppressed because it is too large Load Diff

View File

@ -159,6 +159,7 @@ func TestClient_Fingerprint(t *testing.T) {
}
func TestClient_Fingerprint_Periodic(t *testing.T) {
t.Skip("missing mock driver plugin implementation")
t.Parallel()
c1 := TestClient(t, func(c *config.Config) {
@ -399,6 +400,7 @@ func TestClient_Heartbeat(t *testing.T) {
}
func TestClient_UpdateAllocStatus(t *testing.T) {
t.Skip("missing exec driver plugin implementation")
t.Parallel()
s1, _ := testServer(t, nil)
defer s1.Shutdown()
@ -650,7 +652,7 @@ func TestClient_Init(t *testing.T) {
config: &config.Config{
AllocDir: allocDir,
},
logger: testlog.Logger(t),
logger: testlog.HCLogger(t),
}
if err := client.init(); err != nil {
t.Fatalf("err: %s", err)
@ -662,6 +664,7 @@ func TestClient_Init(t *testing.T) {
}
func TestClient_BlockedAllocations(t *testing.T) {
t.Skip("missing mock driver plugin implementation")
t.Parallel()
s1, _ := testServer(t, nil)
defer s1.Shutdown()

View File

@ -13,6 +13,7 @@ import (
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/nomad/structs/config"
"github.com/hashicorp/nomad/plugins/shared/loader"
"github.com/hashicorp/nomad/version"
)
@ -206,6 +207,13 @@ type Config struct {
// This period is meant to be long enough for a leader election to take
// place, and a small jitter is applied to avoid a thundering herd.
RPCHoldTimeout time.Duration
// PluginLoader is used to load plugins.
PluginLoader loader.PluginCatalog
// PluginSingletonLoader is a plugin loader that will returns singleton
// instances of the plugins.
PluginSingletonLoader loader.PluginCatalog
}
func (c *Config) Copy() *Config {

23
client/config/testing.go Normal file
View File

@ -0,0 +1,23 @@
package config
import (
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/structs"
)
// TestClientConfig returns a default client configuration for test clients.
func TestClientConfig() *Config {
conf := DefaultConfig()
conf.VaultConfig.Enabled = helper.BoolToPtr(false)
conf.DevMode = true
conf.Node = &structs.Node{
Reserved: &structs.Resources{
DiskMB: 0,
},
}
// Loosen GC threshold
conf.GCDiskUsageThreshold = 98.0
conf.GCInodeUsageThreshold = 98.0
return conf
}

View File

@ -31,7 +31,7 @@ func NewMockConsulOp(op, allocID, task string) MockConsulOp {
// MockConsulServiceClient implements the ConsulServiceAPI interface to record
// and log task registration/deregistration.
type MockConsulServiceClient struct {
Ops []MockConsulOp
ops []MockConsulOp
mu sync.Mutex
logger log.Logger
@ -44,7 +44,7 @@ type MockConsulServiceClient struct {
func NewMockConsulServiceClient(t testing.T, logger log.Logger) *MockConsulServiceClient {
logger = logger.Named("mock_consul")
m := MockConsulServiceClient{
Ops: make([]MockConsulOp, 0, 20),
ops: make([]MockConsulOp, 0, 20),
logger: logger,
}
return &m
@ -54,7 +54,7 @@ func (m *MockConsulServiceClient) UpdateTask(old, new *consul.TaskServices) erro
m.mu.Lock()
defer m.mu.Unlock()
m.logger.Trace("UpdateTask", "alloc_id", new.AllocID, "task", new.Name)
m.Ops = append(m.Ops, NewMockConsulOp("update", new.AllocID, new.Name))
m.ops = append(m.ops, NewMockConsulOp("update", new.AllocID, new.Name))
return nil
}
@ -62,7 +62,7 @@ func (m *MockConsulServiceClient) RegisterTask(task *consul.TaskServices) error
m.mu.Lock()
defer m.mu.Unlock()
m.logger.Trace("RegisterTask", "alloc_id", task.AllocID, "task", task.Name)
m.Ops = append(m.Ops, NewMockConsulOp("add", task.AllocID, task.Name))
m.ops = append(m.ops, NewMockConsulOp("add", task.AllocID, task.Name))
return nil
}
@ -70,14 +70,14 @@ func (m *MockConsulServiceClient) RemoveTask(task *consul.TaskServices) {
m.mu.Lock()
defer m.mu.Unlock()
m.logger.Trace("RemoveTask", "alloc_id", task.AllocID, "task", task.Name)
m.Ops = append(m.Ops, NewMockConsulOp("remove", task.AllocID, task.Name))
m.ops = append(m.ops, NewMockConsulOp("remove", task.AllocID, task.Name))
}
func (m *MockConsulServiceClient) AllocRegistrations(allocID string) (*consul.AllocRegistration, error) {
m.mu.Lock()
defer m.mu.Unlock()
m.logger.Trace("AllocRegistrations", "alloc_id", allocID)
m.Ops = append(m.Ops, NewMockConsulOp("alloc_registrations", allocID, ""))
m.ops = append(m.ops, NewMockConsulOp("alloc_registrations", allocID, ""))
if m.AllocRegistrationsFn != nil {
return m.AllocRegistrationsFn(allocID)
@ -85,3 +85,9 @@ func (m *MockConsulServiceClient) AllocRegistrations(allocID string) (*consul.Al
return nil, nil
}
func (m *MockConsulServiceClient) GetOps() []MockConsulOp {
m.mu.Lock()
defer m.mu.Unlock()
return m.ops
}

View File

@ -17,18 +17,16 @@ import (
"time"
"github.com/armon/circbuf"
"github.com/fsouza/go-dockerclient"
metrics "github.com/armon/go-metrics"
docker "github.com/fsouza/go-dockerclient"
"github.com/docker/docker/cli/config/configfile"
"github.com/docker/docker/reference"
"github.com/docker/cli/cli/config/configfile"
"github.com/docker/distribution/reference"
"github.com/docker/docker/registry"
"github.com/armon/go-metrics"
"github.com/hashicorp/go-multierror"
"github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/client/driver/executor"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper"
@ -477,12 +475,9 @@ type dockerPID struct {
ContainerID string
KillTimeout time.Duration
MaxKillTimeout time.Duration
PluginConfig *PluginReattachConfig
}
type DockerHandle struct {
pluginClient *plugin.Client
executor executor.Executor
client *docker.Client
waitClient *docker.Client
logger *log.Logger
@ -500,6 +495,7 @@ type DockerHandle struct {
waitCh chan *dstructs.WaitResult
doneCh chan bool
removeContainerOnExit bool
net *cstructs.DriverNetwork
}
func NewDockerDriver(ctx *DriverContext) Driver {
@ -823,50 +819,10 @@ func (d *DockerDriver) Prestart(ctx *ExecContext, task *structs.Task) (*Prestart
}
func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
pluginLogFile := filepath.Join(ctx.TaskDir.Dir, "executor.out")
executorConfig := &dstructs.ExecutorConfig{
LogFile: pluginLogFile,
LogLevel: d.config.LogLevel,
}
exec, pluginClient, err := createExecutor(d.config.LogOutput, d.config, executorConfig)
if err != nil {
return nil, err
}
executorCtx := &executor.ExecutorContext{
TaskEnv: ctx.TaskEnv,
Task: task,
Driver: "docker",
LogDir: ctx.TaskDir.LogDir,
TaskDir: ctx.TaskDir.Dir,
PortLowerBound: d.config.ClientMinPort,
PortUpperBound: d.config.ClientMaxPort,
}
if err := exec.SetContext(executorCtx); err != nil {
pluginClient.Kill()
return nil, fmt.Errorf("failed to set executor context: %v", err)
}
// The user hasn't specified any logging options so launch our own syslog
// server if possible.
syslogAddr := ""
if len(d.driverConfig.Logging) == 0 {
if runtime.GOOS == "darwin" {
d.logger.Printf("[DEBUG] driver.docker: disabling syslog driver as Docker for Mac workaround")
} else {
ss, err := exec.LaunchSyslogServer()
if err != nil {
pluginClient.Kill()
return nil, fmt.Errorf("failed to start syslog collector: %v", err)
}
syslogAddr = ss.Addr
}
}
config, err := d.createContainerConfig(ctx, task, d.driverConfig, syslogAddr)
// TODO: implement alternative to launching a syslog server in the executor
config, err := d.createContainerConfig(ctx, task, d.driverConfig, "")
if err != nil {
d.logger.Printf("[ERR] driver.docker: failed to create container configuration for image %q (%q): %v", d.driverConfig.ImageName, d.imageID, err)
pluginClient.Kill()
return nil, fmt.Errorf("Failed to create container configuration for image %q (%q): %v", d.driverConfig.ImageName, d.imageID, err)
}
@ -874,7 +830,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
if err != nil {
wrapped := fmt.Sprintf("Failed to create container: %v", err)
d.logger.Printf("[ERR] driver.docker: %s", wrapped)
pluginClient.Kill()
return nil, structs.WrapRecoverable(wrapped, err)
}
@ -887,7 +842,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
// Start the container
if err := d.startContainer(container); err != nil {
d.logger.Printf("[ERR] driver.docker: failed to start container %s: %s", container.ID, err)
pluginClient.Kill()
return nil, structs.NewRecoverableError(fmt.Errorf("Failed to start container %s: %s", container.ID, err), structs.IsRecoverable(err))
}
@ -898,7 +852,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
if err != nil {
err = fmt.Errorf("failed to inspect started container %s: %s", container.ID, err)
d.logger.Printf("[ERR] driver.docker: %v", err)
pluginClient.Kill()
return nil, structs.NewRecoverableError(err, true)
}
container = runningContainer
@ -908,13 +861,20 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
container.ID, container.State.String())
}
// Detect container address
ip, autoUse := d.detectIP(container)
net := &cstructs.DriverNetwork{
PortMap: d.driverConfig.PortMap,
IP: ip,
AutoAdvertise: autoUse,
}
// Return a driver handle
maxKill := d.DriverContext.config.MaxKillTimeout
h := &DockerHandle{
client: client,
waitClient: waitClient,
executor: exec,
pluginClient: pluginClient,
logger: d.logger,
jobName: d.DriverContext.jobName,
taskGroupName: d.DriverContext.taskGroupName,
@ -928,22 +888,17 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
doneCh: make(chan bool),
waitCh: make(chan *dstructs.WaitResult, 1),
removeContainerOnExit: d.config.ReadBoolDefault(dockerCleanupContainerConfigOption, dockerCleanupContainerConfigDefault),
net: net,
}
go h.collectStats()
go h.run()
// Detect container address
ip, autoUse := d.detectIP(container)
// Create a response with the driver handle and container network metadata
resp := &StartResponse{
Handle: h,
Network: &cstructs.DriverNetwork{
PortMap: d.driverConfig.PortMap,
IP: ip,
AutoAdvertise: autoUse,
},
Handle: h,
Network: net,
}
return resp, nil
}
@ -1747,9 +1702,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
}
d.logger.Printf("[INFO] driver.docker: re-attaching to docker process: %s", pid.ContainerID)
d.logger.Printf("[DEBUG] driver.docker: re-attached to handle: %s", handleID)
pluginConfig := &plugin.ClientConfig{
Reattach: pid.PluginConfig.PluginConfig(),
}
client, waitClient, err := d.dockerClients()
if err != nil {
@ -1775,18 +1727,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
if !found {
return nil, fmt.Errorf("Failed to find container %s", pid.ContainerID)
}
exec, pluginClient, err := createExecutorWithConfig(pluginConfig, d.config.LogOutput)
if err != nil {
d.logger.Printf("[INFO] driver.docker: couldn't re-attach to the plugin process: %v", err)
d.logger.Printf("[DEBUG] driver.docker: stopping container %q", pid.ContainerID)
if e := client.StopContainer(pid.ContainerID, uint(pid.KillTimeout.Seconds())); e != nil {
d.logger.Printf("[DEBUG] driver.docker: couldn't stop container: %v", e)
}
return nil, err
}
ver, _ := exec.Version()
d.logger.Printf("[DEBUG] driver.docker: version of executor: %v", ver.Version)
// Increment the reference count since we successfully attached to this
// container
@ -1797,8 +1737,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
h := &DockerHandle{
client: client,
waitClient: waitClient,
executor: exec,
pluginClient: pluginClient,
logger: d.logger,
jobName: d.DriverContext.jobName,
taskGroupName: d.DriverContext.taskGroupName,
@ -1811,6 +1749,7 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
maxKillTimeout: pid.MaxKillTimeout,
doneCh: make(chan bool),
waitCh: make(chan *dstructs.WaitResult, 1),
net: nil, //FIXME Need to get driver network
}
go h.collectStats()
go h.run()
@ -1826,7 +1765,6 @@ func (h *DockerHandle) ID() string {
ImageID: h.ImageID,
KillTimeout: h.killTimeout,
MaxKillTimeout: h.maxKillTimeout,
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
}
data, err := json.Marshal(pid)
if err != nil {
@ -1843,12 +1781,13 @@ func (h *DockerHandle) WaitCh() chan *dstructs.WaitResult {
return h.waitCh
}
func (h *DockerHandle) Network() *cstructs.DriverNetwork {
return h.net
}
func (h *DockerHandle) Update(task *structs.Task) error {
// Store the updated kill timeout.
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
if err := h.executor.UpdateTask(task); err != nil {
h.logger.Printf("[DEBUG] driver.docker: failed to update log config: %v", err)
}
// Update is not possible
return nil
@ -1913,10 +1852,8 @@ func (h *DockerHandle) Signal(s os.Signal) error {
// Kill is used to terminate the task. This uses `docker stop -t killTimeout`
func (h *DockerHandle) Kill() error {
// Stop the container
err := h.waitClient.StopContainer(h.containerID, uint(h.killTimeout.Seconds()))
err := h.client.StopContainer(h.containerID, uint(h.killTimeout.Seconds()))
if err != nil {
h.executor.Exit()
h.pluginClient.Kill()
// Container has already been removed.
if strings.Contains(err.Error(), NoSuchContainerError) {
@ -1976,10 +1913,6 @@ func (h *DockerHandle) run() {
close(h.doneCh)
// 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)
}
h.pluginClient.Kill()
// Stop the container just incase the docker daemon's wait returned
// incorrectly
@ -2180,13 +2113,13 @@ func authFromHelper(helperName string) authBackend {
helper := dockerAuthHelperPrefix + helperName
cmd := exec.Command(helper, "get")
repoParsed, err := reference.ParseNamed(repo)
repoInfo, err := parseRepositoryInfo(repo)
if err != nil {
return nil, err
}
// Ensure that the HTTPs prefix exists
repoAddr := fmt.Sprintf("https://%s", repoParsed.Hostname())
repoAddr := fmt.Sprintf("https://%s", repoInfo.Index.Name)
cmd.Stdin = strings.NewReader(repoAddr)
output, err := cmd.Output()

View File

@ -58,7 +58,7 @@ func TestDockerDriver_PidsLimit(t *testing.T) {
task.Config["args"] = []string{"-c", "sleep 2 & sleep 2"}
ctx := testDockerDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
// Copy the image into the task's directory

View File

@ -117,7 +117,7 @@ func dockerSetupWithClient(t *testing.T, task *structs.Task, client *docker.Clie
if presp != nil && presp.CreatedResources != nil {
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
}
tctx.AllocDir.Destroy()
tctx.Destroy()
t.Fatalf("error in prestart: %v", err)
}
// Update the exec ctx with the driver network env vars
@ -126,13 +126,13 @@ func dockerSetupWithClient(t *testing.T, task *structs.Task, client *docker.Clie
sresp, err := driver.Start(tctx.ExecCtx, task)
if err != nil {
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
tctx.AllocDir.Destroy()
tctx.Destroy()
t.Fatalf("Failed to start driver: %s\nStack\n%s", err, debug.Stack())
}
if sresp.Handle == nil {
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
tctx.AllocDir.Destroy()
tctx.Destroy()
t.Fatalf("handle is nil\nStack\n%s", debug.Stack())
}
@ -142,7 +142,7 @@ func dockerSetupWithClient(t *testing.T, task *structs.Task, client *docker.Clie
cleanup := func() {
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
sresp.Handle.Kill()
tctx.AllocDir.Destroy()
tctx.Destroy()
}
return client, sresp.Handle.(*DockerHandle), cleanup
@ -169,7 +169,7 @@ func TestDockerDriver_Fingerprint(t *testing.T) {
ctx := testDockerDriverContexts(t, &structs.Task{Name: "foo", Driver: "docker", Resources: basicResources})
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
node := &structs.Node{
Attributes: make(map[string]string),
@ -319,7 +319,7 @@ func TestDockerDriver_StartOpen_Wait(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
@ -423,7 +423,7 @@ func TestDockerDriver_Start_StoppedContainer(t *testing.T) {
}
tctx := testDockerDriverContexts(t, task)
defer tctx.AllocDir.Destroy()
defer tctx.Destroy()
copyImage(t, tctx.ExecCtx.TaskDir, "busybox.tar")
client := newTestDockerClient(t)
@ -495,7 +495,7 @@ func TestDockerDriver_Start_LoadImage(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
// Copy the image into the task's directory
@ -563,7 +563,7 @@ func TestDockerDriver_Start_BadPull_Recoverable(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
_, err := d.Prestart(ctx.ExecCtx, task)
@ -616,7 +616,7 @@ func TestDockerDriver_Start_Wait_AllocDir(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
@ -769,7 +769,7 @@ func TestDockerDriver_StartN(t *testing.T) {
for idx, task := range taskList {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
@ -834,7 +834,7 @@ func TestDockerDriver_StartNVersions(t *testing.T) {
for idx, task := range taskList {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
copyImage(t, ctx.ExecCtx.TaskDir, "busybox_musl.tar")
@ -1063,7 +1063,7 @@ func TestDockerDriver_Sysctl_Ulimit_Errors(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
driver := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
_, err := driver.Prestart(ctx.ExecCtx, task)
assert.NotNil(t, err, "Expected non nil error")
@ -1118,7 +1118,7 @@ func TestDockerDriver_ForcePull_IsInvalidConfig(t *testing.T) {
task.Config["force_pull"] = "nothing"
ctx := testDockerDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
driver := NewDockerDriver(ctx.DriverCtx)
@ -1279,7 +1279,7 @@ func TestDockerDriver_Capabilities(t *testing.T) {
driver := NewDockerDriver(tctx.DriverCtx)
copyImage(t, tctx.ExecCtx.TaskDir, "busybox.tar")
defer tctx.AllocDir.Destroy()
defer tctx.Destroy()
presp, err := driver.Prestart(tctx.ExecCtx, task)
defer driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
@ -1571,7 +1571,7 @@ func TestDockerDriver_User(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
driver := NewDockerDriver(ctx.DriverCtx)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
_, err := driver.Prestart(ctx.ExecCtx, task)
@ -1731,7 +1731,7 @@ func setupDockerVolumes(t *testing.T, cfg *config.Config, hostpath string) (*str
}
// Build alloc and task directory structure
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(cfg.AllocDir, uuid.Generate()))
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(cfg.AllocDir, uuid.Generate()))
if err := allocDir.Build(); err != nil {
t.Fatalf("failed to build alloc dir: %v", err)
}
@ -2055,7 +2055,7 @@ func TestDockerDriver_Mounts(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
driver := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
_, err := driver.Prestart(ctx.ExecCtx, task)
if err == nil && c.Error != "" {
@ -2089,7 +2089,7 @@ func TestDockerDriver_Cleanup(t *testing.T) {
},
}
tctx := testDockerDriverContexts(t, task)
defer tctx.AllocDir.Destroy()
defer tctx.Destroy()
// Run Prestart
driver := NewDockerDriver(tctx.DriverCtx).(*DockerDriver)
@ -2277,7 +2277,7 @@ func TestDockerDriver_Devices_IsInvalidConfig(t *testing.T) {
ctx := testDockerDriverContexts(t, task)
driver := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
if _, err := driver.Prestart(ctx.ExecCtx, task); err == nil || err.Error() != tc.err.Error() {
t.Fatalf("error expected in prestart, got %v, expected %v", err, tc.err)
@ -2382,7 +2382,7 @@ func TestDockerDriver_Kill(t *testing.T) {
}
ctx := testDockerDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
@ -2502,7 +2502,7 @@ func TestDockerDriver_AdvertiseIPv6Address(t *testing.T) {
tctx := testDockerDriverContexts(t, task)
driver := NewDockerDriver(tctx.DriverCtx)
copyImage(t, tctx.ExecCtx.TaskDir, "busybox.tar")
defer tctx.AllocDir.Destroy()
defer tctx.Destroy()
presp, err := driver.Prestart(tctx.ExecCtx, task)
defer driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)

View File

@ -43,7 +43,7 @@ func TestDockerDriver_Signal(t *testing.T) {
}
ctx := testDockerDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewDockerDriver(ctx.DriverCtx)
// Copy the image into the task's directory

View File

@ -320,6 +320,10 @@ type DriverHandle interface {
// ScriptExecutor is an interface used to execute commands such as
// health check scripts in the a DriverHandle's context.
ScriptExecutor
// Network returns the driver's network or nil if the driver did not
// create a network.
Network() *cstructs.DriverNetwork
}
// ScriptExecutor is an interface that supports Exec()ing commands in the
@ -335,6 +339,12 @@ type ExecContext struct {
// TaskEnv contains the task's environment variables.
TaskEnv *env.TaskEnv
// StdoutFifo is the path to the named pipe to write stdout to
StdoutFifo string
// StderrFifo is the path to the named pipe to write stderr to
StderrFifo string
}
// NewExecContext is used to create a new execution context

View File

@ -1,20 +1,25 @@
package driver
import (
"fmt"
"io"
"io/ioutil"
"math/rand"
"os"
"path/filepath"
"reflect"
"runtime"
"testing"
"time"
plugin "github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/client/logmon"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/helper/testtask"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
)
@ -95,26 +100,36 @@ func testConfig(t *testing.T) *config.Config {
conf.MaxKillTimeout = 10 * time.Second
conf.Region = "global"
conf.Node = mock.Node()
conf.LogLevel = "DEBUG"
conf.LogOutput = testlog.NewWriter(t)
return conf
}
type testContext struct {
AllocDir *allocdir.AllocDir
DriverCtx *DriverContext
ExecCtx *ExecContext
EnvBuilder *env.Builder
AllocDir *allocdir.AllocDir
DriverCtx *DriverContext
ExecCtx *ExecContext
EnvBuilder *env.Builder
logmon logmon.LogMon
logmonPlugin *plugin.Client
}
func (ctx *testContext) Destroy() {
ctx.AllocDir.Destroy()
ctx.logmon.Stop()
ctx.logmonPlugin.Kill()
}
// testDriverContext sets up an alloc dir, task dir, DriverContext, and ExecContext.
//
// It is up to the caller to call AllocDir.Destroy to cleanup.
// It is up to the caller to call Destroy to cleanup.
func testDriverContexts(t *testing.T, task *structs.Task) *testContext {
cfg := testConfig(t)
cfg.Node = mock.Node()
alloc := mock.Alloc()
alloc.NodeID = cfg.Node.ID
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(cfg.AllocDir, alloc.ID))
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(cfg.AllocDir, alloc.ID))
if err := allocDir.Build(); err != nil {
t.Fatalf("AllocDir.Build() failed: %v", err)
}
@ -139,12 +154,45 @@ func testDriverContexts(t *testing.T, task *structs.Task) *testContext {
execCtx := NewExecContext(td, eb.Build())
logger := testlog.Logger(t)
hcLogger := testlog.HCLogger(t)
emitter := func(m string, args ...interface{}) {
logger.Printf("[EVENT] "+m, args...)
hcLogger.Info(fmt.Sprintf("[EVENT] "+m, args...))
}
driverCtx := NewDriverContext(alloc.Job.Name, alloc.TaskGroup, task.Name, alloc.ID, cfg, cfg.Node, logger, emitter)
l, c, err := logmon.LaunchLogMon(hcLogger)
if err != nil {
allocDir.Destroy()
t.Fatalf("LaunchLogMon() failed: %v", err)
}
return &testContext{allocDir, driverCtx, execCtx, eb}
var stdoutFifo, stderrFifo string
if runtime.GOOS == "windows" {
id := uuid.Generate()[:8]
stdoutFifo = fmt.Sprintf("//./pipe/%s.stdout.%s", id, task.Name)
stderrFifo = fmt.Sprintf("//./pipe/%s.stderr.%s", id, task.Name)
} else {
stdoutFifo = filepath.Join(td.LogDir, fmt.Sprintf("%s.stdout", task.Name))
stderrFifo = filepath.Join(td.LogDir, fmt.Sprintf("%s.stderr", task.Name))
}
err = l.Start(&logmon.LogConfig{
LogDir: td.LogDir,
StdoutLogFile: fmt.Sprintf("%s.stdout", task.Name),
StderrLogFile: fmt.Sprintf("%s.stderr", task.Name),
StdoutFifo: stdoutFifo,
StderrFifo: stderrFifo,
MaxFiles: 10,
MaxFileSizeMB: 10,
})
if err != nil {
allocDir.Destroy()
t.Fatalf("LogMon.Start() failed: %v", err)
}
execCtx.StdoutFifo = stdoutFifo
execCtx.StderrFifo = stderrFifo
return &testContext{allocDir, driverCtx, execCtx, eb, l, c}
}
// setupTaskEnv creates a test env for GetTaskEnv testing. Returns task dir,
@ -179,7 +227,7 @@ func setupTaskEnv(t *testing.T, driver string) (*allocdir.TaskDir, map[string]st
alloc.Name = "Bar"
alloc.AllocatedResources.Tasks["web"].Networks[0].DynamicPorts[0].Value = 2000
conf := testConfig(t)
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(conf.AllocDir, alloc.ID))
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(conf.AllocDir, alloc.ID))
taskDir := allocDir.NewTaskDir(task.Name)
eb := env.NewBuilder(conf.Node, alloc, task, conf.Region)
tmpDriver, err := NewDriver(driver, NewEmptyDriverContext())

View File

@ -247,7 +247,8 @@ func NewBuilder(node *structs.Node, alloc *structs.Allocation, task *structs.Tas
// NewEmptyBuilder creates a new environment builder.
func NewEmptyBuilder() *Builder {
return &Builder{
mu: &sync.RWMutex{},
mu: &sync.RWMutex{},
envvars: make(map[string]string),
}
}
@ -362,6 +363,16 @@ func (b *Builder) UpdateTask(alloc *structs.Allocation, task *structs.Task) *Bui
return b.setTask(task).setAlloc(alloc)
}
func (b *Builder) SetGenericEnv(envs map[string]string) *Builder {
b.mu.Lock()
defer b.mu.Unlock()
for k, v := range envs {
b.envvars[k] = v
}
return b
}
// setTask is called from NewBuilder to populate task related environment
// variables.
func (b *Builder) setTask(task *structs.Task) *Builder {

View File

@ -37,17 +37,17 @@ type ExecDriverConfig struct {
// execHandle is returned from Start/Open as a handle to the PID
type execHandle struct {
pluginClient *plugin.Client
executor executor.Executor
isolationConfig *dstructs.IsolationConfig
userPid int
taskDir *allocdir.TaskDir
killTimeout time.Duration
maxKillTimeout time.Duration
logger *log.Logger
waitCh chan *dstructs.WaitResult
doneCh chan struct{}
version string
pluginClient *plugin.Client
executor executor.Executor
userPid int
taskShutdownSignal string
taskDir *allocdir.TaskDir
killTimeout time.Duration
maxKillTimeout time.Duration
logger *log.Logger
waitCh chan *dstructs.WaitResult
doneCh chan struct{}
version string
}
// NewExecDriver is used to create a new exec driver
@ -110,26 +110,16 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
pluginLogFile := filepath.Join(ctx.TaskDir.Dir, "executor.out")
executorConfig := &dstructs.ExecutorConfig{
LogFile: pluginLogFile,
LogLevel: d.config.LogLevel,
LogFile: pluginLogFile,
LogLevel: d.config.LogLevel,
FSIsolation: true,
}
exec, pluginClient, err := createExecutor(d.config.LogOutput, d.config, executorConfig)
if err != nil {
return nil, err
}
executorCtx := &executor.ExecutorContext{
TaskEnv: ctx.TaskEnv,
Driver: "exec",
LogDir: ctx.TaskDir.LogDir,
TaskDir: ctx.TaskDir.Dir,
Task: task,
}
if err := exec.SetContext(executorCtx); err != nil {
pluginClient.Kill()
return nil, fmt.Errorf("failed to set executor context: %v", err)
}
taskKillSignal, err := getTaskKillSignal(task.KillSignal)
_, err = getTaskKillSignal(task.KillSignal)
if err != nil {
return nil, err
}
@ -137,13 +127,21 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
execCmd := &executor.ExecCommand{
Cmd: command,
Args: driverConfig.Args,
TaskKillSignal: taskKillSignal,
FSIsolation: true,
ResourceLimits: true,
User: getExecutorUser(task),
Resources: &executor.Resources{
CPU: task.Resources.CPU,
MemoryMB: task.Resources.MemoryMB,
IOPS: task.Resources.IOPS,
DiskMB: task.Resources.DiskMB,
},
Env: ctx.TaskEnv.List(),
TaskDir: ctx.TaskDir.Dir,
StdoutPath: ctx.StdoutFifo,
StderrPath: ctx.StderrFifo,
}
ps, err := exec.LaunchCmd(execCmd)
ps, err := exec.Launch(execCmd)
if err != nil {
pluginClient.Kill()
return nil, err
@ -154,17 +152,17 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
// Return a driver handle
maxKill := d.DriverContext.config.MaxKillTimeout
h := &execHandle{
pluginClient: pluginClient,
userPid: ps.Pid,
executor: exec,
isolationConfig: ps.IsolationConfig,
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
maxKillTimeout: maxKill,
logger: d.logger,
version: d.config.Version.VersionNumber(),
doneCh: make(chan struct{}),
waitCh: make(chan *dstructs.WaitResult, 1),
taskDir: ctx.TaskDir,
pluginClient: pluginClient,
userPid: ps.Pid,
taskShutdownSignal: task.KillSignal,
executor: exec,
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
maxKillTimeout: maxKill,
logger: d.logger,
version: d.config.Version.VersionNumber(),
doneCh: make(chan struct{}),
waitCh: make(chan *dstructs.WaitResult, 1),
taskDir: ctx.TaskDir,
}
go h.run()
return &StartResponse{Handle: h}, nil
@ -173,12 +171,11 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
func (d *ExecDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }
type execId struct {
Version string
KillTimeout time.Duration
MaxKillTimeout time.Duration
UserPid int
IsolationConfig *dstructs.IsolationConfig
PluginConfig *PluginReattachConfig
Version string
KillTimeout time.Duration
MaxKillTimeout time.Duration
UserPid int
PluginConfig *PluginReattachConfig
}
func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error) {
@ -198,12 +195,6 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
if e := destroyPlugin(id.PluginConfig.Pid, id.UserPid); e != nil {
merrs.Errors = append(merrs.Errors, fmt.Errorf("error destroying plugin and userpid: %v", e))
}
if id.IsolationConfig != nil {
ePid := pluginConfig.Reattach.Pid
if e := executor.ClientCleanup(id.IsolationConfig, ePid); e != nil {
merrs.Errors = append(merrs.Errors, fmt.Errorf("destroying cgroup failed: %v", e))
}
}
return nil, fmt.Errorf("error connecting to plugin: %v", merrs.ErrorOrNil())
}
@ -211,17 +202,16 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
d.logger.Printf("[DEBUG] driver.exec : version of executor: %v", ver.Version)
// Return a driver handle
h := &execHandle{
pluginClient: client,
executor: exec,
userPid: id.UserPid,
isolationConfig: id.IsolationConfig,
logger: d.logger,
version: id.Version,
killTimeout: id.KillTimeout,
maxKillTimeout: id.MaxKillTimeout,
doneCh: make(chan struct{}),
waitCh: make(chan *dstructs.WaitResult, 1),
taskDir: ctx.TaskDir,
pluginClient: client,
executor: exec,
userPid: id.UserPid,
logger: d.logger,
version: id.Version,
killTimeout: id.KillTimeout,
maxKillTimeout: id.MaxKillTimeout,
doneCh: make(chan struct{}),
waitCh: make(chan *dstructs.WaitResult, 1),
taskDir: ctx.TaskDir,
}
go h.run()
return h, nil
@ -229,12 +219,11 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
func (h *execHandle) ID() string {
id := execId{
Version: h.version,
KillTimeout: h.killTimeout,
MaxKillTimeout: h.maxKillTimeout,
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
UserPid: h.userPid,
IsolationConfig: h.isolationConfig,
Version: h.version,
KillTimeout: h.killTimeout,
MaxKillTimeout: h.maxKillTimeout,
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
UserPid: h.userPid,
}
data, err := json.Marshal(id)
@ -251,7 +240,12 @@ func (h *execHandle) WaitCh() chan *dstructs.WaitResult {
func (h *execHandle) Update(task *structs.Task) error {
// Store the updated kill timeout.
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
h.executor.UpdateTask(task)
h.executor.UpdateResources(&executor.Resources{
CPU: task.Resources.CPU,
MemoryMB: task.Resources.MemoryMB,
IOPS: task.Resources.IOPS,
DiskMB: task.Resources.DiskMB,
})
// Update is not possible
return nil
@ -270,12 +264,16 @@ func (h *execHandle) Signal(s os.Signal) error {
return h.executor.Signal(s)
}
func (d *execHandle) Network() *cstructs.DriverNetwork {
return nil
}
func (h *execHandle) Kill() error {
if err := h.executor.ShutDown(); err != nil {
if err := h.executor.Shutdown(h.taskShutdownSignal, h.killTimeout); err != nil {
if h.pluginClient.Exited() {
return nil
}
return fmt.Errorf("executor Shutdown failed: %v", err)
return fmt.Errorf("executor Kill failed: %v", err)
}
select {
@ -284,8 +282,8 @@ func (h *execHandle) Kill() error {
if h.pluginClient.Exited() {
break
}
if err := h.executor.Exit(); err != nil {
return fmt.Errorf("executor Exit failed: %v", err)
if err := h.executor.Shutdown(h.taskShutdownSignal, h.killTimeout); err != nil {
return fmt.Errorf("executor Destroy failed: %v", err)
}
}
return nil
@ -299,22 +297,8 @@ func (h *execHandle) run() {
ps, werr := h.executor.Wait()
close(h.doneCh)
// If the exitcode is 0 and we had an error that means the plugin didn't
// connect and doesn't know the state of the user process so we are killing
// the user process so that when we create a new executor on restarting the
// new user process doesn't have collisions with resources that the older
// user pid might be holding onto.
if ps.ExitCode == 0 && werr != nil {
if h.isolationConfig != nil {
ePid := h.pluginClient.ReattachConfig().Pid
if e := executor.ClientCleanup(h.isolationConfig, ePid); e != nil {
h.logger.Printf("[ERR] driver.exec: destroying resource container failed: %v", e)
}
}
}
// Exit the executor
if err := h.executor.Exit(); err != nil {
// Destroy the executor
if err := h.executor.Shutdown(h.taskShutdownSignal, 0); err != nil {
h.logger.Printf("[ERR] driver.exec: error destroying executor: %v", err)
}
h.pluginClient.Kill()

View File

@ -56,7 +56,7 @@ func TestExecDriver_Fingerprint(t *testing.T) {
Resources: structs.DefaultResources(),
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
node := &structs.Node{
Attributes: map[string]string{
@ -100,7 +100,7 @@ func TestExecDriver_StartOpen_Wait(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -144,7 +144,7 @@ func TestExecDriver_Start_Wait(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -198,7 +198,7 @@ func TestExecDriver_Start_Wait_AllocDir(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -252,7 +252,7 @@ func TestExecDriver_Start_Kill_Wait(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -304,7 +304,7 @@ func TestExecDriverUser(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -343,7 +343,7 @@ func TestExecDriver_HandlerExec(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
//defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -397,7 +397,7 @@ func TestExecDriver_HandlerExec(t *testing.T) {
if line == "" {
continue
}
if !strings.Contains(line, ":/nomad/") {
if !strings.Contains(line, ":/nomad/") && !strings.Contains(line, ":name=") {
t.Errorf("Not a member of the alloc's cgroup: expected=...:/nomad/... -- found=%q", line)
continue
}
@ -420,6 +420,7 @@ func TestExecDriver_HandlerExec(t *testing.T) {
}
if err := handle.Kill(); err != nil {
t.Logf("Check allocdir: %x", ctx.AllocDir.AllocDir)
t.Fatalf("error killing exec handle: %v", err)
}
}

View File

@ -38,7 +38,7 @@ func TestExecDriver_KillUserPid_OnPluginReconnectFailure(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
@ -111,7 +111,7 @@ func TestExecDriver_Signal(t *testing.T) {
}
ctx := testDriverContexts(t, task)
defer ctx.AllocDir.Destroy()
defer ctx.Destroy()
d := NewExecDriver(ctx.DriverCtx)
testFile := filepath.Join(ctx.ExecCtx.TaskDir.Dir, "test.sh")

File diff suppressed because it is too large Load Diff

View File

@ -1,46 +1,15 @@
// +build darwin dragonfly freebsd netbsd openbsd solaris windows
// +build !linux
package executor
import (
"os"
import hclog "github.com/hashicorp/go-hclog"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/mitchellh/go-ps"
)
func (e *UniversalExecutor) configureChroot() error {
return nil
func NewExecutorWithIsolation(logger hclog.Logger) Executor {
logger = logger.Named("executor")
logger.Error("isolation executor is not supported on this platform, using default")
return NewExecutor(logger)
}
func (e *UniversalExecutor) removeChrootMounts() error {
return nil
}
func (e *UniversalExecutor) configureResourceContainer(_ int) error { return nil }
func (e *UniversalExecutor) runAs(userid string) error {
return nil
}
func (e *UniversalExecutor) applyLimits(pid int) error {
return nil
}
func (e *UniversalExecutor) configureIsolation() error {
return nil
}
func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
pidStats, err := e.pidStats()
if err != nil {
return nil, err
}
return e.aggregatedResourceUsage(pidStats), nil
}
func (e *UniversalExecutor) getAllPids() (map[int]*nomadPid, error) {
allProcesses, err := ps.Processes()
if err != nil {
return nil, err
}
return e.scanPids(os.Getpid(), allProcesses)
}
func (e *UniversalExecutor) runAs(_ string) error { return nil }

View File

@ -1,188 +1,333 @@
// +build linux
package executor
import (
"context"
"fmt"
"os"
"os/user"
"os/exec"
"path"
"path/filepath"
"strconv"
"strings"
"syscall"
"time"
"github.com/hashicorp/go-multierror"
"github.com/mitchellh/go-ps"
cgroupFs "github.com/opencontainers/runc/libcontainer/cgroups/fs"
cgroupConfig "github.com/opencontainers/runc/libcontainer/configs"
"github.com/armon/circbuf"
"github.com/hashicorp/consul-template/signals"
hclog "github.com/hashicorp/go-hclog"
multierror "github.com/hashicorp/go-multierror"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
"github.com/hashicorp/nomad/client/stats"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/discover"
shelpers "github.com/hashicorp/nomad/helper/stats"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/opencontainers/runc/libcontainer"
"github.com/opencontainers/runc/libcontainer/cgroups"
cgroupFs "github.com/opencontainers/runc/libcontainer/cgroups/fs"
lconfigs "github.com/opencontainers/runc/libcontainer/configs"
"github.com/syndtr/gocapability/capability"
)
const (
defaultCgroupParent = "nomad"
)
var (
// The statistics the executor exposes when using cgroups
ExecutorCgroupMeasuredMemStats = []string{"RSS", "Cache", "Swap", "Max Usage", "Kernel Usage", "Kernel Max Usage"}
ExecutorCgroupMeasuredCpuStats = []string{"System Mode", "User Mode", "Throttled Periods", "Throttled Time", "Percent"}
// allCaps is all linux capabilities which is used to configure libcontainer
allCaps []string
)
// configureIsolation configures chroot and creates cgroups
func (e *UniversalExecutor) configureIsolation() error {
if e.command.FSIsolation {
if err := e.configureChroot(); err != nil {
return err
}
// initialize the allCaps var with all capabilities available on the system
func init() {
last := capability.CAP_LAST_CAP
// workaround for RHEL6 which has no /proc/sys/kernel/cap_last_cap
if last == capability.Cap(63) {
last = capability.CAP_BLOCK_SUSPEND
}
if e.command.ResourceLimits || e.command.BasicProcessCgroup {
if err := e.configureCgroups(e.ctx.Task.Resources); err != nil {
return fmt.Errorf("error creating cgroups: %v", err)
for _, cap := range capability.List() {
if cap > last {
continue
}
allCaps = append(allCaps, fmt.Sprintf("CAP_%s", strings.ToUpper(cap.String())))
}
return nil
}
// applyLimits puts a process in a pre-configured cgroup
func (e *UniversalExecutor) applyLimits(pid int) error {
if !(e.command.ResourceLimits || e.command.BasicProcessCgroup) {
return nil
}
// LibcontainerExecutor implements an Executor with the runc/libcontainer api
type LibcontainerExecutor struct {
id string
command *ExecCommand
// Entering the process in the cgroup
manager := getCgroupManager(e.resConCtx.groups, nil)
if err := manager.Apply(pid); err != nil {
e.logger.Printf("[ERR] executor: error applying pid to cgroup: %v", err)
return err
}
logger hclog.Logger
e.resConCtx.cgPaths = manager.GetPaths()
totalCpuStats *stats.CpuStats
userCpuStats *stats.CpuStats
systemCpuStats *stats.CpuStats
pidCollector *pidCollector
// Don't enter all the cgroups since we will inherit resources limits. Only
// use devices (required by libcontainer) and freezer. Freezer allows us to
// capture all pids and stop any fork/execs from happening while we are
// cleaning up.
if !e.command.ResourceLimits {
// Move the executor into the global cgroup so that the task specific
// cgroup can be destroyed.
nilGroup := &cgroupConfig.Cgroup{}
nilGroup.Path = "/"
nilGroup.Resources = e.resConCtx.groups.Resources
nilManager := getCgroupManager(nilGroup, nil)
err := nilManager.Apply(pid)
if err != nil {
return fmt.Errorf("failed to remove executor pid %d: %v", pid, err)
}
// Grab the freezer and devices cgroup paths. We do this from the old
// manager after the executor pid has been applied since there is no
// other way to determine what the proper cgroup paths would be.
freezer := &cgroupFs.FreezerGroup{}
devices := &cgroupFs.DevicesGroup{}
freezerName, devicesName := freezer.Name(), devices.Name()
newPath := map[string]string{
freezerName: e.resConCtx.cgPaths[freezerName],
devicesName: e.resConCtx.cgPaths[devicesName],
}
// Clear the cgroups paths so that everything is properly cleaned except
// the groups we want our process to stay in. This will delete the
// directories from disk.
manager.Cgroups.Paths = nil
delete(manager.Paths, freezerName)
delete(manager.Paths, devicesName)
if err := manager.Destroy(); err != nil {
e.logger.Printf("[ERR] executor: failed to destroy original: %v", err)
return err
}
// Update our context such that the new cgroup manager only is tracking
// the paths we care about now.
e.resConCtx.cgPaths = newPath
e.resConCtx.groups.Paths = newPath
// Apply just the freezer and devices now
manager = getCgroupManager(e.resConCtx.groups, e.resConCtx.cgPaths)
if err := manager.Apply(pid); err != nil {
e.logger.Printf("[ERR] executor: error applying pid to cgroup subset %v: %v", e.resConCtx.cgPaths, err)
return err
}
}
cgConfig := cgroupConfig.Config{Cgroups: e.resConCtx.groups}
if err := manager.Set(&cgConfig); err != nil {
e.logger.Printf("[ERR] executor: error setting cgroup config: %v", err)
if er := DestroyCgroup(e.resConCtx.groups, e.resConCtx.cgPaths, os.Getpid()); er != nil {
e.logger.Printf("[ERR] executor: error destroying cgroup: %v", er)
}
return err
}
return nil
container libcontainer.Container
userProc *libcontainer.Process
userProcExited chan interface{}
exitState *ProcessState
}
// configureCgroups converts a Nomad Resources specification into the equivalent
// cgroup configuration. It returns an error if the resources are invalid.
func (e *UniversalExecutor) configureCgroups(resources *structs.Resources) error {
e.resConCtx.groups = &cgroupConfig.Cgroup{}
e.resConCtx.groups.Resources = &cgroupConfig.Resources{}
cgroupName := uuid.Generate()
e.resConCtx.groups.Path = filepath.Join("/nomad", cgroupName)
// Allow access to /dev/
e.resConCtx.groups.Resources.AllowAllDevices = true
// Use a cgroup but don't apply limits
if !e.command.ResourceLimits {
return nil
func NewExecutorWithIsolation(logger hclog.Logger) Executor {
logger = logger.Named("isolated_executor")
if err := shelpers.Init(); err != nil {
logger.Error("unable to initialize stats", "error", err)
}
if resources.MemoryMB > 0 {
// Total amount of memory allowed to consume
e.resConCtx.groups.Resources.Memory = int64(resources.MemoryMB * 1024 * 1024)
// Disable swap to avoid issues on the machine
var memSwappiness int64 = 0
e.resConCtx.groups.Resources.MemorySwappiness = &memSwappiness
return &LibcontainerExecutor{
id: strings.Replace(uuid.Generate(), "-", "_", 0),
logger: logger,
totalCpuStats: stats.NewCpuStats(),
userCpuStats: stats.NewCpuStats(),
systemCpuStats: stats.NewCpuStats(),
pidCollector: newPidCollector(logger),
}
if resources.CPU < 2 {
return fmt.Errorf("resources.CPU must be equal to or greater than 2: %v", resources.CPU)
}
// Set the relative CPU shares for this cgroup.
e.resConCtx.groups.Resources.CpuShares = int64(resources.CPU)
if resources.IOPS != 0 {
// Validate it is in an acceptable range.
if resources.IOPS < 10 || resources.IOPS > 1000 {
return fmt.Errorf("resources.IOPS must be between 10 and 1000: %d", resources.IOPS)
}
e.resConCtx.groups.Resources.BlkioWeight = uint16(resources.IOPS)
}
return nil
}
// Stats reports the resource utilization of the cgroup. If there is no resource
// isolation we aggregate the resource utilization of all the pids launched by
// the executor.
func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
// If we don't use full resource limits fallback to normal collection. It is
// not enough to be in the Cgroup since you must be in the memory, cpu, and
// cpuacct cgroup to gather the correct statistics.
if !e.command.ResourceLimits {
pidStats, err := e.pidStats()
if err != nil {
return nil, err
}
return e.aggregatedResourceUsage(pidStats), nil
// Launch creates a new container in libcontainer and starts a new process with it
func (l *LibcontainerExecutor) Launch(command *ExecCommand) (*ProcessState, error) {
l.logger.Info("launching command", "command", command.Cmd, "args", strings.Join(command.Args, " "))
// Find the nomad executable to launch the executor process with
bin, err := discover.NomadExecutable()
if err != nil {
return nil, fmt.Errorf("unable to find the nomad binary: %v", err)
}
ts := time.Now()
manager := getCgroupManager(e.resConCtx.groups, e.resConCtx.cgPaths)
stats, err := manager.GetStats()
l.command = command
// Move to the root cgroup until process is started
subsystems, err := cgroups.GetAllSubsystems()
if err != nil {
return nil, err
}
if err := JoinRootCgroup(subsystems); err != nil {
return nil, err
}
// create a new factory which will store the container state in the allocDir
factory, err := libcontainer.New(
path.Join(command.TaskDir, "../alloc/container"),
libcontainer.Cgroupfs,
libcontainer.InitArgs(bin, "libcontainer-shim"),
)
if err != nil {
return nil, fmt.Errorf("failed to create factory: %v", err)
}
// A container groups processes under the same isolation enforcement
container, err := factory.Create(l.id, newLibcontainerConfig(command))
if err != nil {
return nil, fmt.Errorf("failed to create container(%s): %v", l.id, err)
}
l.container = container
combined := append([]string{command.Cmd}, command.Args...)
stdout, err := command.Stdout()
if err != nil {
return nil, err
}
stderr, err := command.Stderr()
if err != nil {
return nil, err
}
// the task process will be started by the container
process := &libcontainer.Process{
Args: combined,
Env: command.Env,
Stdout: stdout,
Stderr: stderr,
Init: true,
}
if command.User != "" {
process.User = command.User
}
l.userProc = process
l.totalCpuStats = stats.NewCpuStats()
l.userCpuStats = stats.NewCpuStats()
l.systemCpuStats = stats.NewCpuStats()
// Starts the task
if err := container.Run(process); err != nil {
container.Destroy()
return nil, err
}
pid, err := process.Pid()
if err != nil {
container.Destroy()
return nil, err
}
// Join process cgroups
containerState, err := container.State()
if err != nil {
l.logger.Error("error entering user process cgroups", "executor_pid", os.Getpid(), "error", err)
}
if err := cgroups.EnterPid(containerState.CgroupPaths, os.Getpid()); err != nil {
l.logger.Error("error entering user process cgroups", "executor_pid", os.Getpid(), "error", err)
}
// start a goroutine to wait on the process to complete, so Wait calls can
// be multiplexed
l.userProcExited = make(chan interface{})
go l.pidCollector.collectPids(l.userProcExited, l.getAllPids)
go l.wait()
return &ProcessState{
Pid: pid,
ExitCode: -1,
Time: time.Now(),
}, nil
}
func (l *LibcontainerExecutor) getAllPids() (map[int]*nomadPid, error) {
pids, err := l.container.Processes()
if err != nil {
return nil, err
}
nPids := make(map[int]*nomadPid)
for _, pid := range pids {
nPids[pid] = &nomadPid{
pid: pid,
cpuStatsTotal: stats.NewCpuStats(),
cpuStatsUser: stats.NewCpuStats(),
cpuStatsSys: stats.NewCpuStats(),
}
}
return nPids, nil
}
// Wait waits until a process has exited and returns it's exitcode and errors
func (l *LibcontainerExecutor) Wait() (*ProcessState, error) {
<-l.userProcExited
return l.exitState, nil
}
func (l *LibcontainerExecutor) wait() {
defer close(l.userProcExited)
ps, err := l.userProc.Wait()
if err != nil {
// If the process has exited before we called wait an error is returned
// the process state is embedded in the error
if exitErr, ok := err.(*exec.ExitError); ok {
ps = exitErr.ProcessState
} else {
l.logger.Error("failed to call wait on user process", "error", err)
l.exitState = &ProcessState{Pid: 0, ExitCode: 0, Time: time.Now()}
return
}
}
l.command.Close()
exitCode := 1
var signal int
if status, ok := ps.Sys().(syscall.WaitStatus); ok {
exitCode = status.ExitStatus()
if status.Signaled() {
const exitSignalBase = 128
signal = int(status.Signal())
exitCode = exitSignalBase + signal
}
}
l.exitState = &ProcessState{
Pid: ps.Pid(),
ExitCode: exitCode,
Signal: signal,
Time: time.Now(),
}
}
// Shutdown stops all processes started and cleans up any resources
// created (such as mountpoints, devices, etc).
func (l *LibcontainerExecutor) Shutdown(signal string, grace time.Duration) error {
if l.container == nil {
return nil
}
// move executor to root cgroup
subsystems, err := cgroups.GetAllSubsystems()
if err != nil {
return err
}
if err := JoinRootCgroup(subsystems); err != nil {
return err
}
status, err := l.container.Status()
if err != nil {
return err
}
defer l.container.Destroy()
if status == libcontainer.Stopped {
return nil
}
if grace > 0 {
if signal == "" {
signal = "SIGINT"
}
sig, ok := signals.SignalLookup[signal]
if !ok {
return fmt.Errorf("error unknown signal given for shutdown: %s", signal)
}
err = l.container.Signal(sig, false)
if err != nil {
return err
}
select {
case <-l.userProcExited:
return nil
case <-time.After(grace):
return l.container.Signal(os.Kill, false)
}
} else {
return l.container.Signal(os.Kill, false)
}
}
// UpdateResources updates the resource isolation with new values to be enforced
func (l *LibcontainerExecutor) UpdateResources(resources *Resources) error {
return nil
}
// Version returns the api version of the executor
func (l *LibcontainerExecutor) Version() (*ExecutorVersion, error) {
return &ExecutorVersion{Version: ExecutorVersionLatest}, nil
}
// Stats returns the resource statistics for processes managed by the executor
func (l *LibcontainerExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
lstats, err := l.container.Stats()
if err != nil {
return nil, err
}
pidStats, err := l.pidCollector.pidStats()
if err != nil {
return nil, err
}
ts := time.Now()
stats := lstats.CgroupStats
// Memory Related Stats
swap := stats.MemoryStats.SwapUsage
@ -204,14 +349,14 @@ func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
userModeTime := float64(stats.CpuStats.CpuUsage.UsageInUsermode)
kernelModeTime := float64(stats.CpuStats.CpuUsage.UsageInKernelmode)
totalPercent := e.totalCpuStats.Percent(totalProcessCPUUsage)
totalPercent := l.totalCpuStats.Percent(totalProcessCPUUsage)
cs := &cstructs.CpuStats{
SystemMode: e.systemCpuStats.Percent(kernelModeTime),
UserMode: e.userCpuStats.Percent(userModeTime),
SystemMode: l.systemCpuStats.Percent(kernelModeTime),
UserMode: l.userCpuStats.Percent(userModeTime),
Percent: totalPercent,
ThrottledPeriods: stats.CpuStats.ThrottlingData.ThrottledPeriods,
ThrottledTime: stats.CpuStats.ThrottlingData.ThrottledTime,
TotalTicks: e.systemCpuStats.TicksConsumed(totalPercent),
TotalTicks: l.systemCpuStats.TicksConsumed(totalPercent),
Measured: ExecutorCgroupMeasuredCpuStats,
}
taskResUsage := cstructs.TaskResourceUsage{
@ -220,181 +365,246 @@ func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
CpuStats: cs,
},
Timestamp: ts.UTC().UnixNano(),
Pids: pidStats,
}
if pidStats, err := e.pidStats(); err == nil {
taskResUsage.Pids = pidStats
}
return &taskResUsage, nil
}
// runAs takes a user id as a string and looks up the user, and sets the command
// to execute as that user.
func (e *UniversalExecutor) runAs(userid string) error {
u, err := user.Lookup(userid)
if err != nil {
return fmt.Errorf("Failed to identify user %v: %v", userid, err)
}
// Get the groups the user is a part of
gidStrings, err := u.GroupIds()
if err != nil {
return fmt.Errorf("Unable to lookup user's group membership: %v", err)
}
gids := make([]uint32, len(gidStrings))
for _, gidString := range gidStrings {
u, err := strconv.Atoi(gidString)
if err != nil {
return fmt.Errorf("Unable to convert user's group to int %s: %v", gidString, err)
}
gids = append(gids, uint32(u))
}
// Convert the uid and gid
uid, err := strconv.ParseUint(u.Uid, 10, 32)
if err != nil {
return fmt.Errorf("Unable to convert userid to uint32: %s", err)
}
gid, err := strconv.ParseUint(u.Gid, 10, 32)
if err != nil {
return fmt.Errorf("Unable to convert groupid to uint32: %s", err)
}
// Set the command to run as that user and group.
if e.cmd.SysProcAttr == nil {
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
}
if e.cmd.SysProcAttr.Credential == nil {
e.cmd.SysProcAttr.Credential = &syscall.Credential{}
}
e.cmd.SysProcAttr.Credential.Uid = uint32(uid)
e.cmd.SysProcAttr.Credential.Gid = uint32(gid)
e.cmd.SysProcAttr.Credential.Groups = gids
e.logger.Printf("[DEBUG] executor: running as user:group %d:%d with group membership in %v", uid, gid, gids)
return nil
// Signal sends a signal to the process managed by the executor
func (l *LibcontainerExecutor) Signal(s os.Signal) error {
return l.userProc.Signal(s)
}
// configureChroot configures a chroot
func (e *UniversalExecutor) configureChroot() error {
if e.cmd.SysProcAttr == nil {
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
// Exec starts an additional process inside the container
func (l *LibcontainerExecutor) Exec(deadline time.Time, cmd string, args []string) ([]byte, int, error) {
combined := append([]string{cmd}, args...)
// Capture output
buf, _ := circbuf.NewBuffer(int64(dstructs.CheckBufSize))
process := &libcontainer.Process{
Args: combined,
Env: l.command.Env,
Stdout: buf,
Stderr: buf,
}
e.cmd.SysProcAttr.Chroot = e.ctx.TaskDir
e.cmd.Dir = "/"
e.fsIsolationEnforced = true
return nil
}
err := l.container.Run(process)
if err != nil {
return nil, 0, err
}
// getAllPids returns the pids of all the processes spun up by the executor. We
// use the libcontainer apis to get the pids when the user is using cgroup
// isolation and we scan the entire process table if the user is not using any
// isolation
func (e *UniversalExecutor) getAllPids() (map[int]*nomadPid, error) {
if e.command.ResourceLimits || e.command.BasicProcessCgroup {
manager := getCgroupManager(e.resConCtx.groups, e.resConCtx.cgPaths)
pids, err := manager.GetAllPids()
if err != nil {
return nil, err
}
np := make(map[int]*nomadPid, len(pids))
for _, pid := range pids {
np[pid] = &nomadPid{
pid: pid,
cpuStatsTotal: stats.NewCpuStats(),
cpuStatsSys: stats.NewCpuStats(),
cpuStatsUser: stats.NewCpuStats(),
waitCh := make(chan *waitResult)
defer close(waitCh)
go l.handleExecWait(waitCh, process)
select {
case result := <-waitCh:
ps := result.ps
if result.err != nil {
if exitErr, ok := result.err.(*exec.ExitError); ok {
ps = exitErr.ProcessState
} else {
return nil, 0, result.err
}
}
return np, nil
var exitCode int
if status, ok := ps.Sys().(syscall.WaitStatus); ok {
exitCode = status.ExitStatus()
}
return buf.Bytes(), exitCode, nil
case <-time.After(time.Until(deadline)):
process.Signal(os.Kill)
return nil, 0, context.DeadlineExceeded
}
allProcesses, err := ps.Processes()
if err != nil {
return nil, err
}
return e.scanPids(os.Getpid(), allProcesses)
}
// destroyCgroup kills all processes in the cgroup and removes the cgroup
// configuration from the host. This function is idempotent.
func DestroyCgroup(groups *cgroupConfig.Cgroup, cgPaths map[string]string, executorPid int) error {
mErrs := new(multierror.Error)
if groups == nil {
return fmt.Errorf("Can't destroy: cgroup configuration empty")
type waitResult struct {
ps *os.ProcessState
err error
}
func (l *LibcontainerExecutor) handleExecWait(ch chan *waitResult, process *libcontainer.Process) {
ps, err := process.Wait()
ch <- &waitResult{ps, err}
}
func configureCapabilities(cfg *lconfigs.Config, command *ExecCommand) {
// TODO: allow better control of these
cfg.Capabilities = &lconfigs.Capabilities{
Bounding: allCaps,
Permitted: allCaps,
Inheritable: allCaps,
Ambient: allCaps,
Effective: allCaps,
}
// Move the executor into the global cgroup so that the task specific
// cgroup can be destroyed.
nilGroup := &cgroupConfig.Cgroup{}
nilGroup.Path = "/"
nilGroup.Resources = groups.Resources
nilManager := getCgroupManager(nilGroup, nil)
err := nilManager.Apply(executorPid)
if err != nil && !strings.Contains(err.Error(), "no such process") {
return fmt.Errorf("failed to remove executor pid %d: %v", executorPid, err)
}
func configureIsolation(cfg *lconfigs.Config, command *ExecCommand) {
defaultMountFlags := syscall.MS_NOEXEC | syscall.MS_NOSUID | syscall.MS_NODEV
// set the new root directory for the container
cfg.Rootfs = command.TaskDir
// launch with mount namespace
cfg.Namespaces = lconfigs.Namespaces{
{Type: lconfigs.NEWNS},
}
// Freeze the Cgroup so that it can not continue to fork/exec.
manager := getCgroupManager(groups, cgPaths)
err = manager.Freeze(cgroupConfig.Frozen)
if err != nil && !strings.Contains(err.Error(), "no such file or directory") {
return fmt.Errorf("failed to freeze cgroup: %v", err)
// paths to mask using a bind mount to /dev/null to prevent reading
cfg.MaskPaths = []string{
"/proc/kcore",
"/sys/firmware",
}
var procs []*os.Process
pids, err := manager.GetAllPids()
if err != nil {
multierror.Append(mErrs, fmt.Errorf("error getting pids: %v", err))
// paths that should be remounted as readonly inside the container
cfg.ReadonlyPaths = []string{
"/proc/sys", "/proc/sysrq-trigger", "/proc/irq", "/proc/bus",
}
// Unfreeze the cgroup.
err = manager.Freeze(cgroupConfig.Thawed)
if err != nil && !strings.Contains(err.Error(), "no such file or directory") {
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
cfg.Devices = lconfigs.DefaultAutoCreatedDevices
cfg.Mounts = []*lconfigs.Mount{
{
Source: "tmpfs",
Destination: "/dev",
Device: "tmpfs",
Flags: syscall.MS_NOSUID | syscall.MS_STRICTATIME,
Data: "mode=755",
},
{
Source: "devpts",
Destination: "/dev/pts",
Device: "devpts",
Flags: syscall.MS_NOSUID | syscall.MS_NOEXEC,
Data: "newinstance,ptmxmode=0666,mode=0620,gid=5",
},
{
Device: "tmpfs",
Source: "shm",
Destination: "/dev/shm",
Data: "mode=1777,size=65536k",
Flags: defaultMountFlags,
},
{
Source: "mqueue",
Destination: "/dev/mqueue",
Device: "mqueue",
Flags: defaultMountFlags,
},
{
Source: "sysfs",
Destination: "/sys",
Device: "sysfs",
Flags: defaultMountFlags | syscall.MS_RDONLY,
},
}
}
func configureCgroups(cfg *lconfigs.Config, command *ExecCommand) error {
// If resources are not limited then manually create cgroups needed
if !command.ResourceLimits {
return configureBasicCgroups(cfg)
}
id := uuid.Generate()
cfg.Cgroups.Path = filepath.Join(defaultCgroupParent, id)
if command.Resources.MemoryMB > 0 {
// Total amount of memory allowed to consume
cfg.Cgroups.Resources.Memory = int64(command.Resources.MemoryMB * 1024 * 1024)
// Disable swap to avoid issues on the machine
var memSwappiness uint64 = 0
cfg.Cgroups.Resources.MemorySwappiness = &memSwappiness
}
if command.Resources.CPU < 2 {
return fmt.Errorf("resources.CPU must be equal to or greater than 2: %v", command.Resources.CPU)
}
// Set the relative CPU shares for this cgroup.
cfg.Cgroups.Resources.CpuShares = uint64(command.Resources.CPU)
if command.Resources.IOPS != 0 {
// Validate it is in an acceptable range.
if command.Resources.IOPS < 10 || command.Resources.IOPS > 1000 {
return fmt.Errorf("resources.IOPS must be between 10 and 1000: %d", command.Resources.IOPS)
}
return mErrs.ErrorOrNil()
cfg.Cgroups.Resources.BlkioWeight = uint16(command.Resources.IOPS)
}
return nil
}
func configureBasicCgroups(cfg *lconfigs.Config) error {
id := uuid.Generate()
// Manually create freezer cgroup
cfg.Cgroups.Paths = map[string]string{}
root, err := cgroups.FindCgroupMountpointDir()
if err != nil {
return err
}
// Kill the processes in the cgroup
for _, pid := range pids {
proc, err := os.FindProcess(pid)
if _, err := os.Stat(root); err != nil {
return err
}
freezer := cgroupFs.FreezerGroup{}
subsystem := freezer.Name()
path, err := cgroups.FindCgroupMountpoint(subsystem)
if err != nil {
return fmt.Errorf("failed to find %s cgroup mountpoint: %v", subsystem, err)
}
// Sometimes subsystems can be mounted together as 'cpu,cpuacct'.
path = filepath.Join(root, filepath.Base(path), defaultCgroupParent, id)
if err = os.MkdirAll(path, 0755); err != nil {
return err
}
cfg.Cgroups.Paths[subsystem] = path
return nil
}
func newLibcontainerConfig(command *ExecCommand) *lconfigs.Config {
cfg := &lconfigs.Config{
Cgroups: &lconfigs.Cgroup{
Resources: &lconfigs.Resources{
AllowAllDevices: nil,
MemorySwappiness: nil,
AllowedDevices: lconfigs.DefaultAllowedDevices,
},
},
Version: "1.0.0",
}
configureCapabilities(cfg, command)
configureIsolation(cfg, command)
configureCgroups(cfg, command)
return cfg
}
// JoinRootCgroup moves the current process to the cgroups of the init process
func JoinRootCgroup(subsystems []string) error {
mErrs := new(multierror.Error)
paths := map[string]string{}
for _, s := range subsystems {
mnt, _, err := cgroups.FindCgroupMountpointAndRoot(s)
if err != nil {
multierror.Append(mErrs, fmt.Errorf("error finding process %v: %v", pid, err))
multierror.Append(mErrs, fmt.Errorf("error getting cgroup path for subsystem: %s", s))
continue
}
procs = append(procs, proc)
if e := proc.Kill(); e != nil {
multierror.Append(mErrs, fmt.Errorf("error killing process %v: %v", pid, e))
}
paths[s] = mnt
}
// Unfreeze the cgroug so we can wait.
err = manager.Freeze(cgroupConfig.Thawed)
if err != nil && !strings.Contains(err.Error(), "no such file or directory") {
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
err := cgroups.EnterPid(paths, os.Getpid())
if err != nil {
multierror.Append(mErrs, err)
}
// Wait on the killed processes to ensure they are cleaned up.
for _, proc := range procs {
// Don't capture the error because we expect this to fail for
// processes we didn't fork.
proc.Wait()
}
// Clear the cgroups paths so that everything is properly cleaned
manager.Cgroups.Paths = nil
// Remove the cgroup.
if err := manager.Destroy(); err != nil {
multierror.Append(mErrs, fmt.Errorf("failed to delete the cgroup directories: %v", err))
}
return mErrs.ErrorOrNil()
}
// getCgroupManager returns the correct libcontainer cgroup manager.
func getCgroupManager(groups *cgroupConfig.Cgroup, paths map[string]string) *cgroupFs.Manager {
return &cgroupFs.Manager{Cgroups: groups, Paths: paths}
}

View File

@ -1,6 +1,7 @@
package executor
import (
"fmt"
"io/ioutil"
"os"
"path/filepath"
@ -9,20 +10,30 @@ import (
"testing"
"time"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/driver/env"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/client/testutil"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/nomad/mock"
tu "github.com/hashicorp/nomad/testutil"
"github.com/stretchr/testify/require"
)
func init() {
executorFactories["LibcontainerExecutor"] = libcontainerFactory
}
func libcontainerFactory(l hclog.Logger) Executor {
return NewExecutorWithIsolation(l)
}
// testExecutorContextWithChroot returns an ExecutorContext and AllocDir with
// chroot. Use testExecutorContext if you don't need a chroot.
//
// The caller is responsible for calling AllocDir.Destroy() to cleanup.
func testExecutorContextWithChroot(t *testing.T) (*ExecutorContext, *allocdir.AllocDir) {
func testExecutorCommandWithChroot(t *testing.T) (*ExecCommand, *allocdir.AllocDir) {
chrootEnv := map[string]string{
"/etc/ld.so.cache": "/etc/ld.so.cache",
"/etc/ld.so.conf": "/etc/ld.so.conf",
@ -41,7 +52,7 @@ func testExecutorContextWithChroot(t *testing.T) (*ExecutorContext, *allocdir.Al
task := alloc.Job.TaskGroups[0].Tasks[0]
taskEnv := env.NewBuilder(mock.Node(), alloc, task, "global").Build()
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(os.TempDir(), alloc.ID))
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(os.TempDir(), alloc.ID))
if err := allocDir.Build(); err != nil {
t.Fatalf("AllocDir.Build() failed: %v", err)
}
@ -50,69 +61,69 @@ func testExecutorContextWithChroot(t *testing.T) (*ExecutorContext, *allocdir.Al
t.Fatalf("allocDir.NewTaskDir(%q) failed: %v", task.Name, err)
}
td := allocDir.TaskDirs[task.Name]
ctx := &ExecutorContext{
TaskEnv: taskEnv,
Task: task,
cmd := &ExecCommand{
Env: taskEnv.List(),
TaskDir: td.Dir,
LogDir: td.LogDir,
Resources: &Resources{
CPU: task.Resources.CPU,
MemoryMB: task.Resources.MemoryMB,
IOPS: task.Resources.IOPS,
DiskMB: task.Resources.DiskMB,
},
}
return ctx, allocDir
configureTLogging(cmd)
return cmd, allocDir
}
func TestExecutor_IsolationAndConstraints(t *testing.T) {
t.Parallel()
require := require.New(t)
testutil.ExecCompatible(t)
execCmd := ExecCommand{Cmd: "/bin/ls", Args: []string{"-F", "/", "/etc/"}}
ctx, allocDir := testExecutorContextWithChroot(t)
execCmd, allocDir := testExecutorCommandWithChroot(t)
execCmd.Cmd = "/bin/ls"
execCmd.Args = []string{"-F", "/", "/etc/"}
defer allocDir.Destroy()
execCmd.FSIsolation = true
execCmd.ResourceLimits = true
execCmd.User = dstructs.DefaultUnprivilegedUser
executor := NewExecutor(testlog.Logger(t))
executor := libcontainerFactory(testlog.HCLogger(t))
defer executor.Shutdown("SIGKILL", 0)
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error: %v", err)
}
ps, err := executor.Launch(execCmd)
require.NoError(err)
require.NotZero(ps.Pid)
ps, err := executor.LaunchCmd(&execCmd)
if err != nil {
t.Fatalf("error in launching command: %v", err)
}
if ps.Pid == 0 {
t.Fatalf("expected process to start and have non zero pid")
}
state, err := executor.Wait()
if err != nil {
t.Fatalf("error in waiting for command: %v", err)
}
if state.ExitCode != 0 {
t.Errorf("exited with non-zero code: %v", state.ExitCode)
}
require.NoError(err)
require.Zero(state.ExitCode)
// Check if the resource constraints were applied
memLimits := filepath.Join(ps.IsolationConfig.CgroupPaths["memory"], "memory.limit_in_bytes")
data, err := ioutil.ReadFile(memLimits)
if err != nil {
t.Fatalf("err: %v", err)
}
expectedMemLim := strconv.Itoa(ctx.Task.Resources.MemoryMB * 1024 * 1024)
actualMemLim := strings.TrimSpace(string(data))
if actualMemLim != expectedMemLim {
t.Fatalf("actual mem limit: %v, expected: %v", string(data), expectedMemLim)
}
if lexec, ok := executor.(*LibcontainerExecutor); ok {
state, err := lexec.container.State()
require.NoError(err)
if err := executor.Exit(); err != nil {
t.Fatalf("error: %v", err)
}
memLimits := filepath.Join(state.CgroupPaths["memory"], "memory.limit_in_bytes")
data, err := ioutil.ReadFile(memLimits)
require.NoError(err)
// Check if Nomad has actually removed the cgroups
if _, err := os.Stat(memLimits); err == nil {
t.Fatalf("file %v hasn't been removed", memLimits)
}
expectedMemLim := strconv.Itoa(execCmd.Resources.MemoryMB * 1024 * 1024)
actualMemLim := strings.TrimSpace(string(data))
require.Equal(actualMemLim, expectedMemLim)
require.NoError(executor.Shutdown("", 0))
executor.Wait()
// Check if Nomad has actually removed the cgroups
tu.WaitForResult(func() (bool, error) {
_, err = os.Stat(memLimits)
if err == nil {
return false, fmt.Errorf("expected an error from os.Stat %s", memLimits)
}
return true, nil
}, func(err error) { t.Error(err) })
}
expected := `/:
alloc/
bin/
@ -123,6 +134,7 @@ lib64/
local/
proc/
secrets/
sys/
tmp/
usr/
@ -130,66 +142,43 @@ usr/
ld.so.cache
ld.so.conf
ld.so.conf.d/`
file := filepath.Join(ctx.LogDir, "web.stdout.0")
output, err := ioutil.ReadFile(file)
if err != nil {
t.Fatalf("Couldn't read file %v", file)
}
act := strings.TrimSpace(string(output))
if act != expected {
t.Errorf("Command output incorrectly: want %v; got %v", expected, act)
}
tu.WaitForResult(func() (bool, error) {
output := execCmd.stdout.(*bufferCloser).String()
act := strings.TrimSpace(string(output))
if act != expected {
return false, fmt.Errorf("Command output incorrectly: want %v; got %v", expected, act)
}
return true, nil
}, func(err error) { t.Error(err) })
}
func TestExecutor_ClientCleanup(t *testing.T) {
t.Parallel()
testutil.ExecCompatible(t)
require := require.New(t)
ctx, allocDir := testExecutorContextWithChroot(t)
ctx.Task.LogConfig.MaxFiles = 1
ctx.Task.LogConfig.MaxFileSizeMB = 300
execCmd, allocDir := testExecutorCommandWithChroot(t)
defer allocDir.Destroy()
executor := NewExecutor(testlog.Logger(t))
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error")
}
executor := libcontainerFactory(testlog.HCLogger(t))
defer executor.Shutdown("", 0)
// Need to run a command which will produce continuous output but not
// too quickly to ensure executor.Exit() stops the process.
execCmd := ExecCommand{Cmd: "/bin/bash", Args: []string{"-c", "while true; do /bin/echo X; /bin/sleep 1; done"}}
execCmd.FSIsolation = true
execCmd.Cmd = "/bin/bash"
execCmd.Args = []string{"-c", "while true; do /bin/echo X; /bin/sleep 1; done"}
execCmd.ResourceLimits = true
execCmd.User = "nobody"
ps, err := executor.LaunchCmd(&execCmd)
if err != nil {
t.Fatalf("error in launching command: %v", err)
}
if ps.Pid == 0 {
t.Fatalf("expected process to start and have non zero pid")
}
ps, err := executor.Launch(execCmd)
require.NoError(err)
require.NotZero(ps.Pid)
time.Sleep(500 * time.Millisecond)
if err := executor.Exit(); err != nil {
t.Fatalf("err: %v", err)
}
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
executor.Wait()
file := filepath.Join(ctx.LogDir, "web.stdout.0")
finfo, err := os.Stat(file)
if err != nil {
t.Fatalf("error stating stdout file: %v", err)
}
if finfo.Size() == 0 {
t.Fatal("Nothing in stdout; expected at least one byte.")
}
output := execCmd.stdout.(*bufferCloser).String()
require.NotZero(len(output))
time.Sleep(2 * time.Second)
finfo1, err := os.Stat(file)
if err != nil {
t.Fatalf("error stating stdout file: %v", err)
}
if finfo.Size() != finfo1.Size() {
t.Fatalf("Expected size: %v, actual: %v", finfo.Size(), finfo1.Size())
}
output1 := execCmd.stdout.(*bufferCloser).String()
require.Equal(len(output), len(output1))
}

View File

@ -1,6 +1,8 @@
package executor
import (
"bytes"
"fmt"
"io/ioutil"
"os"
"path/filepath"
@ -9,24 +11,35 @@ import (
"testing"
"time"
tu "github.com/hashicorp/nomad/testutil"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/driver/env"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/nomad/mock"
tu "github.com/hashicorp/nomad/testutil"
"github.com/mitchellh/go-ps"
"github.com/stretchr/testify/require"
)
var executorFactories = map[string]func(hclog.Logger) Executor{}
var universalFactory = func(l hclog.Logger) Executor {
return NewExecutor(l)
}
func init() {
executorFactories["UniversalExecutor"] = universalFactory
}
// testExecutorContext returns an ExecutorContext and AllocDir.
//
// The caller is responsible for calling AllocDir.Destroy() to cleanup.
func testExecutorContext(t *testing.T) (*ExecutorContext, *allocdir.AllocDir) {
func testExecutorCommand(t *testing.T) (*ExecCommand, *allocdir.AllocDir) {
alloc := mock.Alloc()
task := alloc.Job.TaskGroups[0].Tasks[0]
taskEnv := env.NewBuilder(mock.Node(), alloc, task, "global").Build()
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(os.TempDir(), alloc.ID))
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(os.TempDir(), alloc.ID))
if err := allocDir.Build(); err != nil {
t.Fatalf("AllocDir.Build() failed: %v", err)
}
@ -35,184 +48,176 @@ func testExecutorContext(t *testing.T) (*ExecutorContext, *allocdir.AllocDir) {
t.Fatalf("allocDir.NewTaskDir(%q) failed: %v", task.Name, err)
}
td := allocDir.TaskDirs[task.Name]
ctx := &ExecutorContext{
TaskEnv: taskEnv,
Task: task,
cmd := &ExecCommand{
Env: taskEnv.List(),
TaskDir: td.Dir,
LogDir: td.LogDir,
Resources: &Resources{
CPU: task.Resources.CPU,
MemoryMB: task.Resources.MemoryMB,
IOPS: task.Resources.IOPS,
DiskMB: task.Resources.DiskMB,
},
}
return ctx, allocDir
configureTLogging(cmd)
return cmd, allocDir
}
func TestExecutor_Start_Invalid(t *testing.T) {
t.Parallel()
type bufferCloser struct {
bytes.Buffer
}
func (_ *bufferCloser) Close() error { return nil }
func configureTLogging(cmd *ExecCommand) (stdout bufferCloser, stderr bufferCloser) {
cmd.stdout = &stdout
cmd.stderr = &stderr
return
}
func TestExecutor_Start_Invalid(pt *testing.T) {
pt.Parallel()
invalid := "/bin/foobar"
execCmd := ExecCommand{Cmd: invalid, Args: []string{"1"}}
ctx, allocDir := testExecutorContext(t)
defer allocDir.Destroy()
executor := NewExecutor(testlog.Logger(t))
for name, factory := range executorFactories {
pt.Run(name, func(t *testing.T) {
t.Parallel()
require := require.New(t)
execCmd, allocDir := testExecutorCommand(t)
execCmd.Cmd = invalid
execCmd.Args = []string{"1"}
defer allocDir.Destroy()
executor := factory(testlog.HCLogger(t))
defer executor.Shutdown("", 0)
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error")
}
if _, err := executor.LaunchCmd(&execCmd); err == nil {
t.Fatalf("Expected error")
_, err := executor.Launch(execCmd)
require.Error(err)
})
}
}
func TestExecutor_Start_Wait_Failure_Code(t *testing.T) {
t.Parallel()
execCmd := ExecCommand{Cmd: "/bin/date", Args: []string{"fail"}}
ctx, allocDir := testExecutorContext(t)
defer allocDir.Destroy()
executor := NewExecutor(testlog.Logger(t))
func TestExecutor_Start_Wait_Failure_Code(pt *testing.T) {
pt.Parallel()
for name, factory := range executorFactories {
pt.Run(name, func(t *testing.T) {
t.Parallel()
require := require.New(t)
execCmd, allocDir := testExecutorCommand(t)
execCmd.Cmd = "/bin/date"
execCmd.Args = []string{"fail"}
defer allocDir.Destroy()
executor := factory(testlog.HCLogger(t))
defer executor.Shutdown("", 0)
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error")
}
ps, err := executor.LaunchCmd(&execCmd)
if err != nil {
t.Fatalf("Unexpected error")
}
if ps.Pid == 0 {
t.Fatalf("expected process to start and have non zero pid")
}
ps, _ = executor.Wait()
if ps.ExitCode < 1 {
t.Fatalf("expected exit code to be non zero, actual: %v", ps.ExitCode)
}
if err := executor.Exit(); err != nil {
t.Fatalf("error: %v", err)
ps, err := executor.Launch(execCmd)
require.NoError(err)
require.NotZero(ps.Pid)
ps, _ = executor.Wait()
require.NotZero(ps.ExitCode, "expected exit code to be non zero")
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
})
}
}
func TestExecutor_Start_Wait(t *testing.T) {
t.Parallel()
execCmd := ExecCommand{Cmd: "/bin/echo", Args: []string{"hello world"}}
ctx, allocDir := testExecutorContext(t)
defer allocDir.Destroy()
executor := NewExecutor(testlog.Logger(t))
func TestExecutor_Start_Wait(pt *testing.T) {
pt.Parallel()
for name, factory := range executorFactories {
pt.Run(name, func(t *testing.T) {
t.Parallel()
require := require.New(t)
execCmd, allocDir := testExecutorCommand(t)
execCmd.Cmd = "/bin/echo"
execCmd.Args = []string{"hello world"}
defer allocDir.Destroy()
executor := factory(testlog.HCLogger(t))
defer executor.Shutdown("", 0)
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error")
}
ps, err := executor.Launch(execCmd)
require.NoError(err)
require.NotZero(ps.Pid)
ps, err := executor.LaunchCmd(&execCmd)
if err != nil {
t.Fatalf("error in launching command: %v", err)
}
if ps.Pid == 0 {
t.Fatalf("expected process to start and have non zero pid")
}
ps, err = executor.Wait()
if err != nil {
t.Fatalf("error in waiting for command: %v", err)
}
if err := executor.Exit(); err != nil {
t.Fatalf("error: %v", err)
}
ps, err = executor.Wait()
require.NoError(err)
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
expected := "hello world"
file := filepath.Join(ctx.LogDir, "web.stdout.0")
output, err := ioutil.ReadFile(file)
if err != nil {
t.Fatalf("Couldn't read file %v", file)
}
act := strings.TrimSpace(string(output))
if act != expected {
t.Fatalf("Command output incorrectly: want %v; got %v", expected, act)
expected := "hello world"
tu.WaitForResult(func() (bool, error) {
output := execCmd.stdout.(*bufferCloser).String()
act := strings.TrimSpace(string(output))
if expected != act {
return false, fmt.Errorf("expected: '%s' actual: '%s'", expected, act)
}
return true, nil
}, func(err error) {
require.NoError(err)
})
})
}
}
func TestExecutor_WaitExitSignal(t *testing.T) {
t.Parallel()
execCmd := ExecCommand{Cmd: "/bin/sleep", Args: []string{"10000"}}
ctx, allocDir := testExecutorContext(t)
defer allocDir.Destroy()
executor := NewExecutor(testlog.Logger(t))
func TestExecutor_WaitExitSignal(pt *testing.T) {
pt.Parallel()
for name, factory := range executorFactories {
pt.Run(name, func(t *testing.T) {
t.Parallel()
require := require.New(t)
execCmd, allocDir := testExecutorCommand(t)
execCmd.Cmd = "/bin/sleep"
execCmd.Args = []string{"10000"}
defer allocDir.Destroy()
executor := factory(testlog.HCLogger(t))
defer executor.Shutdown("", 0)
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error")
}
ps, err := executor.Launch(execCmd)
require.NoError(err)
ps, err := executor.LaunchCmd(&execCmd)
if err != nil {
t.Fatalf("err: %v", err)
}
go func() {
time.Sleep(2 * time.Second)
_, err := executor.Stats()
require.NoError(err)
//require.NotEmpty(ru.Pids)
proc, err := os.FindProcess(ps.Pid)
require.NoError(err)
err = proc.Signal(syscall.SIGKILL)
require.NoError(err)
}()
go func() {
time.Sleep(2 * time.Second)
ru, err := executor.Stats()
if err != nil {
t.Fatalf("err: %v", err)
}
if len(ru.Pids) == 0 {
t.Fatalf("expected pids")
}
proc, err := os.FindProcess(ps.Pid)
if err != nil {
t.Fatalf("err: %v", err)
}
if err := proc.Signal(syscall.SIGKILL); err != nil {
t.Fatalf("err: %v", err)
}
}()
ps, err = executor.Wait()
if err != nil {
t.Fatalf("err: %v", err)
}
if ps.Signal != int(syscall.SIGKILL) {
t.Fatalf("expected signal: %v, actual: %v", int(syscall.SIGKILL), ps.Signal)
ps, err = executor.Wait()
require.NoError(err)
require.Equal(ps.Signal, int(syscall.SIGKILL))
})
}
}
func TestExecutor_Start_Kill(t *testing.T) {
t.Parallel()
execCmd := ExecCommand{Cmd: "/bin/sleep", Args: []string{"10 && hello world"}}
ctx, allocDir := testExecutorContext(t)
defer allocDir.Destroy()
executor := NewExecutor(testlog.Logger(t))
func TestExecutor_Start_Kill(pt *testing.T) {
pt.Parallel()
for name, factory := range executorFactories {
pt.Run(name, func(t *testing.T) {
t.Parallel()
require := require.New(t)
execCmd, allocDir := testExecutorCommand(t)
execCmd.Cmd = "/bin/sleep"
execCmd.Args = []string{"10 && hello world"}
defer allocDir.Destroy()
executor := factory(testlog.HCLogger(t))
defer executor.Shutdown("", 0)
if err := executor.SetContext(ctx); err != nil {
t.Fatalf("Unexpected error")
}
ps, err := executor.Launch(execCmd)
require.NoError(err)
require.NotZero(ps.Pid)
ps, err := executor.LaunchCmd(&execCmd)
if err != nil {
t.Fatalf("error in launching command: %v", err)
}
if ps.Pid == 0 {
t.Fatalf("expected process to start and have non zero pid")
}
ps, err = executor.Wait()
if err != nil {
t.Fatalf("error in waiting for command: %v", err)
}
if err := executor.Exit(); err != nil {
t.Fatalf("error: %v", err)
}
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
file := filepath.Join(ctx.LogDir, "web.stdout.0")
time.Sleep(time.Duration(tu.TestMultiplier()*2) * time.Second)
output, err := ioutil.ReadFile(file)
if err != nil {
t.Fatalf("Couldn't read file %v", file)
}
expected := ""
act := strings.TrimSpace(string(output))
if act != expected {
t.Fatalf("Command output incorrectly: want %v; got %v", expected, act)
time.Sleep(time.Duration(tu.TestMultiplier()*2) * time.Second)
output := execCmd.stdout.(*bufferCloser).String()
expected := ""
act := strings.TrimSpace(string(output))
if act != expected {
t.Fatalf("Command output incorrectly: want %v; got %v", expected, act)
}
})
}
}
func TestExecutor_MakeExecutable(t *testing.T) {
func TestUniversalExecutor_MakeExecutable(t *testing.T) {
t.Parallel()
// Create a temp file
f, err := ioutil.TempFile("", "")
@ -226,9 +231,9 @@ func TestExecutor_MakeExecutable(t *testing.T) {
f.Chmod(os.FileMode(0610))
// Make a fake executor
executor := NewExecutor(testlog.Logger(t))
executor := NewExecutor(testlog.HCLogger(t)).(*UniversalExecutor)
err = executor.(*UniversalExecutor).makeExecutable(f.Name())
err = executor.makeExecutable(f.Name())
if err != nil {
t.Fatalf("makeExecutable() failed: %v", err)
}
@ -245,44 +250,3 @@ func TestExecutor_MakeExecutable(t *testing.T) {
t.Fatalf("expected permissions %v; got %v", exp, act)
}
}
func TestScanPids(t *testing.T) {
t.Parallel()
p1 := NewFakeProcess(2, 5)
p2 := NewFakeProcess(10, 2)
p3 := NewFakeProcess(15, 6)
p4 := NewFakeProcess(3, 10)
p5 := NewFakeProcess(20, 18)
// Make a fake executor
executor := NewExecutor(testlog.Logger(t)).(*UniversalExecutor)
nomadPids, err := executor.scanPids(5, []ps.Process{p1, p2, p3, p4, p5})
if err != nil {
t.Fatalf("error: %v", err)
}
if len(nomadPids) != 4 {
t.Fatalf("expected: 4, actual: %v", len(nomadPids))
}
}
type FakeProcess struct {
pid int
ppid int
}
func (f FakeProcess) Pid() int {
return f.pid
}
func (f FakeProcess) PPid() int {
return f.ppid
}
func (f FakeProcess) Executable() string {
return "fake"
}
func NewFakeProcess(pid int, ppid int) ps.Process {
return FakeProcess{pid: pid, ppid: ppid}
}

View File

@ -0,0 +1,156 @@
package executor
import (
"fmt"
"os"
"os/user"
"strconv"
"syscall"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/helper"
"github.com/opencontainers/runc/libcontainer/cgroups"
cgroupFs "github.com/opencontainers/runc/libcontainer/cgroups/fs"
lconfigs "github.com/opencontainers/runc/libcontainer/configs"
)
// runAs takes a user id as a string and looks up the user, and sets the command
// to execute as that user.
func (e *UniversalExecutor) runAs(userid string) error {
u, err := user.Lookup(userid)
if err != nil {
return fmt.Errorf("Failed to identify user %v: %v", userid, err)
}
// Get the groups the user is a part of
gidStrings, err := u.GroupIds()
if err != nil {
return fmt.Errorf("Unable to lookup user's group membership: %v", err)
}
gids := make([]uint32, len(gidStrings))
for _, gidString := range gidStrings {
u, err := strconv.Atoi(gidString)
if err != nil {
return fmt.Errorf("Unable to convert user's group to int %s: %v", gidString, err)
}
gids = append(gids, uint32(u))
}
// Convert the uid and gid
uid, err := strconv.ParseUint(u.Uid, 10, 32)
if err != nil {
return fmt.Errorf("Unable to convert userid to uint32: %s", err)
}
gid, err := strconv.ParseUint(u.Gid, 10, 32)
if err != nil {
return fmt.Errorf("Unable to convert groupid to uint32: %s", err)
}
// Set the command to run as that user and group.
if e.childCmd.SysProcAttr == nil {
e.childCmd.SysProcAttr = &syscall.SysProcAttr{}
}
if e.childCmd.SysProcAttr.Credential == nil {
e.childCmd.SysProcAttr.Credential = &syscall.Credential{}
}
e.childCmd.SysProcAttr.Credential.Uid = uint32(uid)
e.childCmd.SysProcAttr.Credential.Gid = uint32(gid)
e.childCmd.SysProcAttr.Credential.Groups = gids
e.logger.Debug("setting process user", "user", uid, "group", gid, "additional_groups", gids)
return nil
}
// configureResourceContainer configured the cgroups to be used to track pids
// created by the executor
func (e *UniversalExecutor) configureResourceContainer(pid int) error {
cfg := &lconfigs.Config{
Cgroups: &lconfigs.Cgroup{
Resources: &lconfigs.Resources{
AllowAllDevices: helper.BoolToPtr(true),
},
},
}
configureBasicCgroups(cfg)
e.resConCtx.groups = cfg.Cgroups
return cgroups.EnterPid(cfg.Cgroups.Paths, pid)
}
// DestroyCgroup kills all processes in the cgroup and removes the cgroup
// configuration from the host. This function is idempotent.
func DestroyCgroup(groups *lconfigs.Cgroup, executorPid int) error {
mErrs := new(multierror.Error)
if groups == nil {
return fmt.Errorf("Can't destroy: cgroup configuration empty")
}
// Move the executor into the global cgroup so that the task specific
// cgroup can be destroyed.
path, err := cgroups.GetInitCgroupPath("freezer")
if err != nil {
return err
}
if err := cgroups.EnterPid(map[string]string{"freezer": path}, executorPid); err != nil {
return err
}
// Freeze the Cgroup so that it can not continue to fork/exec.
groups.Resources.Freezer = lconfigs.Frozen
freezer := cgroupFs.FreezerGroup{}
if err := freezer.Set(groups.Paths[freezer.Name()], groups); err != nil {
return err
}
var procs []*os.Process
pids, err := cgroups.GetAllPids(groups.Paths[freezer.Name()])
if err != nil {
multierror.Append(mErrs, fmt.Errorf("error getting pids: %v", err))
// Unfreeze the cgroup.
groups.Resources.Freezer = lconfigs.Thawed
freezer := cgroupFs.FreezerGroup{}
if err := freezer.Set(groups.Paths[freezer.Name()], groups); err != nil {
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
return mErrs.ErrorOrNil()
}
}
// Kill the processes in the cgroup
for _, pid := range pids {
proc, err := os.FindProcess(pid)
if err != nil {
multierror.Append(mErrs, fmt.Errorf("error finding process %v: %v", pid, err))
continue
}
procs = append(procs, proc)
if e := proc.Kill(); e != nil {
multierror.Append(mErrs, fmt.Errorf("error killing process %v: %v", pid, e))
}
}
// Unfreeze the cgroug so we can wait.
groups.Resources.Freezer = lconfigs.Thawed
if err := freezer.Set(groups.Paths[freezer.Name()], groups); err != nil {
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
return mErrs.ErrorOrNil()
}
// Wait on the killed processes to ensure they are cleaned up.
for _, proc := range procs {
// Don't capture the error because we expect this to fail for
// processes we didn't fork.
proc.Wait()
}
// Remove the cgroup.
if err := cgroups.RemovePaths(groups.Paths); err != nil {
multierror.Append(mErrs, fmt.Errorf("failed to delete the cgroup directories: %v", err))
}
return mErrs.ErrorOrNil()
}

View File

@ -10,10 +10,10 @@ import (
// configure new process group for child process
func (e *UniversalExecutor) setNewProcessGroup() error {
if e.cmd.SysProcAttr == nil {
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
if e.childCmd.SysProcAttr == nil {
e.childCmd.SysProcAttr = &syscall.SysProcAttr{}
}
e.cmd.SysProcAttr.Setpgid = true
e.childCmd.SysProcAttr.Setpgid = true
return nil
}
@ -21,7 +21,7 @@ func (e *UniversalExecutor) setNewProcessGroup() error {
func (e *UniversalExecutor) cleanupChildProcesses(proc *os.Process) error {
// If new process group was created upon command execution
// we can kill the whole process group now to cleanup any leftovers.
if e.cmd.SysProcAttr != nil && e.cmd.SysProcAttr.Setpgid {
if e.childCmd.SysProcAttr != nil && e.childCmd.SysProcAttr.Setpgid {
if err := syscall.Kill(-proc.Pid, syscall.SIGKILL); err != nil && err.Error() != noSuchProcessErr {
return err
}
@ -32,18 +32,13 @@ func (e *UniversalExecutor) cleanupChildProcesses(proc *os.Process) error {
// Only send the process a shutdown signal (default INT), doesn't
// necessarily kill it.
func (e *UniversalExecutor) shutdownProcess(proc *os.Process) error {
// Set default kill signal, as some drivers don't support configurable
// signals (such as rkt)
var osSignal os.Signal
if e.command.TaskKillSignal != nil {
osSignal = e.command.TaskKillSignal
} else {
osSignal = os.Interrupt
func (e *UniversalExecutor) shutdownProcess(sig os.Signal, proc *os.Process) error {
if sig == nil {
sig = os.Interrupt
}
if err := proc.Signal(osSignal); err != nil && err.Error() != finishedErr {
return fmt.Errorf("executor.shutdown error: %v", err)
if err := proc.Signal(sig); err != nil && err.Error() != finishedErr {
return fmt.Errorf("executor shutdown error: %v", err)
}
return nil

View File

@ -9,10 +9,10 @@ import (
// configure new process group for child process
func (e *UniversalExecutor) setNewProcessGroup() error {
// We need to check that as build flags includes windows for this file
if e.cmd.SysProcAttr == nil {
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
if e.childCmd.SysProcAttr == nil {
e.childCmd.SysProcAttr = &syscall.SysProcAttr{}
}
e.cmd.SysProcAttr.CreationFlags = syscall.CREATE_NEW_PROCESS_GROUP
e.childCmd.SysProcAttr.CreationFlags = syscall.CREATE_NEW_PROCESS_GROUP
return nil
}
@ -59,11 +59,11 @@ func sendCtrlBreak(pid int) error {
// Send the process a Ctrl-Break event, allowing it to shutdown by itself
// before being Terminate.
func (e *UniversalExecutor) shutdownProcess(proc *os.Process) error {
func (e *UniversalExecutor) shutdownProcess(_ os.Signal, proc *os.Process) error {
if err := sendCtrlBreak(proc.Pid); err != nil {
return fmt.Errorf("executor.shutdown error: %v", err)
return fmt.Errorf("executor shutdown error: %v", err)
}
e.logger.Printf("[INFO] executor: sent Ctrl-Break to process %v", proc.Pid)
e.logger.Info("sent Ctrl-Break to process", "pid", proc.Pid)
return nil
}

View File

@ -0,0 +1,219 @@
package executor
import (
"os"
"strconv"
"sync"
"time"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/stats"
cstructs "github.com/hashicorp/nomad/client/structs"
ps "github.com/mitchellh/go-ps"
"github.com/shirou/gopsutil/process"
)
var (
// pidScanInterval is the interval at which the executor scans the process
// tree for finding out the pids that the executor and it's child processes
// have forked
pidScanInterval = 5 * time.Second
)
// pidCollector is a utility that can be embedded in an executor to collect pid
// stats
type pidCollector struct {
pids map[int]*nomadPid
pidLock sync.RWMutex
logger hclog.Logger
}
// nomadPid holds a pid and it's cpu percentage calculator
type nomadPid struct {
pid int
cpuStatsTotal *stats.CpuStats
cpuStatsUser *stats.CpuStats
cpuStatsSys *stats.CpuStats
}
// allPidGetter is a func which is used by the pid collector to gather
// stats on
type allPidGetter func() (map[int]*nomadPid, error)
func newPidCollector(logger hclog.Logger) *pidCollector {
return &pidCollector{
pids: make(map[int]*nomadPid),
logger: logger.Named("pid_collector"),
}
}
// collectPids collects the pids of the child processes that the executor is
// running every 5 seconds
func (c *pidCollector) collectPids(stopCh chan interface{}, pidGetter allPidGetter) {
// Fire the timer right away when the executor starts from there on the pids
// are collected every scan interval
timer := time.NewTimer(0)
defer timer.Stop()
for {
select {
case <-timer.C:
pids, err := pidGetter()
if err != nil {
c.logger.Debug("error collecting pids", "error", err)
}
c.pidLock.Lock()
// Adding pids which are not being tracked
for pid, np := range pids {
if _, ok := c.pids[pid]; !ok {
c.pids[pid] = np
}
}
// Removing pids which are no longer present
for pid := range c.pids {
if _, ok := pids[pid]; !ok {
delete(c.pids, pid)
}
}
c.pidLock.Unlock()
timer.Reset(pidScanInterval)
case <-stopCh:
return
}
}
}
// scanPids scans all the pids on the machine running the current executor and
// returns the child processes of the executor.
func scanPids(parentPid int, allPids []ps.Process) (map[int]*nomadPid, error) {
processFamily := make(map[int]struct{})
processFamily[parentPid] = struct{}{}
// A mapping of pids to their parent pids. It is used to build the process
// tree of the executing task
pidsRemaining := make(map[int]int, len(allPids))
for _, pid := range allPids {
pidsRemaining[pid.Pid()] = pid.PPid()
}
for {
// flag to indicate if we have found a match
foundNewPid := false
for pid, ppid := range pidsRemaining {
_, childPid := processFamily[ppid]
// checking if the pid is a child of any of the parents
if childPid {
processFamily[pid] = struct{}{}
delete(pidsRemaining, pid)
foundNewPid = true
}
}
// not scanning anymore if we couldn't find a single match
if !foundNewPid {
break
}
}
res := make(map[int]*nomadPid)
for pid := range processFamily {
np := nomadPid{
pid: pid,
cpuStatsTotal: stats.NewCpuStats(),
cpuStatsUser: stats.NewCpuStats(),
cpuStatsSys: stats.NewCpuStats(),
}
res[pid] = &np
}
return res, nil
}
// pidStats returns the resource usage stats per pid
func (c *pidCollector) pidStats() (map[string]*cstructs.ResourceUsage, error) {
stats := make(map[string]*cstructs.ResourceUsage)
c.pidLock.RLock()
pids := make(map[int]*nomadPid, len(c.pids))
for k, v := range c.pids {
pids[k] = v
}
c.pidLock.RUnlock()
for pid, np := range pids {
p, err := process.NewProcess(int32(pid))
if err != nil {
c.logger.Trace("unable to create new process", "pid", pid, "error", err)
continue
}
ms := &cstructs.MemoryStats{}
if memInfo, err := p.MemoryInfo(); err == nil {
ms.RSS = memInfo.RSS
ms.Swap = memInfo.Swap
ms.Measured = ExecutorBasicMeasuredMemStats
}
cs := &cstructs.CpuStats{}
if cpuStats, err := p.Times(); err == nil {
cs.SystemMode = np.cpuStatsSys.Percent(cpuStats.System * float64(time.Second))
cs.UserMode = np.cpuStatsUser.Percent(cpuStats.User * float64(time.Second))
cs.Measured = ExecutorBasicMeasuredCpuStats
// calculate cpu usage percent
cs.Percent = np.cpuStatsTotal.Percent(cpuStats.Total() * float64(time.Second))
}
stats[strconv.Itoa(pid)] = &cstructs.ResourceUsage{MemoryStats: ms, CpuStats: cs}
}
return stats, nil
}
// aggregatedResourceUsage aggregates the resource usage of all the pids and
// returns a TaskResourceUsage data point
func aggregatedResourceUsage(systemCpuStats *stats.CpuStats, pidStats map[string]*cstructs.ResourceUsage) *cstructs.TaskResourceUsage {
ts := time.Now().UTC().UnixNano()
var (
systemModeCPU, userModeCPU, percent float64
totalRSS, totalSwap uint64
)
for _, pidStat := range pidStats {
systemModeCPU += pidStat.CpuStats.SystemMode
userModeCPU += pidStat.CpuStats.UserMode
percent += pidStat.CpuStats.Percent
totalRSS += pidStat.MemoryStats.RSS
totalSwap += pidStat.MemoryStats.Swap
}
totalCPU := &cstructs.CpuStats{
SystemMode: systemModeCPU,
UserMode: userModeCPU,
Percent: percent,
Measured: ExecutorBasicMeasuredCpuStats,
TotalTicks: systemCpuStats.TicksConsumed(percent),
}
totalMemory := &cstructs.MemoryStats{
RSS: totalRSS,
Swap: totalSwap,
Measured: ExecutorBasicMeasuredMemStats,
}
resourceUsage := cstructs.ResourceUsage{
MemoryStats: totalMemory,
CpuStats: totalCPU,
}
return &cstructs.TaskResourceUsage{
ResourceUsage: &resourceUsage,
Timestamp: ts,
Pids: pidStats,
}
}
func getAllPids() (map[int]*nomadPid, error) {
allProcesses, err := ps.Processes()
if err != nil {
return nil, err
}
return scanPids(os.Getpid(), allProcesses)
}

View File

@ -0,0 +1,45 @@
package executor
import (
"testing"
ps "github.com/mitchellh/go-ps"
)
func TestScanPids(t *testing.T) {
t.Parallel()
p1 := NewFakeProcess(2, 5)
p2 := NewFakeProcess(10, 2)
p3 := NewFakeProcess(15, 6)
p4 := NewFakeProcess(3, 10)
p5 := NewFakeProcess(20, 18)
nomadPids, err := scanPids(5, []ps.Process{p1, p2, p3, p4, p5})
if err != nil {
t.Fatalf("error: %v", err)
}
if len(nomadPids) != 4 {
t.Fatalf("expected: 4, actual: %v", len(nomadPids))
}
}
type FakeProcess struct {
pid int
ppid int
}
func (f FakeProcess) Pid() int {
return f.pid
}
func (f FakeProcess) PPid() int {
return f.ppid
}
func (f FakeProcess) Executable() string {
return "fake"
}
func NewFakeProcess(pid int, ppid int) ps.Process {
return FakeProcess{pid: pid, ppid: ppid}
}

View File

@ -2,23 +2,11 @@
package executor
import (
dstructs "github.com/hashicorp/nomad/client/driver/structs"
)
// resourceContainerContext is a platform-specific struct for managing a
// resource container.
type resourceContainerContext struct {
}
func clientCleanup(ic *dstructs.IsolationConfig, pid int) error {
return nil
}
func (rc *resourceContainerContext) executorCleanup() error {
return nil
}
func (rc *resourceContainerContext) getIsolationConfig() *dstructs.IsolationConfig {
return nil
}

View File

@ -4,39 +4,22 @@ import (
"os"
"sync"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
cgroupConfig "github.com/opencontainers/runc/libcontainer/configs"
)
// resourceContainerContext is a platform-specific struct for managing a
// resource container. In the case of Linux, this is used to control Cgroups.
type resourceContainerContext struct {
groups *cgroupConfig.Cgroup
cgPaths map[string]string
cgLock sync.Mutex
}
// clientCleanup removes this host's Cgroup from the Nomad Client's context
func clientCleanup(ic *dstructs.IsolationConfig, pid int) error {
if err := DestroyCgroup(ic.Cgroup, ic.CgroupPaths, pid); err != nil {
return err
}
return nil
groups *cgroupConfig.Cgroup
cgLock sync.Mutex
}
// cleanup removes this host's Cgroup from within an Executor's context
func (rc *resourceContainerContext) executorCleanup() error {
rc.cgLock.Lock()
defer rc.cgLock.Unlock()
if err := DestroyCgroup(rc.groups, rc.cgPaths, os.Getpid()); err != nil {
if err := DestroyCgroup(rc.groups, os.Getpid()); err != nil {
return err
}
return nil
}
func (rc *resourceContainerContext) getIsolationConfig() *dstructs.IsolationConfig {
return &dstructs.IsolationConfig{
Cgroup: rc.groups,
CgroupPaths: rc.cgPaths,
}
}

View File

@ -2,16 +2,15 @@ package driver
import (
"encoding/gob"
"log"
"net/rpc"
"os"
"syscall"
"time"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/client/driver/executor"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
// Registering these types since we have to serialize and de-serialize the Task
@ -26,65 +25,53 @@ func init() {
type ExecutorRPC struct {
client *rpc.Client
logger *log.Logger
logger hclog.Logger
}
// LaunchCmdArgs wraps a user command and the args for the purposes of RPC
type LaunchCmdArgs struct {
type LaunchArgs struct {
Cmd *executor.ExecCommand
}
type ExecCmdArgs struct {
// ShutdownArgs wraps shutdown signal and grace period
type ShutdownArgs struct {
Signal string
GracePeriod time.Duration
}
type ExecArgs struct {
Deadline time.Time
Name string
Args []string
}
type ExecCmdReturn struct {
type ExecReturn struct {
Output []byte
Code int
}
func (e *ExecutorRPC) LaunchCmd(cmd *executor.ExecCommand) (*executor.ProcessState, error) {
func (e *ExecutorRPC) Launch(cmd *executor.ExecCommand) (*executor.ProcessState, error) {
var ps *executor.ProcessState
err := e.client.Call("Plugin.LaunchCmd", LaunchCmdArgs{Cmd: cmd}, &ps)
err := e.client.Call("Plugin.Launch", LaunchArgs{Cmd: cmd}, &ps)
return ps, err
}
func (e *ExecutorRPC) LaunchSyslogServer() (*executor.SyslogServerState, error) {
var ss *executor.SyslogServerState
err := e.client.Call("Plugin.LaunchSyslogServer", new(interface{}), &ss)
return ss, err
}
func (e *ExecutorRPC) Wait() (*executor.ProcessState, error) {
var ps executor.ProcessState
err := e.client.Call("Plugin.Wait", new(interface{}), &ps)
return &ps, err
}
func (e *ExecutorRPC) ShutDown() error {
return e.client.Call("Plugin.ShutDown", new(interface{}), new(interface{}))
func (e *ExecutorRPC) Kill() error {
return e.client.Call("Plugin.Kill", new(interface{}), new(interface{}))
}
func (e *ExecutorRPC) Exit() error {
return e.client.Call("Plugin.Exit", new(interface{}), new(interface{}))
func (e *ExecutorRPC) Shutdown(signal string, grace time.Duration) error {
return e.client.Call("Plugin.Shutdown", &ShutdownArgs{signal, grace}, new(interface{}))
}
func (e *ExecutorRPC) SetContext(ctx *executor.ExecutorContext) error {
return e.client.Call("Plugin.SetContext", ctx, new(interface{}))
}
func (e *ExecutorRPC) UpdateLogConfig(logConfig *structs.LogConfig) error {
return e.client.Call("Plugin.UpdateLogConfig", logConfig, new(interface{}))
}
func (e *ExecutorRPC) UpdateTask(task *structs.Task) error {
return e.client.Call("Plugin.UpdateTask", task, new(interface{}))
}
func (e *ExecutorRPC) DeregisterServices() error {
return e.client.Call("Plugin.DeregisterServices", new(interface{}), new(interface{}))
func (e *ExecutorRPC) UpdateResources(resources *executor.Resources) error {
return e.client.Call("Plugin.UpdateResources", resources, new(interface{}))
}
func (e *ExecutorRPC) Version() (*executor.ExecutorVersion, error) {
@ -104,12 +91,12 @@ func (e *ExecutorRPC) Signal(s os.Signal) error {
}
func (e *ExecutorRPC) Exec(deadline time.Time, name string, args []string) ([]byte, int, error) {
req := ExecCmdArgs{
req := ExecArgs{
Deadline: deadline,
Name: name,
Args: args,
}
var resp *ExecCmdReturn
var resp *ExecReturn
err := e.client.Call("Plugin.Exec", req, &resp)
if resp == nil {
return nil, 0, err
@ -119,25 +106,17 @@ func (e *ExecutorRPC) Exec(deadline time.Time, name string, args []string) ([]by
type ExecutorRPCServer struct {
Impl executor.Executor
logger *log.Logger
logger hclog.Logger
}
func (e *ExecutorRPCServer) LaunchCmd(args LaunchCmdArgs, ps *executor.ProcessState) error {
state, err := e.Impl.LaunchCmd(args.Cmd)
func (e *ExecutorRPCServer) Launch(args LaunchArgs, ps *executor.ProcessState) error {
state, err := e.Impl.Launch(args.Cmd)
if state != nil {
*ps = *state
}
return err
}
func (e *ExecutorRPCServer) LaunchSyslogServer(args interface{}, ss *executor.SyslogServerState) error {
state, err := e.Impl.LaunchSyslogServer()
if state != nil {
*ss = *state
}
return err
}
func (e *ExecutorRPCServer) Wait(args interface{}, ps *executor.ProcessState) error {
state, err := e.Impl.Wait()
if state != nil {
@ -146,29 +125,12 @@ func (e *ExecutorRPCServer) Wait(args interface{}, ps *executor.ProcessState) er
return err
}
func (e *ExecutorRPCServer) ShutDown(args interface{}, resp *interface{}) error {
return e.Impl.ShutDown()
func (e *ExecutorRPCServer) Shutdown(args ShutdownArgs, resp *interface{}) error {
return e.Impl.Shutdown(args.Signal, args.GracePeriod)
}
func (e *ExecutorRPCServer) Exit(args interface{}, resp *interface{}) error {
return e.Impl.Exit()
}
func (e *ExecutorRPCServer) SetContext(args *executor.ExecutorContext, resp *interface{}) error {
return e.Impl.SetContext(args)
}
func (e *ExecutorRPCServer) UpdateLogConfig(args *structs.LogConfig, resp *interface{}) error {
return e.Impl.UpdateLogConfig(args)
}
func (e *ExecutorRPCServer) UpdateTask(args *structs.Task, resp *interface{}) error {
return e.Impl.UpdateTask(args)
}
func (e *ExecutorRPCServer) DeregisterServices(args interface{}, resp *interface{}) error {
// In 0.6 this is a noop. Goes away in 0.7.
return nil
func (e *ExecutorRPCServer) UpdateResources(args *executor.Resources, resp *interface{}) error {
return e.Impl.UpdateResources(args)
}
func (e *ExecutorRPCServer) Version(args interface{}, version *executor.ExecutorVersion) error {
@ -191,9 +153,9 @@ func (e *ExecutorRPCServer) Signal(args os.Signal, resp *interface{}) error {
return e.Impl.Signal(args)
}
func (e *ExecutorRPCServer) Exec(args ExecCmdArgs, result *ExecCmdReturn) error {
func (e *ExecutorRPCServer) Exec(args ExecArgs, result *ExecReturn) error {
out, code, err := e.Impl.Exec(args.Deadline, args.Name, args.Args)
ret := &ExecCmdReturn{
ret := &ExecReturn{
Output: out,
Code: code,
}
@ -202,13 +164,18 @@ func (e *ExecutorRPCServer) Exec(args ExecCmdArgs, result *ExecCmdReturn) error
}
type ExecutorPlugin struct {
logger *log.Logger
Impl *ExecutorRPCServer
logger hclog.Logger
fsIsolation bool
Impl *ExecutorRPCServer
}
func (p *ExecutorPlugin) Server(*plugin.MuxBroker) (interface{}, error) {
if p.Impl == nil {
p.Impl = &ExecutorRPCServer{Impl: executor.NewExecutor(p.logger), logger: p.logger}
if p.fsIsolation {
p.Impl = &ExecutorRPCServer{Impl: executor.NewExecutorWithIsolation(p.logger), logger: p.logger}
} else {
p.Impl = &ExecutorRPCServer{Impl: executor.NewExecutor(p.logger), logger: p.logger}
}
}
return p.Impl, nil
}

View File

@ -0,0 +1,220 @@
package driver
/*
import (
"encoding/gob"
"log"
"net/rpc"
"os"
"syscall"
"time"
"github.com/hashicorp/go-plugin"
executorv0 "github.com/hashicorp/nomad/client/driver/executorv0"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
// Registering these types since we have to serialize and de-serialize the Task
// structs over the wire between drivers and the executorv0.
func init() {
gob.Register([]interface{}{})
gob.Register(map[string]interface{}{})
gob.Register([]map[string]string{})
gob.Register([]map[string]int{})
gob.Register(syscall.Signal(0x1))
}
type ExecutorRPCPre0_9_0 struct {
client *rpc.Client
logger *log.Logger
}
// LaunchCmdArgs wraps a user command and the args for the purposes of RPC
type LaunchCmdArgs struct {
Cmd *executorv0.ExecCommand
}
type ExecCmdArgs struct {
Deadline time.Time
Name string
Args []string
}
type ExecCmdReturn struct {
Output []byte
Code int
}
func (e *ExecutorRPCPre0_9_0) LaunchCmd(cmd *executorv0.ExecCommand) (*executorv0.ProcessState, error) {
var ps *executorv0.ProcessState
err := e.client.Call("Plugin.LaunchCmd", LaunchCmdArgs{Cmd: cmd}, &ps)
return ps, err
}
func (e *ExecutorRPCPre0_9_0) LaunchSyslogServer() (*executorv0.SyslogServerState, error) {
var ss *executorv0.SyslogServerState
err := e.client.Call("Plugin.LaunchSyslogServer", new(interface{}), &ss)
return ss, err
}
func (e *ExecutorRPCPre0_9_0) Wait() (*executorv0.ProcessState, error) {
var ps executorv0.ProcessState
err := e.client.Call("Plugin.Wait", new(interface{}), &ps)
return &ps, err
}
func (e *ExecutorRPCPre0_9_0) ShutDown() error {
return e.client.Call("Plugin.ShutDown", new(interface{}), new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) Exit() error {
return e.client.Call("Plugin.Exit", new(interface{}), new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) SetContext(ctx *executorv0.ExecutorContext) error {
return e.client.Call("Plugin.SetContext", ctx, new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) UpdateLogConfig(logConfig *executorv0.LogConfig) error {
return e.client.Call("Plugin.UpdateLogConfig", logConfig, new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) UpdateTask(task *structs.Task) error {
return e.client.Call("Plugin.UpdateTask", task, new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) DeregisterServices() error {
return e.client.Call("Plugin.DeregisterServices", new(interface{}), new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) Version() (*executorv0.ExecutorVersion, error) {
var version executorv0.ExecutorVersion
err := e.client.Call("Plugin.Version", new(interface{}), &version)
return &version, err
}
func (e *ExecutorRPCPre0_9_0) Stats() (*cstructs.TaskResourceUsage, error) {
var resourceUsage cstructs.TaskResourceUsage
err := e.client.Call("Plugin.Stats", new(interface{}), &resourceUsage)
return &resourceUsage, err
}
func (e *ExecutorRPCPre0_9_0) Signal(s os.Signal) error {
return e.client.Call("Plugin.Signal", &s, new(interface{}))
}
func (e *ExecutorRPCPre0_9_0) Exec(deadline time.Time, name string, args []string) ([]byte, int, error) {
req := ExecCmdArgs{
Deadline: deadline,
Name: name,
Args: args,
}
var resp *ExecCmdReturn
err := e.client.Call("Plugin.Exec", req, &resp)
if resp == nil {
return nil, 0, err
}
return resp.Output, resp.Code, err
}
type ExecutorRPCServerPre0_9_0 struct {
Impl executorv0.Executor
logger *log.Logger
}
func (e *ExecutorRPCServerPre0_9_0) LaunchCmd(args LaunchCmdArgs, ps *executorv0.ProcessState) error {
state, err := e.Impl.LaunchCmd(args.Cmd)
if state != nil {
*ps = *state
}
return err
}
func (e *ExecutorRPCServerPre0_9_0) LaunchSyslogServer(args interface{}, ss *executorv0.SyslogServerState) error {
state, err := e.Impl.LaunchSyslogServer()
if state != nil {
*ss = *state
}
return err
}
func (e *ExecutorRPCServerPre0_9_0) Wait(args interface{}, ps *executorv0.ProcessState) error {
state, err := e.Impl.Wait()
if state != nil {
*ps = *state
}
return err
}
func (e *ExecutorRPCServerPre0_9_0) ShutDown(args interface{}, resp *interface{}) error {
return e.Impl.ShutDown()
}
func (e *ExecutorRPCServerPre0_9_0) Exit(args interface{}, resp *interface{}) error {
return e.Impl.Exit()
}
func (e *ExecutorRPCServerPre0_9_0) SetContext(args *executorv0.ExecutorContext, resp *interface{}) error {
return e.Impl.SetContext(args)
}
func (e *ExecutorRPCServerPre0_9_0) UpdateLogConfig(args *executorv0.LogConfig, resp *interface{}) error {
return e.Impl.UpdateLogConfig(args)
}
func (e *ExecutorRPCServerPre0_9_0) UpdateTask(args *structs.Task, resp *interface{}) error {
return e.Impl.UpdateTask(args)
}
func (e *ExecutorRPCServerPre0_9_0) DeregisterServices(args interface{}, resp *interface{}) error {
// In 0.6 this is a noop. Goes away in 0.7.
return nil
}
func (e *ExecutorRPCServerPre0_9_0) Version(args interface{}, version *executorv0.ExecutorVersion) error {
ver, err := e.Impl.Version()
if ver != nil {
*version = *ver
}
return err
}
func (e *ExecutorRPCServerPre0_9_0) Stats(args interface{}, resourceUsage *cstructs.TaskResourceUsage) error {
ru, err := e.Impl.Stats()
if ru != nil {
*resourceUsage = *ru
}
return err
}
func (e *ExecutorRPCServerPre0_9_0) Signal(args os.Signal, resp *interface{}) error {
return e.Impl.Signal(args)
}
func (e *ExecutorRPCServerPre0_9_0) Exec(args ExecCmdArgs, result *ExecCmdReturn) error {
out, code, err := e.Impl.Exec(args.Deadline, args.Name, args.Args)
ret := &ExecCmdReturn{
Output: out,
Code: code,
}
*result = *ret
return err
}
type ExecutorPluginPre0_9_0 struct {
logger *log.Logger
Impl *ExecutorRPCServerPre0_9_0
}
func (p *ExecutorPluginPre0_9_0) Server(*plugin.MuxBroker) (interface{}, error) {
if p.Impl == nil {
p.Impl = &ExecutorRPCServerPre0_9_0{Impl: executorv0.NewExecutor(p.logger), logger: p.logger}
}
return p.Impl, nil
}
func (p *ExecutorPluginPre0_9_0) Client(b *plugin.MuxBroker, c *rpc.Client) (interface{}, error) {
return &ExecutorRPCPre0_9_0{client: c, logger: p.logger}, nil
}
*/

Some files were not shown because too many files have changed in this diff Show More