Commit Graph

210 Commits

Author SHA1 Message Date
hc-github-team-nomad-core 90cb544835
scheduler: ensure dup alloc names are fixed before plan submit. (#18873) (#18891)
This change fixes a bug within the generic scheduler which meant
duplicate alloc indexes (names) could be submitted to the plan
applier and written to state. The bug originates from the
placements calculation notion that names of allocations being
replaced are blindly copied to their replacement. This is not
correct in all cases, particularly when dealing with canaries.

The fix updates the alloc name index tracker to include minor
duplicate tracking. This can be used when computing placements to
ensure duplicate are found, and a new name picked before the plan
is submitted. The name index tracking is now passed from the
reconciler to the generic scheduler via the results, so this does
not have to be regenerated, or another data structure used.

Co-authored-by: James Rasell <jrasell@users.noreply.github.com>
2023-10-27 17:04:04 +01:00
Luiz Aoqui e66a7bbefe
core: remove unnecessary call to SetNodes and adds DC downgrade test (#17655) 2023-06-22 13:26:14 -04:00
Luiz Aoqui ac08fc751b
node pools: apply node pool scheduler configuration (#17598) 2023-06-21 20:31:50 -04:00
Tim Gross fbaf4c8b69
node pools: implement support in scheduler (#17443)
Implement scheduler support for node pool:

* When a scheduler is invoked, we get a set of the ready nodes in the DCs that
  are allowed for that job. Extend the filter to include the node pool.
* Ensure that changes to a job's node pool are picked up as destructive
  allocation updates.
* Add `NodesInPool` as a metric to all reporting done by the scheduler.
* Add the node-in-pool the filter to the `Node.Register` RPC so that we don't
  generate spurious evals for nodes in the wrong pool.
2023-06-07 10:39:03 -04:00
hashicorp-copywrite[bot] 005636afa0 [COMPLIANCE] Add Copyright and License Headers 2023-04-10 15:36:59 +00:00
Tim Gross 1335543731
ephemeral disk: `migrate` should imply `sticky` (#16826)
The `ephemeral_disk` block's `migrate` field allows for best-effort migration of
the ephemeral disk data to new nodes. The documentation says the `migrate` field
is only respected if `sticky=true`, but in fact if client ACLs are not set the
data is migrated even if `sticky=false`.

The existing behavior when client ACLs are disabled has existed since the early
implementation, so "fixing" that case now would silently break backwards
compatibility. Additionally, having `migrate` not imply `sticky` seems
nonsensical: it suggests that if we place on a new node we migrate the data but
if we place on the same node, we throw the data away!

Update so that `migrate=true` implies `sticky=true` as follows:

* The failure mode when client ACLs are enabled comes from the server not passing
  along a migration token. Update the server so that the server provides a
  migration token whenever `migrate=true` and not just when `sticky=true` too.
* Update the scheduler so that `migrate` implies `sticky`.
* Update the client so that we check for `migrate || sticky` where appropriate.
* Refactor the E2E tests to move them off the old framework and make the intention
  of the test more clear.
2023-04-07 16:33:45 -04:00
Piotr Kazmierczak 14b53df3b6
renamed stanza to block for consistency with other projects (#15941) 2023-01-30 15:48:43 +01:00
Luiz Aoqui 909435a0e7
scheduler: log stack in case of panic (#15303) 2022-11-17 18:59:33 -05:00
Luiz Aoqui e4c8b59919
Update alloc after reconnect and enforece client heartbeat order (#15068)
* scheduler: allow updates after alloc reconnects

When an allocation reconnects to a cluster the scheduler needs to run
special logic to handle the reconnection, check if a replacement was
create and stop one of them.

If the allocation kept running while the node was disconnected, it will
be reconnected with `ClientStatus: running` and the node will have
`Status: ready`. This combination is the same as the normal steady state
of allocation, where everything is running as expected.

In order to differentiate between the two states (an allocation that is
reconnecting and one that is just running) the scheduler needs an extra
piece of state.

The current implementation uses the presence of a
`TaskClientReconnected` task event to detect when the allocation has
reconnected and thus must go through the reconnection process. But this
event remains even after the allocation is reconnected, causing all
future evals to consider the allocation as still reconnecting.

This commit changes the reconnect logic to use an `AllocState` to
register when the allocation was reconnected. This provides the
following benefits:

  - Only a limited number of task states are kept, and they are used for
    many other events. It's possible that, upon reconnecting, several
    actions are triggered that could cause the `TaskClientReconnected`
    event to be dropped.
  - Task events are set by clients and so their timestamps are subject
    to time skew from servers. This prevents using time to determine if
    an allocation reconnected after a disconnect event.
  - Disconnect events are already stored as `AllocState` and so storing
    reconnects there as well makes it the only source of information
    required.

With the new logic, the reconnection logic is only triggered if the
last `AllocState` is a disconnect event, meaning that the allocation has
not been reconnected yet. After the reconnection is handled, the new
`ClientStatus` is store in `AllocState` allowing future evals to skip
the reconnection logic.

* scheduler: prevent spurious placement on reconnect

When a client reconnects it makes two independent RPC calls:

  - `Node.UpdateStatus` to heartbeat and set its status as `ready`.
  - `Node.UpdateAlloc` to update the status of its allocations.

These two calls can happen in any order, and in case the allocations are
updated before a heartbeat it causes the state to be the same as a node
being disconnected: the node status will still be `disconnected` while
the allocation `ClientStatus` is set to `running`.

The current implementation did not handle this order of events properly,
and the scheduler would create an unnecessary placement since it
considered the allocation was being disconnected. This extra allocation
would then be quickly stopped by the heartbeat eval.

This commit adds a new code path to handle this order of events. If the
node is `disconnected` and the allocation `ClientStatus` is `running`
the scheduler will check if the allocation is actually reconnecting
using its `AllocState` events.

* rpc: only allow alloc updates from `ready` nodes

Clients interact with servers using three main RPC methods:

  - `Node.GetAllocs` reads allocation data from the server and writes it
    to the client.
  - `Node.UpdateAlloc` reads allocation from from the client and writes
    them to the server.
  - `Node.UpdateStatus` writes the client status to the server and is
    used as the heartbeat mechanism.

These three methods are called periodically by the clients and are done
so independently from each other, meaning that there can't be any
assumptions in their ordering.

This can generate scenarios that are hard to reason about and to code
for. For example, when a client misses too many heartbeats it will be
considered `down` or `disconnected` and the allocations it was running
are set to `lost` or `unknown`.

When connectivity is restored the to rest of the cluster, the natural
mental model is to think that the client will heartbeat first and then
update its allocations status into the servers.

But since there's no inherit order in these calls the reverse is just as
possible: the client updates the alloc status and then heartbeats. This
results in a state where allocs are, for example, `running` while the
client is still `disconnected`.

This commit adds a new verification to the `Node.UpdateAlloc` method to
reject updates from nodes that are not `ready`, forcing clients to
heartbeat first. Since this check is done server-side there is no need
to coordinate operations client-side: they can continue sending these
requests independently and alloc update will succeed after the heartbeat
is done.

* chagelog: add entry for #15068

* code review

* client: skip terminal allocations on reconnect

When the client reconnects with the server it synchronizes the state of
its allocations by sending data using the `Node.UpdateAlloc` RPC and
fetching data using the `Node.GetClientAllocs` RPC.

If the data fetch happens before the data write, `unknown` allocations
will still be in this state and would trigger the
`allocRunner.Reconnect` flow.

But when the server `DesiredStatus` for the allocation is `stop` the
client should not reconnect the allocation.

* apply more code review changes

* scheduler: persist changes to reconnected allocs

Reconnected allocs have a new AllocState entry that must be persisted by
the plan applier.

* rpc: read node ID from allocs in UpdateAlloc

The AllocUpdateRequest struct is used in three disjoint use cases:

1. Stripped allocs from clients Node.UpdateAlloc RPC using the Allocs,
   and WriteRequest fields
2. Raft log message using the Allocs, Evals, and WriteRequest fields
3. Plan updates using the AllocsStopped, AllocsUpdated, and Job fields

Adding a new field that would only be used in one these cases (1) made
things more confusing and error prone. While in theory an
AllocUpdateRequest could send allocations from different nodes, in
practice this never actually happens since only clients call this method
with their own allocations.

* scheduler: remove logic to handle exceptional case

This condition could only be hit if, somehow, the allocation status was
set to "running" while the client was "unknown". This was addressed by
enforcing an order in "Node.UpdateStatus" and "Node.UpdateAlloc" RPC
calls, so this scenario is not expected to happen.

Adding unnecessary code to the scheduler makes it harder to read and
reason about it.

* more code review

* remove another unused test
2022-11-04 16:25:11 -04:00
Derek Strickland 0891218ee9
system_scheduler: support disconnected clients (#12555)
* structs: Add helper method for checking if alloc is configured to disconnect
* system_scheduler: Add support for disconnected clients
2022-04-15 09:31:32 -04:00
Tim Gross 9e53906782
set minimum version for disconnected client mode to 1.3.0 (#12530) 2022-04-08 16:48:37 -04:00
Derek Strickland bd719bc7b8 reconciler: 2 phase reconnects and tests (#12333)
* structs: Add alloc.Expired & alloc.Reconnected functions. Add Reconnect eval trigger by.

* node_endpoint: Emit new eval for reconnecting unknown allocs.

* filterByTainted: handle 2 phase commit filtering rules.

* reconciler: Append AllocState on disconnect. Logic updates from testing and 2 phase reconnects.

* allocs: Set reconnect timestamp. Destroy if not DesiredStatusRun. Watch for unknown status.
2022-04-05 17:13:10 -04:00
Derek Strickland cc854d842e Add description for allocs stopped due to reconnect (#12270) 2022-04-05 17:12:23 -04:00
Derek Strickland fd04a24ac7 disconnected clients: ensure servers meet minimum required version (#12202)
* planner: expose ServerMeetsMinimumVersion via Planner interface
* filterByTainted: add flag indicating disconnect support
* allocReconciler: accept and pass disconnect support flag
* tests: update dependent tests
2022-04-05 17:12:23 -04:00
Derek Strickland 3cbd76ea9d disconnected clients: Add reconnect task event (#12133)
* Add TaskClientReconnectedEvent constant
* Add allocRunner.Reconnect function to manage task state manually
* Removes server-side push
2022-04-05 17:12:23 -04:00
Derek Strickland b128769e19 reconciler: support disconnected clients (#12058)
* Add merge helper for string maps
* structs: add statuses, MaxClientDisconnect, and helper funcs
* taintedNodes: Include disconnected nodes
* upsertAllocsImpl: don't use existing ClientStatus when upserting unknown
* allocSet: update filterByTainted and add delayByMaxClientDisconnect
* allocReconciler: support disconnecting and reconnecting allocs
* GenericScheduler: upsert unknown and queue reconnecting

Co-authored-by: Tim Gross <tgross@hashicorp.com>
2022-04-05 17:10:37 -04:00
Derek Strickland e1f9c442e1
reconciler: refactor `computeGroup` (#12033)
The allocReconciler's computeGroup function contained a significant amount of inline logic that was difficult to understand the intent of. This commit extracts inline logic into the following intention revealing subroutines. It also includes updates to the function internals also aimed at improving maintainability and renames some existing functions for the same purpose. New or renamed functions include.

Renamed functions

- handleGroupCanaries -> cancelUnneededCanaries
- handleDelayedLost -> createLostLaterEvals
- handeDelayedReschedules -> createRescheduleLaterEvals

New functions

- filterAndStopAll
- initializeDeploymentState
- requiresCanaries
- computeCanaries
- computeUnderProvisionedBy
- computeReplacements
- computeDestructiveUpdates
- computeMigrations
- createDeployment
- isDeploymentComplete
2022-02-10 16:24:51 -05:00
Tim Gross 464026c87b
scheduler: recover from panic (#12009)
If processing a specific evaluation causes the scheduler (and
therefore the entire server) to panic, that evaluation will never
get a chance to be nack'd and cleared from the state store. It will
get dequeued by another scheduler, causing that server to panic, and
so forth until all servers are in a panic loop. This prevents the
operator from intervening to remove the evaluation or update the
state.

Recover the goroutine from the top-level `Process` methods for each
scheduler so that this condition can be detected without panicking the
server process. This will lead to a loop of recovering the scheduler
goroutine until the eval can be removed or nack'd, but that's much
better than taking a downtime.
2022-02-07 11:47:53 -05:00
Luiz Aoqui b1753d0568
scheduler: detect and log unexpected scheduling collisions (#11793) 2022-01-14 20:09:14 -05:00
James Rasell 751c8217d1
core: allow setting and propagation of eval priority on job de/registration (#11532)
This change modifies the Nomad job register and deregister RPCs to
accept an updated option set which includes eval priority. This
param is optional and override the use of the job priority to set
the eval priority.

In order to ensure all evaluations as a result of the request use
the same eval priority, the priority is shared to the
allocReconciler and deploymentWatcher. This creates a new
distinction between eval priority and job priority.

The Nomad agent HTTP API has been modified to allow setting the
eval priority on job update and delete. To keep consistency with
the current v1 API, job update accepts this as a payload param;
job delete accepts this as a query param.

Any user supplied value is validated within the agent HTTP handler
removing the need to pass invalid requests to the server.

The register and deregister opts functions now all for setting
the eval priority on requests.

The change includes a small change to the DeregisterOpts function
which handles nil opts. This brings the function inline with the
RegisterOpts.
2021-11-23 09:23:31 +01:00
Mahmood Ali e06ff1d613
scheduler: stop allocs in unrelated nodes (#11391)
The system scheduler should leave allocs on draining nodes as-is, but
stop node stop allocs on nodes that are no longer part of the job
datacenters.

Previously, the scheduler did not make the distinction and left system
job allocs intact if they are already running.

I've added a failing test first, which you can see in https://app.circleci.com/jobs/github/hashicorp/nomad/179661 .

Fixes https://github.com/hashicorp/nomad/issues/11373
2021-10-27 07:04:13 -07:00
Seth Hoenig 3371214431 core: implement system batch scheduler
This PR implements a new "System Batch" scheduler type. Jobs can
make use of this new scheduler by setting their type to 'sysbatch'.

Like the name implies, sysbatch can be thought of as a hybrid between
system and batch jobs - it is for running short lived jobs intended to
run on every compatible node in the cluster.

As with batch jobs, sysbatch jobs can also be periodic and/or parameterized
dispatch jobs. A sysbatch job is considered complete when it has been run
on all compatible nodes until reaching a terminal state (success or failed
on retries).

Feasibility and preemption are governed the same as with system jobs. In
this PR, the update stanza is not yet supported. The update stanza is sill
limited in functionality for the underlying system scheduler, and is
not useful yet for sysbatch jobs. Further work in #4740 will improve
support for the update stanza and deployments.

Closes #2527
2021-08-03 10:30:47 -04:00
Michael Schurter 547a718ef6
Merge pull request #10248 from hashicorp/f-remotetask-2021
core: propagate remote task handles
2021-04-30 08:57:26 -07:00
Michael Schurter 641eb1dc1a clarify docs from pr comments 2021-04-30 08:31:31 -07:00
Luiz Aoqui f1b9055d21
Add metrics for blocked eval resources (#10454)
* add metrics for blocked eval resources

* docs: add new blocked_evals metrics

* fix to call `pruneStats` instead of `stats.prune` directly
2021-04-29 15:03:45 -04:00
Michael Schurter e62795798d core: propagate remote task handles
Add a new driver capability: RemoteTasks.

When a task is run by a driver with RemoteTasks set, its TaskHandle will
be propagated to the server in its allocation's TaskState. If the task
is replaced due to a down node or draining, its TaskHandle will be
propagated to its replacement allocation.

This allows tasks to be scheduled in remote systems whose lifecycles are
disconnected from the Nomad node's lifecycle.

See https://github.com/hashicorp/nomad-driver-ecs for an example ECS
remote task driver.
2021-04-27 15:07:03 -07:00
Tim Gross fa25e048b2
CSI: unique volume per allocation
Add a `PerAlloc` field to volume requests that directs the scheduler to test
feasibility for volumes with a source ID that includes the allocation index
suffix (ex. `[0]`), rather than the exact source ID.

Read the `PerAlloc` field when making the volume claim at the client to
determine if the allocation index suffix (ex. `[0]`) should be added to the
volume source ID.
2021-03-18 15:35:11 -04:00
Kris Hicks 0a3a748053
Add gosimple linter (#9590) 2020-12-09 11:05:18 -08:00
Kris Hicks 62972cc839
scheduler: Fix always-false sort func (#9547)
Co-authored-by: Mahmood Ali <mahmood@hashicorp.com>
2020-12-08 09:57:47 -08:00
Michael Schurter 8ccbd92cb6 api: add field filters to /v1/{allocations,nodes}
Fixes #9017

The ?resources=true query parameter includes resources in the object
stub listings. Specifically:

- For `/v1/nodes?resources=true` both the `NodeResources` and
  `ReservedResources` field are included.
- For `/v1/allocations?resources=true` the `AllocatedResources` field is
  included.

The ?task_states=false query parameter removes TaskStates from
/v1/allocations responses. (By default TaskStates are included.)
2020-10-14 10:35:22 -07:00
Mahmood Ali 8837c9a45d Handle migration of non-deployment jobs
This handles the case where a job when from no-deployment to deployment
with canaries.

Consider a case where a `max_parallel=0` job is submitted as version 0,
then an update is submitted with `max_parallel=1, canary=1` as verion 1.
In this case, we will have 1 canary alloc, and all remaining allocs will
be version 0.  Until the deployment is promoted, we ought to replace the
canaries with version 0 job (which isn't associated with a deployment).
2020-08-26 10:36:34 -04:00
Mahmood Ali e4bb88dfcf tweak stack job manipulation
To address review comments
2020-08-25 17:37:19 -04:00
Mahmood Ali def768728e Have Plan.AppendAlloc accept the job 2020-08-25 17:22:09 -04:00
Mahmood Ali 8a342926b7 Respect alloc job version for lost/failed allocs
This change fixes a bug where lost/failed allocations are replaced by
allocations with the latest versions, even if the version hasn't been
promoted yet.

Now, when generating a plan for lost/failed allocations, the scheduler
first checks if the current deployment is in Canary stage, and if so, it
ensures that any lost/failed allocations is replaced one with the latest
promoted version instead.
2020-08-19 09:52:48 -04:00
Mahmood Ali 1c1fb5da0a this is OSS 2020-06-22 10:28:45 -04:00
Nick Ethier f0559a8162
multi-interface network support 2020-06-19 09:42:10 -04:00
Lang Martin 069840bef8
scheduler/reconcile: set FollowupEvalID on lost stop_after_client_disconnect (#8105) (#8138)
* scheduler/reconcile: set FollowupEvalID on lost stop_after_client_disconnect

* scheduler/reconcile: thread follupEvalIDs through to results.stop

* scheduler/reconcile: comment typo

* nomad/_test: correct arguments for plan.AppendStoppedAlloc

* scheduler/reconcile: avoid nil, cleanup handleDelayed(Lost|Reschedules)
2020-06-09 17:13:53 -04:00
Lang Martin ac7c39d3d3
Delayed evaluations for `stop_after_client_disconnect` can cause unwanted extra followup evaluations around job garbage collection (#8099)
* client/heartbeatstop: reversed time condition for startup grace

* scheduler/generic_sched: use `delayInstead` to avoid a loop

Without protecting the loop that creates followUpEvals, a delayed eval
is allowed to create an immediate subsequent delayed eval. For both
`stop_after_client_disconnect` and the `reschedule` block, a delayed
eval should always produce some immediate result (running or blocked)
and then only after the outcome of that eval produce a second delayed
eval.

* scheduler/reconcile: lostLater are different than delayedReschedules

Just slightly. `lostLater` allocs should be used to create batched
evaluations, but `handleDelayedReschedules` assumes that the
allocations are in the untainted set. When it creates the in-place
updates to those allocations at the end, it causes the allocation to
be treated as running over in the planner, which causes the initial
`stop_after_client_disconnect` evaluation to be retried by the worker.
2020-06-03 09:48:38 -04:00
Mahmood Ali d9792777d9 Open source Preemption code
Nomad 0.12 OSS is to include preemption feature.

This commit moves the private code for managing preemption to OSS
repository.
2020-05-27 15:02:01 -04:00
Lang Martin d3c4700cd3
server: stop after client disconnect (#7939)
* jobspec, api: add stop_after_client_disconnect

* nomad/state/state_store: error message typo

* structs: alloc methods to support stop_after_client_disconnect

1. a global AllocStates to track status changes with timestamps. We
   need this to track the time at which the alloc became lost
   originally.

2. ShouldClientStop() and WaitClientStop() to actually do the math

* scheduler/reconcile_util: delayByStopAfterClientDisconnect

* scheduler/reconcile: use delayByStopAfterClientDisconnect

* scheduler/util: updateNonTerminalAllocsToLost comments

This was setup to only update allocs to lost if the DesiredStatus had
already been set by the scheduler. It seems like the intention was to
update the status from any non-terminal state, and not all lost allocs
have been marked stop or evict by now

* scheduler/testing: AssertEvalStatus just use require

* scheduler/generic_sched: don't create a blocked eval if delayed

* scheduler/generic_sched_test: several scheduling cases
2020-05-13 16:39:04 -04:00
Chris Baker 179ab68258 wip: added job.scale rpc endpoint, needs explicit test (tested via http now) 2020-03-24 13:57:09 +00:00
Mahmood Ali b880607bad update scheduler to account for hooks 2020-03-21 17:52:45 -04:00
Drew Bailey f51a3d1f37
nomad state store must be modified through raft, rm local state change 2020-02-03 13:57:34 -05:00
Tim Gross c50057bf1f
scheduler: fix job update placement on prev node penalized (#6781)
Fixes #5856

When the scheduler looks for a placement for an allocation that's
replacing another allocation, it's supposed to penalize the previous
node if the allocation had been rescheduled or failed. But we're
currently always penalizing the node, which leads to unnecessary
migrations on job update.

This commit leaves in place the existing behavior where if the
previous alloc was itself rescheduled, its previous nodes are also
penalized. This is conservative but the right behavior especially on
larger clusters where a group of hosts might be having correlated
trouble (like an AZ failure).

Co-Authored-By: Michael Schurter <mschurter@hashicorp.com>
2019-12-03 06:14:49 -08:00
Preetha Appan d21c708c4a
Fix inplace updates bug with group level networks
During inplace updates, we should be using network information
from the previous allocation being updated.
2019-09-05 18:37:24 -05:00
Nick Ethier 7c9520b404
scheduler: fix disk constraints 2019-07-31 01:04:08 -04:00
Nick Ethier 09a4cfd8d7
fix failing tests 2019-07-31 01:04:07 -04:00
Nick Ethier af66a35924
networking: Add new bridge networking mode implementation 2019-07-31 01:04:06 -04:00
Lang Martin 8157a7b6f8 system_sched submits failed evals as blocked 2019-07-18 10:32:12 -04:00
Mahmood Ali fd8fb8c22b Stop allocs to be rescheduled
Currently, when an alloc fails and is rescheduled, the alloc desired
state remains as "run" and the nomad client may not free the resources.

Here, we ensure that an alloc is marked as stopped when it's
rescheduled.

Notice the Desired Status and Description before and after this change:

Before:
```
mars-2:nomad notnoop$ nomad alloc status 02aba49e
ID                   = 02aba49e
Eval ID              = bb9ed1d2
Name                 = example-reschedule.nodes[0]
Node ID              = 5853d547
Node Name            = mars-2.local
Job ID               = example-reschedule
Job Version          = 0
Client Status        = failed
Client Description   = Failed tasks
Desired Status       = run
Desired Description  = <none>
Created              = 10s ago
Modified             = 5s ago
Replacement Alloc ID = d6bf872b

Task "payload" is "dead"
Task Resources
CPU        Memory          Disk     Addresses
0/100 MHz  24 MiB/300 MiB  300 MiB

Task Events:
Started At     = 2019-06-06T21:12:45Z
Finished At    = 2019-06-06T21:12:50Z
Total Restarts = 0
Last Restart   = N/A

Recent Events:
Time                       Type            Description
2019-06-06T17:12:50-04:00  Not Restarting  Policy allows no restarts
2019-06-06T17:12:50-04:00  Terminated      Exit Code: 1
2019-06-06T17:12:45-04:00  Started         Task started by client
2019-06-06T17:12:45-04:00  Task Setup      Building Task Directory
2019-06-06T17:12:45-04:00  Received        Task received by client

```

After:

```
ID                   = 5001ccd1
Eval ID              = 53507a02
Name                 = example-reschedule.nodes[0]
Node ID              = a3b04364
Node Name            = mars-2.local
Job ID               = example-reschedule
Job Version          = 0
Client Status        = failed
Client Description   = Failed tasks
Desired Status       = stop
Desired Description  = alloc was rescheduled because it failed
Created              = 13s ago
Modified             = 3s ago
Replacement Alloc ID = 7ba7ac20

Task "payload" is "dead"
Task Resources
CPU         Memory          Disk     Addresses
21/100 MHz  24 MiB/300 MiB  300 MiB

Task Events:
Started At     = 2019-06-06T21:22:50Z
Finished At    = 2019-06-06T21:22:55Z
Total Restarts = 0
Last Restart   = N/A

Recent Events:
Time                       Type            Description
2019-06-06T17:22:55-04:00  Not Restarting  Policy allows no restarts
2019-06-06T17:22:55-04:00  Terminated      Exit Code: 1
2019-06-06T17:22:50-04:00  Started         Task started by client
2019-06-06T17:22:50-04:00  Task Setup      Building Task Directory
2019-06-06T17:22:50-04:00  Received        Task received by client
```
2019-06-06 17:27:12 -04:00