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")) + }) + } }) } }