From 20cfbc82d39e944aa63a22f8170fb8062bc2adcf Mon Sep 17 00:00:00 2001 From: Juana De La Cuesta Date: Mon, 19 Feb 2024 16:41:35 +0100 Subject: [PATCH] Introduces `Disconnect` block into the `TaskGroup` configuration (#19886) This PR is the first on two that will implement the new Disconnect block. In this PR the new block is introduced to be backwards compatible with the fields it will replace. For more information refer to this RFC and this ticket. --- api/tasks.go | 101 +- client/client.go | 2 +- client/heartbeatstop.go | 12 +- client/heartbeatstop_test.go | 67 +- command/agent/job_endpoint.go | 15 + command/agent/job_endpoint_test.go | 8 + command/testdata/example-short-bad.json | 4 + command/testdata/example-short.json | 1 + jobspec/parse.go | 36 + jobspec/parse_group.go | 9 + jobspec/parse_test.go | 14 +- jobspec/test-fixtures/basic.hcl | 7 + nomad/heartbeat_test.go | 74 +- nomad/node_endpoint_test.go | 376 ++-- nomad/plan_apply.go | 2 +- nomad/structs/alloc_test.go | 503 ++++- nomad/structs/diff.go | 29 + nomad/structs/diff_test.go | 203 ++ nomad/structs/group.go | 129 ++ nomad/structs/group_test.go | 240 +++ nomad/structs/structs.go | 201 +- nomad/structs/structs_test.go | 323 --- scheduler/generic_sched_test.go | 315 ++- scheduler/reconcile.go | 2 +- scheduler/reconcile_test.go | 2 + scheduler/reconcile_util.go | 6 +- scheduler/reconcile_util_test.go | 2593 ++++++++++++----------- 27 files changed, 3316 insertions(+), 1958 deletions(-) create mode 100644 nomad/structs/group.go create mode 100644 nomad/structs/group_test.go diff --git a/api/tasks.go b/api/tasks.go index d589d2889..db94b76a9 100644 --- a/api/tasks.go +++ b/api/tasks.go @@ -11,6 +11,8 @@ import ( "time" ) +type ReconcileOption = string + const ( // RestartPolicyModeDelay causes an artificial delay till the next interval is // reached when the specified attempts have been reached in the interval. @@ -19,6 +21,14 @@ const ( // RestartPolicyModeFail causes a job to fail if the specified number of // attempts are reached within an interval. RestartPolicyModeFail = "fail" + + // ReconcileOption is used to specify the behavior of the reconciliation process + // between the original allocations and the replacements when a previously + // disconnected client comes back online. + ReconcileOptionKeepOriginal = "keep_original" + ReconcileOptionKeepReplacement = "keep_replacement" + ReconcileOptionBestScore = "best_score" + ReconcileOptionLongestRunning = "longest_running" ) // MemoryStats holds memory usage related stats @@ -113,6 +123,37 @@ func (r *RestartPolicy) Merge(rp *RestartPolicy) { } } +// Disconnect strategy defines how both clients and server should behave in case of +// disconnection between them. +type DisconnectStrategy struct { + // Defines for how long the server will consider the unresponsive node as + // disconnected but alive instead of lost. + LostAfter *time.Duration `mapstructure:"lost_after" hcl:"lost_after,optional"` + + // Defines for how long a disconnected client will keep its allocations running. + StopOnClientAfter *time.Duration `mapstructure:"stop_on_client_after" hcl:"stop_on_client_after,optional"` + + // A boolean field used to define if the allocations should be replaced while + // it's considered disconnected. + Replace *bool `mapstructure:"replace" hcl:"replace,optional"` + + // Once the disconnected node starts reporting again, it will define which + // instances to keep: the original allocations, the replacement, the one + // running on the node with the best score as it is currently implemented, + // or the allocation that has been running continuously the longest. + Reconcile *ReconcileOption `mapstructure:"reconcile" hcl:"reconcile,optional"` +} + +func (ds *DisconnectStrategy) Canonicalize() { + if ds.Replace == nil { + ds.Replace = pointerOf(true) + } + + if ds.Reconcile == nil { + ds.Reconcile = pointerOf(ReconcileOptionBestScore) + } +} + // Reschedule configures how Tasks are rescheduled when they crash or fail. type ReschedulePolicy struct { // Attempts limits the number of rescheduling attempts that can occur in an interval. @@ -205,6 +246,14 @@ func (a *Affinity) Canonicalize() { } } +func NewDefaultDisconnectStrategy() *DisconnectStrategy { + return &DisconnectStrategy{ + LostAfter: pointerOf(0 * time.Minute), + Replace: pointerOf(true), + Reconcile: pointerOf(ReconcileOptionBestScore), + } +} + func NewDefaultReschedulePolicy(jobType string) *ReschedulePolicy { var dp *ReschedulePolicy switch jobType { @@ -445,27 +494,31 @@ func (vm *VolumeMount) Canonicalize() { // TaskGroup is the unit of scheduling. type TaskGroup struct { - Name *string `hcl:"name,label"` - Count *int `hcl:"count,optional"` - Constraints []*Constraint `hcl:"constraint,block"` - Affinities []*Affinity `hcl:"affinity,block"` - Tasks []*Task `hcl:"task,block"` - Spreads []*Spread `hcl:"spread,block"` - Volumes map[string]*VolumeRequest `hcl:"volume,block"` - RestartPolicy *RestartPolicy `hcl:"restart,block"` - ReschedulePolicy *ReschedulePolicy `hcl:"reschedule,block"` - EphemeralDisk *EphemeralDisk `hcl:"ephemeral_disk,block"` - Update *UpdateStrategy `hcl:"update,block"` - Migrate *MigrateStrategy `hcl:"migrate,block"` - Networks []*NetworkResource `hcl:"network,block"` - Meta map[string]string `hcl:"meta,block"` - Services []*Service `hcl:"service,block"` - ShutdownDelay *time.Duration `mapstructure:"shutdown_delay" hcl:"shutdown_delay,optional"` - StopAfterClientDisconnect *time.Duration `mapstructure:"stop_after_client_disconnect" hcl:"stop_after_client_disconnect,optional"` - MaxClientDisconnect *time.Duration `mapstructure:"max_client_disconnect" hcl:"max_client_disconnect,optional"` - Scaling *ScalingPolicy `hcl:"scaling,block"` - Consul *Consul `hcl:"consul,block"` - PreventRescheduleOnLost *bool `hcl:"prevent_reschedule_on_lost,optional"` + Name *string `hcl:"name,label"` + Count *int `hcl:"count,optional"` + Constraints []*Constraint `hcl:"constraint,block"` + Affinities []*Affinity `hcl:"affinity,block"` + Tasks []*Task `hcl:"task,block"` + Spreads []*Spread `hcl:"spread,block"` + Volumes map[string]*VolumeRequest `hcl:"volume,block"` + RestartPolicy *RestartPolicy `hcl:"restart,block"` + Disconnect *DisconnectStrategy `hcl:"disconnect,block"` + ReschedulePolicy *ReschedulePolicy `hcl:"reschedule,block"` + EphemeralDisk *EphemeralDisk `hcl:"ephemeral_disk,block"` + Update *UpdateStrategy `hcl:"update,block"` + Migrate *MigrateStrategy `hcl:"migrate,block"` + Networks []*NetworkResource `hcl:"network,block"` + Meta map[string]string `hcl:"meta,block"` + Services []*Service `hcl:"service,block"` + ShutdownDelay *time.Duration `mapstructure:"shutdown_delay" hcl:"shutdown_delay,optional"` + // Deprecated: StopAfterClientDisconnect is deprecated in Nomad 1.8. Use Disconnect.StopOnClientAfter instead. + StopAfterClientDisconnect *time.Duration `mapstructure:"stop_after_client_disconnect" hcl:"stop_after_client_disconnect,optional"` + // To be deprecated after 1.8.0 infavour of Disconnect.LostAfter + MaxClientDisconnect *time.Duration `mapstructure:"max_client_disconnect" hcl:"max_client_disconnect,optional"` + Scaling *ScalingPolicy `hcl:"scaling,block"` + Consul *Consul `hcl:"consul,block"` + // To be deprecated after 1.8.0 infavour of Disconnect.Replace + PreventRescheduleOnLost *bool `hcl:"prevent_reschedule_on_lost,optional"` } // NewTaskGroup creates a new TaskGroup. @@ -537,6 +590,7 @@ func (g *TaskGroup) Canonicalize(job *Job) { if g.ReschedulePolicy != nil { g.ReschedulePolicy.Canonicalize(*job.Type) } + // Merge the migrate strategy from the job if jm, tm := job.Migrate != nil, g.Migrate != nil; jm && tm { jobMigrate := job.Migrate.Copy() @@ -584,9 +638,14 @@ func (g *TaskGroup) Canonicalize(job *Job) { for _, s := range g.Services { s.Canonicalize(nil, g, job) } + if g.PreventRescheduleOnLost == nil { g.PreventRescheduleOnLost = pointerOf(false) } + + if g.Disconnect != nil { + g.Disconnect.Canonicalize() + } } // These needs to be in sync with DefaultServiceJobRestartPolicy in diff --git a/client/client.go b/client/client.go index 8f041bc41..f79f51095 100644 --- a/client/client.go +++ b/client/client.go @@ -2668,7 +2668,7 @@ func (c *Client) updateAlloc(update *structs.Allocation) { // Reconnect unknown allocations if they were updated and are not terminal. reconnect := update.ClientStatus == structs.AllocClientStatusUnknown && update.AllocModifyIndex > alloc.AllocModifyIndex && - (!update.ServerTerminalStatus() || !alloc.PreventRescheduleOnLost()) + (!update.ServerTerminalStatus() || !alloc.PreventRescheduleOnDisconnect()) if reconnect { err = ar.Reconnect(update) if err != nil { diff --git a/client/heartbeatstop.go b/client/heartbeatstop.go index 1d6ba61ab..de3856786 100644 --- a/client/heartbeatstop.go +++ b/client/heartbeatstop.go @@ -47,7 +47,7 @@ func newHeartbeatStop( // allocation to be stopped if the taskgroup is configured appropriately func (h *heartbeatStop) allocHook(alloc *structs.Allocation) { tg := allocTaskGroup(alloc) - if tg.StopAfterClientDisconnect != nil { + if tg.GetDisconnectStopTimeout() != nil { h.allocHookCh <- alloc } } @@ -56,8 +56,9 @@ func (h *heartbeatStop) allocHook(alloc *structs.Allocation) { // past that it should be prevented from restarting func (h *heartbeatStop) shouldStop(alloc *structs.Allocation) bool { tg := allocTaskGroup(alloc) - if tg.StopAfterClientDisconnect != nil { - return h.shouldStopAfter(time.Now(), *tg.StopAfterClientDisconnect) + timeout := tg.GetDisconnectStopTimeout() + if timeout != nil { + return h.shouldStopAfter(time.Now(), *timeout) } return false } @@ -103,8 +104,9 @@ func (h *heartbeatStop) watch() { case alloc := <-h.allocHookCh: tg := allocTaskGroup(alloc) - if tg.StopAfterClientDisconnect != nil { - h.allocInterval[alloc.ID] = *tg.StopAfterClientDisconnect + timeout := tg.GetDisconnectStopTimeout() + if timeout != nil { + h.allocInterval[alloc.ID] = *timeout } case <-timeout: diff --git a/client/heartbeatstop_test.go b/client/heartbeatstop_test.go index 9de2eb07c..99e904cee 100644 --- a/client/heartbeatstop_test.go +++ b/client/heartbeatstop_test.go @@ -12,7 +12,7 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" - "github.com/stretchr/testify/require" + "github.com/shoenig/test/must" ) func TestHeartbeatStop_allocHook(t *testing.T) { @@ -27,6 +27,63 @@ func TestHeartbeatStop_allocHook(t *testing.T) { }) defer cleanupC1() + // an allocation, with a tiny lease + d := 1 * time.Microsecond + alloc := &structs.Allocation{ + ID: uuid.Generate(), + TaskGroup: "foo", + Job: &structs.Job{ + TaskGroups: []*structs.TaskGroup{ + { + Name: "foo", + Disconnect: &structs.DisconnectStrategy{ + StopOnClientAfter: &d, + }, + }, + }, + }, + Resources: &structs.Resources{ + CPU: 100, + MemoryMB: 100, + DiskMB: 0, + }, + } + + // alloc added to heartbeatStop.allocs + err := client.addAlloc(alloc, "") + must.NoError(t, err) + testutil.WaitForResult(func() (bool, error) { + _, ok := client.heartbeatStop.allocInterval[alloc.ID] + return ok, nil + }, func(err error) { + must.NoError(t, err) + }) + + // the tiny lease causes the watch loop to destroy it + testutil.WaitForResult(func() (bool, error) { + _, ok := client.heartbeatStop.allocInterval[alloc.ID] + return !ok, nil + }, func(err error) { + must.NoError(t, err) + }) + + must.Nil(t, client.allocs[alloc.ID]) +} + +// Test using stop_after_client_disconnect, remove after its deprecated in favor +// of Disconnect.StopOnClientAfter introduced in 1.8.0. +func TestHeartbeatStop_allocHook_Disconnect(t *testing.T) { + ci.Parallel(t) + + server, _, cleanupS1 := testServer(t, nil) + defer cleanupS1() + testutil.WaitForLeader(t, server.RPC) + + client, cleanupC1 := TestClient(t, func(c *config.Config) { + c.RPCHandler = server + }) + defer cleanupC1() + // an allocation, with a tiny lease d := 1 * time.Microsecond alloc := &structs.Allocation{ @@ -49,12 +106,12 @@ func TestHeartbeatStop_allocHook(t *testing.T) { // alloc added to heartbeatStop.allocs err := client.addAlloc(alloc, "") - require.NoError(t, err) + must.NoError(t, err) testutil.WaitForResult(func() (bool, error) { _, ok := client.heartbeatStop.allocInterval[alloc.ID] return ok, nil }, func(err error) { - require.NoError(t, err) + must.NoError(t, err) }) // the tiny lease causes the watch loop to destroy it @@ -62,8 +119,8 @@ func TestHeartbeatStop_allocHook(t *testing.T) { _, ok := client.heartbeatStop.allocInterval[alloc.ID] return !ok, nil }, func(err error) { - require.NoError(t, err) + must.NoError(t, err) }) - require.Empty(t, client.allocs[alloc.ID]) + must.Nil(t, client.allocs[alloc.ID]) } diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index fb1bf617c..04290a1e7 100644 --- a/command/agent/job_endpoint.go +++ b/command/agent/job_endpoint.go @@ -1164,6 +1164,21 @@ func ApiTgToStructsTG(job *structs.Job, taskGroup *api.TaskGroup, tg *structs.Ta } } + if taskGroup.Disconnect != nil { + tg.Disconnect = &structs.DisconnectStrategy{ + StopOnClientAfter: taskGroup.Disconnect.StopOnClientAfter, + Replace: taskGroup.Disconnect.Replace, + } + + if taskGroup.Disconnect.Reconcile != nil { + tg.Disconnect.Reconcile = *taskGroup.Disconnect.Reconcile + } + + if taskGroup.Disconnect.LostAfter != nil { + tg.Disconnect.LostAfter = *taskGroup.Disconnect.LostAfter + } + } + if taskGroup.Migrate != nil { tg.Migrate = &structs.MigrateStrategy{ MaxParallel: *taskGroup.Migrate.MaxParallel, diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index 05c7dcef8..7925740b0 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -2752,6 +2752,9 @@ func TestJobs_ApiJobToStructsJob(t *testing.T) { }, }, }, + Disconnect: &api.DisconnectStrategy{ + LostAfter: pointer.Of(30 * time.Second), + }, MaxClientDisconnect: pointer.Of(30 * time.Second), Tasks: []*api.Task{ { @@ -3185,6 +3188,11 @@ func TestJobs_ApiJobToStructsJob(t *testing.T) { }, }, }, + Disconnect: &structs.DisconnectStrategy{ + LostAfter: 30 * time.Second, + Replace: pointer.Of(true), + Reconcile: structs.ReconcileOptionBestScore, + }, MaxClientDisconnect: pointer.Of(30 * time.Second), Tasks: []*structs.Task{ { diff --git a/command/testdata/example-short-bad.json b/command/testdata/example-short-bad.json index 8f6fcdb6d..d3866e253 100644 --- a/command/testdata/example-short-bad.json +++ b/command/testdata/example-short-bad.json @@ -90,6 +90,10 @@ "ShutdownDelay": null, "StopAfterClientDisconnect": null, "MaxClientDisconnect": null, + "Disconnect":{ + "StopAfterClient": null, + "LostAfter": null + }, "Scaling": null, "Consul": null } diff --git a/command/testdata/example-short.json b/command/testdata/example-short.json index a62b33b64..44c4a000c 100644 --- a/command/testdata/example-short.json +++ b/command/testdata/example-short.json @@ -93,6 +93,7 @@ "ShutdownDelay": null, "StopAfterClientDisconnect": null, "MaxClientDisconnect": null, + "Disconnect": null, "Scaling": null, "Consul": null } diff --git a/jobspec/parse.go b/jobspec/parse.go index 6d622d542..245e6a9dd 100644 --- a/jobspec/parse.go +++ b/jobspec/parse.go @@ -451,6 +451,42 @@ func parseUpdate(result **api.UpdateStrategy, list *ast.ObjectList) error { return dec.Decode(m) } +func parseDisconnect(result **api.DisconnectStrategy, list *ast.ObjectList) error { + list = list.Elem() + if len(list.Items) > 1 { + return fmt.Errorf("only one 'disconnect' block allowed") + } + + // Get our resource object + o := list.Items[0] + + var m map[string]interface{} + if err := hcl.DecodeObject(&m, o.Val); err != nil { + return err + } + + // Check for invalid keys + valid := []string{ + "lost_after", + "replace", + "reconcile", + "stop_on_client_after", + } + if err := checkHCLKeys(o.Val, valid); err != nil { + return err + } + + dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ + DecodeHook: mapstructure.StringToTimeDurationHookFunc(), + WeaklyTypedInput: true, + Result: result, + }) + if err != nil { + return err + } + return dec.Decode(m) +} + func parseMigrate(result **api.MigrateStrategy, list *ast.ObjectList) error { list = list.Elem() if len(list.Items) > 1 { diff --git a/jobspec/parse_group.go b/jobspec/parse_group.go index 94dc3008d..e940955b7 100644 --- a/jobspec/parse_group.go +++ b/jobspec/parse_group.go @@ -50,6 +50,7 @@ func parseGroups(result *api.Job, list *ast.ObjectList) error { "task", "ephemeral_disk", "update", + "disconnect", "reschedule", "vault", "migrate", @@ -79,6 +80,7 @@ func parseGroups(result *api.Job, list *ast.ObjectList) error { delete(m, "restart") delete(m, "ephemeral_disk") delete(m, "update") + delete(m, "disconnect") delete(m, "vault") delete(m, "migrate") delete(m, "spread") @@ -168,6 +170,13 @@ func parseGroups(result *api.Job, list *ast.ObjectList) error { } } + // If we have an disconnect strategy, then parse that + if o := listVal.Filter("disconnect"); len(o.Items) > 0 { + if err := parseDisconnect(&g.Disconnect, o); err != nil { + return multierror.Prefix(err, "update ->") + } + } + // If we have a migration strategy, then parse that if o := listVal.Filter("migrate"); len(o.Items) > 0 { if err := parseMigrate(&g.Migrate, o); err != nil { diff --git a/jobspec/parse_test.go b/jobspec/parse_test.go index 518c5a3b6..609c9441a 100644 --- a/jobspec/parse_test.go +++ b/jobspec/parse_test.go @@ -139,8 +139,10 @@ func TestParse(t *testing.T) { }, { - Name: stringToPtr("binsl"), - Count: intToPtr(5), + Name: stringToPtr("binsl"), + Count: intToPtr(5), + StopAfterClientDisconnect: timeToPtr(120 * time.Second), + MaxClientDisconnect: timeToPtr(120 * time.Hour), Constraints: []*api.Constraint{ { LTarget: "kernel.os", @@ -220,8 +222,12 @@ func TestParse(t *testing.T) { }, }, }, - StopAfterClientDisconnect: timeToPtr(120 * time.Second), - MaxClientDisconnect: timeToPtr(120 * time.Hour), + Disconnect: &api.DisconnectStrategy{ + StopOnClientAfter: timeToPtr(120 * time.Second), + LostAfter: timeToPtr(120 * time.Hour), + Replace: boolToPtr(true), + Reconcile: stringToPtr("best_score"), + }, ReschedulePolicy: &api.ReschedulePolicy{ Interval: timeToPtr(12 * time.Hour), Attempts: intToPtr(5), diff --git a/jobspec/test-fixtures/basic.hcl b/jobspec/test-fixtures/basic.hcl index 1176a9d96..ed60d8799 100644 --- a/jobspec/test-fixtures/basic.hcl +++ b/jobspec/test-fixtures/basic.hcl @@ -165,6 +165,13 @@ job "binstore-storagelocker" { stop_after_client_disconnect = "120s" max_client_disconnect = "120h" + disconnect { + lost_after = "120h" + stop_on_client_after = "120s" + replace = true + reconcile = "best_score" + } + task "binstore" { driver = "docker" user = "bob" diff --git a/nomad/heartbeat_test.go b/nomad/heartbeat_test.go index 80e1014c6..641090355 100644 --- a/nomad/heartbeat_test.go +++ b/nomad/heartbeat_test.go @@ -15,6 +15,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" + "github.com/shoenig/test/must" "github.com/stretchr/testify/require" ) @@ -293,6 +294,71 @@ func TestHeartbeat_Server_HeartbeatTTL_Failover(t *testing.T) { func TestHeartbeat_InvalidateHeartbeat_DisconnectedClient(t *testing.T) { ci.Parallel(t) + testCases := []struct { + name string + now time.Time + lostAfterOnDisconnect time.Duration + expectedNodeStatus string + }{ + { + name: "has-pending-reconnects", + now: time.Now().UTC(), + lostAfterOnDisconnect: 5 * time.Second, + expectedNodeStatus: structs.NodeStatusDisconnected, + }, + { + name: "has-expired-reconnects", + lostAfterOnDisconnect: 5 * time.Second, + now: time.Now().UTC().Add(-10 * time.Second), + expectedNodeStatus: structs.NodeStatusDown, + }, + { + name: "has-expired-reconnects-equal-timestamp", + lostAfterOnDisconnect: 5 * time.Second, + now: time.Now().UTC().Add(-5 * time.Second), + expectedNodeStatus: structs.NodeStatusDown, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + s1, cleanupS1 := TestServer(t, nil) + defer cleanupS1() + testutil.WaitForLeader(t, s1.RPC) + + // Create a node + node := mock.Node() + state := s1.fsm.State() + must.NoError(t, state.UpsertNode(structs.MsgTypeTestSetup, 1, node)) + + alloc := mock.Alloc() + alloc.NodeID = node.ID + alloc.Job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + LostAfter: tc.lostAfterOnDisconnect, + } + alloc.ClientStatus = structs.AllocClientStatusUnknown + alloc.AllocStates = []*structs.AllocState{{ + Field: structs.AllocStateFieldClientStatus, + Value: structs.AllocClientStatusUnknown, + Time: tc.now, + }} + + must.NoError(t, state.UpsertAllocs(structs.MsgTypeTestSetup, 2, []*structs.Allocation{alloc})) + + // Trigger status update + s1.invalidateHeartbeat(node.ID) + out, err := state.NodeByID(nil, node.ID) + must.NoError(t, err) + must.Eq(t, tc.expectedNodeStatus, out.Status) + }) + } +} + +// Test using max_client_disconnect, remove after its deprecated in favor +// of Disconnect.LostAfter introduced in 1.8.0. +func TestHeartbeat_InvalidateHeartbeatDisconnectedClient(t *testing.T) { + ci.Parallel(t) + type testCase struct { name string now time.Time @@ -336,7 +402,7 @@ func TestHeartbeat_InvalidateHeartbeat_DisconnectedClient(t *testing.T) { // Create a node node := mock.Node() state := s1.fsm.State() - require.NoError(t, state.UpsertNode(structs.MsgTypeTestSetup, 1, node)) + must.NoError(t, state.UpsertNode(structs.MsgTypeTestSetup, 1, node)) alloc := mock.Alloc() alloc.NodeID = node.ID @@ -347,13 +413,13 @@ func TestHeartbeat_InvalidateHeartbeat_DisconnectedClient(t *testing.T) { Value: structs.AllocClientStatusUnknown, Time: tc.now, }} - require.NoError(t, state.UpsertAllocs(structs.MsgTypeTestSetup, 2, []*structs.Allocation{alloc})) + must.NoError(t, state.UpsertAllocs(structs.MsgTypeTestSetup, 2, []*structs.Allocation{alloc})) // Trigger status update s1.invalidateHeartbeat(node.ID) out, err := state.NodeByID(nil, node.ID) - require.NoError(t, err) - require.Equal(t, tc.expectedNodeStatus, out.Status) + must.NoError(t, err) + must.Eq(t, tc.expectedNodeStatus, out.Status) }) } } diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 799df7b60..ba76dfeb2 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -919,188 +919,220 @@ func TestClientEndpoint_UpdateStatus_Vault_WorkloadIdentity(t *testing.T) { func TestClientEndpoint_UpdateStatus_Reconnect(t *testing.T) { ci.Parallel(t) - // Setup server with tighter heartbeat so we don't have to wait so long - // for nodes to go down. - heartbeatTTL := time.Duration(500*testutil.TestMultiplier()) * time.Millisecond - s, cleanupS := TestServer(t, func(c *Config) { - c.MinHeartbeatTTL = heartbeatTTL - c.HeartbeatGrace = 2 * heartbeatTTL - }) - codec := rpcClient(t, s) - defer cleanupS() - testutil.WaitForLeader(t, s.RPC) + jobVersions := []struct { + name string + jobSpec func(time.Duration) *structs.Job + }{ + // Test using max_client_disconnect, remove after its deprecated in favor + // of Disconnect.LostAfter introduced in 1.8.0. + { + name: "job-with-max-client-disconnect-deprecated", + jobSpec: func(maxClientDisconnect time.Duration) *structs.Job { + job := mock.Job() + job.TaskGroups[0].MaxClientDisconnect = &maxClientDisconnect - // Register node. - node := mock.Node() - reg := &structs.NodeRegisterRequest{ - Node: node, - WriteRequest: structs.WriteRequest{Region: "global"}, - } - var nodeUpdateResp structs.NodeUpdateResponse - err := msgpackrpc.CallWithCodec(codec, "Node.Register", reg, &nodeUpdateResp) - must.NoError(t, err) - - // Start heartbeat. - heartbeat := func(ctx context.Context) { - ticker := time.NewTicker(heartbeatTTL / 2) - defer ticker.Stop() - - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - if t.Failed() { - return + return job + }, + }, + { + name: "job-with-disconnect-block", + jobSpec: func(lostAfter time.Duration) *structs.Job { + job := mock.Job() + job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + LostAfter: lostAfter, } + return job + }, + }, + } - req := &structs.NodeUpdateStatusRequest{ - NodeID: node.ID, - Status: structs.NodeStatusReady, - WriteRequest: structs.WriteRequest{Region: "global"}, + for _, version := range jobVersions { + t.Run(version.name, func(t *testing.T) { + + // Setup server with tighter heartbeat so we don't have to wait so long + // for nodes to go down. + heartbeatTTL := time.Duration(500*testutil.TestMultiplier()) * time.Millisecond + s, cleanupS := TestServer(t, func(c *Config) { + c.MinHeartbeatTTL = heartbeatTTL + c.HeartbeatGrace = 2 * heartbeatTTL + }) + codec := rpcClient(t, s) + defer cleanupS() + testutil.WaitForLeader(t, s.RPC) + + // Register node. + node := mock.Node() + reg := &structs.NodeRegisterRequest{ + Node: node, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + var nodeUpdateResp structs.NodeUpdateResponse + err := msgpackrpc.CallWithCodec(codec, "Node.Register", reg, &nodeUpdateResp) + must.NoError(t, err) + + // Start heartbeat. + heartbeat := func(ctx context.Context) { + ticker := time.NewTicker(heartbeatTTL / 2) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + if t.Failed() { + return + } + + req := &structs.NodeUpdateStatusRequest{ + NodeID: node.ID, + Status: structs.NodeStatusReady, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + var resp structs.NodeUpdateResponse + // Ignore errors since an unexpected failed heartbeat will cause + // the test conditions to fail. + msgpackrpc.CallWithCodec(codec, "Node.UpdateStatus", req, &resp) + } } - var resp structs.NodeUpdateResponse - // Ignore errors since an unexpected failed heartbeat will cause - // the test conditions to fail. - msgpackrpc.CallWithCodec(codec, "Node.UpdateStatus", req, &resp) } - } - } - heartbeatCtx, cancelHeartbeat := context.WithCancel(context.Background()) - defer cancelHeartbeat() - go heartbeat(heartbeatCtx) + heartbeatCtx, cancelHeartbeat := context.WithCancel(context.Background()) + defer cancelHeartbeat() + go heartbeat(heartbeatCtx) - // Wait for node to be ready. - testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusReady) + // Wait for node to be ready. + testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusReady) - // Register job with max_client_disconnect. - job := mock.Job() - job.Constraints = []*structs.Constraint{} - job.TaskGroups[0].Count = 1 - job.TaskGroups[0].MaxClientDisconnect = pointer.Of(time.Hour) - job.TaskGroups[0].Constraints = []*structs.Constraint{} - job.TaskGroups[0].Tasks[0].Driver = "mock_driver" - job.TaskGroups[0].Tasks[0].Config = map[string]interface{}{ - "run_for": "10m", - } - - jobReq := &structs.JobRegisterRequest{ - Job: job, - WriteRequest: structs.WriteRequest{ - Region: "global", - Namespace: job.Namespace, - }, - } - var jobResp structs.JobRegisterResponse - err = msgpackrpc.CallWithCodec(codec, "Job.Register", jobReq, &jobResp) - must.NoError(t, err) - - // Wait for alloc to be pending in the server. - testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ - structs.AllocClientStatusPending: 1, - }) - - // Get allocs that node should run. - allocsReq := &structs.NodeSpecificRequest{ - NodeID: node.ID, - QueryOptions: structs.QueryOptions{ - Region: "global", - }, - } - var allocsResp structs.NodeAllocsResponse - err = msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", allocsReq, &allocsResp) - must.NoError(t, err) - must.Len(t, 1, allocsResp.Allocs) - - // Tell server the alloc is running. - // Save the alloc so we can reuse the request later. - alloc := allocsResp.Allocs[0].Copy() - alloc.ClientStatus = structs.AllocClientStatusRunning - - allocUpdateReq := &structs.AllocUpdateRequest{ - Alloc: []*structs.Allocation{alloc}, - WriteRequest: structs.WriteRequest{ - Region: "global", - }, - } - var resp structs.GenericResponse - err = msgpackrpc.CallWithCodec(codec, "Node.UpdateAlloc", allocUpdateReq, &resp) - must.NoError(t, err) - - // Wait for alloc to be running in the server. - testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ - structs.AllocClientStatusRunning: 1, - }) - - // Stop heartbeat and wait for the client to be disconnected and the alloc - // to be unknown. - cancelHeartbeat() - testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusDisconnected) - testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ - structs.AllocClientStatusUnknown: 1, - }) - - // Restart heartbeat to reconnect node. - heartbeatCtx, cancelHeartbeat = context.WithCancel(context.Background()) - defer cancelHeartbeat() - go heartbeat(heartbeatCtx) - - // Wait a few heartbeats and check that the node is still initializing. - // - // The heartbeat should not update the node to ready until it updates its - // allocs status with the server so the scheduler have the necessary - // information to avoid unnecessary placements. - time.Sleep(3 * heartbeatTTL) - testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusInit) - - // Get allocs that node should run. - // The node should only have one alloc assigned until it updates its allocs - // status with the server. - allocsReq = &structs.NodeSpecificRequest{ - NodeID: node.ID, - QueryOptions: structs.QueryOptions{ - Region: "global", - }, - } - err = msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", allocsReq, &allocsResp) - must.NoError(t, err) - must.Len(t, 1, allocsResp.Allocs) - - // Tell server the alloc is still running. - err = msgpackrpc.CallWithCodec(codec, "Node.UpdateAlloc", allocUpdateReq, &resp) - must.NoError(t, err) - - // The client must end in the same state as before it disconnected: - // - client status is ready. - // - only 1 alloc and the alloc is running. - // - all evals are terminal, so cluster is in a stable state. - testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusReady) - testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ - structs.AllocClientStatusRunning: 1, - }) - testutil.WaitForResult(func() (bool, error) { - state := s.fsm.State() - ws := memdb.NewWatchSet() - evals, err := state.EvalsByJob(ws, job.Namespace, job.ID) - if err != nil { - return false, fmt.Errorf("failed to read evals: %v", err) - } - for _, eval := range evals { - // TODO: remove this check once the disconnect process stops - // leaking a max-disconnect-timeout eval. - // https://github.com/hashicorp/nomad/issues/12809 - if eval.TriggeredBy == structs.EvalTriggerMaxDisconnectTimeout { - continue + // Register job with Disconnect.LostAfter + job := version.jobSpec(time.Hour) + job.Constraints = []*structs.Constraint{} + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].Constraints = []*structs.Constraint{} + job.TaskGroups[0].Tasks[0].Driver = "mock_driver" + job.TaskGroups[0].Tasks[0].Config = map[string]interface{}{ + "run_for": "10m", } - if !eval.TerminalStatus() { - return false, fmt.Errorf("found %s eval", eval.Status) + jobReq := &structs.JobRegisterRequest{ + Job: job, + WriteRequest: structs.WriteRequest{ + Region: "global", + Namespace: job.Namespace, + }, } - } - return true, nil - }, func(err error) { - must.NoError(t, err) - }) + var jobResp structs.JobRegisterResponse + err = msgpackrpc.CallWithCodec(codec, "Job.Register", jobReq, &jobResp) + must.NoError(t, err) + + // Wait for alloc to be pending in the server. + testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ + structs.AllocClientStatusPending: 1, + }) + + // Get allocs that node should run. + allocsReq := &structs.NodeSpecificRequest{ + NodeID: node.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + }, + } + var allocsResp structs.NodeAllocsResponse + err = msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", allocsReq, &allocsResp) + must.NoError(t, err) + must.Len(t, 1, allocsResp.Allocs) + + // Tell server the alloc is running. + // Save the alloc so we can reuse the request later. + alloc := allocsResp.Allocs[0].Copy() + alloc.ClientStatus = structs.AllocClientStatusRunning + + allocUpdateReq := &structs.AllocUpdateRequest{ + Alloc: []*structs.Allocation{alloc}, + WriteRequest: structs.WriteRequest{ + Region: "global", + }, + } + var resp structs.GenericResponse + err = msgpackrpc.CallWithCodec(codec, "Node.UpdateAlloc", allocUpdateReq, &resp) + must.NoError(t, err) + + // Wait for alloc to be running in the server. + testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ + structs.AllocClientStatusRunning: 1, + }) + + // Stop heartbeat and wait for the client to be disconnected and the alloc + // to be unknown. + cancelHeartbeat() + testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusDisconnected) + testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ + structs.AllocClientStatusUnknown: 1, + }) + + // Restart heartbeat to reconnect node. + heartbeatCtx, cancelHeartbeat = context.WithCancel(context.Background()) + defer cancelHeartbeat() + go heartbeat(heartbeatCtx) + + // Wait a few heartbeats and check that the node is still initializing. + // + // The heartbeat should not update the node to ready until it updates its + // allocs status with the server so the scheduler have the necessary + // information to avoid unnecessary placements. + time.Sleep(3 * heartbeatTTL) + testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusInit) + + // Get allocs that node should run. + // The node should only have one alloc assigned until it updates its allocs + // status with the server. + allocsReq = &structs.NodeSpecificRequest{ + NodeID: node.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + }, + } + err = msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", allocsReq, &allocsResp) + must.NoError(t, err) + must.Len(t, 1, allocsResp.Allocs) + + // Tell server the alloc is still running. + err = msgpackrpc.CallWithCodec(codec, "Node.UpdateAlloc", allocUpdateReq, &resp) + must.NoError(t, err) + + // The client must end in the same state as before it disconnected: + // - client status is ready. + // - only 1 alloc and the alloc is running. + // - all evals are terminal, so cluster is in a stable state. + testutil.WaitForClientStatus(t, s.RPC, node.ID, "global", structs.NodeStatusReady) + testutil.WaitForJobAllocStatus(t, s.RPC, job, map[string]int{ + structs.AllocClientStatusRunning: 1, + }) + testutil.WaitForResult(func() (bool, error) { + state := s.fsm.State() + ws := memdb.NewWatchSet() + evals, err := state.EvalsByJob(ws, job.Namespace, job.ID) + if err != nil { + return false, fmt.Errorf("failed to read evals: %v", err) + } + for _, eval := range evals { + // TODO: remove this check once the disconnect process stops + // leaking a max-disconnect-timeout eval. + // https://github.com/hashicorp/nomad/issues/12809 + if eval.TriggeredBy == structs.EvalTriggerMaxDisconnectTimeout { + continue + } + + if !eval.TerminalStatus() { + return false, fmt.Errorf("found %s eval", eval.Status) + } + } + return true, nil + }, func(err error) { + must.NoError(t, err) + }) + + }) + } } func TestClientEndpoint_UpdateStatus_HeartbeatRecovery(t *testing.T) { diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 40237182c..57a177f73 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -801,7 +801,7 @@ func isValidForDisconnectedNode(plan *structs.Plan, nodeID string) bool { // as non reschedulables when lost or if the allocs are being updated to lost. func isValidForDownNode(plan *structs.Plan, nodeID string) bool { for _, alloc := range plan.NodeAllocation[nodeID] { - if !(alloc.ClientStatus == structs.AllocClientStatusUnknown && alloc.PreventRescheduleOnLost()) && + if !(alloc.ClientStatus == structs.AllocClientStatusUnknown && alloc.PreventRescheduleOnDisconnect()) && (alloc.ClientStatus != structs.AllocClientStatusLost) { return false } diff --git a/nomad/structs/alloc_test.go b/nomad/structs/alloc_test.go index 46efb5e99..f38f01917 100644 --- a/nomad/structs/alloc_test.go +++ b/nomad/structs/alloc_test.go @@ -5,13 +5,16 @@ package structs import ( "testing" + "time" - "github.com/stretchr/testify/require" + "github.com/hashicorp/nomad/ci" + "github.com/hashicorp/nomad/helper/pointer" + "github.com/shoenig/test/must" ) func TestAllocServiceRegistrationsRequest_StaleReadSupport(t *testing.T) { req := &AllocServiceRegistrationsRequest{} - require.True(t, req.IsRead()) + must.True(t, req.IsRead()) } func Test_Allocation_ServiceProviderNamespace(t *testing.T) { @@ -139,7 +142,501 @@ func Test_Allocation_ServiceProviderNamespace(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { actualOutput := tc.inputAllocation.ServiceProviderNamespace() - require.Equal(t, tc.expectedOutput, actualOutput) + must.Eq(t, tc.expectedOutput, actualOutput) + }) + } +} + +// Test using stop_after_client_disconnect, remove after its deprecated in favor +// of Disconnect.StopOnClientAfter introduced in 1.8.0. +func TestAllocation_WaitClientStop(t *testing.T) { + ci.Parallel(t) + type testCase struct { + desc string + stop time.Duration + status string + expectedShould bool + expectedRescheduleTime time.Time + } + now := time.Now().UTC() + testCases := []testCase{ + { + desc: "running", + stop: 2 * time.Second, + status: AllocClientStatusRunning, + expectedShould: true, + }, + { + desc: "no stop_after_client_disconnect", + status: AllocClientStatusLost, + expectedShould: false, + }, + { + desc: "stop", + status: AllocClientStatusLost, + stop: 2 * time.Second, + expectedShould: true, + expectedRescheduleTime: now.Add((2 + 5) * time.Second), + }, + } + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + j := testJob() + a := &Allocation{ + ClientStatus: tc.status, + Job: j, + TaskStates: map[string]*TaskState{}, + } + + if tc.status == AllocClientStatusLost { + a.AppendState(AllocStateFieldClientStatus, AllocClientStatusLost) + } + + j.TaskGroups[0].StopAfterClientDisconnect = &tc.stop + a.TaskGroup = j.TaskGroups[0].Name + + must.Eq(t, tc.expectedShould, a.ShouldClientStop()) + + if !tc.expectedShould || tc.status != AllocClientStatusLost { + return + } + + // the reschedTime is close to the expectedRescheduleTime + reschedTime := a.WaitClientStop() + e := reschedTime.Unix() - tc.expectedRescheduleTime.Unix() + must.Less(t, int64(2), e) + }) + } +} + +func TestAllocation_WaitClientStop_Disconnect(t *testing.T) { + ci.Parallel(t) + type testCase struct { + desc string + stop time.Duration + status string + expectedShould bool + expectedRescheduleTime time.Time + } + now := time.Now().UTC() + testCases := []testCase{ + { + desc: "running", + stop: 2 * time.Second, + status: AllocClientStatusRunning, + expectedShould: true, + }, + { + desc: "no stop_after_client_disconnect", + status: AllocClientStatusLost, + expectedShould: false, + }, + { + desc: "stop", + status: AllocClientStatusLost, + stop: 2 * time.Second, + expectedShould: true, + expectedRescheduleTime: now.Add((2 + 5) * time.Second), + }, + } + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + j := testJob() + a := &Allocation{ + ClientStatus: tc.status, + Job: j, + TaskStates: map[string]*TaskState{}, + } + + if tc.status == AllocClientStatusLost { + a.AppendState(AllocStateFieldClientStatus, AllocClientStatusLost) + } + + j.TaskGroups[0].Disconnect = &DisconnectStrategy{ + StopOnClientAfter: &tc.stop, + } + + a.TaskGroup = j.TaskGroups[0].Name + + must.Eq(t, tc.expectedShould, a.ShouldClientStop()) + + if !tc.expectedShould || tc.status != AllocClientStatusLost { + return + } + + // the reschedTime is close to the expectedRescheduleTime + reschedTime := a.WaitClientStop() + e := reschedTime.Unix() - tc.expectedRescheduleTime.Unix() + must.Less(t, int64(2), e) + }) + } +} + +func TestAllocation_Timeout_Disconnect(t *testing.T) { + type testCase struct { + desc string + maxDisconnect time.Duration + } + + testCases := []testCase{ + { + desc: "has lost_after", + maxDisconnect: 30 * time.Second, + }, + { + desc: "zero lost_after", + maxDisconnect: 0 * time.Second, + }, + } + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + j := testJob() + a := &Allocation{ + Job: j, + } + + j.TaskGroups[0].Disconnect = &DisconnectStrategy{ + LostAfter: tc.maxDisconnect, + } + + a.TaskGroup = j.TaskGroups[0].Name + + now := time.Now() + + reschedTime := a.DisconnectTimeout(now) + + if tc.maxDisconnect == 0 { + must.Equal(t, now, reschedTime, must.Sprint("expected to be now")) + } else { + difference := reschedTime.Sub(now) + must.Eq(t, tc.maxDisconnect, difference, must.Sprint("expected durations to be equal")) + } + + }) + } +} + +// Test using max_client_disconnect, remove after its deprecated in favor +// of Disconnect.LostAfter introduced in 1.8.0. +func TestAllocation_DisconnectTimeout(t *testing.T) { + type testCase struct { + desc string + maxDisconnect *time.Duration + } + + testCases := []testCase{ + { + desc: "no max_client_disconnect", + maxDisconnect: nil, + }, + { + desc: "has max_client_disconnect", + maxDisconnect: pointer.Of(30 * time.Second), + }, + { + desc: "zero max_client_disconnect", + maxDisconnect: pointer.Of(0 * time.Second), + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + j := testJob() + a := &Allocation{ + Job: j, + } + + j.TaskGroups[0].MaxClientDisconnect = tc.maxDisconnect + a.TaskGroup = j.TaskGroups[0].Name + + now := time.Now() + + reschedTime := a.DisconnectTimeout(now) + + if tc.maxDisconnect == nil { + must.Equal(t, now, reschedTime, must.Sprint("expected to be now")) + } else { + difference := reschedTime.Sub(now) + must.Eq(t, *tc.maxDisconnect, difference, must.Sprint("expected durations to be equal")) + } + }) + } +} + +// Test using max_client_disconnect, remove after its deprecated in favor +// of Disconnect.LostAfter introduced in 1.8.0. +func TestAllocation_Expired(t *testing.T) { + type testCase struct { + name string + maxDisconnect string + ellapsed int + expected bool + nilJob bool + badTaskGroup bool + mixedUTC bool + noReconnectEvent bool + status string + } + + testCases := []testCase{ + { + name: "has-expired", + maxDisconnect: "5s", + ellapsed: 10, + expected: true, + }, + { + name: "has-not-expired", + maxDisconnect: "5s", + ellapsed: 3, + expected: false, + }, + { + name: "are-equal", + maxDisconnect: "5s", + ellapsed: 5, + expected: true, + }, + { + name: "nil-job", + maxDisconnect: "5s", + ellapsed: 10, + expected: false, + nilJob: true, + }, + { + name: "wrong-status", + maxDisconnect: "5s", + ellapsed: 10, + expected: false, + status: AllocClientStatusRunning, + }, + { + name: "bad-task-group", + maxDisconnect: "", + badTaskGroup: true, + ellapsed: 10, + expected: false, + }, + { + name: "no-max-disconnect", + maxDisconnect: "", + ellapsed: 10, + expected: false, + }, + { + name: "mixed-utc-has-expired", + maxDisconnect: "5s", + ellapsed: 10, + mixedUTC: true, + expected: true, + }, + { + name: "mixed-utc-has-not-expired", + maxDisconnect: "5s", + ellapsed: 3, + mixedUTC: true, + expected: false, + }, + { + name: "no-reconnect-event", + maxDisconnect: "5s", + ellapsed: 2, + expected: false, + noReconnectEvent: true, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + alloc := MockAlloc() + var err error + var maxDisconnect time.Duration + + if tc.maxDisconnect != "" { + maxDisconnect, err = time.ParseDuration(tc.maxDisconnect) + must.NoError(t, err) + alloc.Job.TaskGroups[0].MaxClientDisconnect = &maxDisconnect + } + + if tc.nilJob { + alloc.Job = nil + } + + if tc.badTaskGroup { + alloc.TaskGroup = "bad" + } + + alloc.ClientStatus = AllocClientStatusUnknown + if tc.status != "" { + alloc.ClientStatus = tc.status + } + + alloc.AllocStates = []*AllocState{{ + Field: AllocStateFieldClientStatus, + Value: AllocClientStatusUnknown, + Time: time.Now(), + }} + + must.NoError(t, err) + now := time.Now().UTC() + if tc.mixedUTC { + now = time.Now() + } + + if !tc.noReconnectEvent { + event := NewTaskEvent(TaskClientReconnected) + event.Time = now.UnixNano() + + alloc.TaskStates = map[string]*TaskState{ + "web": { + Events: []*TaskEvent{event}, + }, + } + } + + ellapsedDuration := time.Duration(tc.ellapsed) * time.Second + now = now.Add(ellapsedDuration) + + must.Eq(t, tc.expected, alloc.Expired(now)) + }) + } +} + +func TestAllocation_Expired_Disconnected(t *testing.T) { + type testCase struct { + name string + maxDisconnect string + ellapsed int + expected bool + nilJob bool + badTaskGroup bool + mixedUTC bool + noReconnectEvent bool + status string + } + + testCases := []testCase{ + { + name: "has-expired", + maxDisconnect: "5s", + ellapsed: 10, + expected: true, + }, + { + name: "has-not-expired", + maxDisconnect: "5s", + ellapsed: 3, + expected: false, + }, + { + name: "are-equal", + maxDisconnect: "5s", + ellapsed: 5, + expected: true, + }, + { + name: "nil-job", + maxDisconnect: "5s", + ellapsed: 10, + expected: false, + nilJob: true, + }, + { + name: "wrong-status", + maxDisconnect: "5s", + ellapsed: 10, + expected: false, + status: AllocClientStatusRunning, + }, + { + name: "bad-task-group", + maxDisconnect: "", + badTaskGroup: true, + ellapsed: 10, + expected: false, + }, + { + name: "no-max-disconnect", + maxDisconnect: "", + ellapsed: 10, + expected: false, + }, + { + name: "mixed-utc-has-expired", + maxDisconnect: "5s", + ellapsed: 10, + mixedUTC: true, + expected: true, + }, + { + name: "mixed-utc-has-not-expired", + maxDisconnect: "5s", + ellapsed: 3, + mixedUTC: true, + expected: false, + }, + { + name: "no-reconnect-event", + maxDisconnect: "5s", + ellapsed: 2, + expected: false, + noReconnectEvent: true, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + alloc := MockAlloc() + var err error + var maxDisconnect time.Duration + + if tc.maxDisconnect != "" { + maxDisconnect, err = time.ParseDuration(tc.maxDisconnect) + must.NoError(t, err) + alloc.Job.TaskGroups[0].Disconnect = &DisconnectStrategy{ + LostAfter: maxDisconnect, + } + } + + if tc.nilJob { + alloc.Job = nil + } + + if tc.badTaskGroup { + alloc.TaskGroup = "bad" + } + + alloc.ClientStatus = AllocClientStatusUnknown + if tc.status != "" { + alloc.ClientStatus = tc.status + } + + alloc.AllocStates = []*AllocState{{ + Field: AllocStateFieldClientStatus, + Value: AllocClientStatusUnknown, + Time: time.Now(), + }} + + must.NoError(t, err) + now := time.Now().UTC() + if tc.mixedUTC { + now = time.Now() + } + + if !tc.noReconnectEvent { + event := NewTaskEvent(TaskClientReconnected) + event.Time = now.UnixNano() + + alloc.TaskStates = map[string]*TaskState{ + "web": { + Events: []*TaskEvent{event}, + }, + } + } + + ellapsedDuration := time.Duration(tc.ellapsed) * time.Second + now = now.Add(ellapsedDuration) + + must.Eq(t, tc.expected, alloc.Expired(now)) }) } } diff --git a/nomad/structs/diff.go b/nomad/structs/diff.go index af5a0ea63..5102b2f4e 100644 --- a/nomad/structs/diff.go +++ b/nomad/structs/diff.go @@ -338,6 +338,11 @@ func (tg *TaskGroup) Diff(other *TaskGroup, contextual bool) (*TaskGroupDiff, er diff.Objects = append(diff.Objects, uDiff) } + // Disconnect diff + if disconnectDiff := disconectStrategyDiffs(tg.Disconnect, other.Disconnect, contextual); disconnectDiff != nil { + diff.Objects = append(diff.Objects, disconnectDiff) + } + // Network Resources diff if nDiffs := networkResourceDiffs(tg.Networks, other.Networks, contextual); nDiffs != nil { diff.Objects = append(diff.Objects, nDiffs...) @@ -2397,6 +2402,30 @@ func (d *DNSConfig) Diff(other *DNSConfig, contextual bool) *ObjectDiff { return diff } +func disconectStrategyDiffs(old, new *DisconnectStrategy, contextual bool) *ObjectDiff { + diff := &ObjectDiff{Type: DiffTypeNone, Name: "Disconnect"} + var oldDisconnectFlat, newDisconnectFlat map[string]string + + if reflect.DeepEqual(old, new) { + return nil + } else if old == nil { + diff.Type = DiffTypeAdded + newDisconnectFlat = flatmap.Flatten(new, nil, false) + } else if new == nil { + diff.Type = DiffTypeDeleted + oldDisconnectFlat = flatmap.Flatten(old, nil, false) + } else { + diff.Type = DiffTypeEdited + oldDisconnectFlat = flatmap.Flatten(old, nil, false) + newDisconnectFlat = flatmap.Flatten(new, nil, false) + } + + // Diff the primitive fields. + diff.Fields = fieldDiffs(oldDisconnectFlat, newDisconnectFlat, contextual) + + return diff +} + // networkResourceDiffs diffs a set of NetworkResources. If contextual diff is enabled, // non-changed fields will still be returned. func networkResourceDiffs(old, new []*NetworkResource, contextual bool) []*ObjectDiff { diff --git a/nomad/structs/diff_test.go b/nomad/structs/diff_test.go index 0da2c800c..d123b69f7 100644 --- a/nomad/structs/diff_test.go +++ b/nomad/structs/diff_test.go @@ -2970,6 +2970,209 @@ func TestTaskGroupDiff(t *testing.T) { }, }, }, + { + TestCase: "Disconnect strategy deleted", + Old: &TaskGroup{ + Disconnect: &DisconnectStrategy{ + LostAfter: 1 * time.Second, + Replace: pointer.Of(true), + Reconcile: ReconcileOptionLongestRunning, + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + }, + New: &TaskGroup{}, + Expected: &TaskGroupDiff{ + Type: DiffTypeEdited, + Objects: []*ObjectDiff{ + { + Type: DiffTypeDeleted, + Name: "Disconnect", + Fields: []*FieldDiff{ + { + Type: DiffTypeDeleted, + Name: "LostAfter", + Old: "1000000000", + New: "", + }, + { + Type: DiffTypeDeleted, + Name: "Reconcile", + Old: ReconcileOptionLongestRunning, + New: "", + }, + { + Type: DiffTypeDeleted, + Name: "Replace", + Old: "true", + New: "", + }, + { + Type: DiffTypeDeleted, + Name: "StopOnClientAfter", + Old: "1000000000", + New: "", + }, + }, + }, + }, + }, + }, + { + TestCase: "Disconnect strategy added", + Old: &TaskGroup{}, + New: &TaskGroup{ + Disconnect: &DisconnectStrategy{ + LostAfter: time.Second, + Replace: pointer.Of(true), + Reconcile: ReconcileOptionLongestRunning, + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + }, + Expected: &TaskGroupDiff{ + Type: DiffTypeEdited, + Objects: []*ObjectDiff{ + { + Type: DiffTypeAdded, + Name: "Disconnect", + Fields: []*FieldDiff{ + { + Type: DiffTypeAdded, + Name: "LostAfter", + Old: "", + New: "1000000000", + }, + { + Type: DiffTypeAdded, + Name: "Reconcile", + Old: "", + New: ReconcileOptionLongestRunning, + }, + { + Type: DiffTypeAdded, + Name: "Replace", + Old: "", + New: "true", + }, + { + Type: DiffTypeAdded, + Name: "StopOnClientAfter", + Old: "", + New: "1000000000", + }, + }, + }, + }, + }, + }, + { + TestCase: "Disconnect strategy edited", + Old: &TaskGroup{ + Disconnect: &DisconnectStrategy{ + LostAfter: time.Second, + Replace: pointer.Of(false), + Reconcile: ReconcileOptionLongestRunning, + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + }, + New: &TaskGroup{ + Disconnect: &DisconnectStrategy{ + LostAfter: time.Minute, + Replace: pointer.Of(true), + Reconcile: ReconcileOptionBestScore, + StopOnClientAfter: pointer.Of(1 * time.Minute), + }, + }, + Expected: &TaskGroupDiff{ + Type: DiffTypeEdited, + Objects: []*ObjectDiff{ + { + Type: DiffTypeEdited, + Name: "Disconnect", + Fields: []*FieldDiff{ + { + Type: DiffTypeEdited, + Name: "LostAfter", + Old: "1000000000", + New: "60000000000", + }, + { + Type: DiffTypeEdited, + Name: "Reconcile", + Old: ReconcileOptionLongestRunning, + New: ReconcileOptionBestScore, + }, + { + Type: DiffTypeEdited, + Name: "Replace", + Old: "false", + New: "true", + }, + { + Type: DiffTypeEdited, + Name: "StopOnClientAfter", + Old: "1000000000", + New: "60000000000", + }, + }, + }, + }, + }, + }, + { + TestCase: "Disconnect strategy edited with context", + Contextual: true, + Old: &TaskGroup{ + Disconnect: &DisconnectStrategy{ + LostAfter: time.Second, + Replace: pointer.Of(false), + Reconcile: ReconcileOptionLongestRunning, + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + }, + New: &TaskGroup{ + Disconnect: &DisconnectStrategy{ + LostAfter: time.Minute, + Replace: pointer.Of(true), + Reconcile: ReconcileOptionBestScore, + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + }, + Expected: &TaskGroupDiff{ + Type: DiffTypeEdited, + Objects: []*ObjectDiff{ + { + Type: DiffTypeEdited, + Name: "Disconnect", + Fields: []*FieldDiff{ + { + Type: DiffTypeEdited, + Name: "LostAfter", + Old: "1000000000", + New: "60000000000", + }, + { + Type: DiffTypeEdited, + Name: "Reconcile", + Old: ReconcileOptionLongestRunning, + New: ReconcileOptionBestScore, + }, + { + Type: DiffTypeEdited, + Name: "Replace", + Old: "false", + New: "true", + }, + { + Type: DiffTypeNone, + Name: "StopOnClientAfter", + Old: "1000000000", + New: "1000000000", + }, + }, + }, + }, + }, + }, { TestCase: "EphemeralDisk added", Old: &TaskGroup{}, diff --git a/nomad/structs/group.go b/nomad/structs/group.go new file mode 100644 index 000000000..01108c310 --- /dev/null +++ b/nomad/structs/group.go @@ -0,0 +1,129 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package structs + +import ( + "errors" + "fmt" + "time" + + "github.com/hashicorp/go-multierror" + "github.com/hashicorp/nomad/helper/pointer" +) + +const ( + // ReconcileOption is used to specify the behavior of the reconciliation process + // between the original allocations and the replacements when a previously + // disconnected client comes back online. + ReconcileOptionKeepOriginal = "keep_original" + ReconcileOptionKeepReplacement = "keep_replacement" + ReconcileOptionBestScore = "best_score" + ReconcileOptionLongestRunning = "longest_running" +) + +var ( + // Disconnect strategy validation errors + errStopAndLost = errors.New("Disconnect cannot be configured with both lost_after and stop_after") + errNegativeLostAfter = errors.New("lost_after cannot be a negative duration") + errNegativeStopAfter = errors.New("stop_after cannot be a negative duration") + errStopAfterNonService = errors.New("stop_after can only be used with service or batch job types") + errInvalidReconcile = errors.New("reconcile option is invalid") +) + +func NewDefaultDisconnectStrategy() *DisconnectStrategy { + return &DisconnectStrategy{ + Replace: pointer.Of(true), + Reconcile: ReconcileOptionBestScore, + } +} + +// Disconnect strategy defines how both clients and server should behave in case of +// disconnection between them. +type DisconnectStrategy struct { + // Defines for how long the server will consider the unresponsive node as + // disconnected but alive instead of lost. + LostAfter time.Duration `mapstructure:"lost_after" hcl:"lost_after,optional"` + + // Defines for how long a disconnected client will keep its allocations running. + // This option has a different behavior for nil, the default, and time.Duration(0), + // and needs to be intentionally set/unset. + StopOnClientAfter *time.Duration `mapstructure:"stop_on_client_after" hcl:"stop_on_client_after,optional"` + + // A boolean field used to define if the allocations should be replaced while + // its considered disconnected. + // This option has a different behavior for nil, the default, and false, + // and needs to be intentionally set/unset. It needs to be set to true + // for compatibility. + Replace *bool `mapstructure:"replace" hcl:"replace,optional"` + + // Once the disconnected node starts reporting again, it will define which + // instances to keep: the original allocations, the replacement, the one + // running on the node with the best score as it is currently implemented, + // or the allocation that has been running continuously the longest. + Reconcile string `mapstructure:"reconcile" hcl:"reconcile,optional"` +} + +func (ds *DisconnectStrategy) Validate(job *Job) error { + if ds == nil { + return nil + } + + var mErr *multierror.Error + + if ds.StopOnClientAfter != nil { + if *ds.StopOnClientAfter < 0 { + mErr = multierror.Append(mErr, errNegativeStopAfter) + } + + if job.Type != JobTypeService && job.Type != JobTypeBatch { + mErr = multierror.Append(mErr, errStopAfterNonService) + } + } + + if ds.LostAfter < 0 { + mErr = multierror.Append(mErr, errNegativeLostAfter) + } + + if ds.StopOnClientAfter != nil && ds.LostAfter != 0 { + mErr = multierror.Append(mErr, errStopAndLost) + } + + switch ds.Reconcile { + case "", ReconcileOptionBestScore, ReconcileOptionLongestRunning, + ReconcileOptionKeepOriginal, ReconcileOptionKeepReplacement: + default: + mErr = multierror.Append(mErr, fmt.Errorf("%w: %s", errInvalidReconcile, ds.Reconcile)) + } + + return mErr.ErrorOrNil() +} + +func (ds *DisconnectStrategy) Copy() *DisconnectStrategy { + if ds == nil { + return nil + } + + nds := new(DisconnectStrategy) + *nds = *ds + + if ds.StopOnClientAfter != nil { + nds.StopOnClientAfter = pointer.Of(*ds.StopOnClientAfter) + } + + if ds.Replace != nil { + nds.Replace = pointer.Of(*ds.Replace) + } + + return nds +} + +func (ds *DisconnectStrategy) Canonicalize() { + if ds.Replace == nil { + ds.Replace = pointer.Of(true) + } + + if ds.Reconcile == "" { + ds.Reconcile = ReconcileOptionBestScore + } +} diff --git a/nomad/structs/group_test.go b/nomad/structs/group_test.go new file mode 100644 index 000000000..fda421b02 --- /dev/null +++ b/nomad/structs/group_test.go @@ -0,0 +1,240 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package structs + +import ( + "errors" + "fmt" + "testing" + "time" + + "github.com/hashicorp/nomad/ci" + "github.com/hashicorp/nomad/helper/pointer" + "github.com/shoenig/test/must" +) + +func TestJobConfig_Validate_LostAfter_Disconnect(t *testing.T) { + // Set up a job with an invalid Disconnect.LostAfter value + job := testJob() + timeout := -1 * time.Minute + job.TaskGroups[0].Disconnect = &DisconnectStrategy{ + LostAfter: timeout, + StopOnClientAfter: &timeout, + } + + err := job.Validate() + must.Error(t, err) + err = errors.Unwrap(err) + + must.StrContains(t, err.Error(), errNegativeLostAfter.Error()) + must.StrContains(t, err.Error(), errNegativeStopAfter.Error()) + must.StrContains(t, err.Error(), errStopAndLost.Error()) + + // Modify the job with a valid Disconnect.LostAfter value + timeout = 1 * time.Minute + job.TaskGroups[0].Disconnect = &DisconnectStrategy{ + LostAfter: timeout, + StopOnClientAfter: nil, + } + err = job.Validate() + must.NoError(t, err) +} + +func TestDisconnectStategy_Validate(t *testing.T) { + ci.Parallel(t) + + cases := []struct { + name string + strategy *DisconnectStrategy + jobType string + err error + }{ + { + name: "negative-stop-after", + strategy: &DisconnectStrategy{ + StopOnClientAfter: pointer.Of(-1 * time.Second), + }, + jobType: JobTypeService, + err: errNegativeStopAfter, + }, + { + name: "stop-after-on-system", + strategy: &DisconnectStrategy{ + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + jobType: JobTypeSystem, + err: errStopAfterNonService, + }, + { + name: "negative-lost-after", + strategy: &DisconnectStrategy{ + LostAfter: -1 * time.Second, + }, + jobType: JobTypeService, + err: errNegativeLostAfter, + }, + { + name: "lost-after-and-stop-after-enabled", + strategy: &DisconnectStrategy{ + LostAfter: 1 * time.Second, + StopOnClientAfter: pointer.Of(1 * time.Second), + }, + jobType: JobTypeService, + err: errStopAndLost, + }, + { + name: "invalid-reconcile", + strategy: &DisconnectStrategy{ + LostAfter: 1 * time.Second, + Reconcile: "invalid", + }, + jobType: JobTypeService, + err: errInvalidReconcile, + }, + { + name: "valid-configuration", + strategy: &DisconnectStrategy{ + LostAfter: 1 * time.Second, + Reconcile: ReconcileOptionKeepOriginal, + Replace: pointer.Of(true), + StopOnClientAfter: nil, + }, + jobType: JobTypeService, + err: nil, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + job := testJob() + job.Type = c.jobType + err := c.strategy.Validate(job) + if !errors.Is(err, c.err) { + t.Errorf("expected error %v, got %v", c.err, err) + } + }) + } +} + +func TestJobConfig_Validate_StopAferClient_Disconnect(t *testing.T) { + ci.Parallel(t) + // Setup a system Job with Disconnect.StopOnClientAfter set, which is invalid + job := testJob() + job.Type = JobTypeSystem + stop := 1 * time.Minute + job.TaskGroups[0].Disconnect = &DisconnectStrategy{ + StopOnClientAfter: &stop, + } + + err := job.Validate() + must.Error(t, err) + must.StrContains(t, err.Error(), errStopAfterNonService.Error()) + + // Modify the job to a batch job with an invalid Disconnect.StopOnClientAfter value + job.Type = JobTypeBatch + invalid := -1 * time.Minute + job.TaskGroups[0].Disconnect = &DisconnectStrategy{ + StopOnClientAfter: &invalid, + } + + err = job.Validate() + must.Error(t, err) + must.StrContains(t, err.Error(), errNegativeStopAfter.Error()) + + // Modify the job to a batch job with a valid Disconnect.StopOnClientAfter value + job.Type = JobTypeBatch + job.TaskGroups[0].Disconnect = &DisconnectStrategy{ + StopOnClientAfter: &stop, + } + err = job.Validate() + must.NoError(t, err) +} + +// Test using stop_after_client_disconnect, remove after its deprecated in favor +// of Disconnect.StopOnClientAfter introduced in 1.8.0. +func TestJobConfig_Validate_StopAferClientDisconnect(t *testing.T) { + ci.Parallel(t) + // Setup a system Job with stop_after_client_disconnect set, which is invalid + job := testJob() + job.Type = JobTypeSystem + stop := 1 * time.Minute + job.TaskGroups[0].StopAfterClientDisconnect = &stop + + err := job.Validate() + must.Error(t, err) + must.StrContains(t, err.Error(), "stop_after_client_disconnect can only be set in batch and service jobs") + + // Modify the job to a batch job with an invalid stop_after_client_disconnect value + job.Type = JobTypeBatch + invalid := -1 * time.Minute + job.TaskGroups[0].StopAfterClientDisconnect = &invalid + + err = job.Validate() + must.Error(t, err) + must.StrContains(t, err.Error(), "stop_after_client_disconnect must be a positive value") + + // Modify the job to a batch job with a valid stop_after_client_disconnect value + job.Type = JobTypeBatch + job.TaskGroups[0].StopAfterClientDisconnect = &stop + err = job.Validate() + must.NoError(t, err) +} + +func TestJob_Validate_DisconnectRescheduleLost(t *testing.T) { + ci.Parallel(t) + + // Craft our speciality jobspec to test this particular use-case. + testDisconnectRescheduleLostJob := &Job{ + ID: "gh19644", + Name: "gh19644", + Region: "global", + Type: JobTypeSystem, + TaskGroups: []*TaskGroup{ + { + Name: "cache", + Disconnect: &DisconnectStrategy{ + LostAfter: 1 * time.Hour, + Replace: pointer.Of(false), + }, + Tasks: []*Task{ + { + Name: "redis", + Driver: "docker", + Config: map[string]interface{}{ + "image": "redis:7", + }, + LogConfig: DefaultLogConfig(), + }, + }, + }, + }, + } + + testDisconnectRescheduleLostJob.Canonicalize() + + must.NoError(t, testDisconnectRescheduleLostJob.Validate()) +} + +// Test using max_client_disconnect, remove after its deprecated in favor +// of Disconnect.LostAfter introduced in 1.8.0. +func TestJobConfig_Validate_MaxClientDisconnect(t *testing.T) { + // Set up a job with an invalid max_client_disconnect value + job := testJob() + timeout := -1 * time.Minute + job.TaskGroups[0].MaxClientDisconnect = &timeout + job.TaskGroups[0].StopAfterClientDisconnect = &timeout + + err := job.Validate() + must.Error(t, errors.Unwrap(err)) + fmt.Println("what?", err.Error(), "what?") + must.StrContains(t, err.Error(), "max_client_disconnect cannot be negative") + must.StrContains(t, err.Error(), "Task group cannot be configured with both max_client_disconnect and stop_after_client_disconnect") + + // Modify the job with a valid max_client_disconnect value + timeout = 1 * time.Minute + job.TaskGroups[0].MaxClientDisconnect = &timeout + job.TaskGroups[0].StopAfterClientDisconnect = nil + err = job.Validate() + must.NoError(t, err) +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 971356050..45b73d6d4 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4596,6 +4596,7 @@ func (j *Job) Validate() error { } else if strings.Contains(j.Name, "\000") { mErr.Errors = append(mErr.Errors, errors.New("Job Name contains a null character")) } + if j.Namespace == "" { mErr.Errors = append(mErr.Errors, errors.New("Job must be in a namespace")) } @@ -6613,6 +6614,10 @@ type TaskGroup struct { // RestartPolicy of a TaskGroup RestartPolicy *RestartPolicy + // Disconnect strategy defines how both clients and server should behave in case of + // disconnection between them. + Disconnect *DisconnectStrategy + // Tasks are the collection of tasks that this task group needs to run Tasks []*Task @@ -6654,14 +6659,18 @@ type TaskGroup struct { // StopAfterClientDisconnect, if set, configures the client to stop the task group // after this duration since the last known good heartbeat + // To be deprecated after 1.8.0 infavor of Disconnect.StopOnClientAfter StopAfterClientDisconnect *time.Duration // MaxClientDisconnect, if set, configures the client to allow placed // allocations for tasks in this group to attempt to resume running without a restart. + // To be deprecated after 1.8.0 infavor of Disconnect.LostAfter MaxClientDisconnect *time.Duration // PreventRescheduleOnLost is used to signal that an allocation should not // be rescheduled if its node goes down or is disconnected. + // To be deprecated after 1.8.0 + // To be deprecated after 1.8.0 infavor of Disconnect.Replace PreventRescheduleOnLost bool } @@ -6674,6 +6683,7 @@ func (tg *TaskGroup) Copy() *TaskGroup { ntg.Update = ntg.Update.Copy() ntg.Constraints = CopySliceConstraints(ntg.Constraints) ntg.RestartPolicy = ntg.RestartPolicy.Copy() + ntg.Disconnect = ntg.Disconnect.Copy() ntg.ReschedulePolicy = ntg.ReschedulePolicy.Copy() ntg.Affinities = CopySliceAffinities(ntg.Affinities) ntg.Spreads = CopySliceSpreads(ntg.Spreads) @@ -6743,6 +6753,22 @@ func (tg *TaskGroup) Canonicalize(job *Job) { tg.ReschedulePolicy = NewReschedulePolicy(job.Type) } + if tg.Disconnect != nil { + tg.Disconnect.Canonicalize() + + if tg.MaxClientDisconnect != nil && tg.Disconnect.LostAfter == 0 { + tg.Disconnect.LostAfter = *tg.MaxClientDisconnect + } + + if tg.StopAfterClientDisconnect != nil && tg.Disconnect.StopOnClientAfter == nil { + tg.Disconnect.StopOnClientAfter = tg.StopAfterClientDisconnect + } + + if tg.PreventRescheduleOnLost && tg.Disconnect.Replace == nil { + tg.Disconnect.Replace = pointer.Of(false) + } + } + // Canonicalize Migrate for service jobs if job.Type == JobTypeService && tg.Migrate == nil { tg.Migrate = DefaultMigrateStrategy() @@ -6807,88 +6833,109 @@ func (tg *TaskGroup) filterServices(f func(s *Service) bool) []*Service { // Validate is used to check a task group for reasonable configuration func (tg *TaskGroup) Validate(j *Job) error { - var mErr multierror.Error + var mErr *multierror.Error + if tg.Name == "" { - mErr.Errors = append(mErr.Errors, errors.New("Missing task group name")) + mErr = multierror.Append(mErr, errors.New("Missing task group name")) } else if strings.Contains(tg.Name, "\000") { - mErr.Errors = append(mErr.Errors, errors.New("Task group name contains null character")) + mErr = multierror.Append(mErr, errors.New("Task group name contains null character")) } + if tg.Count < 0 { - mErr.Errors = append(mErr.Errors, errors.New("Task group count can't be negative")) + mErr = multierror.Append(mErr, errors.New("Task group count can't be negative")) } + if len(tg.Tasks) == 0 { // could be a lone consul gateway inserted by the connect mutator - mErr.Errors = append(mErr.Errors, errors.New("Missing tasks for task group")) + mErr = multierror.Append(mErr, errors.New("Missing tasks for task group")) } if tg.MaxClientDisconnect != nil && tg.StopAfterClientDisconnect != nil { - mErr.Errors = append(mErr.Errors, errors.New("Task group cannot be configured with both max_client_disconnect and stop_after_client_disconnect")) + mErr = multierror.Append(mErr, errors.New("Task group cannot be configured with both max_client_disconnect and stop_after_client_disconnect")) } if tg.MaxClientDisconnect != nil && *tg.MaxClientDisconnect < 0 { - mErr.Errors = append(mErr.Errors, errors.New("max_client_disconnect cannot be negative")) + mErr = multierror.Append(mErr, errors.New("max_client_disconnect cannot be negative")) + } + + if tg.Disconnect != nil { + if tg.MaxClientDisconnect != nil && tg.Disconnect.LostAfter > 0 { + return multierror.Append(mErr, errors.New("using both lost_after and max_client_disconnect is not allowed")) + } + + if tg.StopAfterClientDisconnect != nil && tg.Disconnect.StopOnClientAfter != nil { + return multierror.Append(mErr, errors.New("using both stop_after_client_disconnect and stop_on_client_after is not allowed")) + } + + if tg.PreventRescheduleOnLost && tg.Disconnect.Replace != nil { + return multierror.Append(mErr, errors.New("using both prevent_reschedule_on_lost and replace is not allowed")) + } + + if err := tg.Disconnect.Validate(j); err != nil { + mErr = multierror.Append(mErr, err) + } } for idx, constr := range tg.Constraints { if err := constr.Validate(); err != nil { outer := fmt.Errorf("Constraint %d validation failed: %s", idx+1, err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } } if j.Type == JobTypeSystem { if tg.Affinities != nil { - mErr.Errors = append(mErr.Errors, fmt.Errorf("System jobs may not have an affinity block")) + mErr = multierror.Append(mErr, fmt.Errorf("System jobs may not have an affinity block")) } } else { for idx, affinity := range tg.Affinities { if err := affinity.Validate(); err != nil { outer := fmt.Errorf("Affinity %d validation failed: %s", idx+1, err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } } } if tg.RestartPolicy != nil { if err := tg.RestartPolicy.Validate(); err != nil { - mErr.Errors = append(mErr.Errors, err) + mErr = multierror.Append(mErr, err) } } else { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Task Group %v should have a restart policy", tg.Name)) + mErr = multierror.Append(mErr, fmt.Errorf("Task Group %v should have a restart policy", tg.Name)) } if j.Type == JobTypeSystem { if tg.Spreads != nil { - mErr.Errors = append(mErr.Errors, fmt.Errorf("System jobs may not have a spread block")) + mErr = multierror.Append(mErr, fmt.Errorf("System jobs may not have a spread block")) } } else { for idx, spread := range tg.Spreads { if err := spread.Validate(); err != nil { outer := fmt.Errorf("Spread %d validation failed: %s", idx+1, err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } } } if j.Type == JobTypeSystem { if tg.ReschedulePolicy != nil { - mErr.Errors = append(mErr.Errors, fmt.Errorf("System jobs should not have a reschedule policy")) + mErr = multierror.Append(mErr, fmt.Errorf("System jobs should not have a reschedule policy")) } } else { if tg.ReschedulePolicy != nil { if err := tg.ReschedulePolicy.Validate(); err != nil { - mErr.Errors = append(mErr.Errors, err) + mErr = multierror.Append(mErr, err) } } else { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Task Group %v should have a reschedule policy", tg.Name)) + mErr = multierror.Append(mErr, fmt.Errorf("Task Group %v should have a reschedule policy", tg.Name)) } } if tg.EphemeralDisk != nil { if err := tg.EphemeralDisk.Validate(); err != nil { - mErr.Errors = append(mErr.Errors, err) + mErr = multierror.Append(mErr, err) } } else { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Task Group %v should have an ephemeral disk object", tg.Name)) + mErr = multierror.Append(mErr, fmt.Errorf("Task Group %v should have an ephemeral disk object", tg.Name)) } // Validate the update strategy @@ -6896,10 +6943,10 @@ func (tg *TaskGroup) Validate(j *Job) error { switch j.Type { case JobTypeService, JobTypeSystem: default: - mErr.Errors = append(mErr.Errors, fmt.Errorf("Job type %q does not allow update block", j.Type)) + mErr = multierror.Append(mErr, fmt.Errorf("Job type %q does not allow update block", j.Type)) } if err := u.Validate(); err != nil { - mErr.Errors = append(mErr.Errors, err) + mErr = multierror.Append(mErr, err) } } @@ -6908,12 +6955,12 @@ func (tg *TaskGroup) Validate(j *Job) error { case JobTypeService: if tg.Migrate != nil { if err := tg.Migrate.Validate(); err != nil { - mErr.Errors = append(mErr.Errors, err) + mErr = multierror.Append(mErr, err) } } default: if tg.Migrate != nil { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Job type %q does not allow migrate block", j.Type)) + mErr = multierror.Append(mErr, fmt.Errorf("Job type %q does not allow migrate block", j.Type)) } } @@ -6922,9 +6969,9 @@ func (tg *TaskGroup) Validate(j *Job) error { leaderTasks := 0 for idx, task := range tg.Tasks { if task.Name == "" { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Task %d missing name", idx+1)) + mErr = multierror.Append(mErr, fmt.Errorf("Task %d missing name", idx+1)) } else if existing, ok := tasks[task.Name]; ok { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Task %d redefines '%s' from task %d", idx+1, task.Name, existing+1)) + mErr = multierror.Append(mErr, fmt.Errorf("Task %d redefines '%s' from task %d", idx+1, task.Name, existing+1)) } else { tasks[task.Name] = idx } @@ -6935,7 +6982,7 @@ func (tg *TaskGroup) Validate(j *Job) error { } if leaderTasks > 1 { - mErr.Errors = append(mErr.Errors, fmt.Errorf("Only one task may be marked as leader")) + mErr = multierror.Append(mErr, fmt.Errorf("Only one task may be marked as leader")) } // Validate the volume requests @@ -6945,7 +6992,7 @@ func (tg *TaskGroup) Validate(j *Job) error { } for name, volReq := range tg.Volumes { if err := volReq.Validate(j.Type, tg.Count, canaries); err != nil { - mErr.Errors = append(mErr.Errors, fmt.Errorf( + mErr = multierror.Append(mErr, fmt.Errorf( "Task group volume validation for %s failed: %v", name, err)) } } @@ -6953,32 +7000,32 @@ func (tg *TaskGroup) Validate(j *Job) error { // Validate task group and task network resources if err := tg.validateNetworks(); err != nil { outer := fmt.Errorf("Task group network validation failed: %v", err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } // Validate task group and task services if err := tg.validateServices(); err != nil { outer := fmt.Errorf("Task group service validation failed: %v", err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } // Validate group service script-checks if err := tg.validateScriptChecksInGroupServices(); err != nil { outer := fmt.Errorf("Task group service check validation failed: %v", err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } // Validate the scaling policy if err := tg.validateScalingPolicy(j); err != nil { outer := fmt.Errorf("Task group scaling policy validation failed: %v", err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } // Validate the tasks for _, task := range tg.Tasks { if err := task.Validate(j.Type, tg); err != nil { outer := fmt.Errorf("Task %s validation failed: %v", task.Name, err) - mErr.Errors = append(mErr.Errors, outer) + mErr = multierror.Append(mErr, outer) } } @@ -7277,6 +7324,18 @@ func (tg *TaskGroup) Warnings(j *Job) error { } } + if tg.MaxClientDisconnect != nil { + mErr.Errors = append(mErr.Errors, errors.New("MaxClientDisconnect will be deprecated favor of Disconnect.LostAfter")) + } + + if tg.StopAfterClientDisconnect != nil { + mErr.Errors = append(mErr.Errors, errors.New("StopAfterClientDisconnect will be deprecated favor of Disconnect.StopOnClientAfter")) + } + + if tg.PreventRescheduleOnLost { + mErr.Errors = append(mErr.Errors, errors.New("PreventRescheduleOnLost will be deprecated favor of Disconnect.Replace")) + } + // Check for mbits network field if len(tg.Networks) > 0 && tg.Networks[0].MBits > 0 { mErr.Errors = append(mErr.Errors, fmt.Errorf("mbits has been deprecated as of Nomad 0.12.0. Please remove mbits from the network block")) @@ -7343,6 +7402,51 @@ func (tg *TaskGroup) GoString() string { return fmt.Sprintf("*%#v", *tg) } +// Replace is a helper meant to simplify the future depracation of +// PreventRescheduleOnLost in favor of Disconnect.Replace +// introduced in 1.8.0. +func (tg *TaskGroup) Replace() bool { + if tg.PreventRescheduleOnLost { + return false + } + + if tg.Disconnect == nil || tg.Disconnect.Replace == nil { + return true + } + + return *tg.Disconnect.Replace +} + +// GetDisconnectLostTimeout is a helper meant to simplify the future depracation of +// MaxClientDisconnect in favor of Disconnect.LostAfter +// introduced in 1.8.0. +func (tg *TaskGroup) GetDisconnectLostTimeout() time.Duration { + if tg.MaxClientDisconnect != nil { + return *tg.MaxClientDisconnect + } + + if tg.Disconnect != nil { + return tg.Disconnect.LostAfter + } + + return 0 +} + +// GetDisconnectStopTimeout is a helper meant to simplify the future depracation of +// StopAfterClientDisconnect in favor of Disconnect.StopOnClientAfter +// introduced in 1.8.0. +func (tg *TaskGroup) GetDisconnectStopTimeout() *time.Duration { + if tg.StopAfterClientDisconnect != nil { + return tg.StopAfterClientDisconnect + } + + if tg.Disconnect != nil && tg.Disconnect.StopOnClientAfter != nil { + return tg.Disconnect.StopOnClientAfter + } + + return nil +} + // CheckRestart describes if and when a task should be restarted based on // failing health checks. type CheckRestart struct { @@ -10970,12 +11074,14 @@ func (a *Allocation) NextRescheduleTimeByTime(t time.Time) (time.Time, bool) { return a.nextRescheduleTime(t, reschedulePolicy) } -// ShouldClientStop tests an alloc for StopAfterClientDisconnect configuration +// ShouldClientStop tests an alloc for StopAfterClient on the Disconnect configuration func (a *Allocation) ShouldClientStop() bool { tg := a.Job.LookupTaskGroup(a.TaskGroup) + timeout := tg.GetDisconnectStopTimeout() + if tg == nil || - tg.StopAfterClientDisconnect == nil || - *tg.StopAfterClientDisconnect == 0*time.Nanosecond { + timeout == nil || + *timeout == 0*time.Nanosecond { return false } return true @@ -11010,7 +11116,7 @@ func (a *Allocation) WaitClientStop() time.Time { } } - return t.Add(*tg.StopAfterClientDisconnect + kill) + return t.Add(*tg.GetDisconnectStopTimeout() + kill) } // DisconnectTimeout uses the MaxClientDisconnect to compute when the allocation @@ -11022,12 +11128,12 @@ func (a *Allocation) DisconnectTimeout(now time.Time) time.Time { tg := a.Job.LookupTaskGroup(a.TaskGroup) - timeout := tg.MaxClientDisconnect - if timeout == nil { + timeout := tg.GetDisconnectLostTimeout() + if timeout == 0 { return now } - return now.Add(*timeout) + return now.Add(timeout) } // SupportsDisconnectedClients determines whether both the server and the task group @@ -11041,7 +11147,7 @@ func (a *Allocation) SupportsDisconnectedClients(serverSupportsDisconnectedClien if a.Job != nil { tg := a.Job.LookupTaskGroup(a.TaskGroup) if tg != nil { - return tg.MaxClientDisconnect != nil + return tg.GetDisconnectLostTimeout() != 0 } } @@ -11049,12 +11155,14 @@ func (a *Allocation) SupportsDisconnectedClients(serverSupportsDisconnectedClien } // PreventRescheduleOnLost determines if an alloc allows to have a replacement -// when lost. -func (a *Allocation) PreventRescheduleOnLost() bool { +// when Disconnected. +func (a *Allocation) PreventRescheduleOnDisconnect() bool { if a.Job != nil { tg := a.Job.LookupTaskGroup(a.TaskGroup) if tg != nil { - return tg.PreventRescheduleOnLost + return (tg.Disconnect != nil && tg.Disconnect.Replace != nil && + !*tg.Disconnect.Replace) || + tg.PreventRescheduleOnLost } } @@ -11252,7 +11360,7 @@ func (a *Allocation) AllocationDiff() *AllocationDiff { return (*AllocationDiff)(a) } -// Expired determines whether an allocation has exceeded its MaxClientDisonnect +// Expired determines whether an allocation has exceeded its Disconnect.LostAfter // duration relative to the passed time stamp. func (a *Allocation) Expired(now time.Time) bool { if a == nil || a.Job == nil { @@ -11274,11 +11382,12 @@ func (a *Allocation) Expired(now time.Time) bool { return false } - if tg.MaxClientDisconnect == nil && !tg.PreventRescheduleOnLost { + timeout := tg.GetDisconnectLostTimeout() + if timeout == 0 && tg.Replace() { return false } - expiry := lastUnknown.Add(*tg.MaxClientDisconnect) + expiry := lastUnknown.Add(timeout) return expiry.Sub(now) <= 0 } diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index 4c8f77eda..6188c1830 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -470,39 +470,6 @@ func TestJob_ValidateNullChar(t *testing.T) { assert.Error(job.Validate(), "null character in task name should not validate") } -func TestJob_Validate_DisconnectRescheduleLost(t *testing.T) { - ci.Parallel(t) - - // Craft our speciality jobspec to test this particular use-case. - testDisconnectRescheduleLostJob := &Job{ - ID: "gh19644", - Name: "gh19644", - Region: "global", - Type: JobTypeSystem, - TaskGroups: []*TaskGroup{ - { - Name: "cache", - MaxClientDisconnect: pointer.Of(1 * time.Hour), - PreventRescheduleOnLost: true, - Tasks: []*Task{ - { - Name: "redis", - Driver: "docker", - Config: map[string]interface{}{ - "image": "redis:7", - }, - LogConfig: DefaultLogConfig(), - }, - }, - }, - }, - } - - testDisconnectRescheduleLostJob.Canonicalize() - - must.NoError(t, testDisconnectRescheduleLostJob.Validate()) -} - func TestJob_Warnings(t *testing.T) { ci.Parallel(t) @@ -5820,248 +5787,6 @@ func TestAllocation_NextDelay(t *testing.T) { } -func TestAllocation_WaitClientStop(t *testing.T) { - ci.Parallel(t) - type testCase struct { - desc string - stop time.Duration - status string - expectedShould bool - expectedRescheduleTime time.Time - } - now := time.Now().UTC() - testCases := []testCase{ - { - desc: "running", - stop: 2 * time.Second, - status: AllocClientStatusRunning, - expectedShould: true, - }, - { - desc: "no stop_after_client_disconnect", - status: AllocClientStatusLost, - expectedShould: false, - }, - { - desc: "stop", - status: AllocClientStatusLost, - stop: 2 * time.Second, - expectedShould: true, - expectedRescheduleTime: now.Add((2 + 5) * time.Second), - }, - } - for _, tc := range testCases { - t.Run(tc.desc, func(t *testing.T) { - j := testJob() - a := &Allocation{ - ClientStatus: tc.status, - Job: j, - TaskStates: map[string]*TaskState{}, - } - - if tc.status == AllocClientStatusLost { - a.AppendState(AllocStateFieldClientStatus, AllocClientStatusLost) - } - - j.TaskGroups[0].StopAfterClientDisconnect = &tc.stop - a.TaskGroup = j.TaskGroups[0].Name - - require.Equal(t, tc.expectedShould, a.ShouldClientStop()) - - if !tc.expectedShould || tc.status != AllocClientStatusLost { - return - } - - // the reschedTime is close to the expectedRescheduleTime - reschedTime := a.WaitClientStop() - e := reschedTime.Unix() - tc.expectedRescheduleTime.Unix() - require.Less(t, e, int64(2)) - }) - } -} - -func TestAllocation_DisconnectTimeout(t *testing.T) { - type testCase struct { - desc string - maxDisconnect *time.Duration - } - - testCases := []testCase{ - { - desc: "no max_client_disconnect", - maxDisconnect: nil, - }, - { - desc: "has max_client_disconnect", - maxDisconnect: pointer.Of(30 * time.Second), - }, - { - desc: "zero max_client_disconnect", - maxDisconnect: pointer.Of(0 * time.Second), - }, - } - for _, tc := range testCases { - t.Run(tc.desc, func(t *testing.T) { - j := testJob() - a := &Allocation{ - Job: j, - } - - j.TaskGroups[0].MaxClientDisconnect = tc.maxDisconnect - a.TaskGroup = j.TaskGroups[0].Name - - now := time.Now() - - reschedTime := a.DisconnectTimeout(now) - - if tc.maxDisconnect == nil { - require.Equal(t, now, reschedTime, "expected to be now") - } else { - difference := reschedTime.Sub(now) - require.Equal(t, *tc.maxDisconnect, difference, "expected durations to be equal") - } - - }) - } -} - -func TestAllocation_Expired(t *testing.T) { - type testCase struct { - name string - maxDisconnect string - ellapsed int - expected bool - nilJob bool - badTaskGroup bool - mixedUTC bool - noReconnectEvent bool - status string - } - - testCases := []testCase{ - { - name: "has-expired", - maxDisconnect: "5s", - ellapsed: 10, - expected: true, - }, - { - name: "has-not-expired", - maxDisconnect: "5s", - ellapsed: 3, - expected: false, - }, - { - name: "are-equal", - maxDisconnect: "5s", - ellapsed: 5, - expected: true, - }, - { - name: "nil-job", - maxDisconnect: "5s", - ellapsed: 10, - expected: false, - nilJob: true, - }, - { - name: "wrong-status", - maxDisconnect: "5s", - ellapsed: 10, - expected: false, - status: AllocClientStatusRunning, - }, - { - name: "bad-task-group", - maxDisconnect: "", - badTaskGroup: true, - ellapsed: 10, - expected: false, - }, - { - name: "no-max-disconnect", - maxDisconnect: "", - ellapsed: 10, - expected: false, - }, - { - name: "mixed-utc-has-expired", - maxDisconnect: "5s", - ellapsed: 10, - mixedUTC: true, - expected: true, - }, - { - name: "mixed-utc-has-not-expired", - maxDisconnect: "5s", - ellapsed: 3, - mixedUTC: true, - expected: false, - }, - { - name: "no-reconnect-event", - maxDisconnect: "5s", - ellapsed: 2, - expected: false, - noReconnectEvent: true, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - alloc := MockAlloc() - var err error - var maxDisconnect time.Duration - - if tc.maxDisconnect != "" { - maxDisconnect, err = time.ParseDuration(tc.maxDisconnect) - require.NoError(t, err) - alloc.Job.TaskGroups[0].MaxClientDisconnect = &maxDisconnect - } - - if tc.nilJob { - alloc.Job = nil - } - - if tc.badTaskGroup { - alloc.TaskGroup = "bad" - } - - alloc.ClientStatus = AllocClientStatusUnknown - if tc.status != "" { - alloc.ClientStatus = tc.status - } - - alloc.AllocStates = []*AllocState{{ - Field: AllocStateFieldClientStatus, - Value: AllocClientStatusUnknown, - Time: time.Now(), - }} - - require.NoError(t, err) - now := time.Now().UTC() - if tc.mixedUTC { - now = time.Now() - } - - if !tc.noReconnectEvent { - event := NewTaskEvent(TaskClientReconnected) - event.Time = now.UnixNano() - - alloc.TaskStates = map[string]*TaskState{ - "web": { - Events: []*TaskEvent{event}, - }, - } - } - - ellapsedDuration := time.Duration(tc.ellapsed) * time.Second - now = now.Add(ellapsedDuration) - - require.Equal(t, tc.expected, alloc.Expired(now)) - }) - } -} - func TestAllocation_NeedsToReconnect(t *testing.T) { ci.Parallel(t) @@ -6344,54 +6069,6 @@ func TestParameterizedJobConfig_Validate_NonBatch(t *testing.T) { } } -func TestJobConfig_Validate_StopAferClientDisconnect(t *testing.T) { - ci.Parallel(t) - // Setup a system Job with stop_after_client_disconnect set, which is invalid - job := testJob() - job.Type = JobTypeSystem - stop := 1 * time.Minute - job.TaskGroups[0].StopAfterClientDisconnect = &stop - - err := job.Validate() - require.Error(t, err) - require.Contains(t, err.Error(), "stop_after_client_disconnect can only be set in batch and service jobs") - - // Modify the job to a batch job with an invalid stop_after_client_disconnect value - job.Type = JobTypeBatch - invalid := -1 * time.Minute - job.TaskGroups[0].StopAfterClientDisconnect = &invalid - - err = job.Validate() - require.Error(t, err) - require.Contains(t, err.Error(), "stop_after_client_disconnect must be a positive value") - - // Modify the job to a batch job with a valid stop_after_client_disconnect value - job.Type = JobTypeBatch - job.TaskGroups[0].StopAfterClientDisconnect = &stop - err = job.Validate() - require.NoError(t, err) -} - -func TestJobConfig_Validate_MaxClientDisconnect(t *testing.T) { - // Set up a job with an invalid max_client_disconnect value - job := testJob() - timeout := -1 * time.Minute - job.TaskGroups[0].MaxClientDisconnect = &timeout - job.TaskGroups[0].StopAfterClientDisconnect = &timeout - - err := job.Validate() - require.Error(t, err) - require.Contains(t, err.Error(), "max_client_disconnect cannot be negative") - require.Contains(t, err.Error(), "Task group cannot be configured with both max_client_disconnect and stop_after_client_disconnect") - - // Modify the job with a valid max_client_disconnect value - timeout = 1 * time.Minute - job.TaskGroups[0].MaxClientDisconnect = &timeout - job.TaskGroups[0].StopAfterClientDisconnect = nil - err = job.Validate() - require.NoError(t, err) -} - func TestParameterizedJobConfig_Canonicalize(t *testing.T) { ci.Parallel(t) diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index edbca23ac..8a330c308 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -3644,24 +3644,78 @@ func TestServiceSched_StopAfterClientDisconnect(t *testing.T) { ci.Parallel(t) cases := []struct { - stop time.Duration + jobSpecFn func(*structs.Job) when time.Time rescheduled bool }{ + // Test using stop_after_client_disconnect, remove after its deprecated in favor + // of Disconnect.StopOnClientAfter introduced in 1.8.0. { rescheduled: true, + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].StopAfterClientDisconnect = nil + }, }, { - stop: 1 * time.Second, + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].StopAfterClientDisconnect = pointer.Of(1 * time.Second) + }, rescheduled: false, }, { - stop: 1 * time.Second, + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].StopAfterClientDisconnect = pointer.Of(1 * time.Second) + }, when: time.Now().UTC().Add(-10 * time.Second), rescheduled: true, }, { - stop: 1 * time.Second, + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].StopAfterClientDisconnect = pointer.Of(1 * time.Second) + }, + when: time.Now().UTC().Add(10 * time.Minute), + rescheduled: false, + }, + // Tests using the new disconnect block + { + rescheduled: true, + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + StopOnClientAfter: nil, + } + }, + }, + { + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + StopOnClientAfter: pointer.Of(1 * time.Second), + } + }, + rescheduled: false, + }, + { + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + StopOnClientAfter: pointer.Of(1 * time.Second), + } + }, + when: time.Now().UTC().Add(-10 * time.Second), + rescheduled: true, + }, + { + jobSpecFn: func(job *structs.Job) { + job.TaskGroups[0].Count = 1 + job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + StopOnClientAfter: pointer.Of(1 * time.Second), + } + }, when: time.Now().UTC().Add(10 * time.Minute), rescheduled: false, }, @@ -3676,10 +3730,9 @@ func TestServiceSched_StopAfterClientDisconnect(t *testing.T) { node.Status = structs.NodeStatusDown require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node)) - // Job with allocations and stop_after_client_disconnect job := mock.Job() - job.TaskGroups[0].Count = 1 - job.TaskGroups[0].StopAfterClientDisconnect = &tc.stop + + tc.jobSpecFn(job) require.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job)) // Alloc for the running group @@ -7217,129 +7270,160 @@ func TestPropagateTaskState(t *testing.T) { // Tests that a client disconnect generates attribute updates and follow up evals. func TestServiceSched_Client_Disconnect_Creates_Updates_and_Evals(t *testing.T) { - h := NewHarness(t) - count := 1 - maxClientDisconnect := 10 * time.Minute - disconnectedNode, job, unknownAllocs := initNodeAndAllocs(t, h, count, maxClientDisconnect, - structs.NodeStatusReady, structs.AllocClientStatusRunning) + jobVersions := []struct { + name string + jobSpec func(time.Duration) *structs.Job + }{ + // Test using max_client_disconnect, remove after its deprecated in favor + // of Disconnect.LostAfter introduced in 1.8.0. + { + name: "job-with-max-client-disconnect-deprecated", + jobSpec: func(maxClientDisconnect time.Duration) *structs.Job { + job := mock.Job() + job.TaskGroups[0].MaxClientDisconnect = &maxClientDisconnect - // Now disconnect the node - disconnectedNode.Status = structs.NodeStatusDisconnected - require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), disconnectedNode)) - - // Create an evaluation triggered by the disconnect - evals := []*structs.Evaluation{{ - Namespace: structs.DefaultNamespace, - ID: uuid.Generate(), - Priority: 50, - TriggeredBy: structs.EvalTriggerNodeUpdate, - JobID: job.ID, - NodeID: disconnectedNode.ID, - Status: structs.EvalStatusPending, - }} - - nodeStatusUpdateEval := evals[0] - require.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), evals)) - - // Process the evaluation - err := h.Process(NewServiceScheduler, nodeStatusUpdateEval) - require.NoError(t, err) - require.Equal(t, structs.EvalStatusComplete, h.Evals[0].Status) - require.Len(t, h.Plans, 1, "plan") - - // Two followup delayed eval created - require.Len(t, h.CreateEvals, 2) - followUpEval1 := h.CreateEvals[0] - require.Equal(t, nodeStatusUpdateEval.ID, followUpEval1.PreviousEval) - require.Equal(t, "pending", followUpEval1.Status) - require.NotEmpty(t, followUpEval1.WaitUntil) - - followUpEval2 := h.CreateEvals[1] - require.Equal(t, nodeStatusUpdateEval.ID, followUpEval2.PreviousEval) - require.Equal(t, "pending", followUpEval2.Status) - require.NotEmpty(t, followUpEval2.WaitUntil) - - // Insert eval1 in the state store - testutil.WaitForResult(func() (bool, error) { - found, err := h.State.EvalByID(nil, followUpEval1.ID) - if err != nil { - return false, err - } - if found == nil { - return false, nil - } - - require.Equal(t, nodeStatusUpdateEval.ID, found.PreviousEval) - require.Equal(t, "pending", found.Status) - require.NotEmpty(t, found.WaitUntil) - - return true, nil - }, func(err error) { - - require.NoError(t, err) - }) - - // Insert eval2 in the state store - testutil.WaitForResult(func() (bool, error) { - found, err := h.State.EvalByID(nil, followUpEval2.ID) - if err != nil { - return false, err - } - if found == nil { - return false, nil - } - - require.Equal(t, nodeStatusUpdateEval.ID, found.PreviousEval) - require.Equal(t, "pending", found.Status) - require.NotEmpty(t, found.WaitUntil) - - return true, nil - }, func(err error) { - - require.NoError(t, err) - }) - - // Validate that the ClientStatus updates are part of the plan. - require.Len(t, h.Plans[0].NodeAllocation[disconnectedNode.ID], count) - // Pending update should have unknown status. - - for _, nodeAlloc := range h.Plans[0].NodeAllocation[disconnectedNode.ID] { - require.Equal(t, nodeAlloc.ClientStatus, structs.AllocClientStatusUnknown) + return job + }, + }, + { + name: "job-with-disconnect-block", + jobSpec: func(lostAfter time.Duration) *structs.Job { + job := mock.Job() + job.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + LostAfter: lostAfter, + } + return job + }, + }, } - // Simulate that NodeAllocation got processed. - err = h.State.UpsertAllocs(structs.MsgTypeTestSetup, h.NextIndex(), h.Plans[0].NodeAllocation[disconnectedNode.ID]) - require.NoError(t, err, "plan.NodeUpdate") + for _, version := range jobVersions { + t.Run(version.name, func(t *testing.T) { - // Validate that the StateStore Upsert applied the ClientStatus we specified. + h := NewHarness(t) + count := 1 + maxClientDisconnect := 10 * time.Minute - for _, alloc := range unknownAllocs { - alloc, err = h.State.AllocByID(nil, alloc.ID) - require.NoError(t, err) - require.Equal(t, alloc.ClientStatus, structs.AllocClientStatusUnknown) + job := version.jobSpec(maxClientDisconnect) + job.TaskGroups[0].Count = count + require.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job)) - // Allocations have been transitioned to unknown - require.Equal(t, structs.AllocDesiredStatusRun, alloc.DesiredStatus) - require.Equal(t, structs.AllocClientStatusUnknown, alloc.ClientStatus) + disconnectedNode, job, unknownAllocs := initNodeAndAllocs(t, h, job, + structs.NodeStatusReady, structs.AllocClientStatusRunning) + + // Now disconnect the node + disconnectedNode.Status = structs.NodeStatusDisconnected + require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), disconnectedNode)) + + // Create an evaluation triggered by the disconnect + evals := []*structs.Evaluation{{ + Namespace: structs.DefaultNamespace, + ID: uuid.Generate(), + Priority: 50, + TriggeredBy: structs.EvalTriggerNodeUpdate, + JobID: job.ID, + NodeID: disconnectedNode.ID, + Status: structs.EvalStatusPending, + }} + + nodeStatusUpdateEval := evals[0] + require.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), evals)) + + // Process the evaluation + err := h.Process(NewServiceScheduler, nodeStatusUpdateEval) + require.NoError(t, err) + require.Equal(t, structs.EvalStatusComplete, h.Evals[0].Status) + require.Len(t, h.Plans, 1, "plan") + + // Two followup delayed eval created + require.Len(t, h.CreateEvals, 2) + followUpEval1 := h.CreateEvals[0] + require.Equal(t, nodeStatusUpdateEval.ID, followUpEval1.PreviousEval) + require.Equal(t, "pending", followUpEval1.Status) + require.NotEmpty(t, followUpEval1.WaitUntil) + + followUpEval2 := h.CreateEvals[1] + require.Equal(t, nodeStatusUpdateEval.ID, followUpEval2.PreviousEval) + require.Equal(t, "pending", followUpEval2.Status) + require.NotEmpty(t, followUpEval2.WaitUntil) + + // Insert eval1 in the state store + testutil.WaitForResult(func() (bool, error) { + found, err := h.State.EvalByID(nil, followUpEval1.ID) + if err != nil { + return false, err + } + if found == nil { + return false, nil + } + + require.Equal(t, nodeStatusUpdateEval.ID, found.PreviousEval) + require.Equal(t, "pending", found.Status) + require.NotEmpty(t, found.WaitUntil) + + return true, nil + }, func(err error) { + + require.NoError(t, err) + }) + + // Insert eval2 in the state store + testutil.WaitForResult(func() (bool, error) { + found, err := h.State.EvalByID(nil, followUpEval2.ID) + if err != nil { + return false, err + } + if found == nil { + return false, nil + } + + require.Equal(t, nodeStatusUpdateEval.ID, found.PreviousEval) + require.Equal(t, "pending", found.Status) + require.NotEmpty(t, found.WaitUntil) + + return true, nil + }, func(err error) { + + require.NoError(t, err) + }) + + // Validate that the ClientStatus updates are part of the plan. + require.Len(t, h.Plans[0].NodeAllocation[disconnectedNode.ID], count) + // Pending update should have unknown status. + + for _, nodeAlloc := range h.Plans[0].NodeAllocation[disconnectedNode.ID] { + require.Equal(t, nodeAlloc.ClientStatus, structs.AllocClientStatusUnknown) + } + + // Simulate that NodeAllocation got processed. + err = h.State.UpsertAllocs(structs.MsgTypeTestSetup, h.NextIndex(), h.Plans[0].NodeAllocation[disconnectedNode.ID]) + require.NoError(t, err, "plan.NodeUpdate") + + // Validate that the StateStore Upsert applied the ClientStatus we specified. + + for _, alloc := range unknownAllocs { + alloc, err = h.State.AllocByID(nil, alloc.ID) + require.NoError(t, err) + require.Equal(t, alloc.ClientStatus, structs.AllocClientStatusUnknown) + + // Allocations have been transitioned to unknown + require.Equal(t, structs.AllocDesiredStatusRun, alloc.DesiredStatus) + require.Equal(t, structs.AllocClientStatusUnknown, alloc.ClientStatus) + } + }) } } -func initNodeAndAllocs(t *testing.T, h *Harness, allocCount int, - maxClientDisconnect time.Duration, nodeStatus, clientStatus string) (*structs.Node, *structs.Job, []*structs.Allocation) { +func initNodeAndAllocs(t *testing.T, h *Harness, job *structs.Job, + nodeStatus, clientStatus string) (*structs.Node, *structs.Job, []*structs.Allocation) { // Node, which is ready node := mock.Node() node.Status = nodeStatus require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node)) - // Job with allocations and max_client_disconnect - job := mock.Job() - job.TaskGroups[0].Count = allocCount - job.TaskGroups[0].MaxClientDisconnect = &maxClientDisconnect - require.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job)) - - allocs := make([]*structs.Allocation, allocCount) - for i := 0; i < allocCount; i++ { + allocs := make([]*structs.Allocation, job.TaskGroups[0].Count) + for i := 0; i < job.TaskGroups[0].Count; i++ { // Alloc for the running group alloc := mock.Alloc() alloc.Job = job @@ -7354,4 +7438,5 @@ func initNodeAndAllocs(t *testing.T, h *Harness, allocCount int, require.NoError(t, h.State.UpsertAllocs(structs.MsgTypeTestSetup, h.NextIndex(), allocs)) return node, job, allocs + } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 3ab3cc760..50477dab5 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -498,7 +498,7 @@ func (a *allocReconciler) computeGroup(groupName string, all allocSet) bool { // If MaxClientDisconnect is enabled as well as tg.PreventRescheduleOnLost, // the reschedule policy won't be enabled and the lost allocations // wont be rescheduled, and PreventRescheduleOnLost is ignored. - if tg.MaxClientDisconnect != nil { + if tg.GetDisconnectLostTimeout() != 0 { untaintedDisconnecting, rescheduleDisconnecting, laterDisconnecting := disconnecting.filterByRescheduleable(a.batch, true, a.now, a.evalID, a.deployment) rescheduleNow = rescheduleNow.union(rescheduleDisconnecting) diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 3e13276a8..be0afdc4c 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -893,6 +893,8 @@ func TestReconciler_Destructive_ScaleDown(t *testing.T) { // PreventRescheduleOnLost, MaxClientDisconnect and ReschedulePolicy. // Having the 3 configurations enabled is not a valid option and is not // included in the test. +// Test using max_client_disconnect, remove after its deprecated in favor +// of Disconnect.LostAfter introduced in 1.8.0. func TestReconciler_LostNode_PreventRescheduleOnLost(t *testing.T) { disabledReschedulePolicy := &structs.ReschedulePolicy{ Attempts: 0, diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index 18d8af5c4..9943112f8 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -277,7 +277,7 @@ func (a allocSet) filterByTainted(taintedNodes map[string]*structs.Node, serverS } } else { - if alloc.PreventRescheduleOnLost() { + if alloc.PreventRescheduleOnDisconnect() { if alloc.ClientStatus == structs.AllocClientStatusRunning { disconnecting[alloc.ID] = alloc continue @@ -364,7 +364,7 @@ func (a allocSet) filterByTainted(taintedNodes map[string]*structs.Node, serverS // Allocs on terminal nodes that can't be rescheduled need to be treated // differently than those that can. if taintedNode.TerminalStatus() { - if alloc.PreventRescheduleOnLost() { + if alloc.PreventRescheduleOnDisconnect() { if alloc.ClientStatus == structs.AllocClientStatusUnknown { untainted[alloc.ID] = alloc continue @@ -554,7 +554,7 @@ func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { } // delayByStopAfterClientDisconnect returns a delay for any lost allocation that's got a -// stop_after_client_disconnect configured +// disconnect.stop_on_client_after configured func (a allocSet) delayByStopAfterClientDisconnect() (later []*delayedRescheduleInfo) { now := time.Now().UTC() for _, a := range a { diff --git a/scheduler/reconcile_util_test.go b/scheduler/reconcile_util_test.go index 268c963d5..55e2011dc 100644 --- a/scheduler/reconcile_util_test.go +++ b/scheduler/reconcile_util_test.go @@ -4,6 +4,7 @@ package scheduler import ( + "fmt" "testing" "time" @@ -14,9 +15,76 @@ import ( "github.com/shoenig/test/must" ) +func testJob_Deprecated() *structs.Job { + testJob := mock.Job() + testJob.TaskGroups[0].MaxClientDisconnect = pointer.Of(5 * time.Second) + + return testJob +} + +func testJobSingle_Deprecated() *structs.Job { + testJobSingle := mock.Job() + testJobSingle.TaskGroups[0].MaxClientDisconnect = pointer.Of(5 * time.Second) + testJobSingle.TaskGroups[0].PreventRescheduleOnLost = true + + return testJobSingle +} + +func testJobNoMaxDisconnect_Deprecated() *structs.Job { + testJobNoMaxDisconnect := mock.Job() + testJobNoMaxDisconnect.TaskGroups[0].MaxClientDisconnect = nil + + return testJobNoMaxDisconnect +} + +func testJobNoMaxDisconnectSingle_Deprecated() *structs.Job { + testJobNoMaxDisconnectSingle := mock.Job() + testJobNoMaxDisconnectSingle.TaskGroups[0].MaxClientDisconnect = nil + testJobNoMaxDisconnectSingle.TaskGroups[0].PreventRescheduleOnLost = true + + return testJobNoMaxDisconnectSingle +} + +func testJob_Disconnected() *structs.Job { + testJob := mock.Job() + testJob.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + LostAfter: 5 * time.Second, + Replace: pointer.Of(true), + } + + return testJob +} + +func testJobSingle_Disconnected() *structs.Job { + testJobSingle := mock.Job() + testJobSingle.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + LostAfter: 5 * time.Second, + Replace: pointer.Of(true), + } + return testJobSingle +} + +func testJobNoMaxDisconnect_Disconnected() *structs.Job { + testJobNoMaxDisconnect := mock.Job() + testJobNoMaxDisconnect.TaskGroups[0].Disconnect = nil + + return testJobNoMaxDisconnect +} + +func testJobNoMaxDisconnectSingle_Disconnected() *structs.Job { + testJobNoMaxDisconnectSingle := mock.Job() + testJobNoMaxDisconnectSingle.TaskGroups[0].Disconnect = &structs.DisconnectStrategy{ + LostAfter: 0 * time.Second, + Replace: pointer.Of(false), + } + + return testJobNoMaxDisconnectSingle +} + func TestAllocSet_filterByTainted(t *testing.T) { ci.Parallel(t) + now := time.Now() nodes := map[string]*structs.Node{ "draining": { ID: "draining", @@ -37,21 +105,6 @@ func TestAllocSet_filterByTainted(t *testing.T) { }, } - testJob := mock.Job() - testJob.TaskGroups[0].MaxClientDisconnect = pointer.Of(5 * time.Second) - now := time.Now() - - testJobSingle := mock.Job() - testJobSingle.TaskGroups[0].MaxClientDisconnect = pointer.Of(5 * time.Second) - testJobSingle.TaskGroups[0].PreventRescheduleOnLost = true - - testJobNoMaxDisconnect := mock.Job() - testJobNoMaxDisconnect.TaskGroups[0].MaxClientDisconnect = nil - - testJobNoMaxDisconnectSingle := mock.Job() - testJobNoMaxDisconnectSingle.TaskGroups[0].MaxClientDisconnect = nil - testJobNoMaxDisconnectSingle.TaskGroups[0].PreventRescheduleOnLost = true - unknownAllocState := []*structs.AllocState{{ Field: structs.AllocStateFieldClientStatus, Value: structs.AllocClientStatusUnknown, @@ -76,1255 +129,1287 @@ func TestAllocSet_filterByTainted(t *testing.T) { Time: now, }, } + fmt.Println(unknownAllocState, expiredAllocState, reconnectedAllocState) - type testCase struct { - name string - all allocSet - taintedNodes map[string]*structs.Node - supportsDisconnectedClients bool - skipNilNodeTest bool - now time.Time - PreventRescheduleOnLost bool - // expected results - untainted allocSet - migrate allocSet - lost allocSet - disconnecting allocSet - reconnecting allocSet - ignore allocSet - expiring allocSet - } - - testCases := []testCase{ - // These two cases test that we maintain parity with pre-disconnected-clients behavior. + jobDefinitions := []struct { + name string + testJob func() *structs.Job + testJobSingle func() *structs.Job + testJobNoMaxDisconnect func() *structs.Job + testJobNoMaxDisconnectSingle func() *structs.Job + }{ + // Test using max_client_disconnect, remove after its deprecated in favor + // of Disconnect.LostAfter introduced in 1.8.0. { - name: "lost-client", - supportsDisconnectedClients: false, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "untainted1": { - ID: "untainted1", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJob, - NodeID: "normal", - }, - // Terminal allocs are always untainted - "untainted2": { - ID: "untainted2", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJob, - NodeID: "normal", - }, - // Terminal allocs are always untainted, even on draining nodes - "untainted3": { - ID: "untainted3", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJob, - NodeID: "draining", - }, - // Terminal allocs are always untainted, even on lost nodes - "untainted4": { - ID: "untainted4", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJob, - NodeID: "lost", - }, - // Non-terminal alloc with migrate=true should migrate on a draining node - "migrating1": { - ID: "migrating1", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJob, - NodeID: "draining", - }, - // Non-terminal alloc with migrate=true should migrate on an unknown node - "migrating2": { - ID: "migrating2", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJob, - NodeID: "nil", - }, - }, - untainted: allocSet{ - "untainted1": { - ID: "untainted1", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJob, - NodeID: "normal", - }, - // Terminal allocs are always untainted - "untainted2": { - ID: "untainted2", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJob, - NodeID: "normal", - }, - // Terminal allocs are always untainted, even on draining nodes - "untainted3": { - ID: "untainted3", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJob, - NodeID: "draining", - }, - // Terminal allocs are always untainted, even on lost nodes - "untainted4": { - ID: "untainted4", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJob, - NodeID: "lost", - }, - }, - migrate: allocSet{ - // Non-terminal alloc with migrate=true should migrate on a draining node - "migrating1": { - ID: "migrating1", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJob, - NodeID: "draining", - }, - // Non-terminal alloc with migrate=true should migrate on an unknown node - "migrating2": { - ID: "migrating2", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJob, - NodeID: "nil", - }, - }, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, + name: "old_definitions_deprecated", + testJob: testJob_Deprecated, + testJobSingle: testJobSingle_Deprecated, + testJobNoMaxDisconnect: testJobNoMaxDisconnect_Deprecated, + testJobNoMaxDisconnectSingle: testJobNoMaxDisconnectSingle_Deprecated, }, { - name: "lost-client-only-tainted-nodes", - supportsDisconnectedClients: false, - now: time.Now(), - taintedNodes: nodes, - // The logic associated with this test case can only trigger if there - // is a tainted node. Therefore, testing with a nil node set produces - // false failures, so don't perform that test if in this case. - skipNilNodeTest: true, - all: allocSet{ - // Non-terminal allocs on lost nodes are lost - "lost1": { - ID: "lost1", - ClientStatus: structs.AllocClientStatusPending, - Job: testJob, - NodeID: "lost", - }, - // Non-terminal allocs on lost nodes are lost - "lost2": { - ID: "lost2", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJob, - NodeID: "lost", - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{ - // Non-terminal allocs on lost nodes are lost - "lost1": { - ID: "lost1", - ClientStatus: structs.AllocClientStatusPending, - Job: testJob, - NodeID: "lost", - }, - // Non-terminal allocs on lost nodes are lost - "lost2": { - ID: "lost2", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJob, - NodeID: "lost", - }, - }, - expiring: allocSet{}, - }, - { - name: "disco-client-disconnect-unset-max-disconnect", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: true, - all: allocSet{ - // Non-terminal allocs on disconnected nodes w/o max-disconnect are lost - "lost-running": { - ID: "lost-running", - Name: "lost-running", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnect, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{ - "lost-running": { - ID: "lost-running", - Name: "lost-running", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnect, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - expiring: allocSet{}, - }, - // Everything below this line tests the disconnected client mode. - { - name: "disco-client-untainted-reconnect-failed-and-replaced", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "failed-original", - }, - // Failed and replaced allocs on reconnected nodes - // that are still desired-running are reconnected so - // we can stop them - "failed-original": { - ID: "failed-original", - Name: "web", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "failed-original", - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{ - "failed-original": { - ID: "failed-original", - Name: "web", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "disco-client-reconnecting-running-no-replacement", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - // Running allocs on reconnected nodes with no replacement are reconnecting. - // Node.UpdateStatus has already handled syncing client state so this - // should be a noop. - "reconnecting-running-no-replacement": { - ID: "reconnecting-running-no-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{ - "reconnecting-running-no-replacement": { - ID: "reconnecting-running-no-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "disco-client-terminal", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - // Allocs on reconnected nodes that are complete are ignored - "ignored-reconnect-complete": { - ID: "ignored-reconnect-complete", - Name: "ignored-reconnect-complete", - ClientStatus: structs.AllocClientStatusComplete, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - // Failed allocs on reconnected nodes are in reconnecting so that - // they be marked with desired status stop at the server. - "reconnecting-failed": { - ID: "reconnecting-failed", - Name: "reconnecting-failed", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - // Lost allocs on reconnected nodes don't get restarted - "ignored-reconnect-lost": { - ID: "ignored-reconnect-lost", - Name: "ignored-reconnect-lost", - ClientStatus: structs.AllocClientStatusLost, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - // Replacement allocs that are complete are ignored - "ignored-reconnect-complete-replacement": { - ID: "ignored-reconnect-complete-replacement", - Name: "ignored-reconnect-complete", - ClientStatus: structs.AllocClientStatusComplete, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - PreviousAllocation: "untainted-reconnect-complete", - }, - // Replacement allocs on reconnected nodes that are failed are ignored - "ignored-reconnect-failed-replacement": { - ID: "ignored-reconnect-failed-replacement", - Name: "ignored-reconnect-failed", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "reconnecting-failed", - }, - // Lost replacement allocs on reconnected nodes don't get restarted - "ignored-reconnect-lost-replacement": { - ID: "ignored-reconnect-lost-replacement", - Name: "ignored-reconnect-lost", - ClientStatus: structs.AllocClientStatusLost, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - PreviousAllocation: "untainted-reconnect-lost", - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{ - "reconnecting-failed": { - ID: "reconnecting-failed", - Name: "reconnecting-failed", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - ignore: allocSet{ - "ignored-reconnect-complete": { - ID: "ignored-reconnect-complete", - Name: "ignored-reconnect-complete", - ClientStatus: structs.AllocClientStatusComplete, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - "ignored-reconnect-lost": { - ID: "ignored-reconnect-lost", - Name: "ignored-reconnect-lost", - ClientStatus: structs.AllocClientStatusLost, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - "ignored-reconnect-complete-replacement": { - ID: "ignored-reconnect-complete-replacement", - Name: "ignored-reconnect-complete", - ClientStatus: structs.AllocClientStatusComplete, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - PreviousAllocation: "untainted-reconnect-complete", - }, - "ignored-reconnect-failed-replacement": { - ID: "ignored-reconnect-failed-replacement", - Name: "ignored-reconnect-failed", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "reconnecting-failed", - }, - "ignored-reconnect-lost-replacement": { - ID: "ignored-reconnect-lost-replacement", - Name: "ignored-reconnect-lost", - ClientStatus: structs.AllocClientStatusLost, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - PreviousAllocation: "untainted-reconnect-lost", - }, - }, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "disco-client-disconnect", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: true, - all: allocSet{ - // Non-terminal allocs on disconnected nodes are disconnecting - "disconnect-running": { - ID: "disconnect-running", - Name: "disconnect-running", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - }, - // Unknown allocs on disconnected nodes are acknowledge, so they wont be rescheduled again - "untainted-unknown": { - ID: "untainted-unknown", - Name: "untainted-unknown", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - // Unknown allocs on disconnected nodes are lost when expired - "expiring-unknown": { - ID: "expiring-unknown", - Name: "expiring-unknown", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - // Pending allocs on disconnected nodes are lost - "lost-pending": { - ID: "lost-pending", - Name: "lost-pending", - ClientStatus: structs.AllocClientStatusPending, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - }, - // Expired allocs on reconnected clients are lost - "expiring-expired": { - ID: "expiring-expired", - Name: "expiring-expired", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - // Failed and stopped allocs on disconnected nodes are ignored - "ignore-reconnected-failed-stopped": { - ID: "ignore-reconnected-failed-stopped", - Name: "ignore-reconnected-failed-stopped", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{ - // Unknown allocs on disconnected nodes are acknowledge, so they wont be rescheduled again - "untainted-unknown": { - ID: "untainted-unknown", - Name: "untainted-unknown", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{ - "disconnect-running": { - ID: "disconnect-running", - Name: "disconnect-running", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - reconnecting: allocSet{}, - ignore: allocSet{ - "ignore-reconnected-failed-stopped": { - ID: "ignore-reconnected-failed-stopped", - Name: "ignore-reconnected-failed-stopped", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusStop, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - lost: allocSet{ - "lost-pending": { - ID: "lost-pending", - Name: "lost-pending", - ClientStatus: structs.AllocClientStatusPending, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - expiring: allocSet{ - "expiring-unknown": { - ID: "expiring-unknown", - Name: "expiring-unknown", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - "expiring-expired": { - ID: "expiring-expired", - Name: "expiring-expired", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - }, - }, - { - name: "disco-client-reconnect", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - // Expired allocs on reconnected clients are lost - "expired-reconnect": { - ID: "expired-reconnect", - Name: "expired-reconnect", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{ - "expired-reconnect": { - ID: "expired-reconnect", - Name: "expired-reconnect", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - }, - }, - { - name: "disco-client-running-reconnecting-and-replacement-untainted", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "running-original", - }, - // Running and replaced allocs on reconnected nodes are reconnecting - "running-original": { - ID: "running-original", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "running-original", - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{ - "running-original": { - ID: "running-original", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - // After an alloc is reconnected, it should be considered - // "untainted" instead of "reconnecting" to allow changes such as - // job updates to be applied properly. - name: "disco-client-reconnected-alloc-untainted", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "running-reconnected": { - ID: "running-reconnected", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: reconnectedAllocState, - }, - }, - untainted: allocSet{ - "running-reconnected": { - ID: "running-reconnected", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJob, - NodeID: "normal", - TaskGroup: "web", - AllocStates: reconnectedAllocState, - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - // Everything below this line tests the single instance on lost mode. - { - name: "lost-client-single-instance-on", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "untainted1": { - ID: "untainted1", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJobSingle, - NodeID: "normal", - }, - // Terminal allocs are always untainted - "untainted2": { - ID: "untainted2", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJobSingle, - NodeID: "normal", - }, - // Terminal allocs are always untainted, even on draining nodes - "untainted3": { - ID: "untainted3", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJobSingle, - NodeID: "draining", - }, - // Terminal allocs are always untainted, even on lost nodes - "untainted4": { - ID: "untainted4", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJobSingle, - NodeID: "lost", - }, - // Non-terminal alloc with migrate=true should migrate on a draining node - "migrating1": { - ID: "migrating1", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJobSingle, - NodeID: "draining", - }, - // Non-terminal alloc with migrate=true should migrate on an unknown node - "migrating2": { - ID: "migrating2", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJobSingle, - NodeID: "nil", - }, - }, - untainted: allocSet{ - "untainted1": { - ID: "untainted1", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJobSingle, - NodeID: "normal", - }, - // Terminal allocs are always untainted - "untainted2": { - ID: "untainted2", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJobSingle, - NodeID: "normal", - }, - // Terminal allocs are always untainted, even on draining nodes - "untainted3": { - ID: "untainted3", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJobSingle, - NodeID: "draining", - }, - // Terminal allocs are always untainted, even on lost nodes - "untainted4": { - ID: "untainted4", - ClientStatus: structs.AllocClientStatusComplete, - Job: testJobSingle, - NodeID: "lost", - }, - }, - migrate: allocSet{ - // Non-terminal alloc with migrate=true should migrate on a draining node - "migrating1": { - ID: "migrating1", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJobSingle, - NodeID: "draining", - }, - // Non-terminal alloc with migrate=true should migrate on an unknown node - "migrating2": { - ID: "migrating2", - ClientStatus: structs.AllocClientStatusRunning, - DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, - Job: testJobSingle, - NodeID: "nil", - }, - }, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "lost-client-only-tainted-nodes-single-instance-on", - supportsDisconnectedClients: false, - now: time.Now(), - taintedNodes: nodes, - // The logic associated with this test case can only trigger if there - // is a tainted node. Therefore, testing with a nil node set produces - // false failures, so don't perform that test if in this case. - skipNilNodeTest: true, - all: allocSet{ - // Non-terminal allocs on lost nodes are lost - "lost1": { - ID: "lost1", - ClientStatus: structs.AllocClientStatusPending, - Job: testJobSingle, - NodeID: "lost", - }, - // Non-terminal allocs on lost nodes are lost - "lost2": { - ID: "lost2", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJobSingle, - NodeID: "lost", - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{ - // Non-terminal allocs on lost nodes are lost - "lost1": { - ID: "lost1", - ClientStatus: structs.AllocClientStatusPending, - Job: testJobSingle, - NodeID: "lost", - }, - // Non-terminal allocs on lost nodes are lost - "lost2": { - ID: "lost2", - ClientStatus: structs.AllocClientStatusRunning, - Job: testJobSingle, - NodeID: "lost", - }, - }, - expiring: allocSet{}, - }, - { - name: "disco-client-disconnect-unset-max-disconnect-single-instance-on", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: true, - all: allocSet{ - // Non-terminal allocs on disconnected nodes w/o max-disconnect are lost - "disconnecting-running": { - ID: "disconnecting-running", - Name: "disconnecting-running", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{"disconnecting-running": { - ID: "disconnecting-running", - Name: "disconnecting-running", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "disconnected", - TaskGroup: "web", - }}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "disco-client-untainted-reconnect-failed-and-replaced-single-instance-on", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "failed-original", - }, - // Failed and replaced allocs on reconnected nodes - // that are still desired-running are reconnected so - // we can stop them - "failed-original": { - ID: "failed-original", - Name: "web", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "failed-original", - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{ - "failed-original": { - ID: "failed-original", - Name: "web", - ClientStatus: structs.AllocClientStatusFailed, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "disco-client-reconnect-single-instance-on", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - // Expired allocs on reconnected clients are lost - "expired-reconnect": { - ID: "expired-reconnect", - Name: "expired-reconnect", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - }, - untainted: allocSet{}, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{ - "expired-reconnect": { - ID: "expired-reconnect", - Name: "expired-reconnect", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: expiredAllocState, - }, - }, - }, - { - name: "disco-client-running-reconnecting-and-replacement-untainted-single-instance-on", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "running-original", - }, - // Running and replaced allocs on reconnected nodes are reconnecting - "running-original": { - ID: "running-original", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{ - "running-replacement": { - ID: "running-replacement", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - PreviousAllocation: "running-original", - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{ - "running-original": { - ID: "running-original", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - // After an alloc is reconnected, it should be considered - // "untainted" instead of "reconnecting" to allow changes such as - // job updates to be applied properly. - name: "disco-client-reconnected-alloc-untainted", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: false, - all: allocSet{ - "running-reconnected": { - ID: "running-reconnected", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: reconnectedAllocState, - }, - }, - untainted: allocSet{ - "running-reconnected": { - ID: "running-reconnected", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobSingle, - NodeID: "normal", - TaskGroup: "web", - AllocStates: reconnectedAllocState, - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{}, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{}, - expiring: allocSet{}, - }, - { - name: "disco-client-reconnected-alloc-untainted-single-instance-on", - supportsDisconnectedClients: true, - now: time.Now(), - taintedNodes: nodes, - skipNilNodeTest: true, - all: allocSet{ - "untainted-unknown": { - ID: "untainted-unknown", - Name: "web", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - "disconnecting-running": { - ID: "disconnecting-running", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "disconnected", - TaskGroup: "web", - }, - "lost-running": { - ID: "lost-running", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnect, - NodeID: "disconnected", - TaskGroup: "web", - }, - "untainted-unknown-on-down-node": { - ID: "untainted-unknown-on-down-node", - Name: "web", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "down", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - untainted: allocSet{ - "untainted-unknown": { - ID: "untainted-unknown", - Name: "web", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "disconnected", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - "untainted-unknown-on-down-node": { - ID: "untainted-unknown-on-down-node", - Name: "web", - ClientStatus: structs.AllocClientStatusUnknown, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "down", - TaskGroup: "web", - AllocStates: unknownAllocState, - }, - }, - migrate: allocSet{}, - disconnecting: allocSet{ - "disconnecting-running": { - ID: "disconnecting-running", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnectSingle, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - reconnecting: allocSet{}, - ignore: allocSet{}, - lost: allocSet{ - "lost-running": { - ID: "lost-running", - Name: "web", - ClientStatus: structs.AllocClientStatusRunning, - DesiredStatus: structs.AllocDesiredStatusRun, - Job: testJobNoMaxDisconnect, - NodeID: "disconnected", - TaskGroup: "web", - }, - }, - expiring: allocSet{}, + name: "new_definitions_using_disconnect_block", + testJob: testJob_Deprecated, + testJobSingle: testJobSingle_Deprecated, + testJobNoMaxDisconnect: testJobNoMaxDisconnect_Deprecated, + testJobNoMaxDisconnectSingle: testJobNoMaxDisconnectSingle_Deprecated, }, } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - // With tainted nodes - untainted, migrate, lost, disconnecting, reconnecting, ignore, expired := tc.all.filterByTainted(tc.taintedNodes, tc.supportsDisconnectedClients, tc.now) - must.Eq(t, tc.untainted, untainted, must.Sprintf("with-nodes: untainted")) - must.Eq(t, tc.migrate, migrate, must.Sprintf("with-nodes: migrate")) - must.Eq(t, tc.lost, lost, must.Sprintf("with-nodes: lost")) - must.Eq(t, tc.disconnecting, disconnecting, must.Sprintf("with-nodes: disconnecting")) - must.Eq(t, tc.reconnecting, reconnecting, must.Sprintf("with-nodes: reconnecting")) - must.Eq(t, tc.ignore, ignore, must.Sprintf("with-nodes: ignore")) - must.Eq(t, tc.expiring, expired, must.Sprintf("with-nodes: expiring")) + for _, jd := range jobDefinitions { + testJob := jd.testJob() + testJobSingle := jd.testJobSingle() + testJobNoMaxDisconnect := jd.testJobNoMaxDisconnect() + testJobNoMaxDisconnectSingle := jd.testJobNoMaxDisconnectSingle() - if tc.skipNilNodeTest { - return + t.Run(jd.name, func(t *testing.T) { + testCases := []struct { + name string + all allocSet + taintedNodes map[string]*structs.Node + supportsDisconnectedClients bool + skipNilNodeTest bool + now time.Time + PreventRescheduleOnLost bool + // expected results + untainted allocSet + migrate allocSet + lost allocSet + disconnecting allocSet + reconnecting allocSet + ignore allocSet + expiring allocSet + }{ // These two cases test that we maintain parity with pre-disconnected-clients behavior. + { + name: "lost-client", + supportsDisconnectedClients: false, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "untainted1": { + ID: "untainted1", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJob, + NodeID: "normal", + }, + // Terminal allocs are always untainted + "untainted2": { + ID: "untainted2", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJob, + NodeID: "normal", + }, + // Terminal allocs are always untainted, even on draining nodes + "untainted3": { + ID: "untainted3", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJob, + NodeID: "draining", + }, + // Terminal allocs are always untainted, even on lost nodes + "untainted4": { + ID: "untainted4", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJob, + NodeID: "lost", + }, + // Non-terminal alloc with migrate=true should migrate on a draining node + "migrating1": { + ID: "migrating1", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJob, + NodeID: "draining", + }, + // Non-terminal alloc with migrate=true should migrate on an unknown node + "migrating2": { + ID: "migrating2", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJob, + NodeID: "nil", + }, + }, + untainted: allocSet{ + "untainted1": { + ID: "untainted1", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJob, + NodeID: "normal", + }, + // Terminal allocs are always untainted + "untainted2": { + ID: "untainted2", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJob, + NodeID: "normal", + }, + // Terminal allocs are always untainted, even on draining nodes + "untainted3": { + ID: "untainted3", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJob, + NodeID: "draining", + }, + // Terminal allocs are always untainted, even on lost nodes + "untainted4": { + ID: "untainted4", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJob, + NodeID: "lost", + }, + }, + migrate: allocSet{ + // Non-terminal alloc with migrate=true should migrate on a draining node + "migrating1": { + ID: "migrating1", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJob, + NodeID: "draining", + }, + // Non-terminal alloc with migrate=true should migrate on an unknown node + "migrating2": { + ID: "migrating2", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJob, + NodeID: "nil", + }, + }, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "lost-client-only-tainted-nodes", + supportsDisconnectedClients: false, + now: time.Now(), + taintedNodes: nodes, + // The logic associated with this test case can only trigger if there + // is a tainted node. Therefore, testing with a nil node set produces + // false failures, so don't perform that test if in this case. + skipNilNodeTest: true, + all: allocSet{ + // Non-terminal allocs on lost nodes are lost + "lost1": { + ID: "lost1", + ClientStatus: structs.AllocClientStatusPending, + Job: testJob, + NodeID: "lost", + }, + // Non-terminal allocs on lost nodes are lost + "lost2": { + ID: "lost2", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJob, + NodeID: "lost", + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{ + // Non-terminal allocs on lost nodes are lost + "lost1": { + ID: "lost1", + ClientStatus: structs.AllocClientStatusPending, + Job: testJob, + NodeID: "lost", + }, + // Non-terminal allocs on lost nodes are lost + "lost2": { + ID: "lost2", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJob, + NodeID: "lost", + }, + }, + expiring: allocSet{}, + }, + { + name: "disco-client-disconnect-unset-max-disconnect", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: true, + all: allocSet{ + // Non-terminal allocs on disconnected nodes w/o max-disconnect are lost + "lost-running": { + ID: "lost-running", + Name: "lost-running", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnect, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{ + "lost-running": { + ID: "lost-running", + Name: "lost-running", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnect, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + expiring: allocSet{}, + }, + // Everything below this line tests the disconnected client mode. + { + name: "disco-client-untainted-reconnect-failed-and-replaced", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "failed-original", + }, + // Failed and replaced allocs on reconnected nodes + // that are still desired-running are reconnected so + // we can stop them + "failed-original": { + ID: "failed-original", + Name: "web", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "failed-original", + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{ + "failed-original": { + ID: "failed-original", + Name: "web", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "disco-client-reconnecting-running-no-replacement", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + // Running allocs on reconnected nodes with no replacement are reconnecting. + // Node.UpdateStatus has already handled syncing client state so this + // should be a noop. + "reconnecting-running-no-replacement": { + ID: "reconnecting-running-no-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{ + "reconnecting-running-no-replacement": { + ID: "reconnecting-running-no-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "disco-client-terminal", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + // Allocs on reconnected nodes that are complete are ignored + "ignored-reconnect-complete": { + ID: "ignored-reconnect-complete", + Name: "ignored-reconnect-complete", + ClientStatus: structs.AllocClientStatusComplete, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + // Failed allocs on reconnected nodes are in reconnecting so that + // they be marked with desired status stop at the server. + "reconnecting-failed": { + ID: "reconnecting-failed", + Name: "reconnecting-failed", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + // Lost allocs on reconnected nodes don't get restarted + "ignored-reconnect-lost": { + ID: "ignored-reconnect-lost", + Name: "ignored-reconnect-lost", + ClientStatus: structs.AllocClientStatusLost, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + // Replacement allocs that are complete are ignored + "ignored-reconnect-complete-replacement": { + ID: "ignored-reconnect-complete-replacement", + Name: "ignored-reconnect-complete", + ClientStatus: structs.AllocClientStatusComplete, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + PreviousAllocation: "untainted-reconnect-complete", + }, + // Replacement allocs on reconnected nodes that are failed are ignored + "ignored-reconnect-failed-replacement": { + ID: "ignored-reconnect-failed-replacement", + Name: "ignored-reconnect-failed", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "reconnecting-failed", + }, + // Lost replacement allocs on reconnected nodes don't get restarted + "ignored-reconnect-lost-replacement": { + ID: "ignored-reconnect-lost-replacement", + Name: "ignored-reconnect-lost", + ClientStatus: structs.AllocClientStatusLost, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + PreviousAllocation: "untainted-reconnect-lost", + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{ + "reconnecting-failed": { + ID: "reconnecting-failed", + Name: "reconnecting-failed", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + ignore: allocSet{ + "ignored-reconnect-complete": { + ID: "ignored-reconnect-complete", + Name: "ignored-reconnect-complete", + ClientStatus: structs.AllocClientStatusComplete, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + "ignored-reconnect-lost": { + ID: "ignored-reconnect-lost", + Name: "ignored-reconnect-lost", + ClientStatus: structs.AllocClientStatusLost, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + "ignored-reconnect-complete-replacement": { + ID: "ignored-reconnect-complete-replacement", + Name: "ignored-reconnect-complete", + ClientStatus: structs.AllocClientStatusComplete, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + PreviousAllocation: "untainted-reconnect-complete", + }, + "ignored-reconnect-failed-replacement": { + ID: "ignored-reconnect-failed-replacement", + Name: "ignored-reconnect-failed", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "reconnecting-failed", + }, + "ignored-reconnect-lost-replacement": { + ID: "ignored-reconnect-lost-replacement", + Name: "ignored-reconnect-lost", + ClientStatus: structs.AllocClientStatusLost, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + PreviousAllocation: "untainted-reconnect-lost", + }, + }, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "disco-client-disconnect", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: true, + all: allocSet{ + // Non-terminal allocs on disconnected nodes are disconnecting + "disconnect-running": { + ID: "disconnect-running", + Name: "disconnect-running", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + }, + // Unknown allocs on disconnected nodes are acknowledge, so they wont be rescheduled again + "untainted-unknown": { + ID: "untainted-unknown", + Name: "untainted-unknown", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + // Unknown allocs on disconnected nodes are lost when expired + "expiring-unknown": { + ID: "expiring-unknown", + Name: "expiring-unknown", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + // Pending allocs on disconnected nodes are lost + "lost-pending": { + ID: "lost-pending", + Name: "lost-pending", + ClientStatus: structs.AllocClientStatusPending, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + }, + // Expired allocs on reconnected clients are lost + "expiring-expired": { + ID: "expiring-expired", + Name: "expiring-expired", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + // Failed and stopped allocs on disconnected nodes are ignored + "ignore-reconnected-failed-stopped": { + ID: "ignore-reconnected-failed-stopped", + Name: "ignore-reconnected-failed-stopped", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{ + // Unknown allocs on disconnected nodes are acknowledge, so they wont be rescheduled again + "untainted-unknown": { + ID: "untainted-unknown", + Name: "untainted-unknown", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{ + "disconnect-running": { + ID: "disconnect-running", + Name: "disconnect-running", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + reconnecting: allocSet{}, + ignore: allocSet{ + "ignore-reconnected-failed-stopped": { + ID: "ignore-reconnected-failed-stopped", + Name: "ignore-reconnected-failed-stopped", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusStop, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + lost: allocSet{ + "lost-pending": { + ID: "lost-pending", + Name: "lost-pending", + ClientStatus: structs.AllocClientStatusPending, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + expiring: allocSet{ + "expiring-unknown": { + ID: "expiring-unknown", + Name: "expiring-unknown", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + "expiring-expired": { + ID: "expiring-expired", + Name: "expiring-expired", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + }, + }, + { + name: "disco-client-reconnect", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + // Expired allocs on reconnected clients are lost + "expired-reconnect": { + ID: "expired-reconnect", + Name: "expired-reconnect", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{ + "expired-reconnect": { + ID: "expired-reconnect", + Name: "expired-reconnect", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + }, + }, + { + name: "disco-client-running-reconnecting-and-replacement-untainted", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "running-original", + }, + // Running and replaced allocs on reconnected nodes are reconnecting + "running-original": { + ID: "running-original", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "running-original", + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{ + "running-original": { + ID: "running-original", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + // After an alloc is reconnected, it should be considered + // "untainted" instead of "reconnecting" to allow changes such as + // job updates to be applied properly. + name: "disco-client-reconnected-alloc-untainted", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "running-reconnected": { + ID: "running-reconnected", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: reconnectedAllocState, + }, + }, + untainted: allocSet{ + "running-reconnected": { + ID: "running-reconnected", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJob, + NodeID: "normal", + TaskGroup: "web", + AllocStates: reconnectedAllocState, + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + // Everything below this line tests the single instance on lost mode. + { + name: "lost-client-single-instance-on", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "untainted1": { + ID: "untainted1", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJobSingle, + NodeID: "normal", + }, + // Terminal allocs are always untainted + "untainted2": { + ID: "untainted2", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJobSingle, + NodeID: "normal", + }, + // Terminal allocs are always untainted, even on draining nodes + "untainted3": { + ID: "untainted3", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJobSingle, + NodeID: "draining", + }, + // Terminal allocs are always untainted, even on lost nodes + "untainted4": { + ID: "untainted4", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJobSingle, + NodeID: "lost", + }, + // Non-terminal alloc with migrate=true should migrate on a draining node + "migrating1": { + ID: "migrating1", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJobSingle, + NodeID: "draining", + }, + // Non-terminal alloc with migrate=true should migrate on an unknown node + "migrating2": { + ID: "migrating2", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJobSingle, + NodeID: "nil", + }, + }, + untainted: allocSet{ + "untainted1": { + ID: "untainted1", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJobSingle, + NodeID: "normal", + }, + // Terminal allocs are always untainted + "untainted2": { + ID: "untainted2", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJobSingle, + NodeID: "normal", + }, + // Terminal allocs are always untainted, even on draining nodes + "untainted3": { + ID: "untainted3", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJobSingle, + NodeID: "draining", + }, + // Terminal allocs are always untainted, even on lost nodes + "untainted4": { + ID: "untainted4", + ClientStatus: structs.AllocClientStatusComplete, + Job: testJobSingle, + NodeID: "lost", + }, + }, + migrate: allocSet{ + // Non-terminal alloc with migrate=true should migrate on a draining node + "migrating1": { + ID: "migrating1", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJobSingle, + NodeID: "draining", + }, + // Non-terminal alloc with migrate=true should migrate on an unknown node + "migrating2": { + ID: "migrating2", + ClientStatus: structs.AllocClientStatusRunning, + DesiredTransition: structs.DesiredTransition{Migrate: pointer.Of(true)}, + Job: testJobSingle, + NodeID: "nil", + }, + }, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "lost-client-only-tainted-nodes-single-instance-on", + supportsDisconnectedClients: false, + now: time.Now(), + taintedNodes: nodes, + // The logic associated with this test case can only trigger if there + // is a tainted node. Therefore, testing with a nil node set produces + // false failures, so don't perform that test if in this case. + skipNilNodeTest: true, + all: allocSet{ + // Non-terminal allocs on lost nodes are lost + "lost1": { + ID: "lost1", + ClientStatus: structs.AllocClientStatusPending, + Job: testJobSingle, + NodeID: "lost", + }, + // Non-terminal allocs on lost nodes are lost + "lost2": { + ID: "lost2", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJobSingle, + NodeID: "lost", + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{ + // Non-terminal allocs on lost nodes are lost + "lost1": { + ID: "lost1", + ClientStatus: structs.AllocClientStatusPending, + Job: testJobSingle, + NodeID: "lost", + }, + // Non-terminal allocs on lost nodes are lost + "lost2": { + ID: "lost2", + ClientStatus: structs.AllocClientStatusRunning, + Job: testJobSingle, + NodeID: "lost", + }, + }, + expiring: allocSet{}, + }, + { + name: "disco-client-disconnect-unset-max-disconnect-single-instance-on", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: true, + all: allocSet{ + // Non-terminal allocs on disconnected nodes w/o max-disconnect are lost + "disconnecting-running": { + ID: "disconnecting-running", + Name: "disconnecting-running", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{"disconnecting-running": { + ID: "disconnecting-running", + Name: "disconnecting-running", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "disconnected", + TaskGroup: "web", + }}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "disco-client-untainted-reconnect-failed-and-replaced-single-instance-on", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "failed-original", + }, + // Failed and replaced allocs on reconnected nodes + // that are still desired-running are reconnected so + // we can stop them + "failed-original": { + ID: "failed-original", + Name: "web", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "failed-original", + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{ + "failed-original": { + ID: "failed-original", + Name: "web", + ClientStatus: structs.AllocClientStatusFailed, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "disco-client-reconnect-single-instance-on", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + // Expired allocs on reconnected clients are lost + "expired-reconnect": { + ID: "expired-reconnect", + Name: "expired-reconnect", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + }, + untainted: allocSet{}, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{ + "expired-reconnect": { + ID: "expired-reconnect", + Name: "expired-reconnect", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: expiredAllocState, + }, + }, + }, + { + name: "disco-client-running-reconnecting-and-replacement-untainted-single-instance-on", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "running-original", + }, + // Running and replaced allocs on reconnected nodes are reconnecting + "running-original": { + ID: "running-original", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{ + "running-replacement": { + ID: "running-replacement", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + PreviousAllocation: "running-original", + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{ + "running-original": { + ID: "running-original", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + // After an alloc is reconnected, it should be considered + // "untainted" instead of "reconnecting" to allow changes such as + // job updates to be applied properly. + name: "disco-client-reconnected-alloc-untainted", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: false, + all: allocSet{ + "running-reconnected": { + ID: "running-reconnected", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: reconnectedAllocState, + }, + }, + untainted: allocSet{ + "running-reconnected": { + ID: "running-reconnected", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobSingle, + NodeID: "normal", + TaskGroup: "web", + AllocStates: reconnectedAllocState, + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{}, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{}, + expiring: allocSet{}, + }, + { + name: "disco-client-reconnected-alloc-untainted-single-instance-on", + supportsDisconnectedClients: true, + now: time.Now(), + taintedNodes: nodes, + skipNilNodeTest: true, + all: allocSet{ + "untainted-unknown": { + ID: "untainted-unknown", + Name: "web", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + "disconnecting-running": { + ID: "disconnecting-running", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "disconnected", + TaskGroup: "web", + }, + "lost-running": { + ID: "lost-running", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnect, + NodeID: "disconnected", + TaskGroup: "web", + }, + "untainted-unknown-on-down-node": { + ID: "untainted-unknown-on-down-node", + Name: "web", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "down", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + untainted: allocSet{ + "untainted-unknown": { + ID: "untainted-unknown", + Name: "web", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "disconnected", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + "untainted-unknown-on-down-node": { + ID: "untainted-unknown-on-down-node", + Name: "web", + ClientStatus: structs.AllocClientStatusUnknown, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "down", + TaskGroup: "web", + AllocStates: unknownAllocState, + }, + }, + migrate: allocSet{}, + disconnecting: allocSet{ + "disconnecting-running": { + ID: "disconnecting-running", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnectSingle, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + reconnecting: allocSet{}, + ignore: allocSet{}, + lost: allocSet{ + "lost-running": { + ID: "lost-running", + Name: "web", + ClientStatus: structs.AllocClientStatusRunning, + DesiredStatus: structs.AllocDesiredStatusRun, + Job: testJobNoMaxDisconnect, + NodeID: "disconnected", + TaskGroup: "web", + }, + }, + expiring: allocSet{}, + }, } - // Now again with nodes nil - untainted, migrate, lost, disconnecting, reconnecting, ignore, expired = tc.all.filterByTainted(nil, tc.supportsDisconnectedClients, tc.now) - must.Eq(t, tc.untainted, untainted, must.Sprintf("with-nodes: untainted")) - must.Eq(t, tc.migrate, migrate, must.Sprintf("with-nodes: migrate")) - must.Eq(t, tc.lost, lost, must.Sprintf("with-nodes: lost")) - must.Eq(t, tc.disconnecting, disconnecting, must.Sprintf("with-nodes: disconnecting")) - must.Eq(t, tc.reconnecting, reconnecting, must.Sprintf("with-nodes: reconnecting")) - must.Eq(t, tc.ignore, ignore, must.Sprintf("with-nodes: ignore")) - must.Eq(t, tc.ignore, ignore, must.Sprintf("with-nodes: expiring")) - must.Eq(t, tc.expiring, expired, must.Sprintf("with-nodes: expiring")) + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + // With tainted nodes + untainted, migrate, lost, disconnecting, reconnecting, ignore, expired := tc.all.filterByTainted(tc.taintedNodes, tc.supportsDisconnectedClients, tc.now) + must.Eq(t, tc.untainted, untainted, must.Sprintf("with-nodes: untainted")) + must.Eq(t, tc.migrate, migrate, must.Sprintf("with-nodes: migrate")) + must.Eq(t, tc.lost, lost, must.Sprintf("with-nodes: lost")) + must.Eq(t, tc.disconnecting, disconnecting, must.Sprintf("with-nodes: disconnecting")) + must.Eq(t, tc.reconnecting, reconnecting, must.Sprintf("with-nodes: reconnecting")) + must.Eq(t, tc.ignore, ignore, must.Sprintf("with-nodes: ignore")) + must.Eq(t, tc.expiring, expired, must.Sprintf("with-nodes: expiring")) + + if tc.skipNilNodeTest { + return + } + + // Now again with nodes nil + untainted, migrate, lost, disconnecting, reconnecting, ignore, expired = tc.all.filterByTainted(nil, tc.supportsDisconnectedClients, tc.now) + must.Eq(t, tc.untainted, untainted, must.Sprintf("with-nodes: untainted")) + must.Eq(t, tc.migrate, migrate, must.Sprintf("with-nodes: migrate")) + must.Eq(t, tc.lost, lost, must.Sprintf("with-nodes: lost")) + must.Eq(t, tc.disconnecting, disconnecting, must.Sprintf("with-nodes: disconnecting")) + must.Eq(t, tc.reconnecting, reconnecting, must.Sprintf("with-nodes: reconnecting")) + must.Eq(t, tc.ignore, ignore, must.Sprintf("with-nodes: ignore")) + must.Eq(t, tc.ignore, ignore, must.Sprintf("with-nodes: expiring")) + must.Eq(t, tc.expiring, expired, must.Sprintf("with-nodes: expiring")) + }) + } }) } }