node pools: register a node in a node pool (#17405)

This commit is contained in:
Luiz Aoqui 2023-06-02 17:50:50 -04:00 committed by GitHub
parent b770f2b1ef
commit 6039c18ab6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 590 additions and 51 deletions

View File

@ -553,6 +553,7 @@ type Node struct {
Links map[string]string
Meta map[string]string
NodeClass string
NodePool string
CgroupParent string
Drain bool
DrainStrategy *DrainStrategy
@ -914,6 +915,7 @@ type NodeListStub struct {
Datacenter string
Name string
NodeClass string
NodePool string
Version string
Drain bool
SchedulingEligibility string

View File

@ -60,6 +60,7 @@ func TestNodes_List(t *testing.T) {
nodeListStub, queryMeta := queryNodeList(t, nodes)
must.Len(t, 1, nodeListStub)
must.Eq(t, NodePoolDefault, nodeListStub[0].NodePool)
// Check that we got valid QueryMeta.
assertQueryMeta(t, queryMeta)
@ -140,6 +141,7 @@ func TestNodes_Info(t *testing.T) {
// Check that the result is what we expect
must.Eq(t, nodeID, result.ID)
must.Eq(t, dc, result.Datacenter)
must.Eq(t, NodePoolDefault, result.NodePool)
must.Eq(t, 20000, result.NodeResources.MinDynamicPort)
must.Eq(t, 32000, result.NodeResources.MaxDynamicPort)

View File

@ -391,6 +391,82 @@ func TestClient_Register(t *testing.T) {
})
}
func TestClient_Register_NodePool(t *testing.T) {
ci.Parallel(t)
s1, _, cleanupS1 := testServer(t, nil)
defer cleanupS1()
testutil.WaitForLeader(t, s1.RPC)
// Create client with a node pool configured.
c1, cleanupC1 := TestClient(t, func(c *config.Config) {
c.RPCHandler = s1
c.Node.NodePool = "dev"
})
defer cleanupC1()
// Create client with no node pool configured.
c2, cleanupC2 := TestClient(t, func(c *config.Config) {
c.RPCHandler = s1
c.Node.NodePool = ""
})
defer cleanupC2()
nodeReq := structs.NodeSpecificRequest{
QueryOptions: structs.QueryOptions{Region: "global"},
}
var nodeResp structs.SingleNodeResponse
poolReq := structs.NodePoolSpecificRequest{
Name: "dev",
QueryOptions: structs.QueryOptions{Region: "global"},
}
var poolResp structs.SingleNodePoolResponse
// Register should succeed and node pool should be created.
// Client without node pool configuration should be in the default pool.
testutil.WaitForResult(func() (bool, error) {
// Fetch node1.
nodeReq.NodeID = c1.Node().ID
err := s1.RPC("Node.GetNode", &nodeReq, &nodeResp)
if err != nil {
return false, err
}
if nodeResp.Node == nil {
return false, fmt.Errorf("c1 is missing")
}
if nodeResp.Node.NodePool != "dev" {
return false, fmt.Errorf("c1 has wrong node pool")
}
// Fetch node1 node pool.
err = s1.RPC("NodePool.GetNodePool", &poolReq, &poolResp)
if err != nil {
return false, err
}
if poolResp.NodePool == nil {
return false, fmt.Errorf("dev node pool is nil")
}
// Fetch node2.
nodeReq.NodeID = c2.Node().ID
err = s1.RPC("Node.GetNode", &nodeReq, &nodeResp)
if err != nil {
return false, err
}
if nodeResp.Node == nil {
return false, fmt.Errorf("c2 is missing")
}
if nodeResp.Node.NodePool != structs.NodePoolDefault {
return false, fmt.Errorf("c2 has wrong node pool")
}
return true, nil
}, func(err error) {
t.Fatalf("err: %v", err)
})
}
func TestClient_Heartbeat(t *testing.T) {
ci.Parallel(t)

View File

@ -750,6 +750,7 @@ func convertClientConfig(agentConfig *Config) (*clientconfig.Config, error) {
conf.Node.Name = agentConfig.NodeName
conf.Node.Meta = agentConfig.Client.Meta
conf.Node.NodeClass = agentConfig.Client.NodeClass
conf.Node.NodePool = agentConfig.Client.NodePool
// Set up the HTTP advertise address
conf.Node.HTTPAddr = agentConfig.AdvertiseAddrs.HTTP

View File

@ -106,6 +106,7 @@ func (c *Command) readConfig() *Config {
flags.StringVar(&cmdConfig.Client.StateDir, "state-dir", "", "")
flags.StringVar(&cmdConfig.Client.AllocDir, "alloc-dir", "", "")
flags.StringVar(&cmdConfig.Client.NodeClass, "node-class", "", "")
flags.StringVar(&cmdConfig.Client.NodePool, "node-pool", "", "")
flags.StringVar(&servers, "servers", "", "")
flags.Var((*flaghelper.StringFlag)(&meta), "meta", "")
flags.StringVar(&cmdConfig.Client.NetworkInterface, "network-interface", "", "")
@ -379,6 +380,19 @@ func (c *Command) IsValidConfig(config, cmdConfig *Config) bool {
return false
}
// Validate node pool name early to prevent agent from starting but the
// client failing to register.
if pool := config.Client.NodePool; pool != "" {
if err := structs.ValidateNodePoolName(pool); err != nil {
c.Ui.Error(fmt.Sprintf("Invalid node pool: %v", err))
return false
}
if pool == structs.NodePoolAll {
c.Ui.Error(fmt.Sprintf("Invalid node pool: node is not allowed to register in node pool %q", structs.NodePoolAll))
return false
}
}
if config.Client.MinDynamicPort < 0 || config.Client.MinDynamicPort > structs.MaxValidPort {
c.Ui.Error(fmt.Sprintf("Invalid dynamic port range: min_dynamic_port=%d", config.Client.MinDynamicPort))
return false
@ -629,6 +643,7 @@ func (c *Command) AutocompleteFlags() complete.Flags {
"-state-dir": complete.PredictDirs("*"),
"-alloc-dir": complete.PredictDirs("*"),
"-node-class": complete.PredictAnything,
"-node-pool": complete.PredictAnything,
"-servers": complete.PredictAnything,
"-meta": complete.PredictAnything,
"-config": configFilePredictor,
@ -1407,6 +1422,10 @@ Client Options:
Mark this node as a member of a node-class. This can be used to label
similar node types.
-node-pool
Register this node in this node pool. If the node pool does not exist it
will be created automatically when the node registers.
-meta
User specified metadata to associated with the node. Each instance of -meta
parses a single KEY=VALUE pair. Repeat the meta flag for each key/value pair

View File

@ -71,6 +71,10 @@ func TestCommand_Args(t *testing.T) {
[]string{"-client", "-data-dir=" + tmpDir, "-meta=invalid.=inaccessible-value"},
"Invalid Client.Meta key: invalid.",
},
{
[]string{"-client", "-node-pool=not@valid"},
"Invalid node pool",
},
}
for _, tc := range tcases {
// Make a new command. We preemptively close the shutdownCh
@ -305,6 +309,26 @@ func TestIsValidConfig(t *testing.T) {
},
err: "must be given as an absolute",
},
{
name: "InvalidNodePoolChar",
conf: Config{
Client: &ClientConfig{
Enabled: true,
NodePool: "not@valid",
},
},
err: "Invalid node pool",
},
{
name: "InvalidNodePoolName",
conf: Config{
Client: &ClientConfig{
Enabled: true,
NodePool: structs.NodePoolAll,
},
},
err: "not allowed",
},
{
name: "NegativeMinDynamicPort",
conf: Config{

View File

@ -204,6 +204,11 @@ type ClientConfig struct {
// NodeClass is used to group the node by class
NodeClass string `hcl:"node_class"`
// NodePool defines the node pool in which the client is registered. If the
// node pool does not exist it will be created automatically when the node
// registers.
NodePool string `hcl:"node_pool"`
// Options is used for configuration of nomad internals,
// like fingerprinters and drivers. The format is:
//
@ -1269,6 +1274,7 @@ func DefaultConfig() *Config {
UI: config.DefaultUIConfig(),
Client: &ClientConfig{
Enabled: false,
NodePool: structs.NodePoolDefault,
MaxKillTimeout: "30s",
ClientMinPort: 14000,
ClientMaxPort: 14512,
@ -2056,6 +2062,9 @@ func (a *ClientConfig) Merge(b *ClientConfig) *ClientConfig {
if b.NodeClass != "" {
result.NodeClass = b.NodeClass
}
if b.NodePool != "" {
result.NodePool = b.NodePool
}
if b.NetworkInterface != "" {
result.NetworkInterface = b.NetworkInterface
}

View File

@ -109,6 +109,7 @@ func TestConfig_Merge(t *testing.T) {
StateDir: "/tmp/state1",
AllocDir: "/tmp/alloc1",
NodeClass: "class1",
NodePool: "dev",
Options: map[string]string{
"foo": "bar",
},
@ -297,6 +298,7 @@ func TestConfig_Merge(t *testing.T) {
StateDir: "/tmp/state2",
AllocDir: "/tmp/alloc2",
NodeClass: "class2",
NodePool: "dev",
Servers: []string{"server2"},
Meta: map[string]string{
"baz": "zip",

View File

@ -250,7 +250,7 @@ func (c *NodeStatusCommand) Run(args []string) int {
return 0
}
out[0] = "ID|DC|Name|Class|"
out[0] = "ID|Node Pool|DC|Name|Class|"
if c.os {
out[0] += "OS|"
@ -267,8 +267,9 @@ func (c *NodeStatusCommand) Run(args []string) int {
}
for i, node := range nodes {
out[i+1] = fmt.Sprintf("%s|%s|%s|%s",
out[i+1] = fmt.Sprintf("%s|%s|%s|%s|%s",
limit(node.ID, c.length),
node.NodePool,
node.Datacenter,
node.Name,
node.NodeClass)
@ -480,6 +481,7 @@ func (c *NodeStatusCommand) formatNode(client *api.Client, node *api.Node) int {
basic := []string{
fmt.Sprintf("ID|%s", node.ID),
fmt.Sprintf("Name|%s", node.Name),
fmt.Sprintf("Node Pool|%s", node.NodePool),
fmt.Sprintf("Class|%s", node.NodeClass),
fmt.Sprintf("DC|%s", node.Datacenter),
fmt.Sprintf("Drain|%v", formatDrain(node)),

View File

@ -241,6 +241,98 @@ func TestFSM_UpsertNode_Canonicalize_Ineligible(t *testing.T) {
require.Equal(structs.NodeSchedulingIneligible, n.SchedulingEligibility)
}
func TestFSM_UpsertNode_NodePool(t *testing.T) {
ci.Parallel(t)
fsm := testFSM(t)
// Populate state with a test node pool.
existingPool := mock.NodePool()
err := fsm.State().UpsertNodePools(structs.MsgTypeTestSetup, 1000, []*structs.NodePool{existingPool})
must.NoError(t, err)
testCases := []struct {
name string
previousPool string
pool string
validateFn func(*testing.T, *structs.Node, *structs.NodePool)
}{
{
name: "new node pool",
pool: "new",
validateFn: func(t *testing.T, node *structs.Node, pool *structs.NodePool) {
// Verify node pool was created in the same transaction as the
// node registration.
must.Eq(t, pool.CreateIndex, node.ModifyIndex)
},
},
{
name: "existing node pool",
pool: existingPool.Name,
validateFn: func(t *testing.T, node *structs.Node, pool *structs.NodePool) {
// Verify node pool was not modified.
must.NotEq(t, pool.CreateIndex, node.ModifyIndex)
},
},
{
name: "built-in node pool",
pool: structs.NodePoolDefault,
validateFn: func(t *testing.T, node *structs.Node, pool *structs.NodePool) {
// Verify node pool was not modified.
must.Eq(t, 1, pool.ModifyIndex)
},
},
{
name: "move node to another node pool",
previousPool: structs.NodePoolDefault,
pool: "new",
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
node := mock.Node()
// Pre-create node if test cases specifies if belonged to a
// previous node pool.
if tc.previousPool != "" {
node.NodePool = tc.previousPool
err := fsm.State().UpsertNode(structs.MsgTypeTestSetup, 1001, node)
must.NoError(t, err)
}
// Set the node pool and apply node register log.
node.NodePool = tc.pool
req := structs.NodeRegisterRequest{Node: node}
buf, err := structs.Encode(structs.NodeRegisterRequestType, req)
must.NoError(t, err)
resp := fsm.Apply(makeLog(buf))
must.Nil(t, resp)
// Snapshot the state.
s := fsm.State()
// Verify node exists.
got, err := s.NodeByID(nil, node.ID)
must.NoError(t, err)
must.NotNil(t, got)
// Verify node pool exists.
pool, err := s.NodePoolByName(nil, tc.pool)
must.NoError(t, err)
must.NotNil(t, pool)
// Verify node was assigned to node pool.
must.Eq(t, pool.Name, got.NodePool)
if tc.validateFn != nil {
tc.validateFn(t, got, pool)
}
})
}
}
func TestFSM_DeregisterNode(t *testing.T) {
ci.Parallel(t)
fsm := testFSM(t)

View File

@ -110,6 +110,7 @@ func Node() *structs.Node {
"version": "5.6",
},
NodeClass: "linux-medium-pci",
NodePool: structs.NodePoolDefault,
Status: structs.NodeStatusReady,
SchedulingEligibility: structs.NodeSchedulingEligible,
}

View File

@ -135,6 +135,17 @@ func (n *Node) Register(args *structs.NodeRegisterRequest, reply *structs.NodeUp
if args.Node.SecretID == "" {
return fmt.Errorf("missing node secret ID for client registration")
}
// Validate node pool value if provided. The node is canonicalized in the
// FSM, where an empty node pool is set to "default".
if args.Node.NodePool != "" {
err := structs.ValidateNodePoolName(args.Node.NodePool)
if err != nil {
return fmt.Errorf("invalid node pool: %v", err)
}
if args.Node.NodePool == structs.NodePoolAll {
return fmt.Errorf("node is not allowed to register in node pool %q", structs.NodePoolAll)
}
}
// Default the status if none is given
if args.Node.Status == "" {

View File

@ -215,6 +215,91 @@ func TestClientEndpoint_Register_SecretMismatch(t *testing.T) {
}
}
func TestClientEndpoint_Register_NodePool(t *testing.T) {
ci.Parallel(t)
s, cleanupS := TestServer(t, nil)
defer cleanupS()
codec := rpcClient(t, s)
testutil.WaitForLeader(t, s.RPC)
testCases := []struct {
name string
pool string
expectedErr string
validateFn func(*testing.T, *structs.Node)
}{
{
name: "invalid node pool name",
pool: "not@valid",
expectedErr: "invalid node pool: invalid name",
},
{
name: "built-in pool all not allowed",
pool: structs.NodePoolAll,
expectedErr: `node is not allowed to register in node pool "all"`,
},
{
name: "set default node pool when empty",
pool: "",
validateFn: func(t *testing.T, node *structs.Node) {
state := s.fsm.State()
ws := memdb.NewWatchSet()
// Verify node was registered with default node pool.
got, err := state.NodeByID(ws, node.ID)
must.NoError(t, err)
must.NotNil(t, got)
must.Eq(t, structs.NodePoolDefault, got.NodePool)
},
},
{
name: "set node pool requested",
pool: "my-pool",
validateFn: func(t *testing.T, node *structs.Node) {
state := s.fsm.State()
ws := memdb.NewWatchSet()
// Verify node was registered.
got, err := state.NodeByID(ws, node.ID)
must.NoError(t, err)
must.NotNil(t, got)
// Verify node pool was created.
pool, err := state.NodePoolByName(ws, "my-pool")
must.NoError(t, err)
must.NotNil(t, pool)
// Verify node was added to the pool.
must.Eq(t, "my-pool", got.NodePool)
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
node := mock.Node()
node.NodePool = tc.pool
req := &structs.NodeRegisterRequest{
Node: node,
WriteRequest: structs.WriteRequest{Region: "global"},
}
var resp structs.GenericResponse
err := msgpackrpc.CallWithCodec(codec, "Node.Register", req, &resp)
if tc.expectedErr != "" {
must.ErrorContains(t, err, tc.expectedErr)
} else {
must.NoError(t, err)
if tc.validateFn != nil {
tc.validateFn(t, req.Node)
}
}
})
}
}
// Test the deprecated single node deregistration path
func TestClientEndpoint_DeregisterOne(t *testing.T) {
ci.Parallel(t)

View File

@ -900,6 +900,14 @@ func (s *StateStore) UpsertNode(msgType structs.MessageType, index uint64, node
txn := s.db.WriteTxnMsgT(msgType, index)
defer txn.Abort()
// Create node pool if necessary.
if node.NodePool != "" {
_, err := s.fetchOrCreateNodePoolTxn(txn, index, node.NodePool)
if err != nil {
return err
}
}
err := upsertNodeTxn(txn, index, node)
if err != nil {
return nil

View File

@ -55,7 +55,10 @@ func (s *StateStore) NodePools(ws memdb.WatchSet, sort SortOption) (memdb.Result
// there is no match.
func (s *StateStore) NodePoolByName(ws memdb.WatchSet, name string) (*structs.NodePool, error) {
txn := s.db.ReadTxn()
return s.nodePoolByNameTxn(txn, ws, name)
}
func (s *StateStore) nodePoolByNameTxn(txn *txn, ws memdb.WatchSet, name string) (*structs.NodePool, error) {
watchCh, existing, err := txn.FirstWatch(TableNodePools, "id", name)
if err != nil {
return nil, fmt.Errorf("node pool lookup failed: %w", err)
@ -140,6 +143,25 @@ func (s *StateStore) upsertNodePoolTxn(txn *txn, index uint64, pool *structs.Nod
return nil
}
// fetchOrCreateNodePoolTxn returns an existing node pool with the given name
// or creates a new one if it doesn't exist.
func (s *StateStore) fetchOrCreateNodePoolTxn(txn *txn, index uint64, name string) (*structs.NodePool, error) {
pool, err := s.nodePoolByNameTxn(txn, nil, name)
if err != nil {
return nil, err
}
if pool == nil {
pool = &structs.NodePool{Name: name}
err = s.upsertNodePoolTxn(txn, index, pool)
if err != nil {
return nil, err
}
}
return pool, nil
}
// DeleteNodePools removes the given set of node pools.
func (s *StateStore) DeleteNodePools(msgType structs.MessageType, index uint64, names []string) error {
txn := s.db.WriteTxnMsgT(msgType, index)

View File

@ -1276,6 +1276,136 @@ func TestStateStore_UpsertNode_Node(t *testing.T) {
require.Equal(NodeRegisterEventReregistered, out.Events[1].Message)
}
func TestStateStore_UpsertNode_NodePool(t *testing.T) {
ci.Parallel(t)
devPoolName := "dev"
nodeWithPoolID := uuid.Generate()
nodeWithoutPoolID := uuid.Generate()
testCases := []struct {
name string
nodeID string
poolName string
expectedPool string
expectedNewPool bool
}{
{
name: "register new node in existing pool",
nodeID: "",
poolName: devPoolName,
expectedPool: devPoolName,
},
{
name: "register new node in new pool",
nodeID: "",
poolName: "new",
expectedPool: "new",
expectedNewPool: true,
},
{
name: "register new node with empty pool in default",
nodeID: "",
poolName: "",
expectedPool: structs.NodePoolDefault,
},
{
name: "update existing node in existing pool",
nodeID: nodeWithPoolID,
poolName: devPoolName,
expectedPool: devPoolName,
},
{
name: "update existing node with new pool",
nodeID: nodeWithPoolID,
poolName: "new",
expectedPool: "new",
expectedNewPool: true,
},
{
name: "update existing node with empty pool in default",
nodeID: nodeWithPoolID,
poolName: "",
expectedPool: devPoolName,
},
{
name: "update legacy node with pool",
nodeID: nodeWithoutPoolID,
poolName: devPoolName,
expectedPool: devPoolName,
},
{
name: "update legacy node with new pool",
nodeID: nodeWithoutPoolID,
poolName: "new",
expectedPool: "new",
expectedNewPool: true,
},
{
name: "update legacy node with empty pool places it in default",
nodeID: nodeWithoutPoolID,
poolName: "",
expectedPool: structs.NodePoolDefault,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
state := testStateStore(t)
// Populate state with pre-existing node pool.
devPool := mock.NodePool()
devPool.Name = devPoolName
err := state.UpsertNodePools(structs.MsgTypeTestSetup, 1000, []*structs.NodePool{devPool})
must.NoError(t, err)
// Populate state with pre-existing node assigned to the
// pre-existing node pool.
nodeWithPool := mock.Node()
nodeWithPool.ID = nodeWithPoolID
nodeWithPool.NodePool = devPool.Name
err = state.UpsertNode(structs.MsgTypeTestSetup, 1001, nodeWithPool)
must.NoError(t, err)
// Populate state with pre-existing node with nil node pool to
// simulate an upgrade path.
nodeWithoutPool := mock.Node()
nodeWithoutPool.ID = nodeWithoutPoolID
err = state.UpsertNode(structs.MsgTypeTestSetup, 1002, nodeWithoutPool)
must.NoError(t, err)
// Upsert test node.
var node *structs.Node
switch tc.nodeID {
case nodeWithPoolID:
node = nodeWithPool
case nodeWithoutPoolID:
node = nodeWithoutPool
default:
node = mock.Node()
}
err = state.UpsertNode(structs.MsgTypeTestSetup, 1003, node)
must.NoError(t, err)
// Verify that node is part of the expected pool.
must.Eq(t, tc.expectedPool, node.NodePool)
// Fech pool.
pool, err := state.NodePoolByName(nil, tc.expectedPool)
must.NoError(t, err)
must.NotNil(t, pool)
if tc.expectedNewPool {
// Verify that pool was created along with node registration.
must.Eq(t, node.ModifyIndex, pool.CreateIndex)
} else {
// Verify that pool was not modified.
must.Less(t, node.ModifyIndex, pool.ModifyIndex)
}
})
}
}
func TestStateStore_DeleteNode_Node(t *testing.T) {
ci.Parallel(t)

View File

@ -45,7 +45,7 @@ func (n *Node) ComputeClass() error {
// included in the computed node class.
func (n Node) HashInclude(field string, v interface{}) (bool, error) {
switch field {
case "Datacenter", "Attributes", "Meta", "NodeClass", "NodeResources":
case "Datacenter", "Attributes", "Meta", "NodeClass", "NodePool", "NodeResources":
return true, nil
default:
return false, nil

View File

@ -10,6 +10,7 @@ import (
"github.com/hashicorp/nomad/ci"
"github.com/hashicorp/nomad/helper/uuid"
psstructs "github.com/hashicorp/nomad/plugins/shared/structs"
"github.com/shoenig/test/must"
"github.com/stretchr/testify/require"
)
@ -51,6 +52,7 @@ func testNode() *Node {
"pci-dss": "true",
},
NodeClass: "linux-medium-pci",
NodePool: "dev",
Status: NodeStatusReady,
}
}
@ -218,6 +220,25 @@ func TestNode_ComputedClass_Meta(t *testing.T) {
}
}
func TestNode_ComputedClass_NodePool(t *testing.T) {
ci.Parallel(t)
// Create a node and get its computed class.
n := testNode()
err := n.ComputeClass()
must.NoError(t, err)
must.NotEq(t, "", n.ComputedClass)
old := n.ComputedClass
// Modify node pool and expect computed class to change.
n.NodePool = "prod"
err = n.ComputeClass()
must.NoError(t, err)
must.NotEq(t, "", n.ComputedClass)
must.NotEq(t, old, n.ComputedClass)
old = n.ComputedClass
}
func TestNode_EscapedConstraints(t *testing.T) {
ci.Parallel(t)

View File

@ -32,6 +32,14 @@ var (
validNodePoolName = regexp.MustCompile("^[a-zA-Z0-9-_]{1,128}$")
)
// ValidadeNodePoolName returns an error if a node pool name is invalid.
func ValidateNodePoolName(pool string) error {
if !validNodePoolName.MatchString(pool) {
return fmt.Errorf("invalid name %q, must match regex %s", pool, validNodePoolName)
}
return nil
}
// NodePool allows partioning infrastructure
type NodePool struct {
// Name is the node pool name. It must be unique.
@ -61,9 +69,8 @@ func (n *NodePool) GetID() string {
func (n *NodePool) Validate() error {
var mErr *multierror.Error
if !validNodePoolName.MatchString(n.Name) {
mErr = multierror.Append(mErr, fmt.Errorf("invalid name %q, must match regex %s", n.Name, validNodePoolName))
}
mErr = multierror.Append(mErr, ValidateNodePoolName(n.Name))
if len(n.Description) > maxNodePoolDescriptionLength {
mErr = multierror.Append(mErr, fmt.Errorf("description longer than %d", maxNodePoolDescriptionLength))
}

View File

@ -2109,6 +2109,9 @@ type Node struct {
// together for the purpose of determining scheduling pressure.
NodeClass string
// NodePool is the node pool the node belongs to.
NodePool string
// ComputedClass is a unique id that identifies nodes with a common set of
// attributes and capabilities.
ComputedClass string
@ -2198,6 +2201,10 @@ func (n *Node) Canonicalize() {
return
}
if n.NodePool == "" {
n.NodePool = NodePoolDefault
}
// Ensure SchedulingEligibility is correctly set whenever draining so the plan applier and other scheduling logic
// only need to check SchedulingEligibility when determining whether a placement is feasible on a node.
if n.DrainStrategy != nil {
@ -2356,6 +2363,7 @@ func (n *Node) Stub(fields *NodeStubFields) *NodeListStub {
Datacenter: n.Datacenter,
Name: n.Name,
NodeClass: n.NodeClass,
NodePool: n.NodePool,
Version: n.Attributes["nomad.version"],
Drain: n.DrainStrategy != nil,
SchedulingEligibility: n.SchedulingEligibility,
@ -2393,6 +2401,7 @@ type NodeListStub struct {
Attributes map[string]string `json:",omitempty"`
Datacenter string
Name string
NodePool string
NodeClass string
Version string
Drain bool

View File

@ -141,6 +141,7 @@ $ curl \
"ModifyIndex": 2526,
"Name": "nomad-4",
"NodeClass": "",
"NodePool": "default",
"SchedulingEligibility": "eligible",
"Status": "ready",
"StatusDescription": "",
@ -348,6 +349,7 @@ $ curl \
"ModifyIndex": 14,
"Name": "mew",
"NodeClass": "",
"NodePool": "default",
"NodeResources": {
"Cpu": {
"CpuShares": 32000

View File

@ -121,6 +121,9 @@ via CLI arguments. The `agent` command accepts the following arguments:
- `-node-class=<class>`: Equivalent to the Client [node_class]
config option.
- `-node-pool=<node-pool>`: Equivalent to the Client [node_pool]
config option.
- `-plugin-dir=<path>`: Equivalent to the [plugin_dir] config option.
- `-region=<region>`: Equivalent to the [region] config option.
@ -205,6 +208,7 @@ via CLI arguments. The `agent` command accepts the following arguments:
[name]: /nomad/docs/configuration#name
[network_interface]: /nomad/docs/configuration/client#network_interface
[node_class]: /nomad/docs/configuration/client#node_class
[node_pool]: /nomad/docs/configuration/client#node_pool
[nomad agent]: /nomad/docs/operations/nomad-agent
[plugin_dir]: /nomad/docs/configuration#plugin_dir
[region]: /nomad/docs/configuration#region

View File

@ -67,18 +67,18 @@ List view:
```shell-session
$ nomad node status
ID DC Name Class Drain Eligibility Status
a72dfba2 dc1 node1 <none> false eligible ready
1f3f03ea dc1 node2 <none> false eligible ready
ID Node Pool DC Name Class Drain Eligibility Status
a72dfba2 default dc1 node1 <none> false eligible ready
1f3f03ea dev dc1 node2 <none> false eligible ready
```
List view, with operating system name:
```shell-session
$ nomad node status -os
ID DC Name Class OS Drain Eligibility Status
a72dfba2 dc1 node1 <none> ubuntu false eligible ready
f73e3993 dc1 node2 <none> centos false eligible ready
ID Node Pool DC Name Class OS Drain Eligibility Status
a72dfba2 default dc1 node1 <none> ubuntu false eligible ready
f73e3993 dev dc1 node2 <none> centos false eligible ready
```
List view, with quiet:
@ -91,28 +91,29 @@ $ nomad node status -quiet
f35be281-85a5-d1e6-d268-6e8a6f0684df
```
**NOTE**: `-quiet` cannot be used in conjuction with `-verbose` or `-json`.
**NOTE**: `-quiet` cannot be used in conjunction with `-verbose` or `-json`.
List view, with running allocations:
```shell-session
$ nomad node status -allocs
ID DC Name Class Drain Eligibility Status Running Allocs
4d2ba53b dc1 node1 <none> false eligible ready 1
34dfba32 dc1 node2 <none> false eligible ready 3
ID Node Pool DC Name Class Drain Eligibility Status Running Allocs
4d2ba53b default dc1 node1 <none> false eligible ready 1
34dfba32 dev dc1 node2 <none> false eligible ready 3
```
Single-node view in short mode:
```shell-session
$ nomad node status -short 1f3f03ea
ID = c754da1f
Name = nomad
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h2m25s
ID = c754da1f
Name = nomad
Node Pool = default
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h2m25s
Allocations
ID Eval ID Job ID Task Group Desired Status Client Status
@ -123,13 +124,14 @@ Full output for a single node:
```shell-session
$ nomad node status 1f3f03ea
ID = c754da1f
Name = nomad-server01
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h42m50s
ID = c754da1f
Name = nomad-server01
Node Pool = default
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h42m50s
Drivers
Driver Detected Healthy
@ -166,13 +168,14 @@ Using `-self` when on a Nomad Client:
```shell-session
$ nomad node status -self
ID = c754da1f
Name = nomad-client01
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h7m41s
ID = c754da1f
Name = nomad-client01
Node Pool = default
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h7m41s
Drivers
Driver Detected Healthy
@ -236,13 +239,14 @@ Using `-stats` to see detailed to resource usage information on the node:
```shell-session
$ nomad node status -stats c754da1f
ID = c754da1f
Name = nomad-client01
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h7m41s
ID = c754da1f
Name = nomad-client01
Node Pool = default
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h7m41s
Drivers
Driver Detected Healthy
@ -317,13 +321,14 @@ To view verbose information about the node:
```shell-session
$ nomad node status -verbose c754da1f
ID = c754da1f-6337-b86d-47dc-2ef4c71aca14
Name = nomad
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h7m41s
ID = c754da1f-6337-b86d-47dc-2ef4c71aca14
Name = nomad
Node Pool = default
Class = <none>
DC = dc1
Drain = false
Status = ready
Uptime = 17h7m41s
Host Volumes
Name ReadOnly Source

View File

@ -86,6 +86,11 @@ client {
group client nodes by user-defined class. This can be used during job
placement as a filter.
- `node_pool` `(string: "default")` - Specifies the node pool in which the
client is registered. If the node pool does not exist yet, it will be created
automatically when the node registers with the cluster. If not specified the
`default` node pool is used.
- `options` <code>([Options](#options-parameters): nil)</code> - Specifies a
key-value mapping of internal configuration for clients, such as for driver
configuration.