Merge pull request #5168 from hashicorp/b-kill-race

Improve Kill handling on task runner
This commit is contained in:
Alex Dadgar 2019-01-09 12:05:10 -08:00 committed by GitHub
commit bd12e0b1f7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 129 additions and 57 deletions

View File

@ -4,12 +4,13 @@ import (
"context" "context"
"github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
) )
// Restart a task. Returns immediately if no task is running. Blocks until // Restart a task. Returns immediately if no task is running. Blocks until
// existing task exits or passed-in context is canceled. // existing task exits or passed-in context is canceled.
func (tr *TaskRunner) Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error { func (tr *TaskRunner) Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error {
tr.logger.Trace("Restart requested", "failure", failure)
// Grab the handle // Grab the handle
handle := tr.getDriverHandle() handle := tr.getDriverHandle()
@ -47,6 +48,8 @@ func (tr *TaskRunner) Restart(ctx context.Context, event *structs.TaskEvent, fai
} }
func (tr *TaskRunner) Signal(event *structs.TaskEvent, s string) error { func (tr *TaskRunner) Signal(event *structs.TaskEvent, s string) error {
tr.logger.Trace("Signal requested", "signal", s)
// Grab the handle // Grab the handle
handle := tr.getDriverHandle() handle := tr.getDriverHandle()
@ -65,58 +68,28 @@ func (tr *TaskRunner) Signal(event *structs.TaskEvent, s string) error {
// Kill a task. Blocks until task exits or context is canceled. State is set to // Kill a task. Blocks until task exits or context is canceled. State is set to
// dead. // dead.
func (tr *TaskRunner) Kill(ctx context.Context, event *structs.TaskEvent) error { func (tr *TaskRunner) Kill(ctx context.Context, event *structs.TaskEvent) error {
tr.logger.Trace("Kill requested", "event_type", event.Type, "event_reason", event.KillReason)
// Cancel the task runner to break out of restart delay or the main run // Cancel the task runner to break out of restart delay or the main run
// loop. // loop.
tr.killCtxCancel() tr.killCtxCancel()
// Grab the handle // Emit kill event
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) tr.EmitEvent(event)
// Run the hooks prior to killing the task // Check if the Run method has started yet. If it hasn't we return early,
tr.killing() // since the task hasn't even started so there is nothing to wait for. This
// is still correct since the Run method no-op since the kill context has
// Tell the restart tracker that the task has been killed so it doesn't // already been cancelled.
// attempt to restart it. if !tr.hasRunLaunched() {
tr.restartTracker.SetKilled() return nil
// Kill the task using an exponential backoff in-case of failures.
killErr := tr.killTask(handle)
if killErr != nil {
// We couldn't successfully destroy the resource created.
tr.logger.Error("failed to kill task. Resources may have been leaked", "error", killErr)
}
// 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
} }
select { select {
case <-waitCh: case <-tr.WaitCh():
case <-ctx.Done(): case <-ctx.Done():
return ctx.Err()
} }
if killErr != nil { return tr.getKillErr()
return killErr
} else if err := ctx.Err(); err != nil {
return err
}
return nil
} }

View File

@ -199,6 +199,12 @@ func (h *serviceHook) getTaskServices() *agentconsul.TaskServices {
// interpolateServices returns an interpolated copy of services and checks with // interpolateServices returns an interpolated copy of services and checks with
// values from the task's environment. // values from the task's environment.
func interpolateServices(taskEnv *taskenv.TaskEnv, services []*structs.Service) []*structs.Service { func interpolateServices(taskEnv *taskenv.TaskEnv, services []*structs.Service) []*structs.Service {
// Guard against not having a valid taskEnv. This can be the case if the
// Killing or Exited hook is run before post-run.
if taskEnv == nil || len(services) == 0 {
return nil
}
interpolated := make([]*structs.Service, len(services)) interpolated := make([]*structs.Service, len(services))
for i, origService := range services { for i, origService := range services {

View File

@ -85,6 +85,13 @@ type TaskRunner struct {
// stateDB is for persisting localState and taskState // stateDB is for persisting localState and taskState
stateDB cstate.StateDB stateDB cstate.StateDB
// shutdownCtx is used to exit the TaskRunner *without* affecting task state.
shutdownCtx context.Context
// shutdownCtxCancel causes the TaskRunner to exit immediately without
// affecting task state. Useful for testing or graceful agent shutdown.
shutdownCtxCancel context.CancelFunc
// killCtx is the task runner's context representing the tasks's lifecycle. // killCtx is the task runner's context representing the tasks's lifecycle.
// The context is canceled when the task is killed. // The context is canceled when the task is killed.
killCtx context.Context killCtx context.Context
@ -92,12 +99,10 @@ type TaskRunner struct {
// killCtxCancel is called when killing a task. // killCtxCancel is called when killing a task.
killCtxCancel context.CancelFunc killCtxCancel context.CancelFunc
// ctx is used to exit the TaskRunner *without* affecting task state. // killErr is populated when killing a task. Access should be done use the
ctx context.Context // getter/setter
killErr error
// ctxCancel causes the TaskRunner to exit immediately without killErrLock sync.Mutex
// affecting task state. Useful for testing or graceful agent shutdown.
ctxCancel context.CancelFunc
// Logger is the logger for the task runner. // Logger is the logger for the task runner.
logger log.Logger logger log.Logger
@ -181,6 +186,11 @@ type TaskRunner struct {
// driverManager is used to dispense driver plugins and register event // driverManager is used to dispense driver plugins and register event
// handlers // handlers
driverManager drivermanager.Manager driverManager drivermanager.Manager
// runLaunched marks whether the Run goroutine has been started. It should
// be accessed via helpers
runLaunched bool
runLaunchedLock sync.Mutex
} }
type Config struct { type Config struct {
@ -251,8 +261,8 @@ func NewTaskRunner(config *Config) (*TaskRunner, error) {
deviceStatsReporter: config.DeviceStatsReporter, deviceStatsReporter: config.DeviceStatsReporter,
killCtx: killCtx, killCtx: killCtx,
killCtxCancel: killCancel, killCtxCancel: killCancel,
ctx: trCtx, shutdownCtx: trCtx,
ctxCancel: trCancel, shutdownCtxCancel: trCancel,
triggerUpdateCh: make(chan struct{}, triggerUpdateChCap), triggerUpdateCh: make(chan struct{}, triggerUpdateChCap),
waitCh: make(chan struct{}), waitCh: make(chan struct{}),
devicemanager: config.DeviceManager, devicemanager: config.DeviceManager,
@ -360,6 +370,10 @@ func (tr *TaskRunner) initLabels() {
// Run the TaskRunner. Starts the user's task or reattaches to a restored task. // Run the TaskRunner. Starts the user's task or reattaches to a restored task.
// Run closes WaitCh when it exits. Should be started in a goroutine. // Run closes WaitCh when it exits. Should be started in a goroutine.
func (tr *TaskRunner) Run() { func (tr *TaskRunner) Run() {
// Mark that the run routine has been launched so that other functions can
// decide to use the wait channel or not.
tr.setRunLaunched()
defer close(tr.waitCh) defer close(tr.waitCh)
var result *drivers.ExitResult var result *drivers.ExitResult
@ -372,8 +386,9 @@ MAIN:
for { for {
select { select {
case <-tr.killCtx.Done(): case <-tr.killCtx.Done():
tr.handleKill()
break MAIN break MAIN
case <-tr.ctx.Done(): case <-tr.shutdownCtx.Done():
// TaskRunner was told to exit immediately // TaskRunner was told to exit immediately
return return
default: default:
@ -388,8 +403,9 @@ MAIN:
select { select {
case <-tr.killCtx.Done(): case <-tr.killCtx.Done():
tr.handleKill()
break MAIN break MAIN
case <-tr.ctx.Done(): case <-tr.shutdownCtx.Done():
// TaskRunner was told to exit immediately // TaskRunner was told to exit immediately
return return
default: default:
@ -421,7 +437,11 @@ MAIN:
tr.logger.Error("wait task failed", "error", err) tr.logger.Error("wait task failed", "error", err)
} else { } else {
select { select {
case <-tr.ctx.Done(): case <-tr.killCtx.Done():
// We can go through the normal should restart check since
// the restart tracker knowns it is killed
tr.handleKill()
case <-tr.shutdownCtx.Done():
// TaskRunner was told to exit immediately // TaskRunner was told to exit immediately
return return
case result = <-resultCh: case result = <-resultCh:
@ -455,8 +475,9 @@ MAIN:
case <-time.After(restartDelay): case <-time.After(restartDelay):
case <-tr.killCtx.Done(): case <-tr.killCtx.Done():
tr.logger.Trace("task killed between restarts", "delay", restartDelay) tr.logger.Trace("task killed between restarts", "delay", restartDelay)
tr.handleKill()
break MAIN break MAIN
case <-tr.ctx.Done(): case <-tr.shutdownCtx.Done():
// TaskRunner was told to exit immediately // TaskRunner was told to exit immediately
return return
} }
@ -682,6 +703,50 @@ func (tr *TaskRunner) initDriver() error {
return nil return nil
} }
// handleKill is used to handle the a request to kill a task. It will store any
// error in the task runner killErr value.
func (tr *TaskRunner) handleKill() {
// Run the hooks prior to killing the task
tr.killing()
// Tell the restart tracker that the task has been killed so it doesn't
// attempt to restart it.
tr.restartTracker.SetKilled()
// Check it is running
handle := tr.getDriverHandle()
if handle == nil {
return
}
// Kill the task using an exponential backoff in-case of failures.
killErr := tr.killTask(handle)
if killErr != nil {
// We couldn't successfully destroy the resource created.
tr.logger.Error("failed to kill task. Resources may have been leaked", "error", killErr)
tr.setKillErr(killErr)
}
// Block until task has exited.
waitCh, err := handle.WaitCh(tr.shutdownCtx)
// 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
}
tr.logger.Error("failed to wait on task. Resources may have been leaked", "error", err)
tr.setKillErr(killErr)
return
}
select {
case <-waitCh:
case <-tr.shutdownCtx.Done():
}
}
// killTask kills the task handle. In the case that killing fails, // killTask kills the task handle. In the case that killing fails,
// killTask will retry with an exponential backoff and will give up at a // killTask will retry with an exponential backoff and will give up at a
// given limit. Returns an error if the task could not be killed. // given limit. Returns an error if the task could not be killed.
@ -1009,7 +1074,7 @@ func (tr *TaskRunner) triggerUpdateHooks() {
// Shutdown blocks until the main Run loop exits. // Shutdown blocks until the main Run loop exits.
func (tr *TaskRunner) Shutdown() { func (tr *TaskRunner) Shutdown() {
tr.logger.Trace("shutting down") tr.logger.Trace("shutting down")
tr.ctxCancel() tr.shutdownCtxCancel()
<-tr.WaitCh() <-tr.WaitCh()

View File

@ -85,3 +85,31 @@ func (tr *TaskRunner) clearDriverHandle() {
} }
tr.handle = nil tr.handle = nil
} }
// setKillErr stores any error that arouse while killing the task
func (tr *TaskRunner) setKillErr(err error) {
tr.killErrLock.Lock()
defer tr.killErrLock.Unlock()
tr.killErr = err
}
// getKillErr returns any error that arouse while killing the task
func (tr *TaskRunner) getKillErr() error {
tr.killErrLock.Lock()
defer tr.killErrLock.Unlock()
return tr.killErr
}
// setRunLaunched marks the fact that the Run loop has been started
func (tr *TaskRunner) setRunLaunched() {
tr.runLaunchedLock.Lock()
defer tr.runLaunchedLock.Unlock()
tr.runLaunched = true
}
// hasRunLaunched returns whether the Run loop has been started
func (tr *TaskRunner) hasRunLaunched() bool {
tr.runLaunchedLock.Lock()
defer tr.runLaunchedLock.Unlock()
return tr.runLaunched
}

View File

@ -256,7 +256,7 @@ func (tr *TaskRunner) poststart() error {
// Pass the lazy handle to the hooks so even if the driver exits and we // 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. // launch a new one (external plugin), the handle will refresh.
lazyHandle := NewLazyHandle(tr.ctx, tr.getDriverHandle, tr.logger) lazyHandle := NewLazyHandle(tr.shutdownCtx, tr.getDriverHandle, tr.logger)
var merr multierror.Error var merr multierror.Error
for _, hook := range tr.runnerHooks { for _, hook := range tr.runnerHooks {