open-nomad/client/allocrunner/taskrunner/task_runner_hooks.go

553 lines
15 KiB
Go
Raw Normal View History

2018-06-22 00:35:07 +00:00
package taskrunner
import (
"context"
2018-06-22 00:35:07 +00:00
"fmt"
CSI Plugin Registration (#6555) This changeset implements the initial registration and fingerprinting of CSI Plugins as part of #5378. At a high level, it introduces the following: * A `csi_plugin` stanza as part of a Nomad task configuration, to allow a task to expose that it is a plugin. * A new task runner hook: `csi_plugin_supervisor`. This hook does two things. When the `csi_plugin` stanza is detected, it will automatically configure the plugin task to receive bidirectional mounts to the CSI intermediary directory. At runtime, it will then perform an initial heartbeat of the plugin and handle submitting it to the new `dynamicplugins.Registry` for further use by the client, and then run a lightweight heartbeat loop that will emit task events when health changes. * The `dynamicplugins.Registry` for handling plugins that run as Nomad tasks, in contrast to the existing catalog that requires `go-plugin` type plugins and to know the plugin configuration in advance. * The `csimanager` which fingerprints CSI plugins, in a similar way to `drivermanager` and `devicemanager`. It currently only fingerprints the NodeID from the plugin, and assumes that all plugins are monolithic. Missing features * We do not use the live updates of the `dynamicplugin` registry in the `csimanager` yet. * We do not deregister the plugins from the client when they shutdown yet, they just become indefinitely marked as unhealthy. This is deliberate until we figure out how we should manage deploying new versions of plugins/transitioning them.
2019-10-22 13:20:26 +00:00
"path/filepath"
"sync"
2018-06-22 00:35:07 +00:00
"time"
"github.com/LK4D4/joincontext"
multierror "github.com/hashicorp/go-multierror"
2018-10-04 23:22:01 +00:00
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/state"
2018-06-22 00:35:07 +00:00
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
2018-06-22 00:35:07 +00:00
)
// hookResources captures the resources for the task provided by hooks.
type hookResources struct {
Devices []*drivers.DeviceConfig
Mounts []*drivers.MountConfig
sync.RWMutex
}
func (h *hookResources) setDevices(d []*drivers.DeviceConfig) {
h.Lock()
h.Devices = d
h.Unlock()
}
func (h *hookResources) getDevices() []*drivers.DeviceConfig {
h.RLock()
defer h.RUnlock()
return h.Devices
}
func (h *hookResources) setMounts(m []*drivers.MountConfig) {
h.Lock()
h.Mounts = m
h.Unlock()
}
func (h *hookResources) getMounts() []*drivers.MountConfig {
h.RLock()
defer h.RUnlock()
return h.Mounts
}
// initHooks initializes the tasks hooks.
2018-06-22 00:35:07 +00:00
func (tr *TaskRunner) initHooks() {
hookLogger := tr.logger.Named("task_hook")
task := tr.Task()
2018-06-22 00:35:07 +00:00
tr.logmonHookConfig = newLogMonHookConfig(task.Name, tr.taskDir.LogDir)
// Add the hook resources
tr.hookResources = &hookResources{}
2018-06-22 00:35:07 +00:00
// Create the task directory hook. This is run first to ensure the
2018-10-05 02:36:40 +00:00
// directory path exists for other hooks.
alloc := tr.Alloc()
2018-06-27 23:57:31 +00:00
tr.runnerHooks = []interfaces.TaskHook{
2018-07-18 23:51:50 +00:00
newValidateHook(tr.clientConfig, hookLogger),
2018-06-27 23:57:31 +00:00
newTaskDirHook(tr, hookLogger),
newLogMonHook(tr, hookLogger),
newDispatchHook(alloc, hookLogger),
newVolumeHook(tr, hookLogger),
2018-06-27 23:57:31 +00:00
newArtifactHook(tr, hookLogger),
newStatsHook(tr, tr.clientConfig.StatsCollectionInterval, hookLogger),
newDeviceHook(tr.devicemanager, hookLogger),
2018-06-27 23:57:31 +00:00
}
2018-07-12 23:15:33 +00:00
CSI Plugin Registration (#6555) This changeset implements the initial registration and fingerprinting of CSI Plugins as part of #5378. At a high level, it introduces the following: * A `csi_plugin` stanza as part of a Nomad task configuration, to allow a task to expose that it is a plugin. * A new task runner hook: `csi_plugin_supervisor`. This hook does two things. When the `csi_plugin` stanza is detected, it will automatically configure the plugin task to receive bidirectional mounts to the CSI intermediary directory. At runtime, it will then perform an initial heartbeat of the plugin and handle submitting it to the new `dynamicplugins.Registry` for further use by the client, and then run a lightweight heartbeat loop that will emit task events when health changes. * The `dynamicplugins.Registry` for handling plugins that run as Nomad tasks, in contrast to the existing catalog that requires `go-plugin` type plugins and to know the plugin configuration in advance. * The `csimanager` which fingerprints CSI plugins, in a similar way to `drivermanager` and `devicemanager`. It currently only fingerprints the NodeID from the plugin, and assumes that all plugins are monolithic. Missing features * We do not use the live updates of the `dynamicplugin` registry in the `csimanager` yet. * We do not deregister the plugins from the client when they shutdown yet, they just become indefinitely marked as unhealthy. This is deliberate until we figure out how we should manage deploying new versions of plugins/transitioning them.
2019-10-22 13:20:26 +00:00
// If the task has a CSI stanza, add the hook.
if task.CSIPluginConfig != nil {
tr.runnerHooks = append(tr.runnerHooks, newCSIPluginSupervisorHook(filepath.Join(tr.clientConfig.StateDir, "csi"), tr, tr, hookLogger))
}
2018-07-12 23:15:33 +00:00
// If Vault is enabled, add the hook
if task.Vault != nil {
2018-07-12 23:15:33 +00:00
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,
}))
}
2018-07-13 20:45:57 +00:00
// If there are templates is enabled, add the hook
if len(task.Templates) != 0 {
2018-07-13 20:45:57 +00:00
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 service 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,
}))
}
// If this is a Connect sidecar proxy (or a Connect Native) service,
// add the sidsHook for requesting a Service Identity token (if ACLs).
if task.UsesConnect() {
// Enable the Service Identity hook only if the Nomad client is configured
// with a consul token, indicating that Consul ACLs are enabled
if tr.clientConfig.ConsulConfig.Token != "" {
tr.runnerHooks = append(tr.runnerHooks, newSIDSHook(sidsHookConfig{
alloc: tr.Alloc(),
task: tr.Task(),
sidsClient: tr.siClient,
lifecycle: tr,
logger: hookLogger,
}))
}
// envoy bootstrap must execute after sidsHook maybe sets SI token
tr.runnerHooks = append(tr.runnerHooks, newEnvoyBootstrapHook(&envoyBootstrapHookConfig{
alloc: alloc,
consulHTTPAddr: tr.clientConfig.ConsulConfig.Addr,
logger: hookLogger,
}))
}
// If there are any script checks, add the hook
scriptCheckHook := newScriptCheckHook(scriptCheckHookConfig{
alloc: tr.Alloc(),
task: tr.Task(),
consul: tr.consulClient,
logger: hookLogger,
})
tr.runnerHooks = append(tr.runnerHooks, scriptCheckHook)
2018-06-22 00:35:07 +00:00
}
func (tr *TaskRunner) emitHookError(err error, hookName string) {
var taskEvent *structs.TaskEvent
if herr, ok := err.(*hookError); ok {
taskEvent = herr.taskEvent
} else {
message := fmt.Sprintf("%s: %v", hookName, err)
taskEvent = structs.NewTaskEvent(structs.TaskHookFailed).SetMessage(message)
}
tr.EmitEvent(taskEvent)
}
2018-07-17 00:19:56 +00:00
// prestart is used to run the runners prestart hooks.
func (tr *TaskRunner) prestart() error {
// Determine if the allocation is terminal and we should avoid running
2018-07-17 00:19:56 +00:00
// prestart hooks.
alloc := tr.Alloc()
2018-06-22 00:35:07 +00:00
if alloc.TerminalStatus() {
2018-07-17 00:19:56 +00:00
tr.logger.Trace("skipping prestart hooks since allocation is terminal")
2018-06-22 00:35:07 +00:00
return nil
}
if tr.logger.IsTrace() {
start := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("running prestart hooks", "start", start)
2018-06-22 00:35:07 +00:00
defer func() {
end := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("finished prestart hooks", "end", end, "duration", end.Sub(start))
2018-06-22 00:35:07 +00:00
}()
}
for _, hook := range tr.runnerHooks {
2018-07-17 00:19:56 +00:00
pre, ok := hook.(interfaces.TaskPrestartHook)
2018-06-22 00:35:07 +00:00
if !ok {
continue
}
name := pre.Name()
2018-06-22 00:35:07 +00:00
// Build the request
2018-07-17 00:19:56 +00:00
req := interfaces.TaskPrestartRequest{
Task: tr.Task(),
TaskDir: tr.taskDir,
TaskEnv: tr.envBuilder.Build(),
TaskResources: tr.taskResources,
2018-06-22 00:35:07 +00:00
}
origHookState := tr.hookState(name)
if origHookState != nil {
if origHookState.PrestartDone {
tr.logger.Trace("skipping done prestart hook", "name", pre.Name())
// Always set env vars from hooks
if name == HookNameDevices {
tr.envBuilder.SetDeviceHookEnv(name, origHookState.Env)
} else {
tr.envBuilder.SetHookEnv(name, origHookState.Env)
}
continue
}
// Give the hook it's old data
req.PreviousState = origHookState.Data
2018-06-22 00:35:07 +00:00
}
2018-07-12 23:15:33 +00:00
req.VaultToken = tr.getVaultToken()
2018-06-22 00:35:07 +00:00
2018-07-17 00:19:56 +00:00
// Time the prestart hook
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("running prestart hook", "name", name, "start", start)
}
2018-07-17 00:19:56 +00:00
// Run the prestart hook
// use a joint context to allow any blocking pre-start hooks
// to be canceled by either killCtx or shutdownCtx
joinedCtx, _ := joincontext.Join(tr.killCtx, tr.shutdownCtx)
2018-07-17 00:19:56 +00:00
var resp interfaces.TaskPrestartResponse
if err := pre.Prestart(joinedCtx, &req, &resp); err != nil {
tr.emitHookError(err, name)
2018-07-17 00:19:56 +00:00
return structs.WrapRecoverable(fmt.Sprintf("prestart hook %q failed: %v", name, err), err)
}
2018-06-22 00:35:07 +00:00
// Store the hook state
{
hookState := &state.HookState{
Data: resp.State,
PrestartDone: resp.Done,
Env: resp.Env,
2018-06-22 00:35:07 +00:00
}
// Store and persist local state if the hook state has changed
if !hookState.Equal(origHookState) {
tr.stateLock.Lock()
tr.localState.Hooks[name] = hookState
tr.stateLock.Unlock()
if err := tr.persistLocalState(); err != nil {
return err
}
}
2018-06-22 00:35:07 +00:00
}
// Store the environment variables returned by the hook
if name == HookNameDevices {
tr.envBuilder.SetDeviceHookEnv(name, resp.Env)
} else {
tr.envBuilder.SetHookEnv(name, resp.Env)
}
2018-06-22 00:35:07 +00:00
// Store the resources
if len(resp.Devices) != 0 {
tr.hookResources.setDevices(resp.Devices)
}
if len(resp.Mounts) != 0 {
tr.hookResources.setMounts(resp.Mounts)
}
2018-06-22 00:35:07 +00:00
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished prestart hook", "name", name, "end", end, "duration", end.Sub(start))
2018-06-22 00:35:07 +00:00
}
}
return nil
}
2018-07-17 00:19:56 +00:00
// poststart is used to run the runners poststart hooks.
func (tr *TaskRunner) poststart() error {
2018-06-22 00:35:07 +00:00
if tr.logger.IsTrace() {
start := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("running poststart hooks", "start", start)
2018-06-22 00:35:07 +00:00
defer func() {
end := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("finished poststart hooks", "end", end, "duration", end.Sub(start))
2018-06-22 00:35:07 +00:00
}()
}
handle := tr.getDriverHandle()
net := handle.Network()
2019-01-03 23:00:52 +00:00
// Pass the lazy handle to the hooks so even if the driver exits and we
// launch a new one (external plugin), the handle will refresh.
lazyHandle := NewLazyHandle(tr.shutdownCtx, tr.getDriverHandle, tr.logger)
2019-01-03 23:00:52 +00:00
var merr multierror.Error
2018-06-22 00:35:07 +00:00
for _, hook := range tr.runnerHooks {
2018-07-17 00:19:56 +00:00
post, ok := hook.(interfaces.TaskPoststartHook)
2018-06-22 00:35:07 +00:00
if !ok {
continue
}
name := post.Name()
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("running poststart hook", "name", name, "start", start)
2018-06-22 00:35:07 +00:00
}
req := interfaces.TaskPoststartRequest{
2019-01-03 23:00:52 +00:00
DriverExec: lazyHandle,
DriverNetwork: net,
2019-01-03 23:00:52 +00:00
DriverStats: lazyHandle,
TaskEnv: tr.envBuilder.Build(),
}
2018-07-17 00:19:56 +00:00
var resp interfaces.TaskPoststartResponse
if err := post.Poststart(tr.killCtx, &req, &resp); err != nil {
tr.emitHookError(err, name)
merr.Errors = append(merr.Errors, fmt.Errorf("poststart hook %q failed: %v", name, err))
2018-06-22 00:35:07 +00:00
}
// No need to persist as PoststartResponse is currently empty
2018-06-22 00:35:07 +00:00
if tr.logger.IsTrace() {
end := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("finished poststart hooks", "name", name, "end", end, "duration", end.Sub(start))
2018-06-22 00:35:07 +00:00
}
}
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.killCtx, &req, &resp); err != nil {
tr.emitHookError(err, name)
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()
2018-06-22 00:35:07 +00:00
}
2018-07-17 00:19:56 +00:00
// stop is used to run the stop hooks.
func (tr *TaskRunner) stop() error {
2018-06-22 00:35:07 +00:00
if tr.logger.IsTrace() {
start := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("running stop hooks", "start", start)
2018-06-22 00:35:07 +00:00
defer func() {
end := time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("finished stop hooks", "end", end, "duration", end.Sub(start))
2018-06-22 00:35:07 +00:00
}()
}
var merr multierror.Error
2018-06-22 00:35:07 +00:00
for _, hook := range tr.runnerHooks {
2018-07-17 00:19:56 +00:00
post, ok := hook.(interfaces.TaskStopHook)
2018-06-22 00:35:07 +00:00
if !ok {
continue
}
name := post.Name()
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
2018-07-17 00:19:56 +00:00
tr.logger.Trace("running stop hook", "name", name, "start", start)
2018-06-22 00:35:07 +00:00
}
2018-07-17 00:19:56 +00:00
req := interfaces.TaskStopRequest{}
origHookState := tr.hookState(name)
if origHookState != nil {
// Give the hook data provided by prestart
req.ExistingState = origHookState.Data
}
2018-07-17 00:19:56 +00:00
var resp interfaces.TaskStopResponse
if err := post.Stop(tr.killCtx, &req, &resp); err != nil {
tr.emitHookError(err, name)
merr.Errors = append(merr.Errors, fmt.Errorf("stop hook %q failed: %v", name, err))
2018-06-22 00:35:07 +00:00
}
// Stop hooks cannot alter state and must be idempotent, so
// unlike prestart there's no state to persist here.
2018-06-22 00:35:07 +00:00
if tr.logger.IsTrace() {
end := time.Now()
2018-10-12 01:01:42 +00:00
tr.logger.Trace("finished stop hook", "name", name, "end", end, "duration", end.Sub(start))
2018-06-22 00:35:07 +00:00
}
}
return merr.ErrorOrNil()
2018-06-22 00:35:07 +00:00
}
// 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.
2018-07-12 23:15:33 +00:00
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
2018-07-12 23:15:33 +00:00
for _, hook := range tr.runnerHooks {
upd, ok := hook.(interfaces.TaskUpdateHook)
if !ok {
continue
}
name := upd.Name()
// Build the request
req := interfaces.TaskUpdateRequest{
VaultToken: tr.getVaultToken(),
Alloc: alloc,
TaskEnv: tr.envBuilder.Build(),
2018-07-12 23:15:33 +00:00
}
2018-07-17 00:19:56 +00:00
// Time the update hook
2018-07-12 23:15:33 +00:00
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.killCtx, &req, &resp); err != nil {
tr.emitHookError(err, name)
2018-07-12 23:15:33 +00:00
tr.logger.Error("update hook failed", "name", name, "error", err)
}
// No need to persist as TaskUpdateResponse is currently empty
2018-07-12 23:15:33 +00:00
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished update hooks", "name", name, "end", end, "duration", end.Sub(start))
}
}
}
// preKill is used to run the runners preKill hooks
// preKill hooks contain logic that must be executed before
// a task is killed or restarted
func (tr *TaskRunner) preKill() {
if tr.logger.IsTrace() {
start := time.Now()
2019-01-22 15:45:58 +00:00
tr.logger.Trace("running pre kill hooks", "start", start)
defer func() {
end := time.Now()
2019-01-22 15:45:58 +00:00
tr.logger.Trace("finished pre kill hooks", "end", end, "duration", end.Sub(start))
}()
}
for _, hook := range tr.runnerHooks {
killHook, ok := hook.(interfaces.TaskPreKillHook)
if !ok {
continue
}
name := killHook.Name()
// Time the pre kill hook
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running prekill hook", "name", name, "start", start)
}
// Run the pre kill hook
req := interfaces.TaskPreKillRequest{}
var resp interfaces.TaskPreKillResponse
if err := killHook.PreKilling(context.Background(), &req, &resp); err != nil {
tr.emitHookError(err, name)
tr.logger.Error("prekill 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 prekill hook", "name", name, "end", end, "duration", end.Sub(start))
}
}
}
// shutdownHooks is called when the TaskRunner is gracefully shutdown but the
// task is not being stopped or garbage collected.
func (tr *TaskRunner) shutdownHooks() {
for _, hook := range tr.runnerHooks {
sh, ok := hook.(interfaces.ShutdownHook)
if !ok {
continue
}
name := sh.Name()
// Time the update hook
var start time.Time
if tr.logger.IsTrace() {
start = time.Now()
tr.logger.Trace("running shutdown hook", "name", name, "start", start)
}
sh.Shutdown()
if tr.logger.IsTrace() {
end := time.Now()
tr.logger.Trace("finished shutdown hook", "name", name, "end", end, "duration", end.Sub(start))
}
}
}