2023-04-10 15:36:59 +00:00
|
|
|
// Copyright (c) HashiCorp, Inc.
|
|
|
|
// SPDX-License-Identifier: MPL-2.0
|
|
|
|
|
2015-06-07 19:14:41 +00:00
|
|
|
package nomad
|
|
|
|
|
|
|
|
import (
|
2016-08-18 21:31:44 +00:00
|
|
|
"context"
|
2022-04-02 00:24:02 +00:00
|
|
|
"errors"
|
2015-06-07 19:14:41 +00:00
|
|
|
"fmt"
|
2022-04-21 15:04:33 +00:00
|
|
|
"net/http"
|
2021-08-10 21:17:44 +00:00
|
|
|
"reflect"
|
2016-08-16 06:11:57 +00:00
|
|
|
"strings"
|
2016-02-22 02:51:34 +00:00
|
|
|
"sync"
|
2015-06-07 19:14:41 +00:00
|
|
|
"time"
|
|
|
|
|
2022-04-02 00:24:02 +00:00
|
|
|
"github.com/armon/go-metrics"
|
|
|
|
"github.com/hashicorp/go-hclog"
|
|
|
|
"github.com/hashicorp/go-memdb"
|
|
|
|
"github.com/hashicorp/go-multierror"
|
2022-12-01 15:05:15 +00:00
|
|
|
vapi "github.com/hashicorp/vault/api"
|
|
|
|
"golang.org/x/sync/errgroup"
|
|
|
|
|
2017-09-15 04:42:19 +00:00
|
|
|
"github.com/hashicorp/nomad/acl"
|
2017-09-29 16:58:48 +00:00
|
|
|
"github.com/hashicorp/nomad/helper/uuid"
|
2016-06-01 10:47:19 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/state"
|
2022-04-21 15:04:33 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/state/paginator"
|
2015-06-07 19:14:41 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/structs"
|
2016-10-23 01:08:30 +00:00
|
|
|
"github.com/hashicorp/raft"
|
2015-06-07 19:14:41 +00:00
|
|
|
)
|
|
|
|
|
2016-02-22 02:51:34 +00:00
|
|
|
const (
|
|
|
|
// batchUpdateInterval is how long we wait to batch updates
|
|
|
|
batchUpdateInterval = 50 * time.Millisecond
|
2016-08-18 21:31:44 +00:00
|
|
|
|
|
|
|
// maxParallelRequestsPerDerive is the maximum number of parallel Vault
|
|
|
|
// create token requests that may be outstanding per derive request
|
|
|
|
maxParallelRequestsPerDerive = 16
|
2018-05-10 23:54:43 +00:00
|
|
|
|
|
|
|
// NodeDrainEvents are the various drain messages
|
|
|
|
NodeDrainEventDrainSet = "Node drain strategy set"
|
|
|
|
NodeDrainEventDrainDisabled = "Node drain disabled"
|
2022-03-21 12:37:08 +00:00
|
|
|
NodeDrainEventDrainUpdated = "Node drain strategy updated"
|
2018-05-11 21:32:34 +00:00
|
|
|
|
|
|
|
// NodeEligibilityEventEligible is used when the nodes eligiblity is marked
|
|
|
|
// eligible
|
|
|
|
NodeEligibilityEventEligible = "Node marked as eligible for scheduling"
|
|
|
|
|
|
|
|
// NodeEligibilityEventIneligible is used when the nodes eligiblity is marked
|
|
|
|
// ineligible
|
|
|
|
NodeEligibilityEventIneligible = "Node marked as ineligible for scheduling"
|
2018-05-12 00:26:25 +00:00
|
|
|
|
|
|
|
// NodeHeartbeatEventReregistered is the message used when the node becomes
|
|
|
|
// reregistered by the heartbeat.
|
|
|
|
NodeHeartbeatEventReregistered = "Node reregistered by heartbeat"
|
2016-02-22 02:51:34 +00:00
|
|
|
)
|
|
|
|
|
2015-09-07 03:31:32 +00:00
|
|
|
// Node endpoint is used for client interactions
|
|
|
|
type Node struct {
|
2018-09-15 23:23:13 +00:00
|
|
|
srv *Server
|
2022-04-02 00:24:02 +00:00
|
|
|
logger hclog.Logger
|
2016-02-22 02:51:34 +00:00
|
|
|
|
2018-01-05 21:50:04 +00:00
|
|
|
// ctx provides context regarding the underlying connection
|
|
|
|
ctx *RPCContext
|
|
|
|
|
2016-02-22 02:51:34 +00:00
|
|
|
// updates holds pending client status updates for allocations
|
|
|
|
updates []*structs.Allocation
|
|
|
|
|
2018-04-09 19:05:31 +00:00
|
|
|
// evals holds pending rescheduling eval updates triggered by failed allocations
|
|
|
|
evals []*structs.Evaluation
|
|
|
|
|
2016-02-22 02:51:34 +00:00
|
|
|
// updateFuture is used to wait for the pending batch update
|
|
|
|
// to complete. This may be nil if no batch is pending.
|
2018-03-06 22:37:37 +00:00
|
|
|
updateFuture *structs.BatchFuture
|
2016-02-22 02:51:34 +00:00
|
|
|
|
|
|
|
// updateTimer is the timer that will trigger the next batch
|
|
|
|
// update, and may be nil if there is no batch pending.
|
|
|
|
updateTimer *time.Timer
|
|
|
|
|
|
|
|
// updatesLock synchronizes access to the updates list,
|
|
|
|
// the future and the timer.
|
|
|
|
updatesLock sync.Mutex
|
2015-06-07 19:14:41 +00:00
|
|
|
}
|
|
|
|
|
2022-12-01 15:05:15 +00:00
|
|
|
func NewNodeEndpoint(srv *Server, ctx *RPCContext) *Node {
|
|
|
|
return &Node{
|
|
|
|
srv: srv,
|
|
|
|
ctx: ctx,
|
|
|
|
logger: srv.logger.Named("client"),
|
|
|
|
updates: []*structs.Allocation{},
|
|
|
|
evals: []*structs.Evaluation{},
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-06-07 19:14:41 +00:00
|
|
|
// Register is used to upsert a client that is available for scheduling
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) Register(args *structs.NodeRegisterRequest, reply *structs.NodeUpdateResponse) error {
|
2023-01-24 15:52:07 +00:00
|
|
|
// note that we trust-on-first use and the identity will be anonymous for
|
|
|
|
// that initial request; we lean on mTLS for handling that safely
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
|
|
|
|
2020-03-18 01:35:56 +00:00
|
|
|
isForwarded := args.IsForwarded()
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.Register", args, args, reply); done {
|
2018-03-27 01:10:43 +00:00
|
|
|
// We have a valid node connection since there is no error from the
|
|
|
|
// forwarded server, so add the mapping to cache the
|
|
|
|
// connection and allow the server to send RPCs to the client.
|
2020-03-18 01:35:56 +00:00
|
|
|
if err == nil && n.ctx != nil && n.ctx.NodeID == "" && !isForwarded {
|
2018-03-27 01:10:43 +00:00
|
|
|
n.ctx.NodeID = args.Node.ID
|
|
|
|
n.srv.addNodeConn(n.ctx)
|
|
|
|
}
|
|
|
|
|
2015-06-07 19:14:41 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-24 15:52:07 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2015-06-07 19:14:41 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "register"}, time.Now())
|
|
|
|
|
|
|
|
// Validate the arguments
|
2015-07-04 00:47:55 +00:00
|
|
|
if args.Node == nil {
|
|
|
|
return fmt.Errorf("missing node for client registration")
|
|
|
|
}
|
|
|
|
if args.Node.ID == "" {
|
|
|
|
return fmt.Errorf("missing node ID for client registration")
|
|
|
|
}
|
|
|
|
if args.Node.Datacenter == "" {
|
2015-06-07 19:14:41 +00:00
|
|
|
return fmt.Errorf("missing datacenter for client registration")
|
|
|
|
}
|
2015-07-04 00:47:55 +00:00
|
|
|
if args.Node.Name == "" {
|
2015-06-07 19:14:41 +00:00
|
|
|
return fmt.Errorf("missing node name for client registration")
|
|
|
|
}
|
2016-08-16 06:11:57 +00:00
|
|
|
if len(args.Node.Attributes) == 0 {
|
|
|
|
return fmt.Errorf("missing attributes for client registration")
|
|
|
|
}
|
2017-10-13 18:28:47 +00:00
|
|
|
if args.Node.SecretID == "" {
|
2016-08-19 17:50:49 +00:00
|
|
|
return fmt.Errorf("missing node secret ID for client registration")
|
2016-08-16 06:11:57 +00:00
|
|
|
}
|
2015-06-07 19:14:41 +00:00
|
|
|
|
|
|
|
// Default the status if none is given
|
2015-07-04 00:47:55 +00:00
|
|
|
if args.Node.Status == "" {
|
|
|
|
args.Node.Status = structs.NodeStatusInit
|
2015-06-07 19:14:41 +00:00
|
|
|
}
|
2015-08-06 23:39:20 +00:00
|
|
|
if !structs.ValidNodeStatus(args.Node.Status) {
|
|
|
|
return fmt.Errorf("invalid status for node")
|
|
|
|
}
|
2015-06-07 19:14:41 +00:00
|
|
|
|
2018-01-24 00:47:00 +00:00
|
|
|
// Default to eligible for scheduling if unset
|
|
|
|
if args.Node.SchedulingEligibility == "" {
|
|
|
|
args.Node.SchedulingEligibility = structs.NodeSchedulingEligible
|
|
|
|
}
|
|
|
|
|
2016-07-12 17:29:23 +00:00
|
|
|
// Set the timestamp when the node is registered
|
|
|
|
args.Node.StatusUpdatedAt = time.Now().Unix()
|
|
|
|
|
2016-01-21 01:30:02 +00:00
|
|
|
// Compute the node class
|
|
|
|
if err := args.Node.ComputeClass(); err != nil {
|
|
|
|
return fmt.Errorf("failed to computed node class: %v", err)
|
|
|
|
}
|
|
|
|
|
2017-08-07 21:13:05 +00:00
|
|
|
// Look for the node so we can detect a state transition
|
2016-07-21 22:22:02 +00:00
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-02-08 04:31:23 +00:00
|
|
|
|
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
originalNode, err := snap.NodeByID(ws, args.Node.ID)
|
2016-07-21 22:22:02 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2017-10-13 18:28:47 +00:00
|
|
|
if originalNode != nil {
|
2023-01-25 20:53:59 +00:00
|
|
|
// Check if the SecretID has been tampered with
|
2016-10-27 05:05:44 +00:00
|
|
|
if args.Node.SecretID != originalNode.SecretID && originalNode.SecretID != "" {
|
2016-08-19 17:50:49 +00:00
|
|
|
return fmt.Errorf("node secret ID does not match. Not registering node.")
|
|
|
|
}
|
2023-01-25 20:53:59 +00:00
|
|
|
|
|
|
|
// Don't allow the Register method to update the node status. Only the
|
|
|
|
// UpdateStatus method should be able to do this.
|
|
|
|
if originalNode.Status != "" {
|
|
|
|
args.Node.Status = originalNode.Status
|
|
|
|
}
|
2016-08-19 17:50:49 +00:00
|
|
|
}
|
|
|
|
|
2018-01-05 21:50:04 +00:00
|
|
|
// We have a valid node connection, so add the mapping to cache the
|
2018-03-27 01:10:43 +00:00
|
|
|
// connection and allow the server to send RPCs to the client. We only cache
|
|
|
|
// the connection if it is not being forwarded from another server.
|
|
|
|
if n.ctx != nil && n.ctx.NodeID == "" && !args.IsForwarded() {
|
2018-01-05 21:50:04 +00:00
|
|
|
n.ctx.NodeID = args.Node.ID
|
|
|
|
n.srv.addNodeConn(n.ctx)
|
|
|
|
}
|
|
|
|
|
2015-06-07 19:14:41 +00:00
|
|
|
// Commit this update via Raft
|
2015-09-07 03:31:32 +00:00
|
|
|
_, index, err := n.srv.raftApply(structs.NodeRegisterRequestType, args)
|
2015-06-07 19:14:41 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("register failed", "error", err)
|
2015-06-07 19:14:41 +00:00
|
|
|
return err
|
|
|
|
}
|
2015-08-06 23:39:20 +00:00
|
|
|
reply.NodeModifyIndex = index
|
|
|
|
|
|
|
|
// Check if we should trigger evaluations
|
2021-08-10 21:17:44 +00:00
|
|
|
if shouldCreateNodeEval(originalNode, args.Node) {
|
2022-07-06 21:35:18 +00:00
|
|
|
evalIDs, evalIndex, err := n.createNodeEvals(args.Node, index)
|
2015-08-06 23:39:20 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eval creation failed", "error", err)
|
2015-08-06 23:39:20 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.EvalIDs = evalIDs
|
|
|
|
reply.EvalCreateIndex = evalIndex
|
|
|
|
}
|
2015-07-06 20:34:32 +00:00
|
|
|
|
2015-08-23 00:37:50 +00:00
|
|
|
// Check if we need to setup a heartbeat
|
|
|
|
if !args.Node.TerminalStatus() {
|
2015-09-07 03:31:32 +00:00
|
|
|
ttl, err := n.srv.resetHeartbeatTimer(args.Node.ID)
|
2015-08-23 00:37:50 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("heartbeat reset failed", "error", err)
|
2015-08-23 00:37:50 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.HeartbeatTTL = ttl
|
|
|
|
}
|
|
|
|
|
2015-07-06 20:34:32 +00:00
|
|
|
// Set the reply index
|
|
|
|
reply.Index = index
|
2016-07-21 22:22:02 +00:00
|
|
|
snap, err = n.srv.fsm.State().Snapshot()
|
2016-06-10 05:16:02 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2016-06-01 10:47:19 +00:00
|
|
|
|
|
|
|
n.srv.peerLock.RLock()
|
|
|
|
defer n.srv.peerLock.RUnlock()
|
2022-09-08 18:31:36 +00:00
|
|
|
if err := n.constructNodeServerInfoResponse(args.Node.ID, snap, reply); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("failed to populate NodeUpdateResponse", "error", err)
|
2016-06-01 10:47:19 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-08-10 21:17:44 +00:00
|
|
|
// shouldCreateNodeEval returns true if the node update may result into
|
|
|
|
// allocation updates, so the node should be re-evaluating.
|
|
|
|
//
|
|
|
|
// Such cases might be:
|
|
|
|
// * node health/drain status changes that may result into alloc rescheduling
|
|
|
|
// * node drivers or attributes changing that may cause system job placement changes
|
|
|
|
func shouldCreateNodeEval(original, updated *structs.Node) bool {
|
|
|
|
if structs.ShouldDrainNode(updated.Status) {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
if original == nil {
|
2022-04-15 13:47:45 +00:00
|
|
|
return nodeStatusTransitionRequiresEval(updated.Status, structs.NodeStatusInit)
|
2021-08-10 21:17:44 +00:00
|
|
|
}
|
|
|
|
|
2022-04-15 13:47:45 +00:00
|
|
|
if nodeStatusTransitionRequiresEval(updated.Status, original.Status) {
|
2021-08-10 21:17:44 +00:00
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
// check fields used by the feasibility checks in ../scheduler/feasible.go,
|
|
|
|
// whether through a Constraint explicitly added by user or an implicit constraint
|
|
|
|
// added through a driver/volume check.
|
|
|
|
//
|
|
|
|
// Node Resources (e.g. CPU/Memory) are handled differently, using blocked evals,
|
|
|
|
// and not relevant in this check.
|
|
|
|
return !(original.ID == updated.ID &&
|
|
|
|
original.Datacenter == updated.Datacenter &&
|
|
|
|
original.Name == updated.Name &&
|
|
|
|
original.NodeClass == updated.NodeClass &&
|
|
|
|
reflect.DeepEqual(original.Attributes, updated.Attributes) &&
|
|
|
|
reflect.DeepEqual(original.Meta, updated.Meta) &&
|
|
|
|
reflect.DeepEqual(original.Drivers, updated.Drivers) &&
|
|
|
|
reflect.DeepEqual(original.HostVolumes, updated.HostVolumes) &&
|
|
|
|
equalDevices(original, updated))
|
|
|
|
}
|
|
|
|
|
|
|
|
func equalDevices(n1, n2 *structs.Node) bool {
|
|
|
|
// ignore super old nodes, mostly to avoid nil dereferencing
|
|
|
|
if n1.NodeResources == nil || n2.NodeResources == nil {
|
|
|
|
return n1.NodeResources == n2.NodeResources
|
|
|
|
}
|
|
|
|
|
|
|
|
// treat nil and empty value as equal
|
|
|
|
if len(n1.NodeResources.Devices) == 0 {
|
|
|
|
return len(n1.NodeResources.Devices) == len(n2.NodeResources.Devices)
|
|
|
|
}
|
|
|
|
|
|
|
|
return reflect.DeepEqual(n1.NodeResources.Devices, n2.NodeResources.Devices)
|
|
|
|
}
|
|
|
|
|
2016-06-01 10:47:19 +00:00
|
|
|
// updateNodeUpdateResponse assumes the n.srv.peerLock is held for reading.
|
2022-09-08 18:31:36 +00:00
|
|
|
func (n *Node) constructNodeServerInfoResponse(nodeID string, snap *state.StateSnapshot, reply *structs.NodeUpdateResponse) error {
|
2017-02-03 00:07:15 +00:00
|
|
|
reply.LeaderRPCAddr = string(n.srv.raft.Leader())
|
2016-06-01 10:47:19 +00:00
|
|
|
|
|
|
|
// Reply with config information required for future RPC requests
|
|
|
|
reply.Servers = make([]*structs.NodeServerInfo, 0, len(n.srv.localPeers))
|
2018-03-16 22:53:14 +00:00
|
|
|
for _, v := range n.srv.localPeers {
|
2016-06-01 10:47:19 +00:00
|
|
|
reply.Servers = append(reply.Servers,
|
|
|
|
&structs.NodeServerInfo{
|
2018-03-16 22:53:14 +00:00
|
|
|
RPCAdvertiseAddr: v.RPCAddr.String(),
|
2016-06-01 10:47:19 +00:00
|
|
|
Datacenter: v.Datacenter,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2022-09-08 18:31:36 +00:00
|
|
|
// Add ClientStatus information to heartbeat response.
|
|
|
|
node, _ := snap.NodeByID(nil, nodeID)
|
|
|
|
reply.SchedulingEligibility = node.SchedulingEligibility
|
|
|
|
|
2016-06-10 05:16:02 +00:00
|
|
|
// TODO(sean@): Use an indexed node count instead
|
|
|
|
//
|
|
|
|
// Snapshot is used only to iterate over all nodes to create a node
|
|
|
|
// count to send back to Nomad Clients in their heartbeat so Clients
|
|
|
|
// can estimate the size of the cluster.
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
iter, err := snap.Nodes(ws)
|
2016-06-01 10:47:19 +00:00
|
|
|
if err == nil {
|
|
|
|
for {
|
|
|
|
raw := iter.Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
reply.NumNodes++
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-05-27 17:46:52 +00:00
|
|
|
reply.Features = n.srv.EnterpriseState.Features()
|
|
|
|
|
2015-07-06 20:34:32 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
// Deregister is used to remove a client from the cluster. If a client should
|
|
|
|
// just be made unavailable for scheduling, a status update is preferred.
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) Deregister(args *structs.NodeDeregisterRequest, reply *structs.NodeUpdateResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2019-06-26 20:14:36 +00:00
|
|
|
if done, err := n.srv.forward("Node.Deregister", args, args, reply); done {
|
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2019-06-26 20:14:36 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "deregister"}, time.Now())
|
|
|
|
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID for client deregistration")
|
|
|
|
}
|
|
|
|
|
|
|
|
// deregister takes a batch
|
|
|
|
repack := &structs.NodeBatchDeregisterRequest{
|
2019-06-13 13:46:05 +00:00
|
|
|
NodeIDs: []string{args.NodeID},
|
|
|
|
WriteRequest: args.WriteRequest,
|
2019-06-26 20:14:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return n.deregister(repack, reply, func() (interface{}, uint64, error) {
|
|
|
|
return n.srv.raftApply(structs.NodeDeregisterRequestType, args)
|
|
|
|
})
|
2019-06-13 13:46:05 +00:00
|
|
|
}
|
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
// BatchDeregister is used to remove client nodes from the cluster.
|
|
|
|
func (n *Node) BatchDeregister(args *structs.NodeBatchDeregisterRequest, reply *structs.NodeUpdateResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2019-06-26 14:57:58 +00:00
|
|
|
if done, err := n.srv.forward("Node.BatchDeregister", args, args, reply); done {
|
2015-07-06 20:42:33 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2019-06-26 20:14:36 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "batch_deregister"}, time.Now())
|
2015-07-06 20:42:33 +00:00
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
if len(args.NodeIDs) == 0 {
|
|
|
|
return fmt.Errorf("missing node IDs for client deregistration")
|
|
|
|
}
|
|
|
|
|
|
|
|
return n.deregister(args, reply, func() (interface{}, uint64, error) {
|
|
|
|
return n.srv.raftApply(structs.NodeBatchDeregisterRequestType, args)
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
// deregister takes a raftMessage closure, to support both Deregister and BatchDeregister
|
|
|
|
func (n *Node) deregister(args *structs.NodeBatchDeregisterRequest,
|
|
|
|
reply *structs.NodeUpdateResponse,
|
|
|
|
raftApplyFn func() (interface{}, uint64, error),
|
|
|
|
) error {
|
|
|
|
// Check request permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
if aclObj, err := n.srv.ResolveACL(args); err != nil {
|
2017-10-26 21:12:17 +00:00
|
|
|
return err
|
|
|
|
} else if aclObj != nil && !aclObj.AllowNodeWrite() {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
// Look for the node
|
2017-10-26 06:51:53 +00:00
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-07-06 21:35:18 +00:00
|
|
|
nodes := make([]*structs.Node, 0, len(args.NodeIDs))
|
2019-06-13 13:46:05 +00:00
|
|
|
for _, nodeID := range args.NodeIDs {
|
2022-07-06 21:35:18 +00:00
|
|
|
node, err := snap.NodeByID(nil, nodeID)
|
2019-06-05 14:49:57 +00:00
|
|
|
if err != nil {
|
2019-06-26 20:14:36 +00:00
|
|
|
return err
|
2019-06-05 14:49:57 +00:00
|
|
|
}
|
|
|
|
if node == nil {
|
2019-06-26 20:14:36 +00:00
|
|
|
return fmt.Errorf("node not found")
|
2019-06-05 14:49:57 +00:00
|
|
|
}
|
2022-07-06 21:35:18 +00:00
|
|
|
nodes = append(nodes, node)
|
2019-06-07 15:25:55 +00:00
|
|
|
}
|
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
// Commit this update via Raft
|
|
|
|
_, index, err := raftApplyFn()
|
2019-06-07 15:25:55 +00:00
|
|
|
if err != nil {
|
2019-06-26 20:14:36 +00:00
|
|
|
n.logger.Error("raft message failed", "error", err)
|
2019-06-07 15:25:55 +00:00
|
|
|
return err
|
|
|
|
}
|
2019-06-05 14:49:57 +00:00
|
|
|
|
2022-07-06 21:35:18 +00:00
|
|
|
for _, node := range nodes {
|
|
|
|
nodeID := node.ID
|
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
// Clear the heartbeat timer if any
|
2019-06-05 14:49:57 +00:00
|
|
|
n.srv.clearHeartbeatTimer(nodeID)
|
|
|
|
|
2019-06-26 20:14:36 +00:00
|
|
|
// Create the evaluations for this node
|
2022-07-06 21:35:18 +00:00
|
|
|
evalIDs, evalIndex, err := n.createNodeEvals(node, index)
|
2019-06-26 20:14:36 +00:00
|
|
|
if err != nil {
|
|
|
|
n.logger.Error("eval creation failed", "error", err)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Determine if there are any Vault accessors on the node
|
2022-07-06 21:35:18 +00:00
|
|
|
if accessors, err := snap.VaultAccessorsByNode(nil, nodeID); err != nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
n.logger.Error("looking up vault accessors for node failed", "node_id", nodeID, "error", err)
|
2019-06-05 14:49:57 +00:00
|
|
|
return err
|
2019-12-06 20:46:46 +00:00
|
|
|
} else if l := len(accessors); l > 0 {
|
|
|
|
n.logger.Debug("revoking vault accessors on node due to deregister", "num_accessors", l, "node_id", nodeID)
|
|
|
|
if err := n.srv.vault.RevokeTokens(context.Background(), accessors, true); err != nil {
|
|
|
|
n.logger.Error("revoking vault accessors for node failed", "node_id", nodeID, "error", err)
|
|
|
|
return err
|
|
|
|
}
|
2019-06-05 14:49:57 +00:00
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
// Determine if there are any SI token accessors on the node
|
2022-07-06 21:35:18 +00:00
|
|
|
if accessors, err := snap.SITokenAccessorsByNode(nil, nodeID); err != nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
n.logger.Error("looking up si accessors for node failed", "node_id", nodeID, "error", err)
|
|
|
|
return err
|
|
|
|
} else if l := len(accessors); l > 0 {
|
|
|
|
n.logger.Debug("revoking si accessors on node due to deregister", "num_accessors", l, "node_id", nodeID)
|
2020-01-07 17:58:29 +00:00
|
|
|
// Unlike with the Vault integration, there's no error returned here, since
|
|
|
|
// bootstrapping the Consul client is elsewhere. Errors in revocation trigger
|
|
|
|
// background retry attempts rather than inline error handling.
|
2020-01-02 15:03:05 +00:00
|
|
|
_ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, true)
|
2019-06-05 14:49:57 +00:00
|
|
|
}
|
2015-07-06 20:42:33 +00:00
|
|
|
|
2019-06-05 14:49:57 +00:00
|
|
|
reply.EvalIDs = append(reply.EvalIDs, evalIDs...)
|
2019-06-26 20:14:36 +00:00
|
|
|
// Set the reply eval create index just the first time
|
2019-06-05 16:45:42 +00:00
|
|
|
if reply.EvalCreateIndex == 0 {
|
|
|
|
reply.EvalCreateIndex = evalIndex
|
|
|
|
}
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
|
|
|
|
2015-08-06 23:39:20 +00:00
|
|
|
reply.NodeModifyIndex = index
|
2015-07-06 20:42:33 +00:00
|
|
|
reply.Index = index
|
2015-06-07 19:14:41 +00:00
|
|
|
return nil
|
|
|
|
}
|
2015-07-06 20:50:40 +00:00
|
|
|
|
2023-01-25 20:53:59 +00:00
|
|
|
// UpdateStatus is used to update the status of a client node.
|
|
|
|
//
|
|
|
|
// Clients with non-terminal allocations must first call UpdateAlloc to be able
|
|
|
|
// to transition from the initializing status to ready.
|
|
|
|
//
|
|
|
|
// ┌────────────────────────────────────── No ───┐
|
|
|
|
// │ │
|
|
|
|
// ┌──▼───┐ ┌─────────────┐ ┌────────┴────────┐
|
|
|
|
// ── Register ─► init ├─ ready ──► Has allocs? ├─ Yes ─► Allocs updated? │
|
|
|
|
// └──▲───┘ └─────┬───────┘ └────────┬────────┘
|
|
|
|
// │ │ │
|
|
|
|
// ready └─ No ─┐ ┌─────── Yes ──┘
|
|
|
|
// │ │ │
|
|
|
|
// ┌──────┴───────┐ ┌──▼──▼─┐ ┌──────┐
|
|
|
|
// │ disconnected ◄─ disconnected ─┤ ready ├─ down ──► down │
|
|
|
|
// └──────────────┘ └───▲───┘ └──┬───┘
|
|
|
|
// │ │
|
|
|
|
// └──── ready ─────┘
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) UpdateStatus(args *structs.NodeUpdateStatusRequest, reply *structs.NodeUpdateResponse) error {
|
2023-01-24 15:52:07 +00:00
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
|
|
|
|
2020-03-18 01:35:56 +00:00
|
|
|
isForwarded := args.IsForwarded()
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.UpdateStatus", args, args, reply); done {
|
2018-03-27 01:10:43 +00:00
|
|
|
// We have a valid node connection since there is no error from the
|
|
|
|
// forwarded server, so add the mapping to cache the
|
|
|
|
// connection and allow the server to send RPCs to the client.
|
2020-03-18 01:35:56 +00:00
|
|
|
if err == nil && n.ctx != nil && n.ctx.NodeID == "" && !isForwarded {
|
2018-03-27 01:10:43 +00:00
|
|
|
n.ctx.NodeID = args.NodeID
|
|
|
|
n.srv.addNodeConn(n.ctx)
|
|
|
|
}
|
|
|
|
|
2015-07-06 20:50:40 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-24 15:52:07 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2015-07-06 20:50:40 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "update_status"}, time.Now())
|
|
|
|
|
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
2016-08-16 06:11:57 +00:00
|
|
|
return fmt.Errorf("missing node ID for client status update")
|
2015-07-06 20:50:40 +00:00
|
|
|
}
|
2015-08-06 23:39:20 +00:00
|
|
|
if !structs.ValidNodeStatus(args.Status) {
|
2015-07-06 20:50:40 +00:00
|
|
|
return fmt.Errorf("invalid status for node")
|
|
|
|
}
|
|
|
|
|
2015-08-23 00:49:48 +00:00
|
|
|
// Look for the node
|
2015-09-07 03:31:32 +00:00
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
2015-07-06 20:50:40 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-02-08 04:31:23 +00:00
|
|
|
|
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
node, err := snap.NodeByID(ws, args.NodeID)
|
2015-08-23 00:49:48 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if node == nil {
|
|
|
|
return fmt.Errorf("node not found")
|
|
|
|
}
|
|
|
|
|
2018-01-05 21:50:04 +00:00
|
|
|
// We have a valid node connection, so add the mapping to cache the
|
2018-03-27 01:10:43 +00:00
|
|
|
// connection and allow the server to send RPCs to the client. We only cache
|
|
|
|
// the connection if it is not being forwarded from another server.
|
|
|
|
if n.ctx != nil && n.ctx.NodeID == "" && !args.IsForwarded() {
|
2018-01-05 21:50:04 +00:00
|
|
|
n.ctx.NodeID = args.NodeID
|
|
|
|
n.srv.addNodeConn(n.ctx)
|
|
|
|
}
|
|
|
|
|
2016-08-16 06:11:57 +00:00
|
|
|
// XXX: Could use the SecretID here but have to update the heartbeat system
|
|
|
|
// to track SecretIDs.
|
|
|
|
|
2016-07-12 17:29:23 +00:00
|
|
|
// Update the timestamp of when the node status was updated
|
2019-05-21 19:45:00 +00:00
|
|
|
args.UpdatedAt = time.Now().Unix()
|
2016-07-12 17:29:23 +00:00
|
|
|
|
2023-01-25 20:53:59 +00:00
|
|
|
// Compute next status.
|
|
|
|
switch node.Status {
|
|
|
|
case structs.NodeStatusInit:
|
|
|
|
if args.Status == structs.NodeStatusReady {
|
|
|
|
allocs, err := snap.AllocsByNodeTerminal(ws, args.NodeID, false)
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to query node allocs: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
allocsUpdated := node.LastAllocUpdateIndex > node.LastMissedHeartbeatIndex
|
|
|
|
if len(allocs) > 0 && !allocsUpdated {
|
|
|
|
args.Status = structs.NodeStatusInit
|
|
|
|
}
|
|
|
|
}
|
|
|
|
case structs.NodeStatusDisconnected:
|
|
|
|
if args.Status == structs.NodeStatusReady {
|
|
|
|
args.Status = structs.NodeStatusInit
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-08-23 00:49:48 +00:00
|
|
|
// Commit this update via Raft
|
|
|
|
var index uint64
|
|
|
|
if node.Status != args.Status {
|
2018-05-12 00:26:25 +00:00
|
|
|
// Attach an event if we are updating the node status to ready when it
|
|
|
|
// is down via a heartbeat
|
|
|
|
if node.Status == structs.NodeStatusDown && args.NodeEvent == nil {
|
|
|
|
args.NodeEvent = structs.NewNodeEvent().
|
|
|
|
SetSubsystem(structs.NodeEventSubsystemCluster).
|
|
|
|
SetMessage(NodeHeartbeatEventReregistered)
|
|
|
|
}
|
|
|
|
|
2015-09-07 03:31:32 +00:00
|
|
|
_, index, err = n.srv.raftApply(structs.NodeUpdateStatusRequestType, args)
|
2015-08-23 00:49:48 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("status update failed", "error", err)
|
2015-08-23 00:49:48 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.NodeModifyIndex = index
|
|
|
|
}
|
2015-08-06 23:39:20 +00:00
|
|
|
|
|
|
|
// Check if we should trigger evaluations
|
2022-04-15 13:47:45 +00:00
|
|
|
if structs.ShouldDrainNode(args.Status) ||
|
|
|
|
nodeStatusTransitionRequiresEval(args.Status, node.Status) {
|
2022-07-06 21:35:18 +00:00
|
|
|
evalIDs, evalIndex, err := n.createNodeEvals(node, index)
|
2015-08-06 23:39:20 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eval creation failed", "error", err)
|
2015-08-06 23:39:20 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.EvalIDs = evalIDs
|
|
|
|
reply.EvalCreateIndex = evalIndex
|
|
|
|
}
|
2015-07-06 20:50:40 +00:00
|
|
|
|
2015-08-23 00:49:48 +00:00
|
|
|
// Check if we need to setup a heartbeat
|
2016-08-22 20:57:27 +00:00
|
|
|
switch args.Status {
|
|
|
|
case structs.NodeStatusDown:
|
2019-12-06 20:46:46 +00:00
|
|
|
// Determine if there are any Vault accessors on the node to cleanup
|
|
|
|
if accessors, err := n.srv.State().VaultAccessorsByNode(ws, args.NodeID); err != nil {
|
|
|
|
n.logger.Error("looking up vault accessors for node failed", "node_id", args.NodeID, "error", err)
|
2016-08-22 20:57:27 +00:00
|
|
|
return err
|
2019-12-06 20:46:46 +00:00
|
|
|
} else if l := len(accessors); l > 0 {
|
|
|
|
n.logger.Debug("revoking vault accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID)
|
|
|
|
if err := n.srv.vault.RevokeTokens(context.Background(), accessors, true); err != nil {
|
|
|
|
n.logger.Error("revoking vault accessors for node failed", "node_id", args.NodeID, "error", err)
|
|
|
|
return err
|
|
|
|
}
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
// Determine if there are any SI token accessors on the node to cleanup
|
|
|
|
if accessors, err := n.srv.State().SITokenAccessorsByNode(ws, args.NodeID); err != nil {
|
2020-01-07 17:58:29 +00:00
|
|
|
n.logger.Error("looking up SI accessors for node failed", "node_id", args.NodeID, "error", err)
|
2019-12-06 20:46:46 +00:00
|
|
|
return err
|
|
|
|
} else if l := len(accessors); l > 0 {
|
2020-01-07 17:58:29 +00:00
|
|
|
n.logger.Debug("revoking SI accessors on node due to down state", "num_accessors", l, "node_id", args.NodeID)
|
2020-01-02 15:03:05 +00:00
|
|
|
_ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, true)
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
2022-03-23 08:42:46 +00:00
|
|
|
|
|
|
|
// Identify the service registrations current placed on the downed
|
|
|
|
// node.
|
|
|
|
serviceRegistrations, err := n.srv.State().GetServiceRegistrationsByNodeID(ws, args.NodeID)
|
|
|
|
if err != nil {
|
|
|
|
n.logger.Error("looking up service registrations for node failed",
|
|
|
|
"node_id", args.NodeID, "error", err)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// If the node has service registrations assigned to it, delete these
|
|
|
|
// via Raft.
|
|
|
|
if l := len(serviceRegistrations); l > 0 {
|
|
|
|
n.logger.Debug("deleting service registrations on node due to down state",
|
|
|
|
"num_service_registrations", l, "node_id", args.NodeID)
|
|
|
|
|
|
|
|
deleteRegReq := structs.ServiceRegistrationDeleteByNodeIDRequest{NodeID: args.NodeID}
|
|
|
|
|
|
|
|
_, index, err = n.srv.raftApply(structs.ServiceRegistrationDeleteByNodeIDRequestType, &deleteRegReq)
|
|
|
|
if err != nil {
|
|
|
|
n.logger.Error("failed to delete service registrations for node",
|
|
|
|
"node_id", args.NodeID, "error", err)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-08-22 20:57:27 +00:00
|
|
|
default:
|
2015-09-07 03:31:32 +00:00
|
|
|
ttl, err := n.srv.resetHeartbeatTimer(args.NodeID)
|
2015-08-23 00:49:48 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("heartbeat reset failed", "error", err)
|
2015-08-23 00:49:48 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.HeartbeatTTL = ttl
|
|
|
|
}
|
|
|
|
|
2016-05-23 18:09:31 +00:00
|
|
|
// Set the reply index and leader
|
2016-06-01 10:47:19 +00:00
|
|
|
reply.Index = index
|
2016-05-23 18:09:31 +00:00
|
|
|
n.srv.peerLock.RLock()
|
|
|
|
defer n.srv.peerLock.RUnlock()
|
2022-09-08 18:31:36 +00:00
|
|
|
if err := n.constructNodeServerInfoResponse(node.GetID(), snap, reply); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("failed to populate NodeUpdateResponse", "error", err)
|
2016-06-01 10:47:19 +00:00
|
|
|
return err
|
2016-05-23 18:09:31 +00:00
|
|
|
}
|
|
|
|
|
2015-07-06 20:50:40 +00:00
|
|
|
return nil
|
|
|
|
}
|
2015-07-06 21:23:15 +00:00
|
|
|
|
2022-04-15 13:47:45 +00:00
|
|
|
// nodeStatusTransitionRequiresEval is a helper that takes a nodes new and old status and
|
2018-03-11 19:06:05 +00:00
|
|
|
// returns whether it has transitioned to ready.
|
2022-04-15 13:47:45 +00:00
|
|
|
func nodeStatusTransitionRequiresEval(newStatus, oldStatus string) bool {
|
2016-07-21 22:22:02 +00:00
|
|
|
initToReady := oldStatus == structs.NodeStatusInit && newStatus == structs.NodeStatusReady
|
|
|
|
terminalToReady := oldStatus == structs.NodeStatusDown && newStatus == structs.NodeStatusReady
|
2022-04-15 13:47:45 +00:00
|
|
|
disconnectedToOther := oldStatus == structs.NodeStatusDisconnected && newStatus != structs.NodeStatusDisconnected
|
2022-04-21 14:05:58 +00:00
|
|
|
otherToDisconnected := oldStatus != structs.NodeStatusDisconnected && newStatus == structs.NodeStatusDisconnected
|
|
|
|
return initToReady || terminalToReady || disconnectedToOther || otherToDisconnected
|
2016-07-21 22:22:02 +00:00
|
|
|
}
|
|
|
|
|
2015-09-07 03:00:12 +00:00
|
|
|
// UpdateDrain is used to update the drain mode of a client node
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) UpdateDrain(args *structs.NodeUpdateDrainRequest,
|
2015-09-07 03:00:12 +00:00
|
|
|
reply *structs.NodeDrainUpdateResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.UpdateDrain", args, args, reply); done {
|
2015-09-07 03:00:12 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2015-09-07 03:00:12 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "update_drain"}, time.Now())
|
|
|
|
|
2017-09-15 03:33:31 +00:00
|
|
|
// Check node write permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
if aclObj, err := n.srv.ResolveACL(args); err != nil {
|
2017-09-15 03:33:31 +00:00
|
|
|
return err
|
|
|
|
} else if aclObj != nil && !aclObj.AllowNodeWrite() {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2015-09-07 03:00:12 +00:00
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID for drain update")
|
|
|
|
}
|
2018-05-10 23:30:54 +00:00
|
|
|
if args.NodeEvent != nil {
|
2018-05-22 21:01:43 +00:00
|
|
|
return fmt.Errorf("node event must not be set")
|
2018-05-10 23:30:54 +00:00
|
|
|
}
|
2015-09-07 03:00:12 +00:00
|
|
|
|
|
|
|
// Look for the node
|
2015-09-07 03:31:32 +00:00
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
2015-09-07 03:00:12 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2018-03-27 22:53:24 +00:00
|
|
|
node, err := snap.NodeByID(nil, args.NodeID)
|
2015-09-07 03:00:12 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if node == nil {
|
|
|
|
return fmt.Errorf("node not found")
|
|
|
|
}
|
|
|
|
|
2019-11-14 21:06:09 +00:00
|
|
|
now := time.Now().UTC()
|
|
|
|
|
2019-05-21 19:45:00 +00:00
|
|
|
// Update the timestamp of when the node status was updated
|
2019-11-14 21:06:09 +00:00
|
|
|
args.UpdatedAt = now.Unix()
|
2019-05-21 19:45:00 +00:00
|
|
|
|
2019-11-14 21:06:09 +00:00
|
|
|
// Setup drain strategy
|
2019-11-13 22:26:35 +00:00
|
|
|
if args.DrainStrategy != nil {
|
2019-11-14 21:06:09 +00:00
|
|
|
// Mark start time for the drain
|
|
|
|
if node.DrainStrategy == nil {
|
|
|
|
args.DrainStrategy.StartedAt = now
|
2019-11-13 22:26:35 +00:00
|
|
|
} else {
|
|
|
|
args.DrainStrategy.StartedAt = node.DrainStrategy.StartedAt
|
|
|
|
}
|
|
|
|
|
2019-11-14 21:06:09 +00:00
|
|
|
// Mark the deadline time
|
|
|
|
if args.DrainStrategy.Deadline.Nanoseconds() > 0 {
|
|
|
|
args.DrainStrategy.ForceDeadline = now.Add(args.DrainStrategy.Deadline)
|
|
|
|
}
|
2018-03-01 00:25:56 +00:00
|
|
|
}
|
|
|
|
|
2018-05-10 23:54:43 +00:00
|
|
|
// Construct the node event
|
|
|
|
args.NodeEvent = structs.NewNodeEvent().SetSubsystem(structs.NodeEventSubsystemDrain)
|
2018-05-30 17:28:46 +00:00
|
|
|
if node.DrainStrategy == nil && args.DrainStrategy != nil {
|
2018-05-10 23:54:43 +00:00
|
|
|
args.NodeEvent.SetMessage(NodeDrainEventDrainSet)
|
|
|
|
} else if node.DrainStrategy != nil && args.DrainStrategy != nil {
|
|
|
|
args.NodeEvent.SetMessage(NodeDrainEventDrainUpdated)
|
|
|
|
} else if node.DrainStrategy != nil && args.DrainStrategy == nil {
|
|
|
|
args.NodeEvent.SetMessage(NodeDrainEventDrainDisabled)
|
2018-06-06 18:02:10 +00:00
|
|
|
} else {
|
|
|
|
args.NodeEvent = nil
|
2018-05-10 23:54:43 +00:00
|
|
|
}
|
|
|
|
|
2015-09-07 03:00:12 +00:00
|
|
|
// Commit this update via Raft
|
2018-02-23 18:42:43 +00:00
|
|
|
_, index, err := n.srv.raftApply(structs.NodeUpdateDrainRequestType, args)
|
2016-04-19 01:43:52 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("drain update failed", "error", err)
|
2016-04-19 01:43:52 +00:00
|
|
|
return err
|
2015-09-07 03:00:12 +00:00
|
|
|
}
|
2018-02-23 18:42:43 +00:00
|
|
|
reply.NodeModifyIndex = index
|
2015-09-07 03:00:12 +00:00
|
|
|
|
2018-04-10 22:02:52 +00:00
|
|
|
// If the node is transitioning to be eligible, create Node evaluations
|
2018-03-27 22:53:24 +00:00
|
|
|
// because there may be a System job registered that should be evaluated.
|
|
|
|
if node.SchedulingEligibility == structs.NodeSchedulingIneligible && args.MarkEligible && args.DrainStrategy == nil {
|
2022-09-06 12:03:33 +00:00
|
|
|
n.logger.Info("node transitioning to eligible state", "node_id", node.ID)
|
2022-07-06 21:35:18 +00:00
|
|
|
evalIDs, evalIndex, err := n.createNodeEvals(node, index)
|
2018-03-27 22:53:24 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eval creation failed", "error", err)
|
2018-03-27 22:53:24 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.EvalIDs = evalIDs
|
|
|
|
reply.EvalCreateIndex = evalIndex
|
|
|
|
}
|
|
|
|
|
2015-09-07 03:00:12 +00:00
|
|
|
// Set the reply index
|
|
|
|
reply.Index = index
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-02-27 00:34:42 +00:00
|
|
|
// UpdateEligibility is used to update the scheduling eligibility of a node
|
|
|
|
func (n *Node) UpdateEligibility(args *structs.NodeUpdateEligibilityRequest,
|
2018-03-27 22:53:24 +00:00
|
|
|
reply *structs.NodeEligibilityUpdateResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2018-02-27 00:34:42 +00:00
|
|
|
if done, err := n.srv.forward("Node.UpdateEligibility", args, args, reply); done {
|
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2018-02-27 00:34:42 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "update_eligibility"}, time.Now())
|
|
|
|
|
|
|
|
// Check node write permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
if aclObj, err := n.srv.ResolveACL(args); err != nil {
|
2018-02-27 00:34:42 +00:00
|
|
|
return err
|
|
|
|
} else if aclObj != nil && !aclObj.AllowNodeWrite() {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID for setting scheduling eligibility")
|
|
|
|
}
|
2018-05-11 21:32:34 +00:00
|
|
|
if args.NodeEvent != nil {
|
2018-05-22 21:02:44 +00:00
|
|
|
return fmt.Errorf("node event must not be set")
|
2018-05-11 21:32:34 +00:00
|
|
|
}
|
2018-02-27 00:34:42 +00:00
|
|
|
|
2018-03-27 22:53:24 +00:00
|
|
|
// Check that only allowed types are set
|
|
|
|
switch args.Eligibility {
|
|
|
|
case structs.NodeSchedulingEligible, structs.NodeSchedulingIneligible:
|
|
|
|
default:
|
|
|
|
return fmt.Errorf("invalid scheduling eligibility %q", args.Eligibility)
|
|
|
|
}
|
|
|
|
|
2018-02-27 00:34:42 +00:00
|
|
|
// Look for the node
|
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2018-03-27 22:53:24 +00:00
|
|
|
node, err := snap.NodeByID(nil, args.NodeID)
|
2018-02-27 00:34:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if node == nil {
|
|
|
|
return fmt.Errorf("node not found")
|
|
|
|
}
|
|
|
|
|
|
|
|
if node.DrainStrategy != nil && args.Eligibility == structs.NodeSchedulingEligible {
|
|
|
|
return fmt.Errorf("can not set node's scheduling eligibility to eligible while it is draining")
|
|
|
|
}
|
|
|
|
|
2018-02-27 20:59:27 +00:00
|
|
|
switch args.Eligibility {
|
|
|
|
case structs.NodeSchedulingEligible, structs.NodeSchedulingIneligible:
|
|
|
|
default:
|
|
|
|
return fmt.Errorf("invalid scheduling eligibility %q", args.Eligibility)
|
|
|
|
}
|
|
|
|
|
2019-05-21 19:45:00 +00:00
|
|
|
// Update the timestamp of when the node status was updated
|
|
|
|
args.UpdatedAt = time.Now().Unix()
|
|
|
|
|
2018-05-11 21:32:34 +00:00
|
|
|
// Construct the node event
|
|
|
|
args.NodeEvent = structs.NewNodeEvent().SetSubsystem(structs.NodeEventSubsystemCluster)
|
|
|
|
if node.SchedulingEligibility == args.Eligibility {
|
|
|
|
return nil // Nothing to do
|
|
|
|
} else if args.Eligibility == structs.NodeSchedulingEligible {
|
2022-09-06 12:03:33 +00:00
|
|
|
n.logger.Info("node transitioning to eligible state", "node_id", node.ID)
|
2018-05-11 21:32:34 +00:00
|
|
|
args.NodeEvent.SetMessage(NodeEligibilityEventEligible)
|
|
|
|
} else {
|
2022-09-06 12:03:33 +00:00
|
|
|
n.logger.Info("node transitioning to ineligible state", "node_id", node.ID)
|
2018-05-11 21:32:34 +00:00
|
|
|
args.NodeEvent.SetMessage(NodeEligibilityEventIneligible)
|
|
|
|
}
|
|
|
|
|
2018-02-27 00:34:42 +00:00
|
|
|
// Commit this update via Raft
|
|
|
|
outErr, index, err := n.srv.raftApply(structs.NodeUpdateEligibilityRequestType, args)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eligibility update failed", "error", err)
|
2018-02-27 00:34:42 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
if outErr != nil {
|
|
|
|
if err, ok := outErr.(error); ok && err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eligibility update failed", "error", err)
|
2018-02-27 00:34:42 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-04-10 22:02:52 +00:00
|
|
|
// If the node is transitioning to be eligible, create Node evaluations
|
2018-03-27 22:53:24 +00:00
|
|
|
// because there may be a System job registered that should be evaluated.
|
|
|
|
if node.SchedulingEligibility == structs.NodeSchedulingIneligible && args.Eligibility == structs.NodeSchedulingEligible {
|
2022-07-06 21:35:18 +00:00
|
|
|
evalIDs, evalIndex, err := n.createNodeEvals(node, index)
|
2018-03-27 22:53:24 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eval creation failed", "error", err)
|
2018-03-27 22:53:24 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.EvalIDs = evalIDs
|
|
|
|
reply.EvalCreateIndex = evalIndex
|
|
|
|
}
|
|
|
|
|
2018-02-27 00:34:42 +00:00
|
|
|
// Set the reply index
|
|
|
|
reply.Index = index
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-08-16 01:20:35 +00:00
|
|
|
// Evaluate is used to force a re-evaluation of the node
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) Evaluate(args *structs.NodeEvaluateRequest, reply *structs.NodeUpdateResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.Evaluate", args, args, reply); done {
|
2015-08-16 01:20:35 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2015-08-16 01:20:35 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "evaluate"}, time.Now())
|
|
|
|
|
2017-09-15 03:41:44 +00:00
|
|
|
// Check node write permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
if aclObj, err := n.srv.ResolveACL(args); err != nil {
|
2017-09-15 03:41:44 +00:00
|
|
|
return err
|
|
|
|
} else if aclObj != nil && !aclObj.AllowNodeWrite() {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2015-08-16 01:20:35 +00:00
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID for evaluation")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Look for the node
|
2015-09-07 03:31:32 +00:00
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
2015-08-16 01:20:35 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
node, err := snap.NodeByID(ws, args.NodeID)
|
2015-08-16 01:20:35 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if node == nil {
|
|
|
|
return fmt.Errorf("node not found")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create the evaluation
|
2022-07-06 21:35:18 +00:00
|
|
|
evalIDs, evalIndex, err := n.createNodeEvals(node, node.ModifyIndex)
|
2015-08-16 01:20:35 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("eval creation failed", "error", err)
|
2015-08-16 01:20:35 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.EvalIDs = evalIDs
|
|
|
|
reply.EvalCreateIndex = evalIndex
|
|
|
|
|
|
|
|
// Set the reply index
|
|
|
|
reply.Index = evalIndex
|
2016-06-01 10:47:19 +00:00
|
|
|
|
|
|
|
n.srv.peerLock.RLock()
|
|
|
|
defer n.srv.peerLock.RUnlock()
|
2022-09-08 18:31:36 +00:00
|
|
|
if err := n.constructNodeServerInfoResponse(node.GetID(), snap, reply); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("failed to populate NodeUpdateResponse", "error", err)
|
2016-06-01 10:47:19 +00:00
|
|
|
return err
|
|
|
|
}
|
2015-08-16 01:20:35 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-10-20 17:57:53 +00:00
|
|
|
// GetNode is used to request information about a specific node
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) GetNode(args *structs.NodeSpecificRequest,
|
2015-07-06 21:23:15 +00:00
|
|
|
reply *structs.SingleNodeResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.GetNode", args, args, reply); done {
|
2015-07-06 21:23:15 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricRead, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2015-07-06 21:23:15 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "get_node"}, time.Now())
|
|
|
|
|
2017-09-15 03:59:18 +00:00
|
|
|
// Check node read permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
aclObj, err := n.srv.ResolveClientOrACL(args)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if aclObj != nil && !aclObj.AllowNodeRead() {
|
2017-09-15 03:59:18 +00:00
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2015-10-29 22:48:44 +00:00
|
|
|
// Setup the blocking query
|
|
|
|
opts := blockingOptions{
|
|
|
|
queryOpts: &args.QueryOptions,
|
|
|
|
queryMeta: &reply.QueryMeta,
|
2017-02-08 04:31:23 +00:00
|
|
|
run: func(ws memdb.WatchSet, state *state.StateStore) error {
|
2015-10-29 22:48:44 +00:00
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Look for the node
|
2017-02-08 04:31:23 +00:00
|
|
|
out, err := state.NodeByID(ws, args.NodeID)
|
2015-10-29 22:48:44 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Setup the output
|
|
|
|
if out != nil {
|
2021-03-26 17:03:15 +00:00
|
|
|
out = out.Sanitize()
|
2021-02-11 15:40:59 +00:00
|
|
|
reply.Node = out
|
2015-10-29 22:48:44 +00:00
|
|
|
reply.Index = out.ModifyIndex
|
|
|
|
} else {
|
|
|
|
// Use the last index that affected the nodes table
|
2017-02-08 04:31:23 +00:00
|
|
|
index, err := state.Index("nodes")
|
2015-10-29 22:48:44 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2016-08-16 06:11:57 +00:00
|
|
|
reply.Node = nil
|
2015-10-29 22:48:44 +00:00
|
|
|
reply.Index = index
|
|
|
|
}
|
|
|
|
|
|
|
|
// Set the query response
|
|
|
|
n.srv.setQueryMeta(&reply.QueryMeta)
|
|
|
|
return nil
|
|
|
|
}}
|
|
|
|
return n.srv.blockingRPC(&opts)
|
2015-07-06 21:23:15 +00:00
|
|
|
}
|
2015-08-06 23:39:20 +00:00
|
|
|
|
2015-10-20 17:57:53 +00:00
|
|
|
// GetAllocs is used to request allocations for a specific node
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) GetAllocs(args *structs.NodeSpecificRequest,
|
2015-08-23 02:17:49 +00:00
|
|
|
reply *structs.NodeAllocsResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.GetAllocs", args, args, reply); done {
|
2015-08-23 02:17:49 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricList, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2015-08-23 02:17:49 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "get_allocs"}, time.Now())
|
|
|
|
|
2017-09-15 04:42:19 +00:00
|
|
|
// Check node read and namespace job read permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
aclObj, err := n.srv.ResolveACL(args)
|
2017-09-15 21:27:11 +00:00
|
|
|
if err != nil {
|
2017-09-15 04:42:19 +00:00
|
|
|
return err
|
2017-09-15 21:27:11 +00:00
|
|
|
}
|
|
|
|
if aclObj != nil && !aclObj.AllowNodeRead() {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
|
|
|
// cache namespace perms
|
|
|
|
readableNamespaces := map[string]bool{}
|
|
|
|
|
|
|
|
// readNS is a caching namespace read-job helper
|
|
|
|
readNS := func(ns string) bool {
|
|
|
|
if aclObj == nil {
|
|
|
|
// ACLs are disabled; everything is readable
|
|
|
|
return true
|
2017-09-15 04:42:19 +00:00
|
|
|
}
|
2017-09-15 21:27:11 +00:00
|
|
|
|
|
|
|
if readable, ok := readableNamespaces[ns]; ok {
|
|
|
|
// cache hit
|
|
|
|
return readable
|
2017-09-15 04:42:19 +00:00
|
|
|
}
|
2017-09-15 21:27:11 +00:00
|
|
|
|
|
|
|
// cache miss
|
|
|
|
readable := aclObj.AllowNsOp(ns, acl.NamespaceCapabilityReadJob)
|
|
|
|
readableNamespaces[ns] = readable
|
|
|
|
return readable
|
2017-09-15 04:42:19 +00:00
|
|
|
}
|
|
|
|
|
2015-08-23 02:17:49 +00:00
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Setup the blocking query
|
|
|
|
opts := blockingOptions{
|
2015-10-29 21:47:39 +00:00
|
|
|
queryOpts: &args.QueryOptions,
|
|
|
|
queryMeta: &reply.QueryMeta,
|
2017-02-08 04:31:23 +00:00
|
|
|
run: func(ws memdb.WatchSet, state *state.StateStore) error {
|
2015-08-23 02:17:49 +00:00
|
|
|
// Look for the node
|
2017-02-08 04:31:23 +00:00
|
|
|
allocs, err := state.AllocsByNode(ws, args.NodeID)
|
2015-08-23 02:17:49 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Setup the output
|
2017-09-15 21:27:11 +00:00
|
|
|
if n := len(allocs); n != 0 {
|
|
|
|
reply.Allocs = make([]*structs.Allocation, 0, n)
|
2015-08-23 02:17:49 +00:00
|
|
|
for _, alloc := range allocs {
|
2017-09-15 21:27:11 +00:00
|
|
|
if readNS(alloc.Namespace) {
|
|
|
|
reply.Allocs = append(reply.Allocs, alloc)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Get the max of all allocs since
|
|
|
|
// subsequent requests need to start
|
|
|
|
// from the latest index
|
2015-08-23 02:17:49 +00:00
|
|
|
reply.Index = maxUint64(reply.Index, alloc.ModifyIndex)
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
reply.Allocs = nil
|
|
|
|
|
|
|
|
// Use the last index that affected the nodes table
|
2017-02-08 04:31:23 +00:00
|
|
|
index, err := state.Index("allocs")
|
2015-08-23 02:17:49 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Must provide non-zero index to prevent blocking
|
2016-01-29 14:29:52 +00:00
|
|
|
// Index 1 is impossible anyways (due to Raft internals)
|
|
|
|
if index == 0 {
|
|
|
|
reply.Index = 1
|
|
|
|
} else {
|
|
|
|
reply.Index = index
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}}
|
|
|
|
return n.srv.blockingRPC(&opts)
|
|
|
|
}
|
|
|
|
|
2016-02-01 21:57:35 +00:00
|
|
|
// GetClientAllocs is used to request a lightweight list of alloc modify indexes
|
2016-01-29 14:29:52 +00:00
|
|
|
// per allocation.
|
|
|
|
func (n *Node) GetClientAllocs(args *structs.NodeSpecificRequest,
|
|
|
|
reply *structs.NodeClientAllocsResponse) error {
|
2023-01-26 20:04:25 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2020-03-18 01:35:56 +00:00
|
|
|
isForwarded := args.IsForwarded()
|
2016-01-29 14:29:52 +00:00
|
|
|
if done, err := n.srv.forward("Node.GetClientAllocs", args, args, reply); done {
|
2018-03-27 01:10:43 +00:00
|
|
|
// We have a valid node connection since there is no error from the
|
|
|
|
// forwarded server, so add the mapping to cache the
|
|
|
|
// connection and allow the server to send RPCs to the client.
|
2020-03-18 01:35:56 +00:00
|
|
|
if err == nil && n.ctx != nil && n.ctx.NodeID == "" && !isForwarded {
|
2018-03-27 01:10:43 +00:00
|
|
|
n.ctx.NodeID = args.NodeID
|
|
|
|
n.srv.addNodeConn(n.ctx)
|
|
|
|
}
|
|
|
|
|
2016-01-29 14:29:52 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-26 20:04:25 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricList, args)
|
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2016-01-29 14:29:52 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "get_client_allocs"}, time.Now())
|
|
|
|
|
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID")
|
|
|
|
}
|
|
|
|
|
2017-10-26 23:59:37 +00:00
|
|
|
// numOldAllocs is used to detect if there is a garbage collection event
|
|
|
|
// that effects the node. When an allocation is garbage collected, that does
|
|
|
|
// not change the modify index changes and thus the query won't unblock,
|
|
|
|
// even though the set of allocations on the node has changed.
|
|
|
|
var numOldAllocs int
|
|
|
|
|
2016-01-29 14:29:52 +00:00
|
|
|
// Setup the blocking query
|
|
|
|
opts := blockingOptions{
|
|
|
|
queryOpts: &args.QueryOptions,
|
|
|
|
queryMeta: &reply.QueryMeta,
|
2017-02-08 04:31:23 +00:00
|
|
|
run: func(ws memdb.WatchSet, state *state.StateStore) error {
|
2016-01-29 14:29:52 +00:00
|
|
|
// Look for the node
|
2017-02-08 04:31:23 +00:00
|
|
|
node, err := state.NodeByID(ws, args.NodeID)
|
2016-01-29 14:29:52 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2016-08-16 06:11:57 +00:00
|
|
|
var allocs []*structs.Allocation
|
|
|
|
if node != nil {
|
|
|
|
if args.SecretID == "" {
|
2017-10-13 18:28:47 +00:00
|
|
|
return fmt.Errorf("missing node secret ID for client status update")
|
2016-08-16 06:11:57 +00:00
|
|
|
} else if args.SecretID != node.SecretID {
|
|
|
|
return fmt.Errorf("node secret ID does not match")
|
|
|
|
}
|
|
|
|
|
2018-01-05 21:50:04 +00:00
|
|
|
// We have a valid node connection, so add the mapping to cache the
|
2018-03-27 01:10:43 +00:00
|
|
|
// connection and allow the server to send RPCs to the client. We only cache
|
|
|
|
// the connection if it is not being forwarded from another server.
|
|
|
|
if n.ctx != nil && n.ctx.NodeID == "" && !args.IsForwarded() {
|
2018-01-05 21:50:04 +00:00
|
|
|
n.ctx.NodeID = args.NodeID
|
|
|
|
n.srv.addNodeConn(n.ctx)
|
|
|
|
}
|
|
|
|
|
2016-08-16 06:11:57 +00:00
|
|
|
var err error
|
2017-02-08 04:31:23 +00:00
|
|
|
allocs, err = state.AllocsByNode(ws, args.NodeID)
|
2016-08-16 06:11:57 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-01-29 14:29:52 +00:00
|
|
|
reply.Allocs = make(map[string]uint64)
|
2017-10-02 19:18:33 +00:00
|
|
|
reply.MigrateTokens = make(map[string]string)
|
2017-10-12 00:04:09 +00:00
|
|
|
|
2017-10-26 23:59:37 +00:00
|
|
|
// preferTableIndex is used to determine whether we should build the
|
|
|
|
// response index based on the full table indexes versus the modify
|
2017-10-31 20:32:31 +00:00
|
|
|
// indexes of the allocations on the specific node. This is
|
|
|
|
// preferred in the case that the node doesn't yet have allocations
|
|
|
|
// or when we detect a GC that effects the node.
|
2017-10-26 23:59:37 +00:00
|
|
|
preferTableIndex := true
|
|
|
|
|
2016-01-29 14:29:52 +00:00
|
|
|
// Setup the output
|
2017-10-26 23:59:37 +00:00
|
|
|
if numAllocs := len(allocs); numAllocs != 0 {
|
|
|
|
preferTableIndex = false
|
|
|
|
|
2016-01-29 14:29:52 +00:00
|
|
|
for _, alloc := range allocs {
|
2016-02-01 21:57:35 +00:00
|
|
|
reply.Allocs[alloc.ID] = alloc.AllocModifyIndex
|
2017-10-02 19:18:33 +00:00
|
|
|
|
2017-10-12 00:04:09 +00:00
|
|
|
// If the allocation is going to do a migration, create a
|
|
|
|
// migration token so that the client can authenticate with
|
|
|
|
// the node hosting the previous allocation.
|
2017-10-10 00:23:26 +00:00
|
|
|
if alloc.ShouldMigrate() {
|
|
|
|
prevAllocation, err := state.AllocByID(ws, alloc.PreviousAllocation)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2017-10-12 00:04:09 +00:00
|
|
|
if prevAllocation != nil && prevAllocation.NodeID != alloc.NodeID {
|
2017-10-10 00:23:26 +00:00
|
|
|
allocNode, err := state.NodeByID(ws, prevAllocation.NodeID)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-10-12 00:04:09 +00:00
|
|
|
if allocNode == nil {
|
|
|
|
// Node must have been GC'd so skip the token
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2018-01-12 21:58:44 +00:00
|
|
|
token, err := structs.GenerateMigrateToken(prevAllocation.ID, allocNode.SecretID)
|
2017-10-10 00:23:26 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.MigrateTokens[alloc.ID] = token
|
|
|
|
}
|
2017-10-02 19:18:33 +00:00
|
|
|
}
|
|
|
|
|
2016-01-29 14:29:52 +00:00
|
|
|
reply.Index = maxUint64(reply.Index, alloc.ModifyIndex)
|
|
|
|
}
|
2017-10-26 23:59:37 +00:00
|
|
|
|
|
|
|
// Determine if we have less allocations than before. This
|
|
|
|
// indicates there was a garbage collection
|
|
|
|
if numAllocs < numOldAllocs {
|
|
|
|
preferTableIndex = true
|
|
|
|
}
|
|
|
|
|
|
|
|
// Store the new number of allocations
|
|
|
|
numOldAllocs = numAllocs
|
|
|
|
}
|
|
|
|
|
|
|
|
if preferTableIndex {
|
2016-01-29 14:29:52 +00:00
|
|
|
// Use the last index that affected the nodes table
|
2017-02-08 04:31:23 +00:00
|
|
|
index, err := state.Index("allocs")
|
2016-01-29 14:29:52 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Must provide non-zero index to prevent blocking
|
2015-08-23 02:17:49 +00:00
|
|
|
// Index 1 is impossible anyways (due to Raft internals)
|
|
|
|
if index == 0 {
|
|
|
|
reply.Index = 1
|
|
|
|
} else {
|
|
|
|
reply.Index = index
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}}
|
2015-09-07 03:31:32 +00:00
|
|
|
return n.srv.blockingRPC(&opts)
|
2015-08-23 02:17:49 +00:00
|
|
|
}
|
|
|
|
|
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 20:25:11 +00:00
|
|
|
// UpdateAlloc is used to update the client status of an allocation. It should
|
|
|
|
// only be called by clients.
|
|
|
|
//
|
2023-01-25 20:53:59 +00:00
|
|
|
// Calling this method returns an error when:
|
|
|
|
// - The node is not registered in the server yet. Clients must first call the
|
|
|
|
// Register method.
|
|
|
|
// - The node status is down or disconnected. Clients must call the
|
|
|
|
// UpdateStatus method to update its status in the server.
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) UpdateAlloc(args *structs.AllocUpdateRequest, reply *structs.GenericResponse) error {
|
2023-01-26 20:04:25 +00:00
|
|
|
|
2023-01-24 15:52:07 +00:00
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
|
|
|
|
2022-02-05 01:35:20 +00:00
|
|
|
// Ensure the connection was initiated by another client if TLS is used.
|
|
|
|
err := validateTLSCertificateLevel(n.srv, n.ctx, tlsCertificateLevelClient)
|
|
|
|
if err != nil {
|
2015-08-26 01:12:51 +00:00
|
|
|
return err
|
|
|
|
}
|
2022-02-05 01:35:20 +00:00
|
|
|
if done, err := n.srv.forward("Node.UpdateAlloc", args, args, reply); done {
|
|
|
|
return err
|
2022-02-02 20:03:18 +00:00
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
2023-01-24 15:52:07 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2022-02-05 01:35:20 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "update_alloc"}, time.Now())
|
2022-02-02 20:03:18 +00:00
|
|
|
|
2016-02-22 05:03:24 +00:00
|
|
|
// Ensure at least a single alloc
|
2016-02-22 02:00:46 +00:00
|
|
|
if len(args.Alloc) == 0 {
|
|
|
|
return fmt.Errorf("must update at least one allocation")
|
2015-08-26 01:12:51 +00:00
|
|
|
}
|
|
|
|
|
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 20:25:11 +00:00
|
|
|
// Ensure the node is allowed to update allocs.
|
|
|
|
// The node needs to successfully heartbeat before updating its allocs.
|
|
|
|
nodeID := args.Alloc[0].NodeID
|
|
|
|
if nodeID == "" {
|
|
|
|
return fmt.Errorf("missing node ID")
|
|
|
|
}
|
|
|
|
|
|
|
|
node, err := n.srv.State().NodeByID(nil, nodeID)
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to retrieve node %s: %v", nodeID, err)
|
|
|
|
}
|
|
|
|
if node == nil {
|
|
|
|
return fmt.Errorf("node %s not found", nodeID)
|
|
|
|
}
|
2023-01-25 20:53:59 +00:00
|
|
|
if node.UnresponsiveStatus() {
|
|
|
|
return fmt.Errorf("node %s is not allowed to update allocs while in status %s", nodeID, node.Status)
|
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 20:25:11 +00:00
|
|
|
}
|
|
|
|
|
2018-01-20 02:48:37 +00:00
|
|
|
// Ensure that evals aren't set from client RPCs
|
|
|
|
// We create them here before the raft update
|
2018-01-16 14:55:35 +00:00
|
|
|
if len(args.Evals) != 0 {
|
2018-02-21 18:58:04 +00:00
|
|
|
return fmt.Errorf("evals field must not be set")
|
2018-01-16 14:55:35 +00:00
|
|
|
}
|
|
|
|
|
2017-10-26 13:52:57 +00:00
|
|
|
// Update modified timestamp for client initiated allocation updates
|
2018-01-17 22:24:57 +00:00
|
|
|
now := time.Now()
|
2018-01-16 14:55:35 +00:00
|
|
|
var evals []*structs.Evaluation
|
|
|
|
|
2020-02-05 18:27:37 +00:00
|
|
|
for _, allocToUpdate := range args.Alloc {
|
2022-04-06 13:33:32 +00:00
|
|
|
evalTriggerBy := ""
|
2020-02-05 18:27:37 +00:00
|
|
|
allocToUpdate.ModifyTime = now.UTC().UnixNano()
|
|
|
|
|
2022-03-31 15:32:18 +00:00
|
|
|
alloc, _ := n.srv.State().AllocByID(nil, allocToUpdate.ID)
|
|
|
|
if alloc == nil {
|
2020-02-05 18:27:37 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2022-03-31 15:32:18 +00:00
|
|
|
if !allocToUpdate.TerminalStatus() && alloc.ClientStatus != structs.AllocClientStatusUnknown {
|
2020-02-05 18:27:37 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2022-04-06 13:33:32 +00:00
|
|
|
var job *structs.Job
|
|
|
|
var jobType string
|
|
|
|
var jobPriority int
|
|
|
|
|
|
|
|
job, err = n.srv.State().JobByID(nil, alloc.Namespace, alloc.JobID)
|
2020-02-05 18:27:37 +00:00
|
|
|
if err != nil {
|
2020-04-05 14:47:40 +00:00
|
|
|
n.logger.Debug("UpdateAlloc unable to find job", "job", alloc.JobID, "error", err)
|
2020-02-05 18:27:37 +00:00
|
|
|
continue
|
|
|
|
}
|
2022-04-06 13:33:32 +00:00
|
|
|
|
|
|
|
// If the job is nil it means it has been de-registered.
|
2020-02-05 18:27:37 +00:00
|
|
|
if job == nil {
|
2022-04-06 13:33:32 +00:00
|
|
|
jobType = alloc.Job.Type
|
|
|
|
jobPriority = alloc.Job.Priority
|
|
|
|
evalTriggerBy = structs.EvalTriggerJobDeregister
|
|
|
|
allocToUpdate.DesiredStatus = structs.AllocDesiredStatusStop
|
|
|
|
n.logger.Debug("UpdateAlloc unable to find job - shutting down alloc", "job", alloc.JobID)
|
2020-02-05 18:27:37 +00:00
|
|
|
}
|
|
|
|
|
2022-04-06 13:33:32 +00:00
|
|
|
var taskGroup *structs.TaskGroup
|
|
|
|
if job != nil {
|
|
|
|
jobType = job.Type
|
|
|
|
jobPriority = job.Priority
|
|
|
|
taskGroup = job.LookupTaskGroup(alloc.TaskGroup)
|
2020-02-05 18:27:37 +00:00
|
|
|
}
|
|
|
|
|
2022-04-06 13:33:32 +00:00
|
|
|
// If we cannot find the task group for a failed alloc we cannot continue, unless it is an orphan.
|
|
|
|
if evalTriggerBy != structs.EvalTriggerJobDeregister &&
|
|
|
|
allocToUpdate.ClientStatus == structs.AllocClientStatusFailed &&
|
|
|
|
alloc.FollowupEvalID == "" {
|
|
|
|
|
|
|
|
if taskGroup == nil {
|
|
|
|
n.logger.Debug("UpdateAlloc unable to find task group for job", "job", alloc.JobID, "alloc", alloc.ID, "task_group", alloc.TaskGroup)
|
|
|
|
continue
|
|
|
|
}
|
2022-03-31 15:32:18 +00:00
|
|
|
|
2022-04-06 13:33:32 +00:00
|
|
|
// Set trigger by failed if not an orphan.
|
|
|
|
if alloc.RescheduleEligible(taskGroup.ReschedulePolicy, now) {
|
|
|
|
evalTriggerBy = structs.EvalTriggerRetryFailedAlloc
|
|
|
|
}
|
2022-03-31 15:32:18 +00:00
|
|
|
}
|
|
|
|
|
2022-04-06 13:33:32 +00:00
|
|
|
var eval *structs.Evaluation
|
|
|
|
// If unknown, and not an orphan, set the trigger by.
|
|
|
|
if evalTriggerBy != structs.EvalTriggerJobDeregister &&
|
|
|
|
alloc.ClientStatus == structs.AllocClientStatusUnknown {
|
2022-03-31 15:32:18 +00:00
|
|
|
evalTriggerBy = structs.EvalTriggerReconnect
|
|
|
|
}
|
|
|
|
|
2022-04-06 13:33:32 +00:00
|
|
|
// If we weren't able to determine one of our expected eval triggers,
|
|
|
|
// continue and don't create an eval.
|
|
|
|
if evalTriggerBy == "" {
|
|
|
|
continue
|
2018-01-16 14:55:35 +00:00
|
|
|
}
|
2022-04-06 13:33:32 +00:00
|
|
|
|
|
|
|
eval = &structs.Evaluation{
|
|
|
|
ID: uuid.Generate(),
|
|
|
|
Namespace: alloc.Namespace,
|
|
|
|
TriggeredBy: evalTriggerBy,
|
|
|
|
JobID: alloc.JobID,
|
|
|
|
Type: jobType,
|
|
|
|
Priority: jobPriority,
|
|
|
|
Status: structs.EvalStatusPending,
|
|
|
|
CreateTime: now.UTC().UnixNano(),
|
|
|
|
ModifyTime: now.UTC().UnixNano(),
|
|
|
|
}
|
|
|
|
evals = append(evals, eval)
|
2018-01-16 14:55:35 +00:00
|
|
|
}
|
2018-04-10 19:00:07 +00:00
|
|
|
|
2017-10-26 13:52:57 +00:00
|
|
|
// Add this to the batch
|
|
|
|
n.updatesLock.Lock()
|
2016-02-22 02:51:34 +00:00
|
|
|
n.updates = append(n.updates, args.Alloc...)
|
2018-04-09 19:05:31 +00:00
|
|
|
n.evals = append(n.evals, evals...)
|
2016-02-22 02:51:34 +00:00
|
|
|
|
|
|
|
// Start a new batch if none
|
|
|
|
future := n.updateFuture
|
|
|
|
if future == nil {
|
2018-03-06 22:37:37 +00:00
|
|
|
future = structs.NewBatchFuture()
|
2016-02-22 02:51:34 +00:00
|
|
|
n.updateFuture = future
|
|
|
|
n.updateTimer = time.AfterFunc(batchUpdateInterval, func() {
|
|
|
|
// Get the pending updates
|
|
|
|
n.updatesLock.Lock()
|
|
|
|
updates := n.updates
|
2018-04-09 19:05:31 +00:00
|
|
|
evals := n.evals
|
2016-02-22 02:51:34 +00:00
|
|
|
future := n.updateFuture
|
2020-11-24 17:14:00 +00:00
|
|
|
|
|
|
|
// Assume future update patterns will be similar to
|
|
|
|
// current batch and set cap appropriately to avoid
|
|
|
|
// slice resizing.
|
|
|
|
n.updates = make([]*structs.Allocation, 0, len(updates))
|
|
|
|
n.evals = make([]*structs.Evaluation, 0, len(evals))
|
|
|
|
|
2016-02-22 02:51:34 +00:00
|
|
|
n.updateFuture = nil
|
|
|
|
n.updateTimer = nil
|
|
|
|
n.updatesLock.Unlock()
|
|
|
|
|
|
|
|
// Perform the batch update
|
2018-01-16 14:55:35 +00:00
|
|
|
n.batchUpdate(future, updates, evals)
|
2016-02-22 02:51:34 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
n.updatesLock.Unlock()
|
|
|
|
|
|
|
|
// Wait for the future
|
|
|
|
if err := future.Wait(); err != nil {
|
2020-08-06 18:51:46 +00:00
|
|
|
return err
|
2015-08-26 01:12:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Setup the response
|
2016-02-22 02:51:34 +00:00
|
|
|
reply.Index = future.Index()
|
2020-08-06 18:51:46 +00:00
|
|
|
return nil
|
2015-08-26 01:12:51 +00:00
|
|
|
}
|
|
|
|
|
2016-02-22 02:51:34 +00:00
|
|
|
// batchUpdate is used to update all the allocations
|
2018-03-06 22:37:37 +00:00
|
|
|
func (n *Node) batchUpdate(future *structs.BatchFuture, updates []*structs.Allocation, evals []*structs.Evaluation) {
|
2022-04-06 13:33:32 +00:00
|
|
|
var mErr multierror.Error
|
2018-04-09 19:05:31 +00:00
|
|
|
// Group pending evals by jobID to prevent creating unnecessary evals
|
2018-04-10 19:00:07 +00:00
|
|
|
evalsByJobId := make(map[structs.NamespacedID]struct{})
|
2018-04-09 19:05:31 +00:00
|
|
|
var trimmedEvals []*structs.Evaluation
|
|
|
|
for _, eval := range evals {
|
|
|
|
namespacedID := structs.NamespacedID{
|
|
|
|
ID: eval.JobID,
|
|
|
|
Namespace: eval.Namespace,
|
|
|
|
}
|
|
|
|
_, exists := evalsByJobId[namespacedID]
|
|
|
|
if !exists {
|
2019-08-07 16:50:35 +00:00
|
|
|
now := time.Now().UTC().UnixNano()
|
|
|
|
eval.CreateTime = now
|
|
|
|
eval.ModifyTime = now
|
2018-04-09 19:05:31 +00:00
|
|
|
trimmedEvals = append(trimmedEvals, eval)
|
2018-04-10 19:00:07 +00:00
|
|
|
evalsByJobId[namespacedID] = struct{}{}
|
2018-04-09 19:05:31 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-04-10 19:00:07 +00:00
|
|
|
if len(trimmedEvals) > 0 {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Debug("adding evaluations for rescheduling failed allocations", "num_evals", len(trimmedEvals))
|
2018-04-10 19:00:07 +00:00
|
|
|
}
|
2016-02-22 02:51:34 +00:00
|
|
|
// Prepare the batch update
|
|
|
|
batch := &structs.AllocUpdateRequest{
|
|
|
|
Alloc: updates,
|
2018-04-09 19:05:31 +00:00
|
|
|
Evals: trimmedEvals,
|
2016-02-22 02:51:34 +00:00
|
|
|
WriteRequest: structs.WriteRequest{Region: n.srv.config.Region},
|
|
|
|
}
|
|
|
|
|
|
|
|
// Commit this update via Raft
|
|
|
|
_, index, err := n.srv.raftApply(structs.AllocClientUpdateRequestType, batch)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("alloc update failed", "error", err)
|
2016-08-22 20:57:27 +00:00
|
|
|
mErr.Errors = append(mErr.Errors, err)
|
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
// For each allocation we are updating, check if we should revoke any
|
|
|
|
// - Vault token accessors
|
|
|
|
// - Service Identity token accessors
|
|
|
|
var (
|
|
|
|
revokeVault []*structs.VaultAccessor
|
|
|
|
revokeSI []*structs.SITokenAccessor
|
|
|
|
)
|
|
|
|
|
2016-08-22 20:57:27 +00:00
|
|
|
for _, alloc := range updates {
|
|
|
|
// Skip any allocation that isn't dead on the client
|
|
|
|
if !alloc.Terminated() {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
2019-12-06 20:46:46 +00:00
|
|
|
|
|
|
|
// Determine if there are any orphaned Vault accessors for the allocation
|
|
|
|
if accessors, err := n.srv.State().VaultAccessorsByAlloc(ws, alloc.ID); err != nil {
|
|
|
|
n.logger.Error("looking up vault accessors for alloc failed", "alloc_id", alloc.ID, "error", err)
|
2016-08-22 20:57:27 +00:00
|
|
|
mErr.Errors = append(mErr.Errors, err)
|
2019-12-06 20:46:46 +00:00
|
|
|
} else {
|
|
|
|
revokeVault = append(revokeVault, accessors...)
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
// Determine if there are any orphaned SI accessors for the allocation
|
|
|
|
if accessors, err := n.srv.State().SITokenAccessorsByAlloc(ws, alloc.ID); err != nil {
|
|
|
|
n.logger.Error("looking up si accessors for alloc failed", "alloc_id", alloc.ID, "error", err)
|
|
|
|
mErr.Errors = append(mErr.Errors, err)
|
|
|
|
} else {
|
|
|
|
revokeSI = append(revokeSI, accessors...)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-02 15:03:05 +00:00
|
|
|
// Revoke any orphaned Vault token accessors
|
2019-12-06 20:46:46 +00:00
|
|
|
if l := len(revokeVault); l > 0 {
|
|
|
|
n.logger.Debug("revoking vault accessors due to terminal allocations", "num_accessors", l)
|
|
|
|
if err := n.srv.vault.RevokeTokens(context.Background(), revokeVault, true); err != nil {
|
|
|
|
n.logger.Error("batched vault accessor revocation failed", "error", err)
|
|
|
|
mErr.Errors = append(mErr.Errors, err)
|
|
|
|
}
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
|
|
|
|
2020-01-02 15:03:05 +00:00
|
|
|
// Revoke any orphaned SI token accessors
|
2019-12-06 20:46:46 +00:00
|
|
|
if l := len(revokeSI); l > 0 {
|
|
|
|
n.logger.Debug("revoking si accessors due to terminal allocations", "num_accessors", l)
|
2020-01-02 15:03:05 +00:00
|
|
|
_ = n.srv.consulACLs.RevokeTokens(context.Background(), revokeSI, true)
|
2016-02-22 02:51:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Respond to the future
|
2016-08-22 20:57:27 +00:00
|
|
|
future.Respond(index, mErr.ErrorOrNil())
|
2016-02-22 02:51:34 +00:00
|
|
|
}
|
|
|
|
|
2015-09-06 21:28:29 +00:00
|
|
|
// List is used to list the available nodes
|
2015-09-07 03:31:32 +00:00
|
|
|
func (n *Node) List(args *structs.NodeListRequest,
|
2015-09-06 21:28:29 +00:00
|
|
|
reply *structs.NodeListResponse) error {
|
2023-01-25 19:33:06 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
2015-09-07 03:31:32 +00:00
|
|
|
if done, err := n.srv.forward("Node.List", args, args, reply); done {
|
2015-09-06 21:28:29 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-25 21:37:24 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricList, args)
|
2023-01-25 19:33:06 +00:00
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2015-09-06 21:28:29 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "list"}, time.Now())
|
|
|
|
|
2017-09-15 05:01:18 +00:00
|
|
|
// Check node read permissions
|
2023-01-25 19:33:06 +00:00
|
|
|
if aclObj, err := n.srv.ResolveACL(args); err != nil {
|
2017-09-15 05:01:18 +00:00
|
|
|
return err
|
|
|
|
} else if aclObj != nil && !aclObj.AllowNodeRead() {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
|
|
|
|
2022-04-21 15:04:33 +00:00
|
|
|
// Set up the blocking query.
|
2015-10-28 18:21:39 +00:00
|
|
|
opts := blockingOptions{
|
2015-10-29 21:47:39 +00:00
|
|
|
queryOpts: &args.QueryOptions,
|
|
|
|
queryMeta: &reply.QueryMeta,
|
2017-02-08 04:31:23 +00:00
|
|
|
run: func(ws memdb.WatchSet, state *state.StateStore) error {
|
2022-04-21 15:04:33 +00:00
|
|
|
|
2017-02-08 04:31:23 +00:00
|
|
|
var err error
|
2015-12-22 22:44:33 +00:00
|
|
|
var iter memdb.ResultIterator
|
|
|
|
if prefix := args.QueryOptions.Prefix; prefix != "" {
|
2017-02-08 04:31:23 +00:00
|
|
|
iter, err = state.NodesByIDPrefix(ws, prefix)
|
2015-12-22 22:44:33 +00:00
|
|
|
} else {
|
2017-02-08 04:31:23 +00:00
|
|
|
iter, err = state.Nodes(ws)
|
2015-12-22 22:44:33 +00:00
|
|
|
}
|
2015-10-28 18:21:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2015-09-06 21:28:29 +00:00
|
|
|
|
2022-04-21 15:04:33 +00:00
|
|
|
// Generate the tokenizer to use for pagination using the populated
|
|
|
|
// paginatorOpts object. The ID of a node must be unique within the
|
|
|
|
// region, therefore we only need WithID on the paginator options.
|
|
|
|
tokenizer := paginator.NewStructsTokenizer(iter, paginator.StructsTokenizerOptions{WithID: true})
|
|
|
|
|
2015-10-28 19:29:06 +00:00
|
|
|
var nodes []*structs.NodeListStub
|
2022-04-21 15:04:33 +00:00
|
|
|
|
|
|
|
// Build the paginator. This includes the function that is
|
|
|
|
// responsible for appending a node to the nodes array.
|
|
|
|
paginatorImpl, err := paginator.NewPaginator(iter, tokenizer, nil, args.QueryOptions,
|
|
|
|
func(raw interface{}) error {
|
|
|
|
nodes = append(nodes, raw.(*structs.Node).Stub(args.Fields))
|
|
|
|
return nil
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
return structs.NewErrRPCCodedf(
|
|
|
|
http.StatusBadRequest, "failed to create result paginator: %v", err)
|
2015-10-28 18:21:39 +00:00
|
|
|
}
|
2022-04-21 15:04:33 +00:00
|
|
|
|
|
|
|
// Calling page populates our output nodes array as well as returns
|
|
|
|
// the next token.
|
|
|
|
nextToken, err := paginatorImpl.Page()
|
|
|
|
if err != nil {
|
|
|
|
return structs.NewErrRPCCodedf(
|
|
|
|
http.StatusBadRequest, "failed to read result page: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Populate the reply.
|
2015-10-28 19:29:06 +00:00
|
|
|
reply.Nodes = nodes
|
2022-04-21 15:04:33 +00:00
|
|
|
reply.NextToken = nextToken
|
2015-09-06 21:28:29 +00:00
|
|
|
|
2015-10-28 18:21:39 +00:00
|
|
|
// Use the last index that affected the jobs table
|
2017-02-08 04:31:23 +00:00
|
|
|
index, err := state.Index("nodes")
|
2015-10-28 18:21:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
reply.Index = index
|
|
|
|
|
|
|
|
// Set the query response
|
|
|
|
n.srv.setQueryMeta(&reply.QueryMeta)
|
|
|
|
return nil
|
|
|
|
}}
|
|
|
|
return n.srv.blockingRPC(&opts)
|
2015-09-06 21:28:29 +00:00
|
|
|
}
|
|
|
|
|
2015-08-06 23:39:20 +00:00
|
|
|
// createNodeEvals is used to create evaluations for each alloc on a node.
|
|
|
|
// Each Eval is scoped to a job, so we need to potentially trigger many evals.
|
2022-07-06 21:35:18 +00:00
|
|
|
func (n *Node) createNodeEvals(node *structs.Node, nodeIndex uint64) ([]string, uint64, error) {
|
|
|
|
nodeID := node.ID
|
|
|
|
|
2015-08-06 23:39:20 +00:00
|
|
|
// Snapshot the state
|
2015-09-07 03:31:32 +00:00
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
2015-08-06 23:39:20 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, 0, fmt.Errorf("failed to snapshot state: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Find all the allocations for this node
|
2022-07-06 21:35:18 +00:00
|
|
|
allocs, err := snap.AllocsByNode(nil, nodeID)
|
2015-08-06 23:39:20 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, 0, fmt.Errorf("failed to find allocs for '%s': %v", nodeID, err)
|
|
|
|
}
|
|
|
|
|
2022-07-06 21:35:18 +00:00
|
|
|
sysJobsIter, err := snap.JobsByScheduler(nil, "system")
|
2015-10-20 17:57:53 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, 0, fmt.Errorf("failed to find system jobs for '%s': %v", nodeID, err)
|
|
|
|
}
|
2015-10-21 00:11:57 +00:00
|
|
|
|
|
|
|
var sysJobs []*structs.Job
|
2022-07-06 21:35:18 +00:00
|
|
|
for jobI := sysJobsIter.Next(); jobI != nil; jobI = sysJobsIter.Next() {
|
|
|
|
job := jobI.(*structs.Job)
|
|
|
|
// Avoid creating evals for jobs that don't run in this
|
|
|
|
// datacenter. We could perform an entire feasibility check
|
|
|
|
// here, but datacenter is a good optimization to start with as
|
|
|
|
// datacenter cardinality tends to be low so the check
|
|
|
|
// shouldn't add much work.
|
2023-03-07 15:05:59 +00:00
|
|
|
if node.IsInAnyDC(job.Datacenters) {
|
|
|
|
sysJobs = append(sysJobs, job)
|
2022-07-06 21:35:18 +00:00
|
|
|
}
|
2015-10-21 00:11:57 +00:00
|
|
|
}
|
2015-10-20 17:57:53 +00:00
|
|
|
|
2015-08-06 23:39:20 +00:00
|
|
|
// Fast-path if nothing to do
|
2015-10-21 00:11:57 +00:00
|
|
|
if len(allocs) == 0 && len(sysJobs) == 0 {
|
2015-08-06 23:39:20 +00:00
|
|
|
return nil, 0, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create an eval for each JobID affected
|
|
|
|
var evals []*structs.Evaluation
|
|
|
|
var evalIDs []string
|
2021-08-18 13:50:37 +00:00
|
|
|
jobIDs := map[structs.NamespacedID]struct{}{}
|
2019-08-07 16:50:35 +00:00
|
|
|
now := time.Now().UTC().UnixNano()
|
2015-08-06 23:39:20 +00:00
|
|
|
|
|
|
|
for _, alloc := range allocs {
|
|
|
|
// Deduplicate on JobID
|
2021-08-18 13:50:37 +00:00
|
|
|
if _, ok := jobIDs[alloc.JobNamespacedID()]; ok {
|
2015-08-06 23:39:20 +00:00
|
|
|
continue
|
|
|
|
}
|
2021-08-18 13:50:37 +00:00
|
|
|
jobIDs[alloc.JobNamespacedID()] = struct{}{}
|
2015-08-06 23:39:20 +00:00
|
|
|
|
|
|
|
// Create a new eval
|
|
|
|
eval := &structs.Evaluation{
|
2017-09-29 16:58:48 +00:00
|
|
|
ID: uuid.Generate(),
|
2017-09-07 23:56:15 +00:00
|
|
|
Namespace: alloc.Namespace,
|
2015-08-06 23:39:20 +00:00
|
|
|
Priority: alloc.Job.Priority,
|
|
|
|
Type: alloc.Job.Type,
|
|
|
|
TriggeredBy: structs.EvalTriggerNodeUpdate,
|
|
|
|
JobID: alloc.JobID,
|
|
|
|
NodeID: nodeID,
|
|
|
|
NodeModifyIndex: nodeIndex,
|
|
|
|
Status: structs.EvalStatusPending,
|
2019-08-07 16:50:35 +00:00
|
|
|
CreateTime: now,
|
|
|
|
ModifyTime: now,
|
2015-08-06 23:39:20 +00:00
|
|
|
}
|
2022-03-31 15:32:18 +00:00
|
|
|
|
2015-08-06 23:39:20 +00:00
|
|
|
evals = append(evals, eval)
|
|
|
|
evalIDs = append(evalIDs, eval.ID)
|
|
|
|
}
|
|
|
|
|
2015-10-20 17:57:53 +00:00
|
|
|
// Create an evaluation for each system job.
|
2015-10-20 20:02:55 +00:00
|
|
|
for _, job := range sysJobs {
|
2015-10-20 17:57:53 +00:00
|
|
|
// Still dedup on JobID as the node may already have the system job.
|
2021-08-18 13:50:37 +00:00
|
|
|
if _, ok := jobIDs[job.NamespacedID()]; ok {
|
2015-10-20 17:57:53 +00:00
|
|
|
continue
|
|
|
|
}
|
2021-08-18 13:50:37 +00:00
|
|
|
jobIDs[job.NamespacedID()] = struct{}{}
|
2015-10-20 17:57:53 +00:00
|
|
|
|
|
|
|
// Create a new eval
|
|
|
|
eval := &structs.Evaluation{
|
2017-09-29 16:58:48 +00:00
|
|
|
ID: uuid.Generate(),
|
2017-09-07 23:56:15 +00:00
|
|
|
Namespace: job.Namespace,
|
2015-10-20 20:02:55 +00:00
|
|
|
Priority: job.Priority,
|
|
|
|
Type: job.Type,
|
2015-10-20 17:57:53 +00:00
|
|
|
TriggeredBy: structs.EvalTriggerNodeUpdate,
|
2015-10-20 20:02:55 +00:00
|
|
|
JobID: job.ID,
|
2015-10-20 17:57:53 +00:00
|
|
|
NodeID: nodeID,
|
|
|
|
NodeModifyIndex: nodeIndex,
|
|
|
|
Status: structs.EvalStatusPending,
|
2019-08-07 16:50:35 +00:00
|
|
|
CreateTime: now,
|
|
|
|
ModifyTime: now,
|
2015-10-20 17:57:53 +00:00
|
|
|
}
|
|
|
|
evals = append(evals, eval)
|
|
|
|
evalIDs = append(evalIDs, eval.ID)
|
|
|
|
}
|
|
|
|
|
2015-08-06 23:39:20 +00:00
|
|
|
// Create the Raft transaction
|
|
|
|
update := &structs.EvalUpdateRequest{
|
|
|
|
Evals: evals,
|
2015-09-07 03:31:32 +00:00
|
|
|
WriteRequest: structs.WriteRequest{Region: n.srv.config.Region},
|
2015-08-06 23:39:20 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Commit this evaluation via Raft
|
2015-08-16 01:03:05 +00:00
|
|
|
// XXX: There is a risk of partial failure where the node update succeeds
|
|
|
|
// but that the EvalUpdate does not.
|
2015-09-07 03:31:32 +00:00
|
|
|
_, evalIndex, err := n.srv.raftApply(structs.EvalUpdateRequestType, update)
|
2015-08-06 23:39:20 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, 0, err
|
|
|
|
}
|
|
|
|
return evalIDs, evalIndex, nil
|
|
|
|
}
|
2016-02-22 02:51:34 +00:00
|
|
|
|
2016-08-18 17:50:47 +00:00
|
|
|
// DeriveVaultToken is used by the clients to request wrapped Vault tokens for
|
|
|
|
// tasks
|
2019-12-06 20:46:46 +00:00
|
|
|
func (n *Node) DeriveVaultToken(args *structs.DeriveVaultTokenRequest, reply *structs.DeriveVaultTokenResponse) error {
|
2023-01-26 20:04:25 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
setError := func(e error, recoverable bool) {
|
|
|
|
if e != nil {
|
|
|
|
if re, ok := e.(*structs.RecoverableError); ok {
|
|
|
|
reply.Error = re // No need to wrap if error is already a RecoverableError
|
|
|
|
} else {
|
|
|
|
reply.Error = structs.NewRecoverableError(e, recoverable).(*structs.RecoverableError)
|
|
|
|
}
|
|
|
|
n.logger.Error("DeriveVaultToken failed", "recoverable", recoverable, "error", e)
|
2018-03-13 22:09:03 +00:00
|
|
|
}
|
2016-10-28 22:50:35 +00:00
|
|
|
}
|
|
|
|
|
2016-08-18 17:50:47 +00:00
|
|
|
if done, err := n.srv.forward("Node.DeriveVaultToken", args, args, reply); done {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(err, structs.IsRecoverable(err) || err == structs.ErrNoLeader)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
2023-01-26 20:04:25 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2016-08-18 17:50:47 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "derive_vault_token"}, time.Now())
|
|
|
|
|
|
|
|
// Verify the arguments
|
|
|
|
if args.NodeID == "" {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("missing node ID"), false)
|
2016-10-28 22:50:35 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
if args.SecretID == "" {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("missing node SecretID"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
if args.AllocID == "" {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("missing allocation ID"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
if len(args.Tasks) == 0 {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("no tasks specified"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Verify the following:
|
|
|
|
// * The Node exists and has the correct SecretID
|
2019-12-06 20:46:46 +00:00
|
|
|
// * The Allocation exists on the specified Node
|
|
|
|
// * The Allocation contains the given tasks and they each require Vault
|
2016-08-18 17:50:47 +00:00
|
|
|
// tokens
|
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
|
|
|
if err != nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(err, false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
node, err := snap.NodeByID(ws, args.NodeID)
|
2016-08-18 17:50:47 +00:00
|
|
|
if err != nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(err, false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
if node == nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("Node %q does not exist", args.NodeID), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
2016-08-19 01:57:33 +00:00
|
|
|
if node.SecretID != args.SecretID {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("SecretID mismatch"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-19 01:57:33 +00:00
|
|
|
}
|
2016-08-18 17:50:47 +00:00
|
|
|
|
2017-02-08 04:31:23 +00:00
|
|
|
alloc, err := snap.AllocByID(ws, args.AllocID)
|
2016-08-18 17:50:47 +00:00
|
|
|
if err != nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(err, false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
if alloc == nil {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("Allocation %q does not exist", args.AllocID), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
if alloc.NodeID != args.NodeID {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("Allocation %q not running on Node %q", args.AllocID, args.NodeID), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
2016-08-19 20:13:51 +00:00
|
|
|
if alloc.TerminalStatus() {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf("Can't request Vault token for terminal allocation"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-19 20:13:51 +00:00
|
|
|
}
|
2016-08-18 17:50:47 +00:00
|
|
|
|
2022-04-05 18:18:10 +00:00
|
|
|
// Check if alloc has Vault
|
|
|
|
vaultBlocks := alloc.Job.Vault()
|
|
|
|
if vaultBlocks == nil {
|
|
|
|
setError(fmt.Errorf("Job does not require Vault token"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
2022-04-05 18:18:10 +00:00
|
|
|
tg, ok := vaultBlocks[alloc.TaskGroup]
|
2016-08-18 17:50:47 +00:00
|
|
|
if !ok {
|
2022-04-05 18:18:10 +00:00
|
|
|
setError(fmt.Errorf("Task group does not require Vault token"), false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var unneeded []string
|
|
|
|
for _, task := range args.Tasks {
|
|
|
|
taskVault := tg[task]
|
2016-08-19 20:13:51 +00:00
|
|
|
if taskVault == nil || len(taskVault.Policies) == 0 {
|
2016-08-18 17:50:47 +00:00
|
|
|
unneeded = append(unneeded, task)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(unneeded) != 0 {
|
2016-10-23 01:08:30 +00:00
|
|
|
e := fmt.Errorf("Requested Vault tokens for tasks without defined Vault policies: %s",
|
2016-08-18 17:50:47 +00:00
|
|
|
strings.Join(unneeded, ", "))
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(e, false)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-18 17:50:47 +00:00
|
|
|
}
|
|
|
|
|
2016-08-18 21:31:44 +00:00
|
|
|
// At this point the request is valid and we should contact Vault for
|
|
|
|
// tokens.
|
|
|
|
|
|
|
|
// Create an error group where we will spin up a fixed set of goroutines to
|
|
|
|
// handle deriving tokens but where if any fails the whole group is
|
|
|
|
// canceled.
|
|
|
|
g, ctx := errgroup.WithContext(context.Background())
|
|
|
|
|
|
|
|
// Cap the handlers
|
|
|
|
handlers := len(args.Tasks)
|
|
|
|
if handlers > maxParallelRequestsPerDerive {
|
|
|
|
handlers = maxParallelRequestsPerDerive
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create the Vault Tokens
|
|
|
|
input := make(chan string, handlers)
|
|
|
|
results := make(map[string]*vapi.Secret, len(args.Tasks))
|
|
|
|
for i := 0; i < handlers; i++ {
|
|
|
|
g.Go(func() error {
|
2016-08-20 02:55:06 +00:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case task, ok := <-input:
|
|
|
|
if !ok {
|
|
|
|
return nil
|
|
|
|
}
|
2016-08-18 21:31:44 +00:00
|
|
|
|
2016-08-20 02:55:06 +00:00
|
|
|
secret, err := n.srv.vault.CreateToken(ctx, alloc, task)
|
|
|
|
if err != nil {
|
2018-03-13 22:09:03 +00:00
|
|
|
return err
|
2016-08-20 02:55:06 +00:00
|
|
|
}
|
2016-08-18 21:31:44 +00:00
|
|
|
|
2016-08-20 02:55:06 +00:00
|
|
|
results[task] = secret
|
|
|
|
case <-ctx.Done():
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
2016-08-18 21:31:44 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
// Send the input
|
|
|
|
go func() {
|
2016-08-20 02:55:06 +00:00
|
|
|
defer close(input)
|
2016-08-18 21:31:44 +00:00
|
|
|
for _, task := range args.Tasks {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case input <- task:
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Wait for everything to complete or for an error
|
2016-10-23 01:08:30 +00:00
|
|
|
createErr := g.Wait()
|
2016-08-19 01:57:33 +00:00
|
|
|
|
2016-10-23 01:08:30 +00:00
|
|
|
// Retrieve the results
|
2016-08-19 01:57:33 +00:00
|
|
|
accessors := make([]*structs.VaultAccessor, 0, len(results))
|
|
|
|
tokens := make(map[string]string, len(results))
|
|
|
|
for task, secret := range results {
|
|
|
|
w := secret.WrapInfo
|
|
|
|
tokens[task] = w.Token
|
|
|
|
accessor := &structs.VaultAccessor{
|
|
|
|
Accessor: w.WrappedAccessor,
|
|
|
|
Task: task,
|
|
|
|
NodeID: alloc.NodeID,
|
|
|
|
AllocID: alloc.ID,
|
|
|
|
CreationTTL: w.TTL,
|
|
|
|
}
|
2016-08-18 17:50:47 +00:00
|
|
|
|
2016-08-19 01:57:33 +00:00
|
|
|
accessors = append(accessors, accessor)
|
|
|
|
}
|
|
|
|
|
2016-08-22 20:57:27 +00:00
|
|
|
// If there was an error revoke the created tokens
|
2016-10-23 01:08:30 +00:00
|
|
|
if createErr != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("Vault token creation for alloc failed", "alloc_id", alloc.ID, "error", createErr)
|
2016-10-28 22:50:35 +00:00
|
|
|
|
2016-10-23 01:08:30 +00:00
|
|
|
if revokeErr := n.srv.vault.RevokeTokens(context.Background(), accessors, false); revokeErr != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("Vault token revocation for alloc failed", "alloc_id", alloc.ID, "error", revokeErr)
|
2016-10-23 01:08:30 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if rerr, ok := createErr.(*structs.RecoverableError); ok {
|
|
|
|
reply.Error = rerr
|
2017-03-29 20:59:43 +00:00
|
|
|
} else {
|
2017-02-01 21:18:12 +00:00
|
|
|
reply.Error = structs.NewRecoverableError(createErr, false).(*structs.RecoverableError)
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
2016-10-23 01:08:30 +00:00
|
|
|
|
|
|
|
return nil
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
|
|
|
|
2016-10-23 01:08:30 +00:00
|
|
|
// Commit to Raft before returning any of the tokens
|
2016-08-22 20:57:27 +00:00
|
|
|
req := structs.VaultAccessorsRequest{Accessors: accessors}
|
2016-08-19 01:57:33 +00:00
|
|
|
_, index, err := n.srv.raftApply(structs.VaultAccessorRegisterRequestType, &req)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("registering Vault accessors for alloc failed", "alloc_id", alloc.ID, "error", err)
|
2016-10-23 01:08:30 +00:00
|
|
|
|
|
|
|
// Determine if we can recover from the error
|
|
|
|
retry := false
|
|
|
|
switch err {
|
|
|
|
case raft.ErrNotLeader, raft.ErrLeadershipLost, raft.ErrRaftShutdown, raft.ErrEnqueueTimeout:
|
|
|
|
retry = true
|
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(err, retry)
|
2016-10-23 01:08:30 +00:00
|
|
|
return nil
|
2016-08-19 01:57:33 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
reply.Index = index
|
|
|
|
reply.Tasks = tokens
|
|
|
|
n.srv.setQueryMeta(&reply.QueryMeta)
|
2016-08-18 17:50:47 +00:00
|
|
|
return nil
|
|
|
|
}
|
2018-03-14 00:52:12 +00:00
|
|
|
|
2020-05-18 19:21:12 +00:00
|
|
|
type connectTask struct {
|
|
|
|
TaskKind structs.TaskKind
|
|
|
|
TaskName string
|
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
func (n *Node) DeriveSIToken(args *structs.DeriveSITokenRequest, reply *structs.DeriveSITokenResponse) error {
|
2023-01-26 20:04:25 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
setError := func(e error, recoverable bool) {
|
|
|
|
if e != nil {
|
|
|
|
if re, ok := e.(*structs.RecoverableError); ok {
|
|
|
|
reply.Error = re // No need to wrap if error is already a RecoverableError
|
|
|
|
} else {
|
|
|
|
reply.Error = structs.NewRecoverableError(e, recoverable).(*structs.RecoverableError)
|
|
|
|
}
|
|
|
|
n.logger.Error("DeriveSIToken failed", "recoverable", recoverable, "error", e)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if done, err := n.srv.forward("Node.DeriveSIToken", args, args, reply); done {
|
|
|
|
setError(err, structs.IsRecoverable(err) || err == structs.ErrNoLeader)
|
|
|
|
return nil
|
|
|
|
}
|
2023-01-26 20:04:25 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2019-12-06 20:46:46 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "derive_si_token"}, time.Now())
|
|
|
|
|
|
|
|
// Verify the arguments
|
|
|
|
if err := args.Validate(); err != nil {
|
|
|
|
setError(err, false)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Get the ClusterID
|
|
|
|
clusterID, err := n.srv.ClusterID()
|
|
|
|
if err != nil {
|
|
|
|
setError(err, false)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Verify the following:
|
|
|
|
// * The Node exists and has the correct SecretID.
|
|
|
|
// * The Allocation exists on the specified Node.
|
|
|
|
// * The Allocation contains the given tasks, and each task requires a
|
|
|
|
// SI token.
|
|
|
|
|
|
|
|
snap, err := n.srv.fsm.State().Snapshot()
|
|
|
|
if err != nil {
|
|
|
|
setError(err, false)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
node, err := snap.NodeByID(nil, args.NodeID)
|
|
|
|
if err != nil {
|
|
|
|
setError(err, false)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if node == nil {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(fmt.Errorf("Node %q does not exist", args.NodeID), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if node.SecretID != args.SecretID {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(errors.New("SecretID mismatch"), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
alloc, err := snap.AllocByID(nil, args.AllocID)
|
|
|
|
if err != nil {
|
|
|
|
setError(err, false)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if alloc == nil {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(fmt.Errorf("Allocation %q does not exist", args.AllocID), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if alloc.NodeID != args.NodeID {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(fmt.Errorf("Allocation %q not running on node %q", args.AllocID, args.NodeID), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if alloc.TerminalStatus() {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(errors.New("Cannot request SI token for terminal allocation"), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// make sure task group contains at least one connect enabled service
|
|
|
|
tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup)
|
|
|
|
if tg == nil {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(fmt.Errorf("Allocation %q does not contain TaskGroup %q", args.AllocID, alloc.TaskGroup), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
2020-01-30 16:49:07 +00:00
|
|
|
if !tg.UsesConnect() {
|
2022-04-02 00:24:02 +00:00
|
|
|
setError(fmt.Errorf("TaskGroup %q does not use Connect", tg.Name), false)
|
2019-12-06 20:46:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// make sure each task in args.Tasks is a connect-enabled task
|
2020-05-18 19:21:12 +00:00
|
|
|
notConnect, tasks := connectTasks(tg, args.Tasks)
|
|
|
|
if len(notConnect) > 0 {
|
2019-12-06 20:46:46 +00:00
|
|
|
setError(fmt.Errorf(
|
|
|
|
"Requested Consul Service Identity tokens for tasks that are not Connect enabled: %v",
|
2020-05-18 19:21:12 +00:00
|
|
|
strings.Join(notConnect, ", "),
|
2019-12-06 20:46:46 +00:00
|
|
|
), false)
|
|
|
|
}
|
|
|
|
|
|
|
|
// At this point the request is valid and we should contact Consul for tokens.
|
|
|
|
|
|
|
|
// A lot of the following is copied from DeriveVaultToken which has been
|
|
|
|
// working fine for years.
|
|
|
|
|
|
|
|
// Create an error group where we will spin up a fixed set of goroutines to
|
|
|
|
// handle deriving tokens but where if any fails the whole group is
|
|
|
|
// canceled.
|
|
|
|
g, ctx := errgroup.WithContext(context.Background())
|
|
|
|
|
|
|
|
// Cap the worker threads
|
|
|
|
numWorkers := len(args.Tasks)
|
|
|
|
if numWorkers > maxParallelRequestsPerDerive {
|
|
|
|
numWorkers = maxParallelRequestsPerDerive
|
|
|
|
}
|
|
|
|
|
|
|
|
// would like to pull some of this out...
|
|
|
|
|
2020-05-18 19:21:12 +00:00
|
|
|
// Create the SI tokens from a slice of task name + connect service
|
|
|
|
input := make(chan connectTask, numWorkers)
|
2019-12-06 20:46:46 +00:00
|
|
|
results := make(map[string]*structs.SIToken, numWorkers)
|
|
|
|
for i := 0; i < numWorkers; i++ {
|
|
|
|
g.Go(func() error {
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case task, ok := <-input:
|
|
|
|
if !ok {
|
|
|
|
return nil
|
|
|
|
}
|
2020-05-18 19:21:12 +00:00
|
|
|
secret, err := n.srv.consulACLs.CreateToken(ctx, ServiceIdentityRequest{
|
2021-03-16 18:22:21 +00:00
|
|
|
ConsulNamespace: tg.Consul.GetNamespace(),
|
|
|
|
TaskKind: task.TaskKind,
|
|
|
|
TaskName: task.TaskName,
|
|
|
|
ClusterID: clusterID,
|
|
|
|
AllocID: alloc.ID,
|
2020-05-18 19:21:12 +00:00
|
|
|
})
|
2019-12-06 20:46:46 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-05-18 19:21:12 +00:00
|
|
|
results[task.TaskName] = secret
|
2019-12-06 20:46:46 +00:00
|
|
|
case <-ctx.Done():
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
// Send the input
|
|
|
|
go func() {
|
|
|
|
defer close(input)
|
2020-05-18 19:21:12 +00:00
|
|
|
for _, connectTask := range tasks {
|
2019-12-06 20:46:46 +00:00
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
2020-05-18 19:21:12 +00:00
|
|
|
case input <- connectTask:
|
2019-12-06 20:46:46 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Wait for everything to complete or for an error
|
|
|
|
createErr := g.Wait()
|
|
|
|
|
|
|
|
accessors := make([]*structs.SITokenAccessor, 0, len(results))
|
|
|
|
tokens := make(map[string]string, len(results))
|
|
|
|
for task, secret := range results {
|
|
|
|
tokens[task] = secret.SecretID
|
|
|
|
accessor := &structs.SITokenAccessor{
|
2021-03-16 18:22:21 +00:00
|
|
|
ConsulNamespace: tg.Consul.GetNamespace(),
|
|
|
|
NodeID: alloc.NodeID,
|
|
|
|
AllocID: alloc.ID,
|
|
|
|
TaskName: task,
|
|
|
|
AccessorID: secret.AccessorID,
|
2019-12-06 20:46:46 +00:00
|
|
|
}
|
|
|
|
accessors = append(accessors, accessor)
|
|
|
|
}
|
|
|
|
|
2020-01-02 15:03:05 +00:00
|
|
|
// If there was an error, revoke all created tokens. These tokens have not
|
|
|
|
// yet been committed to the persistent store.
|
2019-12-06 20:46:46 +00:00
|
|
|
if createErr != nil {
|
|
|
|
n.logger.Error("Consul Service Identity token creation for alloc failed", "alloc_id", alloc.ID, "error", createErr)
|
2020-01-02 15:03:05 +00:00
|
|
|
_ = n.srv.consulACLs.RevokeTokens(context.Background(), accessors, false)
|
2019-12-06 20:46:46 +00:00
|
|
|
|
|
|
|
if recoverable, ok := createErr.(*structs.RecoverableError); ok {
|
|
|
|
reply.Error = recoverable
|
|
|
|
} else {
|
|
|
|
reply.Error = structs.NewRecoverableError(createErr, false).(*structs.RecoverableError)
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Commit the derived tokens to raft before returning them
|
|
|
|
requested := structs.SITokenAccessorsRequest{Accessors: accessors}
|
|
|
|
_, index, err := n.srv.raftApply(structs.ServiceIdentityAccessorRegisterRequestType, &requested)
|
|
|
|
if err != nil {
|
|
|
|
n.logger.Error("registering Service Identity token accessors for alloc failed", "alloc_id", alloc.ID, "error", err)
|
|
|
|
|
|
|
|
// Determine if we can recover from the error
|
|
|
|
retry := false
|
|
|
|
switch err {
|
|
|
|
case raft.ErrNotLeader, raft.ErrLeadershipLost, raft.ErrRaftShutdown, raft.ErrEnqueueTimeout:
|
|
|
|
retry = true
|
|
|
|
}
|
|
|
|
setError(err, retry)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// We made it! Now we can set the reply.
|
|
|
|
reply.Index = index
|
|
|
|
reply.Tokens = tokens
|
|
|
|
n.srv.setQueryMeta(&reply.QueryMeta)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-05-18 19:21:12 +00:00
|
|
|
func connectTasks(tg *structs.TaskGroup, tasks []string) ([]string, []connectTask) {
|
|
|
|
var notConnect []string
|
|
|
|
var usesConnect []connectTask
|
2019-12-06 20:46:46 +00:00
|
|
|
for _, task := range tasks {
|
|
|
|
tgTask := tg.LookupTask(task)
|
|
|
|
if !taskUsesConnect(tgTask) {
|
2020-05-18 19:21:12 +00:00
|
|
|
notConnect = append(notConnect, task)
|
|
|
|
} else {
|
|
|
|
usesConnect = append(usesConnect, connectTask{
|
|
|
|
TaskName: task,
|
|
|
|
TaskKind: tgTask.Kind,
|
|
|
|
})
|
2019-12-06 20:46:46 +00:00
|
|
|
}
|
|
|
|
}
|
2020-05-18 19:21:12 +00:00
|
|
|
return notConnect, usesConnect
|
2019-12-06 20:46:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func taskUsesConnect(task *structs.Task) bool {
|
|
|
|
if task == nil {
|
|
|
|
// not even in the task group
|
|
|
|
return false
|
|
|
|
}
|
2020-05-13 20:15:55 +00:00
|
|
|
return task.UsesConnect()
|
2019-12-06 20:46:46 +00:00
|
|
|
}
|
|
|
|
|
2018-03-14 00:52:12 +00:00
|
|
|
func (n *Node) EmitEvents(args *structs.EmitNodeEventsRequest, reply *structs.EmitNodeEventsResponse) error {
|
2023-01-26 20:04:25 +00:00
|
|
|
|
|
|
|
authErr := n.srv.Authenticate(n.ctx, args)
|
|
|
|
|
2022-02-05 01:35:20 +00:00
|
|
|
// Ensure the connection was initiated by another client if TLS is used.
|
|
|
|
err := validateTLSCertificateLevel(n.srv, n.ctx, tlsCertificateLevelClient)
|
|
|
|
if err != nil {
|
2018-03-14 00:52:12 +00:00
|
|
|
return err
|
|
|
|
}
|
2022-02-05 01:35:20 +00:00
|
|
|
if done, err := n.srv.forward("Node.EmitEvents", args, args, reply); done {
|
|
|
|
return err
|
2022-02-02 20:03:18 +00:00
|
|
|
}
|
2023-01-26 20:04:25 +00:00
|
|
|
n.srv.MeasureRPCRate("node", structs.RateMetricWrite, args)
|
|
|
|
if authErr != nil {
|
|
|
|
return structs.ErrPermissionDenied
|
|
|
|
}
|
2022-02-05 01:35:20 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "client", "emit_events"}, time.Now())
|
2022-02-02 20:03:18 +00:00
|
|
|
|
2018-03-14 00:52:12 +00:00
|
|
|
if len(args.NodeEvents) == 0 {
|
|
|
|
return fmt.Errorf("no node events given")
|
|
|
|
}
|
|
|
|
for nodeID, events := range args.NodeEvents {
|
|
|
|
if len(events) == 0 {
|
|
|
|
return fmt.Errorf("no node events given for node %q", nodeID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
_, index, err := n.srv.raftApply(structs.UpsertNodeEventsType, args)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
n.logger.Error("upserting node events failed", "error", err)
|
2018-03-14 00:52:12 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
reply.Index = index
|
|
|
|
return nil
|
|
|
|
}
|