From 19081875bc13ba7cc01f3cfe2c25a3c1d2e0605f Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 19 Nov 2021 22:53:31 -0500 Subject: [PATCH 01/30] Working POC --- nomad/leader.go | 14 +++--- nomad/server.go | 43 +++++++++++++++++- nomad/worker.go | 101 ++++++++++++++++++++++++++++++++----------- nomad/worker_test.go | 8 ++-- 4 files changed, 128 insertions(+), 38 deletions(-) diff --git a/nomad/leader.go b/nomad/leader.go index 64f06b84867..89f61865e9a 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -230,9 +230,7 @@ func (s *Server) establishLeadership(stopCh chan struct{}) error { // Disable workers to free half the cores for use in the plan queue and // evaluation broker - for _, w := range s.pausableWorkers() { - w.SetPause(true) - } + s.handlePausableWorkers(true) // Initialize and start the autopilot routine s.getOrCreateAutopilotConfig() @@ -442,6 +440,12 @@ ERR_WAIT: } } +func (s *Server) handlePausableWorkers(isLeader bool) { + for _, w := range s.pausableWorkers() { + w.SetPause(isLeader) + } +} + // diffNamespaces is used to perform a two-way diff between the local namespaces // and the remote namespaces to determine which namespaces need to be deleted or // updated. @@ -1081,9 +1085,7 @@ func (s *Server) revokeLeadership() error { } // Unpause our worker if we paused previously - for _, w := range s.pausableWorkers() { - w.SetPause(false) - } + s.handlePausableWorkers(false) return nil } diff --git a/nomad/server.go b/nomad/server.go index 557ed175962..c6442c84d37 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -809,6 +809,15 @@ func (s *Server) Reload(newConfig *Config) error { s.EnterpriseState.ReloadLicense(newConfig) } + if newConfig.NumSchedulers != s.config.NumSchedulers { + s.logger.Debug("changing number of schedulers", "from", s.config.NumSchedulers, "to", newConfig.NumSchedulers) + s.config.NumSchedulers = newConfig.NumSchedulers + if err := s.SetupNewWorkers(); err != nil { + s.logger.Error("error creating new workers", "error", err) + _ = multierror.Append(&mErr, err) + } + } + return mErr.ErrorOrNil() } @@ -1453,19 +1462,49 @@ func (s *Server) setupWorkers() error { if !foundCore { return fmt.Errorf("invalid configuration: %q scheduler not enabled", structs.JobTypeCore) } - + s.logger.Info("starting scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) // Start the workers for i := 0; i < s.config.NumSchedulers; i++ { if w, err := NewWorker(s); err != nil { return err } else { + s.logger.Debug("started scheduling worker", "id", w.ID(), "index", i+1, "of", s.config.NumSchedulers) + s.workers = append(s.workers, w) } } - s.logger.Info("starting scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) + s.logger.Info("started scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) + return nil +} + +// SetupNewWorkers() is used to start a new set of workers after a configuration +// change and a hot reload. +func (s *Server) SetupNewWorkers() error { + // make a copy of the s.workers array so we can safely stop those goroutines + oldWorkers := make([]*Worker, len(s.workers)) + defer s.stopOldWorkers(oldWorkers) + for i, w := range s.workers { + oldWorkers[i] = w + } + s.logger.Info(fmt.Sprintf("marking %v current schedulers for shutdown", len(oldWorkers))) + s.workers = make([]*Worker, 0, s.config.NumSchedulers) + err := s.setupWorkers() + if err != nil { + return err + } + s.handlePausableWorkers(s.IsLeader()) + return nil } +func (s *Server) stopOldWorkers(oldWorkers []*Worker) { + workerCount := len(oldWorkers) + for i, w := range oldWorkers { + s.logger.Debug("stopping old scheduling worker", "id", w.ID(), "index", i+1, "of", workerCount) + go w.Shutdown() + } +} + // numPeers is used to check on the number of known peers, including the local // node. func (s *Server) numPeers() (int, error) { diff --git a/nomad/worker.go b/nomad/worker.go index d5b9699fd0f..5180c2d0601 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -10,6 +10,7 @@ import ( metrics "github.com/armon/go-metrics" log "github.com/hashicorp/go-hclog" memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" @@ -52,11 +53,14 @@ const ( // lifecycle around making task allocations. They bridge the business logic // of the scheduler with the plumbing required to make it all work. type Worker struct { - srv *Server - logger log.Logger - start time.Time + srv *Server + logger log.Logger + start time.Time + id string + paused bool + stop bool // Signals that the worker should stop + stopped bool // Indicates that the worker is in a terminal state; read with Stopped() - paused bool pauseLock sync.Mutex pauseCond *sync.Cond @@ -73,10 +77,11 @@ type Worker struct { // NewWorker starts a new worker associated with the given server func NewWorker(srv *Server) (*Worker, error) { w := &Worker{ - srv: srv, - logger: srv.logger.ResetNamed("worker"), - start: time.Now(), + srv: srv, + start: time.Now(), + id: uuid.Generate(), } + w.logger = srv.logger.ResetNamed("worker").With("worker_id", w.id) w.pauseCond = sync.NewCond(&w.pauseLock) go w.run() return w, nil @@ -101,8 +106,52 @@ func (w *Worker) checkPaused() { w.pauseLock.Unlock() } +// Shutdown is used to signal that the worker should shutdown. +func (w *Worker) Shutdown() { + w.pauseLock.Lock() + w.paused = false + w.stop = true + w.pauseLock.Unlock() + w.pauseCond.Broadcast() +} + +// Stopped returns a boolean indicating if this worker has been stopped. +func (w *Worker) Stopped() bool { + w.pauseLock.Lock() + defer w.pauseLock.Unlock() + return w.stopped +} + +// ID returns a string ID for the worker. +func (w *Worker) ID() string { + return w.id +} + +// shouldStop is used to check the worker state to see if conditions indicate +// that it should shutdown. +func (w *Worker) shouldStop() bool { + var localStop, shouldStop bool + w.pauseLock.Lock() + localStop = w.stop + w.pauseLock.Unlock() + + shouldStop = localStop || w.srv.IsShutdown() + return shouldStop +} + +// markStopped is used to mark the worker as stopped and should be called in a +// defer immediately upon entering the run() function. +func (w *Worker) markStopped() { + w.pauseLock.Lock() + w.stopped = true + w.pauseLock.Unlock() + w.logger.Trace("stopped") +} + // run is the long-lived goroutine which is used to run the worker func (w *Worker) run() { + defer w.markStopped() + w.logger.Trace("running") for { // Dequeue a pending evaluation eval, token, waitIndex, shutdown := w.dequeueEvaluation(dequeueTimeout) @@ -111,9 +160,9 @@ func (w *Worker) run() { } // Check for a shutdown - if w.srv.IsShutdown() { + if w.shouldStop() { w.logger.Error("nacking eval because the server is shutting down", "eval", log.Fmt("%#v", eval)) - w.sendNack(eval.ID, token) + w.sendNack(eval, token) return } @@ -121,19 +170,19 @@ func (w *Worker) run() { snap, err := w.snapshotMinIndex(waitIndex, raftSyncLimit) if err != nil { w.logger.Error("error waiting for Raft index", "error", err, "index", waitIndex) - w.sendNack(eval.ID, token) + w.sendNack(eval, token) continue } // Invoke the scheduler to determine placements if err := w.invokeScheduler(snap, eval, token); err != nil { w.logger.Error("error invoking scheduler", "error", err) - w.sendNack(eval.ID, token) + w.sendNack(eval, token) continue } // Complete the evaluation - w.sendAck(eval.ID, token) + w.sendAck(eval, token) } } @@ -155,13 +204,17 @@ func (w *Worker) dequeueEvaluation(timeout time.Duration) ( REQ: // Check if we are paused w.checkPaused() + // Immediately check to see if the worker has been shutdown + if w.shouldStop() { + return nil, "", 0, true + } // Make a blocking RPC start := time.Now() err := w.srv.RPC("Eval.Dequeue", &req, &resp) metrics.MeasureSince([]string{"nomad", "worker", "dequeue_eval"}, start) if err != nil { - if time.Since(w.start) > dequeueErrGrace && !w.srv.IsShutdown() { + if time.Since(w.start) > dequeueErrGrace && !w.shouldStop() { w.logger.Error("failed to dequeue evaluation", "error", err) } @@ -182,25 +235,21 @@ REQ: // Check if we got a response if resp.Eval != nil { - w.logger.Debug("dequeued evaluation", "eval_id", resp.Eval.ID) + w.logger.Debug("dequeued evaluation", "eval_id", resp.Eval.ID, "type", resp.Eval.Type, "namespace", resp.Eval.Namespace, "job_id", resp.Eval.JobID, "node_id", resp.Eval.NodeID, "triggered_by", resp.Eval.TriggeredBy) return resp.Eval, resp.Token, resp.GetWaitIndex(), false } - // Check for potential shutdown - if w.srv.IsShutdown() { - return nil, "", 0, true - } goto REQ } // sendAcknowledgement should not be called directly. Call `sendAck` or `sendNack` instead. // This function implements `ack`ing or `nack`ing the evaluation generally. // Any errors are logged but swallowed. -func (w *Worker) sendAcknowledgement(evalID, token string, ack bool) { +func (w *Worker) sendAcknowledgement(eval *structs.Evaluation, token string, ack bool) { defer metrics.MeasureSince([]string{"nomad", "worker", "send_ack"}, time.Now()) // Setup the request req := structs.EvalAckRequest{ - EvalID: evalID, + EvalID: eval.ID, Token: token, WriteRequest: structs.WriteRequest{ Region: w.srv.config.Region, @@ -219,22 +268,22 @@ func (w *Worker) sendAcknowledgement(evalID, token string, ack bool) { // Make the RPC call err := w.srv.RPC(endpoint, &req, &resp) if err != nil { - w.logger.Error(fmt.Sprintf("failed to %s evaluation", verb), "eval_id", evalID, "error", err) + w.logger.Error(fmt.Sprintf("failed to %s evaluation", verb), "eval_id", eval.ID, "error", err) } else { - w.logger.Debug(fmt.Sprintf("%s evaluation", verb), "eval_id", evalID) + w.logger.Debug(fmt.Sprintf("%s evaluation", verb), "eval_id", eval.ID, "type", eval.Type, "namespace", eval.Namespace, "job_id", eval.JobID, "node_id", eval.NodeID, "triggered_by", eval.TriggeredBy) } } // sendNack makes a best effort to nack the evaluation. // Any errors are logged but swallowed. -func (w *Worker) sendNack(evalID, token string) { - w.sendAcknowledgement(evalID, token, false) +func (w *Worker) sendNack(eval *structs.Evaluation, token string) { + w.sendAcknowledgement(eval, token, false) } // sendAck makes a best effort to ack the evaluation. // Any errors are logged but swallowed. -func (w *Worker) sendAck(evalID, token string) { - w.sendAcknowledgement(evalID, token, true) +func (w *Worker) sendAck(eval *structs.Evaluation, token string) { + w.sendAcknowledgement(eval, token, true) } // snapshotMinIndex times calls to StateStore.SnapshotAfter which may block. diff --git a/nomad/worker_test.go b/nomad/worker_test.go index 34327e61fac..0569d1d6675 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -133,7 +133,7 @@ func TestWorker_dequeueEvaluation_SerialJobs(t *testing.T) { } // Send the Ack - w.sendAck(eval1.ID, token) + w.sendAck(eval1, token) // Attempt second dequeue eval, token, waitIndex, shutdown = w.dequeueEvaluation(10 * time.Millisecond) @@ -258,7 +258,7 @@ func TestWorker_sendAck(t *testing.T) { } // Send the Nack - w.sendNack(eval.ID, token) + w.sendNack(eval, token) // Check the depth is 1, nothing unacked stats = s1.evalBroker.Stats() @@ -270,7 +270,7 @@ func TestWorker_sendAck(t *testing.T) { eval, token, _, _ = w.dequeueEvaluation(10 * time.Millisecond) // Send the Ack - w.sendAck(eval.ID, token) + w.sendAck(eval, token) // Check the depth is 0 stats = s1.evalBroker.Stats() @@ -674,7 +674,7 @@ func TestWorker_ReblockEval(t *testing.T) { } // Ack the eval - w.sendAck(evalOut.ID, token) + w.sendAck(evalOut, token) // Check that it is blocked bStats := s1.blockedEvals.Stats() From 0071e558384398e29ee07d69d293e8aab01d7682 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Tue, 23 Nov 2021 12:13:16 -0500 Subject: [PATCH 02/30] Unexport setupNewWorkers; improve comments --- nomad/server.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/nomad/server.go b/nomad/server.go index c6442c84d37..732f998cd0c 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -812,7 +812,7 @@ func (s *Server) Reload(newConfig *Config) error { if newConfig.NumSchedulers != s.config.NumSchedulers { s.logger.Debug("changing number of schedulers", "from", s.config.NumSchedulers, "to", newConfig.NumSchedulers) s.config.NumSchedulers = newConfig.NumSchedulers - if err := s.SetupNewWorkers(); err != nil { + if err := s.setupNewWorkers(); err != nil { s.logger.Error("error creating new workers", "error", err) _ = multierror.Append(&mErr, err) } @@ -1477,26 +1477,32 @@ func (s *Server) setupWorkers() error { return nil } -// SetupNewWorkers() is used to start a new set of workers after a configuration +// setupNewWorkers() is used to start a new set of workers after a configuration // change and a hot reload. -func (s *Server) SetupNewWorkers() error { - // make a copy of the s.workers array so we can safely stop those goroutines +func (s *Server) setupNewWorkers() error { + // make a copy of the s.workers array so we can safely stop those goroutines asynchronously oldWorkers := make([]*Worker, len(s.workers)) defer s.stopOldWorkers(oldWorkers) for i, w := range s.workers { oldWorkers[i] = w } s.logger.Info(fmt.Sprintf("marking %v current schedulers for shutdown", len(oldWorkers))) + + // build a clean backing array and call setupWorkers like in the normal startup path s.workers = make([]*Worker, 0, s.config.NumSchedulers) err := s.setupWorkers() if err != nil { return err } + + // if we're the leader, we need to pause all of the pausable workers. s.handlePausableWorkers(s.IsLeader()) return nil } +// stopOldWorkers is called once setupNewWorkers has created the new worker +// array to asynchronously stop each of the old workers individually. func (s *Server) stopOldWorkers(oldWorkers []*Worker) { workerCount := len(oldWorkers) for i, w := range oldWorkers { From 763671a19162fbb717460f689da513b3a27ea089 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Tue, 23 Nov 2021 12:13:50 -0500 Subject: [PATCH 03/30] Added some VSCode codetours --- .tours/scheduler-worker---hot-reload.tour | 57 ++++++++++++++++++++ .tours/scheduler-worker---pause.tour | 66 +++++++++++++++++++++++ .tours/scheduler-worker---unpause.tour | 51 ++++++++++++++++++ .tours/scheduler-worker.tour | 36 +++++++++++++ 4 files changed, 210 insertions(+) create mode 100644 .tours/scheduler-worker---hot-reload.tour create mode 100644 .tours/scheduler-worker---pause.tour create mode 100644 .tours/scheduler-worker---unpause.tour create mode 100644 .tours/scheduler-worker.tour diff --git a/.tours/scheduler-worker---hot-reload.tour b/.tours/scheduler-worker---hot-reload.tour new file mode 100644 index 00000000000..f733733306b --- /dev/null +++ b/.tours/scheduler-worker---hot-reload.tour @@ -0,0 +1,57 @@ +{ + "$schema": "https://aka.ms/codetour-schema", + "title": "Scheduler Worker - Hot Reload", + "steps": [ + { + "file": "nomad/server.go", + "description": "## Server.Reload()\n\nServer configuration reloads start here.", + "line": 782, + "selection": { + "start": { + "line": 780, + "character": 4 + }, + "end": { + "line": 780, + "character": 10 + } + } + }, + { + "file": "nomad/server.go", + "description": "## Did NumSchedulers change?\nIf the number of schedulers has changed between the running configuration and the new one we need to adopt that change in realtime.", + "line": 812 + }, + { + "file": "nomad/server.go", + "description": "## Server.setupNewWorkers()\n\nsetupNewWorkers performs three tasks:\n\n- makes a copy of the existing worker pointers\n\n- creates a fresh array and loads a new set of workers into them\n\n- iterates through the \"old\" workers and shuts them down in individual\n goroutines for maximum parallelism", + "line": 1482, + "selection": { + "start": { + "line": 1480, + "character": 4 + }, + "end": { + "line": 1480, + "character": 12 + } + } + }, + { + "file": "nomad/server.go", + "description": "Once all of the work in setupNewWorkers is complete, we stop the old ones.", + "line": 1485 + }, + { + "file": "nomad/server.go", + "description": "The `stopOldWorkers` function iterates through the array of workers and calls their `Shutdown` method\nas a goroutine to prevent blocking.", + "line": 1505 + }, + { + "file": "nomad/worker.go", + "description": "The `Shutdown` method sets `w.stop` to true signaling that we intend for the `Worker` to stop the next time we consult it. We also manually unpause the `Worker` by setting w.paused to false and sending a `Broadcast()` via the cond.", + "line": 110 + } + ], + "ref": "f-reload-num-schedulers" +} \ No newline at end of file diff --git a/.tours/scheduler-worker---pause.tour b/.tours/scheduler-worker---pause.tour new file mode 100644 index 00000000000..5b9d21fc22e --- /dev/null +++ b/.tours/scheduler-worker---pause.tour @@ -0,0 +1,66 @@ +{ + "$schema": "https://aka.ms/codetour-schema", + "title": "Scheduler Worker - Pause", + "steps": [ + { + "file": "nomad/leader.go", + "description": "## Server.establishLeadership()\n\nUpon becoming a leader, the server pauses a subset of the workers to allow for the additional burden of the leader's goroutines. The `handlePausableWorkers` function takes a boolean that states whether or not the current node is a leader or not. Because we are in `establishLeadership` we use `true` rather than calling `s.IsLeader()`", + "line": 233, + "selection": { + "start": { + "line": 233, + "character": 4 + }, + "end": { + "line": 233, + "character": 12 + } + } + }, + { + "file": "nomad/leader.go", + "description": "## Server.handlePausableWorkers()\n\nhandlePausableWorkers ranges over a slice of Workers and manipulates their paused state by calling their `SetPause` method.", + "line": 443, + "selection": { + "start": { + "line": 443, + "character": 18 + }, + "end": { + "line": 443, + "character": 26 + } + } + }, + { + "file": "nomad/leader.go", + "description": "## Server.pausableWorkers()\n\nThe pausableWorkers function provides a consistent slice of workers that the server can pause and unpause. Since the Worker array is never mutated, the same slice is returned by pausableWorkers on every invocation.\nThis comment is interesting/potentially confusing\n\n```golang\n // Disabling 3/4 of the workers frees CPU for raft and the\n\t// plan applier which uses 1/2 the cores.\n``` \n\nHowever, the key point is that it will return a slice containg 3/4th of the workers.", + "line": 1100, + "selection": { + "start": { + "line": 1104, + "character": 1 + }, + "end": { + "line": 1105, + "character": 43 + } + } + }, + { + "file": "nomad/worker.go", + "description": "## Worker.SetPause()\n\nThe `SetPause` function is used to signal an intention to pause the worker. Because the worker's work is happening in the `run()` goroutine, pauses happen asynchronously.", + "line": 91 + }, + { + "file": "nomad/worker.go", + "description": "## Worker.dequeueEvaluation()\n\nCalls checkPaused, which will be the function we wait in if the scheduler is set to be paused. \n\n> **NOTE:** This is called here rather than in run() because this function loops in case of an error fetching a evaluation.", + "line": 206 + }, + { + "file": "nomad/worker.go", + "description": "## Worker.checkPaused()\n\nWhen `w.paused` is `true`, we call the `Wait()` function on the condition. Execution of this goroutine will stop here until it receives a `Broadcast()` or a `Signal()`. At this point, the `Worker` is paused.", + "line": 104 + } + ] +} \ No newline at end of file diff --git a/.tours/scheduler-worker---unpause.tour b/.tours/scheduler-worker---unpause.tour new file mode 100644 index 00000000000..9c1c3a796ac --- /dev/null +++ b/.tours/scheduler-worker---unpause.tour @@ -0,0 +1,51 @@ +{ + "$schema": "https://aka.ms/codetour-schema", + "title": "Scheduler Worker - Unpause", + "steps": [ + { + "file": "nomad/leader.go", + "description": "## revokeLeadership()\n\nAs a server transistions from leader to non-leader, the pausableWorkers are resumed since the other leader goroutines are stopped providing extra capacity.", + "line": 1040, + "selection": { + "start": { + "line": 1038, + "character": 10 + }, + "end": { + "line": 1038, + "character": 20 + } + } + }, + { + "file": "nomad/leader.go", + "description": "## handlePausableWorkers()\n\nThe handlePausableWorkers method is called with `false`. We fetch the pausableWorkers and call their SetPause method with `false`.\n", + "line": 443, + "selection": { + "start": { + "line": 443, + "character": 18 + }, + "end": { + "line": 443, + "character": 27 + } + } + }, + { + "file": "nomad/worker.go", + "description": "## Worker.SetPause()\n\nDuring unpause, p is false. We update w.paused in the mutex, and then call Broadcast on the cond. This wakes the goroutine sitting in the Wait() inside of `checkPaused()`", + "line": 91 + }, + { + "file": "nomad/worker.go", + "description": "## Worker.checkPaused()\n\nOnce the goroutine receives the `Broadcast()` message from `SetPause()`, execution continues here. Now that `w.paused == false`, we exit the loop and return to the caller (the `dequeueEvaluation()` function).", + "line": 104 + }, + { + "file": "nomad/worker.go", + "description": "## Worker.dequeueEvaluation\n\nWe return back into dequeueEvaluation after the call to checkPaused. At this point the worker will either stop (if that signal boolean has been set) or continue looping after returning to run().", + "line": 207 + } + ] +} \ No newline at end of file diff --git a/.tours/scheduler-worker.tour b/.tours/scheduler-worker.tour new file mode 100644 index 00000000000..bbd6fca599b --- /dev/null +++ b/.tours/scheduler-worker.tour @@ -0,0 +1,36 @@ +{ + "$schema": "https://aka.ms/codetour-schema", + "title": "Scheduler Worker - Start", + "steps": [ + { + "file": "nomad/server.go", + "description": "## Server.NewServer()\n\nScheduler workers are started as the agent starts the `server` go routines.", + "line": 402 + }, + { + "file": "nomad/server.go", + "description": "## Server.setupWorkers()\n\nThe `setupWorkers()` function validates that there are enabled Schedulers by type and count. It then creates s.config.NumSchedulers by calling `NewWorker()`\n\nThe `_core` scheduler _**must**_ be enabled. **TODO: why?**\n", + "line": 1443, + "selection": { + "start": { + "line": 1442, + "character": 4 + }, + "end": { + "line": 1442, + "character": 12 + } + } + }, + { + "file": "nomad/worker.go", + "description": "## Worker.NewWorker\n\nNewWorker creates the Worker and starts `run()` in a goroutine.", + "line": 78 + }, + { + "file": "nomad/worker.go", + "description": "## Worker.run()\n\nThe `run()` function runs in a loop until it's paused, it's stopped, or the server indicates that it is shutting down. All of the work the `Worker` performs should be\nimplemented in or called from here.\n", + "line": 152 + } + ] +} \ No newline at end of file From 339316a774e8e97968ae58f7da9df1f212520de3 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Tue, 30 Nov 2021 17:51:13 -0500 Subject: [PATCH 04/30] Update shutdown to use context --- nomad/server.go | 10 ++--- nomad/server_test.go | 23 ++++++++++- nomad/testing.go | 2 +- nomad/worker.go | 74 ++++++++++++++++++++------------- nomad/worker_test.go | 97 ++++++++++++++++++++++++++++++++++++++------ 5 files changed, 157 insertions(+), 49 deletions(-) diff --git a/nomad/server.go b/nomad/server.go index 732f998cd0c..37ad35b454b 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -399,7 +399,7 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI, consulConfigEntr } // Initialize the scheduling workers - if err := s.setupWorkers(); err != nil { + if err := s.setupWorkers(s.shutdownCtx); err != nil { s.Shutdown() s.logger.Error("failed to start workers", "error", err) return nil, fmt.Errorf("Failed to start workers: %v", err) @@ -558,7 +558,7 @@ func (s *Server) reloadTLSConnections(newTLSConfig *config.TLSConfig) error { // Check if we can reload the RPC listener if s.rpcListener == nil || s.rpcCancel == nil { - s.logger.Warn("unable to reload configuration due to uninitialized rpc listner") + s.logger.Warn("unable to reload configuration due to uninitialized rpc listener") return fmt.Errorf("can't reload uninitialized RPC listener") } @@ -1440,7 +1440,7 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string) ( } // setupWorkers is used to start the scheduling workers -func (s *Server) setupWorkers() error { +func (s *Server) setupWorkers(ctx context.Context) error { // Check if all the schedulers are disabled if len(s.config.EnabledSchedulers) == 0 || s.config.NumSchedulers == 0 { s.logger.Warn("no enabled schedulers") @@ -1465,7 +1465,7 @@ func (s *Server) setupWorkers() error { s.logger.Info("starting scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) // Start the workers for i := 0; i < s.config.NumSchedulers; i++ { - if w, err := NewWorker(s); err != nil { + if w, err := NewWorker(ctx, s); err != nil { return err } else { s.logger.Debug("started scheduling worker", "id", w.ID(), "index", i+1, "of", s.config.NumSchedulers) @@ -1490,7 +1490,7 @@ func (s *Server) setupNewWorkers() error { // build a clean backing array and call setupWorkers like in the normal startup path s.workers = make([]*Worker, 0, s.config.NumSchedulers) - err := s.setupWorkers() + err := s.setupWorkers(s.shutdownCtx) if err != nil { return err } diff --git a/nomad/server_test.go b/nomad/server_test.go index 64e34de576b..77570fee9b6 100644 --- a/nomad/server_test.go +++ b/nomad/server_test.go @@ -540,13 +540,13 @@ func TestServer_InvalidSchedulers(t *testing.T) { } config.EnabledSchedulers = []string{"batch"} - err := s.setupWorkers() + err := s.setupWorkers(s.shutdownCtx) require.NotNil(err) require.Contains(err.Error(), "scheduler not enabled") // Set the config to have an unknown scheduler config.EnabledSchedulers = []string{"batch", structs.JobTypeCore, "foo"} - err = s.setupWorkers() + err = s.setupWorkers(s.shutdownCtx) require.NotNil(err) require.Contains(err.Error(), "foo") } @@ -577,3 +577,22 @@ func TestServer_RPCNameAndRegionValidation(t *testing.T) { tc.name, tc.region, tc.expected) } } + +func TestServer_Reload_NumSchedulers(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.NumSchedulers = 8 + c.Region = "global" + }) + defer cleanupS1() + + require.Equal(t, s1.config.NumSchedulers, len(s1.workers)) + + config := DefaultConfig() + config.NumSchedulers = 4 + require.NoError(t, s1.Reload(config)) + + time.Sleep(1 * time.Second) + require.Equal(t, config.NumSchedulers, len(s1.workers)) +} diff --git a/nomad/testing.go b/nomad/testing.go index 2822dc5fbfa..7c86f91e8fb 100644 --- a/nomad/testing.go +++ b/nomad/testing.go @@ -56,7 +56,7 @@ func TestServer(t testing.T, cb func(*Config)) (*Server, func()) { nodeNum := atomic.AddUint32(&nodeNumber, 1) config.NodeName = fmt.Sprintf("nomad-%03d", nodeNum) - // configer logger + // configure logger level := hclog.Trace if envLogLevel := os.Getenv("NOMAD_TEST_LOG_LEVEL"); envLogLevel != "" { level = hclog.LevelFromString(envLogLevel) diff --git a/nomad/worker.go b/nomad/worker.go index 5180c2d0601..5cc8f842668 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -58,9 +58,11 @@ type Worker struct { start time.Time id string paused bool - stop bool // Signals that the worker should stop stopped bool // Indicates that the worker is in a terminal state; read with Stopped() + ctx context.Context + cancelFn context.CancelFunc + pauseLock sync.Mutex pauseCond *sync.Cond @@ -75,7 +77,7 @@ type Worker struct { } // NewWorker starts a new worker associated with the given server -func NewWorker(srv *Server) (*Worker, error) { +func NewWorker(ctx context.Context, srv *Server) (*Worker, error) { w := &Worker{ srv: srv, start: time.Now(), @@ -83,6 +85,9 @@ func NewWorker(srv *Server) (*Worker, error) { } w.logger = srv.logger.ResetNamed("worker").With("worker_id", w.id) w.pauseCond = sync.NewCond(&w.pauseLock) + + w.ctx, w.cancelFn = context.WithCancel(ctx) + go w.run() return w, nil } @@ -109,10 +114,15 @@ func (w *Worker) checkPaused() { // Shutdown is used to signal that the worker should shutdown. func (w *Worker) Shutdown() { w.pauseLock.Lock() + wasPaused := w.paused w.paused = false - w.stop = true w.pauseLock.Unlock() - w.pauseCond.Broadcast() + + w.logger.Trace("shutdown request received") + w.cancelFn() + if wasPaused { + w.pauseCond.Broadcast() + } } // Stopped returns a boolean indicating if this worker has been stopped. @@ -127,40 +137,44 @@ func (w *Worker) ID() string { return w.id } -// shouldStop is used to check the worker state to see if conditions indicate -// that it should shutdown. -func (w *Worker) shouldStop() bool { - var localStop, shouldStop bool - w.pauseLock.Lock() - localStop = w.stop - w.pauseLock.Unlock() - - shouldStop = localStop || w.srv.IsShutdown() - return shouldStop -} - // markStopped is used to mark the worker as stopped and should be called in a // defer immediately upon entering the run() function. func (w *Worker) markStopped() { w.pauseLock.Lock() w.stopped = true w.pauseLock.Unlock() - w.logger.Trace("stopped") + w.logger.Debug("stopped") +} + +func (w *Worker) isDone() bool { + select { + case <-w.ctx.Done(): + return true + default: + return false + } } // run is the long-lived goroutine which is used to run the worker func (w *Worker) run() { defer w.markStopped() - w.logger.Trace("running") + w.logger.Debug("running") for { + // Check to see if the context has been cancelled. Server shutdown and Shutdown() + // should do this. + if w.isDone() { + return + } // Dequeue a pending evaluation eval, token, waitIndex, shutdown := w.dequeueEvaluation(dequeueTimeout) if shutdown { return } - // Check for a shutdown - if w.shouldStop() { + // since dequeue takes time, we could have shutdown the server after getting an eval that + // needs to be nacked before we exit. Explicitly checking the server to allow this eval + // to be processed on worker shutdown. + if w.srv.IsShutdown() { w.logger.Error("nacking eval because the server is shutting down", "eval", log.Fmt("%#v", eval)) w.sendNack(eval, token) return @@ -205,7 +219,7 @@ REQ: // Check if we are paused w.checkPaused() // Immediately check to see if the worker has been shutdown - if w.shouldStop() { + if w.isDone() { return nil, "", 0, true } @@ -214,7 +228,7 @@ REQ: err := w.srv.RPC("Eval.Dequeue", &req, &resp) metrics.MeasureSince([]string{"nomad", "worker", "dequeue_eval"}, start) if err != nil { - if time.Since(w.start) > dequeueErrGrace && !w.shouldStop() { + if time.Since(w.start) > dequeueErrGrace && !w.isDone() { w.logger.Error("failed to dequeue evaluation", "error", err) } @@ -289,7 +303,7 @@ func (w *Worker) sendAck(eval *structs.Evaluation, token string) { // snapshotMinIndex times calls to StateStore.SnapshotAfter which may block. func (w *Worker) snapshotMinIndex(waitIndex uint64, timeout time.Duration) (*state.StateSnapshot, error) { start := time.Now() - ctx, cancel := context.WithTimeout(w.srv.shutdownCtx, timeout) + ctx, cancel := context.WithTimeout(w.ctx, timeout) snap, err := w.srv.fsm.State().SnapshotMinIndex(ctx, waitIndex) cancel() metrics.MeasureSince([]string{"nomad", "worker", "wait_for_index"}, start) @@ -337,7 +351,8 @@ func (w *Worker) invokeScheduler(snap *state.StateSnapshot, eval *structs.Evalua // SubmitPlan is used to submit a plan for consideration. This allows // the worker to act as the planner for the scheduler. func (w *Worker) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, scheduler.State, error) { - // Check for a shutdown before plan submission + // Check for a shutdown before plan submission. Checking server state rathen than + // worker state to allow work in flight to complete before stopping. if w.srv.IsShutdown() { return nil, nil, fmt.Errorf("shutdown while planning") } @@ -407,7 +422,8 @@ SUBMIT: // UpdateEval is used to submit an updated evaluation. This allows // the worker to act as the planner for the scheduler. func (w *Worker) UpdateEval(eval *structs.Evaluation) error { - // Check for a shutdown before plan submission + // Check for a shutdown before plan submission. Checking server state rathen than + // worker state to allow a workers work in flight to complete before stopping. if w.srv.IsShutdown() { return fmt.Errorf("shutdown while planning") } @@ -445,7 +461,8 @@ SUBMIT: // CreateEval is used to create a new evaluation. This allows // the worker to act as the planner for the scheduler. func (w *Worker) CreateEval(eval *structs.Evaluation) error { - // Check for a shutdown before plan submission + // Check for a shutdown before plan submission. This consults the server Shutdown state + // instead of the worker's to prevent aborting work in flight. if w.srv.IsShutdown() { return fmt.Errorf("shutdown while planning") } @@ -486,7 +503,8 @@ SUBMIT: // ReblockEval is used to reinsert a blocked evaluation into the blocked eval // tracker. This allows the worker to act as the planner for the scheduler. func (w *Worker) ReblockEval(eval *structs.Evaluation) error { - // Check for a shutdown before plan submission + // Check for a shutdown before plan submission. This checks the server state rather than + // the worker's to prevent erroring on work in flight that would complete otherwise. if w.srv.IsShutdown() { return fmt.Errorf("shutdown while planning") } @@ -573,7 +591,7 @@ func (w *Worker) backoffErr(base, limit time.Duration) bool { select { case <-time.After(backoff): return false - case <-w.srv.shutdownCh: + case <-w.ctx.Done(): return true } } diff --git a/nomad/worker_test.go b/nomad/worker_test.go index 0569d1d6675..947fd80fa7b 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "fmt" "reflect" "sync" @@ -47,6 +48,19 @@ func init() { } } +// NewTestWorker returns the worker without calling it's run method. +func NewTestWorker(shutdownCtx context.Context, srv *Server) *Worker { + w := &Worker{ + srv: srv, + start: time.Now(), + id: uuid.Generate(), + } + w.logger = srv.logger.ResetNamed("worker").With("worker_id", w.id) + w.pauseCond = sync.NewCond(&w.pauseLock) + w.ctx, w.cancelFn = context.WithCancel(shutdownCtx) + return w +} + func TestWorker_dequeueEvaluation(t *testing.T) { t.Parallel() @@ -62,7 +76,7 @@ func TestWorker_dequeueEvaluation(t *testing.T) { s1.evalBroker.Enqueue(eval1) // Create a worker - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) // Attempt dequeue eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond) @@ -108,7 +122,7 @@ func TestWorker_dequeueEvaluation_SerialJobs(t *testing.T) { s1.evalBroker.Enqueue(eval2) // Create a worker - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) // Attempt dequeue eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond) @@ -168,7 +182,7 @@ func TestWorker_dequeueEvaluation_paused(t *testing.T) { s1.evalBroker.Enqueue(eval1) // Create a worker - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) w.pauseCond = sync.NewCond(&w.pauseLock) // PAUSE the worker @@ -212,7 +226,7 @@ func TestWorker_dequeueEvaluation_shutdown(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) // Create a worker - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) go func() { time.Sleep(10 * time.Millisecond) @@ -231,6 +245,53 @@ func TestWorker_dequeueEvaluation_shutdown(t *testing.T) { } } +func TestWorker_Shutdown(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.NumSchedulers = 0 + c.EnabledSchedulers = []string{structs.JobTypeService} + }) + defer cleanupS1() + testutil.WaitForLeader(t, s1.RPC) + + // Create a worker; since this tests the shutdown, you need the cancelFn too. + w := NewTestWorker(s1.shutdownCtx, s1) + + go func() { + time.Sleep(10 * time.Millisecond) + w.Shutdown() + }() + + // Attempt dequeue + eval, _, _, shutdown := w.dequeueEvaluation(10 * time.Millisecond) + require.True(t, shutdown) + require.Nil(t, eval) +} + +func TestWorker_Shutdown_paused(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.NumSchedulers = 0 + c.EnabledSchedulers = []string{structs.JobTypeService} + }) + defer cleanupS1() + testutil.WaitForLeader(t, s1.RPC) + + w, _ := NewWorker(s1.shutdownCtx, s1) + w.SetPause(true) + + go func() { + time.Sleep(10 * time.Millisecond) + w.Shutdown() + }() + + time.Sleep(511 * time.Millisecond) + // Verify that the worker is stopped. + require.True(t, w.Stopped()) +} + func TestWorker_sendAck(t *testing.T) { t.Parallel() @@ -246,7 +307,7 @@ func TestWorker_sendAck(t *testing.T) { s1.evalBroker.Enqueue(eval1) // Create a worker - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) // Attempt dequeue eval, token, _, _ := w.dequeueEvaluation(10 * time.Millisecond) @@ -301,7 +362,7 @@ func TestWorker_waitForIndex(t *testing.T) { }() // Wait for a future index - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) snap, err := w.snapshotMinIndex(index+1, time.Second) require.NoError(t, err) require.NotNil(t, snap) @@ -327,7 +388,7 @@ func TestWorker_invokeScheduler(t *testing.T) { }) defer cleanupS1() - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) eval := mock.Eval() eval.Type = "noop" @@ -380,7 +441,9 @@ func TestWorker_SubmitPlan(t *testing.T) { } // Attempt to submit a plan - w := &Worker{srv: s1, logger: s1.logger, evalToken: token} + w := NewTestWorker(s1.shutdownCtx, s1) + w.evalToken = token + result, state, err := w.SubmitPlan(plan) if err != nil { t.Fatalf("err: %v", err) @@ -442,7 +505,7 @@ func TestWorker_SubmitPlanNormalizedAllocations(t *testing.T) { plan.AppendPreemptedAlloc(preemptedAlloc, preemptingAllocID) // Attempt to submit a plan - w := &Worker{srv: s1, logger: s1.logger} + w := NewTestWorker(s1.shutdownCtx, s1) w.SubmitPlan(plan) assert.Equal(t, &structs.Allocation{ @@ -499,7 +562,9 @@ func TestWorker_SubmitPlan_MissingNodeRefresh(t *testing.T) { } // Attempt to submit a plan - w := &Worker{srv: s1, logger: s1.logger, evalToken: token} + w := NewTestWorker(s1.shutdownCtx, s1) + w.evalToken = token + result, state, err := w.SubmitPlan(plan) if err != nil { t.Fatalf("err: %v", err) @@ -556,7 +621,9 @@ func TestWorker_UpdateEval(t *testing.T) { eval2.Status = structs.EvalStatusComplete // Attempt to update eval - w := &Worker{srv: s1, logger: s1.logger, evalToken: token} + w := NewTestWorker(s1.shutdownCtx, s1) + w.evalToken = token + err = w.UpdateEval(eval2) if err != nil { t.Fatalf("err: %v", err) @@ -605,7 +672,9 @@ func TestWorker_CreateEval(t *testing.T) { eval2.PreviousEval = eval1.ID // Attempt to create eval - w := &Worker{srv: s1, logger: s1.logger, evalToken: token} + w := NewTestWorker(s1.shutdownCtx, s1) + w.evalToken = token + err = w.CreateEval(eval2) if err != nil { t.Fatalf("err: %v", err) @@ -667,7 +736,9 @@ func TestWorker_ReblockEval(t *testing.T) { eval2.QueuedAllocations = map[string]int{"web": 50} // Attempt to reblock eval - w := &Worker{srv: s1, logger: s1.logger, evalToken: token} + w := NewTestWorker(s1.shutdownCtx, s1) + w.evalToken = token + err = w.ReblockEval(eval2) if err != nil { t.Fatalf("err: %v", err) From 1a985b37810310d15a1b2b7870106058ef54e63f Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Wed, 1 Dec 2021 09:07:33 -0600 Subject: [PATCH 05/30] Apply suggestions from code review Co-authored-by: Derek Strickland <1111455+DerekStrickland@users.noreply.github.com> --- nomad/worker.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nomad/worker.go b/nomad/worker.go index 5cc8f842668..d1be5a0476e 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -351,7 +351,7 @@ func (w *Worker) invokeScheduler(snap *state.StateSnapshot, eval *structs.Evalua // SubmitPlan is used to submit a plan for consideration. This allows // the worker to act as the planner for the scheduler. func (w *Worker) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, scheduler.State, error) { - // Check for a shutdown before plan submission. Checking server state rathen than + // Check for a shutdown before plan submission. Checking server state rather than // worker state to allow work in flight to complete before stopping. if w.srv.IsShutdown() { return nil, nil, fmt.Errorf("shutdown while planning") @@ -422,7 +422,7 @@ SUBMIT: // UpdateEval is used to submit an updated evaluation. This allows // the worker to act as the planner for the scheduler. func (w *Worker) UpdateEval(eval *structs.Evaluation) error { - // Check for a shutdown before plan submission. Checking server state rathen than + // Check for a shutdown before plan submission. Checking server state rather than // worker state to allow a workers work in flight to complete before stopping. if w.srv.IsShutdown() { return fmt.Errorf("shutdown while planning") From 22f93b7abe2de4837345b6dec455c045d1d5e636 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Wed, 1 Dec 2021 18:16:36 -0500 Subject: [PATCH 06/30] Implement GET for SchedulerWorker API + tests --- command/agent/agent_endpoint.go | 79 +++++++- command/agent/agent_endpoint_test.go | 280 +++++++++++++++++++++++++++ command/agent/http.go | 1 + 3 files changed, 359 insertions(+), 1 deletion(-) diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index dc4afd1ef6c..474aeea0e81 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -362,7 +362,7 @@ func (s *HTTPServer) agentPprof(reqType pprof.ReqType, resp http.ResponseWriter, // Parse query param int values // Errors are dropped here and default to their zero values. - // This is to mimick the functionality that net/pprof implements. + // This is to mimic the functionality that net/pprof implements. seconds, _ := strconv.Atoi(req.URL.Query().Get("seconds")) debug, _ := strconv.Atoi(req.URL.Query().Get("debug")) gc, _ := strconv.Atoi(req.URL.Query().Get("gc")) @@ -740,3 +740,80 @@ func (s *HTTPServer) AgentHostRequest(resp http.ResponseWriter, req *http.Reques return reply, rpcErr } + +// AgentSchedulerWorkerConfig +type AgentSchedulerWorkerConfig struct { + NumSchedulers uint `json:"num_schedulers"` + EnabledSchedulers []string `json:"enabled_schedulers"` +} + +// AgentSchedulerWorkerRequest is used to query the count (and state eventually) +// of the scheduler workers running in a Nomad server agent. +// This endpoint can also be used to update the count of running workers for a +// given agent. +func (s *HTTPServer) AgentSchedulerWorkerRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + if s.agent.Server() == nil { + return nil, CodedError(http.StatusBadRequest, "server only endpoint") + } + switch req.Method { + case "PUT", "POST": + return s.updateScheduleWorkers(resp, req) + case "GET": + return s.getScheduleWorkersInfo(resp, req) + default: + return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod) + } +} + +func (s *HTTPServer) getScheduleWorkersInfo(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + var secret string + s.parseToken(req, &secret) + + // Check agent read permissions + if aclObj, err := s.agent.Server().ResolveToken(secret); err != nil { + return nil, CodedError(http.StatusInternalServerError, err.Error()) + } else if aclObj != nil && !aclObj.AllowAgentRead() { + return nil, CodedError(http.StatusForbidden, structs.ErrPermissionDenied.Error()) + } + + config := s.agent.server.GetConfig() + response := &AgentSchedulerWorkerConfig{ + NumSchedulers: uint(config.NumSchedulers), + EnabledSchedulers: config.EnabledSchedulers, + } + + return response, nil +} + +func (s *HTTPServer) updateScheduleWorkers(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + server := s.agent.Server() + if server == nil { + return nil, CodedError(400, "server only endpoint") + } + + // // Get the servers from the request + // qsNumSchedulers := req.URL.Query()["num_schedulers"] + // if len(qsNumSchedulers) != 1 + // if newNumSchedulers == 0 { + // return nil, CodedError(400, "missing server address") + // } + + // var secret string + // s.parseToken(req, &secret) + + // // Check agent write permissions + // if aclObj, err := s.agent.Client().ResolveToken(secret); err != nil { + // return nil, err + // } else if aclObj != nil && !aclObj.AllowAgentWrite() { + // return nil, structs.ErrPermissionDenied + // } + + // // Set the servers list into the client + // s.agent.logger.Trace("adding servers to the client's primary server list", "servers", servers, "path", "/v1/agent/servers", "method", "PUT") + // if _, err := client.SetServers(servers); err != nil { + // s.agent.logger.Error("failed adding servers to client's server list", "servers", servers, "error", err, "path", "/v1/agent/servers", "method", "PUT") + // //TODO is this the right error to return? + // return nil, CodedError(400, err.Error()) + // } + return nil, nil +} diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index 795dffd385a..f0b770eba30 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -1463,3 +1463,283 @@ func TestHTTP_XSS_Monitor(t *testing.T) { }) } } + +/* + SchedulerWorkerAPI tests +*/ +type scheduleWorkerTest_workerRequestTest struct { + name string // test case name + requiresACL bool // prevents test cases that require ACLs from running in the non-ACL version + serverConfig AgentSchedulerWorkerConfig + request schedulerWorkerTest_testRequest + whenACLNotEnabled schedulerWorkerTest_testExpect + whenACLEnabled schedulerWorkerTest_testExpect +} +type schedulerWorkerTest_testRequest struct { + verb string + aclToken string + requestBody string +} +type schedulerWorkerTest_testExpect struct { + expectedResponseCode int + expectedResponse interface{} +} + +// schedulerWorkerTest_serverConfig creates a test function that can merge +// in an existing Config. Passing nil as the parameter will not override any +// thing. +func schedulerWorkerTest_serverConfig(inConfig *Config) func(*Config) { + if inConfig != nil { + return func(c *Config) { + inConfig.Merge(c) + } + } else { + return func(c *Config) {} + } +} + +// These test cases are run for both the ACL and Non-ACL enabled servers. When +// ACLS are not enabled, the request.aclTokens are ignored. +func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { + forbidden := schedulerWorkerTest_testExpect{ + expectedResponseCode: http.StatusForbidden, + expectedResponse: structs.ErrPermissionDenied.Error(), + } + invalidMethod := schedulerWorkerTest_testExpect{ + expectedResponseCode: http.StatusMethodNotAllowed, + expectedResponse: ErrInvalidMethod, + } + success1 := schedulerWorkerTest_testExpect{ + expectedResponseCode: http.StatusOK, + expectedResponse: &AgentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "system"}, NumSchedulers: 8}, + } + + return []scheduleWorkerTest_workerRequestTest{ + { + name: "bad verb", + request: schedulerWorkerTest_testRequest{ + verb: "FOO", + aclToken: "", + requestBody: "", + }, + whenACLNotEnabled: invalidMethod, + whenACLEnabled: invalidMethod, + }, + { + name: "get without token", + request: schedulerWorkerTest_testRequest{ + verb: "GET", + aclToken: "", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: forbidden, + }, + { + name: "get with management token", + request: schedulerWorkerTest_testRequest{ + verb: "GET", + aclToken: "management", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: success1, + }, + { + name: "get with read token", + requiresACL: true, + request: schedulerWorkerTest_testRequest{ + verb: "GET", + aclToken: "agent_read", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: success1, + }, + { + name: "get with write token", + requiresACL: true, + request: schedulerWorkerTest_testRequest{ + verb: "GET", + aclToken: "agent_write", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: success1, + }, + { + name: "post with no token", + request: schedulerWorkerTest_testRequest{ + verb: "POST", + aclToken: "", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: forbidden, + }, + { + name: "put with no token", + request: schedulerWorkerTest_testRequest{ + verb: "PUT", + aclToken: "", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: forbidden, + }, + { + name: "post with invalid token", + request: schedulerWorkerTest_testRequest{ + verb: "POST", + aclToken: "node_write", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: forbidden, + }, + { + name: "put with invalid token", + request: schedulerWorkerTest_testRequest{ + verb: "PUT", + aclToken: "node_write", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: forbidden, + }, + { + name: "post with valid token", + request: schedulerWorkerTest_testRequest{ + verb: "POST", + aclToken: "agent_write", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: success1, + }, + { + name: "put with valid token", + request: schedulerWorkerTest_testRequest{ + verb: "PUT", + aclToken: "agent_write", + requestBody: "", + }, + whenACLNotEnabled: success1, + whenACLEnabled: success1, + }, + } +} + +func TestHTTP_AgentSchedulerWorkerRequest_NoACL(t *testing.T) { + configFn := func(c *Config) { + var numSchedulers = 8 + c.Server.NumSchedulers = &numSchedulers + c.Server.EnabledSchedulers = []string{"_core", "system"} + c.Client.Enabled = false + } + testFn := func(s *TestAgent) { + for _, tc := range schedulerWorkerTest_testCases() { + t.Run(tc.name, func(t *testing.T) { + + req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", nil) + require.Nil(t, err) + respW := httptest.NewRecorder() + workersI, err := s.Server.AgentSchedulerWorkerRequest(respW, req) + + switch tc.whenACLNotEnabled.expectedResponseCode { + case http.StatusBadRequest, http.StatusForbidden, http.StatusMethodNotAllowed: + schedulerWorkerTest_parseError(t, false, tc, workersI, err) + case http.StatusOK: + schedulerWorkerTest_parseSuccess(t, false, tc, workersI, err) + default: + require.Failf(t, "unexpected status code", "code: %v", tc.whenACLNotEnabled.expectedResponseCode) + } + }) + } + } + + httpTest(t, configFn, testFn) +} + +func TestHTTP_AgentSchedulerWorkerRequest_ACL(t *testing.T) { + configFn := func(c *Config) { + var numSchedulers = 8 + c.Server.NumSchedulers = &numSchedulers + c.Server.EnabledSchedulers = []string{"_core", "system"} + c.Client.Enabled = false + } + + tests := func(s *TestAgent) { + state := s.Agent.server.State() + var tokens map[string]*structs.ACLToken = make(map[string]*structs.ACLToken) + + tokens["management"] = s.RootToken + tokens["agent_read"] = mock.CreatePolicyAndToken(t, state, 1005, "agent_read", mock.AgentPolicy(acl.PolicyRead)) + tokens["agent_write"] = mock.CreatePolicyAndToken(t, state, 1007, "agent_write", mock.AgentPolicy(acl.PolicyWrite)) + tokens["node_write"] = mock.CreatePolicyAndToken(t, state, 1009, "node_write", mock.NodePolicy(acl.PolicyWrite)) + + for _, tc := range schedulerWorkerTest_testCases() { + t.Run(tc.name, func(t *testing.T) { + + req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", nil) + if tc.request.aclToken != "" { + setToken(req, tokens[tc.request.aclToken]) + } + require.Nil(t, err) + + respW := httptest.NewRecorder() + workersI, err := s.Server.AgentSchedulerWorkerRequest(respW, req) + + switch tc.whenACLEnabled.expectedResponseCode { + case http.StatusOK: + schedulerWorkerTest_parseSuccess(t, true, tc, workersI, err) + case http.StatusBadRequest, http.StatusForbidden, http.StatusMethodNotAllowed: + schedulerWorkerTest_parseError(t, true, tc, workersI, err) + default: + require.Failf(t, "unexpected status code", "code: %v", tc.whenACLEnabled.expectedResponseCode) + } + }) + } + } + + httpACLTest(t, configFn, tests) +} + +func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc scheduleWorkerTest_workerRequestTest, workersI interface{}, err error) { + require.NoError(t, err) + require.NotNil(t, workersI) + + testExpect := tc.whenACLNotEnabled + if isACLEnabled { + testExpect = tc.whenACLNotEnabled + } + + // test into the response when we expect an okay + tcConfig, ok := testExpect.expectedResponse.(*AgentSchedulerWorkerConfig) + require.True(t, ok, "expected response malformed - this is an issue with a test case.") + + workersConfig, ok := workersI.(*AgentSchedulerWorkerConfig) + require.True(t, ok, "response can not cast to an AgentSchedulerWorkerConfig") + require.NotNil(t, workersConfig) + + require.Equal(t, tcConfig.NumSchedulers, workersConfig.NumSchedulers) + require.ElementsMatch(t, tcConfig.EnabledSchedulers, workersConfig.EnabledSchedulers) +} + +// schedulerWorkerTest_parseError parses the error response given +// from the +func schedulerWorkerTest_parseError(t *testing.T, isACLEnabled bool, tc scheduleWorkerTest_workerRequestTest, workersI interface{}, err error) { + require.Error(t, err) + require.Nil(t, workersI) + + codedError, ok := err.(HTTPCodedError) + require.True(t, ok, "expected an HTTPCodedError") + testExpect := tc.whenACLNotEnabled + + if isACLEnabled { + testExpect = tc.whenACLNotEnabled + } + + require.Equal(t, testExpect.expectedResponseCode, codedError.Code()) + require.Equal(t, testExpect.expectedResponse, codedError.Error()) +} diff --git a/command/agent/http.go b/command/agent/http.go index 4a93ae74f76..19dc7fce2cb 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -292,6 +292,7 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/agent/members", s.wrap(s.AgentMembersRequest)) s.mux.HandleFunc("/v1/agent/force-leave", s.wrap(s.AgentForceLeaveRequest)) s.mux.HandleFunc("/v1/agent/servers", s.wrap(s.AgentServersRequest)) + s.mux.HandleFunc("/v1/agent/workers", s.wrap(s.AgentSchedulerWorkerRequest)) s.mux.HandleFunc("/v1/agent/keyring/", s.wrap(s.KeyringOperationRequest)) s.mux.HandleFunc("/v1/agent/health", s.wrap(s.HealthRequest)) s.mux.HandleFunc("/v1/agent/host", s.wrap(s.AgentHostRequest)) From 48428e7df345e62aea2982def34860a4ff1ba38c Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 3 Dec 2021 17:49:38 -0500 Subject: [PATCH 07/30] Wired API, refactors, more testing --- api/agent.go | 43 +++++++++ api/agent_test.go | 15 +++ command/agent/agent_endpoint.go | 103 +++++++++++--------- command/agent/agent_endpoint_test.go | 65 ++++++------- command/agent/http.go | 2 +- nomad/server.go | 139 ++++++++++++++++++++++++--- nomad/server_test.go | 52 +++++++++- 7 files changed, 324 insertions(+), 95 deletions(-) diff --git a/api/agent.go b/api/agent.go index 39bfb95443c..c95f3829955 100644 --- a/api/agent.go +++ b/api/agent.go @@ -483,3 +483,46 @@ type HostDataResponse struct { AgentID string HostData *HostData `json:",omitempty"` } + +// GetSchedulerWorkerConfig returns the targeted agent's worker pool configuration +func (a *Agent) GetSchedulerWorkerConfig() (*SchedulerWorkerPoolArgs, error) { + var resp AgentSchedulerWorkerConfigResponse + _, err := a.client.query("/v1/agent/workers", &resp, nil) + if err != nil { + return nil, err + } + + return &SchedulerWorkerPoolArgs{NumSchedulers: resp.NumSchedulers, EnabledSchedulers: resp.EnabledSchedulers}, nil +} + +// SetSchedulerWorkerConfig attempts to update the targeted agent's worker pool configuration +func (a *Agent) SetSchedulerWorkerConfig(args SchedulerWorkerPoolArgs) (*SchedulerWorkerPoolArgs, error) { + req := AgentSchedulerWorkerConfigRequest{ + NumSchedulers: args.NumSchedulers, + EnabledSchedulers: args.EnabledSchedulers, + } + + var resp AgentSchedulerWorkerConfigResponse + _, err := a.client.write("/v1/agent/workers", &req, &resp, nil) + if err != nil { + return nil, err + } + + return &SchedulerWorkerPoolArgs{NumSchedulers: resp.NumSchedulers, EnabledSchedulers: resp.EnabledSchedulers}, nil +} + +type SchedulerWorkerPoolArgs struct { + NumSchedulers int + EnabledSchedulers []string +} + +// AgentSchedulerWorkerConfig +type AgentSchedulerWorkerConfigRequest struct { + NumSchedulers int `json:"num_schedulers"` + EnabledSchedulers []string `json:"enabled_schedulers"` +} + +type AgentSchedulerWorkerConfigResponse struct { + NumSchedulers int `json:"num_schedulers"` + EnabledSchedulers []string `json:"enabled_schedulers"` +} diff --git a/api/agent_test.go b/api/agent_test.go index 8fff6cfb0c5..715db417cbe 100644 --- a/api/agent_test.go +++ b/api/agent_test.go @@ -456,3 +456,18 @@ func TestAgentProfile(t *testing.T) { require.Nil(t, resp) } } + +func TestAgent_SchedulerWorkerConfig(t *testing.T) { + t.Parallel() + + c, s := makeClient(t, nil, nil) + defer s.Stop() + a := c.Agent() + + config, err := a.GetSchedulerWorkerConfig() + require.Nil(t, err) + newConfig := SchedulerWorkerPoolArgs{NumSchedulers: 0, EnabledSchedulers: []string{"_core", "system"}} + resp, err := a.SetSchedulerWorkerConfig(newConfig) + require.NoError(t, err) + assert.NotEqual(t, config, resp) +} diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index 474aeea0e81..2593a74d62a 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -19,6 +19,7 @@ import ( cstructs "github.com/hashicorp/nomad/client/structs" "github.com/hashicorp/nomad/command/agent/host" "github.com/hashicorp/nomad/command/agent/pprof" + "github.com/hashicorp/nomad/nomad" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/serf/serf" "github.com/mitchellh/copystructure" @@ -741,31 +742,30 @@ func (s *HTTPServer) AgentHostRequest(resp http.ResponseWriter, req *http.Reques return reply, rpcErr } -// AgentSchedulerWorkerConfig -type AgentSchedulerWorkerConfig struct { - NumSchedulers uint `json:"num_schedulers"` - EnabledSchedulers []string `json:"enabled_schedulers"` -} - -// AgentSchedulerWorkerRequest is used to query the count (and state eventually) +// AgentSchedulerWorkerConfigRequest is used to query the count (and state eventually) // of the scheduler workers running in a Nomad server agent. // This endpoint can also be used to update the count of running workers for a // given agent. -func (s *HTTPServer) AgentSchedulerWorkerRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { +func (s *HTTPServer) AgentSchedulerWorkerConfigRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { if s.agent.Server() == nil { return nil, CodedError(http.StatusBadRequest, "server only endpoint") } switch req.Method { case "PUT", "POST": - return s.updateScheduleWorkers(resp, req) + return s.updateScheduleWorkersConfig(resp, req) case "GET": - return s.getScheduleWorkersInfo(resp, req) + return s.getScheduleWorkersConfig(resp, req) default: return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod) } } -func (s *HTTPServer) getScheduleWorkersInfo(resp http.ResponseWriter, req *http.Request) (interface{}, error) { +func (s *HTTPServer) getScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + srv := s.agent.Server() + if srv == nil { + return nil, CodedError(http.StatusBadRequest, "server only endpoint") + } + var secret string s.parseToken(req, &secret) @@ -776,44 +776,61 @@ func (s *HTTPServer) getScheduleWorkersInfo(resp http.ResponseWriter, req *http. return nil, CodedError(http.StatusForbidden, structs.ErrPermissionDenied.Error()) } - config := s.agent.server.GetConfig() - response := &AgentSchedulerWorkerConfig{ - NumSchedulers: uint(config.NumSchedulers), + config := srv.GetSchedulerWorkerConfig() + response := &agentSchedulerWorkerConfig{ + NumSchedulers: config.NumSchedulers, EnabledSchedulers: config.EnabledSchedulers, } - return response, nil } -func (s *HTTPServer) updateScheduleWorkers(resp http.ResponseWriter, req *http.Request) (interface{}, error) { - server := s.agent.Server() - if server == nil { +func (s *HTTPServer) updateScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + srv := s.agent.Server() + if srv == nil { return nil, CodedError(400, "server only endpoint") } - // // Get the servers from the request - // qsNumSchedulers := req.URL.Query()["num_schedulers"] - // if len(qsNumSchedulers) != 1 - // if newNumSchedulers == 0 { - // return nil, CodedError(400, "missing server address") - // } - - // var secret string - // s.parseToken(req, &secret) - - // // Check agent write permissions - // if aclObj, err := s.agent.Client().ResolveToken(secret); err != nil { - // return nil, err - // } else if aclObj != nil && !aclObj.AllowAgentWrite() { - // return nil, structs.ErrPermissionDenied - // } - - // // Set the servers list into the client - // s.agent.logger.Trace("adding servers to the client's primary server list", "servers", servers, "path", "/v1/agent/servers", "method", "PUT") - // if _, err := client.SetServers(servers); err != nil { - // s.agent.logger.Error("failed adding servers to client's server list", "servers", servers, "error", err, "path", "/v1/agent/servers", "method", "PUT") - // //TODO is this the right error to return? - // return nil, CodedError(400, err.Error()) - // } - return nil, nil + var secret string + s.parseToken(req, &secret) + + // Check agent write permissions + if aclObj, err := srv.ResolveToken(secret); err != nil { + return nil, CodedError(http.StatusInternalServerError, err.Error()) + } else if aclObj != nil && !aclObj.AllowAgentWrite() { + return nil, CodedError(http.StatusForbidden, structs.ErrPermissionDenied.Error()) + } + + var args agentSchedulerWorkerConfig + + if err := decodeBody(req, &args); err != nil { + return nil, CodedError(http.StatusBadRequest, err.Error()) + } + newArgs := nomad.SchedulerWorkerPoolArgs{ + NumSchedulers: args.NumSchedulers, + EnabledSchedulers: args.EnabledSchedulers, + } + if newArgs.IsInvalid() { + return nil, CodedError(http.StatusBadRequest, "invalid arguments") + } + reply := srv.SetSchedulerWorkerConfig(newArgs) + + response := &agentSchedulerWorkerConfig{ + NumSchedulers: reply.NumSchedulers, + EnabledSchedulers: reply.EnabledSchedulers, + } + + return response, nil +} + +type agentSchedulerWorkerConfig struct { + NumSchedulers int `json:"num_schedulers"` + EnabledSchedulers []string `json:"enabled_schedulers"` +} +type agentSchedulerWorkerConfigRequest struct { + agentSchedulerWorkerConfig +} + +type agentSchedulerWorkerConfigResponse struct { + NumSchedulers int `json:"num_schedulers"` + EnabledSchedulers []string `json:"enabled_schedulers"` } diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index f0b770eba30..0ac02966ebe 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -1470,7 +1470,6 @@ func TestHTTP_XSS_Monitor(t *testing.T) { type scheduleWorkerTest_workerRequestTest struct { name string // test case name requiresACL bool // prevents test cases that require ACLs from running in the non-ACL version - serverConfig AgentSchedulerWorkerConfig request schedulerWorkerTest_testRequest whenACLNotEnabled schedulerWorkerTest_testExpect whenACLEnabled schedulerWorkerTest_testExpect @@ -1485,19 +1484,6 @@ type schedulerWorkerTest_testExpect struct { expectedResponse interface{} } -// schedulerWorkerTest_serverConfig creates a test function that can merge -// in an existing Config. Passing nil as the parameter will not override any -// thing. -func schedulerWorkerTest_serverConfig(inConfig *Config) func(*Config) { - if inConfig != nil { - return func(c *Config) { - inConfig.Merge(c) - } - } else { - return func(c *Config) {} - } -} - // These test cases are run for both the ACL and Non-ACL enabled servers. When // ACLS are not enabled, the request.aclTokens are ignored. func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { @@ -1511,7 +1497,12 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { } success1 := schedulerWorkerTest_testExpect{ expectedResponseCode: http.StatusOK, - expectedResponse: &AgentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "system"}, NumSchedulers: 8}, + expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, + } + + success2 := schedulerWorkerTest_testExpect{ + expectedResponseCode: http.StatusOK, + expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 9}, } return []scheduleWorkerTest_workerRequestTest{ @@ -1572,9 +1563,9 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { request: schedulerWorkerTest_testRequest{ verb: "POST", aclToken: "", - requestBody: "", + requestBody: `{"num_schedulers":9,"enabled_schedulers":["_core", "batch"]}`, }, - whenACLNotEnabled: success1, + whenACLNotEnabled: success2, whenACLEnabled: forbidden, }, { @@ -1582,7 +1573,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { request: schedulerWorkerTest_testRequest{ verb: "PUT", aclToken: "", - requestBody: "", + requestBody: `{"num_schedulers":8,"enabled_schedulers":["_core", "batch"]}`, }, whenACLNotEnabled: success1, whenACLEnabled: forbidden, @@ -1592,9 +1583,9 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { request: schedulerWorkerTest_testRequest{ verb: "POST", aclToken: "node_write", - requestBody: "", + requestBody: `{"num_schedulers":9,"enabled_schedulers":["_core", "batch"]}`, }, - whenACLNotEnabled: success1, + whenACLNotEnabled: success2, whenACLEnabled: forbidden, }, { @@ -1602,7 +1593,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { request: schedulerWorkerTest_testRequest{ verb: "PUT", aclToken: "node_write", - requestBody: "", + requestBody: `{"num_schedulers":8,"enabled_schedulers":["_core", "batch"]}`, }, whenACLNotEnabled: success1, whenACLEnabled: forbidden, @@ -1612,17 +1603,17 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { request: schedulerWorkerTest_testRequest{ verb: "POST", aclToken: "agent_write", - requestBody: "", + requestBody: `{"num_schedulers":9,"enabled_schedulers":["_core", "batch"]}`, }, - whenACLNotEnabled: success1, - whenACLEnabled: success1, + whenACLNotEnabled: success2, + whenACLEnabled: success2, }, { name: "put with valid token", request: schedulerWorkerTest_testRequest{ verb: "PUT", aclToken: "agent_write", - requestBody: "", + requestBody: `{"num_schedulers":8,"enabled_schedulers":["_core", "batch"]}`, }, whenACLNotEnabled: success1, whenACLEnabled: success1, @@ -1634,17 +1625,17 @@ func TestHTTP_AgentSchedulerWorkerRequest_NoACL(t *testing.T) { configFn := func(c *Config) { var numSchedulers = 8 c.Server.NumSchedulers = &numSchedulers - c.Server.EnabledSchedulers = []string{"_core", "system"} + c.Server.EnabledSchedulers = []string{"_core", "batch"} c.Client.Enabled = false } testFn := func(s *TestAgent) { for _, tc := range schedulerWorkerTest_testCases() { t.Run(tc.name, func(t *testing.T) { - req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", nil) + req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) require.Nil(t, err) respW := httptest.NewRecorder() - workersI, err := s.Server.AgentSchedulerWorkerRequest(respW, req) + workersI, err := s.Server.AgentSchedulerWorkerConfigRequest(respW, req) switch tc.whenACLNotEnabled.expectedResponseCode { case http.StatusBadRequest, http.StatusForbidden, http.StatusMethodNotAllowed: @@ -1665,7 +1656,7 @@ func TestHTTP_AgentSchedulerWorkerRequest_ACL(t *testing.T) { configFn := func(c *Config) { var numSchedulers = 8 c.Server.NumSchedulers = &numSchedulers - c.Server.EnabledSchedulers = []string{"_core", "system"} + c.Server.EnabledSchedulers = []string{"_core", "batch"} c.Client.Enabled = false } @@ -1681,14 +1672,13 @@ func TestHTTP_AgentSchedulerWorkerRequest_ACL(t *testing.T) { for _, tc := range schedulerWorkerTest_testCases() { t.Run(tc.name, func(t *testing.T) { - req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", nil) + req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) if tc.request.aclToken != "" { setToken(req, tokens[tc.request.aclToken]) } require.Nil(t, err) - respW := httptest.NewRecorder() - workersI, err := s.Server.AgentSchedulerWorkerRequest(respW, req) + workersI, err := s.Server.AgentSchedulerWorkerConfigRequest(respW, req) switch tc.whenACLEnabled.expectedResponseCode { case http.StatusOK: @@ -1715,11 +1705,11 @@ func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc schedu } // test into the response when we expect an okay - tcConfig, ok := testExpect.expectedResponse.(*AgentSchedulerWorkerConfig) + tcConfig, ok := testExpect.expectedResponse.(*agentSchedulerWorkerConfig) require.True(t, ok, "expected response malformed - this is an issue with a test case.") - workersConfig, ok := workersI.(*AgentSchedulerWorkerConfig) - require.True(t, ok, "response can not cast to an AgentSchedulerWorkerConfig") + workersConfig, ok := workersI.(*agentSchedulerWorkerConfig) + require.True(t, ok, "response can not cast to an agentSchedulerWorkerConfig") require.NotNil(t, workersConfig) require.Equal(t, tcConfig.NumSchedulers, workersConfig.NumSchedulers) @@ -1727,7 +1717,8 @@ func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc schedu } // schedulerWorkerTest_parseError parses the error response given -// from the +// from the API call to make sure that it's a coded error and is the +// expected value from the test case func schedulerWorkerTest_parseError(t *testing.T, isACLEnabled bool, tc scheduleWorkerTest_workerRequestTest, workersI interface{}, err error) { require.Error(t, err) require.Nil(t, workersI) @@ -1737,7 +1728,7 @@ func schedulerWorkerTest_parseError(t *testing.T, isACLEnabled bool, tc schedule testExpect := tc.whenACLNotEnabled if isACLEnabled { - testExpect = tc.whenACLNotEnabled + testExpect = tc.whenACLEnabled } require.Equal(t, testExpect.expectedResponseCode, codedError.Code()) diff --git a/command/agent/http.go b/command/agent/http.go index 19dc7fce2cb..00cdf41c0fb 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -292,7 +292,7 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/agent/members", s.wrap(s.AgentMembersRequest)) s.mux.HandleFunc("/v1/agent/force-leave", s.wrap(s.AgentForceLeaveRequest)) s.mux.HandleFunc("/v1/agent/servers", s.wrap(s.AgentServersRequest)) - s.mux.HandleFunc("/v1/agent/workers", s.wrap(s.AgentSchedulerWorkerRequest)) + s.mux.HandleFunc("/v1/agent/workers", s.wrap(s.AgentSchedulerWorkerConfigRequest)) s.mux.HandleFunc("/v1/agent/keyring/", s.wrap(s.KeyringOperationRequest)) s.mux.HandleFunc("/v1/agent/health", s.wrap(s.HealthRequest)) s.mux.HandleFunc("/v1/agent/host", s.wrap(s.AgentHostRequest)) diff --git a/nomad/server.go b/nomad/server.go index 37ad35b454b..7f0de78e9f0 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -226,7 +226,10 @@ type Server struct { vault VaultClient // Worker used for processing - workers []*Worker + workers []*Worker + currentNumSchedulers int + currentEnabledSchedulers []string + workerLock sync.RWMutex // aclCache is used to maintain the parsed ACL objects aclCache *lru.TwoQueueCache @@ -809,13 +812,12 @@ func (s *Server) Reload(newConfig *Config) error { s.EnterpriseState.ReloadLicense(newConfig) } - if newConfig.NumSchedulers != s.config.NumSchedulers { - s.logger.Debug("changing number of schedulers", "from", s.config.NumSchedulers, "to", newConfig.NumSchedulers) - s.config.NumSchedulers = newConfig.NumSchedulers - if err := s.setupNewWorkers(); err != nil { - s.logger.Error("error creating new workers", "error", err) - _ = multierror.Append(&mErr, err) + workerPoolArgs := unsafeGetSchedulerWorkerPoolArgsFromConfig(newConfig) + if reload, newVals := shouldReloadSchedulers(s, workerPoolArgs); reload { + if newVals.IsValid() { + reloadSchedulers(s, newVals) } + reloadSchedulers(s, newVals) } return mErr.ErrorOrNil() @@ -1439,8 +1441,119 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string) ( return serf.Create(conf) } +// shouldReloadSchedulers checks the new config to determine if the scheduler worker pool +// needs to be updated. If so, returns true and a pointer to a populated SchedulerWorkerPoolArgs +func shouldReloadSchedulers(s *Server, newPoolArgs *SchedulerWorkerPoolArgs) (bool, *SchedulerWorkerPoolArgs) { + s.workerLock.RLock() + defer s.workerLock.RUnlock() + + newSchedulers := make([]string, len(newPoolArgs.EnabledSchedulers)) + copy(newSchedulers, newPoolArgs.EnabledSchedulers) + sort.Strings(newSchedulers) + + if s.config.NumSchedulers != newPoolArgs.NumSchedulers { + return true, newPoolArgs + } + + oldSchedulers := make([]string, len(s.config.EnabledSchedulers)) + copy(oldSchedulers, s.config.EnabledSchedulers) + sort.Strings(oldSchedulers) + + for i, v := range newSchedulers { + if oldSchedulers[i] != v { + return true, newPoolArgs + } + } + + return false, nil +} + +// SchedulerWorkerPoolArgs are the two key configuration options for a Nomad server's +// scheduler worker pool. Before using, you should always verify that they are rational +// using IsValid() or IsInvalid() +type SchedulerWorkerPoolArgs struct { + NumSchedulers int + EnabledSchedulers []string +} + +// IsInvalid returns true when the SchedulerWorkerPoolArgs.IsValid is false +func (swpa SchedulerWorkerPoolArgs) IsInvalid() bool { + return !swpa.IsValid() +} + +// IsValid verifies that the pool arguments are valid. That is, they have a non-negative +// numSchedulers value and the enabledSchedulers list has _core and only refers to known +// schedulers. +func (swpa SchedulerWorkerPoolArgs) IsValid() bool { + if swpa.NumSchedulers < 0 { + // the pool has to be non-negative + return false + } + + // validate the scheduler list against the builtin types and _core + foundCore := false + for _, sched := range swpa.EnabledSchedulers { + if sched == structs.JobTypeCore { + foundCore = true + continue // core is not in the BuiltinSchedulers map, so we need to skip that check + } + + if _, ok := scheduler.BuiltinSchedulers[sched]; !ok { + return false // found an unknown scheduler in the list; bailing out + } + } + + return foundCore +} + +func unsafeGetSchedulerWorkerPoolArgsFromConfig(c *Config) *SchedulerWorkerPoolArgs { + return &SchedulerWorkerPoolArgs{ + NumSchedulers: c.NumSchedulers, + EnabledSchedulers: c.EnabledSchedulers, + } +} + +func (s *Server) GetSchedulerWorkerConfig() SchedulerWorkerPoolArgs { + s.workerLock.RLock() + defer s.workerLock.RUnlock() + return *unsafeGetSchedulerWorkerPoolArgsFromConfig(s.config) +} + +func (s *Server) SetSchedulerWorkerConfig(newArgs SchedulerWorkerPoolArgs) SchedulerWorkerPoolArgs { + if reload, newVals := shouldReloadSchedulers(s, &newArgs); reload { + if newVals.IsValid() { + reloadSchedulers(s, newVals) + } + reloadSchedulers(s, newVals) + } + return s.GetSchedulerWorkerConfig() +} + +// reloadSchedulers validates the passed scheduler worker pool arguments, locks the +// workerLock, applies the new values to the s.config, and restarts the pool +func reloadSchedulers(s *Server, newArgs *SchedulerWorkerPoolArgs) { + if newArgs == nil || newArgs.IsInvalid() { + s.logger.Info("received invalid arguments for scheduler pool reload; ignoring") + return + } + s.workerLock.Lock() + defer s.workerLock.Unlock() + // TODO: If EnabledSchedulers didn't change, we can scale rather than drain and rebuild + s.config.NumSchedulers = newArgs.NumSchedulers + s.config.EnabledSchedulers = newArgs.EnabledSchedulers + s.unsafeSetupNewWorkers() +} + // setupWorkers is used to start the scheduling workers func (s *Server) setupWorkers(ctx context.Context) error { + s.workerLock.Lock() + defer s.workerLock.Unlock() + return s.unsafeSetupWorkers(ctx) +} + +// unsafeSetupWorkers directly manipulates the server.config, so it is not safe to +// call concurrently. Use setupWorkers() or call this with server.workerLock set. +func (s *Server) unsafeSetupWorkers(ctx context.Context) error { // Check if all the schedulers are disabled if len(s.config.EnabledSchedulers) == 0 || s.config.NumSchedulers == 0 { s.logger.Warn("no enabled schedulers") @@ -1462,6 +1575,7 @@ func (s *Server) setupWorkers(ctx context.Context) error { if !foundCore { return fmt.Errorf("invalid configuration: %q scheduler not enabled", structs.JobTypeCore) } + s.logger.Info("starting scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) // Start the workers for i := 0; i < s.config.NumSchedulers; i++ { @@ -1477,9 +1591,9 @@ func (s *Server) setupWorkers(ctx context.Context) error { return nil } -// setupNewWorkers() is used to start a new set of workers after a configuration -// change and a hot reload. -func (s *Server) setupNewWorkers() error { +// unsafeSetupNewWorkers directly manipulates the server.config, so it is not safe to +// call concurrently. Use reloadWorkers() or call this with server.workerLock set. +func (s *Server) unsafeSetupNewWorkers() error { // make a copy of the s.workers array so we can safely stop those goroutines asynchronously oldWorkers := make([]*Worker, len(s.workers)) defer s.stopOldWorkers(oldWorkers) @@ -1488,9 +1602,10 @@ func (s *Server) setupNewWorkers() error { } s.logger.Info(fmt.Sprintf("marking %v current schedulers for shutdown", len(oldWorkers))) - // build a clean backing array and call setupWorkers like in the normal startup path + // build a clean backing array and call unsafeSetupWorkers like setupWorkers does in the normal startup path + s.workers = make([]*Worker, 0, s.config.NumSchedulers) - err := s.setupWorkers(s.shutdownCtx) + err := s.unsafeSetupWorkers(s.shutdownCtx) if err != nil { return err } diff --git a/nomad/server_test.go b/nomad/server_test.go index 77570fee9b6..5872f3ceb10 100644 --- a/nomad/server_test.go +++ b/nomad/server_test.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "fmt" "io/ioutil" "os" @@ -578,12 +579,11 @@ func TestServer_RPCNameAndRegionValidation(t *testing.T) { } } -func TestServer_Reload_NumSchedulers(t *testing.T) { +func TestServer_ReloadSchedulers_NumSchedulers(t *testing.T) { t.Parallel() s1, cleanupS1 := TestServer(t, func(c *Config) { c.NumSchedulers = 8 - c.Region = "global" }) defer cleanupS1() @@ -596,3 +596,51 @@ func TestServer_Reload_NumSchedulers(t *testing.T) { time.Sleep(1 * time.Second) require.Equal(t, config.NumSchedulers, len(s1.workers)) } + +func TestServer_ReloadSchedulers_EnabledSchedulers(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.EnabledSchedulers = []string{structs.JobTypeCore, structs.JobTypeSystem} + }) + defer cleanupS1() + + require.Equal(t, s1.config.NumSchedulers, len(s1.workers)) + + config := DefaultConfig() + config.EnabledSchedulers = []string{structs.JobTypeCore, structs.JobTypeSystem, structs.JobTypeBatch} + require.NoError(t, s1.Reload(config)) + + time.Sleep(1 * time.Second) + require.Equal(t, config.NumSchedulers, len(s1.workers)) + require.ElementsMatch(t, config.EnabledSchedulers, s1.GetSchedulerWorkerConfig().EnabledSchedulers) + +} + +func TestServer_ReloadSchedulers_InvalidSchedulers(t *testing.T) { + t.Parallel() + + // Set the config to not have the core scheduler + config := DefaultConfig() + logger := testlog.HCLogger(t) + s := &Server{ + config: config, + logger: logger, + } + s.config.NumSchedulers = 0 + s.shutdownCtx, s.shutdownCancel = context.WithCancel(context.Background()) + s.shutdownCh = s.shutdownCtx.Done() + + config.EnabledSchedulers = []string{"_core", "batch"} + err := s.setupWorkers(s.shutdownCtx) + require.Nil(t, err) + origWC := s.GetSchedulerWorkerConfig() + reloadSchedulers(s, &SchedulerWorkerPoolArgs{NumSchedulers: config.NumSchedulers, EnabledSchedulers: []string{"batch"}}) + currentWC := s.GetSchedulerWorkerConfig() + require.Equal(t, origWC, currentWC) + + // Set the config to have an unknown scheduler + reloadSchedulers(s, &SchedulerWorkerPoolArgs{NumSchedulers: config.NumSchedulers, EnabledSchedulers: []string{"_core", "foo"}}) + currentWC = s.GetSchedulerWorkerConfig() + require.Equal(t, origWC, currentWC) +} From 18455771c17b2d70611b9c31ea62e5c98f876cb7 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Mon, 6 Dec 2021 18:32:14 -0500 Subject: [PATCH 08/30] Fix linter complaints --- command/agent/agent_endpoint.go | 8 -------- nomad/server.go | 6 ++---- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index 2593a74d62a..0c58c6cbc7b 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -826,11 +826,3 @@ type agentSchedulerWorkerConfig struct { NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } -type agentSchedulerWorkerConfigRequest struct { - agentSchedulerWorkerConfig -} - -type agentSchedulerWorkerConfigResponse struct { - NumSchedulers int `json:"num_schedulers"` - EnabledSchedulers []string `json:"enabled_schedulers"` -} diff --git a/nomad/server.go b/nomad/server.go index 7f0de78e9f0..65003bee2fb 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -226,10 +226,8 @@ type Server struct { vault VaultClient // Worker used for processing - workers []*Worker - currentNumSchedulers int - currentEnabledSchedulers []string - workerLock sync.RWMutex + workers []*Worker + workerLock sync.RWMutex // aclCache is used to maintain the parsed ACL objects aclCache *lru.TwoQueueCache From 9c4e5c410667a0a618641b60978dd836df030ccb Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Mon, 6 Dec 2021 18:40:21 -0500 Subject: [PATCH 09/30] Updating worker to cache EnabledScheduler list --- nomad/worker.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/nomad/worker.go b/nomad/worker.go index d1be5a0476e..1280750dc88 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -60,6 +60,10 @@ type Worker struct { paused bool stopped bool // Indicates that the worker is in a terminal state; read with Stopped() + // the Server.Config.EnabledSchedulers value is not safe for concurrent access, so + // the worker needs a cached copy of it. Workers are stopped if this value changes. + enabledSchedulers []string + ctx context.Context cancelFn context.CancelFunc @@ -79,9 +83,10 @@ type Worker struct { // NewWorker starts a new worker associated with the given server func NewWorker(ctx context.Context, srv *Server) (*Worker, error) { w := &Worker{ - srv: srv, - start: time.Now(), - id: uuid.Generate(), + srv: srv, + start: time.Now(), + id: uuid.Generate(), + enabledSchedulers: srv.GetSchedulerWorkerConfig().EnabledSchedulers, } w.logger = srv.logger.ResetNamed("worker").With("worker_id", w.id) w.pauseCond = sync.NewCond(&w.pauseLock) @@ -206,7 +211,7 @@ func (w *Worker) dequeueEvaluation(timeout time.Duration) ( eval *structs.Evaluation, token string, waitIndex uint64, shutdown bool) { // Setup the request req := structs.EvalDequeueRequest{ - Schedulers: w.srv.config.EnabledSchedulers, + Schedulers: w.enabledSchedulers, Timeout: timeout, SchedulerVersion: scheduler.SchedulerVersion, WriteRequest: structs.WriteRequest{ From 0d8b7ec81b5584595c01df8baf3c90462a56d87f Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Wed, 8 Dec 2021 16:42:29 -0500 Subject: [PATCH 10/30] Refactor `unsafe...` func names to `...Locked` --- nomad/server.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/nomad/server.go b/nomad/server.go index 65003bee2fb..9e1e59ce404 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -810,7 +810,7 @@ func (s *Server) Reload(newConfig *Config) error { s.EnterpriseState.ReloadLicense(newConfig) } - workerPoolArgs := unsafeGetSchedulerWorkerPoolArgsFromConfig(newConfig) + workerPoolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(newConfig) if reload, newVals := shouldReloadSchedulers(s, workerPoolArgs); reload { if newVals.IsValid() { reloadSchedulers(s, newVals) @@ -1504,7 +1504,7 @@ func (swpa SchedulerWorkerPoolArgs) IsValid() bool { return foundCore } -func unsafeGetSchedulerWorkerPoolArgsFromConfig(c *Config) *SchedulerWorkerPoolArgs { +func getSchedulerWorkerPoolArgsFromConfigLocked(c *Config) *SchedulerWorkerPoolArgs { return &SchedulerWorkerPoolArgs{ NumSchedulers: c.NumSchedulers, EnabledSchedulers: c.EnabledSchedulers, @@ -1514,7 +1514,7 @@ func unsafeGetSchedulerWorkerPoolArgsFromConfig(c *Config) *SchedulerWorkerPoolA func (s *Server) GetSchedulerWorkerConfig() SchedulerWorkerPoolArgs { s.workerLock.RLock() defer s.workerLock.RUnlock() - return *unsafeGetSchedulerWorkerPoolArgsFromConfig(s.config) + return *getSchedulerWorkerPoolArgsFromConfigLocked(s.config) } func (s *Server) SetSchedulerWorkerConfig(newArgs SchedulerWorkerPoolArgs) SchedulerWorkerPoolArgs { @@ -1539,19 +1539,19 @@ func reloadSchedulers(s *Server, newArgs *SchedulerWorkerPoolArgs) { // TODO: If EnabledSchedulers didn't change, we can scale rather than drain and rebuild s.config.NumSchedulers = newArgs.NumSchedulers s.config.EnabledSchedulers = newArgs.EnabledSchedulers - s.unsafeSetupNewWorkers() + s.setupNewWorkersLocked() } // setupWorkers is used to start the scheduling workers func (s *Server) setupWorkers(ctx context.Context) error { s.workerLock.Lock() defer s.workerLock.Unlock() - return s.unsafeSetupWorkers(ctx) + return s.setupWorkersLocked(ctx) } -// unsafeSetupWorkers directly manipulates the server.config, so it is not safe to +// setupWorkersLocked directly manipulates the server.config, so it is not safe to // call concurrently. Use setupWorkers() or call this with server.workerLock set. -func (s *Server) unsafeSetupWorkers(ctx context.Context) error { +func (s *Server) setupWorkersLocked(ctx context.Context) error { // Check if all the schedulers are disabled if len(s.config.EnabledSchedulers) == 0 || s.config.NumSchedulers == 0 { s.logger.Warn("no enabled schedulers") @@ -1589,9 +1589,9 @@ func (s *Server) unsafeSetupWorkers(ctx context.Context) error { return nil } -// unsafeSetupNewWorkers directly manipulates the server.config, so it is not safe to +// setupNewWorkersLocked directly manipulates the server.config, so it is not safe to // call concurrently. Use reloadWorkers() or call this with server.workerLock set. -func (s *Server) unsafeSetupNewWorkers() error { +func (s *Server) setupNewWorkersLocked() error { // make a copy of the s.workers array so we can safely stop those goroutines asynchronously oldWorkers := make([]*Worker, len(s.workers)) defer s.stopOldWorkers(oldWorkers) @@ -1600,10 +1600,10 @@ func (s *Server) unsafeSetupNewWorkers() error { } s.logger.Info(fmt.Sprintf("marking %v current schedulers for shutdown", len(oldWorkers))) - // build a clean backing array and call unsafeSetupWorkers like setupWorkers does in the normal startup path - + // build a clean backing array and call setupWorkersLocked like setupWorkers + // does in the normal startup path s.workers = make([]*Worker, 0, s.config.NumSchedulers) - err := s.unsafeSetupWorkers(s.shutdownCtx) + err := s.setupWorkersLocked(s.shutdownCtx) if err != nil { return err } From f5bb22762f0316413cb87f4eef47ade08ab58975 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 10 Dec 2021 09:42:09 -0500 Subject: [PATCH 11/30] Passing enabled schedulers list to worker - add mutex for contended server data - `workerLock` for the `workers` slice - `workerConfigLock` for the `Server.Config.NumSchedulers` and `Server.Config.EnabledSchedulers` values - add Copy() func for `SchedulerWorkerPoolArgs` - refactor NewWorker to allow for creating an unstarted worker for testing --- nomad/server.go | 26 +++++++++++++++++++++---- nomad/worker.go | 14 ++++++++++---- nomad/worker_test.go | 45 +++++++++++++++++++++++++++++--------------- 3 files changed, 62 insertions(+), 23 deletions(-) diff --git a/nomad/server.go b/nomad/server.go index 9e1e59ce404..8199219f875 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -226,8 +226,9 @@ type Server struct { vault VaultClient // Worker used for processing - workers []*Worker - workerLock sync.RWMutex + workers []*Worker + workerLock sync.RWMutex + workerConfigLock sync.RWMutex // aclCache is used to maintain the parsed ACL objects aclCache *lru.TwoQueueCache @@ -1504,6 +1505,18 @@ func (swpa SchedulerWorkerPoolArgs) IsValid() bool { return foundCore } +// Copy returns a clone of a SchedulerWorkerPoolArgs struct. Concurrent access +// concerns should be managed by the caller. +func (swpa SchedulerWorkerPoolArgs) Copy() SchedulerWorkerPoolArgs { + out := SchedulerWorkerPoolArgs{ + NumSchedulers: swpa.NumSchedulers, + EnabledSchedulers: make([]string, len(swpa.EnabledSchedulers)), + } + copy(out.EnabledSchedulers, swpa.EnabledSchedulers) + + return out +} + func getSchedulerWorkerPoolArgsFromConfigLocked(c *Config) *SchedulerWorkerPoolArgs { return &SchedulerWorkerPoolArgs{ NumSchedulers: c.NumSchedulers, @@ -1511,10 +1524,12 @@ func getSchedulerWorkerPoolArgsFromConfigLocked(c *Config) *SchedulerWorkerPoolA } } +// GetSchedulerWorkerConfig returns a clean copy of the server's current scheduler +// worker config. func (s *Server) GetSchedulerWorkerConfig() SchedulerWorkerPoolArgs { s.workerLock.RLock() defer s.workerLock.RUnlock() - return *getSchedulerWorkerPoolArgsFromConfigLocked(s.config) + return getSchedulerWorkerPoolArgsFromConfigLocked(s.config).Copy() } func (s *Server) SetSchedulerWorkerConfig(newArgs SchedulerWorkerPoolArgs) SchedulerWorkerPoolArgs { @@ -1576,8 +1591,11 @@ func (s *Server) setupWorkersLocked(ctx context.Context) error { s.logger.Info("starting scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) // Start the workers + + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s.config).Copy() + for i := 0; i < s.config.NumSchedulers; i++ { - if w, err := NewWorker(ctx, s); err != nil { + if w, err := NewWorker(ctx, s, poolArgs); err != nil { return err } else { s.logger.Debug("started scheduling worker", "id", w.ID(), "index", i+1, "of", s.config.NumSchedulers) diff --git a/nomad/worker.go b/nomad/worker.go index 1280750dc88..468e341c2d7 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -81,19 +81,25 @@ type Worker struct { } // NewWorker starts a new worker associated with the given server -func NewWorker(ctx context.Context, srv *Server) (*Worker, error) { +func NewWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { + w, _ := newWorker(ctx, srv, args) + go w.run() + return w, nil +} + +func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { w := &Worker{ srv: srv, start: time.Now(), id: uuid.Generate(), - enabledSchedulers: srv.GetSchedulerWorkerConfig().EnabledSchedulers, + enabledSchedulers: make([]string, len(args.EnabledSchedulers)), } + copy(w.enabledSchedulers, args.EnabledSchedulers) + w.logger = srv.logger.ResetNamed("worker").With("worker_id", w.id) w.pauseCond = sync.NewCond(&w.pauseLock) w.ctx, w.cancelFn = context.WithCancel(ctx) - - go w.run() return w, nil } diff --git a/nomad/worker_test.go b/nomad/worker_test.go index 947fd80fa7b..2cadf70e36e 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -76,7 +76,8 @@ func TestWorker_dequeueEvaluation(t *testing.T) { s1.evalBroker.Enqueue(eval1) // Create a worker - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := NewWorker(s1.shutdownCtx, s1, poolArgs) // Attempt dequeue eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond) @@ -122,7 +123,8 @@ func TestWorker_dequeueEvaluation_SerialJobs(t *testing.T) { s1.evalBroker.Enqueue(eval2) // Create a worker - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) // Attempt dequeue eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond) @@ -182,7 +184,8 @@ func TestWorker_dequeueEvaluation_paused(t *testing.T) { s1.evalBroker.Enqueue(eval1) // Create a worker - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.pauseCond = sync.NewCond(&w.pauseLock) // PAUSE the worker @@ -226,7 +229,8 @@ func TestWorker_dequeueEvaluation_shutdown(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) // Create a worker - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) go func() { time.Sleep(10 * time.Millisecond) @@ -256,7 +260,8 @@ func TestWorker_Shutdown(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) // Create a worker; since this tests the shutdown, you need the cancelFn too. - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) go func() { time.Sleep(10 * time.Millisecond) @@ -279,7 +284,8 @@ func TestWorker_Shutdown_paused(t *testing.T) { defer cleanupS1() testutil.WaitForLeader(t, s1.RPC) - w, _ := NewWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := NewWorker(s1.shutdownCtx, s1, poolArgs) w.SetPause(true) go func() { @@ -307,7 +313,8 @@ func TestWorker_sendAck(t *testing.T) { s1.evalBroker.Enqueue(eval1) // Create a worker - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) // Attempt dequeue eval, token, _, _ := w.dequeueEvaluation(10 * time.Millisecond) @@ -362,7 +369,8 @@ func TestWorker_waitForIndex(t *testing.T) { }() // Wait for a future index - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) snap, err := w.snapshotMinIndex(index+1, time.Second) require.NoError(t, err) require.NotNil(t, snap) @@ -388,7 +396,8 @@ func TestWorker_invokeScheduler(t *testing.T) { }) defer cleanupS1() - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) eval := mock.Eval() eval.Type = "noop" @@ -441,7 +450,8 @@ func TestWorker_SubmitPlan(t *testing.T) { } // Attempt to submit a plan - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token result, state, err := w.SubmitPlan(plan) @@ -505,7 +515,8 @@ func TestWorker_SubmitPlanNormalizedAllocations(t *testing.T) { plan.AppendPreemptedAlloc(preemptedAlloc, preemptingAllocID) // Attempt to submit a plan - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.SubmitPlan(plan) assert.Equal(t, &structs.Allocation{ @@ -562,7 +573,8 @@ func TestWorker_SubmitPlan_MissingNodeRefresh(t *testing.T) { } // Attempt to submit a plan - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token result, state, err := w.SubmitPlan(plan) @@ -621,7 +633,8 @@ func TestWorker_UpdateEval(t *testing.T) { eval2.Status = structs.EvalStatusComplete // Attempt to update eval - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token err = w.UpdateEval(eval2) @@ -672,7 +685,8 @@ func TestWorker_CreateEval(t *testing.T) { eval2.PreviousEval = eval1.ID // Attempt to create eval - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token err = w.CreateEval(eval2) @@ -736,7 +750,8 @@ func TestWorker_ReblockEval(t *testing.T) { eval2.QueuedAllocations = map[string]int{"web": 50} // Attempt to reblock eval - w := NewTestWorker(s1.shutdownCtx, s1) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token err = w.ReblockEval(eval2) From 292518bddaf68cc3136f2a893cc8763a9330d4a0 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 10 Dec 2021 09:47:32 -0500 Subject: [PATCH 12/30] Add note about scheduler death --- nomad/worker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nomad/worker.go b/nomad/worker.go index 468e341c2d7..c25199a37a7 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -251,6 +251,8 @@ REQ: limit = backoffSchedulerVersionMismatch } + // !: while I don't know _what_ to do here, shutting down the worker seems like it + // could end with a server with no running scheduler workers. if w.backoffErr(base, limit) { return nil, "", 0, true } From 1337f04939a276ca29bba5d913bbb9e2d3710d3d Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 10 Dec 2021 11:46:08 -0500 Subject: [PATCH 13/30] Worker API refactor - Start, Pause, Resume, Stop for func names - made setPause, shutdown implementing funcs for abobe functions - updated tests --- nomad/leader.go | 6 +++- nomad/server.go | 36 +++++++++++++-------- nomad/worker.go | 76 ++++++++++++++++++++++++++++++++++---------- nomad/worker_test.go | 20 ++++++------ 4 files changed, 98 insertions(+), 40 deletions(-) diff --git a/nomad/leader.go b/nomad/leader.go index 89f61865e9a..c6dca9dc677 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -442,7 +442,11 @@ ERR_WAIT: func (s *Server) handlePausableWorkers(isLeader bool) { for _, w := range s.pausableWorkers() { - w.SetPause(isLeader) + if isLeader { + w.Pause() + } else { + w.Resume() + } } } diff --git a/nomad/server.go b/nomad/server.go index 8199219f875..d1f2b3c7465 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -1443,8 +1443,8 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string) ( // shouldReloadSchedulers checks the new config to determine if the scheduler worker pool // needs to be updated. If so, returns true and a pointer to a populated SchedulerWorkerPoolArgs func shouldReloadSchedulers(s *Server, newPoolArgs *SchedulerWorkerPoolArgs) (bool, *SchedulerWorkerPoolArgs) { - s.workerLock.RLock() - defer s.workerLock.RUnlock() + s.workerConfigLock.RLock() + defer s.workerConfigLock.RUnlock() newSchedulers := make([]string, len(newPoolArgs.EnabledSchedulers)) copy(newSchedulers, newPoolArgs.EnabledSchedulers) @@ -1527,8 +1527,8 @@ func getSchedulerWorkerPoolArgsFromConfigLocked(c *Config) *SchedulerWorkerPoolA // GetSchedulerWorkerConfig returns a clean copy of the server's current scheduler // worker config. func (s *Server) GetSchedulerWorkerConfig() SchedulerWorkerPoolArgs { - s.workerLock.RLock() - defer s.workerLock.RUnlock() + s.workerConfigLock.RLock() + defer s.workerConfigLock.RUnlock() return getSchedulerWorkerPoolArgsFromConfigLocked(s.config).Copy() } @@ -1549,8 +1549,15 @@ func reloadSchedulers(s *Server, newArgs *SchedulerWorkerPoolArgs) { s.logger.Info("received invalid arguments for scheduler pool reload; ignoring") return } + + // reload will modify the server.config so it needs a write lock + s.workerConfigLock.Lock() + defer s.workerConfigLock.Unlock() + + // reload modifies the worker slice so it needs a write lock s.workerLock.Lock() defer s.workerLock.Unlock() + // TODO: If EnabledSchedulers didn't change, we can scale rather than drain and rebuild s.config.NumSchedulers = newArgs.NumSchedulers s.config.EnabledSchedulers = newArgs.EnabledSchedulers @@ -1559,23 +1566,27 @@ func reloadSchedulers(s *Server, newArgs *SchedulerWorkerPoolArgs) { // setupWorkers is used to start the scheduling workers func (s *Server) setupWorkers(ctx context.Context) error { + poolArgs := s.GetSchedulerWorkerConfig() + + // we will be writing to the worker slice s.workerLock.Lock() defer s.workerLock.Unlock() - return s.setupWorkersLocked(ctx) + + return s.setupWorkersLocked(ctx, poolArgs) } // setupWorkersLocked directly manipulates the server.config, so it is not safe to // call concurrently. Use setupWorkers() or call this with server.workerLock set. -func (s *Server) setupWorkersLocked(ctx context.Context) error { +func (s *Server) setupWorkersLocked(ctx context.Context, poolArgs SchedulerWorkerPoolArgs) error { // Check if all the schedulers are disabled - if len(s.config.EnabledSchedulers) == 0 || s.config.NumSchedulers == 0 { + if len(poolArgs.EnabledSchedulers) == 0 || poolArgs.NumSchedulers == 0 { s.logger.Warn("no enabled schedulers") return nil } // Check if the core scheduler is not enabled foundCore := false - for _, sched := range s.config.EnabledSchedulers { + for _, sched := range poolArgs.EnabledSchedulers { if sched == structs.JobTypeCore { foundCore = true continue @@ -1589,11 +1600,9 @@ func (s *Server) setupWorkersLocked(ctx context.Context) error { return fmt.Errorf("invalid configuration: %q scheduler not enabled", structs.JobTypeCore) } - s.logger.Info("starting scheduling worker(s)", "num_workers", s.config.NumSchedulers, "schedulers", s.config.EnabledSchedulers) + s.logger.Info("starting scheduling worker(s)", "num_workers", poolArgs.NumSchedulers, "schedulers", poolArgs.EnabledSchedulers) // Start the workers - poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s.config).Copy() - for i := 0; i < s.config.NumSchedulers; i++ { if w, err := NewWorker(ctx, s, poolArgs); err != nil { return err @@ -1621,7 +1630,8 @@ func (s *Server) setupNewWorkersLocked() error { // build a clean backing array and call setupWorkersLocked like setupWorkers // does in the normal startup path s.workers = make([]*Worker, 0, s.config.NumSchedulers) - err := s.setupWorkersLocked(s.shutdownCtx) + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s.config).Copy() + err := s.setupWorkersLocked(s.shutdownCtx, poolArgs) if err != nil { return err } @@ -1638,7 +1648,7 @@ func (s *Server) stopOldWorkers(oldWorkers []*Worker) { workerCount := len(oldWorkers) for i, w := range oldWorkers { s.logger.Debug("stopping old scheduling worker", "id", w.ID(), "index", i+1, "of", workerCount) - go w.Shutdown() + go w.Stop() } } diff --git a/nomad/worker.go b/nomad/worker.go index c25199a37a7..afb306068b0 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -83,7 +83,7 @@ type Worker struct { // NewWorker starts a new worker associated with the given server func NewWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { w, _ := newWorker(ctx, srv, args) - go w.run() + w.Start() return w, nil } @@ -98,13 +98,63 @@ func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) ( w.logger = srv.logger.ResetNamed("worker").With("worker_id", w.id) w.pauseCond = sync.NewCond(&w.pauseLock) - w.ctx, w.cancelFn = context.WithCancel(ctx) + return w, nil } -// SetPause is used to pause or unpause a worker -func (w *Worker) SetPause(p bool) { +// ID returns a string ID for the worker. +func (w *Worker) ID() string { + return w.id +} + +// Start transitions a worker to the starting state. Check +// to see if it paused using IsStarted() +func (w *Worker) Start() { + go w.run() +} + +// Pause transitions a worker to the pausing state. Check +// to see if it paused using IsPaused() +func (w *Worker) Pause() { + w.setPause(true) +} + +// Resume transitions a worker to the resuming state. Check +// to see if the worker restarted by checking IsStarted() +func (w *Worker) Resume() { + w.setPause(false) +} + +// Resume transitions a worker to the stopping state. Check +// to see if the worker stopped by checking IsStopped() +func (w *Worker) Stop() { + w.shutdown() +} + +// IsStarted returns a boolean indicating if this worker has been started. +func (w *Worker) IsStarted() bool { + w.pauseLock.Lock() + defer w.pauseLock.Unlock() + return !w.paused && !w.stopped +} + +// IsPaused returns a boolean indicating if this worker has been paused. +func (w *Worker) IsPaused() bool { + w.pauseLock.Lock() + defer w.pauseLock.Unlock() + return w.paused +} + +// IsStopped returns a boolean indicating if this worker has been stopped. +func (w *Worker) IsStopped() bool { + w.pauseLock.Lock() + defer w.pauseLock.Unlock() + return w.stopped +} + +// setPause is internally used to pause or unpause a worker +func (w *Worker) setPause(p bool) { w.pauseLock.Lock() w.paused = p w.pauseLock.Unlock() @@ -123,7 +173,7 @@ func (w *Worker) checkPaused() { } // Shutdown is used to signal that the worker should shutdown. -func (w *Worker) Shutdown() { +func (w *Worker) shutdown() { w.pauseLock.Lock() wasPaused := w.paused w.paused = false @@ -136,18 +186,6 @@ func (w *Worker) Shutdown() { } } -// Stopped returns a boolean indicating if this worker has been stopped. -func (w *Worker) Stopped() bool { - w.pauseLock.Lock() - defer w.pauseLock.Unlock() - return w.stopped -} - -// ID returns a string ID for the worker. -func (w *Worker) ID() string { - return w.id -} - // markStopped is used to mark the worker as stopped and should be called in a // defer immediately upon entering the run() function. func (w *Worker) markStopped() { @@ -166,6 +204,10 @@ func (w *Worker) isDone() bool { } } +/*----------------- + Behavior code +-----------------*/ + // run is the long-lived goroutine which is used to run the worker func (w *Worker) run() { defer w.markStopped() diff --git a/nomad/worker_test.go b/nomad/worker_test.go index 2cadf70e36e..d3a9d238f4c 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -189,11 +189,11 @@ func TestWorker_dequeueEvaluation_paused(t *testing.T) { w.pauseCond = sync.NewCond(&w.pauseLock) // PAUSE the worker - w.SetPause(true) + w.Pause() go func() { time.Sleep(100 * time.Millisecond) - w.SetPause(false) + w.Resume() }() // Attempt dequeue @@ -265,7 +265,7 @@ func TestWorker_Shutdown(t *testing.T) { go func() { time.Sleep(10 * time.Millisecond) - w.Shutdown() + w.Stop() }() // Attempt dequeue @@ -286,16 +286,18 @@ func TestWorker_Shutdown_paused(t *testing.T) { poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() w, _ := NewWorker(s1.shutdownCtx, s1, poolArgs) - w.SetPause(true) + w.Pause() + + // pausing can take up to 500ms because of the blocking query timeout in dequeueEvaluation. + require.Eventually(t, w.IsPaused, 550*time.Millisecond, 10*time.Millisecond, "should paused") go func() { - time.Sleep(10 * time.Millisecond) - w.Shutdown() + w.Stop() }() - time.Sleep(511 * time.Millisecond) - // Verify that the worker is stopped. - require.True(t, w.Stopped()) + // transitioning to stopped from paused should be very quick, + // but might not be immediate. + require.Eventually(t, w.IsStopped, 100*time.Millisecond, 10*time.Millisecond, "should stop when paused") } func TestWorker_sendAck(t *testing.T) { From bd345e01cefeacd1cee992e391288d2a4481dc81 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 10 Dec 2021 11:59:46 -0500 Subject: [PATCH 14/30] Made handler methods public for OpenAPI, remove unused test bool --- command/agent/agent_endpoint.go | 8 ++++---- command/agent/agent_endpoint_test.go | 7 ++----- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index 0c58c6cbc7b..01b3b220b72 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -752,15 +752,15 @@ func (s *HTTPServer) AgentSchedulerWorkerConfigRequest(resp http.ResponseWriter, } switch req.Method { case "PUT", "POST": - return s.updateScheduleWorkersConfig(resp, req) + return s.UpdateScheduleWorkersConfig(resp, req) case "GET": - return s.getScheduleWorkersConfig(resp, req) + return s.GetScheduleWorkersConfig(resp, req) default: return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod) } } -func (s *HTTPServer) getScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { +func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { return nil, CodedError(http.StatusBadRequest, "server only endpoint") @@ -784,7 +784,7 @@ func (s *HTTPServer) getScheduleWorkersConfig(resp http.ResponseWriter, req *htt return response, nil } -func (s *HTTPServer) updateScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { +func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { return nil, CodedError(400, "server only endpoint") diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index 0ac02966ebe..0111e681716 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -1469,7 +1469,6 @@ func TestHTTP_XSS_Monitor(t *testing.T) { */ type scheduleWorkerTest_workerRequestTest struct { name string // test case name - requiresACL bool // prevents test cases that require ACLs from running in the non-ACL version request schedulerWorkerTest_testRequest whenACLNotEnabled schedulerWorkerTest_testExpect whenACLEnabled schedulerWorkerTest_testExpect @@ -1537,8 +1536,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { whenACLEnabled: success1, }, { - name: "get with read token", - requiresACL: true, + name: "get with read token", request: schedulerWorkerTest_testRequest{ verb: "GET", aclToken: "agent_read", @@ -1548,8 +1546,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { whenACLEnabled: success1, }, { - name: "get with write token", - requiresACL: true, + name: "get with write token", request: schedulerWorkerTest_testRequest{ verb: "GET", aclToken: "agent_write", From 31687cdf10b3fac182d3df025d2747710c1ad829 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 10 Dec 2021 16:58:00 -0500 Subject: [PATCH 15/30] Implement SchedulerWorker status part 1 --- nomad/leader_test.go | 19 +- nomad/worker.go | 176 +++++++++++++------ nomad/worker_string_schedulerworkerstatus.go | 30 ++++ nomad/worker_string_workerstatus.go | 30 ++++ nomad/worker_test.go | 3 +- 5 files changed, 202 insertions(+), 56 deletions(-) create mode 100644 nomad/worker_string_schedulerworkerstatus.go create mode 100644 nomad/worker_string_workerstatus.go diff --git a/nomad/leader_test.go b/nomad/leader_test.go index ea6c4670e15..3f5a1add5b0 100644 --- a/nomad/leader_test.go +++ b/nomad/leader_test.go @@ -1331,22 +1331,29 @@ func TestLeader_PausingWorkers(t *testing.T) { pausedWorkers := func() int { c := 0 for _, w := range s1.workers { - w.pauseLock.Lock() - if w.paused { + if w.IsPaused() { c++ } - w.pauseLock.Unlock() } return c } - // pause 3/4 of the workers - require.Equal(t, 9, pausedWorkers()) + // this satisfies the require.Eventually test interface + checkPaused := func(count int) func() bool { + return func() bool { + workers := pausedWorkers() + return workers == count + } + } + + // acquiring leadership should have paused 3/4 of the workers + require.Eventually(t, checkPaused(9), 1*time.Second, 10*time.Millisecond, "scheduler workers did not pause within a second at leadership change") err := s1.revokeLeadership() require.NoError(t, err) - require.Zero(t, pausedWorkers()) + // unpausing is a relatively quick activity + require.Eventually(t, checkPaused(0), 50*time.Millisecond, 10*time.Millisecond, "scheduler workers should have unpaused after losing leadership") } // Test doing an inplace upgrade on a server from raft protocol 2 to 3 diff --git a/nomad/worker.go b/nomad/worker.go index afb306068b0..25e723b8a72 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -47,31 +47,62 @@ const ( dequeueErrGrace = 10 * time.Second ) +type WorkerStatus int + +//go:generate stringer -trimprefix=Worker -output worker_string_workerstatus.go -linecomment -type=WorkerStatus +const ( + WorkerUnknownStatus WorkerStatus = iota // Unknown + WorkerStarting + WorkerStarted + WorkerPausing + WorkerPaused + WorkerResuming + WorkerStopping + WorkerStopped +) + +type SchedulerWorkerStatus int + +//go:generate stringer -trimprefix=Workload -output worker_string_schedulerworkerstatus.go -linecomment -type=SchedulerWorkerStatus +const ( + WorkloadUnknownStatus SchedulerWorkerStatus = iota + WorkloadRunning + WorkloadWaiting + WorkloadScheduling + WorkloadSubmitting + WorkloadBackoff + WorkloadStopped + WorkloadPaused +) + // Worker is a single threaded scheduling worker. There may be multiple // running per server (leader or follower). They are responsible for dequeuing // pending evaluations, invoking schedulers, plan submission and the // lifecycle around making task allocations. They bridge the business logic // of the scheduler with the plumbing required to make it all work. type Worker struct { - srv *Server - logger log.Logger - start time.Time - id string - paused bool - stopped bool // Indicates that the worker is in a terminal state; read with Stopped() + srv *Server + logger log.Logger + start time.Time + id string - // the Server.Config.EnabledSchedulers value is not safe for concurrent access, so - // the worker needs a cached copy of it. Workers are stopped if this value changes. - enabledSchedulers []string - - ctx context.Context - cancelFn context.CancelFunc + status WorkerStatus + workloadStatus SchedulerWorkerStatus + statusLock sync.RWMutex + pauseFlag bool pauseLock sync.Mutex pauseCond *sync.Cond + ctx context.Context + cancelFn context.CancelFunc - failures uint + // the Server.Config.EnabledSchedulers value is not safe for concurrent access, so + // the worker needs a cached copy of it. Workers are stopped if this value changes. + enabledSchedulers []string + // failures is the count of errors encountered while dequeueing evaluations + // and is used to calculate backoff. + failures uint evalToken string // snapshotIndex is the index of the snapshot in which the scheduler was @@ -89,9 +120,10 @@ func NewWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) ( func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { w := &Worker{ + id: uuid.Generate(), srv: srv, start: time.Now(), - id: uuid.Generate(), + status: WorkerStarting, enabledSchedulers: make([]string, len(args.EnabledSchedulers)), } copy(w.enabledSchedulers, args.EnabledSchedulers) @@ -111,72 +143,114 @@ func (w *Worker) ID() string { // Start transitions a worker to the starting state. Check // to see if it paused using IsStarted() func (w *Worker) Start() { + w.setStatus(WorkerStarting) go w.run() } // Pause transitions a worker to the pausing state. Check // to see if it paused using IsPaused() func (w *Worker) Pause() { - w.setPause(true) + w.setStatus(WorkerPausing) + w.pauseLock.Lock() + w.pauseFlag = true + w.pauseLock.Unlock() } // Resume transitions a worker to the resuming state. Check // to see if the worker restarted by checking IsStarted() func (w *Worker) Resume() { - w.setPause(false) + if w.IsPaused() { + w.setStatus(WorkerResuming) + w.pauseCond.Broadcast() + } } // Resume transitions a worker to the stopping state. Check // to see if the worker stopped by checking IsStopped() func (w *Worker) Stop() { + w.setStatus(WorkerStopping) w.shutdown() } // IsStarted returns a boolean indicating if this worker has been started. func (w *Worker) IsStarted() bool { - w.pauseLock.Lock() - defer w.pauseLock.Unlock() - return !w.paused && !w.stopped + return w.GetStatus() == WorkerStarted } // IsPaused returns a boolean indicating if this worker has been paused. func (w *Worker) IsPaused() bool { - w.pauseLock.Lock() - defer w.pauseLock.Unlock() - return w.paused + return w.GetStatus() == WorkerPaused } // IsStopped returns a boolean indicating if this worker has been stopped. func (w *Worker) IsStopped() bool { - w.pauseLock.Lock() - defer w.pauseLock.Unlock() - return w.stopped + return w.GetStatus() == WorkerStopped } -// setPause is internally used to pause or unpause a worker -func (w *Worker) setPause(p bool) { - w.pauseLock.Lock() - w.paused = p - w.pauseLock.Unlock() - if !p { - w.pauseCond.Broadcast() - } +// GetStatus returns the status of the Worker +func (w *Worker) GetStatus() WorkerStatus { + w.statusLock.RLock() + defer w.statusLock.RUnlock() + return w.status +} + +// setStatus is used internally to the worker to update the +// status of the worker based on calls to the Worker API. +func (w *Worker) setStatus(newStatus WorkerStatus) { + w.statusLock.Lock() + defer w.statusLock.Unlock() + w.status = newStatus +} + +// GetStatus returns the status of the Worker's Workload. +func (w *Worker) GetWorkloadStatus() SchedulerWorkerStatus { + w.statusLock.RLock() + defer w.statusLock.RUnlock() + return w.workloadStatus +} + +// setWorkloadStatus is used internally to the worker to update the +// status of the worker based updates from the workload. +func (w *Worker) setWorkloadStatus(newStatus SchedulerWorkerStatus) { + w.statusLock.Lock() + defer w.statusLock.Unlock() + w.workloadStatus = newStatus } -// checkPaused is used to park the worker when paused -func (w *Worker) checkPaused() { +// maybeWait is responsible for making the transition from `pausing` +// to `paused`, waiting, and then transitioning back to the running +// values. +func (w *Worker) maybeWait() { w.pauseLock.Lock() - for w.paused { + if !w.pauseFlag { + w.pauseLock.Unlock() + return + } + + w.statusLock.Lock() + w.status = WorkerPaused + w.workloadStatus = WorkloadPaused + w.statusLock.Unlock() + + for w.pauseFlag { w.pauseCond.Wait() } + w.pauseFlag = false w.pauseLock.Unlock() + + w.statusLock.Lock() + w.status = WorkerStarted + w.workloadStatus = WorkloadRunning + w.statusLock.Unlock() + w.logger.Debug("resumed") + } // Shutdown is used to signal that the worker should shutdown. func (w *Worker) shutdown() { w.pauseLock.Lock() - wasPaused := w.paused - w.paused = false + wasPaused := w.pauseFlag + w.pauseFlag = false w.pauseLock.Unlock() w.logger.Trace("shutdown request received") @@ -189,13 +263,11 @@ func (w *Worker) shutdown() { // markStopped is used to mark the worker as stopped and should be called in a // defer immediately upon entering the run() function. func (w *Worker) markStopped() { - w.pauseLock.Lock() - w.stopped = true - w.pauseLock.Unlock() w.logger.Debug("stopped") + w.setStatus(WorkerStopped) } -func (w *Worker) isDone() bool { +func (w *Worker) workerShuttingDown() bool { select { case <-w.ctx.Done(): return true @@ -210,12 +282,16 @@ func (w *Worker) isDone() bool { // run is the long-lived goroutine which is used to run the worker func (w *Worker) run() { - defer w.markStopped() + defer func() { + w.setWorkloadStatus(WorkloadStopped) + w.markStopped() + }() + w.setWorkloadStatus(WorkloadRunning) w.logger.Debug("running") for { // Check to see if the context has been cancelled. Server shutdown and Shutdown() // should do this. - if w.isDone() { + if w.workerShuttingDown() { return } // Dequeue a pending evaluation @@ -269,10 +345,10 @@ func (w *Worker) dequeueEvaluation(timeout time.Duration) ( var resp structs.EvalDequeueResponse REQ: - // Check if we are paused - w.checkPaused() - // Immediately check to see if the worker has been shutdown - if w.isDone() { + // Wait inside this function if the worker is paused. + w.maybeWait() + // Immediately check to see if the worker has been shutdown. + if w.workerShuttingDown() { return nil, "", 0, true } @@ -281,7 +357,7 @@ REQ: err := w.srv.RPC("Eval.Dequeue", &req, &resp) metrics.MeasureSince([]string{"nomad", "worker", "dequeue_eval"}, start) if err != nil { - if time.Since(w.start) > dequeueErrGrace && !w.isDone() { + if time.Since(w.start) > dequeueErrGrace && !w.workerShuttingDown() { w.logger.Error("failed to dequeue evaluation", "error", err) } @@ -636,6 +712,8 @@ func (w *Worker) shouldResubmit(err error) bool { // backoffErr is used to do an exponential back off on error. This is // maintained statefully for the worker. Returns if attempts should be // abandoned due to shutdown. +// This uses the worker's context in order to immediately stop the +// backoff if the server or the worker is shutdown. func (w *Worker) backoffErr(base, limit time.Duration) bool { backoff := (1 << (2 * w.failures)) * base if backoff > limit { diff --git a/nomad/worker_string_schedulerworkerstatus.go b/nomad/worker_string_schedulerworkerstatus.go new file mode 100644 index 00000000000..13614343e90 --- /dev/null +++ b/nomad/worker_string_schedulerworkerstatus.go @@ -0,0 +1,30 @@ +// Code generated by "stringer -trimprefix=Workload -output worker_string_schedulerworkerstatus.go -linecomment -type=SchedulerWorkerStatus"; DO NOT EDIT. + +package nomad + +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[WorkloadUnknownStatus-0] + _ = x[WorkloadRunning-1] + _ = x[WorkloadWaiting-2] + _ = x[WorkloadScheduling-3] + _ = x[WorkloadSubmitting-4] + _ = x[WorkloadBackoff-5] + _ = x[WorkloadStopped-6] + _ = x[WorkloadPaused-7] +} + +const _SchedulerWorkerStatus_name = "UnknownStatusRunningWaitingSchedulingSubmittingBackoffStoppedPaused" + +var _SchedulerWorkerStatus_index = [...]uint8{0, 13, 20, 27, 37, 47, 54, 61, 67} + +func (i SchedulerWorkerStatus) String() string { + if i < 0 || i >= SchedulerWorkerStatus(len(_SchedulerWorkerStatus_index)-1) { + return "SchedulerWorkerStatus(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _SchedulerWorkerStatus_name[_SchedulerWorkerStatus_index[i]:_SchedulerWorkerStatus_index[i+1]] +} diff --git a/nomad/worker_string_workerstatus.go b/nomad/worker_string_workerstatus.go new file mode 100644 index 00000000000..1eda2d9acfe --- /dev/null +++ b/nomad/worker_string_workerstatus.go @@ -0,0 +1,30 @@ +// Code generated by "stringer -trimprefix=Worker -output worker_string_workerstatus.go -linecomment -type=WorkerStatus"; DO NOT EDIT. + +package nomad + +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[WorkerUnknownStatus-0] + _ = x[WorkerStarting-1] + _ = x[WorkerStarted-2] + _ = x[WorkerPausing-3] + _ = x[WorkerPaused-4] + _ = x[WorkerResuming-5] + _ = x[WorkerStopping-6] + _ = x[WorkerStopped-7] +} + +const _WorkerStatus_name = "UnknownStartingStartedPausingPausedResumingStoppingStopped" + +var _WorkerStatus_index = [...]uint8{0, 7, 15, 22, 29, 35, 43, 51, 58} + +func (i WorkerStatus) String() string { + if i < 0 || i >= WorkerStatus(len(_WorkerStatus_index)-1) { + return "WorkerStatus(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _WorkerStatus_name[_WorkerStatus_index[i]:_WorkerStatus_index[i+1]] +} diff --git a/nomad/worker_test.go b/nomad/worker_test.go index d3a9d238f4c..a8993bb18d4 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -286,10 +286,11 @@ func TestWorker_Shutdown_paused(t *testing.T) { poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() w, _ := NewWorker(s1.shutdownCtx, s1, poolArgs) + w.Pause() // pausing can take up to 500ms because of the blocking query timeout in dequeueEvaluation. - require.Eventually(t, w.IsPaused, 550*time.Millisecond, 10*time.Millisecond, "should paused") + require.Eventually(t, w.IsPaused, 550*time.Millisecond, 10*time.Millisecond, "should pause") go func() { w.Stop() From 373998730d2b3b05e12f236cdfd1b3c505ead561 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Fri, 10 Dec 2021 19:06:50 -0500 Subject: [PATCH 16/30] Fix broken Pause logic; split WorkloadWaiting status There are two places in the code where the scheduler worker code waits: to dequeue an eval and for a raft index --- nomad/worker.go | 62 +++++++++++++++----- nomad/worker_string_schedulerworkerstatus.go | 17 +++--- 2 files changed, 57 insertions(+), 22 deletions(-) diff --git a/nomad/worker.go b/nomad/worker.go index 25e723b8a72..884eee7dc9d 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -67,7 +67,8 @@ type SchedulerWorkerStatus int const ( WorkloadUnknownStatus SchedulerWorkerStatus = iota WorkloadRunning - WorkloadWaiting + WorkloadWaitingToDequeue + WorkloadWaitingForRaft WorkloadScheduling WorkloadSubmitting WorkloadBackoff @@ -111,13 +112,14 @@ type Worker struct { snapshotIndex uint64 } -// NewWorker starts a new worker associated with the given server +// NewWorker starts a new scheduler worker associated with the given server func NewWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { w, _ := newWorker(ctx, srv, args) w.Start() return w, nil } +// _newWorker creates a worker without calling its Start func. This is useful for testing. func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { w := &Worker{ id: uuid.Generate(), @@ -151,22 +153,21 @@ func (w *Worker) Start() { // to see if it paused using IsPaused() func (w *Worker) Pause() { w.setStatus(WorkerPausing) - w.pauseLock.Lock() - w.pauseFlag = true - w.pauseLock.Unlock() + w.setPauseFlag(true) } // Resume transitions a worker to the resuming state. Check -// to see if the worker restarted by checking IsStarted() +// to see if the worker restarted by calling IsStarted() func (w *Worker) Resume() { if w.IsPaused() { w.setStatus(WorkerResuming) + w.setPauseFlag(false) w.pauseCond.Broadcast() } } // Resume transitions a worker to the stopping state. Check -// to see if the worker stopped by checking IsStopped() +// to see if the worker stopped by calling IsStopped() func (w *Worker) Stop() { w.setStatus(WorkerStopping) w.shutdown() @@ -199,6 +200,10 @@ func (w *Worker) GetStatus() WorkerStatus { func (w *Worker) setStatus(newStatus WorkerStatus) { w.statusLock.Lock() defer w.statusLock.Unlock() + if newStatus == w.status { + return + } + w.logger.Trace("changed worker status", "from", w.status, "to", newStatus) w.status = newStatus } @@ -214,9 +219,24 @@ func (w *Worker) GetWorkloadStatus() SchedulerWorkerStatus { func (w *Worker) setWorkloadStatus(newStatus SchedulerWorkerStatus) { w.statusLock.Lock() defer w.statusLock.Unlock() + if newStatus == w.workloadStatus { + return + } + w.logger.Trace("changed workload status", "from", w.workloadStatus, "to", newStatus) w.workloadStatus = newStatus } +// ---------------------------------- +// Pause Implementation +// These functions are used to support the worker's pause behaviors. +// ---------------------------------- + +func (w *Worker) setPauseFlag(pause bool) { + w.pauseLock.Lock() + defer w.pauseLock.Unlock() + w.pauseFlag = pause +} + // maybeWait is responsible for making the transition from `pausing` // to `paused`, waiting, and then transitioning back to the running // values. @@ -229,21 +249,24 @@ func (w *Worker) maybeWait() { w.statusLock.Lock() w.status = WorkerPaused + originalWorkloadStatus := w.workloadStatus w.workloadStatus = WorkloadPaused + w.logger.Trace("changed workload status", "from", originalWorkloadStatus, "to", w.workloadStatus) + w.statusLock.Unlock() for w.pauseFlag { w.pauseCond.Wait() } - w.pauseFlag = false + w.pauseLock.Unlock() w.statusLock.Lock() w.status = WorkerStarted - w.workloadStatus = WorkloadRunning + w.workloadStatus = originalWorkloadStatus + w.logger.Trace("changed workload status", "from", w.workloadStatus, "to", originalWorkloadStatus) + w.logger.Trace("changed worker status", "from", WorkerPaused, "to", WorkerStarted) w.statusLock.Unlock() - w.logger.Debug("resumed") - } // Shutdown is used to signal that the worker should shutdown. @@ -276,9 +299,9 @@ func (w *Worker) workerShuttingDown() bool { } } -/*----------------- - Behavior code ------------------*/ +// ---------------------------------- +// Workload behavior code +// ---------------------------------- // run is the long-lived goroutine which is used to run the worker func (w *Worker) run() { @@ -310,6 +333,7 @@ func (w *Worker) run() { } // Wait for the raft log to catchup to the evaluation + w.setWorkloadStatus(WorkloadWaitingForRaft) snap, err := w.snapshotMinIndex(waitIndex, raftSyncLimit) if err != nil { w.logger.Error("error waiting for Raft index", "error", err, "index", waitIndex) @@ -318,6 +342,7 @@ func (w *Worker) run() { } // Invoke the scheduler to determine placements + w.setWorkloadStatus(WorkloadScheduling) if err := w.invokeScheduler(snap, eval, token); err != nil { w.logger.Error("error invoking scheduler", "error", err) w.sendNack(eval, token) @@ -354,6 +379,7 @@ REQ: // Make a blocking RPC start := time.Now() + w.setWorkloadStatus(WorkloadWaitingToDequeue) err := w.srv.RPC("Eval.Dequeue", &req, &resp) metrics.MeasureSince([]string{"nomad", "worker", "dequeue_eval"}, start) if err != nil { @@ -715,6 +741,7 @@ func (w *Worker) shouldResubmit(err error) bool { // This uses the worker's context in order to immediately stop the // backoff if the server or the worker is shutdown. func (w *Worker) backoffErr(base, limit time.Duration) bool { + w.setWorkloadStatus(WorkloadBackoff) backoff := (1 << (2 * w.failures)) * base if backoff > limit { backoff = limit @@ -734,3 +761,10 @@ func (w *Worker) backoffErr(base, limit time.Duration) bool { func (w *Worker) backoffReset() { w.failures = 0 } + +// Test helpers +// +func (w *Worker) _start(inFunc func(w *Worker)) { + w.setStatus(WorkerStarting) + go inFunc(w) +} diff --git a/nomad/worker_string_schedulerworkerstatus.go b/nomad/worker_string_schedulerworkerstatus.go index 13614343e90..42181ffd0e2 100644 --- a/nomad/worker_string_schedulerworkerstatus.go +++ b/nomad/worker_string_schedulerworkerstatus.go @@ -10,17 +10,18 @@ func _() { var x [1]struct{} _ = x[WorkloadUnknownStatus-0] _ = x[WorkloadRunning-1] - _ = x[WorkloadWaiting-2] - _ = x[WorkloadScheduling-3] - _ = x[WorkloadSubmitting-4] - _ = x[WorkloadBackoff-5] - _ = x[WorkloadStopped-6] - _ = x[WorkloadPaused-7] + _ = x[WorkloadWaitingToDequeue-2] + _ = x[WorkloadWaitingForRaft-3] + _ = x[WorkloadScheduling-4] + _ = x[WorkloadSubmitting-5] + _ = x[WorkloadBackoff-6] + _ = x[WorkloadStopped-7] + _ = x[WorkloadPaused-8] } -const _SchedulerWorkerStatus_name = "UnknownStatusRunningWaitingSchedulingSubmittingBackoffStoppedPaused" +const _SchedulerWorkerStatus_name = "UnknownStatusRunningWaitingToDequeueWaitingForRaftSchedulingSubmittingBackoffStoppedPaused" -var _SchedulerWorkerStatus_index = [...]uint8{0, 13, 20, 27, 37, 47, 54, 61, 67} +var _SchedulerWorkerStatus_index = [...]uint8{0, 13, 20, 36, 50, 60, 70, 77, 84, 90} func (i SchedulerWorkerStatus) String() string { if i < 0 || i >= SchedulerWorkerStatus(len(_SchedulerWorkerStatus_index)-1) { From 7fe5949a08582c6cf18f878669759644a76eb87a Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Sat, 11 Dec 2021 00:41:07 -0500 Subject: [PATCH 17/30] Added scheduler info api --- command/agent/agent_endpoint.go | 56 ++++++ command/agent/agent_endpoint_test.go | 268 +++++++++++++++++++++++---- command/agent/http.go | 3 +- nomad/server.go | 13 ++ nomad/worker.go | 41 ++++ nomad/worker_test.go | 96 ++++++++++ 6 files changed, 443 insertions(+), 34 deletions(-) diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index 01b3b220b72..b8e712e9d50 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -11,6 +11,7 @@ import ( "sort" "strconv" "strings" + "time" "github.com/docker/docker/pkg/ioutils" log "github.com/hashicorp/go-hclog" @@ -742,6 +743,48 @@ func (s *HTTPServer) AgentHostRequest(resp http.ResponseWriter, req *http.Reques return reply, rpcErr } +// AgentSchedulerWorkerConfigRequest is used to query the count (and state eventually) +// of the scheduler workers running in a Nomad server agent. +// This endpoint can also be used to update the count of running workers for a +// given agent. +func (s *HTTPServer) AgentSchedulerWorkerInfoRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + srv := s.agent.Server() + if srv == nil { + return nil, CodedError(http.StatusBadRequest, "server only endpoint") + } + if req.Method != http.MethodGet { + return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod) + } + + var secret string + s.parseToken(req, &secret) + + // Check agent read permissions + if aclObj, err := s.agent.Server().ResolveToken(secret); err != nil { + return nil, CodedError(http.StatusInternalServerError, err.Error()) + } else if aclObj != nil && !aclObj.AllowAgentRead() { + return nil, CodedError(http.StatusForbidden, structs.ErrPermissionDenied.Error()) + } + + schedulersInfo := srv.GetSchedulerWorkersInfo() + response := &agentSchedulerWorkersInfo{ + Workers: make([]agentSchedulerWorkerInfo, len(schedulersInfo)), + } + + for i, workerInfo := range schedulersInfo { + response.Workers[i] = agentSchedulerWorkerInfo{ + ID: workerInfo.ID, + EnabledSchedulers: make([]string, len(workerInfo.EnabledSchedulers)), + Started: workerInfo.Started.UTC().Format(time.RFC3339Nano), + Status: workerInfo.Status, + WorkloadStatus: workerInfo.WorkloadStatus, + } + copy(response.Workers[i].EnabledSchedulers, workerInfo.EnabledSchedulers) + } + + return response, nil +} + // AgentSchedulerWorkerConfigRequest is used to query the count (and state eventually) // of the scheduler workers running in a Nomad server agent. // This endpoint can also be used to update the count of running workers for a @@ -781,6 +824,7 @@ func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *htt NumSchedulers: config.NumSchedulers, EnabledSchedulers: config.EnabledSchedulers, } + return response, nil } @@ -826,3 +870,15 @@ type agentSchedulerWorkerConfig struct { NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } + +type agentSchedulerWorkersInfo struct { + Workers []agentSchedulerWorkerInfo `json:"workers"` +} + +type agentSchedulerWorkerInfo struct { + ID string `json:"id"` + EnabledSchedulers []string `json:"enabled_schedulers"` + Started string `json:"started"` + Status string `json:"status"` + WorkloadStatus string `json:"workload_status"` +} diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index 0111e681716..1e729158e8a 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -11,6 +11,7 @@ import ( "net/http/httptest" "net/url" "os" + "reflect" "strings" "sync" "syscall" @@ -1464,50 +1465,251 @@ func TestHTTP_XSS_Monitor(t *testing.T) { } } -/* - SchedulerWorkerAPI tests -*/ -type scheduleWorkerTest_workerRequestTest struct { +// ---------------------------- +// SchedulerWorkerInfoAPI tests +// ---------------------------- +type schedulerWorkerAPITest_testCase struct { name string // test case name - request schedulerWorkerTest_testRequest - whenACLNotEnabled schedulerWorkerTest_testExpect - whenACLEnabled schedulerWorkerTest_testExpect + request schedulerWorkerAPITest_testRequest + whenACLNotEnabled schedulerWorkerAPITest_testExpect + whenACLEnabled schedulerWorkerAPITest_testExpect } -type schedulerWorkerTest_testRequest struct { + +type schedulerWorkerAPITest_testRequest struct { verb string aclToken string requestBody string } -type schedulerWorkerTest_testExpect struct { + +type schedulerWorkerAPITest_testExpect struct { + statusCode int + response interface{} + err error + isError bool +} + +func (te schedulerWorkerAPITest_testExpect) Code() int { + return te.statusCode +} + +func schedulerWorkerInfoTest_testCases() []schedulerWorkerAPITest_testCase { + forbidden := schedulerWorkerAPITest_testExpect{ + statusCode: http.StatusForbidden, + response: structs.ErrPermissionDenied.Error(), + isError: true, + } + invalidMethod := schedulerWorkerAPITest_testExpect{ + statusCode: http.StatusMethodNotAllowed, + response: ErrInvalidMethod, + isError: true, + } + success := schedulerWorkerAPITest_testExpect{ + statusCode: http.StatusOK, + response: &agentSchedulerWorkersInfo{ + Workers: []agentSchedulerWorkerInfo{ + { + ID: "9b3713e0-6f74-0e1b-3b3e-d94f0c22dbf9", + EnabledSchedulers: []string{"_core", "batch"}, + Started: "2021-12-10 22:13:12.595366 -0500 EST m=+0.039016232", + Status: "Pausing", + WorkloadStatus: "WaitingToDequeue", + }, + { + ID: "ebda23e2-7f68-0c82-f0b2-f91d4581094d", + EnabledSchedulers: []string{"_core", "batch"}, + Started: "2021-12-10 22:13:12.595478 -0500 EST m=+0.039127886", + Status: "Pausing", + WorkloadStatus: "WaitingToDequeue", + }, + { + ID: "b3869c9b-64ff-686c-a003-e7d059d3a573", + EnabledSchedulers: []string{"_core", "batch"}, + Started: "2021-12-10 22:13:12.595501 -0500 EST m=+0.039151276", + Status: "Pausing", + WorkloadStatus: "WaitingToDequeue", + }, + { + ID: "cc5907c0-552e-bf36-0ca1-f150af7273c2", + EnabledSchedulers: []string{"_core", "batch"}, + Started: "2021-12-10 22:13:12.595691 -0500 EST m=+0.039341541", + Status: "Starting", + WorkloadStatus: "WaitingToDequeue", + }, + }, + }, + } + return []schedulerWorkerAPITest_testCase{ + { + name: "bad verb", + request: schedulerWorkerAPITest_testRequest{ + verb: "FOO", + aclToken: "", + requestBody: "", + }, + whenACLNotEnabled: invalidMethod, + whenACLEnabled: invalidMethod, + }, + { + name: "get without token", + request: schedulerWorkerAPITest_testRequest{ + verb: "GET", + aclToken: "", + requestBody: "", + }, + whenACLNotEnabled: success, + whenACLEnabled: forbidden, + }, + { + name: "get with management token", + request: schedulerWorkerAPITest_testRequest{ + verb: "GET", + aclToken: "management", + requestBody: "", + }, + whenACLNotEnabled: success, + whenACLEnabled: success, + }, + { + name: "get with read token", + request: schedulerWorkerAPITest_testRequest{ + verb: "GET", + aclToken: "agent_read", + requestBody: "", + }, + whenACLNotEnabled: success, + whenACLEnabled: success, + }, + { + name: "get with invalid token", + request: schedulerWorkerAPITest_testRequest{ + verb: "GET", + aclToken: "node_write", + requestBody: "", + }, + whenACLNotEnabled: success, + whenACLEnabled: forbidden, + }, + } +} + +func TestHTTP_AgentSchedulerWorkerInfoRequest(t *testing.T) { + configFn := func(c *Config) { + var numSchedulers = 4 + c.Server.NumSchedulers = &numSchedulers + c.Server.EnabledSchedulers = []string{"_core", "batch"} + c.Client.Enabled = false + } + + for _, runACL := range []string{"no_acl", "acl"} { + t.Run(runACL, func(t *testing.T) { + tests := func(s *TestAgent) { + testingACLS := s.Config.ACL.Enabled + var tokens map[string]*structs.ACLToken + if s.Config.ACL.Enabled { + state := s.Agent.server.State() + tokens = make(map[string]*structs.ACLToken) + + tokens["management"] = s.RootToken + tokens["agent_read"] = mock.CreatePolicyAndToken(t, state, 1005, "agent_read", mock.AgentPolicy(acl.PolicyRead)) + tokens["agent_write"] = mock.CreatePolicyAndToken(t, state, 1007, "agent_write", mock.AgentPolicy(acl.PolicyWrite)) + tokens["node_write"] = mock.CreatePolicyAndToken(t, state, 1009, "node_write", mock.NodePolicy(acl.PolicyWrite)) + } + + for _, tc := range schedulerWorkerInfoTest_testCases() { + t.Run(tc.name, func(t *testing.T) { + req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) + if testingACLS && tc.request.aclToken != "" { + setToken(req, tokens[tc.request.aclToken]) + } + require.Nil(t, err) + respW := httptest.NewRecorder() + workerInfoResp, err := s.Server.AgentSchedulerWorkerInfoRequest(respW, req) + + expected := tc.whenACLNotEnabled + if testingACLS { + expected = tc.whenACLEnabled + } + + if expected.isError { + require.Error(t, err) + codedErr, ok := err.(HTTPCodedError) + require.True(t, ok, "expected a HTTPCodedError") + require.Equal(t, expected.Code(), codedErr.Code()) + require.Equal(t, expected.response, codedErr.Error()) + return + } + + require.NoError(t, err) + workerInfo, ok := workerInfoResp.(*agentSchedulerWorkersInfo) + require.True(t, ok, "expected an *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) + + expectWorkerInfo, ok := expected.response.(*agentSchedulerWorkersInfo) + require.True(t, ok, "expected an *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) + + var schedCount int = *s.Config.Server.NumSchedulers + require.Equal(t, schedCount, len(workerInfo.Workers), "must match num_schedulers") + require.Equal(t, len(expectWorkerInfo.Workers), len(workerInfo.Workers), "lengths must match") + + for i, info := range expectWorkerInfo.Workers { + require.ElementsMatch(t, info.EnabledSchedulers, workerInfo.Workers[i].EnabledSchedulers) + } + }) + } + } + + if runACL == "acl" { + httpACLTest(t, configFn, tests) + } else { + httpTest(t, configFn, tests) + } + }) + } +} + +// ---------------------------- +// SchedulerWorkerConfigAPI tests +// ---------------------------- +type scheduleWorkerConfigTest_workerRequestTest struct { + name string // test case name + request schedulerWorkerConfigTest_testRequest + whenACLNotEnabled schedulerWorkerConfigTest_testExpect + whenACLEnabled schedulerWorkerConfigTest_testExpect +} +type schedulerWorkerConfigTest_testRequest struct { + verb string + aclToken string + requestBody string +} +type schedulerWorkerConfigTest_testExpect struct { expectedResponseCode int expectedResponse interface{} } // These test cases are run for both the ACL and Non-ACL enabled servers. When // ACLS are not enabled, the request.aclTokens are ignored. -func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { - forbidden := schedulerWorkerTest_testExpect{ +func schedulerWorkerConfigTest_testCases() []scheduleWorkerConfigTest_workerRequestTest { + forbidden := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusForbidden, expectedResponse: structs.ErrPermissionDenied.Error(), } - invalidMethod := schedulerWorkerTest_testExpect{ + invalidMethod := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusMethodNotAllowed, expectedResponse: ErrInvalidMethod, } - success1 := schedulerWorkerTest_testExpect{ + success1 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, } - success2 := schedulerWorkerTest_testExpect{ + success2 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 9}, } - return []scheduleWorkerTest_workerRequestTest{ + return []scheduleWorkerConfigTest_workerRequestTest{ { name: "bad verb", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "FOO", aclToken: "", requestBody: "", @@ -1517,7 +1719,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "get without token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "GET", aclToken: "", requestBody: "", @@ -1527,7 +1729,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "get with management token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "GET", aclToken: "management", requestBody: "", @@ -1537,7 +1739,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "get with read token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "GET", aclToken: "agent_read", requestBody: "", @@ -1547,7 +1749,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "get with write token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "GET", aclToken: "agent_write", requestBody: "", @@ -1557,7 +1759,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "post with no token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "POST", aclToken: "", requestBody: `{"num_schedulers":9,"enabled_schedulers":["_core", "batch"]}`, @@ -1567,7 +1769,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "put with no token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "PUT", aclToken: "", requestBody: `{"num_schedulers":8,"enabled_schedulers":["_core", "batch"]}`, @@ -1577,7 +1779,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "post with invalid token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "POST", aclToken: "node_write", requestBody: `{"num_schedulers":9,"enabled_schedulers":["_core", "batch"]}`, @@ -1587,7 +1789,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "put with invalid token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "PUT", aclToken: "node_write", requestBody: `{"num_schedulers":8,"enabled_schedulers":["_core", "batch"]}`, @@ -1597,7 +1799,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "post with valid token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "POST", aclToken: "agent_write", requestBody: `{"num_schedulers":9,"enabled_schedulers":["_core", "batch"]}`, @@ -1607,7 +1809,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { }, { name: "put with valid token", - request: schedulerWorkerTest_testRequest{ + request: schedulerWorkerConfigTest_testRequest{ verb: "PUT", aclToken: "agent_write", requestBody: `{"num_schedulers":8,"enabled_schedulers":["_core", "batch"]}`, @@ -1618,7 +1820,7 @@ func schedulerWorkerTest_testCases() []scheduleWorkerTest_workerRequestTest { } } -func TestHTTP_AgentSchedulerWorkerRequest_NoACL(t *testing.T) { +func TestHTTP_AgentSchedulerWorkerConfigRequest_NoACL(t *testing.T) { configFn := func(c *Config) { var numSchedulers = 8 c.Server.NumSchedulers = &numSchedulers @@ -1626,10 +1828,10 @@ func TestHTTP_AgentSchedulerWorkerRequest_NoACL(t *testing.T) { c.Client.Enabled = false } testFn := func(s *TestAgent) { - for _, tc := range schedulerWorkerTest_testCases() { + for _, tc := range schedulerWorkerConfigTest_testCases() { t.Run(tc.name, func(t *testing.T) { - req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) + req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers/config", bytes.NewReader([]byte(tc.request.requestBody))) require.Nil(t, err) respW := httptest.NewRecorder() workersI, err := s.Server.AgentSchedulerWorkerConfigRequest(respW, req) @@ -1649,7 +1851,7 @@ func TestHTTP_AgentSchedulerWorkerRequest_NoACL(t *testing.T) { httpTest(t, configFn, testFn) } -func TestHTTP_AgentSchedulerWorkerRequest_ACL(t *testing.T) { +func TestHTTP_AgentSchedulerWorkerConfigRequest_ACL(t *testing.T) { configFn := func(c *Config) { var numSchedulers = 8 c.Server.NumSchedulers = &numSchedulers @@ -1666,7 +1868,7 @@ func TestHTTP_AgentSchedulerWorkerRequest_ACL(t *testing.T) { tokens["agent_write"] = mock.CreatePolicyAndToken(t, state, 1007, "agent_write", mock.AgentPolicy(acl.PolicyWrite)) tokens["node_write"] = mock.CreatePolicyAndToken(t, state, 1009, "node_write", mock.NodePolicy(acl.PolicyWrite)) - for _, tc := range schedulerWorkerTest_testCases() { + for _, tc := range schedulerWorkerConfigTest_testCases() { t.Run(tc.name, func(t *testing.T) { req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) @@ -1692,7 +1894,7 @@ func TestHTTP_AgentSchedulerWorkerRequest_ACL(t *testing.T) { httpACLTest(t, configFn, tests) } -func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc scheduleWorkerTest_workerRequestTest, workersI interface{}, err error) { +func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc scheduleWorkerConfigTest_workerRequestTest, workersI interface{}, err error) { require.NoError(t, err) require.NotNil(t, workersI) @@ -1716,7 +1918,7 @@ func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc schedu // schedulerWorkerTest_parseError parses the error response given // from the API call to make sure that it's a coded error and is the // expected value from the test case -func schedulerWorkerTest_parseError(t *testing.T, isACLEnabled bool, tc scheduleWorkerTest_workerRequestTest, workersI interface{}, err error) { +func schedulerWorkerTest_parseError(t *testing.T, isACLEnabled bool, tc scheduleWorkerConfigTest_workerRequestTest, workersI interface{}, err error) { require.Error(t, err) require.Nil(t, workersI) diff --git a/command/agent/http.go b/command/agent/http.go index 13c79a530bb..db657991296 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -293,7 +293,8 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/agent/members", s.wrap(s.AgentMembersRequest)) s.mux.HandleFunc("/v1/agent/force-leave", s.wrap(s.AgentForceLeaveRequest)) s.mux.HandleFunc("/v1/agent/servers", s.wrap(s.AgentServersRequest)) - s.mux.HandleFunc("/v1/agent/workers", s.wrap(s.AgentSchedulerWorkerConfigRequest)) + s.mux.HandleFunc("/v1/agent/workers", s.wrap(s.AgentSchedulerWorkerInfoRequest)) + s.mux.HandleFunc("/v1/agent/workers/config", s.wrap(s.AgentSchedulerWorkerConfigRequest)) s.mux.HandleFunc("/v1/agent/keyring/", s.wrap(s.KeyringOperationRequest)) s.mux.HandleFunc("/v1/agent/health", s.wrap(s.HealthRequest)) s.mux.HandleFunc("/v1/agent/host", s.wrap(s.AgentHostRequest)) diff --git a/nomad/server.go b/nomad/server.go index d1f2b3c7465..e283c13e838 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -1524,6 +1524,19 @@ func getSchedulerWorkerPoolArgsFromConfigLocked(c *Config) *SchedulerWorkerPoolA } } +// GetSchedulerWorkerInfo returns a slice of WorkerInfos from all of +// the running scheduler workers. +func (s *Server) GetSchedulerWorkersInfo() []WorkerInfo { + s.workerLock.RLock() + defer s.workerLock.RUnlock() + out := make([]WorkerInfo, len(s.workers)) + for i := 0; i < len(s.workers); i = i + 1 { + workerInfo := s.workers[i].Info() + out[i] = workerInfo.Copy() + } + return out +} + // GetSchedulerWorkerConfig returns a clean copy of the server's current scheduler // worker config. func (s *Server) GetSchedulerWorkerConfig() SchedulerWorkerPoolArgs { diff --git a/nomad/worker.go b/nomad/worker.go index 884eee7dc9d..04433faa0fa 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -2,6 +2,7 @@ package nomad import ( "context" + "encoding/json" "fmt" "strings" "sync" @@ -226,6 +227,46 @@ func (w *Worker) setWorkloadStatus(newStatus SchedulerWorkerStatus) { w.workloadStatus = newStatus } +type WorkerInfo struct { + ID string `json:"id"` + EnabledSchedulers []string `json:"enabled_schedulers"` + Started time.Time `json:"started"` + Status string `json:"status"` + WorkloadStatus string `json:"workload_status"` +} + +func (w WorkerInfo) Copy() WorkerInfo { + out := WorkerInfo{ + ID: w.ID, + EnabledSchedulers: make([]string, len(w.EnabledSchedulers)), + Started: w.Started, + Status: w.Status, + WorkloadStatus: w.WorkloadStatus, + } + copy(out.EnabledSchedulers, w.EnabledSchedulers) + return out +} + +func (w WorkerInfo) String() string { + // lazy implementation of WorkerInfo to string + out, _ := json.Marshal(w) + return string(out) +} + +func (w *Worker) Info() WorkerInfo { + w.pauseLock.Lock() + defer w.pauseLock.Unlock() + out := WorkerInfo{ + ID: w.id, + Status: w.status.String(), + WorkloadStatus: w.workloadStatus.String(), + EnabledSchedulers: make([]string, len(w.enabledSchedulers)), + } + out.Started = w.start + copy(out.EnabledSchedulers, w.enabledSchedulers) + return out +} + // ---------------------------------- // Pause Implementation // These functions are used to support the worker's pause behaviors. diff --git a/nomad/worker_test.go b/nomad/worker_test.go index a8993bb18d4..08e5cb993f7 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -2,6 +2,7 @@ package nomad import ( "context" + "encoding/json" "fmt" "reflect" "sync" @@ -12,6 +13,7 @@ import ( "github.com/hashicorp/go-memdb" "github.com/stretchr/testify/require" + "github.com/hashicorp/nomad/helper/testlog" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" @@ -802,3 +804,97 @@ func TestWorker_ReblockEval(t *testing.T) { reblockedEval.SnapshotIndex, w.snapshotIndex) } } + +func TestWorker_Info(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.NumSchedulers = 0 + c.EnabledSchedulers = []string{structs.JobTypeService} + }) + defer cleanupS1() + testutil.WaitForLeader(t, s1.RPC) + + poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() + + // Create a worker + w, err := newWorker(s1.shutdownCtx, s1, poolArgs) + require.NoError(t, err) + + require.Equal(t, WorkerStarting, w.GetStatus()) + workerInfo := w.Info() + require.Equal(t, WorkerStarting.String(), workerInfo.Status) +} + +func TestWorker_WorkerInfo_String(t *testing.T) { + t.Parallel() + startTime := time.Date(2009, time.November, 10, 23, 0, 0, 0, time.UTC) + w := &Worker{ + id: "uuid", + start: startTime, + status: WorkerStarted, + workloadStatus: WorkloadBackoff, + enabledSchedulers: []string{structs.JobTypeCore, structs.JobTypeBatch, structs.JobTypeSystem}, + } + _, err := json.Marshal(w) + require.NoError(t, err) + + require.Equal(t, `{"id":"uuid","enabled_schedulers":["_core","batch","system"],"started":"2009-11-10T23:00:00Z","status":"Started","workload_status":"Backoff"}`, fmt.Sprint(w.Info())) +} + +const ( + longWait = 100 * time.Millisecond + tinyWait = 10 * time.Millisecond +) + +func TestWorker_SetPause(t *testing.T) { + t.Parallel() + logger := testlog.HCLogger(t) + srv := &Server{ + logger: logger, + shutdownCtx: context.Background(), + } + args := SchedulerWorkerPoolArgs{ + EnabledSchedulers: []string{structs.JobTypeCore, structs.JobTypeBatch, structs.JobTypeSystem}, + } + w, err := newWorker(context.Background(), srv, args) + require.NoError(t, err) + + w._start(testWorkload) + require.Eventually(t, w.IsStarted, longWait, tinyWait, "should have started") + + go func() { + time.Sleep(tinyWait) + w.Pause() + }() + require.Eventually(t, w.IsPaused, longWait, tinyWait, "should have paused") + + go func() { + time.Sleep(tinyWait) + w.Resume() + }() + require.Eventually(t, w.IsStarted, longWait, tinyWait, "should have restarted from pause") + + go func() { + time.Sleep(tinyWait) + w.Stop() + }() + require.Eventually(t, w.IsStopped, longWait, tinyWait, "should have shutdown") +} + +func testWorkload(w *Worker) { + defer w.markStopped() + w.setStatus(WorkerStarted) + w.setWorkloadStatus(WorkloadRunning) + w.logger.Debug("testWorkload running") + for { + // ensure state variables are happy after resuming. + w.maybeWait() + if w.workerShuttingDown() { + w.logger.Debug("testWorkload stopped") + return + } + // do some fake work + time.Sleep(10 * time.Millisecond) + } +} From 60d53fa535221210e8484e16870e5f0b40e310c7 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Sat, 11 Dec 2021 01:21:02 -0500 Subject: [PATCH 18/30] Added worker info api to api package --- api/agent.go | 35 ++++++++++++++++++++++++++++------- api/agent_test.go | 16 ++++++++++++++++ 2 files changed, 44 insertions(+), 7 deletions(-) diff --git a/api/agent.go b/api/agent.go index c95f3829955..505df65b0bb 100644 --- a/api/agent.go +++ b/api/agent.go @@ -487,7 +487,7 @@ type HostDataResponse struct { // GetSchedulerWorkerConfig returns the targeted agent's worker pool configuration func (a *Agent) GetSchedulerWorkerConfig() (*SchedulerWorkerPoolArgs, error) { var resp AgentSchedulerWorkerConfigResponse - _, err := a.client.query("/v1/agent/workers", &resp, nil) + _, err := a.client.query("/v1/agent/workers/config", &resp, nil) if err != nil { return nil, err } @@ -497,13 +497,10 @@ func (a *Agent) GetSchedulerWorkerConfig() (*SchedulerWorkerPoolArgs, error) { // SetSchedulerWorkerConfig attempts to update the targeted agent's worker pool configuration func (a *Agent) SetSchedulerWorkerConfig(args SchedulerWorkerPoolArgs) (*SchedulerWorkerPoolArgs, error) { - req := AgentSchedulerWorkerConfigRequest{ - NumSchedulers: args.NumSchedulers, - EnabledSchedulers: args.EnabledSchedulers, - } - + req := AgentSchedulerWorkerConfigRequest(args) var resp AgentSchedulerWorkerConfigResponse - _, err := a.client.write("/v1/agent/workers", &req, &resp, nil) + + _, err := a.client.write("/v1/agent/workers/config", &req, &resp, nil) if err != nil { return nil, err } @@ -526,3 +523,27 @@ type AgentSchedulerWorkerConfigResponse struct { NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } + +func (a *Agent) GetSchedulerWorkersInfo() (*AgentSchedulerWorkersInfo, error) { + var out *AgentSchedulerWorkersInfo + + // Query the self endpoint on the agent + _, err := a.client.query("/v1/agent/workers", &out, nil) + if err != nil { + return nil, err + } + + return out, nil +} + +type AgentSchedulerWorkersInfo struct { + Workers []AgentSchedulerWorkerInfo `json:"workers"` +} + +type AgentSchedulerWorkerInfo struct { + ID string `json:"id"` + EnabledSchedulers []string `json:"enabled_schedulers"` + Started string `json:"started"` + Status string `json:"status"` + WorkloadStatus string `json:"workload_status"` +} diff --git a/api/agent_test.go b/api/agent_test.go index 715db417cbe..868d3ef3fd9 100644 --- a/api/agent_test.go +++ b/api/agent_test.go @@ -466,8 +466,24 @@ func TestAgent_SchedulerWorkerConfig(t *testing.T) { config, err := a.GetSchedulerWorkerConfig() require.Nil(t, err) + require.NotNil(t, config) newConfig := SchedulerWorkerPoolArgs{NumSchedulers: 0, EnabledSchedulers: []string{"_core", "system"}} resp, err := a.SetSchedulerWorkerConfig(newConfig) require.NoError(t, err) assert.NotEqual(t, config, resp) } + +func TestAgent_SchedulerWorkersInfo(t *testing.T) { + t.Parallel() + c, s := makeClient(t, nil, nil) + defer s.Stop() + a := c.Agent() + + info, err := a.GetSchedulerWorkersInfo() + require.Nil(t, err) + require.NotNil(t, info) + defaultSchedulers := []string{"batch", "system", "sysbatch", "service", "_core"} + for _, worker := range info.Workers { + require.ElementsMatch(t, defaultSchedulers, worker.EnabledSchedulers) + } +} From 3d755aa71fb8efaff3175537bc3f130816cb62f4 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Sat, 11 Dec 2021 01:45:11 -0500 Subject: [PATCH 19/30] bugfixes --- nomad/server.go | 1 - nomad/worker.go | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/nomad/server.go b/nomad/server.go index e283c13e838..905389a912d 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -1550,7 +1550,6 @@ func (s *Server) SetSchedulerWorkerConfig(newArgs SchedulerWorkerPoolArgs) Sched if newVals.IsValid() { reloadSchedulers(s, newVals) } - reloadSchedulers(s, newVals) } return s.GetSchedulerWorkerConfig() } diff --git a/nomad/worker.go b/nomad/worker.go index 04433faa0fa..cfc56bcf8fd 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -350,6 +350,7 @@ func (w *Worker) run() { w.setWorkloadStatus(WorkloadStopped) w.markStopped() }() + w.setStatus(WorkerStarted) w.setWorkloadStatus(WorkloadRunning) w.logger.Debug("running") for { From 4ee6b8cf0034c8149eaeeb3554a2608983d2eaf7 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Mon, 13 Dec 2021 12:41:31 -0500 Subject: [PATCH 20/30] Adding stringer to build deps --- GNUmakefile | 1 + 1 file changed, 1 insertion(+) diff --git a/GNUmakefile b/GNUmakefile index cdc685e5531..95d2463ea0b 100644 --- a/GNUmakefile +++ b/GNUmakefile @@ -124,6 +124,7 @@ deps: ## Install build and development dependencies go install github.com/hashicorp/go-msgpack/codec/codecgen@v1.1.5 go install github.com/bufbuild/buf/cmd/buf@v0.36.0 go install github.com/hashicorp/go-changelog/cmd/changelog-build@latest + go install golang.org/x/tools/cmd/stringer@v0.1.8 .PHONY: lint-deps lint-deps: ## Install linter dependencies From 71dab369328c68d678378ce98cc865031be4bee0 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Mon, 20 Dec 2021 17:59:40 -0500 Subject: [PATCH 21/30] Changing route to /v1/agent/schedulers --- api/agent.go | 10 +- api/agent_test.go | 2 +- command/agent/agent_endpoint.go | 33 ++++--- command/agent/agent_endpoint_test.go | 134 +++++++++++++++++++++++++-- command/agent/http.go | 8 +- 5 files changed, 156 insertions(+), 31 deletions(-) diff --git a/api/agent.go b/api/agent.go index 505df65b0bb..2ccd56846b2 100644 --- a/api/agent.go +++ b/api/agent.go @@ -487,7 +487,7 @@ type HostDataResponse struct { // GetSchedulerWorkerConfig returns the targeted agent's worker pool configuration func (a *Agent) GetSchedulerWorkerConfig() (*SchedulerWorkerPoolArgs, error) { var resp AgentSchedulerWorkerConfigResponse - _, err := a.client.query("/v1/agent/workers/config", &resp, nil) + _, err := a.client.query("/v1/agent/schedulers/config", &resp, nil) if err != nil { return nil, err } @@ -500,7 +500,7 @@ func (a *Agent) SetSchedulerWorkerConfig(args SchedulerWorkerPoolArgs) (*Schedul req := AgentSchedulerWorkerConfigRequest(args) var resp AgentSchedulerWorkerConfigResponse - _, err := a.client.write("/v1/agent/workers/config", &req, &resp, nil) + _, err := a.client.write("/v1/agent/schedulers/config", &req, &resp, nil) if err != nil { return nil, err } @@ -520,6 +520,7 @@ type AgentSchedulerWorkerConfigRequest struct { } type AgentSchedulerWorkerConfigResponse struct { + ServerID string `json:"server_id"` NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } @@ -528,7 +529,7 @@ func (a *Agent) GetSchedulerWorkersInfo() (*AgentSchedulerWorkersInfo, error) { var out *AgentSchedulerWorkersInfo // Query the self endpoint on the agent - _, err := a.client.query("/v1/agent/workers", &out, nil) + _, err := a.client.query("/v1/agent/schedulers", &out, nil) if err != nil { return nil, err } @@ -537,7 +538,8 @@ func (a *Agent) GetSchedulerWorkersInfo() (*AgentSchedulerWorkersInfo, error) { } type AgentSchedulerWorkersInfo struct { - Workers []AgentSchedulerWorkerInfo `json:"workers"` + ServerID string `json:"server_id"` + Schedulers []AgentSchedulerWorkerInfo `json:"schedulers"` } type AgentSchedulerWorkerInfo struct { diff --git a/api/agent_test.go b/api/agent_test.go index 868d3ef3fd9..238be8def41 100644 --- a/api/agent_test.go +++ b/api/agent_test.go @@ -483,7 +483,7 @@ func TestAgent_SchedulerWorkersInfo(t *testing.T) { require.Nil(t, err) require.NotNil(t, info) defaultSchedulers := []string{"batch", "system", "sysbatch", "service", "_core"} - for _, worker := range info.Workers { + for _, worker := range info.Schedulers { require.ElementsMatch(t, defaultSchedulers, worker.EnabledSchedulers) } } diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index b8e712e9d50..3796bae912b 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -743,14 +743,12 @@ func (s *HTTPServer) AgentHostRequest(resp http.ResponseWriter, req *http.Reques return reply, rpcErr } -// AgentSchedulerWorkerConfigRequest is used to query the count (and state eventually) -// of the scheduler workers running in a Nomad server agent. -// This endpoint can also be used to update the count of running workers for a -// given agent. +// AgentSchedulerWorkerInfoRequest is used to query the running state of the +// agent's scheduler workers. func (s *HTTPServer) AgentSchedulerWorkerInfoRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { - return nil, CodedError(http.StatusBadRequest, "server only endpoint") + return nil, CodedError(http.StatusBadRequest, ErrServerOnly) } if req.Method != http.MethodGet { return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod) @@ -768,18 +766,19 @@ func (s *HTTPServer) AgentSchedulerWorkerInfoRequest(resp http.ResponseWriter, r schedulersInfo := srv.GetSchedulerWorkersInfo() response := &agentSchedulerWorkersInfo{ - Workers: make([]agentSchedulerWorkerInfo, len(schedulersInfo)), + ServerID: srv.LocalMember().Name, + Schedulers: make([]agentSchedulerWorkerInfo, len(schedulersInfo)), } for i, workerInfo := range schedulersInfo { - response.Workers[i] = agentSchedulerWorkerInfo{ + response.Schedulers[i] = agentSchedulerWorkerInfo{ ID: workerInfo.ID, EnabledSchedulers: make([]string, len(workerInfo.EnabledSchedulers)), Started: workerInfo.Started.UTC().Format(time.RFC3339Nano), Status: workerInfo.Status, WorkloadStatus: workerInfo.WorkloadStatus, } - copy(response.Workers[i].EnabledSchedulers, workerInfo.EnabledSchedulers) + copy(response.Schedulers[i].EnabledSchedulers, workerInfo.EnabledSchedulers) } return response, nil @@ -791,7 +790,7 @@ func (s *HTTPServer) AgentSchedulerWorkerInfoRequest(resp http.ResponseWriter, r // given agent. func (s *HTTPServer) AgentSchedulerWorkerConfigRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { if s.agent.Server() == nil { - return nil, CodedError(http.StatusBadRequest, "server only endpoint") + return nil, CodedError(http.StatusBadRequest, ErrServerOnly) } switch req.Method { case "PUT", "POST": @@ -806,7 +805,7 @@ func (s *HTTPServer) AgentSchedulerWorkerConfigRequest(resp http.ResponseWriter, func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { - return nil, CodedError(http.StatusBadRequest, "server only endpoint") + return nil, CodedError(http.StatusBadRequest, ErrServerOnly) } var secret string @@ -821,6 +820,7 @@ func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *htt config := srv.GetSchedulerWorkerConfig() response := &agentSchedulerWorkerConfig{ + ServerID: srv.LocalMember().Name, NumSchedulers: config.NumSchedulers, EnabledSchedulers: config.EnabledSchedulers, } @@ -831,7 +831,7 @@ func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *htt func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { - return nil, CodedError(400, "server only endpoint") + return nil, CodedError(http.StatusBadRequest, ErrServerOnly) } var secret string @@ -847,18 +847,21 @@ func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req * var args agentSchedulerWorkerConfig if err := decodeBody(req, &args); err != nil { - return nil, CodedError(http.StatusBadRequest, err.Error()) + return nil, CodedError(http.StatusBadRequest, fmt.Sprintf("Invalid request: %s", err.Error())) } + // the server_id provided in the payload is ignored to allow the + // response to be roundtripped right into a PUT. newArgs := nomad.SchedulerWorkerPoolArgs{ NumSchedulers: args.NumSchedulers, EnabledSchedulers: args.EnabledSchedulers, } if newArgs.IsInvalid() { - return nil, CodedError(http.StatusBadRequest, "invalid arguments") + return nil, CodedError(http.StatusBadRequest, "Invalid request") } reply := srv.SetSchedulerWorkerConfig(newArgs) response := &agentSchedulerWorkerConfig{ + ServerID: srv.LocalMember().Name, NumSchedulers: reply.NumSchedulers, EnabledSchedulers: reply.EnabledSchedulers, } @@ -867,12 +870,14 @@ func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req * } type agentSchedulerWorkerConfig struct { + ServerID string `json:"server_id,omitempty"` NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } type agentSchedulerWorkersInfo struct { - Workers []agentSchedulerWorkerInfo `json:"workers"` + ServerID string `json:"server_id"` + Schedulers []agentSchedulerWorkerInfo `json:"schedulers"` } type agentSchedulerWorkerInfo struct { diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index 1e729158e8a..b2074c5c309 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -1506,7 +1506,7 @@ func schedulerWorkerInfoTest_testCases() []schedulerWorkerAPITest_testCase { success := schedulerWorkerAPITest_testExpect{ statusCode: http.StatusOK, response: &agentSchedulerWorkersInfo{ - Workers: []agentSchedulerWorkerInfo{ + Schedulers: []agentSchedulerWorkerInfo{ { ID: "9b3713e0-6f74-0e1b-3b3e-d94f0c22dbf9", EnabledSchedulers: []string{"_core", "batch"}, @@ -1617,7 +1617,7 @@ func TestHTTP_AgentSchedulerWorkerInfoRequest(t *testing.T) { for _, tc := range schedulerWorkerInfoTest_testCases() { t.Run(tc.name, func(t *testing.T) { - req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) + req, err := http.NewRequest(tc.request.verb, "/v1/agent/schedulers", bytes.NewReader([]byte(tc.request.requestBody))) if testingACLS && tc.request.aclToken != "" { setToken(req, tokens[tc.request.aclToken]) } @@ -1644,14 +1644,14 @@ func TestHTTP_AgentSchedulerWorkerInfoRequest(t *testing.T) { require.True(t, ok, "expected an *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) expectWorkerInfo, ok := expected.response.(*agentSchedulerWorkersInfo) - require.True(t, ok, "expected an *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) + require.True(t, ok, "error casting test case to *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) var schedCount int = *s.Config.Server.NumSchedulers - require.Equal(t, schedCount, len(workerInfo.Workers), "must match num_schedulers") - require.Equal(t, len(expectWorkerInfo.Workers), len(workerInfo.Workers), "lengths must match") + require.Equal(t, schedCount, len(workerInfo.Schedulers), "must match num_schedulers") + require.Equal(t, len(expectWorkerInfo.Schedulers), len(workerInfo.Schedulers), "lengths must match") - for i, info := range expectWorkerInfo.Workers { - require.ElementsMatch(t, info.EnabledSchedulers, workerInfo.Workers[i].EnabledSchedulers) + for i, info := range expectWorkerInfo.Schedulers { + require.ElementsMatch(t, info.EnabledSchedulers, workerInfo.Schedulers[i].EnabledSchedulers) } }) } @@ -1696,6 +1696,10 @@ func schedulerWorkerConfigTest_testCases() []scheduleWorkerConfigTest_workerRequ expectedResponseCode: http.StatusMethodNotAllowed, expectedResponse: ErrInvalidMethod, } + invalidRequest := schedulerWorkerConfigTest_testExpect{ + expectedResponseCode: http.StatusBadRequest, + expectedResponse: "Invalid request", + } success1 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, @@ -1817,6 +1821,66 @@ func schedulerWorkerConfigTest_testCases() []scheduleWorkerConfigTest_workerRequ whenACLNotEnabled: success1, whenACLEnabled: success1, }, + { + name: "post with good token and bad value", + request: schedulerWorkerConfigTest_testRequest{ + verb: "POST", + aclToken: "agent_write", + requestBody: `{"num_schedulers":-1,"enabled_schedulers":["_core", "batch"]}`, + }, + whenACLNotEnabled: invalidRequest, + whenACLEnabled: invalidRequest, + }, + { + name: "post with bad token and bad value", + request: schedulerWorkerConfigTest_testRequest{ + verb: "POST", + aclToken: "node_write", + requestBody: `{"num_schedulers":-1,"enabled_schedulers":["_core", "batch"]}`, + }, + whenACLNotEnabled: invalidRequest, + whenACLEnabled: forbidden, + }, + { + name: "put with good token and bad value", + request: schedulerWorkerConfigTest_testRequest{ + verb: "PUT", + aclToken: "agent_write", + requestBody: `{"num_schedulers":-1,"enabled_schedulers":["_core", "batch"]}`, + }, + whenACLNotEnabled: invalidRequest, + whenACLEnabled: invalidRequest, + }, + { + name: "put with bad token and bad value", + request: schedulerWorkerConfigTest_testRequest{ + verb: "PUT", + aclToken: "node_write", + requestBody: `{"num_schedulers":-1,"enabled_schedulers":["_core", "batch"]}`, + }, + whenACLNotEnabled: invalidRequest, + whenACLEnabled: forbidden, + }, + { + name: "post with bad json", + request: schedulerWorkerConfigTest_testRequest{ + verb: "POST", + aclToken: "agent_write", + requestBody: `{num_schedulers:-1,"enabled_schedulers":["_core", "batch"]}`, + }, + whenACLNotEnabled: invalidRequest, + whenACLEnabled: invalidRequest, + }, + { + name: "put with bad json", + request: schedulerWorkerConfigTest_testRequest{ + verb: "PUT", + aclToken: "agent_write", + requestBody: `{num_schedulers:-1,"enabled_schedulers":["_core", "batch"]}`, + }, + whenACLNotEnabled: invalidRequest, + whenACLEnabled: invalidRequest, + }, } } @@ -1831,7 +1895,7 @@ func TestHTTP_AgentSchedulerWorkerConfigRequest_NoACL(t *testing.T) { for _, tc := range schedulerWorkerConfigTest_testCases() { t.Run(tc.name, func(t *testing.T) { - req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers/config", bytes.NewReader([]byte(tc.request.requestBody))) + req, err := http.NewRequest(tc.request.verb, "/v1/agent/schedulers/config", bytes.NewReader([]byte(tc.request.requestBody))) require.Nil(t, err) respW := httptest.NewRecorder() workersI, err := s.Server.AgentSchedulerWorkerConfigRequest(respW, req) @@ -1871,7 +1935,7 @@ func TestHTTP_AgentSchedulerWorkerConfigRequest_ACL(t *testing.T) { for _, tc := range schedulerWorkerConfigTest_testCases() { t.Run(tc.name, func(t *testing.T) { - req, err := http.NewRequest(tc.request.verb, "/v1/agent/workers", bytes.NewReader([]byte(tc.request.requestBody))) + req, err := http.NewRequest(tc.request.verb, "/v1/agent/schedulers", bytes.NewReader([]byte(tc.request.requestBody))) if tc.request.aclToken != "" { setToken(req, tokens[tc.request.aclToken]) } @@ -1931,5 +1995,55 @@ func schedulerWorkerTest_parseError(t *testing.T, isACLEnabled bool, tc schedule } require.Equal(t, testExpect.expectedResponseCode, codedError.Code()) - require.Equal(t, testExpect.expectedResponse, codedError.Error()) + // this is a relaxed test to allow us to not have to create a case + // for concatenated error strings. + require.Contains(t, codedError.Error(), testExpect.expectedResponse) +} + +func TestHTTP_AgentSchedulerWorkerInfoRequest_Client(t *testing.T) { + verbs := []string{"GET", "POST", "PUT"} + path := "schedulers" + + for _, verb := range verbs { + t.Run(verb, func(t *testing.T) { + httpTest(t, nil, func(s *TestAgent) { + s.Agent.server = nil + req, err := http.NewRequest(verb, fmt.Sprintf("/v1/agent/%v", path), nil) + require.Nil(t, err) + respW := httptest.NewRecorder() + + _, err = s.Server.AgentSchedulerWorkerInfoRequest(respW, req) + + require.Error(t, err) + codedErr, ok := err.(HTTPCodedError) + require.True(t, ok, "expected a HTTPCodedError") + require.Equal(t, http.StatusBadRequest, codedErr.Code()) + require.Equal(t, ErrServerOnly, codedErr.Error()) + }) + }) + } +} + +func TestHTTP_AgentSchedulerWorkerConfigRequest_Client(t *testing.T) { + verbs := []string{"GET", "POST", "PUT"} + path := "schedulers/config" + + for _, verb := range verbs { + t.Run(verb, func(t *testing.T) { + httpTest(t, nil, func(s *TestAgent) { + s.Agent.server = nil + req, err := http.NewRequest(verb, fmt.Sprintf("/v1/agent/%v", path), nil) + require.Nil(t, err) + respW := httptest.NewRecorder() + + _, err = s.Server.AgentSchedulerWorkerInfoRequest(respW, req) + + require.Error(t, err) + codedErr, ok := err.(HTTPCodedError) + require.True(t, ok, "expected a HTTPCodedError") + require.Equal(t, http.StatusBadRequest, codedErr.Code()) + require.Equal(t, ErrServerOnly, codedErr.Error()) + }) + }) + } } diff --git a/command/agent/http.go b/command/agent/http.go index db657991296..a3e682902b8 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -35,6 +35,10 @@ const ( // endpoint ErrEntOnly = "Nomad Enterprise only endpoint" + // ErrServerOnly is the error text returned if accessing a server only + // endpoint + ErrServerOnly = "Server only endpoint" + // ContextKeyReqID is a unique ID for a given request ContextKeyReqID = "requestID" @@ -293,8 +297,8 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/agent/members", s.wrap(s.AgentMembersRequest)) s.mux.HandleFunc("/v1/agent/force-leave", s.wrap(s.AgentForceLeaveRequest)) s.mux.HandleFunc("/v1/agent/servers", s.wrap(s.AgentServersRequest)) - s.mux.HandleFunc("/v1/agent/workers", s.wrap(s.AgentSchedulerWorkerInfoRequest)) - s.mux.HandleFunc("/v1/agent/workers/config", s.wrap(s.AgentSchedulerWorkerConfigRequest)) + s.mux.HandleFunc("/v1/agent/schedulers", s.wrap(s.AgentSchedulerWorkerInfoRequest)) + s.mux.HandleFunc("/v1/agent/schedulers/config", s.wrap(s.AgentSchedulerWorkerConfigRequest)) s.mux.HandleFunc("/v1/agent/keyring/", s.wrap(s.KeyringOperationRequest)) s.mux.HandleFunc("/v1/agent/health", s.wrap(s.HealthRequest)) s.mux.HandleFunc("/v1/agent/host", s.wrap(s.AgentHostRequest)) From 1dc9f9689a295b3ac9ff8086fe60535586b77459 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Tue, 21 Dec 2021 18:00:33 -0500 Subject: [PATCH 22/30] Adding docs for scheduler worker api --- website/content/api-docs/agent.mdx | 201 +++++++++++++++++++++++++++++ 1 file changed, 201 insertions(+) diff --git a/website/content/api-docs/agent.mdx b/website/content/api-docs/agent.mdx index e62a451054b..f24be64c872 100644 --- a/website/content/api-docs/agent.mdx +++ b/website/content/api-docs/agent.mdx @@ -725,3 +725,204 @@ $ curl -O -J \ go tool trace trace ``` + + +## Fetch all scheduler worker's status + +The `/agent/schedulers` endpoint allow Nomad operators to inspect the state of +a Nomad server agent's scheduler workers. + +| Method | Path | Produces | +| ------ | ------------------- | ------------------ | +| `GET` | `/agent/schedulers` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api-docs#blocking-queries) and +[required ACLs](/api-docs#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `agent:read` | + +### Parameters + +This endpoint accepts no additional parameters. + +### Sample Request + +```shell-session +$ curl \ + https://localhost:4646/v1/agent/schedulers +``` + +### Sample Response + +```json +{ + "schedulers": [ + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "5669d6fa-0def-7369-6558-a47c35fdc675", + "started": "2021-12-21T19:25:00.911883Z", + "status": "Paused", + "workload_status": "Paused" + }, + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "c919709d-6d14-66bf-b425-80b8167a267e", + "started": "2021-12-21T19:25:00.91189Z", + "status": "Paused", + "workload_status": "Paused" + }, + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "f5edb69a-6122-be8f-b32a-23cd8511dba5", + "started": "2021-12-21T19:25:00.911961Z", + "status": "Paused", + "workload_status": "Paused" + }, + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "458816ae-83cf-0710-d8d4-35d2ad2e42d7", + "started": "2021-12-21T19:25:00.912119Z", + "status": "Started", + "workload_status": "WaitingToDequeue" + } + ], + "server_id": "server1.global" +} +``` + +## Read scheduler worker configuration + +This endpoint returns data about the agent's scheduler configuration from +the perspective of the agent. This is only applicable for servers. + +| Method | Path | Produces | +| ------ | -------------------------- | ------------------ | +| `GET` | `/agent/schedulers/config` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api-docs#blocking-queries) and +[required ACLs](/api-docs#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `agent:read` | + +### Parameters + +This endpoint accepts no additional parameters. + +### Sample Request + +```shell-session +$ curl \ + --request PUT \ + --data @payload.json \ + https://localhost:4646/v1/jobs +``` + +### Sample Response + +```json +{ + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "num_schedulers": 8, + "server_id": "cv-macbook.localdomain.global" +} +``` + +## Update scheduler worker configuration + +This allows a Nomad operator to modify the server's running scheduler +configuration, which will remain in effect until another update or until the +node is restarted. For durable changes to this value, set the corresponding +values—[`num_schedulers`][] and [`enabled_schedulers`][]—in the node's +configuration file. The response contains the configuration after attempting +to apply the provided values. This is only applicable for servers. + +| Method | Path | Produces | +| ------ | -------------------------- | ------------------ | +| `PUT` | `/agent/schedulers/config` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api-docs#blocking-queries) and +[required ACLs](/api-docs#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------- | +| `NO` | `agent:write` | + +### Sample Payload + +```json +{ + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "num_schedulers": 12 +} +``` + +### Sample Request + +```shell-session +$ curl \ + --request PUT \ + --data @payload.json \ + https://localhost:4646/v1/jobs +``` + +### Sample Response + +```json +{ + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "num_schedulers": 12, + "server_id": "server1.global" +} +``` + +[`enabled_schedulers`]: /docs/configuration/server#enabled_schedulers +[`num_schedulers`]: /docs/configuration/server#num_schedulers From 0417332926a077d21dbf4ebe545040e132a4e652 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Wed, 22 Dec 2021 13:43:53 -0500 Subject: [PATCH 23/30] Adding API test for bad worker info --- api/agent.go | 1 - api/agent_test.go | 17 +++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/api/agent.go b/api/agent.go index 2ccd56846b2..8e8af7f9525 100644 --- a/api/agent.go +++ b/api/agent.go @@ -528,7 +528,6 @@ type AgentSchedulerWorkerConfigResponse struct { func (a *Agent) GetSchedulerWorkersInfo() (*AgentSchedulerWorkersInfo, error) { var out *AgentSchedulerWorkersInfo - // Query the self endpoint on the agent _, err := a.client.query("/v1/agent/schedulers", &out, nil) if err != nil { return nil, err diff --git a/api/agent_test.go b/api/agent_test.go index 238be8def41..6569b07faa9 100644 --- a/api/agent_test.go +++ b/api/agent_test.go @@ -2,6 +2,7 @@ package api import ( "fmt" + "net/http" "reflect" "sort" "strings" @@ -473,6 +474,22 @@ func TestAgent_SchedulerWorkerConfig(t *testing.T) { assert.NotEqual(t, config, resp) } +func TestAgent_SchedulerWorkerConfig_BadRequest(t *testing.T) { + t.Parallel() + + c, s := makeClient(t, nil, nil) + defer s.Stop() + a := c.Agent() + + config, err := a.GetSchedulerWorkerConfig() + require.NoError(t, err) + require.NotNil(t, config) + newConfig := SchedulerWorkerPoolArgs{NumSchedulers: -1, EnabledSchedulers: []string{"_core", "system"}} + _, err = a.SetSchedulerWorkerConfig(newConfig) + require.Error(t, err) + require.Contains(t, err.Error(), fmt.Sprintf("%v (%s)", http.StatusBadRequest, "Invalid request")) +} + func TestAgent_SchedulerWorkersInfo(t *testing.T) { t.Parallel() c, s := makeClient(t, nil, nil) From 420a158a7a0ff180471300b309028dc77d3fdfcc Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Thu, 23 Dec 2021 09:59:22 -0500 Subject: [PATCH 24/30] Add changelog message --- .changelog/11593.txt | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 .changelog/11593.txt diff --git a/.changelog/11593.txt b/.changelog/11593.txt new file mode 100644 index 00000000000..ff072b530ef --- /dev/null +++ b/.changelog/11593.txt @@ -0,0 +1,3 @@ +```release-note:improvement +server: Make num_schedulers and enabled_scheulers hot reloadable; add agent API endpoint to enable dynamic modifications of these values. +``` \ No newline at end of file From fd016de91919b2127ae24d822155d5c878d40ed7 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Thu, 23 Dec 2021 11:00:48 -0500 Subject: [PATCH 25/30] =?UTF-8?q?typo=20in=20changelog=20=F0=9F=A4=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .changelog/11593.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.changelog/11593.txt b/.changelog/11593.txt index ff072b530ef..855ba6d3d48 100644 --- a/.changelog/11593.txt +++ b/.changelog/11593.txt @@ -1,3 +1,3 @@ ```release-note:improvement -server: Make num_schedulers and enabled_scheulers hot reloadable; add agent API endpoint to enable dynamic modifications of these values. -``` \ No newline at end of file +server: Make num_schedulers and enabled_schedulers hot reloadable; add agent API endpoint to enable dynamic modifications of these values. +``` From 167c6a306da79518b950233290cf6212e886baf9 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Mon, 3 Jan 2022 14:54:22 -0500 Subject: [PATCH 26/30] Incorporate API code review feedback --- api/agent.go | 24 +++++++++----- api/agent_test.go | 14 ++++----- command/agent/agent_endpoint.go | 44 ++++++++------------------ command/agent/agent_endpoint_test.go | 47 ++++++++++++++-------------- 4 files changed, 61 insertions(+), 68 deletions(-) diff --git a/api/agent.go b/api/agent.go index 8e8af7f9525..eeab8500584 100644 --- a/api/agent.go +++ b/api/agent.go @@ -485,9 +485,9 @@ type HostDataResponse struct { } // GetSchedulerWorkerConfig returns the targeted agent's worker pool configuration -func (a *Agent) GetSchedulerWorkerConfig() (*SchedulerWorkerPoolArgs, error) { +func (a *Agent) GetSchedulerWorkerConfig(q *QueryOptions) (*SchedulerWorkerPoolArgs, error) { var resp AgentSchedulerWorkerConfigResponse - _, err := a.client.query("/v1/agent/schedulers/config", &resp, nil) + _, err := a.client.query("/v1/agent/schedulers/config", &resp, q) if err != nil { return nil, err } @@ -496,11 +496,11 @@ func (a *Agent) GetSchedulerWorkerConfig() (*SchedulerWorkerPoolArgs, error) { } // SetSchedulerWorkerConfig attempts to update the targeted agent's worker pool configuration -func (a *Agent) SetSchedulerWorkerConfig(args SchedulerWorkerPoolArgs) (*SchedulerWorkerPoolArgs, error) { +func (a *Agent) SetSchedulerWorkerConfig(args SchedulerWorkerPoolArgs, q *WriteOptions) (*SchedulerWorkerPoolArgs, error) { req := AgentSchedulerWorkerConfigRequest(args) var resp AgentSchedulerWorkerConfigResponse - _, err := a.client.write("/v1/agent/schedulers/config", &req, &resp, nil) + _, err := a.client.write("/v1/agent/schedulers/config", &req, &resp, q) if err != nil { return nil, err } @@ -513,22 +513,29 @@ type SchedulerWorkerPoolArgs struct { EnabledSchedulers []string } -// AgentSchedulerWorkerConfig +// AgentSchedulerWorkerConfigRequest is used to provide new scheduler worker configuration +// to a specific Nomad server. EnabledSchedulers must contain at least the `_core` scheduler +// to be valid. type AgentSchedulerWorkerConfigRequest struct { NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } +// AgentSchedulerWorkerConfigResponse contains the Nomad server's current running configuration +// as well as the server's id as a convenience. This can be used to provide starting values for +// creating an AgentSchedulerWorkerConfigRequest to make changes to the running configuration. type AgentSchedulerWorkerConfigResponse struct { ServerID string `json:"server_id"` NumSchedulers int `json:"num_schedulers"` EnabledSchedulers []string `json:"enabled_schedulers"` } -func (a *Agent) GetSchedulerWorkersInfo() (*AgentSchedulerWorkersInfo, error) { +// GetSchedulerWorkersInfo returns the current status of all of the scheduler workers on +// a Nomad server. +func (a *Agent) GetSchedulerWorkersInfo(q *QueryOptions) (*AgentSchedulerWorkersInfo, error) { var out *AgentSchedulerWorkersInfo - _, err := a.client.query("/v1/agent/schedulers", &out, nil) + _, err := a.client.query("/v1/agent/schedulers", &out, q) if err != nil { return nil, err } @@ -536,11 +543,14 @@ func (a *Agent) GetSchedulerWorkersInfo() (*AgentSchedulerWorkersInfo, error) { return out, nil } +// AgentSchedulerWorkersInfo is the response from the scheduler information endpoint containing +// a detailed status of each scheduler worker running on the server. type AgentSchedulerWorkersInfo struct { ServerID string `json:"server_id"` Schedulers []AgentSchedulerWorkerInfo `json:"schedulers"` } +// AgentSchedulerWorkerInfo holds the detailed status information for a single scheduler worker. type AgentSchedulerWorkerInfo struct { ID string `json:"id"` EnabledSchedulers []string `json:"enabled_schedulers"` diff --git a/api/agent_test.go b/api/agent_test.go index 6569b07faa9..cd13bab1892 100644 --- a/api/agent_test.go +++ b/api/agent_test.go @@ -465,11 +465,11 @@ func TestAgent_SchedulerWorkerConfig(t *testing.T) { defer s.Stop() a := c.Agent() - config, err := a.GetSchedulerWorkerConfig() - require.Nil(t, err) + config, err := a.GetSchedulerWorkerConfig(nil) + require.NoError(t, err) require.NotNil(t, config) newConfig := SchedulerWorkerPoolArgs{NumSchedulers: 0, EnabledSchedulers: []string{"_core", "system"}} - resp, err := a.SetSchedulerWorkerConfig(newConfig) + resp, err := a.SetSchedulerWorkerConfig(newConfig, nil) require.NoError(t, err) assert.NotEqual(t, config, resp) } @@ -481,11 +481,11 @@ func TestAgent_SchedulerWorkerConfig_BadRequest(t *testing.T) { defer s.Stop() a := c.Agent() - config, err := a.GetSchedulerWorkerConfig() + config, err := a.GetSchedulerWorkerConfig(nil) require.NoError(t, err) require.NotNil(t, config) newConfig := SchedulerWorkerPoolArgs{NumSchedulers: -1, EnabledSchedulers: []string{"_core", "system"}} - _, err = a.SetSchedulerWorkerConfig(newConfig) + _, err = a.SetSchedulerWorkerConfig(newConfig, nil) require.Error(t, err) require.Contains(t, err.Error(), fmt.Sprintf("%v (%s)", http.StatusBadRequest, "Invalid request")) } @@ -496,8 +496,8 @@ func TestAgent_SchedulerWorkersInfo(t *testing.T) { defer s.Stop() a := c.Agent() - info, err := a.GetSchedulerWorkersInfo() - require.Nil(t, err) + info, err := a.GetSchedulerWorkersInfo(nil) + require.NoError(t, err) require.NotNil(t, info) defaultSchedulers := []string{"batch", "system", "sysbatch", "service", "_core"} for _, worker := range info.Schedulers { diff --git a/command/agent/agent_endpoint.go b/command/agent/agent_endpoint.go index 3796bae912b..1ff4a98feff 100644 --- a/command/agent/agent_endpoint.go +++ b/command/agent/agent_endpoint.go @@ -17,6 +17,7 @@ import ( log "github.com/hashicorp/go-hclog" "github.com/hashicorp/go-msgpack/codec" "github.com/hashicorp/nomad/acl" + "github.com/hashicorp/nomad/api" cstructs "github.com/hashicorp/nomad/client/structs" "github.com/hashicorp/nomad/command/agent/host" "github.com/hashicorp/nomad/command/agent/pprof" @@ -765,13 +766,13 @@ func (s *HTTPServer) AgentSchedulerWorkerInfoRequest(resp http.ResponseWriter, r } schedulersInfo := srv.GetSchedulerWorkersInfo() - response := &agentSchedulerWorkersInfo{ + response := &api.AgentSchedulerWorkersInfo{ ServerID: srv.LocalMember().Name, - Schedulers: make([]agentSchedulerWorkerInfo, len(schedulersInfo)), + Schedulers: make([]api.AgentSchedulerWorkerInfo, len(schedulersInfo)), } for i, workerInfo := range schedulersInfo { - response.Schedulers[i] = agentSchedulerWorkerInfo{ + response.Schedulers[i] = api.AgentSchedulerWorkerInfo{ ID: workerInfo.ID, EnabledSchedulers: make([]string, len(workerInfo.EnabledSchedulers)), Started: workerInfo.Started.UTC().Format(time.RFC3339Nano), @@ -793,16 +794,16 @@ func (s *HTTPServer) AgentSchedulerWorkerConfigRequest(resp http.ResponseWriter, return nil, CodedError(http.StatusBadRequest, ErrServerOnly) } switch req.Method { - case "PUT", "POST": - return s.UpdateScheduleWorkersConfig(resp, req) - case "GET": - return s.GetScheduleWorkersConfig(resp, req) + case http.MethodPut, http.MethodPost: + return s.updateScheduleWorkersConfig(resp, req) + case http.MethodGet: + return s.getScheduleWorkersConfig(resp, req) default: return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod) } } -func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { +func (s *HTTPServer) getScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { return nil, CodedError(http.StatusBadRequest, ErrServerOnly) @@ -819,7 +820,7 @@ func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *htt } config := srv.GetSchedulerWorkerConfig() - response := &agentSchedulerWorkerConfig{ + response := &api.AgentSchedulerWorkerConfigResponse{ ServerID: srv.LocalMember().Name, NumSchedulers: config.NumSchedulers, EnabledSchedulers: config.EnabledSchedulers, @@ -828,7 +829,7 @@ func (s *HTTPServer) GetScheduleWorkersConfig(resp http.ResponseWriter, req *htt return response, nil } -func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { +func (s *HTTPServer) updateScheduleWorkersConfig(resp http.ResponseWriter, req *http.Request) (interface{}, error) { srv := s.agent.Server() if srv == nil { return nil, CodedError(http.StatusBadRequest, ErrServerOnly) @@ -844,7 +845,7 @@ func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req * return nil, CodedError(http.StatusForbidden, structs.ErrPermissionDenied.Error()) } - var args agentSchedulerWorkerConfig + var args api.AgentSchedulerWorkerConfigRequest if err := decodeBody(req, &args); err != nil { return nil, CodedError(http.StatusBadRequest, fmt.Sprintf("Invalid request: %s", err.Error())) @@ -860,7 +861,7 @@ func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req * } reply := srv.SetSchedulerWorkerConfig(newArgs) - response := &agentSchedulerWorkerConfig{ + response := &api.AgentSchedulerWorkerConfigResponse{ ServerID: srv.LocalMember().Name, NumSchedulers: reply.NumSchedulers, EnabledSchedulers: reply.EnabledSchedulers, @@ -868,22 +869,3 @@ func (s *HTTPServer) UpdateScheduleWorkersConfig(resp http.ResponseWriter, req * return response, nil } - -type agentSchedulerWorkerConfig struct { - ServerID string `json:"server_id,omitempty"` - NumSchedulers int `json:"num_schedulers"` - EnabledSchedulers []string `json:"enabled_schedulers"` -} - -type agentSchedulerWorkersInfo struct { - ServerID string `json:"server_id"` - Schedulers []agentSchedulerWorkerInfo `json:"schedulers"` -} - -type agentSchedulerWorkerInfo struct { - ID string `json:"id"` - EnabledSchedulers []string `json:"enabled_schedulers"` - Started string `json:"started"` - Status string `json:"status"` - WorkloadStatus string `json:"workload_status"` -} diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index b2074c5c309..5b23a449f35 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -20,6 +20,7 @@ import ( msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc" "github.com/hashicorp/nomad/acl" + "github.com/hashicorp/nomad/api" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/pool" "github.com/hashicorp/nomad/nomad/mock" @@ -264,7 +265,7 @@ func TestHTTP_AgentMonitor(t *testing.T) { t.Run("invalid log_json parameter", func(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { req, err := http.NewRequest("GET", "/v1/agent/monitor?log_json=no", nil) - require.Nil(t, err) + require.NoError(t, err) resp := newClosableRecorder() // Make the request @@ -277,7 +278,7 @@ func TestHTTP_AgentMonitor(t *testing.T) { t.Run("unknown log_level", func(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { req, err := http.NewRequest("GET", "/v1/agent/monitor?log_level=unknown", nil) - require.Nil(t, err) + require.NoError(t, err) resp := newClosableRecorder() // Make the request @@ -290,7 +291,7 @@ func TestHTTP_AgentMonitor(t *testing.T) { t.Run("check for specific log level", func(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { req, err := http.NewRequest("GET", "/v1/agent/monitor?log_level=warn", nil) - require.Nil(t, err) + require.NoError(t, err) resp := newClosableRecorder() defer resp.Close() @@ -324,7 +325,7 @@ func TestHTTP_AgentMonitor(t *testing.T) { t.Run("plain output", func(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { req, err := http.NewRequest("GET", "/v1/agent/monitor?log_level=debug&plain=true", nil) - require.Nil(t, err) + require.NoError(t, err) resp := newClosableRecorder() defer resp.Close() @@ -358,7 +359,7 @@ func TestHTTP_AgentMonitor(t *testing.T) { t.Run("logs for a specific node", func(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { req, err := http.NewRequest("GET", "/v1/agent/monitor?log_level=warn&node_id="+s.client.NodeID(), nil) - require.Nil(t, err) + require.NoError(t, err) resp := newClosableRecorder() defer resp.Close() @@ -398,7 +399,7 @@ func TestHTTP_AgentMonitor(t *testing.T) { t.Run("logs for a local client with no server running on agent", func(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { req, err := http.NewRequest("GET", "/v1/agent/monitor?log_level=warn", nil) - require.Nil(t, err) + require.NoError(t, err) resp := newClosableRecorder() defer resp.Close() @@ -596,7 +597,7 @@ func TestAgent_PprofRequest(t *testing.T) { } req, err := http.NewRequest("GET", url, nil) - require.Nil(t, err) + require.NoError(t, err) respW := httptest.NewRecorder() resp, err := s.Server.AgentPprofRequest(respW, req) @@ -914,7 +915,7 @@ func TestHTTP_AgentListKeys(t *testing.T) { respW := httptest.NewRecorder() out, err := s.Server.KeyringOperationRequest(respW, req) - require.Nil(t, err) + require.NoError(t, err) kresp := out.(structs.KeyringResponse) require.Len(t, kresp.Keys, 1) }) @@ -1505,8 +1506,8 @@ func schedulerWorkerInfoTest_testCases() []schedulerWorkerAPITest_testCase { } success := schedulerWorkerAPITest_testExpect{ statusCode: http.StatusOK, - response: &agentSchedulerWorkersInfo{ - Schedulers: []agentSchedulerWorkerInfo{ + response: &api.AgentSchedulerWorkersInfo{ + Schedulers: []api.AgentSchedulerWorkerInfo{ { ID: "9b3713e0-6f74-0e1b-3b3e-d94f0c22dbf9", EnabledSchedulers: []string{"_core", "batch"}, @@ -1621,7 +1622,7 @@ func TestHTTP_AgentSchedulerWorkerInfoRequest(t *testing.T) { if testingACLS && tc.request.aclToken != "" { setToken(req, tokens[tc.request.aclToken]) } - require.Nil(t, err) + require.NoError(t, err) respW := httptest.NewRecorder() workerInfoResp, err := s.Server.AgentSchedulerWorkerInfoRequest(respW, req) @@ -1640,11 +1641,11 @@ func TestHTTP_AgentSchedulerWorkerInfoRequest(t *testing.T) { } require.NoError(t, err) - workerInfo, ok := workerInfoResp.(*agentSchedulerWorkersInfo) - require.True(t, ok, "expected an *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) + workerInfo, ok := workerInfoResp.(*api.AgentSchedulerWorkersInfo) + require.True(t, ok, "expected an *AgentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) - expectWorkerInfo, ok := expected.response.(*agentSchedulerWorkersInfo) - require.True(t, ok, "error casting test case to *agentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) + expectWorkerInfo, ok := expected.response.(*api.AgentSchedulerWorkersInfo) + require.True(t, ok, "error casting test case to *AgentSchedulersWorkersInfo. received:%s", reflect.TypeOf(workerInfoResp)) var schedCount int = *s.Config.Server.NumSchedulers require.Equal(t, schedCount, len(workerInfo.Schedulers), "must match num_schedulers") @@ -1702,12 +1703,12 @@ func schedulerWorkerConfigTest_testCases() []scheduleWorkerConfigTest_workerRequ } success1 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, - expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, + expectedResponse: &api.AgentSchedulerWorkerConfigRequest{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, } success2 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, - expectedResponse: &agentSchedulerWorkerConfig{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 9}, + expectedResponse: &api.AgentSchedulerWorkerConfigRequest{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 9}, } return []scheduleWorkerConfigTest_workerRequestTest{ @@ -1896,7 +1897,7 @@ func TestHTTP_AgentSchedulerWorkerConfigRequest_NoACL(t *testing.T) { t.Run(tc.name, func(t *testing.T) { req, err := http.NewRequest(tc.request.verb, "/v1/agent/schedulers/config", bytes.NewReader([]byte(tc.request.requestBody))) - require.Nil(t, err) + require.NoError(t, err) respW := httptest.NewRecorder() workersI, err := s.Server.AgentSchedulerWorkerConfigRequest(respW, req) @@ -1939,7 +1940,7 @@ func TestHTTP_AgentSchedulerWorkerConfigRequest_ACL(t *testing.T) { if tc.request.aclToken != "" { setToken(req, tokens[tc.request.aclToken]) } - require.Nil(t, err) + require.NoError(t, err) respW := httptest.NewRecorder() workersI, err := s.Server.AgentSchedulerWorkerConfigRequest(respW, req) @@ -1968,10 +1969,10 @@ func schedulerWorkerTest_parseSuccess(t *testing.T, isACLEnabled bool, tc schedu } // test into the response when we expect an okay - tcConfig, ok := testExpect.expectedResponse.(*agentSchedulerWorkerConfig) + tcConfig, ok := testExpect.expectedResponse.(*api.AgentSchedulerWorkerConfigResponse) require.True(t, ok, "expected response malformed - this is an issue with a test case.") - workersConfig, ok := workersI.(*agentSchedulerWorkerConfig) + workersConfig, ok := workersI.(*api.AgentSchedulerWorkerConfigResponse) require.True(t, ok, "response can not cast to an agentSchedulerWorkerConfig") require.NotNil(t, workersConfig) @@ -2009,7 +2010,7 @@ func TestHTTP_AgentSchedulerWorkerInfoRequest_Client(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { s.Agent.server = nil req, err := http.NewRequest(verb, fmt.Sprintf("/v1/agent/%v", path), nil) - require.Nil(t, err) + require.NoError(t, err) respW := httptest.NewRecorder() _, err = s.Server.AgentSchedulerWorkerInfoRequest(respW, req) @@ -2033,7 +2034,7 @@ func TestHTTP_AgentSchedulerWorkerConfigRequest_Client(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { s.Agent.server = nil req, err := http.NewRequest(verb, fmt.Sprintf("/v1/agent/%v", path), nil) - require.Nil(t, err) + require.NoError(t, err) respW := httptest.NewRecorder() _, err = s.Server.AgentSchedulerWorkerInfoRequest(respW, req) From f4f610b229bf5099d57adb607b343eba29726b2c Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Tue, 4 Jan 2022 10:28:10 -0500 Subject: [PATCH 27/30] Incorporate api-docs feedback --- website/content/api-docs/agent.mdx | 164 ++++++++++++++--------------- 1 file changed, 82 insertions(+), 82 deletions(-) diff --git a/website/content/api-docs/agent.mdx b/website/content/api-docs/agent.mdx index f24be64c872..db51a4a7f0e 100644 --- a/website/content/api-docs/agent.mdx +++ b/website/content/api-docs/agent.mdx @@ -726,7 +726,6 @@ $ curl -O -J \ go tool trace trace ``` - ## Fetch all scheduler worker's status The `/agent/schedulers` endpoint allow Nomad operators to inspect the state of @@ -759,62 +758,63 @@ $ curl \ ```json { - "schedulers": [ - { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "id": "5669d6fa-0def-7369-6558-a47c35fdc675", - "started": "2021-12-21T19:25:00.911883Z", - "status": "Paused", - "workload_status": "Paused" - }, - { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "id": "c919709d-6d14-66bf-b425-80b8167a267e", - "started": "2021-12-21T19:25:00.91189Z", - "status": "Paused", - "workload_status": "Paused" - }, - { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "id": "f5edb69a-6122-be8f-b32a-23cd8511dba5", - "started": "2021-12-21T19:25:00.911961Z", - "status": "Paused", - "workload_status": "Paused" - }, - { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "id": "458816ae-83cf-0710-d8d4-35d2ad2e42d7", - "started": "2021-12-21T19:25:00.912119Z", - "status": "Started", - "workload_status": "WaitingToDequeue" - } - ], - "server_id": "server1.global" + "schedulers": [ + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "5669d6fa-0def-7369-6558-a47c35fdc675", + "started": "2021-12-21T19:25:00.911883Z", + "status": "Paused", + "workload_status": "Paused" + }, + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "c919709d-6d14-66bf-b425-80b8167a267e", + "started": "2021-12-21T19:25:00.91189Z", + "status": "Paused", + "workload_status": "Paused" + }, + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "f5edb69a-6122-be8f-b32a-23cd8511dba5", + "started": "2021-12-21T19:25:00.911961Z", + "status": "Paused", + "workload_status": "Paused" + }, + { + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "id": "458816ae-83cf-0710-d8d4-35d2ad2e42d7", + "started": "2021-12-21T19:25:00.912119Z", + "status": "Started", + "workload_status": "WaitingToDequeue" + } + ], + "server_id": "server1.global" } + ``` ## Read scheduler worker configuration @@ -851,15 +851,15 @@ $ curl \ ```json { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "num_schedulers": 8, - "server_id": "cv-macbook.localdomain.global" + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "num_schedulers": 8, + "server_id": "server1.global" } ``` @@ -888,14 +888,14 @@ The table below shows this endpoint's support for ```json { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "num_schedulers": 12 + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "num_schedulers": 12 } ``` @@ -912,15 +912,15 @@ $ curl \ ```json { - "enabled_schedulers": [ - "service", - "batch", - "system", - "sysbatch", - "_core" - ], - "num_schedulers": 12, - "server_id": "server1.global" + "enabled_schedulers": [ + "service", + "batch", + "system", + "sysbatch", + "_core" + ], + "num_schedulers": 12, + "server_id": "server1.global" } ``` From 689fa77f9453f2cd70663aa87466f6e4a88aefa5 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Tue, 4 Jan 2022 15:21:51 -0500 Subject: [PATCH 28/30] Updates to worker/leader code from code review --- nomad/leader_test.go | 23 +++++----- nomad/server.go | 1 + nomad/worker.go | 73 +++++++++++++++++++++---------- nomad/worker_test.go | 102 +++++++++++++++++++++++++++---------------- 4 files changed, 126 insertions(+), 73 deletions(-) diff --git a/nomad/leader_test.go b/nomad/leader_test.go index 3f5a1add5b0..07f854d09e3 100644 --- a/nomad/leader_test.go +++ b/nomad/leader_test.go @@ -1328,21 +1328,20 @@ func TestLeader_PausingWorkers(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) require.Len(t, s1.workers, 12) - pausedWorkers := func() int { - c := 0 - for _, w := range s1.workers { - if w.IsPaused() { - c++ - } - } - return c - } - // this satisfies the require.Eventually test interface checkPaused := func(count int) func() bool { return func() bool { - workers := pausedWorkers() - return workers == count + pausedWorkers := func() int { + c := 0 + for _, w := range s1.workers { + if w.IsPaused() { + c++ + } + } + return c + } + + return pausedWorkers() == count } } diff --git a/nomad/server.go b/nomad/server.go index 905389a912d..533b2ca01f9 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -811,6 +811,7 @@ func (s *Server) Reload(newConfig *Config) error { s.EnterpriseState.ReloadLicense(newConfig) } + // Because this is a new configuration, we extract the worker pool arguments without acquiring a lock workerPoolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(newConfig) if reload, newVals := shouldReloadSchedulers(s, workerPoolArgs); reload { if newVals.IsValid() { diff --git a/nomad/worker.go b/nomad/worker.go index cfc56bcf8fd..a504b745be7 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -115,13 +115,13 @@ type Worker struct { // NewWorker starts a new scheduler worker associated with the given server func NewWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { - w, _ := newWorker(ctx, srv, args) + w := newWorker(ctx, srv, args) w.Start() return w, nil } // _newWorker creates a worker without calling its Start func. This is useful for testing. -func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) (*Worker, error) { +func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) *Worker { w := &Worker{ id: uuid.Generate(), srv: srv, @@ -135,7 +135,7 @@ func newWorker(ctx context.Context, srv *Server, args SchedulerWorkerPoolArgs) ( w.pauseCond = sync.NewCond(&w.pauseLock) w.ctx, w.cancelFn = context.WithCancel(ctx) - return w, nil + return w } // ID returns a string ID for the worker. @@ -153,8 +153,10 @@ func (w *Worker) Start() { // Pause transitions a worker to the pausing state. Check // to see if it paused using IsPaused() func (w *Worker) Pause() { - w.setStatus(WorkerPausing) - w.setPauseFlag(true) + if w.isPausable() { + w.setStatus(WorkerPausing) + w.setPauseFlag(true) + } } // Resume transitions a worker to the resuming state. Check @@ -189,6 +191,17 @@ func (w *Worker) IsStopped() bool { return w.GetStatus() == WorkerStopped } +func (w *Worker) isPausable() bool { + w.statusLock.RLock() + defer w.statusLock.RUnlock() + switch w.status { + case WorkerPausing, WorkerPaused, WorkerStopping, WorkerStopped: + return false + default: + return true + } +} + // GetStatus returns the status of the Worker func (w *Worker) GetStatus() WorkerStatus { w.statusLock.RLock() @@ -196,11 +209,27 @@ func (w *Worker) GetStatus() WorkerStatus { return w.status } +// setStatuses is used internally to the worker to update the +// status of the worker and workload at one time, since some +// transitions need to update both values using the same lock. +func (w *Worker) setStatuses(newWorkerStatus WorkerStatus, newWorkloadStatus SchedulerWorkerStatus) { + w.statusLock.Lock() + defer w.statusLock.Unlock() + w.setWorkerStatusLocked(newWorkerStatus) + w.setWorkloadStatusLocked(newWorkloadStatus) +} + // setStatus is used internally to the worker to update the -// status of the worker based on calls to the Worker API. +// status of the worker based on calls to the Worker API. For +// atomically updating the scheduler status and the workload +// status, use `setStatuses`. func (w *Worker) setStatus(newStatus WorkerStatus) { w.statusLock.Lock() defer w.statusLock.Unlock() + w.setWorkerStatusLocked(newStatus) +} + +func (w *Worker) setWorkerStatusLocked(newStatus WorkerStatus) { if newStatus == w.status { return } @@ -220,6 +249,10 @@ func (w *Worker) GetWorkloadStatus() SchedulerWorkerStatus { func (w *Worker) setWorkloadStatus(newStatus SchedulerWorkerStatus) { w.statusLock.Lock() defer w.statusLock.Unlock() + w.setWorkloadStatusLocked(newStatus) +} + +func (w *Worker) setWorkloadStatusLocked(newStatus SchedulerWorkerStatus) { if newStatus == w.workloadStatus { return } @@ -283,8 +316,9 @@ func (w *Worker) setPauseFlag(pause bool) { // values. func (w *Worker) maybeWait() { w.pauseLock.Lock() + defer w.pauseLock.Unlock() + if !w.pauseFlag { - w.pauseLock.Unlock() return } @@ -300,13 +334,16 @@ func (w *Worker) maybeWait() { w.pauseCond.Wait() } - w.pauseLock.Unlock() - w.statusLock.Lock() - w.status = WorkerStarted - w.workloadStatus = originalWorkloadStatus + w.logger.Trace("changed workload status", "from", w.workloadStatus, "to", originalWorkloadStatus) - w.logger.Trace("changed worker status", "from", WorkerPaused, "to", WorkerStarted) + w.workloadStatus = originalWorkloadStatus + + // only reset the worker status if the worker is not resuming to stop the paused workload. + if w.status != WorkerStopping { + w.logger.Trace("changed worker status", "from", w.status, "to", WorkerStarted) + w.status = WorkerStarted + } w.statusLock.Unlock() } @@ -350,8 +387,7 @@ func (w *Worker) run() { w.setWorkloadStatus(WorkloadStopped) w.markStopped() }() - w.setStatus(WorkerStarted) - w.setWorkloadStatus(WorkloadRunning) + w.setStatuses(WorkerStarted, WorkloadRunning) w.logger.Debug("running") for { // Check to see if the context has been cancelled. Server shutdown and Shutdown() @@ -437,8 +473,6 @@ REQ: limit = backoffSchedulerVersionMismatch } - // !: while I don't know _what_ to do here, shutting down the worker seems like it - // could end with a server with no running scheduler workers. if w.backoffErr(base, limit) { return nil, "", 0, true } @@ -803,10 +837,3 @@ func (w *Worker) backoffErr(base, limit time.Duration) bool { func (w *Worker) backoffReset() { w.failures = 0 } - -// Test helpers -// -func (w *Worker) _start(inFunc func(w *Worker)) { - w.setStatus(WorkerStarting) - go inFunc(w) -} diff --git a/nomad/worker_test.go b/nomad/worker_test.go index 08e5cb993f7..334790e5f9d 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -2,7 +2,6 @@ package nomad import ( "context" - "encoding/json" "fmt" "reflect" "sync" @@ -126,7 +125,7 @@ func TestWorker_dequeueEvaluation_SerialJobs(t *testing.T) { // Create a worker poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) // Attempt dequeue eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond) @@ -187,7 +186,7 @@ func TestWorker_dequeueEvaluation_paused(t *testing.T) { // Create a worker poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.pauseCond = sync.NewCond(&w.pauseLock) // PAUSE the worker @@ -232,7 +231,7 @@ func TestWorker_dequeueEvaluation_shutdown(t *testing.T) { // Create a worker poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) go func() { time.Sleep(10 * time.Millisecond) @@ -261,9 +260,8 @@ func TestWorker_Shutdown(t *testing.T) { defer cleanupS1() testutil.WaitForLeader(t, s1.RPC) - // Create a worker; since this tests the shutdown, you need the cancelFn too. poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) go func() { time.Sleep(10 * time.Millisecond) @@ -319,7 +317,7 @@ func TestWorker_sendAck(t *testing.T) { // Create a worker poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) // Attempt dequeue eval, token, _, _ := w.dequeueEvaluation(10 * time.Millisecond) @@ -375,7 +373,7 @@ func TestWorker_waitForIndex(t *testing.T) { // Wait for a future index poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) snap, err := w.snapshotMinIndex(index+1, time.Second) require.NoError(t, err) require.NotNil(t, snap) @@ -402,7 +400,7 @@ func TestWorker_invokeScheduler(t *testing.T) { defer cleanupS1() poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) eval := mock.Eval() eval.Type = "noop" @@ -456,7 +454,7 @@ func TestWorker_SubmitPlan(t *testing.T) { // Attempt to submit a plan poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token result, state, err := w.SubmitPlan(plan) @@ -521,7 +519,7 @@ func TestWorker_SubmitPlanNormalizedAllocations(t *testing.T) { // Attempt to submit a plan poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.SubmitPlan(plan) assert.Equal(t, &structs.Allocation{ @@ -579,7 +577,7 @@ func TestWorker_SubmitPlan_MissingNodeRefresh(t *testing.T) { // Attempt to submit a plan poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token result, state, err := w.SubmitPlan(plan) @@ -639,7 +637,7 @@ func TestWorker_UpdateEval(t *testing.T) { // Attempt to update eval poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token err = w.UpdateEval(eval2) @@ -691,7 +689,7 @@ func TestWorker_CreateEval(t *testing.T) { // Attempt to create eval poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token err = w.CreateEval(eval2) @@ -756,7 +754,7 @@ func TestWorker_ReblockEval(t *testing.T) { // Attempt to reblock eval poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() - w, _ := newWorker(s1.shutdownCtx, s1, poolArgs) + w := newWorker(s1.shutdownCtx, s1, poolArgs) w.evalToken = token err = w.ReblockEval(eval2) @@ -818,30 +816,13 @@ func TestWorker_Info(t *testing.T) { poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy() // Create a worker - w, err := newWorker(s1.shutdownCtx, s1, poolArgs) - require.NoError(t, err) + w := newWorker(s1.shutdownCtx, s1, poolArgs) require.Equal(t, WorkerStarting, w.GetStatus()) workerInfo := w.Info() require.Equal(t, WorkerStarting.String(), workerInfo.Status) } -func TestWorker_WorkerInfo_String(t *testing.T) { - t.Parallel() - startTime := time.Date(2009, time.November, 10, 23, 0, 0, 0, time.UTC) - w := &Worker{ - id: "uuid", - start: startTime, - status: WorkerStarted, - workloadStatus: WorkloadBackoff, - enabledSchedulers: []string{structs.JobTypeCore, structs.JobTypeBatch, structs.JobTypeSystem}, - } - _, err := json.Marshal(w) - require.NoError(t, err) - - require.Equal(t, `{"id":"uuid","enabled_schedulers":["_core","batch","system"],"started":"2009-11-10T23:00:00Z","status":"Started","workload_status":"Backoff"}`, fmt.Sprint(w.Info())) -} - const ( longWait = 100 * time.Millisecond tinyWait = 10 * time.Millisecond @@ -857,9 +838,7 @@ func TestWorker_SetPause(t *testing.T) { args := SchedulerWorkerPoolArgs{ EnabledSchedulers: []string{structs.JobTypeCore, structs.JobTypeBatch, structs.JobTypeSystem}, } - w, err := newWorker(context.Background(), srv, args) - require.NoError(t, err) - + w := newWorker(context.Background(), srv, args) w._start(testWorkload) require.Eventually(t, w.IsStarted, longWait, tinyWait, "should have started") @@ -869,6 +848,12 @@ func TestWorker_SetPause(t *testing.T) { }() require.Eventually(t, w.IsPaused, longWait, tinyWait, "should have paused") + go func() { + time.Sleep(tinyWait) + w.Pause() + }() + require.Eventually(t, w.IsPaused, longWait, tinyWait, "pausing a paused should be okay") + go func() { time.Sleep(tinyWait) w.Resume() @@ -882,10 +867,51 @@ func TestWorker_SetPause(t *testing.T) { require.Eventually(t, w.IsStopped, longWait, tinyWait, "should have shutdown") } +func TestWorker_SetPause_OutOfOrderEvents(t *testing.T) { + t.Parallel() + logger := testlog.HCLogger(t) + srv := &Server{ + logger: logger, + shutdownCtx: context.Background(), + } + args := SchedulerWorkerPoolArgs{ + EnabledSchedulers: []string{structs.JobTypeCore, structs.JobTypeBatch, structs.JobTypeSystem}, + } + w := newWorker(context.Background(), srv, args) + w._start(testWorkload) + require.Eventually(t, w.IsStarted, longWait, tinyWait, "should have started") + + go func() { + time.Sleep(tinyWait) + w.Pause() + }() + require.Eventually(t, w.IsPaused, longWait, tinyWait, "should have paused") + + go func() { + time.Sleep(tinyWait) + w.Stop() + }() + require.Eventually(t, w.IsStopped, longWait, tinyWait, "stop from pause should have shutdown") + + go func() { + time.Sleep(tinyWait) + w.Pause() + }() + require.Eventually(t, w.IsStopped, longWait, tinyWait, "pausing a stopped should stay stopped") + +} + +// _start is a test helper function used to start a worker with an alternate workload +func (w *Worker) _start(inFunc func(w *Worker)) { + w.setStatus(WorkerStarting) + go inFunc(w) +} + +// testWorkload is a very simple function that performs the same status updating behaviors that the +// real workload does. func testWorkload(w *Worker) { defer w.markStopped() - w.setStatus(WorkerStarted) - w.setWorkloadStatus(WorkloadRunning) + w.setStatuses(WorkerStarted, WorkloadRunning) w.logger.Debug("testWorkload running") for { // ensure state variables are happy after resuming. From 982c3970d8abae8ca9c93f5fa38d42799e7b2c70 Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Wed, 5 Jan 2022 10:26:00 -0500 Subject: [PATCH 29/30] Fix test response type --- command/agent/agent_endpoint_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/command/agent/agent_endpoint_test.go b/command/agent/agent_endpoint_test.go index 5b23a449f35..efad48880f7 100644 --- a/command/agent/agent_endpoint_test.go +++ b/command/agent/agent_endpoint_test.go @@ -1703,12 +1703,12 @@ func schedulerWorkerConfigTest_testCases() []scheduleWorkerConfigTest_workerRequ } success1 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, - expectedResponse: &api.AgentSchedulerWorkerConfigRequest{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, + expectedResponse: &api.AgentSchedulerWorkerConfigResponse{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 8}, } success2 := schedulerWorkerConfigTest_testExpect{ expectedResponseCode: http.StatusOK, - expectedResponse: &api.AgentSchedulerWorkerConfigRequest{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 9}, + expectedResponse: &api.AgentSchedulerWorkerConfigResponse{EnabledSchedulers: []string{"_core", "batch"}, NumSchedulers: 9}, } return []scheduleWorkerConfigTest_workerRequestTest{ From 758195709523bf426e2169266d55684cf8c0f9fc Mon Sep 17 00:00:00 2001 From: Charlie Voiselle <464492+angrycub@users.noreply.github.com> Date: Thu, 6 Jan 2022 11:03:45 -0500 Subject: [PATCH 30/30] Set both statuses in markStopped so they are atomic --- nomad/worker.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/nomad/worker.go b/nomad/worker.go index a504b745be7..10e8cae0614 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -361,11 +361,11 @@ func (w *Worker) shutdown() { } } -// markStopped is used to mark the worker as stopped and should be called in a +// markStopped is used to mark the worker and workload as stopped. It should be called in a // defer immediately upon entering the run() function. func (w *Worker) markStopped() { + w.setStatuses(WorkerStopped, WorkloadStopped) w.logger.Debug("stopped") - w.setStatus(WorkerStopped) } func (w *Worker) workerShuttingDown() bool { @@ -384,7 +384,6 @@ func (w *Worker) workerShuttingDown() bool { // run is the long-lived goroutine which is used to run the worker func (w *Worker) run() { defer func() { - w.setWorkloadStatus(WorkloadStopped) w.markStopped() }() w.setStatuses(WorkerStarted, WorkloadRunning)