Skip to content

Commit f74f508

Browse files
authored
Task lifecycle restart (#14127)
* allocrunner: handle lifecycle when all tasks die When all tasks die the Coordinator must transition to its terminal state, coordinatorStatePoststop, to unblock poststop tasks. Since this could happen at any time (for example, a prestart task dies), all states must be able to transition to this terminal state. * allocrunner: implement different alloc restarts Add a new alloc restart mode where all tasks are restarted, even if they have already exited. Also unifies the alloc restart logic to use the implementation that restarts tasks concurrently and ignores ErrTaskNotRunning errors since those are expected when restarting the allocation. * allocrunner: allow tasks to run again Prevent the task runner Run() method from exiting to allow a dead task to run again. When the task runner is signaled to restart, the function will jump back to the MAIN loop and run it again. The task runner determines if a task needs to run again based on two new task events that were added to differentiate between a request to restart a specific task, the tasks that are currently running, or all tasks that have already run. * api/cli: add support for all tasks alloc restart Implement the new -all-tasks alloc restart CLI flag and its API counterpar, AllTasks. The client endpoint calls the appropriate restart method from the allocrunner depending on the restart parameters used. * test: fix tasklifecycle Coordinator test * allocrunner: kill taskrunners if all tasks are dead When all non-poststop tasks are dead we need to kill the taskrunners so we don't leak their goroutines, which are blocked in the alloc restart loop. This also ensures the allocrunner exits on its own. * taskrunner: fix tests that waited on WaitCh Now that "dead" tasks may run again, the taskrunner Run() method will not return when the task finishes running, so tests must wait for the task state to be "dead" instead of using the WaitCh, since it won't be closed until the taskrunner is killed. * tests: add tests for all tasks alloc restart * changelog: add entry for #14127 * taskrunner: fix restore logic. The first implementation of the task runner restore process relied on server data (`tr.Alloc().TerminalStatus()`) which may not be available to the client at the time of restore. It also had the incorrect code path. When restoring a dead task the driver handle always needs to be clear cleanly using `clearDriverHandle` otherwise, after exiting the MAIN loop, the task may be killed by `tr.handleKill`. The fix is to store the state of the Run() loop in the task runner local client state: if the task runner ever exits this loop cleanly (not with a shutdown) it will never be able to run again. So if the Run() loops starts with this local state flag set, it must exit early. This local state flag is also being checked on task restart requests. If the task is "dead" and its Run() loop is not active it will never be able to run again. * address code review requests * apply more code review changes * taskrunner: add different Restart modes Using the task event to differentiate between the allocrunner restart methods proved to be confusing for developers to understand how it all worked. So instead of relying on the event type, this commit separated the logic of restarting an taskRunner into two methods: - `Restart` will retain the current behaviour and only will only restart the task if it's currently running. - `ForceRestart` is the new method where a `dead` task is allowed to restart if its `Run()` method is still active. Callers will need to restart the allocRunner taskCoordinator to make sure it will allow the task to run again. * minor fixes
1 parent e886d5d commit f74f508

22 files changed

+1098
-212
lines changed

.changelog/14127.txt

+7
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,7 @@
1+
```release-note:improvement
2+
client: add option to restart all tasks of an allocation, regardless of lifecycle type or state.
3+
```
4+
5+
```release-note:improvement
6+
client: only start poststop tasks after poststart tasks are done.
7+
```

api/allocations.go

+20-1
Original file line numberDiff line numberDiff line change
@@ -141,7 +141,9 @@ func (a *Allocations) GC(alloc *Allocation, q *QueryOptions) error {
141141
return err
142142
}
143143

144-
// Restart restarts an allocation.
144+
// Restart restarts the tasks that are currently running or a specific task if
145+
// taskName is provided. An error is returned if the task to be restarted is
146+
// not running.
145147
//
146148
// Note: for cluster topologies where API consumers don't have network access to
147149
// Nomad clients, set api.ClientConnTimeout to a small value (ex 1ms) to avoid
@@ -156,6 +158,22 @@ func (a *Allocations) Restart(alloc *Allocation, taskName string, q *QueryOption
156158
return err
157159
}
158160

161+
// RestartAllTasks restarts all tasks in the allocation, regardless of
162+
// lifecycle type or state. Tasks will restart following their lifecycle order.
163+
//
164+
// Note: for cluster topologies where API consumers don't have network access to
165+
// Nomad clients, set api.ClientConnTimeout to a small value (ex 1ms) to avoid
166+
// long pauses on this API call.
167+
func (a *Allocations) RestartAllTasks(alloc *Allocation, q *QueryOptions) error {
168+
req := AllocationRestartRequest{
169+
AllTasks: true,
170+
}
171+
172+
var resp struct{}
173+
_, err := a.client.putQuery("/v1/client/allocation/"+alloc.ID+"/restart", &req, &resp, q)
174+
return err
175+
}
176+
159177
// Stop stops an allocation.
160178
//
161179
// Note: for cluster topologies where API consumers don't have network access to
@@ -447,6 +465,7 @@ func (a Allocation) RescheduleInfo(t time.Time) (int, int) {
447465

448466
type AllocationRestartRequest struct {
449467
TaskName string
468+
AllTasks bool
450469
}
451470

452471
type AllocSignalRequest struct {

client/alloc_endpoint.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -102,7 +102,7 @@ func (a *Allocations) Restart(args *nstructs.AllocRestartRequest, reply *nstruct
102102
return nstructs.ErrPermissionDenied
103103
}
104104

105-
return a.c.RestartAllocation(args.AllocID, args.TaskName)
105+
return a.c.RestartAllocation(args.AllocID, args.TaskName, args.AllTasks)
106106
}
107107

108108
// Stats is used to collect allocation statistics

client/alloc_endpoint_test.go

+39
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,45 @@ func TestAllocations_Restart(t *testing.T) {
6868
})
6969
}
7070

71+
func TestAllocations_RestartAllTasks(t *testing.T) {
72+
ci.Parallel(t)
73+
74+
require := require.New(t)
75+
client, cleanup := TestClient(t, nil)
76+
defer cleanup()
77+
78+
alloc := mock.LifecycleAlloc()
79+
require.Nil(client.addAlloc(alloc, ""))
80+
81+
// Can't restart all tasks while specifying a task name.
82+
req := &nstructs.AllocRestartRequest{
83+
AllocID: alloc.ID,
84+
AllTasks: true,
85+
TaskName: "web",
86+
}
87+
var resp nstructs.GenericResponse
88+
err := client.ClientRPC("Allocations.Restart", &req, &resp)
89+
require.Error(err)
90+
91+
// Good request.
92+
req = &nstructs.AllocRestartRequest{
93+
AllocID: alloc.ID,
94+
AllTasks: true,
95+
}
96+
97+
testutil.WaitForResult(func() (bool, error) {
98+
var resp2 nstructs.GenericResponse
99+
err := client.ClientRPC("Allocations.Restart", &req, &resp2)
100+
if err != nil && strings.Contains(err.Error(), "not running") {
101+
return false, err
102+
}
103+
104+
return true, nil
105+
}, func(err error) {
106+
t.Fatalf("err: %v", err)
107+
})
108+
}
109+
71110
func TestAllocations_Restart_ACL(t *testing.T) {
72111
ci.Parallel(t)
73112
require := require.New(t)

client/allocrunner/alloc_runner.go

+88-57
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,6 @@ import (
2828
cstate "github.com/hashicorp/nomad/client/state"
2929
cstructs "github.com/hashicorp/nomad/client/structs"
3030
"github.com/hashicorp/nomad/client/vaultclient"
31-
agentconsul "github.com/hashicorp/nomad/command/agent/consul"
3231
"github.com/hashicorp/nomad/helper/pointer"
3332
"github.com/hashicorp/nomad/nomad/structs"
3433
"github.com/hashicorp/nomad/plugins/device"
@@ -547,40 +546,64 @@ func (ar *allocRunner) handleTaskStateUpdates() {
547546
}
548547
}
549548

550-
// if all live runners are sidecars - kill alloc
551-
if killEvent == nil && hasSidecars && !hasNonSidecarTasks(liveRunners) {
552-
killEvent = structs.NewTaskEvent(structs.TaskMainDead)
553-
}
554-
555-
// If there's a kill event set and live runners, kill them
556-
if killEvent != nil && len(liveRunners) > 0 {
557-
558-
// Log kill reason
559-
switch killEvent.Type {
560-
case structs.TaskLeaderDead:
561-
ar.logger.Debug("leader task dead, destroying all tasks", "leader_task", killTask)
562-
case structs.TaskMainDead:
563-
ar.logger.Debug("main tasks dead, destroying all sidecar tasks")
564-
default:
565-
ar.logger.Debug("task failure, destroying all tasks", "failed_task", killTask)
549+
if len(liveRunners) > 0 {
550+
// if all live runners are sidecars - kill alloc
551+
onlySidecarsRemaining := hasSidecars && !hasNonSidecarTasks(liveRunners)
552+
if killEvent == nil && onlySidecarsRemaining {
553+
killEvent = structs.NewTaskEvent(structs.TaskMainDead)
566554
}
567555

568-
// Emit kill event for live runners
569-
for _, tr := range liveRunners {
570-
tr.EmitEvent(killEvent)
571-
}
556+
// If there's a kill event set and live runners, kill them
557+
if killEvent != nil {
558+
559+
// Log kill reason
560+
switch killEvent.Type {
561+
case structs.TaskLeaderDead:
562+
ar.logger.Debug("leader task dead, destroying all tasks", "leader_task", killTask)
563+
case structs.TaskMainDead:
564+
ar.logger.Debug("main tasks dead, destroying all sidecar tasks")
565+
default:
566+
ar.logger.Debug("task failure, destroying all tasks", "failed_task", killTask)
567+
}
572568

573-
// Kill 'em all
574-
states = ar.killTasks()
569+
// Emit kill event for live runners
570+
for _, tr := range liveRunners {
571+
tr.EmitEvent(killEvent)
572+
}
573+
574+
// Kill 'em all
575+
states = ar.killTasks()
576+
577+
// Wait for TaskRunners to exit before continuing. This will
578+
// prevent looping before TaskRunners have transitioned to
579+
// Dead.
580+
for _, tr := range liveRunners {
581+
ar.logger.Info("waiting for task to exit", "task", tr.Task().Name)
582+
select {
583+
case <-tr.WaitCh():
584+
case <-ar.waitCh:
585+
}
586+
}
587+
}
588+
} else {
589+
// If there are no live runners left kill all non-poststop task
590+
// runners to unblock them from the alloc restart loop.
591+
for _, tr := range ar.tasks {
592+
if tr.IsPoststopTask() {
593+
continue
594+
}
575595

576-
// Wait for TaskRunners to exit before continuing to
577-
// prevent looping before TaskRunners have transitioned
578-
// to Dead.
579-
for _, tr := range liveRunners {
580-
ar.logger.Info("killing task", "task", tr.Task().Name)
581596
select {
582597
case <-tr.WaitCh():
583598
case <-ar.waitCh:
599+
default:
600+
// Kill task runner without setting an event because the
601+
// task is already dead, it's just waiting in the alloc
602+
// restart loop.
603+
err := tr.Kill(context.TODO(), nil)
604+
if err != nil {
605+
ar.logger.Warn("failed to kill task", "task", tr.Task().Name, "error", err)
606+
}
584607
}
585608
}
586609
}
@@ -648,7 +671,7 @@ func (ar *allocRunner) killTasks() map[string]*structs.TaskState {
648671
break
649672
}
650673

651-
// Kill the rest non-sidecar or poststop tasks concurrently
674+
// Kill the rest non-sidecar and non-poststop tasks concurrently
652675
wg := sync.WaitGroup{}
653676
for name, tr := range ar.tasks {
654677
// Filter out poststop and sidecar tasks so that they stop after all the other tasks are killed
@@ -1205,19 +1228,37 @@ func (ar *allocRunner) GetTaskEventHandler(taskName string) drivermanager.EventH
12051228
return nil
12061229
}
12071230

1208-
// RestartTask signalls the task runner for the provided task to restart.
1209-
func (ar *allocRunner) RestartTask(taskName string, taskEvent *structs.TaskEvent) error {
1231+
// Restart satisfies the WorkloadRestarter interface and restarts all tasks
1232+
// that are currently running.
1233+
func (ar *allocRunner) Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error {
1234+
return ar.restartTasks(ctx, event, failure, false)
1235+
}
1236+
1237+
// RestartTask restarts the provided task.
1238+
func (ar *allocRunner) RestartTask(taskName string, event *structs.TaskEvent) error {
12101239
tr, ok := ar.tasks[taskName]
12111240
if !ok {
12121241
return fmt.Errorf("Could not find task runner for task: %s", taskName)
12131242
}
12141243

1215-
return tr.Restart(context.TODO(), taskEvent, false)
1244+
return tr.Restart(context.TODO(), event, false)
12161245
}
12171246

1218-
// Restart satisfies the WorkloadRestarter interface restarts all task runners
1219-
// concurrently
1220-
func (ar *allocRunner) Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error {
1247+
// RestartRunning restarts all tasks that are currently running.
1248+
func (ar *allocRunner) RestartRunning(event *structs.TaskEvent) error {
1249+
return ar.restartTasks(context.TODO(), event, false, false)
1250+
}
1251+
1252+
// RestartAll restarts all tasks in the allocation, including dead ones. They
1253+
// will restart following their lifecycle order.
1254+
func (ar *allocRunner) RestartAll(event *structs.TaskEvent) error {
1255+
// Restart the taskCoordinator to allow dead tasks to run again.
1256+
ar.taskCoordinator.Restart()
1257+
return ar.restartTasks(context.TODO(), event, false, true)
1258+
}
1259+
1260+
// restartTasks restarts all task runners concurrently.
1261+
func (ar *allocRunner) restartTasks(ctx context.Context, event *structs.TaskEvent, failure bool, force bool) error {
12211262
waitCh := make(chan struct{})
12221263
var err *multierror.Error
12231264
var errMutex sync.Mutex
@@ -1230,10 +1271,19 @@ func (ar *allocRunner) Restart(ctx context.Context, event *structs.TaskEvent, fa
12301271
defer close(waitCh)
12311272
for tn, tr := range ar.tasks {
12321273
wg.Add(1)
1233-
go func(taskName string, r agentconsul.WorkloadRestarter) {
1274+
go func(taskName string, taskRunner *taskrunner.TaskRunner) {
12341275
defer wg.Done()
1235-
e := r.Restart(ctx, event, failure)
1236-
if e != nil {
1276+
1277+
var e error
1278+
if force {
1279+
e = taskRunner.ForceRestart(ctx, event.Copy(), failure)
1280+
} else {
1281+
e = taskRunner.Restart(ctx, event.Copy(), failure)
1282+
}
1283+
1284+
// Ignore ErrTaskNotRunning errors since tasks that are not
1285+
// running are expected to not be restarted.
1286+
if e != nil && e != taskrunner.ErrTaskNotRunning {
12371287
errMutex.Lock()
12381288
defer errMutex.Unlock()
12391289
err = multierror.Append(err, fmt.Errorf("failed to restart task %s: %v", taskName, e))
@@ -1251,25 +1301,6 @@ func (ar *allocRunner) Restart(ctx context.Context, event *structs.TaskEvent, fa
12511301
return err.ErrorOrNil()
12521302
}
12531303

1254-
// RestartAll signalls all task runners in the allocation to restart and passes
1255-
// a copy of the task event to each restart event.
1256-
// Returns any errors in a concatenated form.
1257-
func (ar *allocRunner) RestartAll(taskEvent *structs.TaskEvent) error {
1258-
var err *multierror.Error
1259-
1260-
// run alloc task restart hooks
1261-
ar.taskRestartHooks()
1262-
1263-
for tn := range ar.tasks {
1264-
rerr := ar.RestartTask(tn, taskEvent.Copy())
1265-
if rerr != nil {
1266-
err = multierror.Append(err, rerr)
1267-
}
1268-
}
1269-
1270-
return err.ErrorOrNil()
1271-
}
1272-
12731304
// Signal sends a signal request to task runners inside an allocation. If the
12741305
// taskName is empty, then it is sent to all tasks.
12751306
func (ar *allocRunner) Signal(taskName, signal string) error {

0 commit comments

Comments
 (0)