2023-04-10 15:36:59 +00:00
|
|
|
// Copyright (c) HashiCorp, Inc.
|
|
|
|
// SPDX-License-Identifier: MPL-2.0
|
|
|
|
|
2015-07-28 22:15:42 +00:00
|
|
|
package nomad
|
|
|
|
|
2015-07-29 00:02:54 +00:00
|
|
|
import (
|
2022-01-06 16:56:13 +00:00
|
|
|
"context"
|
core: backoff considerably when worker is behind raft (#15523)
Upon dequeuing an evaluation workers snapshot their state store at the
eval's wait index or later. This ensures we process an eval at a point
in time after it was created or updated. Processing an eval on an old
snapshot could cause any number of problems such as:
1. Since job registration atomically updates an eval and job in a single
raft entry, scheduling against indexes before that may not have the
eval's job or may have an older version.
2. The older the scheduler's snapshot, the higher the likelihood
something has changed in the cluster state which will cause the plan
applier to reject the scheduler's plan. This could waste work or
even cause eval's to be failed needlessly.
However, the workers run in parallel with a new server pulling the
cluster state from a peer. During this time, which may be many minutes
long, the state store is likely far behind the minimum index required
to process evaluations.
This PR addresses this by adding an additional long backoff period after
an eval is nacked. If the scheduler's indexes catches up within the
additional backoff, it will unblock early to dequeue the next eval.
When the server shuts down we'll get a `context.Canceled` error from the state
store method. We need to bubble this error up so that other callers can detect
it. Handle this case separately when waiting after dequeue so that we can warn
on shutdown instead of throwing an ambiguous error message with just the text
"canceled."
While there may be more precise ways to block scheduling until the
server catches up, this approach adds little risk and covers additional
cases where a server may be temporarily behind due to a spike in load or
a saturated network.
For testing, we make the `raftSyncLimit` into a parameter on the worker's `run` method
so that we can run backoff tests without waiting 30+ seconds. We haven't followed thru
and made all the worker globals into worker parameters, because there isn't much
use outside of testing, but we can consider that in the future.
Co-authored-by: Tim Gross <tgross@hashicorp.com>
2023-01-24 13:56:35 +00:00
|
|
|
"errors"
|
2019-05-17 21:37:42 +00:00
|
|
|
"fmt"
|
2015-07-29 00:02:54 +00:00
|
|
|
"reflect"
|
2015-08-23 17:52:31 +00:00
|
|
|
"sync"
|
2015-07-29 00:02:54 +00:00
|
|
|
"testing"
|
|
|
|
"time"
|
2015-07-28 22:15:42 +00:00
|
|
|
|
2018-09-15 23:23:13 +00:00
|
|
|
log "github.com/hashicorp/go-hclog"
|
2019-03-05 21:41:41 +00:00
|
|
|
"github.com/hashicorp/go-memdb"
|
2022-03-15 12:42:43 +00:00
|
|
|
"github.com/hashicorp/nomad/ci"
|
core: backoff considerably when worker is behind raft (#15523)
Upon dequeuing an evaluation workers snapshot their state store at the
eval's wait index or later. This ensures we process an eval at a point
in time after it was created or updated. Processing an eval on an old
snapshot could cause any number of problems such as:
1. Since job registration atomically updates an eval and job in a single
raft entry, scheduling against indexes before that may not have the
eval's job or may have an older version.
2. The older the scheduler's snapshot, the higher the likelihood
something has changed in the cluster state which will cause the plan
applier to reject the scheduler's plan. This could waste work or
even cause eval's to be failed needlessly.
However, the workers run in parallel with a new server pulling the
cluster state from a peer. During this time, which may be many minutes
long, the state store is likely far behind the minimum index required
to process evaluations.
This PR addresses this by adding an additional long backoff period after
an eval is nacked. If the scheduler's indexes catches up within the
additional backoff, it will unblock early to dequeue the next eval.
When the server shuts down we'll get a `context.Canceled` error from the state
store method. We need to bubble this error up so that other callers can detect
it. Handle this case separately when waiting after dequeue so that we can warn
on shutdown instead of throwing an ambiguous error message with just the text
"canceled."
While there may be more precise ways to block scheduling until the
server catches up, this approach adds little risk and covers additional
cases where a server may be temporarily behind due to a spike in load or
a saturated network.
For testing, we make the `raftSyncLimit` into a parameter on the worker's `run` method
so that we can run backoff tests without waiting 30+ seconds. We haven't followed thru
and made all the worker globals into worker parameters, because there isn't much
use outside of testing, but we can consider that in the future.
Co-authored-by: Tim Gross <tgross@hashicorp.com>
2023-01-24 13:56:35 +00:00
|
|
|
"github.com/hashicorp/nomad/helper"
|
2022-11-16 21:10:11 +00:00
|
|
|
"github.com/shoenig/test/must"
|
2019-03-12 21:25:14 +00:00
|
|
|
"github.com/stretchr/testify/require"
|
2018-09-15 23:23:13 +00:00
|
|
|
|
2022-01-06 16:56:13 +00:00
|
|
|
"github.com/hashicorp/nomad/helper/testlog"
|
2017-09-29 16:58:48 +00:00
|
|
|
"github.com/hashicorp/nomad/helper/uuid"
|
2015-08-11 21:27:14 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/mock"
|
2015-07-29 00:02:54 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/structs"
|
2015-07-29 00:20:06 +00:00
|
|
|
"github.com/hashicorp/nomad/scheduler"
|
2015-07-29 00:02:54 +00:00
|
|
|
"github.com/hashicorp/nomad/testutil"
|
2019-03-05 21:41:41 +00:00
|
|
|
"github.com/stretchr/testify/assert"
|
2015-07-29 00:02:54 +00:00
|
|
|
)
|
|
|
|
|
2015-07-29 00:20:06 +00:00
|
|
|
type NoopScheduler struct {
|
2022-01-15 01:09:14 +00:00
|
|
|
state scheduler.State
|
|
|
|
planner scheduler.Planner
|
|
|
|
eval *structs.Evaluation
|
|
|
|
eventsCh chan<- interface{}
|
|
|
|
err error
|
2015-07-29 00:20:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (n *NoopScheduler) Process(eval *structs.Evaluation) error {
|
|
|
|
if n.state == nil {
|
|
|
|
panic("missing state")
|
|
|
|
}
|
|
|
|
if n.planner == nil {
|
|
|
|
panic("missing planner")
|
|
|
|
}
|
|
|
|
n.eval = eval
|
|
|
|
return n.err
|
|
|
|
}
|
|
|
|
|
|
|
|
func init() {
|
2022-01-15 01:09:14 +00:00
|
|
|
scheduler.BuiltinSchedulers["noop"] = func(logger log.Logger, eventsCh chan<- interface{}, s scheduler.State, p scheduler.Planner) scheduler.Scheduler {
|
2015-07-29 00:20:06 +00:00
|
|
|
n := &NoopScheduler{
|
2019-03-08 12:48:12 +00:00
|
|
|
state: s,
|
|
|
|
planner: p,
|
2015-07-29 00:20:06 +00:00
|
|
|
}
|
|
|
|
return n
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-01-06 16:56:13 +00:00
|
|
|
// NewTestWorker returns the worker without calling it's run method.
|
|
|
|
func NewTestWorker(shutdownCtx context.Context, srv *Server) *Worker {
|
|
|
|
w := &Worker{
|
core: backoff considerably when worker is behind raft (#15523)
Upon dequeuing an evaluation workers snapshot their state store at the
eval's wait index or later. This ensures we process an eval at a point
in time after it was created or updated. Processing an eval on an old
snapshot could cause any number of problems such as:
1. Since job registration atomically updates an eval and job in a single
raft entry, scheduling against indexes before that may not have the
eval's job or may have an older version.
2. The older the scheduler's snapshot, the higher the likelihood
something has changed in the cluster state which will cause the plan
applier to reject the scheduler's plan. This could waste work or
even cause eval's to be failed needlessly.
However, the workers run in parallel with a new server pulling the
cluster state from a peer. During this time, which may be many minutes
long, the state store is likely far behind the minimum index required
to process evaluations.
This PR addresses this by adding an additional long backoff period after
an eval is nacked. If the scheduler's indexes catches up within the
additional backoff, it will unblock early to dequeue the next eval.
When the server shuts down we'll get a `context.Canceled` error from the state
store method. We need to bubble this error up so that other callers can detect
it. Handle this case separately when waiting after dequeue so that we can warn
on shutdown instead of throwing an ambiguous error message with just the text
"canceled."
While there may be more precise ways to block scheduling until the
server catches up, this approach adds little risk and covers additional
cases where a server may be temporarily behind due to a spike in load or
a saturated network.
For testing, we make the `raftSyncLimit` into a parameter on the worker's `run` method
so that we can run backoff tests without waiting 30+ seconds. We haven't followed thru
and made all the worker globals into worker parameters, because there isn't much
use outside of testing, but we can consider that in the future.
Co-authored-by: Tim Gross <tgross@hashicorp.com>
2023-01-24 13:56:35 +00:00
|
|
|
srv: srv,
|
|
|
|
start: time.Now(),
|
|
|
|
id: uuid.Generate(),
|
|
|
|
enabledSchedulers: srv.config.EnabledSchedulers,
|
2022-01-06 16:56:13 +00:00
|
|
|
}
|
|
|
|
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
|
|
|
|
}
|
|
|
|
|
2015-07-29 00:02:54 +00:00
|
|
|
func TestWorker_dequeueEvaluation(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-07-29 00:02:54 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-07-29 00:02:54 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
2015-07-29 00:05:59 +00:00
|
|
|
// Create the evaluation
|
2015-08-11 21:27:14 +00:00
|
|
|
eval1 := mock.Eval()
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
2015-07-29 00:02:54 +00:00
|
|
|
|
|
|
|
// Create a worker
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w, _ := NewWorker(s1.shutdownCtx, s1, poolArgs)
|
2015-07-29 00:02:54 +00:00
|
|
|
|
|
|
|
// Attempt dequeue
|
2017-09-13 20:47:01 +00:00
|
|
|
eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond)
|
2015-07-29 00:02:54 +00:00
|
|
|
if shutdown {
|
|
|
|
t.Fatalf("should not shutdown")
|
|
|
|
}
|
2015-08-12 22:25:31 +00:00
|
|
|
if token == "" {
|
|
|
|
t.Fatalf("should get token")
|
|
|
|
}
|
2017-09-13 20:47:01 +00:00
|
|
|
if waitIndex != eval1.ModifyIndex {
|
2017-09-26 22:26:33 +00:00
|
|
|
t.Fatalf("bad wait index; got %d; want %d", waitIndex, eval1.ModifyIndex)
|
2017-09-13 20:47:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure we get a sane eval
|
|
|
|
if !reflect.DeepEqual(eval, eval1) {
|
|
|
|
t.Fatalf("bad: %#v %#v", eval, eval1)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Test that the worker picks up the correct wait index when there are multiple
|
|
|
|
// evals for the same job.
|
|
|
|
func TestWorker_dequeueEvaluation_SerialJobs(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2017-09-13 20:47:01 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2017-09-13 20:47:01 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Create the evaluation
|
|
|
|
eval1 := mock.Eval()
|
|
|
|
eval2 := mock.Eval()
|
|
|
|
eval2.JobID = eval1.JobID
|
|
|
|
|
|
|
|
// Insert the evals into the state store
|
2022-11-16 21:10:11 +00:00
|
|
|
must.NoError(t, s1.fsm.State().UpsertEvals(
|
|
|
|
structs.MsgTypeTestSetup, 1000, []*structs.Evaluation{eval1}))
|
|
|
|
must.NoError(t, s1.fsm.State().UpsertEvals(
|
|
|
|
structs.MsgTypeTestSetup, 2000, []*structs.Evaluation{eval2}))
|
2017-09-13 20:47:01 +00:00
|
|
|
|
|
|
|
s1.evalBroker.Enqueue(eval1)
|
|
|
|
s1.evalBroker.Enqueue(eval2)
|
|
|
|
|
|
|
|
// Create a worker
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2017-09-13 20:47:01 +00:00
|
|
|
|
|
|
|
// Attempt dequeue
|
|
|
|
eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond)
|
2022-11-16 21:10:11 +00:00
|
|
|
must.False(t, shutdown, must.Sprint("should not be shutdown"))
|
|
|
|
must.NotEq(t, token, "", must.Sprint("should get a token"))
|
|
|
|
must.NotEq(t, eval1.ModifyIndex, waitIndex, must.Sprintf("bad wait index"))
|
|
|
|
must.Eq(t, eval, eval1)
|
2017-09-13 20:47:01 +00:00
|
|
|
|
|
|
|
// Update the modify index of the first eval
|
2022-11-16 21:10:11 +00:00
|
|
|
must.NoError(t, s1.fsm.State().UpsertEvals(
|
|
|
|
structs.MsgTypeTestSetup, 1500, []*structs.Evaluation{eval1}))
|
2017-09-13 20:47:01 +00:00
|
|
|
|
|
|
|
// Send the Ack
|
2022-01-06 16:56:13 +00:00
|
|
|
w.sendAck(eval1, token)
|
2017-09-13 20:47:01 +00:00
|
|
|
|
2022-11-16 21:10:11 +00:00
|
|
|
// Attempt second dequeue; it should succeed because the 2nd eval has a
|
|
|
|
// lower modify index than the snapshot used to schedule the 1st
|
|
|
|
// eval. Normally this can only happen if the worker is on a follower that's
|
|
|
|
// trailing behind in raft logs
|
2017-09-13 20:47:01 +00:00
|
|
|
eval, token, waitIndex, shutdown = w.dequeueEvaluation(10 * time.Millisecond)
|
|
|
|
|
2022-11-16 21:10:11 +00:00
|
|
|
must.False(t, shutdown, must.Sprint("should not be shutdown"))
|
|
|
|
must.NotEq(t, token, "", must.Sprint("should get a token"))
|
|
|
|
must.Eq(t, waitIndex, 2000, must.Sprintf("bad wait index"))
|
|
|
|
must.Eq(t, eval, eval2)
|
|
|
|
|
2015-07-29 00:02:54 +00:00
|
|
|
}
|
|
|
|
|
2015-08-23 17:52:31 +00:00
|
|
|
func TestWorker_dequeueEvaluation_paused(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-08-23 17:52:31 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-08-23 17:52:31 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Create the evaluation
|
|
|
|
eval1 := mock.Eval()
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
2015-08-23 17:52:31 +00:00
|
|
|
|
|
|
|
// Create a worker
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2015-08-23 17:52:31 +00:00
|
|
|
w.pauseCond = sync.NewCond(&w.pauseLock)
|
|
|
|
|
|
|
|
// PAUSE the worker
|
2022-01-06 16:56:13 +00:00
|
|
|
w.Pause()
|
2015-08-23 17:52:31 +00:00
|
|
|
|
|
|
|
go func() {
|
|
|
|
time.Sleep(100 * time.Millisecond)
|
2022-01-06 16:56:13 +00:00
|
|
|
w.Resume()
|
2015-08-23 17:52:31 +00:00
|
|
|
}()
|
|
|
|
|
|
|
|
// Attempt dequeue
|
|
|
|
start := time.Now()
|
2017-09-13 20:47:01 +00:00
|
|
|
eval, token, waitIndex, shutdown := w.dequeueEvaluation(10 * time.Millisecond)
|
2015-08-23 17:52:31 +00:00
|
|
|
if diff := time.Since(start); diff < 100*time.Millisecond {
|
|
|
|
t.Fatalf("should have paused: %v", diff)
|
|
|
|
}
|
|
|
|
if shutdown {
|
|
|
|
t.Fatalf("should not shutdown")
|
|
|
|
}
|
|
|
|
if token == "" {
|
|
|
|
t.Fatalf("should get token")
|
|
|
|
}
|
2017-09-13 20:47:01 +00:00
|
|
|
if waitIndex != eval1.ModifyIndex {
|
2017-09-26 22:26:33 +00:00
|
|
|
t.Fatalf("bad wait index; got %d; want %d", waitIndex, eval1.ModifyIndex)
|
2017-09-13 20:47:01 +00:00
|
|
|
}
|
2015-08-23 17:52:31 +00:00
|
|
|
|
|
|
|
// Ensure we get a sane eval
|
|
|
|
if !reflect.DeepEqual(eval, eval1) {
|
|
|
|
t.Fatalf("bad: %#v %#v", eval, eval1)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-07-29 00:02:54 +00:00
|
|
|
func TestWorker_dequeueEvaluation_shutdown(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-07-29 00:02:54 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-07-29 00:02:54 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Create a worker
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2015-07-29 00:02:54 +00:00
|
|
|
|
|
|
|
go func() {
|
|
|
|
time.Sleep(10 * time.Millisecond)
|
|
|
|
s1.Shutdown()
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Attempt dequeue
|
2017-09-13 20:47:01 +00:00
|
|
|
eval, _, _, shutdown := w.dequeueEvaluation(10 * time.Millisecond)
|
2015-07-29 00:02:54 +00:00
|
|
|
if !shutdown {
|
|
|
|
t.Fatalf("should not shutdown")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure we get a sane eval
|
|
|
|
if eval != nil {
|
|
|
|
t.Fatalf("bad: %#v", eval)
|
|
|
|
}
|
2015-07-28 22:15:42 +00:00
|
|
|
}
|
2015-07-29 00:05:59 +00:00
|
|
|
|
2022-01-06 16:56:13 +00:00
|
|
|
func TestWorker_Shutdown(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2022-01-06 16:56:13 +00:00
|
|
|
|
|
|
|
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()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
time.Sleep(10 * time.Millisecond)
|
|
|
|
w.Stop()
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Attempt dequeue
|
|
|
|
eval, _, _, shutdown := w.dequeueEvaluation(10 * time.Millisecond)
|
|
|
|
require.True(t, shutdown)
|
|
|
|
require.Nil(t, eval)
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestWorker_Shutdown_paused(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2022-01-06 16:56:13 +00:00
|
|
|
|
|
|
|
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()
|
|
|
|
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 pause")
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
w.Stop()
|
|
|
|
}()
|
|
|
|
|
|
|
|
// 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")
|
|
|
|
}
|
|
|
|
|
2015-07-29 00:05:59 +00:00
|
|
|
func TestWorker_sendAck(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-07-29 00:05:59 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-07-29 00:05:59 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Create the evaluation
|
2015-08-11 21:27:14 +00:00
|
|
|
eval1 := mock.Eval()
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
2015-07-29 00:05:59 +00:00
|
|
|
|
|
|
|
// Create a worker
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2015-07-29 00:05:59 +00:00
|
|
|
|
|
|
|
// Attempt dequeue
|
2017-09-13 20:47:01 +00:00
|
|
|
eval, token, _, _ := w.dequeueEvaluation(10 * time.Millisecond)
|
2015-07-29 00:05:59 +00:00
|
|
|
|
|
|
|
// Check the depth is 0, 1 unacked
|
|
|
|
stats := s1.evalBroker.Stats()
|
|
|
|
if stats.TotalReady != 0 && stats.TotalUnacked != 1 {
|
|
|
|
t.Fatalf("bad: %#v", stats)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Send the Nack
|
2022-01-06 16:56:13 +00:00
|
|
|
w.sendNack(eval, token)
|
2015-07-29 00:05:59 +00:00
|
|
|
|
|
|
|
// Check the depth is 1, nothing unacked
|
|
|
|
stats = s1.evalBroker.Stats()
|
|
|
|
if stats.TotalReady != 1 && stats.TotalUnacked != 0 {
|
|
|
|
t.Fatalf("bad: %#v", stats)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Attempt dequeue
|
2017-09-13 20:47:01 +00:00
|
|
|
eval, token, _, _ = w.dequeueEvaluation(10 * time.Millisecond)
|
2015-07-29 00:05:59 +00:00
|
|
|
|
|
|
|
// Send the Ack
|
2022-01-06 16:56:13 +00:00
|
|
|
w.sendAck(eval, token)
|
2015-07-29 00:05:59 +00:00
|
|
|
|
|
|
|
// Check the depth is 0
|
|
|
|
stats = s1.evalBroker.Stats()
|
|
|
|
if stats.TotalReady != 0 && stats.TotalUnacked != 0 {
|
|
|
|
t.Fatalf("bad: %#v", stats)
|
|
|
|
}
|
|
|
|
}
|
2015-07-29 00:11:00 +00:00
|
|
|
|
core: backoff considerably when worker is behind raft (#15523)
Upon dequeuing an evaluation workers snapshot their state store at the
eval's wait index or later. This ensures we process an eval at a point
in time after it was created or updated. Processing an eval on an old
snapshot could cause any number of problems such as:
1. Since job registration atomically updates an eval and job in a single
raft entry, scheduling against indexes before that may not have the
eval's job or may have an older version.
2. The older the scheduler's snapshot, the higher the likelihood
something has changed in the cluster state which will cause the plan
applier to reject the scheduler's plan. This could waste work or
even cause eval's to be failed needlessly.
However, the workers run in parallel with a new server pulling the
cluster state from a peer. During this time, which may be many minutes
long, the state store is likely far behind the minimum index required
to process evaluations.
This PR addresses this by adding an additional long backoff period after
an eval is nacked. If the scheduler's indexes catches up within the
additional backoff, it will unblock early to dequeue the next eval.
When the server shuts down we'll get a `context.Canceled` error from the state
store method. We need to bubble this error up so that other callers can detect
it. Handle this case separately when waiting after dequeue so that we can warn
on shutdown instead of throwing an ambiguous error message with just the text
"canceled."
While there may be more precise ways to block scheduling until the
server catches up, this approach adds little risk and covers additional
cases where a server may be temporarily behind due to a spike in load or
a saturated network.
For testing, we make the `raftSyncLimit` into a parameter on the worker's `run` method
so that we can run backoff tests without waiting 30+ seconds. We haven't followed thru
and made all the worker globals into worker parameters, because there isn't much
use outside of testing, but we can consider that in the future.
Co-authored-by: Tim Gross <tgross@hashicorp.com>
2023-01-24 13:56:35 +00:00
|
|
|
func TestWorker_runBackoff(t *testing.T) {
|
|
|
|
ci.Parallel(t)
|
|
|
|
|
|
|
|
srv, cleanupSrv := TestServer(t, func(c *Config) {
|
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
|
|
|
defer cleanupSrv()
|
|
|
|
testutil.WaitForLeader(t, srv.RPC)
|
|
|
|
|
|
|
|
eval1 := mock.Eval()
|
|
|
|
eval1.ModifyIndex = 1000
|
|
|
|
srv.evalBroker.Enqueue(eval1)
|
|
|
|
must.Eq(t, 1, srv.evalBroker.Stats().TotalReady)
|
|
|
|
|
|
|
|
// make a new context here so we can still check the broker's state after
|
|
|
|
// we've shut down the worker
|
|
|
|
workerCtx, workerCancel := context.WithCancel(srv.shutdownCtx)
|
|
|
|
defer workerCancel()
|
|
|
|
|
|
|
|
w := NewTestWorker(workerCtx, srv)
|
|
|
|
doneCh := make(chan struct{})
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
w.run(time.Millisecond)
|
|
|
|
doneCh <- struct{}{}
|
|
|
|
}()
|
|
|
|
|
|
|
|
// We expect to be paused for 10ms + 1ms but otherwise can't be all that
|
|
|
|
// precise here because of concurrency. But checking coverage for this test
|
|
|
|
// shows we've covered the logic
|
|
|
|
t1, cancelT1 := helper.NewSafeTimer(100 * time.Millisecond)
|
|
|
|
defer cancelT1()
|
|
|
|
select {
|
|
|
|
case <-doneCh:
|
|
|
|
t.Fatal("returned early")
|
|
|
|
case <-t1.C:
|
|
|
|
}
|
|
|
|
|
|
|
|
workerCancel()
|
|
|
|
<-doneCh
|
|
|
|
|
|
|
|
must.Eq(t, 1, srv.evalBroker.Stats().TotalWaiting)
|
|
|
|
must.Eq(t, 0, srv.evalBroker.Stats().TotalReady)
|
|
|
|
must.Eq(t, 0, srv.evalBroker.Stats().TotalPending)
|
|
|
|
must.Eq(t, 0, srv.evalBroker.Stats().TotalUnacked)
|
|
|
|
}
|
|
|
|
|
2015-07-29 00:11:00 +00:00
|
|
|
func TestWorker_waitForIndex(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-07-29 00:11:00 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-07-29 00:11:00 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Get the current index
|
|
|
|
index := s1.raft.AppliedIndex()
|
|
|
|
|
|
|
|
// Cause an increment
|
2019-03-12 21:25:14 +00:00
|
|
|
errCh := make(chan error, 1)
|
2015-07-29 00:11:00 +00:00
|
|
|
go func() {
|
|
|
|
time.Sleep(10 * time.Millisecond)
|
2016-06-22 16:04:22 +00:00
|
|
|
n := mock.Node()
|
2020-10-19 13:30:15 +00:00
|
|
|
errCh <- s1.fsm.state.UpsertNode(structs.MsgTypeTestSetup, index+1, n)
|
2015-07-29 00:11:00 +00:00
|
|
|
}()
|
|
|
|
|
|
|
|
// Wait for a future index
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2019-06-24 18:59:44 +00:00
|
|
|
snap, err := w.snapshotMinIndex(index+1, time.Second)
|
2019-03-12 21:25:14 +00:00
|
|
|
require.NoError(t, err)
|
|
|
|
require.NotNil(t, snap)
|
|
|
|
|
|
|
|
// No error from upserting
|
|
|
|
require.NoError(t, <-errCh)
|
2015-07-29 00:11:00 +00:00
|
|
|
|
|
|
|
// Cause a timeout
|
2019-05-17 21:37:42 +00:00
|
|
|
waitIndex := index + 100
|
|
|
|
timeout := 10 * time.Millisecond
|
2019-06-24 18:59:44 +00:00
|
|
|
snap, err = w.snapshotMinIndex(index+100, timeout)
|
2019-03-12 21:25:14 +00:00
|
|
|
require.Nil(t, snap)
|
2019-05-17 21:37:42 +00:00
|
|
|
require.EqualError(t, err,
|
|
|
|
fmt.Sprintf("timed out after %s waiting for index=%d", timeout, waitIndex))
|
core: backoff considerably when worker is behind raft (#15523)
Upon dequeuing an evaluation workers snapshot their state store at the
eval's wait index or later. This ensures we process an eval at a point
in time after it was created or updated. Processing an eval on an old
snapshot could cause any number of problems such as:
1. Since job registration atomically updates an eval and job in a single
raft entry, scheduling against indexes before that may not have the
eval's job or may have an older version.
2. The older the scheduler's snapshot, the higher the likelihood
something has changed in the cluster state which will cause the plan
applier to reject the scheduler's plan. This could waste work or
even cause eval's to be failed needlessly.
However, the workers run in parallel with a new server pulling the
cluster state from a peer. During this time, which may be many minutes
long, the state store is likely far behind the minimum index required
to process evaluations.
This PR addresses this by adding an additional long backoff period after
an eval is nacked. If the scheduler's indexes catches up within the
additional backoff, it will unblock early to dequeue the next eval.
When the server shuts down we'll get a `context.Canceled` error from the state
store method. We need to bubble this error up so that other callers can detect
it. Handle this case separately when waiting after dequeue so that we can warn
on shutdown instead of throwing an ambiguous error message with just the text
"canceled."
While there may be more precise ways to block scheduling until the
server catches up, this approach adds little risk and covers additional
cases where a server may be temporarily behind due to a spike in load or
a saturated network.
For testing, we make the `raftSyncLimit` into a parameter on the worker's `run` method
so that we can run backoff tests without waiting 30+ seconds. We haven't followed thru
and made all the worker globals into worker parameters, because there isn't much
use outside of testing, but we can consider that in the future.
Co-authored-by: Tim Gross <tgross@hashicorp.com>
2023-01-24 13:56:35 +00:00
|
|
|
require.True(t, errors.Is(err, context.DeadlineExceeded), "expect error to wrap DeadlineExceeded")
|
2015-07-29 00:11:00 +00:00
|
|
|
}
|
2015-07-29 00:20:06 +00:00
|
|
|
|
|
|
|
func TestWorker_invokeScheduler(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-07-29 00:20:06 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-07-29 00:20:06 +00:00
|
|
|
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2015-08-11 21:27:14 +00:00
|
|
|
eval := mock.Eval()
|
2015-07-29 00:20:06 +00:00
|
|
|
eval.Type = "noop"
|
|
|
|
|
2019-03-12 21:25:14 +00:00
|
|
|
snap, err := s1.fsm.state.Snapshot()
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
err = w.invokeScheduler(snap, eval, uuid.Generate())
|
|
|
|
require.NoError(t, err)
|
2015-07-29 00:20:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func TestWorker_SubmitPlan(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-07-29 00:20:06 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-07-29 00:20:06 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
2015-08-05 23:23:47 +00:00
|
|
|
// Register node
|
2015-08-11 21:27:14 +00:00
|
|
|
node := mock.Node()
|
2015-08-05 23:23:47 +00:00
|
|
|
testRegisterNode(t, s1, node)
|
|
|
|
|
2017-10-13 21:36:02 +00:00
|
|
|
job := mock.Job()
|
2015-08-12 22:44:36 +00:00
|
|
|
eval1 := mock.Eval()
|
2017-10-13 21:36:02 +00:00
|
|
|
eval1.JobID = job.ID
|
2020-10-19 13:30:15 +00:00
|
|
|
s1.fsm.State().UpsertJob(structs.MsgTypeTestSetup, 1000, job)
|
|
|
|
s1.fsm.State().UpsertEvals(structs.MsgTypeTestSetup, 1000, []*structs.Evaluation{eval1})
|
2016-07-25 21:11:32 +00:00
|
|
|
|
|
|
|
// Create the register request
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
|
|
|
|
2015-08-12 22:44:36 +00:00
|
|
|
evalOut, token, err := s1.evalBroker.Dequeue([]string{eval1.Type}, time.Second)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if evalOut != eval1 {
|
|
|
|
t.Fatalf("Bad eval")
|
|
|
|
}
|
|
|
|
|
2015-08-05 23:23:47 +00:00
|
|
|
// Create an allocation plan
|
2015-08-11 21:27:14 +00:00
|
|
|
alloc := mock.Alloc()
|
2015-08-05 23:23:47 +00:00
|
|
|
plan := &structs.Plan{
|
2017-10-13 21:36:02 +00:00
|
|
|
Job: job,
|
2015-08-12 22:44:36 +00:00
|
|
|
EvalID: eval1.ID,
|
2015-08-05 23:23:47 +00:00
|
|
|
NodeAllocation: map[string][]*structs.Allocation{
|
2017-09-26 22:26:33 +00:00
|
|
|
node.ID: {alloc},
|
2015-08-05 23:23:47 +00:00
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
// Attempt to submit a plan
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
w.evalToken = token
|
|
|
|
|
2015-08-05 23:23:47 +00:00
|
|
|
result, state, err := w.SubmitPlan(plan)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Should have no update
|
|
|
|
if state != nil {
|
|
|
|
t.Fatalf("unexpected state update")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Result should have allocated
|
|
|
|
if result == nil {
|
|
|
|
t.Fatalf("missing result")
|
|
|
|
}
|
|
|
|
|
|
|
|
if result.AllocIndex == 0 {
|
|
|
|
t.Fatalf("Bad: %#v", result)
|
|
|
|
}
|
|
|
|
if len(result.NodeAllocation) != 1 {
|
|
|
|
t.Fatalf("Bad: %#v", result)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-03-05 21:41:41 +00:00
|
|
|
func TestWorker_SubmitPlanNormalizedAllocations(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2019-03-05 21:41:41 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
2022-10-06 16:47:02 +00:00
|
|
|
c.Build = "1.4.0"
|
2019-03-05 21:41:41 +00:00
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2019-03-05 21:41:41 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Register node
|
|
|
|
node := mock.Node()
|
|
|
|
testRegisterNode(t, s1, node)
|
|
|
|
|
|
|
|
job := mock.Job()
|
|
|
|
eval1 := mock.Eval()
|
|
|
|
eval1.JobID = job.ID
|
2020-10-19 13:30:15 +00:00
|
|
|
s1.fsm.State().UpsertJob(structs.MsgTypeTestSetup, 0, job)
|
|
|
|
s1.fsm.State().UpsertEvals(structs.MsgTypeTestSetup, 0, []*structs.Evaluation{eval1})
|
2019-03-05 21:41:41 +00:00
|
|
|
|
|
|
|
stoppedAlloc := mock.Alloc()
|
|
|
|
preemptedAlloc := mock.Alloc()
|
2020-10-19 13:30:15 +00:00
|
|
|
s1.fsm.State().UpsertAllocs(structs.MsgTypeTestSetup, 5, []*structs.Allocation{stoppedAlloc, preemptedAlloc})
|
2019-03-05 21:41:41 +00:00
|
|
|
|
|
|
|
// Create an allocation plan
|
|
|
|
plan := &structs.Plan{
|
|
|
|
Job: job,
|
|
|
|
EvalID: eval1.ID,
|
|
|
|
NodeUpdate: make(map[string][]*structs.Allocation),
|
|
|
|
NodePreemptions: make(map[string][]*structs.Allocation),
|
|
|
|
}
|
|
|
|
desiredDescription := "desired desc"
|
2020-06-09 21:13:53 +00:00
|
|
|
plan.AppendStoppedAlloc(stoppedAlloc, desiredDescription, structs.AllocClientStatusLost, "")
|
2019-03-05 21:41:41 +00:00
|
|
|
preemptingAllocID := uuid.Generate()
|
|
|
|
plan.AppendPreemptedAlloc(preemptedAlloc, preemptingAllocID)
|
|
|
|
|
|
|
|
// Attempt to submit a plan
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
2019-03-05 21:41:41 +00:00
|
|
|
w.SubmitPlan(plan)
|
|
|
|
|
|
|
|
assert.Equal(t, &structs.Allocation{
|
|
|
|
ID: preemptedAlloc.ID,
|
|
|
|
PreemptedByAllocation: preemptingAllocID,
|
|
|
|
}, plan.NodePreemptions[preemptedAlloc.NodeID][0])
|
|
|
|
assert.Equal(t, &structs.Allocation{
|
|
|
|
ID: stoppedAlloc.ID,
|
|
|
|
DesiredDescription: desiredDescription,
|
|
|
|
ClientStatus: structs.AllocClientStatusLost,
|
|
|
|
}, plan.NodeUpdate[stoppedAlloc.NodeID][0])
|
|
|
|
}
|
|
|
|
|
2015-08-05 23:23:47 +00:00
|
|
|
func TestWorker_SubmitPlan_MissingNodeRefresh(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-08-05 23:23:47 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-08-05 23:23:47 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Register node
|
2015-08-11 21:27:14 +00:00
|
|
|
node := mock.Node()
|
2015-08-05 23:23:47 +00:00
|
|
|
testRegisterNode(t, s1, node)
|
|
|
|
|
2017-10-13 21:36:02 +00:00
|
|
|
// Create the job
|
|
|
|
job := mock.Job()
|
2020-10-19 13:30:15 +00:00
|
|
|
s1.fsm.State().UpsertJob(structs.MsgTypeTestSetup, 1000, job)
|
2017-10-13 21:36:02 +00:00
|
|
|
|
2015-08-12 22:44:36 +00:00
|
|
|
// Create the register request
|
|
|
|
eval1 := mock.Eval()
|
2017-10-13 21:36:02 +00:00
|
|
|
eval1.JobID = job.ID
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
|
|
|
|
2015-08-12 22:44:36 +00:00
|
|
|
evalOut, token, err := s1.evalBroker.Dequeue([]string{eval1.Type}, time.Second)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if evalOut != eval1 {
|
|
|
|
t.Fatalf("Bad eval")
|
|
|
|
}
|
|
|
|
|
2015-08-05 23:23:47 +00:00
|
|
|
// Create an allocation plan, with unregistered node
|
2015-08-11 21:27:14 +00:00
|
|
|
node2 := mock.Node()
|
|
|
|
alloc := mock.Alloc()
|
2015-08-05 23:23:47 +00:00
|
|
|
plan := &structs.Plan{
|
2017-10-13 21:36:02 +00:00
|
|
|
Job: job,
|
2015-08-12 22:44:36 +00:00
|
|
|
EvalID: eval1.ID,
|
2015-08-05 23:23:47 +00:00
|
|
|
NodeAllocation: map[string][]*structs.Allocation{
|
2017-09-26 22:26:33 +00:00
|
|
|
node2.ID: {alloc},
|
2015-08-05 23:23:47 +00:00
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
// Attempt to submit a plan
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
w.evalToken = token
|
|
|
|
|
2015-08-05 23:23:47 +00:00
|
|
|
result, state, err := w.SubmitPlan(plan)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Result should have allocated
|
|
|
|
if result == nil {
|
|
|
|
t.Fatalf("missing result")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Expect no allocation and forced refresh
|
|
|
|
if result.AllocIndex != 0 {
|
|
|
|
t.Fatalf("Bad: %#v", result)
|
|
|
|
}
|
|
|
|
if result.RefreshIndex == 0 {
|
|
|
|
t.Fatalf("Bad: %#v", result)
|
|
|
|
}
|
|
|
|
if len(result.NodeAllocation) != 0 {
|
|
|
|
t.Fatalf("Bad: %#v", result)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Should have an update
|
|
|
|
if state == nil {
|
|
|
|
t.Fatalf("expected state update")
|
|
|
|
}
|
2015-07-29 00:20:06 +00:00
|
|
|
}
|
2015-08-15 21:25:00 +00:00
|
|
|
|
|
|
|
func TestWorker_UpdateEval(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-08-15 21:25:00 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-08-15 21:25:00 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Register node
|
|
|
|
node := mock.Node()
|
|
|
|
testRegisterNode(t, s1, node)
|
|
|
|
|
|
|
|
// Create the register request
|
|
|
|
eval1 := mock.Eval()
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
2015-08-15 21:25:00 +00:00
|
|
|
evalOut, token, err := s1.evalBroker.Dequeue([]string{eval1.Type}, time.Second)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if evalOut != eval1 {
|
|
|
|
t.Fatalf("Bad eval")
|
|
|
|
}
|
|
|
|
|
|
|
|
eval2 := evalOut.Copy()
|
|
|
|
eval2.Status = structs.EvalStatusComplete
|
|
|
|
|
|
|
|
// Attempt to update eval
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
w.evalToken = token
|
|
|
|
|
2015-08-15 21:25:00 +00:00
|
|
|
err = w.UpdateEval(eval2)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
|
2017-02-08 05:22:48 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
out, err := s1.fsm.State().EvalByID(ws, eval2.ID)
|
2015-08-15 21:25:00 +00:00
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if out.Status != structs.EvalStatusComplete {
|
|
|
|
t.Fatalf("bad: %v", out)
|
|
|
|
}
|
2016-05-21 01:07:10 +00:00
|
|
|
if out.SnapshotIndex != w.snapshotIndex {
|
|
|
|
t.Fatalf("bad: %v", out)
|
|
|
|
}
|
2015-08-15 21:25:00 +00:00
|
|
|
}
|
2015-09-07 21:23:48 +00:00
|
|
|
|
|
|
|
func TestWorker_CreateEval(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2015-09-07 21:23:48 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2015-09-07 21:23:48 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Register node
|
|
|
|
node := mock.Node()
|
|
|
|
testRegisterNode(t, s1, node)
|
|
|
|
|
|
|
|
// Create the register request
|
|
|
|
eval1 := mock.Eval()
|
2016-05-18 18:35:15 +00:00
|
|
|
s1.evalBroker.Enqueue(eval1)
|
|
|
|
|
2015-09-07 21:23:48 +00:00
|
|
|
evalOut, token, err := s1.evalBroker.Dequeue([]string{eval1.Type}, time.Second)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if evalOut != eval1 {
|
|
|
|
t.Fatalf("Bad eval")
|
|
|
|
}
|
|
|
|
|
|
|
|
eval2 := mock.Eval()
|
|
|
|
eval2.PreviousEval = eval1.ID
|
|
|
|
|
|
|
|
// Attempt to create eval
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
w.evalToken = token
|
|
|
|
|
2015-09-07 21:23:48 +00:00
|
|
|
err = w.CreateEval(eval2)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
|
2017-02-08 05:22:48 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
out, err := s1.fsm.State().EvalByID(ws, eval2.ID)
|
2015-09-07 21:23:48 +00:00
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if out.PreviousEval != eval1.ID {
|
|
|
|
t.Fatalf("bad: %v", out)
|
|
|
|
}
|
2016-05-21 01:07:10 +00:00
|
|
|
if out.SnapshotIndex != w.snapshotIndex {
|
|
|
|
t.Fatalf("bad: %v", out)
|
|
|
|
}
|
2015-09-07 21:23:48 +00:00
|
|
|
}
|
2016-05-20 23:03:53 +00:00
|
|
|
|
|
|
|
func TestWorker_ReblockEval(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2019-12-04 00:15:11 +00:00
|
|
|
|
|
|
|
s1, cleanupS1 := TestServer(t, func(c *Config) {
|
2016-05-20 23:03:53 +00:00
|
|
|
c.NumSchedulers = 0
|
|
|
|
c.EnabledSchedulers = []string{structs.JobTypeService}
|
|
|
|
})
|
2019-12-04 00:15:11 +00:00
|
|
|
defer cleanupS1()
|
2016-05-20 23:03:53 +00:00
|
|
|
testutil.WaitForLeader(t, s1.RPC)
|
|
|
|
|
|
|
|
// Create the blocked eval
|
|
|
|
eval1 := mock.Eval()
|
|
|
|
eval1.Status = structs.EvalStatusBlocked
|
2016-07-20 21:09:03 +00:00
|
|
|
eval1.QueuedAllocations = map[string]int{"cache": 100}
|
2016-05-20 23:03:53 +00:00
|
|
|
|
|
|
|
// Insert it into the state store
|
2020-10-19 13:30:15 +00:00
|
|
|
if err := s1.fsm.State().UpsertEvals(structs.MsgTypeTestSetup, 1000, []*structs.Evaluation{eval1}); err != nil {
|
2016-05-20 23:03:53 +00:00
|
|
|
t.Fatal(err)
|
|
|
|
}
|
|
|
|
|
2016-07-20 21:09:03 +00:00
|
|
|
// Create the job summary
|
|
|
|
js := mock.JobSummary(eval1.JobID)
|
2016-07-21 21:43:21 +00:00
|
|
|
tg := js.Summary["web"]
|
2016-07-20 21:09:03 +00:00
|
|
|
tg.Queued = 100
|
2016-07-21 21:43:21 +00:00
|
|
|
js.Summary["web"] = tg
|
2016-07-20 21:09:03 +00:00
|
|
|
if err := s1.fsm.State().UpsertJobSummary(1001, js); err != nil {
|
|
|
|
t.Fatal(err)
|
|
|
|
}
|
|
|
|
|
2016-05-25 03:20:06 +00:00
|
|
|
// Enqueue the eval and then dequeue
|
|
|
|
s1.evalBroker.Enqueue(eval1)
|
2016-05-20 23:03:53 +00:00
|
|
|
evalOut, token, err := s1.evalBroker.Dequeue([]string{eval1.Type}, time.Second)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if evalOut != eval1 {
|
|
|
|
t.Fatalf("Bad eval")
|
|
|
|
}
|
|
|
|
|
|
|
|
eval2 := evalOut.Copy()
|
2016-07-21 21:43:21 +00:00
|
|
|
eval2.QueuedAllocations = map[string]int{"web": 50}
|
2016-05-20 23:03:53 +00:00
|
|
|
|
|
|
|
// Attempt to reblock eval
|
2022-01-06 16:56:13 +00:00
|
|
|
poolArgs := getSchedulerWorkerPoolArgsFromConfigLocked(s1.config).Copy()
|
|
|
|
w := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
w.evalToken = token
|
|
|
|
|
2016-05-20 23:03:53 +00:00
|
|
|
err = w.ReblockEval(eval2)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
|
2016-05-21 01:07:10 +00:00
|
|
|
// Ack the eval
|
2022-01-06 16:56:13 +00:00
|
|
|
w.sendAck(evalOut, token)
|
2016-05-21 01:07:10 +00:00
|
|
|
|
2016-05-20 23:03:53 +00:00
|
|
|
// Check that it is blocked
|
|
|
|
bStats := s1.blockedEvals.Stats()
|
2016-05-21 01:07:10 +00:00
|
|
|
if bStats.TotalBlocked+bStats.TotalEscaped != 1 {
|
|
|
|
t.Fatalf("ReblockEval didn't insert eval into the blocked eval tracker: %#v", bStats)
|
|
|
|
}
|
|
|
|
|
2016-07-20 21:09:03 +00:00
|
|
|
// Check that the eval was updated
|
2017-02-08 05:22:48 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
eval, err := s1.fsm.State().EvalByID(ws, eval2.ID)
|
2016-07-20 21:09:03 +00:00
|
|
|
if err != nil {
|
|
|
|
t.Fatal(err)
|
|
|
|
}
|
|
|
|
if !reflect.DeepEqual(eval.QueuedAllocations, eval2.QueuedAllocations) {
|
|
|
|
t.Fatalf("expected: %#v, actual: %#v", eval2.QueuedAllocations, eval.QueuedAllocations)
|
|
|
|
}
|
|
|
|
|
2016-05-21 01:07:10 +00:00
|
|
|
// Check that the snapshot index was set properly by unblocking the eval and
|
|
|
|
// then dequeuing.
|
2016-05-23 22:24:31 +00:00
|
|
|
s1.blockedEvals.Unblock("foobar", 1000)
|
2016-05-21 01:07:10 +00:00
|
|
|
|
|
|
|
reblockedEval, _, err := s1.evalBroker.Dequeue([]string{eval1.Type}, 1*time.Second)
|
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("err: %v", err)
|
|
|
|
}
|
|
|
|
if reblockedEval == nil {
|
|
|
|
t.Fatalf("Nil eval")
|
|
|
|
}
|
|
|
|
if reblockedEval.ID != eval1.ID {
|
|
|
|
t.Fatalf("Bad eval")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Check that the SnapshotIndex is set
|
|
|
|
if reblockedEval.SnapshotIndex != w.snapshotIndex {
|
|
|
|
t.Fatalf("incorrect snapshot index; got %d; want %d",
|
|
|
|
reblockedEval.SnapshotIndex, w.snapshotIndex)
|
2016-05-20 23:03:53 +00:00
|
|
|
}
|
|
|
|
}
|
2022-01-06 16:56:13 +00:00
|
|
|
|
|
|
|
func TestWorker_Info(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2022-01-06 16:56:13 +00:00
|
|
|
|
|
|
|
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 := newWorker(s1.shutdownCtx, s1, poolArgs)
|
|
|
|
|
|
|
|
require.Equal(t, WorkerStarting, w.GetStatus())
|
|
|
|
workerInfo := w.Info()
|
|
|
|
require.Equal(t, WorkerStarting.String(), workerInfo.Status)
|
|
|
|
}
|
|
|
|
|
|
|
|
const (
|
|
|
|
longWait = 100 * time.Millisecond
|
|
|
|
tinyWait = 10 * time.Millisecond
|
|
|
|
)
|
|
|
|
|
|
|
|
func TestWorker_SetPause(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2022-01-06 16:56:13 +00:00
|
|
|
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.Pause()
|
|
|
|
}()
|
|
|
|
require.Eventually(t, w.IsPaused, longWait, tinyWait, "pausing a paused should be okay")
|
|
|
|
|
|
|
|
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 TestWorker_SetPause_OutOfOrderEvents(t *testing.T) {
|
2022-03-15 12:42:43 +00:00
|
|
|
ci.Parallel(t)
|
2022-01-06 16:56:13 +00:00
|
|
|
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.setStatuses(WorkerStarted, 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)
|
|
|
|
}
|
|
|
|
}
|