client: allow `drain_on_shutdown` configuration (#16827)
Adds a new configuration to clients to optionally allow them to drain their workloads on shutdown. The client sends the `Node.UpdateDrain` RPC targeting itself and then monitors the drain state as seen by the server until the drain is complete or the deadline expires. If it loses connection with the server, it will monitor local client status instead to ensure allocations are stopped before exiting.
This commit is contained in:
parent
99185e2d8f
commit
62548616d4
|
@ -0,0 +1,3 @@
|
|||
```release-note:improvement
|
||||
client: Added `drain_on_shutdown` configuration
|
||||
```
|
|
@ -761,8 +761,12 @@ func (c *Client) Reload(newConfig *config.Config) error {
|
|||
|
||||
// Leave is used to prepare the client to leave the cluster
|
||||
func (c *Client) Leave() error {
|
||||
// TODO
|
||||
return nil
|
||||
if c.GetConfig().DevMode {
|
||||
return nil
|
||||
}
|
||||
|
||||
// In normal mode optionally drain the node
|
||||
return c.DrainSelf()
|
||||
}
|
||||
|
||||
// GetConfig returns the config of the client. Do *not* mutate without first
|
||||
|
|
|
@ -316,6 +316,9 @@ type Config struct {
|
|||
|
||||
// Artifact configuration from the agent's config file.
|
||||
Artifact *ArtifactConfig
|
||||
|
||||
// Drain configuration from the agent's config file.
|
||||
Drain *DrainConfig
|
||||
}
|
||||
|
||||
type APIListenerRegistrar interface {
|
||||
|
|
|
@ -0,0 +1,55 @@
|
|||
package config
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/nomad/nomad/structs/config"
|
||||
)
|
||||
|
||||
// DrainConfig describes a Node's drain behavior on graceful shutdown.
|
||||
type DrainConfig struct {
|
||||
// Deadline is the duration after the drain starts when client will stop
|
||||
// waiting for allocations to stop.
|
||||
Deadline time.Duration
|
||||
|
||||
// IgnoreSystemJobs allows systems jobs to remain on the node even though it
|
||||
// has been marked for draining.
|
||||
IgnoreSystemJobs bool
|
||||
|
||||
// Force causes the drain to stop all the allocations immediately, ignoring
|
||||
// their jobs' migrate blocks.
|
||||
Force bool
|
||||
}
|
||||
|
||||
// DrainConfigFromAgent creates the internal read-only copy of the client
|
||||
// agent's DrainConfig.
|
||||
func DrainConfigFromAgent(c *config.DrainConfig) (*DrainConfig, error) {
|
||||
if c == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
deadline := time.Hour
|
||||
ignoreSystemJobs := false
|
||||
force := false
|
||||
|
||||
if c.Deadline != nil {
|
||||
var err error
|
||||
deadline, err = time.ParseDuration(*c.Deadline)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error parsing Deadline: %w", err)
|
||||
}
|
||||
}
|
||||
if c.IgnoreSystemJobs != nil {
|
||||
ignoreSystemJobs = *c.IgnoreSystemJobs
|
||||
}
|
||||
if c.Force != nil {
|
||||
force = *c.Force
|
||||
}
|
||||
|
||||
return &DrainConfig{
|
||||
Deadline: deadline,
|
||||
IgnoreSystemJobs: ignoreSystemJobs,
|
||||
Force: force,
|
||||
}, nil
|
||||
}
|
|
@ -0,0 +1,163 @@
|
|||
package client
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/nomad/client/config"
|
||||
"github.com/hashicorp/nomad/helper"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
)
|
||||
|
||||
func (c *Client) DrainSelf() error {
|
||||
drainSpec := c.GetConfig().Drain
|
||||
if drainSpec == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
logger := c.logger.Named("drain")
|
||||
|
||||
now := time.Now()
|
||||
drainReq := &structs.NodeUpdateDrainRequest{
|
||||
NodeID: c.NodeID(),
|
||||
DrainStrategy: &structs.DrainStrategy{
|
||||
DrainSpec: structs.DrainSpec{
|
||||
Deadline: drainSpec.Deadline,
|
||||
IgnoreSystemJobs: drainSpec.IgnoreSystemJobs,
|
||||
},
|
||||
StartedAt: now,
|
||||
},
|
||||
MarkEligible: false,
|
||||
Meta: map[string]string{"message": "shutting down"},
|
||||
WriteRequest: structs.WriteRequest{
|
||||
Region: c.Region(), AuthToken: c.secretNodeID()},
|
||||
}
|
||||
if drainSpec.Deadline > 0 {
|
||||
drainReq.DrainStrategy.ForceDeadline = now.Add(drainSpec.Deadline)
|
||||
}
|
||||
|
||||
var drainResp structs.NodeDrainUpdateResponse
|
||||
err := c.RPC("Node.UpdateDrain", drainReq, &drainResp)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// note: the default deadline is 1hr but could be set to "". letting this
|
||||
// run forever seems wrong but init system (ex systemd) will almost always
|
||||
// force kill the client eventually
|
||||
ctx := context.Background()
|
||||
var cancel context.CancelFunc
|
||||
if drainSpec.Deadline > 0 {
|
||||
// if we set this context to the deadline, the server will reach the
|
||||
// deadline but not get a chance to record it before this context
|
||||
// expires, resulting in spurious errors. So extend the deadline here by
|
||||
// a few seconds
|
||||
ctx, cancel = context.WithTimeout(context.Background(), drainSpec.Deadline+(5*time.Second))
|
||||
defer cancel()
|
||||
}
|
||||
statusCheckInterval := time.Second
|
||||
|
||||
logger.Info("monitoring self-drain")
|
||||
err = c.pollServerForDrainStatus(ctx, statusCheckInterval)
|
||||
switch err {
|
||||
case nil:
|
||||
logger.Debug("self-drain complete")
|
||||
return nil
|
||||
case context.DeadlineExceeded, context.Canceled:
|
||||
logger.Error("self-drain exceeded deadline")
|
||||
return fmt.Errorf("self-drain exceeded deadline")
|
||||
default:
|
||||
logger.Error("could not check node status, falling back to local status checks", "error", err)
|
||||
}
|
||||
|
||||
err = c.pollLocalStatusForDrainStatus(ctx, statusCheckInterval, drainSpec)
|
||||
if err != nil {
|
||||
return fmt.Errorf("self-drain exceeded deadline")
|
||||
}
|
||||
|
||||
logger.Debug("self-drain complete")
|
||||
return nil
|
||||
}
|
||||
|
||||
// pollServerForDrainStatus will poll the server periodically for the client's
|
||||
// drain status, returning an error if the context expires or get any error from
|
||||
// the RPC call. If this function returns nil, the drain was successful.
|
||||
func (c *Client) pollServerForDrainStatus(ctx context.Context, interval time.Duration) error {
|
||||
timer, stop := helper.NewSafeTimer(0)
|
||||
defer stop()
|
||||
|
||||
statusReq := &structs.NodeSpecificRequest{
|
||||
NodeID: c.NodeID(),
|
||||
SecretID: c.secretNodeID(),
|
||||
QueryOptions: structs.QueryOptions{
|
||||
Region: c.Region(), AuthToken: c.secretNodeID()},
|
||||
}
|
||||
var statusResp structs.SingleNodeResponse
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-timer.C:
|
||||
err := c.RPC("Node.GetNode", statusReq, &statusResp)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if &statusResp != nil && statusResp.Node.DrainStrategy == nil {
|
||||
return nil
|
||||
}
|
||||
timer.Reset(interval)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// pollLocalStatusForDrainStatus polls the local allocrunner state periodicially
|
||||
// for the client status of all allocation runners, returning an error if the
|
||||
// context expires or get any error from the RPC call. If this function returns
|
||||
// nil, the drain was successful. This is a fallback function in case polling
|
||||
// the server fails.
|
||||
func (c *Client) pollLocalStatusForDrainStatus(ctx context.Context,
|
||||
interval time.Duration, drainSpec *config.DrainConfig) error {
|
||||
|
||||
// drainIsDone is its own function scope so we can release the allocLock
|
||||
// between poll attempts
|
||||
drainIsDone := func() bool {
|
||||
c.allocLock.RLock()
|
||||
defer c.allocLock.RUnlock()
|
||||
for _, runner := range c.allocs {
|
||||
|
||||
// note: allocs in runners should never be nil or have a nil Job but
|
||||
// if they do we can safely assume the runner is done with it
|
||||
alloc := runner.Alloc()
|
||||
if alloc != nil && !alloc.ClientTerminalStatus() {
|
||||
if !drainSpec.IgnoreSystemJobs {
|
||||
return false
|
||||
}
|
||||
if alloc.Job == nil {
|
||||
continue
|
||||
}
|
||||
if alloc.Job.Type != structs.JobTypeSystem {
|
||||
return false
|
||||
}
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
timer, stop := helper.NewSafeTimer(0)
|
||||
defer stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-timer.C:
|
||||
if drainIsDone() {
|
||||
return nil
|
||||
}
|
||||
timer.Reset(interval)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,201 @@
|
|||
package client
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/shoenig/test/must"
|
||||
"github.com/shoenig/test/wait"
|
||||
|
||||
"github.com/hashicorp/nomad/ci"
|
||||
"github.com/hashicorp/nomad/client/config"
|
||||
"github.com/hashicorp/nomad/helper/uuid"
|
||||
"github.com/hashicorp/nomad/nomad"
|
||||
"github.com/hashicorp/nomad/nomad/mock"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
nstructs "github.com/hashicorp/nomad/nomad/structs"
|
||||
"github.com/hashicorp/nomad/testutil"
|
||||
)
|
||||
|
||||
// TestClient_SelfDrainConfig is an integration test of the client's Leave
|
||||
// method that exercises the behavior of the drain_on_shutdown configuration
|
||||
func TestClient_SelfDrainConfig(t *testing.T) {
|
||||
ci.Parallel(t)
|
||||
|
||||
srv, _, cleanupSRV := testServer(t, nil)
|
||||
defer cleanupSRV()
|
||||
testutil.WaitForLeader(t, srv.RPC)
|
||||
|
||||
c1, cleanupC1 := TestClient(t, func(c *config.Config) {
|
||||
c.RPCHandler = srv
|
||||
c.DevMode = false
|
||||
c.Drain = &config.DrainConfig{
|
||||
Deadline: 10 * time.Second,
|
||||
IgnoreSystemJobs: true,
|
||||
}
|
||||
})
|
||||
defer cleanupC1()
|
||||
|
||||
jobID := "service-job-" + uuid.Short()
|
||||
sysJobID := "system-job-" + uuid.Short()
|
||||
testSelfDrainSetup(t, srv, c1.Node().ID, jobID, sysJobID)
|
||||
t.Log("setup complete successful, self-draining node")
|
||||
|
||||
testCtx, cancel := context.WithTimeout(context.Background(), time.Second*10)
|
||||
defer cancel()
|
||||
|
||||
errCh := make(chan error)
|
||||
go func() {
|
||||
errCh <- c1.Leave()
|
||||
}()
|
||||
|
||||
select {
|
||||
case err := <-errCh:
|
||||
must.NoError(t, err)
|
||||
case <-testCtx.Done():
|
||||
t.Fatal("expected drain complete before deadline")
|
||||
}
|
||||
|
||||
c1.allocLock.RLock()
|
||||
defer c1.allocLock.RUnlock()
|
||||
for _, runner := range c1.allocs {
|
||||
if runner.Alloc().JobID == sysJobID {
|
||||
must.Eq(t, structs.AllocClientStatusRunning, runner.AllocState().ClientStatus)
|
||||
} else {
|
||||
must.Eq(t, structs.AllocClientStatusComplete, runner.AllocState().ClientStatus)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// TestClient_SelfDrain_FailLocal is an integration test of the client's Leave
|
||||
// method that exercises the behavior when the client loses connection with the
|
||||
// server
|
||||
func TestClient_SelfDrain_FailLocal(t *testing.T) {
|
||||
ci.Parallel(t)
|
||||
|
||||
srv, _, cleanupSRV := testServer(t, nil)
|
||||
defer cleanupSRV()
|
||||
testutil.WaitForLeader(t, srv.RPC)
|
||||
|
||||
c1, cleanupC1 := TestClient(t, func(c *config.Config) {
|
||||
c.RPCHandler = srv
|
||||
c.DevMode = false
|
||||
c.Drain = &config.DrainConfig{Deadline: 5 * time.Second}
|
||||
})
|
||||
defer cleanupC1()
|
||||
|
||||
jobID := "service-job-" + uuid.Short()
|
||||
sysJobID := "system-job-" + uuid.Short()
|
||||
testSelfDrainSetup(t, srv, c1.Node().ID, jobID, sysJobID)
|
||||
|
||||
t.Log("setup complete successful, self-draining node and disconnecting node from server")
|
||||
|
||||
// note: this timeout has to cover the drain deadline plus the RPC timeout
|
||||
// when we fail to make the RPC to the leader
|
||||
testCtx, cancel := context.WithTimeout(context.Background(), time.Second*20)
|
||||
defer cancel()
|
||||
|
||||
errCh := make(chan error)
|
||||
go func() {
|
||||
errCh <- c1.Leave()
|
||||
}()
|
||||
|
||||
// We want to disconnect the server so that self-drain is forced to fallback
|
||||
// to local drain behavior. But if we disconnect the server before we start
|
||||
// the self-drain, the drain won't happen at all. So this attempts to
|
||||
// interleave disconnecting the server between when the drain starts and the
|
||||
// server marks the drain successful.
|
||||
go func() {
|
||||
req := structs.NodeSpecificRequest{
|
||||
NodeID: c1.Node().ID,
|
||||
QueryOptions: structs.QueryOptions{Region: "global"},
|
||||
}
|
||||
var out structs.SingleNodeResponse
|
||||
for {
|
||||
select {
|
||||
case <-testCtx.Done():
|
||||
return
|
||||
default:
|
||||
}
|
||||
err := srv.RPC("Node.GetNode", &req, &out)
|
||||
must.NoError(t, err)
|
||||
if out.Node.DrainStrategy != nil {
|
||||
cleanupSRV()
|
||||
return
|
||||
} else if out.Node.LastDrain != nil {
|
||||
return // the drain is already complete
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
select {
|
||||
case err := <-errCh:
|
||||
if err != nil {
|
||||
// we might not be able to interleave the disconnection, so it's
|
||||
// possible the Leave works just fine
|
||||
must.EqError(t, err, "self-drain exceeded deadline")
|
||||
}
|
||||
case <-testCtx.Done():
|
||||
t.Fatal("expected drain complete before test timeout")
|
||||
}
|
||||
}
|
||||
|
||||
func testSelfDrainSetup(t *testing.T, srv *nomad.Server, nodeID, jobID, sysJobID string) {
|
||||
req := structs.NodeSpecificRequest{
|
||||
NodeID: nodeID,
|
||||
QueryOptions: structs.QueryOptions{Region: "global"},
|
||||
}
|
||||
var out structs.SingleNodeResponse
|
||||
|
||||
// Wait for the node to register before we drain
|
||||
must.Wait(t, wait.InitialSuccess(
|
||||
wait.BoolFunc(func() bool {
|
||||
err := srv.RPC("Node.GetNode", &req, &out)
|
||||
must.NoError(t, err)
|
||||
return out.Node != nil
|
||||
}),
|
||||
wait.Timeout(5*time.Second),
|
||||
wait.Gap(10*time.Millisecond),
|
||||
))
|
||||
|
||||
// Run a job that starts quickly
|
||||
job := mock.Job()
|
||||
job.ID = jobID
|
||||
job.Constraints = nil
|
||||
job.TaskGroups[0].Constraints = nil
|
||||
job.TaskGroups[0].Count = 1
|
||||
job.TaskGroups[0].Migrate = nstructs.DefaultMigrateStrategy()
|
||||
job.TaskGroups[0].Migrate.MinHealthyTime = 100 * time.Millisecond
|
||||
job.TaskGroups[0].Networks = []*structs.NetworkResource{}
|
||||
job.TaskGroups[0].Tasks[0] = &structs.Task{
|
||||
Name: "mock",
|
||||
Driver: "mock_driver",
|
||||
Config: map[string]interface{}{"run_for": "1m"},
|
||||
LogConfig: structs.DefaultLogConfig(),
|
||||
Resources: &structs.Resources{
|
||||
CPU: 50,
|
||||
MemoryMB: 25,
|
||||
},
|
||||
}
|
||||
testutil.WaitForRunning(t, srv.RPC, job.Copy())
|
||||
|
||||
sysJob := mock.SystemJob()
|
||||
sysJob.ID = sysJobID
|
||||
sysJob.Constraints = nil
|
||||
sysJob.TaskGroups[0].Constraints = nil
|
||||
sysJob.TaskGroups[0].Networks = []*structs.NetworkResource{}
|
||||
sysJob.TaskGroups[0].Tasks[0] = &structs.Task{
|
||||
Name: "mock",
|
||||
Driver: "mock_driver",
|
||||
Config: map[string]interface{}{"run_for": "1m"},
|
||||
LogConfig: structs.DefaultLogConfig(),
|
||||
Resources: &structs.Resources{
|
||||
CPU: 50,
|
||||
MemoryMB: 25,
|
||||
},
|
||||
}
|
||||
testutil.WaitForRunning(t, srv.RPC, sysJob.Copy())
|
||||
|
||||
}
|
|
@ -853,6 +853,12 @@ func convertClientConfig(agentConfig *Config) (*clientconfig.Config, error) {
|
|||
}
|
||||
conf.Artifact = artifactConfig
|
||||
|
||||
drainConfig, err := clientconfig.DrainConfigFromAgent(agentConfig.Client.Drain)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("invalid drain_on_shutdown config: %v", err)
|
||||
}
|
||||
conf.Drain = drainConfig
|
||||
|
||||
return conf, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -344,6 +344,9 @@ type ClientConfig struct {
|
|||
// Artifact contains the configuration for artifacts.
|
||||
Artifact *config.ArtifactConfig `hcl:"artifact"`
|
||||
|
||||
// Drain specifies whether to drain the client on shutdown; ignored in dev mode.
|
||||
Drain *config.DrainConfig `hcl:"drain_on_shutdown"`
|
||||
|
||||
// ExtraKeysHCL is used by hcl to surface unexpected keys
|
||||
ExtraKeysHCL []string `hcl:",unusedKeys" json:"-"`
|
||||
}
|
||||
|
@ -366,6 +369,7 @@ func (c *ClientConfig) Copy() *ClientConfig {
|
|||
nc.HostNetworks = helper.CopySlice(c.HostNetworks)
|
||||
nc.NomadServiceDiscovery = pointer.Copy(c.NomadServiceDiscovery)
|
||||
nc.Artifact = c.Artifact.Copy()
|
||||
nc.Drain = c.Drain.Copy()
|
||||
nc.ExtraKeysHCL = slices.Clone(c.ExtraKeysHCL)
|
||||
return &nc
|
||||
}
|
||||
|
@ -1292,6 +1296,7 @@ func DefaultConfig() *Config {
|
|||
CNIConfigDir: "/opt/cni/config",
|
||||
NomadServiceDiscovery: pointer.Of(true),
|
||||
Artifact: config.DefaultArtifactConfig(),
|
||||
Drain: nil,
|
||||
},
|
||||
Server: &ServerConfig{
|
||||
Enabled: false,
|
||||
|
@ -2199,6 +2204,7 @@ func (a *ClientConfig) Merge(b *ClientConfig) *ClientConfig {
|
|||
}
|
||||
|
||||
result.Artifact = a.Artifact.Merge(b.Artifact)
|
||||
result.Drain = a.Drain.Merge(b.Drain)
|
||||
|
||||
return &result
|
||||
}
|
||||
|
|
15
nomad/fsm.go
15
nomad/fsm.go
|
@ -473,10 +473,19 @@ func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, ind
|
|||
return fmt.Errorf("error looking up ACL token: %v", err)
|
||||
}
|
||||
if token == nil {
|
||||
n.logger.Error("token did not exist during node drain update")
|
||||
return fmt.Errorf("token did not exist during node drain update")
|
||||
node, err := n.state.NodeBySecretID(nil, req.AuthToken)
|
||||
if err != nil {
|
||||
n.logger.Error("error looking up node for drain update", "error", err)
|
||||
return fmt.Errorf("error looking up node for drain update: %v", err)
|
||||
}
|
||||
if node == nil {
|
||||
n.logger.Error("token did not exist during node drain update")
|
||||
return fmt.Errorf("token did not exist during node drain update")
|
||||
}
|
||||
accessorId = node.ID
|
||||
} else {
|
||||
accessorId = token.AccessorID
|
||||
}
|
||||
accessorId = token.AccessorID
|
||||
}
|
||||
|
||||
if err := n.state.UpdateNodeDrain(reqType, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt,
|
||||
|
|
|
@ -0,0 +1,49 @@
|
|||
package config
|
||||
|
||||
import "github.com/hashicorp/nomad/helper/pointer"
|
||||
|
||||
// DrainConfig describes a Node's drain behavior on graceful shutdown.
|
||||
type DrainConfig struct {
|
||||
// Deadline is the duration after the drain starts when client will stop
|
||||
// waiting for allocations to stop.
|
||||
Deadline *string `hcl:"deadline"`
|
||||
|
||||
// IgnoreSystemJobs allows systems jobs to remain on the node even though it
|
||||
// has been marked for draining.
|
||||
IgnoreSystemJobs *bool `hcl:"ignore_system_jobs"`
|
||||
|
||||
// Force causes the drain to stop all the allocations immediately, ignoring
|
||||
// their jobs' migrate blocks.
|
||||
Force *bool `hcl:"force"`
|
||||
}
|
||||
|
||||
func (d *DrainConfig) Copy() *DrainConfig {
|
||||
if d == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
nd := new(DrainConfig)
|
||||
*nd = *d
|
||||
return nd
|
||||
}
|
||||
|
||||
func (d *DrainConfig) Merge(o *DrainConfig) *DrainConfig {
|
||||
switch {
|
||||
case d == nil:
|
||||
return o.Copy()
|
||||
case o == nil:
|
||||
return d.Copy()
|
||||
default:
|
||||
nd := d.Copy()
|
||||
if o.Deadline != nil {
|
||||
nd.Deadline = pointer.Copy(o.Deadline)
|
||||
}
|
||||
if o.IgnoreSystemJobs != nil && *o.IgnoreSystemJobs {
|
||||
nd.IgnoreSystemJobs = pointer.Of(true)
|
||||
}
|
||||
if o.Force != nil && *o.Force {
|
||||
nd.Force = pointer.Of(true)
|
||||
}
|
||||
return nd
|
||||
}
|
||||
}
|
|
@ -172,6 +172,11 @@ client {
|
|||
- `host_network` <code>([host_network](#host_network-block): nil)</code> - Registers
|
||||
additional host networks with the node that can be selected when port mapping.
|
||||
|
||||
- `drain_on_shutdown` <code>([drain_on_shutdown](#drain_on_shutdown-block):
|
||||
nil)</code> - Controls the behavior of the client when
|
||||
[`leave_on_interrupt`][] or [`leave_on_terminate`][] are set and the client
|
||||
receives the appropriate signal.
|
||||
|
||||
- `cgroup_parent` `(string: "/nomad")` - Specifies the cgroup parent for which cgroup
|
||||
subsystems managed by Nomad will be mounted under. Currently this only applies to the
|
||||
`cpuset` subsystems. This field is ignored on non Linux platforms.
|
||||
|
@ -602,6 +607,50 @@ client {
|
|||
[`reserved.reserved_ports`](#reserved_ports) are also reserved on each host
|
||||
network.
|
||||
|
||||
### `drain_on_shutdown` Block
|
||||
|
||||
The `drain_on_shutdown` block controls the behavior of the client when
|
||||
[`leave_on_interrupt`][] or [`leave_on_terminate`][] are set. By default
|
||||
`drain_on_shutdown` is not configured and clients will not drain on any
|
||||
signal.
|
||||
|
||||
If `drain_on_shutdown` is configured, the node will drain itself when receiving
|
||||
the appropriate signal (`SIGINT` for `leave_on_interrupt` or `SIGTERM` on
|
||||
`leave_on_terminate`). By default this acts similarly to running [`nomad node
|
||||
drain -self -no-deadline`][]
|
||||
|
||||
Note that even if no deadline is set, your init system may send `SIGKILL` to
|
||||
Nomad if the drain takes longer than allowed by the service shutdown. For
|
||||
example, when running under Linux with `systemd`, you should adjust the
|
||||
[`TimeoutStopSec`][] value in the `nomad.service` unit file to account for
|
||||
enough time for the client to drain.
|
||||
|
||||
```hcl
|
||||
client {
|
||||
# Either leave_on_interrupt or leave_on_terminate must be set
|
||||
# for this to take effect.
|
||||
drain_on_shutdown {
|
||||
deadline = "1h"
|
||||
force = false
|
||||
ignore_system_jobs = false
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
- `deadline` `(string: "1h")` - Set the deadline by which all allocations must
|
||||
be moved off the client. Remaining allocations after the deadline are removed
|
||||
from the client, regardless of their [`migrate`][] block. Defaults to 1 hour.
|
||||
|
||||
- `force` `(bool: false)` - Setting to `true` drains all the allocations on the
|
||||
client immediately, ignoring the [`migrate`][] block. Note if you have
|
||||
multiple allocations for the same job on the draining client without
|
||||
additional allocations on other clients, this will result in an outage for
|
||||
that job until the drain is complete.
|
||||
|
||||
- `ignore_system_jobs` `(bool: false)` - Setting to `true` allows the drain to
|
||||
complete without stopping system job allocations. By default system jobs (and
|
||||
CSI plugins) are stopped last.
|
||||
|
||||
## `client` Examples
|
||||
|
||||
### Common Setup
|
||||
|
@ -666,3 +715,8 @@ client {
|
|||
[task working directory]: /nomad/docs/runtime/environment#task-directories 'Task directories'
|
||||
[go-sockaddr/template]: https://godoc.org/github.com/hashicorp/go-sockaddr/template
|
||||
[landlock]: https://docs.kernel.org/userspace-api/landlock.html
|
||||
[`leave_on_interrupt`]: /nomad/docs/configuration#leave_on_interrupt
|
||||
[`leave_on_terminate`]: /nomad/docs/configuration#leave_on_terminate
|
||||
[migrate]: /nomad/docs/job-specification/migrate
|
||||
[`nomad node drain -self -no-deadline`]: /nomad/docs/commands/node/drain
|
||||
[`TimeoutStopSec`]: https://www.freedesktop.org/software/systemd/man/systemd.service.html#TimeoutStopSec=
|
||||
|
|
|
@ -171,17 +171,27 @@ testing.
|
|||
- `http_api_response_headers` `(map<string|string>: nil)` - Specifies
|
||||
user-defined headers to add to the HTTP API responses.
|
||||
|
||||
- `leave_on_interrupt` `(bool: false)` - Specifies if the agent should
|
||||
gracefully leave when receiving the interrupt signal. By default, the agent
|
||||
will exit forcefully on any signal. This value should only be set to true on
|
||||
server agents if it is expected that a terminated server instance will never
|
||||
join the cluster again.
|
||||
- `leave_on_interrupt` `(bool: false)` - Specifies if the agent should leave
|
||||
when receiving the interrupt signal. By default, any stop signal to an agent
|
||||
(interrupt or terminate) will cause the agent to exit after ensuring its
|
||||
internal state is committed to disk as needed. If this value is set to true on
|
||||
a server agent, the server will notify other servers of their intention to
|
||||
leave the peer set. You should only set this value to true on server agents if
|
||||
the terminated server will never join the cluster again. If this value is set
|
||||
to true on a client agent and the client is configured with
|
||||
[`drain_on_shutdown`][], the client will drain its workloads before shutting
|
||||
down.
|
||||
|
||||
- `leave_on_terminate` `(bool: false)` - Specifies if the agent should
|
||||
gracefully leave when receiving the terminate signal. By default, the agent
|
||||
will exit forcefully on any signal. This value should only be set to true on
|
||||
server agents if it is expected that a terminated server instance will never
|
||||
join the cluster again.
|
||||
- `leave_on_terminate` `(bool: false)` - Specifies if the agent should leave
|
||||
when receiving the terminate signal. By default, any stop signal to an agent
|
||||
(interrupt or terminate) will cause the agent to exit after ensuring its
|
||||
internal state is committed to disk as needed. If this value is set to true on
|
||||
a server agent, the server will notify other servers of their intention to
|
||||
leave the peer set. You should only set this value to true on server agents if
|
||||
the terminated server will never join the cluster again. If this value is set
|
||||
to true on a client agent and the client is configured with
|
||||
[`drain_on_shutdown`][], the client will drain its workloads before shutting
|
||||
down.
|
||||
|
||||
- `limits` - Available in Nomad 0.10.3 and later, this is a nested object that
|
||||
configures limits that are enforced by the agent. The following parameters
|
||||
|
@ -384,3 +394,4 @@ http_api_response_headers {
|
|||
[tls-reload]: /nomad/docs/configuration/tls#tls-configuration-reloads
|
||||
[vault-reload]: /nomad/docs/configuration/vault#vault-configuration-reloads
|
||||
[gh-3885]: https://github.com/hashicorp/nomad/issues/3885
|
||||
[`drain_on_shutdown`]: /nomad/docs/configuration/client#drain_on_shutdown
|
||||
|
|
|
@ -82,20 +82,23 @@ There are several important messages that `nomad agent` outputs:
|
|||
|
||||
## Stopping an Agent
|
||||
|
||||
An agent can be stopped in two ways: gracefully or forcefully. By default, any
|
||||
stop signal to an agent (interrupt, terminate, kill) will cause the agent to
|
||||
forcefully stop. Graceful termination can be configured by either setting
|
||||
By default, any stop signal to an agent (interrupt or terminate) will cause the
|
||||
agent to exit after ensuring its internal state is committed to disk as
|
||||
needed. You can configuration additonal behaviors by setting shutdown
|
||||
[`leave_on_interrupt`][] or [`leave_on_terminate`][] to respond to the
|
||||
respective signals.
|
||||
|
||||
When gracefully exiting, servers will notify their intention to leave the
|
||||
cluster which allows them to leave the [consensus][] peer set.
|
||||
For servers, when `leave_on_interrupt` or `leave_on_terminate` are set the
|
||||
server will notify other servers of their intention to leave the cluster, which
|
||||
allows them to leave the [consensus][] peer set. It is especially important that
|
||||
a server node be allowed to leave gracefully so that there will be a minimal
|
||||
impact on availability as the server leaves the consensus peer set. If a server
|
||||
does not gracefully leave, and will not return into service, the [`server
|
||||
force-leave` command][] should be used to eject it from the consensus peer set.
|
||||
|
||||
It is especially important that a server node be allowed to leave gracefully so
|
||||
that there will be a minimal impact on availability as the server leaves the
|
||||
consensus peer set. If a server does not gracefully leave, and will not return
|
||||
into service, the [`server force-leave` command][] should be used to eject it
|
||||
from the consensus peer set.
|
||||
For clients, when `leave_on_interrupt` or `leave_on_terminate` are set and the
|
||||
client is configured with [`drain_on_shutdown`][], the client will drain its
|
||||
workloads before shutting down.
|
||||
|
||||
## Signal Handling
|
||||
|
||||
|
@ -180,5 +183,6 @@ owned by `root` with filesystem permissions set to `0700`.
|
|||
[`leave_on_terminate`]: /nomad/docs/configuration#leave_on_terminate
|
||||
[`server force-leave` command]: /nomad/docs/commands/server/force-leave
|
||||
[consensus]: /nomad/docs/concepts/consensus
|
||||
[`drain_on_shutdown`]: /nomad/docs/configuration/client#drain_on_shutdown
|
||||
[reload its configuration]: /nomad/docs/configuration#configuration-reload
|
||||
[metrics]: /nomad/docs/operations/metrics-reference
|
||||
|
|
|
@ -73,6 +73,12 @@ On a single server, install the new version of Nomad. You can do this by
|
|||
joining a new server to the cluster or by replacing or upgrading the binary
|
||||
locally and restarting the Nomad service.
|
||||
|
||||
Note that if you have [`leave_on_terminate`][] or [`leave_on_interrupt`][] set,
|
||||
you should ensure you're using the expected signal for your upgrade process. For
|
||||
example, if you have `leave_on_terminate` set and you intend on updating a
|
||||
server in-place, you should `SIGINT` and not `SIGTERM` when shutting down the
|
||||
server before restarting it.
|
||||
|
||||
### 2. Check cluster health
|
||||
|
||||
[Monitor the Nomad logs][monitor] on the remaining servers to check that the
|
||||
|
@ -244,3 +250,5 @@ configuration to `3` and restart the server.
|
|||
|
||||
[peers-json]: /nomad/tutorials/manage-clusters/outage-recovery#manual-recovery-using-peersjson
|
||||
[`raft_protocol`]: /nomad/docs/configuration/server#raft_protocol
|
||||
[`leave_on_interrupt`]: /nomad/docs/configuration#leave_on_interrupt
|
||||
[`leave_on_terminate`]: /nomad/docs/configuration#leave_on_terminate
|
||||
|
|
Loading…
Reference in New Issue