From 199d12865fdaf0f84176d0d5f8abaffc94e1cb82 Mon Sep 17 00:00:00 2001 From: Piotr Kazmierczak <470696+pkazmierczak@users.noreply.github.com> Date: Wed, 11 Jun 2025 20:11:04 +0200 Subject: [PATCH] scheduler: isolate `feasibility` (#26031) This change isolates all the code that deals with node selection in the scheduler into its own package called feasible. --------- Co-authored-by: Tim Gross --- helper/raftutil/sample_test.go | 3 +- nomad/core_sched.go | 4 +- nomad/fsm.go | 3 +- nomad/host_volume_endpoint.go | 18 +- nomad/job_endpoint.go | 3 +- nomad/server.go | 3 +- nomad/worker.go | 7 +- nomad/worker_test.go | 16 +- scheduler/README.md | 43 +- scheduler/benchmarks/benchmarks_test.go | 7 +- scheduler/benchmarks/helpers_test.go | 14 +- scheduler/{ => feasible}/context.go | 79 +- scheduler/{ => feasible}/context_test.go | 78 +- scheduler/{ => feasible}/device.go | 2 +- scheduler/{ => feasible}/device_test.go | 16 +- scheduler/{ => feasible}/feasible.go | 29 +- scheduler/{ => feasible}/feasible_test.go | 72 +- scheduler/{ => feasible}/numa_ce.go | 2 +- scheduler/{ => feasible}/numa_ce_test.go | 2 +- scheduler/{ => feasible}/preemption.go | 2 +- scheduler/feasible/preemption_test.go | 1422 +++++++++++++++++++++ scheduler/{ => feasible}/propertyset.go | 2 +- scheduler/{ => feasible}/rank.go | 7 +- scheduler/{ => feasible}/rank_test.go | 61 +- scheduler/{ => feasible}/select.go | 2 +- scheduler/{ => feasible}/select_test.go | 8 +- scheduler/{ => feasible}/spread.go | 2 +- scheduler/feasible/spread_test.go | 685 ++++++++++ scheduler/{ => feasible}/stack.go | 44 +- scheduler/{ => feasible}/stack_ce.go | 2 +- scheduler/{ => feasible}/stack_test.go | 38 +- scheduler/generic_sched.go | 31 +- scheduler/generic_sched_test.go | 182 +-- scheduler/preemption_test.go | 1366 +------------------- scheduler/scheduler.go | 128 +- scheduler/scheduler_sysbatch_test.go | 55 +- scheduler/scheduler_system.go | 23 +- scheduler/scheduler_system_test.go | 88 +- scheduler/scheduler_test.go | 6 +- scheduler/spread_test.go | 695 +--------- scheduler/structs/interfaces.go | 133 ++ scheduler/structs/structs.go | 53 + scheduler/tests/numa_ce.go | 53 + scheduler/{ => tests}/testing.go | 17 +- scheduler/util.go | 42 +- scheduler/util_test.go | 68 +- 46 files changed, 2877 insertions(+), 2739 deletions(-) rename scheduler/{ => feasible}/context.go (88%) rename scheduler/{ => feasible}/context_test.go (84%) rename scheduler/{ => feasible}/device.go (99%) rename scheduler/{ => feasible}/device_test.go (98%) rename scheduler/{ => feasible}/feasible.go (98%) rename scheduler/{ => feasible}/feasible_test.go (98%) rename scheduler/{ => feasible}/numa_ce.go (99%) rename scheduler/{ => feasible}/numa_ce_test.go (99%) rename scheduler/{ => feasible}/preemption.go (99%) create mode 100644 scheduler/feasible/preemption_test.go rename scheduler/{ => feasible}/propertyset.go (99%) rename scheduler/{ => feasible}/rank.go (99%) rename scheduler/{ => feasible}/rank_test.go (98%) rename scheduler/{ => feasible}/select.go (99%) rename scheduler/{ => feasible}/select_test.go (98%) rename scheduler/{ => feasible}/spread.go (99%) create mode 100644 scheduler/feasible/spread_test.go rename scheduler/{ => feasible}/stack.go (93%) rename scheduler/{ => feasible}/stack_ce.go (91%) rename scheduler/{ => feasible}/stack_test.go (97%) create mode 100644 scheduler/structs/interfaces.go create mode 100644 scheduler/structs/structs.go create mode 100644 scheduler/tests/numa_ce.go rename scheduler/{ => tests}/testing.go (95%) diff --git a/helper/raftutil/sample_test.go b/helper/raftutil/sample_test.go index 860c48c81..401525523 100644 --- a/helper/raftutil/sample_test.go +++ b/helper/raftutil/sample_test.go @@ -10,6 +10,7 @@ import ( "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/kr/pretty" "github.com/stretchr/testify/require" ) @@ -99,7 +100,7 @@ func TestSchedulerLogic(t *testing.T) { } // Process the evaluation - h := scheduler.NewHarnessWithState(t, state) + h := tests.NewHarnessWithState(t, state) err = h.Process(scheduler.NewServiceScheduler, eval) require.NoError(t, err) diff --git a/nomad/core_sched.go b/nomad/core_sched.go index ca25abcb4..2af7eab7c 100644 --- a/nomad/core_sched.go +++ b/nomad/core_sched.go @@ -17,7 +17,7 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/scheduler" + sstructs "github.com/hashicorp/nomad/scheduler/structs" "golang.org/x/time/rate" ) @@ -37,7 +37,7 @@ type CoreScheduler struct { } // NewCoreScheduler is used to return a new system scheduler instance -func NewCoreScheduler(srv *Server, snap *state.StateSnapshot) scheduler.Scheduler { +func NewCoreScheduler(srv *Server, snap *state.StateSnapshot) sstructs.Scheduler { s := &CoreScheduler{ srv: srv, snap: snap, diff --git a/nomad/fsm.go b/nomad/fsm.go index 1816a7747..717b16cfd 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -21,6 +21,7 @@ import ( "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/hashicorp/raft" ) @@ -2047,7 +2048,7 @@ func (n *nomadFSM) reconcileQueuedAllocations(index uint64) error { if job.IsParameterized() || job.IsPeriodic() { continue } - planner := &scheduler.Harness{ + planner := &tests.Harness{ State: &snap.StateStore, } // Create an eval and mark it as requiring annotations and insert that as well diff --git a/nomad/host_volume_endpoint.go b/nomad/host_volume_endpoint.go index 4cc49d5a0..1809c68ea 100644 --- a/nomad/host_volume_endpoint.go +++ b/nomad/host_volume_endpoint.go @@ -22,7 +22,7 @@ import ( "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/state/paginator" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/scheduler" + "github.com/hashicorp/nomad/scheduler/feasible" ) // HostVolume is the server RPC endpoint for host volumes @@ -538,18 +538,18 @@ func (v *HostVolume) placeHostVolume(snap *state.StateSnapshot, vol *structs.Hos return nil, err } - var checker *scheduler.ConstraintChecker + var checker *feasible.ConstraintChecker ctx := &placementContext{ regexpCache: make(map[string]*regexp.Regexp), - versionCache: make(map[string]scheduler.VerConstraints), - semverCache: make(map[string]scheduler.VerConstraints), + versionCache: make(map[string]feasible.VerConstraints), + semverCache: make(map[string]feasible.VerConstraints), } constraints := []*structs.Constraint{{ LTarget: fmt.Sprintf("${attr.plugins.host_volume.%s.version}", vol.PluginID), Operand: "is_set", }} constraints = append(constraints, vol.Constraints...) - checker = scheduler.NewConstraintChecker(ctx, constraints) + checker = feasible.NewConstraintChecker(ctx, constraints) var ( filteredByExisting int @@ -602,18 +602,18 @@ func (v *HostVolume) placeHostVolume(snap *state.StateSnapshot, vol *structs.Hos // feasibility checker for constraints type placementContext struct { regexpCache map[string]*regexp.Regexp - versionCache map[string]scheduler.VerConstraints - semverCache map[string]scheduler.VerConstraints + versionCache map[string]feasible.VerConstraints + semverCache map[string]feasible.VerConstraints } func (ctx *placementContext) Metrics() *structs.AllocMetric { return &structs.AllocMetric{} } func (ctx *placementContext) RegexpCache() map[string]*regexp.Regexp { return ctx.regexpCache } -func (ctx *placementContext) VersionConstraintCache() map[string]scheduler.VerConstraints { +func (ctx *placementContext) VersionConstraintCache() map[string]feasible.VerConstraints { return ctx.versionCache } -func (ctx *placementContext) SemverConstraintCache() map[string]scheduler.VerConstraints { +func (ctx *placementContext) SemverConstraintCache() map[string]feasible.VerConstraints { return ctx.semverCache } diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 41f920be0..a25b7f1f1 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -26,6 +26,7 @@ import ( "github.com/hashicorp/nomad/nomad/state/paginator" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" + "github.com/hashicorp/nomad/scheduler/tests" ) const ( @@ -1893,7 +1894,7 @@ func (j *Job) Plan(args *structs.JobPlanRequest, reply *structs.JobPlanResponse) // Create an in-memory Planner that returns no errors and stores the // submitted plan and created evals. - planner := &scheduler.Harness{ + planner := &tests.Harness{ State: &snap.StateStore, } diff --git a/nomad/server.go b/nomad/server.go index 82ebc8693..ca255783a 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -50,6 +50,7 @@ import ( "github.com/hashicorp/nomad/nomad/structs/config" "github.com/hashicorp/nomad/nomad/volumewatcher" "github.com/hashicorp/nomad/scheduler" + sstructs "github.com/hashicorp/nomad/scheduler/structs" ) const ( @@ -1860,7 +1861,7 @@ func (s *Server) listenWorkerEvents() { } case e := <-s.workersEventCh: switch event := e.(type) { - case *scheduler.PortCollisionEvent: + case *sstructs.PortCollisionEvent: if event == nil || event.Node == nil { continue } diff --git a/nomad/worker.go b/nomad/worker.go index 0322d7d38..29436e376 100644 --- a/nomad/worker.go +++ b/nomad/worker.go @@ -21,6 +21,7 @@ import ( "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" + sstructs "github.com/hashicorp/nomad/scheduler/structs" ) const ( @@ -620,7 +621,7 @@ func (w *Worker) invokeScheduler(snap *state.StateSnapshot, eval *structs.Evalua } // Create the scheduler, or use the special core scheduler - var sched scheduler.Scheduler + var sched sstructs.Scheduler if eval.Type == structs.JobTypeCore { sched = NewCoreScheduler(w.srv, snap) } else { @@ -647,7 +648,7 @@ func (w *Worker) ServersMeetMinimumVersion(minVersion *version.Version, checkFai // 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) { +func (w *Worker) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, sstructs.State, error) { // 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() { @@ -700,7 +701,7 @@ SUBMIT: // planned based on stale data, which is causing issues. For example, a // node failure since the time we've started planning or conflicting task // allocations. - var state scheduler.State + var state sstructs.State if result.RefreshIndex != 0 { // Wait for the raft log to catchup to the evaluation w.logger.Debug("refreshing state", "refresh_index", result.RefreshIndex, "eval_id", plan.EvalID) diff --git a/nomad/worker_test.go b/nomad/worker_test.go index c63deec3d..4bd18c7ea 100644 --- a/nomad/worker_test.go +++ b/nomad/worker_test.go @@ -16,23 +16,23 @@ import ( "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/ci" "github.com/hashicorp/nomad/helper" - "github.com/shoenig/test/must" - "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" "github.com/hashicorp/nomad/scheduler" + sstructs "github.com/hashicorp/nomad/scheduler/structs" "github.com/hashicorp/nomad/testutil" + "github.com/shoenig/test/must" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) type NoopScheduler struct { - state scheduler.State - planner scheduler.Planner + state sstructs.State + planner sstructs.Planner eval *structs.Evaluation - eventsCh chan<- interface{} + eventsCh chan<- any err error } @@ -48,7 +48,9 @@ func (n *NoopScheduler) Process(eval *structs.Evaluation) error { } func init() { - scheduler.BuiltinSchedulers["noop"] = func(logger log.Logger, eventsCh chan<- interface{}, s scheduler.State, p scheduler.Planner) scheduler.Scheduler { + scheduler.BuiltinSchedulers["noop"] = func( + logger log.Logger, eventsCh chan<- any, s sstructs.State, p sstructs.Planner, + ) sstructs.Scheduler { n := &NoopScheduler{ state: s, planner: p, diff --git a/scheduler/README.md b/scheduler/README.md index de80c0956..d825a510b 100644 --- a/scheduler/README.md +++ b/scheduler/README.md @@ -167,14 +167,26 @@ The following diagram illustrates the logic flow of the cluster reconciler: ### Node Reconciler -TODO +The system scheduler also does a "reconciliation" step, but only on a +per-node basis (system jobs run on all feasible nodes), which makes it +simpler than the service reconciler which takes into account a whole cluster, +and has jobs that can run on arbitrary subset of clients. -## Feasibility checking +Node reconciliation removes tainted nodes, updates terminal allocations to lost, +deals with disconnected nodes and computes placements. -Nomad uses a set of iterators to iterate over nodes and check how feasible -they are for any given allocation. The scheduler uses a `Stack` interface that -lives in `scheduler/stack.go` file in order to make placement decisions, and -feasibility iterators that live in `scheduler/feasible.go` to filter by: +## Finding the right node + +The `scheduler/feasible` package contains all the logic used to finding the +right nodes to place workloads. + +### Feasibility checking + +Nomad uses a set of iterators to iterate over nodes and check how feasible they +are for any given allocation. The scheduler uses a `Stack` interface that lives +in `scheduler/feasible/stack.go` file in order to make placement decisions, +and feasibility iterators that live in `scheduler/feasible/feasible.go` to +filter by: - node eligibiligy, - data center, @@ -191,24 +203,13 @@ Once nodes are filtered, the `Stack` implementations (`GenericStack` and - affinities, - and quotas. -## Node reconciliation - -The system scheduler also does a "reconciliation" step, but only on a -per-node basis (system jobs run on all feasible nodes), which makes it -simpler than the service reconciler which takes into account a whole cluster, -and has jobs that can run on arbitrary subset of clients. The code is in -`scheduler/scheduler_system.go` file. - -Node reconciliation removes tainted nodes, updates terminal allocations to lost, -deals with disconnected nodes and computes placements. - -## Finding the best fit and scoring +### Finding the best fit and scoring Applies only to service and batch jobs, since system and sysbatch jobs are placed on all feasible nodes. -This part of scheduling sits in the `scheduler/rank.go` file. The `RankIterator` -interface, which is implemented by e.g., `SpreadIterator` and `BinPackIterator`, -captures the ranking logic in its `Next()` methods. +This part of scheduling sits in the `scheduler/feasible/rank.go` file. The +`RankIterator` interface, which is implemented by e.g., `SpreadIterator` and +`BinPackIterator`, captures the ranking logic in its `Next()` methods. [0]: https://developer.hashicorp.com/nomad/docs/concepts/scheduling/scheduling diff --git a/scheduler/benchmarks/benchmarks_test.go b/scheduler/benchmarks/benchmarks_test.go index cc2c7668d..2488bb16d 100644 --- a/scheduler/benchmarks/benchmarks_test.go +++ b/scheduler/benchmarks/benchmarks_test.go @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler" + "github.com/hashicorp/nomad/scheduler/tests" ) // BenchmarkSchedulerExample is an example of how to write a one-off @@ -114,7 +115,7 @@ func BenchmarkServiceScheduler(b *testing.B) { for _, bm := range benchmarks { job := generateJob(bm.withSpread, bm.jobSize, bm.racks) - h := scheduler.NewHarness(b) + h := tests.NewHarness(b) h.SetNoSubmit() upsertNodes(h, bm.clusterSize, bm.racks) eval := upsertJob(h, job) @@ -130,7 +131,7 @@ func BenchmarkServiceScheduler(b *testing.B) { } -func upsertJob(h *scheduler.Harness, job *structs.Job) *structs.Evaluation { +func upsertJob(h *tests.Harness, job *structs.Job) *structs.Evaluation { err := h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job) if err != nil { panic(err) @@ -181,7 +182,7 @@ func generateJob(withSpread bool, jobSize int, racks int) *structs.Job { return job } -func upsertNodes(h *scheduler.Harness, count, racks int) { +func upsertNodes(h *tests.Harness, count, racks int) { datacenters := []string{"dc-1", "dc-2"} diff --git a/scheduler/benchmarks/helpers_test.go b/scheduler/benchmarks/helpers_test.go index 5d340b153..7007717c8 100644 --- a/scheduler/benchmarks/helpers_test.go +++ b/scheduler/benchmarks/helpers_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/hashicorp/nomad/helper/raftutil" - "github.com/hashicorp/nomad/scheduler" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test/must" ) @@ -25,7 +25,7 @@ import ( // - NOMAD_BENCHMARK_DATADIR: path to data directory // - NOMAD_BENCHMARK_SNAPSHOT: path to raft snapshot // - neither: empty starting state -func NewBenchmarkingHarness(t testing.TB) *scheduler.Harness { +func NewBenchmarkingHarness(t testing.TB) *tests.Harness { // create the Harness and starting state. datadir := os.Getenv("NOMAD_BENCHMARK_DATADIR") if datadir != "" { @@ -40,12 +40,12 @@ func NewBenchmarkingHarness(t testing.TB) *scheduler.Harness { return h } } - return scheduler.NewHarness(t) + return tests.NewHarness(t) } // NewHarnessFromDataDir creates a new scheduler test harness with // state loaded from an existing datadir. -func NewHarnessFromDataDir(t testing.TB, datadirPath string) (*scheduler.Harness, error) { +func NewHarnessFromDataDir(t testing.TB, datadirPath string) (*tests.Harness, error) { if datadirPath == "" { return nil, errors.New("datadir path was not set") } @@ -58,12 +58,12 @@ func NewHarnessFromDataDir(t testing.TB, datadirPath string) (*scheduler.Harness return nil, err } - return scheduler.NewHarnessWithState(t, fsm.State()), nil + return tests.NewHarnessWithState(t, fsm.State()), nil } // NewHarnessFromDataDir creates a new harness with state loaded // from an existing raft snapshot. -func NewHarnessFromSnapshot(t testing.TB, snapshotPath string) (*scheduler.Harness, error) { +func NewHarnessFromSnapshot(t testing.TB, snapshotPath string) (*tests.Harness, error) { if snapshotPath == "" { return nil, errors.New("snapshot path was not set") } @@ -78,5 +78,5 @@ func NewHarnessFromSnapshot(t testing.TB, snapshotPath string) (*scheduler.Harne return nil, err } - return scheduler.NewHarnessWithState(t, state), nil + return tests.NewHarnessWithState(t, state), nil } diff --git a/scheduler/context.go b/scheduler/feasible/context.go similarity index 88% rename from scheduler/context.go rename to scheduler/feasible/context.go index e48cefc39..a03b62cfe 100644 --- a/scheduler/context.go +++ b/scheduler/feasible/context.go @@ -1,20 +1,25 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "regexp" + "testing" log "github.com/hashicorp/go-hclog" memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" + sstructs "github.com/hashicorp/nomad/scheduler/structs" ) // Context is used to track contextual information used for placement type Context interface { // State is used to inspect the current global state - State() State + State() sstructs.State // Plan returns the current plan Plan() *structs.Plan @@ -86,58 +91,11 @@ func (e *EvalCache) SemverConstraintCache() map[string]VerConstraints { return e.semverCache } -// PortCollisionEvent is an event that can happen during scheduling when -// an unexpected port collision is detected. -type PortCollisionEvent struct { - Reason string - Node *structs.Node - Allocations []*structs.Allocation - - // TODO: this is a large struct, but may be required to debug unexpected - // port collisions. Re-evaluate its need in the future if the bug is fixed - // or not caused by this field. - NetIndex *structs.NetworkIndex -} - -func (ev *PortCollisionEvent) Copy() *PortCollisionEvent { - if ev == nil { - return nil - } - c := new(PortCollisionEvent) - *c = *ev - c.Node = ev.Node.Copy() - if len(ev.Allocations) > 0 { - for i, a := range ev.Allocations { - c.Allocations[i] = a.Copy() - } - - } - c.NetIndex = ev.NetIndex.Copy() - return c -} - -func (ev *PortCollisionEvent) Sanitize() *PortCollisionEvent { - if ev == nil { - return nil - } - clean := ev.Copy() - - clean.Node = ev.Node.Sanitize() - clean.Node.Meta = make(map[string]string) - - for i, alloc := range ev.Allocations { - clean.Allocations[i] = alloc.CopySkipJob() - clean.Allocations[i].Job = nil - } - - return clean -} - // EvalContext is a Context used during an Evaluation type EvalContext struct { EvalCache eventsCh chan<- interface{} - state State + state sstructs.State plan *structs.Plan logger log.Logger metrics *structs.AllocMetric @@ -145,7 +103,7 @@ type EvalContext struct { } // NewEvalContext constructs a new EvalContext -func NewEvalContext(eventsCh chan<- interface{}, s State, p *structs.Plan, log log.Logger) *EvalContext { +func NewEvalContext(eventsCh chan<- interface{}, s sstructs.State, p *structs.Plan, log log.Logger) *EvalContext { ctx := &EvalContext{ eventsCh: eventsCh, state: s, @@ -156,7 +114,7 @@ func NewEvalContext(eventsCh chan<- interface{}, s State, p *structs.Plan, log l return ctx } -func (e *EvalContext) State() State { +func (e *EvalContext) State() sstructs.State { return e.state } @@ -172,7 +130,7 @@ func (e *EvalContext) Metrics() *structs.AllocMetric { return e.metrics } -func (e *EvalContext) SetState(s State) { +func (e *EvalContext) SetState(s sstructs.State) { e.state = s } @@ -432,3 +390,18 @@ func (e *EvalEligibility) SetQuotaLimitReached(quota string) { func (e *EvalEligibility) QuotaLimitReached() string { return e.quotaReached } + +func MockContext(t testing.TB) (*state.StateStore, *EvalContext) { + state := state.TestStateStore(t) + plan := &structs.Plan{ + EvalID: uuid.Generate(), + NodeUpdate: make(map[string][]*structs.Allocation), + NodeAllocation: make(map[string][]*structs.Allocation), + NodePreemptions: make(map[string][]*structs.Allocation), + } + + logger := testlog.HCLogger(t) + + ctx := NewEvalContext(nil, state, plan, logger) + return state, ctx +} diff --git a/scheduler/context_test.go b/scheduler/feasible/context_test.go similarity index 84% rename from scheduler/context_test.go rename to scheduler/feasible/context_test.go index 4346976e6..96551ef42 100644 --- a/scheduler/context_test.go +++ b/scheduler/feasible/context_test.go @@ -1,86 +1,26 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "testing" "github.com/hashicorp/nomad/ci" - "github.com/hashicorp/nomad/client/lib/idset" - "github.com/hashicorp/nomad/client/lib/numalib" - "github.com/hashicorp/nomad/client/lib/numalib/hw" - "github.com/hashicorp/nomad/helper/testlog" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" - "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" + sstructs "github.com/hashicorp/nomad/scheduler/structs" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test/must" ) -func testContext(t testing.TB) (*state.StateStore, *EvalContext) { - state := state.TestStateStore(t) - plan := &structs.Plan{ - EvalID: uuid.Generate(), - NodeUpdate: make(map[string][]*structs.Allocation), - NodeAllocation: make(map[string][]*structs.Allocation), - NodePreemptions: make(map[string][]*structs.Allocation), - } - - logger := testlog.HCLogger(t) - - ctx := NewEvalContext(nil, state, plan, logger) - return state, ctx -} - -// cpuResources creates both the legacy and modern structs concerning cpu -// metrics used for resource accounting -// -// only creates a trivial single node, single core system for the sake of -// compatibility with existing tests -func cpuResources(shares int) (structs.LegacyNodeCpuResources, structs.NodeProcessorResources) { - n := &structs.NodeResources{ - Processors: structs.NodeProcessorResources{ - Topology: &numalib.Topology{ - Distances: numalib.SLIT{[]numalib.Cost{10}}, - Cores: []numalib.Core{{ - SocketID: 0, - NodeID: 0, - ID: 0, - Grade: numalib.Performance, - Disable: false, - BaseSpeed: hw.MHz(shares), - }}, - }, - }, - } - n.Processors.Topology.SetNodes(idset.From[hw.NodeID]([]hw.NodeID{0})) - - // polyfill the legacy struct - n.Compatibility() - - return n.Cpu, n.Processors -} - -func cpuResourcesFrom(top *numalib.Topology) (structs.LegacyNodeCpuResources, structs.NodeProcessorResources) { - n := &structs.NodeResources{ - Processors: structs.NodeProcessorResources{ - Topology: top, - }, - } - - // polyfill the legacy struct - n.Compatibility() - - return n.Cpu, n.Processors -} - func TestEvalContext_ProposedAlloc(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) - legacyCpuResources, processorResources := cpuResources(2048) + legacyCpuResources, processorResources := tests.CpuResources(2048) nodes := []*RankedNode{ { @@ -208,9 +148,9 @@ func TestEvalContext_ProposedAlloc(t *testing.T) { // See https://github.com/hashicorp/nomad/issues/6787 func TestEvalContext_ProposedAlloc_EvictPreempt(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) - legacyCpuResources, processorResources := cpuResources(3 * 1024) + legacyCpuResources, processorResources := tests.CpuResources(3 * 1024) nodes := []*RankedNode{ { @@ -460,7 +400,7 @@ func TestEvalEligibility_GetClasses_JobEligible_TaskGroupIneligible(t *testing.T func TestPortCollisionEvent_Copy(t *testing.T) { ci.Parallel(t) - ev := &PortCollisionEvent{ + ev := &sstructs.PortCollisionEvent{ Reason: "original", Node: mock.Node(), Allocations: []*structs.Allocation{ @@ -492,7 +432,7 @@ func TestPortCollisionEvent_Copy(t *testing.T) { func TestPortCollisionEvent_Sanitize(t *testing.T) { ci.Parallel(t) - ev := &PortCollisionEvent{ + ev := &sstructs.PortCollisionEvent{ Reason: "original", Node: mock.Node(), Allocations: []*structs.Allocation{ diff --git a/scheduler/device.go b/scheduler/feasible/device.go similarity index 99% rename from scheduler/device.go rename to scheduler/feasible/device.go index ce479753a..4bb2e72f6 100644 --- a/scheduler/device.go +++ b/scheduler/feasible/device.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "fmt" diff --git a/scheduler/device_test.go b/scheduler/feasible/device_test.go similarity index 98% rename from scheduler/device_test.go rename to scheduler/feasible/device_test.go index 3345b48d3..4dd4d2b27 100644 --- a/scheduler/device_test.go +++ b/scheduler/feasible/device_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "testing" @@ -102,7 +102,7 @@ func collectInstanceIDs(devices ...*structs.NodeDeviceResource) []string { func TestDeviceAllocator_Allocate_GenericRequest(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) n := devNode() d := newDeviceAllocator(ctx, n) must.NotNil(t, d) @@ -125,7 +125,7 @@ func TestDeviceAllocator_Allocate_GenericRequest(t *testing.T) { func TestDeviceAllocator_Allocate_FullyQualifiedRequest(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) n := devNode() d := newDeviceAllocator(ctx, n) must.NotNil(t, d) @@ -148,7 +148,7 @@ func TestDeviceAllocator_Allocate_FullyQualifiedRequest(t *testing.T) { func TestDeviceAllocator_Allocate_NotEnoughInstances(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) n := devNode() d := newDeviceAllocator(ctx, n) must.NotNil(t, d) @@ -165,7 +165,7 @@ func TestDeviceAllocator_Allocate_NotEnoughInstances(t *testing.T) { func TestDeviceAllocator_Allocate_NUMA_available(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) n := devNode() d := newDeviceAllocator(ctx, n) @@ -184,7 +184,7 @@ func TestDeviceAllocator_Allocate_NUMA_available(t *testing.T) { func TestDeviceAllocator_Allocate_NUMA_node1(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) n := devNode() n.NodeResources.Devices = append(n.NodeResources.Devices, &structs.NodeDeviceResource{ Type: "fpga", @@ -323,7 +323,7 @@ func TestDeviceAllocator_Allocate_Constraints(t *testing.T) { for _, c := range cases { t.Run(c.Name+c.Note, func(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) d := newDeviceAllocator(ctx, n) must.NotNil(t, d) @@ -429,7 +429,7 @@ func TestDeviceAllocator_Allocate_Affinities(t *testing.T) { for _, c := range cases { t.Run(c.Name, func(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) d := newDeviceAllocator(ctx, n) must.NotNil(t, d) diff --git a/scheduler/feasible.go b/scheduler/feasible/feasible.go similarity index 98% rename from scheduler/feasible.go rename to scheduler/feasible/feasible.go index 21e260f32..722a982b0 100644 --- a/scheduler/feasible.go +++ b/scheduler/feasible/feasible.go @@ -1,11 +1,13 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "cmp" + "encoding/binary" "fmt" + "math/rand" "reflect" "regexp" "slices" @@ -124,13 +126,36 @@ func (iter *StaticIterator) SetNodes(nodes []*structs.Node) { iter.seen = 0 } +// ShuffleNodes randomizes the slice order with the Fisher-Yates +// algorithm. We seed the random source with the eval ID (which is +// random) to aid in postmortem debugging of specific evaluations and +// state snapshots. +func ShuffleNodes(plan *structs.Plan, index uint64, nodes []*structs.Node) { + + // use the last 4 bytes because those are the random bits + // if we have sortable IDs + buf := []byte(plan.EvalID) + seed := binary.BigEndian.Uint64(buf[len(buf)-8:]) + + // for retried plans the index is the plan result's RefreshIndex + // so that we don't retry with the exact same shuffle + seed ^= index + r := rand.New(rand.NewSource(int64(seed >> 2))) + + n := len(nodes) + for i := n - 1; i > 0; i-- { + j := r.Intn(i + 1) + nodes[i], nodes[j] = nodes[j], nodes[i] + } +} + // NewRandomIterator constructs a static iterator from a list of nodes // after applying the Fisher-Yates algorithm for a random shuffle. This // is applied in-place func NewRandomIterator(ctx Context, nodes []*structs.Node) *StaticIterator { // shuffle with the Fisher-Yates algorithm idx, _ := ctx.State().LatestIndex() - shuffleNodes(ctx.Plan(), idx, nodes) + ShuffleNodes(ctx.Plan(), idx, nodes) // Create a static iterator return NewStaticIterator(ctx, nodes) diff --git a/scheduler/feasible_test.go b/scheduler/feasible/feasible_test.go similarity index 98% rename from scheduler/feasible_test.go rename to scheduler/feasible/feasible_test.go index 20079487e..2947a47e0 100644 --- a/scheduler/feasible_test.go +++ b/scheduler/feasible/feasible_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "fmt" @@ -20,7 +20,7 @@ import ( func TestStaticIterator_Reset(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) var nodes []*structs.Node for i := 0; i < 3; i++ { nodes = append(nodes, mock.Node()) @@ -52,7 +52,7 @@ func TestStaticIterator_Reset(t *testing.T) { func TestStaticIterator_SetNodes(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) var nodes []*structs.Node for i := 0; i < 3; i++ { nodes = append(nodes, mock.Node()) @@ -70,7 +70,7 @@ func TestStaticIterator_SetNodes(t *testing.T) { func TestRandomIterator(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) var nodes []*structs.Node for i := 0; i < 10; i++ { nodes = append(nodes, mock.Node()) @@ -91,7 +91,7 @@ func TestRandomIterator(t *testing.T) { func TestHostVolumeChecker_Static(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -187,7 +187,7 @@ func TestHostVolumeChecker_Static(t *testing.T) { func TestHostVolumeChecker_Dynamic(t *testing.T) { ci.Parallel(t) - store, ctx := testContext(t) + store, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), @@ -375,7 +375,7 @@ func TestHostVolumeChecker_Dynamic(t *testing.T) { func TestHostVolumeChecker_Sticky(t *testing.T) { ci.Parallel(t) - store, ctx := testContext(t) + store, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), @@ -529,7 +529,7 @@ func TestHostVolumeChecker_Sticky(t *testing.T) { // hostVolumeIsAvailable method func TestDynamicHostVolumeIsAvailable(t *testing.T) { - store, ctx := testContext(t) + store, ctx := MockContext(t) allCaps := []*structs.HostVolumeCapability{} @@ -673,7 +673,7 @@ func TestDynamicHostVolumeIsAvailable(t *testing.T) { func TestCSIVolumeChecker(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -935,7 +935,7 @@ func TestCSIVolumeChecker(t *testing.T) { func TestNetworkChecker(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) node := func(mode string) *structs.Node { n := mock.Node() @@ -1081,7 +1081,7 @@ func TestNetworkChecker(t *testing.T) { func TestNetworkChecker_bridge_upgrade_path(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) t.Run("older client", func(t *testing.T) { // Create a client that is still on v0.11, which does not have the bridge @@ -1113,7 +1113,7 @@ func TestNetworkChecker_bridge_upgrade_path(t *testing.T) { func TestDriverChecker_DriverInfo(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -1164,7 +1164,7 @@ func TestDriverChecker_DriverInfo(t *testing.T) { func TestDriverChecker_Compatibility(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -1217,7 +1217,7 @@ func TestDriverChecker_Compatibility(t *testing.T) { func Test_HealthChecks(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), @@ -1281,7 +1281,7 @@ func Test_HealthChecks(t *testing.T) { func TestConstraintChecker(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -1558,7 +1558,7 @@ func TestCheckConstraint(t *testing.T) { } for _, tc := range cases { - _, ctx := testContext(t) + _, ctx := MockContext(t) if res := checkConstraint(ctx, tc.op, tc.lVal, tc.rVal, tc.lVal != nil, tc.rVal != nil); res != tc.result { t.Fatalf("TC: %#v, Result: %v", tc, res) } @@ -1672,7 +1672,7 @@ func TestCheckVersionConstraint(t *testing.T) { }, } for _, tc := range cases { - _, ctx := testContext(t) + _, ctx := MockContext(t) p := newVersionConstraintParser(ctx) if res := checkVersionMatch(p, tc.lVal, tc.rVal); res != tc.result { t.Fatalf("TC: %#v, Result: %v", tc, res) @@ -1754,7 +1754,7 @@ func TestCheckSemverConstraint(t *testing.T) { for _, tc := range cases { tc := tc t.Run(tc.name, func(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) p := newSemverConstraintParser(ctx) actual := checkVersionMatch(p, tc.lVal, tc.rVal) must.Eq(t, tc.result, actual) @@ -1792,7 +1792,7 @@ func TestCheckRegexpConstraint(t *testing.T) { }, } for _, tc := range cases { - _, ctx := testContext(t) + _, ctx := MockContext(t) if res := checkRegexpMatch(ctx, tc.lVal, tc.rVal); res != tc.result { t.Fatalf("TC: %#v, Result: %v", tc, res) } @@ -1804,7 +1804,7 @@ func TestCheckRegexpConstraint(t *testing.T) { func TestDistinctHostsIterator_JobDistinctHosts(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -1949,7 +1949,7 @@ func TestDistinctHostsIterator_JobDistinctHosts_Table(t *testing.T) { tc := tc ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) static := NewStaticIterator(ctx, []*structs.Node{n1, n2, n3}) j := job.Copy() @@ -1987,7 +1987,7 @@ func TestDistinctHostsIterator_JobDistinctHosts_Table(t *testing.T) { func TestDistinctHostsIterator_JobDistinctHosts_InfeasibleCount(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2040,7 +2040,7 @@ func TestDistinctHostsIterator_JobDistinctHosts_InfeasibleCount(t *testing.T) { func TestDistinctHostsIterator_TaskGroupDistinctHosts(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2110,7 +2110,7 @@ func TestDistinctHostsIterator_TaskGroupDistinctHosts(t *testing.T) { func TestDistinctPropertyIterator_JobDistinctProperty(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2292,7 +2292,7 @@ func TestDistinctPropertyIterator_JobDistinctProperty(t *testing.T) { func TestDistinctPropertyIterator_JobDistinctProperty_Count(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2501,7 +2501,7 @@ func TestDistinctPropertyIterator_JobDistinctProperty_Count(t *testing.T) { func TestDistinctPropertyIterator_JobDistinctProperty_RemoveAndReplace(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), } @@ -2585,7 +2585,7 @@ func TestDistinctPropertyIterator_JobDistinctProperty_RemoveAndReplace(t *testin func TestDistinctPropertyIterator_JobDistinctProperty_Infeasible(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2664,7 +2664,7 @@ func TestDistinctPropertyIterator_JobDistinctProperty_Infeasible(t *testing.T) { func TestDistinctPropertyIterator_JobDistinctProperty_Infeasible_Count(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2761,7 +2761,7 @@ func TestDistinctPropertyIterator_JobDistinctProperty_Infeasible_Count(t *testin func TestDistinctPropertyIterator_TaskGroupDistinctProperty(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -2924,7 +2924,7 @@ func (c *mockFeasibilityChecker) calls() int { return c.i } func TestFeasibilityWrapper_JobIneligible(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{mock.Node()} static := NewStaticIterator(ctx, nodes) mocked := newMockFeasibilityChecker(false) @@ -2944,7 +2944,7 @@ func TestFeasibilityWrapper_JobIneligible(t *testing.T) { func TestFeasibilityWrapper_JobEscapes(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{mock.Node()} static := NewStaticIterator(ctx, nodes) mocked := newMockFeasibilityChecker(false) @@ -2971,7 +2971,7 @@ func TestFeasibilityWrapper_JobEscapes(t *testing.T) { func TestFeasibilityWrapper_JobAndTg_Eligible(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{mock.Node()} static := NewStaticIterator(ctx, nodes) jobMock := newMockFeasibilityChecker(true) @@ -2995,7 +2995,7 @@ func TestFeasibilityWrapper_JobAndTg_Eligible(t *testing.T) { func TestFeasibilityWrapper_JobEligible_TgIneligible(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{mock.Node()} static := NewStaticIterator(ctx, nodes) jobMock := newMockFeasibilityChecker(true) @@ -3019,7 +3019,7 @@ func TestFeasibilityWrapper_JobEligible_TgIneligible(t *testing.T) { func TestFeasibilityWrapper_JobEligible_TgEscaped(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{mock.Node()} static := NewStaticIterator(ctx, nodes) jobMock := newMockFeasibilityChecker(true) @@ -3413,7 +3413,7 @@ func TestDeviceChecker(t *testing.T) { for _, c := range cases { t.Run(c.Name, func(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) checker := NewDeviceChecker(ctx) checker.SetTaskGroup(getTg(c.RequestedDevices...)) if act := checker.Feasible(getNode(c.NodeDevices...)); act != c.Result { @@ -3551,7 +3551,7 @@ func TestCheckAttributeConstraint(t *testing.T) { } for _, tc := range cases { - _, ctx := testContext(t) + _, ctx := MockContext(t) if res := checkAttributeConstraint(ctx, tc.op, tc.lVal, tc.rVal, tc.lVal != nil, tc.rVal != nil); res != tc.result { t.Fatalf("TC: %#v, Result: %v", tc, res) } diff --git a/scheduler/numa_ce.go b/scheduler/feasible/numa_ce.go similarity index 99% rename from scheduler/numa_ce.go rename to scheduler/feasible/numa_ce.go index ea616979d..22a809818 100644 --- a/scheduler/numa_ce.go +++ b/scheduler/feasible/numa_ce.go @@ -3,7 +3,7 @@ //go:build !ent -package scheduler +package feasible import ( "cmp" diff --git a/scheduler/numa_ce_test.go b/scheduler/feasible/numa_ce_test.go similarity index 99% rename from scheduler/numa_ce_test.go rename to scheduler/feasible/numa_ce_test.go index 3db23d4ea..7300e890a 100644 --- a/scheduler/numa_ce_test.go +++ b/scheduler/feasible/numa_ce_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "testing" diff --git a/scheduler/preemption.go b/scheduler/feasible/preemption.go similarity index 99% rename from scheduler/preemption.go rename to scheduler/feasible/preemption.go index 96e1321db..259ea46f1 100644 --- a/scheduler/preemption.go +++ b/scheduler/feasible/preemption.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "maps" diff --git a/scheduler/feasible/preemption_test.go b/scheduler/feasible/preemption_test.go new file mode 100644 index 000000000..649ab40ab --- /dev/null +++ b/scheduler/feasible/preemption_test.go @@ -0,0 +1,1422 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package feasible + +import ( + "fmt" + "strconv" + "testing" + + "github.com/hashicorp/nomad/ci" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + psstructs "github.com/hashicorp/nomad/plugins/shared/structs" + "github.com/hashicorp/nomad/scheduler/tests" + "github.com/shoenig/test/must" +) + +func TestResourceDistance(t *testing.T) { + ci.Parallel(t) + + resourceAsk := &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Cpu: structs.AllocatedCpuResources{ + CpuShares: 2048, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 512, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 4096, + }, + } + + type testCase struct { + allocResource *structs.ComparableResources + expectedDistance string + } + + testCases := []*testCase{ + { + &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Cpu: structs.AllocatedCpuResources{ + CpuShares: 2048, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 512, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 4096, + }, + }, + "0.000", + }, + { + &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Cpu: structs.AllocatedCpuResources{ + CpuShares: 1024, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 400, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 1024, + }, + }, + "0.928", + }, + { + &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Cpu: structs.AllocatedCpuResources{ + CpuShares: 8192, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 200, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 512, + }, + }, + }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 1024, + }, + }, + "3.152", + }, + { + &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Cpu: structs.AllocatedCpuResources{ + CpuShares: 2048, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 500, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 4096, + }, + }, + "0.023", + }, + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + actualDistance := fmt.Sprintf("%3.3f", basicResourceDistance(resourceAsk, tc.allocResource)) + must.Eq(t, tc.expectedDistance, actualDistance) + }) + + } + +} + +func makeDeviceInstance(instanceID, busID string) *structs.NodeDevice { + return &structs.NodeDevice{ + ID: instanceID, + Healthy: true, + Locality: &structs.NodeDeviceLocality{ + PciBusID: busID, + }, + } +} + +func TestPreemption_Normal(t *testing.T) { + ci.Parallel(t) + + type testCase struct { + desc string + currentAllocations []*structs.Allocation + nodeReservedCapacity *structs.NodeReservedResources + nodeCapacity *structs.NodeResources + resourceAsk *structs.Resources + jobPriority int + currentPreemptions []*structs.Allocation + preemptedAllocIDs map[string]struct{} + } + + highPrioJob := mock.Job() + highPrioJob.Priority = 100 + + lowPrioJob := mock.Job() + lowPrioJob.Priority = 30 + + lowPrioJob2 := mock.Job() + lowPrioJob2.Priority = 40 + + // Create some persistent alloc ids to use in test cases + allocIDs := []string{uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate()} + + var deviceIDs []string + for i := 0; i < 10; i++ { + deviceIDs = append(deviceIDs, "dev"+strconv.Itoa(i)) + } + + legacyCpuResources, processorResources := tests.CpuResources(4000) + + defaultNodeResources := &structs.NodeResources{ + Processors: processorResources, + Cpu: legacyCpuResources, + + Memory: structs.NodeMemoryResources{ + MemoryMB: 8192, + }, + Disk: structs.NodeDiskResources{ + DiskMB: 100 * 1024, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + CIDR: "192.168.0.100/32", + MBits: 1000, + }, + }, + Devices: []*structs.NodeDeviceResource{ + { + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + Attributes: map[string]*psstructs.Attribute{ + "memory": psstructs.NewIntAttribute(11, psstructs.UnitGiB), + "cuda_cores": psstructs.NewIntAttribute(3584, ""), + "graphics_clock": psstructs.NewIntAttribute(1480, psstructs.UnitMHz), + "memory_bandwidth": psstructs.NewIntAttribute(11, psstructs.UnitGBPerS), + }, + Instances: []*structs.NodeDevice{ + makeDeviceInstance(deviceIDs[0], "0000:00:00.0"), + makeDeviceInstance(deviceIDs[1], "0000:00:01.0"), + makeDeviceInstance(deviceIDs[2], "0000:00:02.0"), + makeDeviceInstance(deviceIDs[3], "0000:00:03.0"), + }, + }, + { + Type: "gpu", + Vendor: "nvidia", + Name: "2080ti", + Attributes: map[string]*psstructs.Attribute{ + "memory": psstructs.NewIntAttribute(11, psstructs.UnitGiB), + "cuda_cores": psstructs.NewIntAttribute(3584, ""), + "graphics_clock": psstructs.NewIntAttribute(1480, psstructs.UnitMHz), + "memory_bandwidth": psstructs.NewIntAttribute(11, psstructs.UnitGBPerS), + }, + Instances: []*structs.NodeDevice{ + makeDeviceInstance(deviceIDs[4], "0000:00:04.0"), + makeDeviceInstance(deviceIDs[5], "0000:00:05.0"), + makeDeviceInstance(deviceIDs[6], "0000:00:06.0"), + makeDeviceInstance(deviceIDs[7], "0000:00:07.0"), + makeDeviceInstance(deviceIDs[8], "0000:00:08.0"), + }, + }, + { + Type: "fpga", + Vendor: "intel", + Name: "F100", + Attributes: map[string]*psstructs.Attribute{ + "memory": psstructs.NewIntAttribute(4, psstructs.UnitGiB), + }, + Instances: []*structs.NodeDevice{ + makeDeviceInstance("fpga1", "0000:01:00.0"), + makeDeviceInstance("fpga2", "0000:02:01.0"), + }, + }, + }, + } + + reservedNodeResources := &structs.NodeReservedResources{ + Cpu: structs.NodeReservedCpuResources{ + CpuShares: 100, + }, + Memory: structs.NodeReservedMemoryResources{ + MemoryMB: 256, + }, + Disk: structs.NodeReservedDiskResources{ + DiskMB: 4 * 1024, + }, + } + + testCases := []testCase{ + { + desc: "No preemption because existing allocs are not low priority", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 3200, + MemoryMB: 7256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 50, + }, + }, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 2000, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "ssh", Value: 22}}, + MBits: 1, + }, + }, + }, + }, + { + desc: "Preempting low priority allocs not enough to meet resource ask", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 3200, + MemoryMB: 7256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 50, + }, + }, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 4000, + MemoryMB: 8192, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "ssh", Value: 22}}, + MBits: 1, + }, + }, + }, + }, + { + desc: "preemption impossible - static port needed is used by higher priority alloc", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], highPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 600, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 600, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 700, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }, + }, + { + desc: "preempt only from device that has allocation with unused reserved port", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], highPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth1", + IP: "192.168.0.200", + MBits: 600, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 600, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + // This test sets up a node with two NICs + + nodeCapacity: &structs.NodeResources{ + Processors: processorResources, + Cpu: legacyCpuResources, + Memory: structs.NodeMemoryResources{ + MemoryMB: 8192, + }, + Disk: structs.NodeDiskResources{ + DiskMB: 100 * 1024, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + CIDR: "192.168.0.100/32", + MBits: 1000, + }, + { + Device: "eth1", + CIDR: "192.168.1.100/32", + MBits: 1000, + }, + }, + }, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 600, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + IP: "192.168.0.100", + MBits: 700, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[2]: {}, + }, + }, + { + desc: "Combination of high/low priority allocs, without static ports", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 2800, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAllocWithTaskgroupNetwork(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 200, + }, + }, + }, &structs.NetworkResource{ + Device: "eth0", + IP: "192.168.0.201", + MBits: 300, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 300, + }, + }, + }), + createAlloc(allocIDs[3], lowPrioJob, &structs.Resources{ + CPU: 700, + MemoryMB: 256, + DiskMB: 4 * 1024, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1100, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 840, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + allocIDs[2]: {}, + allocIDs[3]: {}, + }, + }, + { + desc: "preempt allocs with network devices", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 2800, + MemoryMB: 2256, + DiskMB: 4 * 1024, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 800, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1100, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 840, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + }, + }, + { + desc: "ignore allocs with close enough priority for network devices", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 2800, + MemoryMB: 2256, + DiskMB: 4 * 1024, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 800, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: lowPrioJob.Priority + 5, + resourceAsk: &structs.Resources{ + CPU: 1100, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 840, + }, + }, + }, + preemptedAllocIDs: nil, + }, + { + desc: "Preemption needed for all resources except network", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 2800, + MemoryMB: 2256, + DiskMB: 40 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 50, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 512, + DiskMB: 25 * 1024, + }), + createAlloc(allocIDs[3], lowPrioJob, &structs.Resources{ + CPU: 700, + MemoryMB: 276, + DiskMB: 20 * 1024, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 3000, + DiskMB: 50 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 50, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + allocIDs[2]: {}, + allocIDs[3]: {}, + }, + }, + { + desc: "Only one low priority alloc needs to be preempted", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 500, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 320, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 300, + MemoryMB: 500, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 320, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[2]: {}, + }, + }, + { + desc: "one alloc meets static port need, another meets remaining mbits needed", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 500, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 200, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 2700, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 800, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + allocIDs[2]: {}, + }, + }, + { + desc: "alloc that meets static port need also meets other needs", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 600, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 100, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 600, + MemoryMB: 1000, + DiskMB: 25 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 700, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + }, + }, + { + desc: "alloc from job that has existing evictions not chosen for preemption", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 500, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob2, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 300, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 300, + MemoryMB: 500, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 320, + }, + }, + }, + currentPreemptions: []*structs.Allocation{ + createAlloc(allocIDs[4], lowPrioJob2, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 300, + }, + }, + }), + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + }, + }, + { + desc: "Preemption with one device instance per alloc", + // Add allocations that use two device instances + currentAllocations: []*structs.Allocation{ + createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 500, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[0]}, + }), + createAllocWithDevice(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[1]}, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 512, + DiskMB: 4 * 1024, + Devices: []*structs.RequestedDevice{ + { + Name: "nvidia/gpu/1080ti", + Count: 4, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[0]: {}, + allocIDs[1]: {}, + }, + }, + { + desc: "Preemption multiple devices used", + currentAllocations: []*structs.Allocation{ + createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 500, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[0], deviceIDs[1], deviceIDs[2], deviceIDs[3]}, + }), + createAllocWithDevice(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "fpga", + Vendor: "intel", + Name: "F100", + DeviceIDs: []string{"fpga1"}, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 512, + DiskMB: 4 * 1024, + Devices: []*structs.RequestedDevice{ + { + Name: "nvidia/gpu/1080ti", + Count: 4, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[0]: {}, + }, + }, + { + // This test cases creates allocations across two GPUs + // Both GPUs are eligible for the task, but only allocs sharing the + // same device should be chosen for preemption + desc: "Preemption with allocs across multiple devices that match", + currentAllocations: []*structs.Allocation{ + createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 500, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[0], deviceIDs[1]}, + }), + createAllocWithDevice(allocIDs[1], highPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 100, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[2]}, + }), + createAllocWithDevice(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "2080ti", + DeviceIDs: []string{deviceIDs[4], deviceIDs[5]}, + }), + createAllocWithDevice(allocIDs[3], lowPrioJob, &structs.Resources{ + CPU: 100, + MemoryMB: 256, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "2080ti", + DeviceIDs: []string{deviceIDs[6], deviceIDs[7]}, + }), + createAllocWithDevice(allocIDs[4], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "fpga", + Vendor: "intel", + Name: "F100", + DeviceIDs: []string{"fpga1"}, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 512, + DiskMB: 4 * 1024, + Devices: []*structs.RequestedDevice{ + { + Name: "gpu", + Count: 4, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[2]: {}, + allocIDs[3]: {}, + }, + }, + { + // This test cases creates allocations across two GPUs + // Both GPUs are eligible for the task, but only allocs with the lower + // priority are chosen + desc: "Preemption with lower/higher priority combinations", + currentAllocations: []*structs.Allocation{ + createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 500, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[0], deviceIDs[1]}, + }), + createAllocWithDevice(allocIDs[1], lowPrioJob2, &structs.Resources{ + CPU: 200, + MemoryMB: 100, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[2], deviceIDs[3]}, + }), + createAllocWithDevice(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "2080ti", + DeviceIDs: []string{deviceIDs[4], deviceIDs[5]}, + }), + createAllocWithDevice(allocIDs[3], lowPrioJob, &structs.Resources{ + CPU: 100, + MemoryMB: 256, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "2080ti", + DeviceIDs: []string{deviceIDs[6], deviceIDs[7]}, + }), + createAllocWithDevice(allocIDs[4], lowPrioJob, &structs.Resources{ + CPU: 100, + MemoryMB: 256, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "2080ti", + DeviceIDs: []string{deviceIDs[8]}, + }), + createAllocWithDevice(allocIDs[5], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "fpga", + Vendor: "intel", + Name: "F100", + DeviceIDs: []string{"fpga1"}, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 512, + DiskMB: 4 * 1024, + Devices: []*structs.RequestedDevice{ + { + Name: "gpu", + Count: 4, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[2]: {}, + allocIDs[3]: {}, + }, + }, + { + desc: "Device preemption not possible due to more instances needed than available", + currentAllocations: []*structs.Allocation{ + createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ + CPU: 500, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "gpu", + Vendor: "nvidia", + Name: "1080ti", + DeviceIDs: []string{deviceIDs[0], deviceIDs[1], deviceIDs[2], deviceIDs[3]}, + }), + createAllocWithDevice(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 512, + DiskMB: 4 * 1024, + }, &structs.AllocatedDeviceResource{ + Type: "fpga", + Vendor: "intel", + Name: "F100", + DeviceIDs: []string{"fpga1"}, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 512, + DiskMB: 4 * 1024, + Devices: []*structs.RequestedDevice{ + { + Name: "gpu", + Count: 6, + }, + }, + }, + }, + // This test case exercises the code path for a final filtering step that tries to + // minimize the number of preemptible allocations + { + desc: "Filter out allocs whose resource usage superset is also in the preemption list", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1800, + MemoryMB: 2256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 150, + }, + }, + }), + createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + CPU: 1500, + MemoryMB: 256, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 100, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 600, + MemoryMB: 256, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 300, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: defaultNodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 1000, + MemoryMB: 256, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 50, + }, + }, + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + node := mock.Node() + node.NodeResources = tc.nodeCapacity + node.ReservedResources = tc.nodeReservedCapacity + + state, ctx := MockContext(t) + + nodes := []*RankedNode{ + { + Node: node, + }, + } + state.UpsertNode(structs.MsgTypeTestSetup, 1000, node) + for _, alloc := range tc.currentAllocations { + alloc.NodeID = node.ID + } + err := state.UpsertAllocs(structs.MsgTypeTestSetup, 1001, tc.currentAllocations) + + must.NoError(t, err) + if tc.currentPreemptions != nil { + ctx.plan.NodePreemptions[node.ID] = tc.currentPreemptions + } + static := NewStaticRankIterator(ctx, nodes) + binPackIter := NewBinPackIterator(ctx, static, true, tc.jobPriority) + job := mock.Job() + job.Priority = tc.jobPriority + binPackIter.SetJob(job) + binPackIter.SetSchedulerConfiguration(testSchedulerConfig) + + taskGroup := &structs.TaskGroup{ + EphemeralDisk: &structs.EphemeralDisk{}, + Tasks: []*structs.Task{ + { + Name: "web", + Resources: tc.resourceAsk, + }, + }, + } + + binPackIter.SetTaskGroup(taskGroup) + option := binPackIter.Next() + if tc.preemptedAllocIDs == nil { + must.Nil(t, option) + } else { + must.NotNil(t, option) + preemptedAllocs := option.PreemptedAllocs + must.Eq(t, len(tc.preemptedAllocIDs), len(preemptedAllocs)) + for _, alloc := range preemptedAllocs { + _, ok := tc.preemptedAllocIDs[alloc.ID] + must.True(t, ok, must.Sprintf("alloc %s was preempted unexpectedly", alloc.ID)) + } + } + }) + } +} + +// helper method to create allocations with given jobs and resources +func createAlloc(id string, job *structs.Job, resource *structs.Resources) *structs.Allocation { + return createAllocInner(id, job, resource, nil, nil) +} + +// helper method to create allocation with network at the task group level +func createAllocWithTaskgroupNetwork(id string, job *structs.Job, resource *structs.Resources, tgNet *structs.NetworkResource) *structs.Allocation { + return createAllocInner(id, job, resource, nil, tgNet) +} + +func createAllocWithDevice(id string, job *structs.Job, resource *structs.Resources, allocatedDevices *structs.AllocatedDeviceResource) *structs.Allocation { + return createAllocInner(id, job, resource, allocatedDevices, nil) +} + +func createAllocInner(id string, job *structs.Job, resource *structs.Resources, allocatedDevices *structs.AllocatedDeviceResource, tgNetwork *structs.NetworkResource) *structs.Allocation { + alloc := &structs.Allocation{ + ID: id, + Job: job, + JobID: job.ID, + TaskResources: map[string]*structs.Resources{ + "web": resource, + }, + Namespace: structs.DefaultNamespace, + EvalID: uuid.Generate(), + DesiredStatus: structs.AllocDesiredStatusRun, + ClientStatus: structs.AllocClientStatusRunning, + TaskGroup: "web", + AllocatedResources: &structs.AllocatedResources{ + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: int64(resource.CPU), + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: int64(resource.MemoryMB), + }, + Networks: resource.Networks, + }, + }, + }, + } + + if allocatedDevices != nil { + alloc.AllocatedResources.Tasks["web"].Devices = []*structs.AllocatedDeviceResource{allocatedDevices} + } + + if tgNetwork != nil { + alloc.AllocatedResources.Shared = structs.AllocatedSharedResources{ + Networks: []*structs.NetworkResource{tgNetwork}, + } + } + return alloc +} diff --git a/scheduler/propertyset.go b/scheduler/feasible/propertyset.go similarity index 99% rename from scheduler/propertyset.go rename to scheduler/feasible/propertyset.go index 0dbe0c949..49fccaf58 100644 --- a/scheduler/propertyset.go +++ b/scheduler/feasible/propertyset.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "fmt" diff --git a/scheduler/rank.go b/scheduler/feasible/rank.go similarity index 99% rename from scheduler/rank.go rename to scheduler/feasible/rank.go index 31229c152..e2c941619 100644 --- a/scheduler/rank.go +++ b/scheduler/feasible/rank.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "fmt" @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/safemath" "github.com/hashicorp/nomad/nomad/structs" + sstructs "github.com/hashicorp/nomad/scheduler/structs" ) const ( @@ -230,7 +231,7 @@ NEXTNODE: // collect as much information as possible. netIdx := structs.NewNetworkIndex() if err := netIdx.SetNode(option.Node); err != nil { - iter.ctx.SendEvent(&PortCollisionEvent{ + iter.ctx.SendEvent(&sstructs.PortCollisionEvent{ Reason: err.Error(), NetIndex: netIdx.Copy(), Node: option.Node, @@ -239,7 +240,7 @@ NEXTNODE: continue } if collide, reason := netIdx.AddAllocs(proposed); collide { - event := &PortCollisionEvent{ + event := &sstructs.PortCollisionEvent{ Reason: reason, NetIndex: netIdx.Copy(), Node: option.Node, diff --git a/scheduler/rank_test.go b/scheduler/feasible/rank_test.go similarity index 98% rename from scheduler/rank_test.go rename to scheduler/feasible/rank_test.go index 04146b406..2657f492d 100644 --- a/scheduler/rank_test.go +++ b/scheduler/feasible/rank_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "sort" @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test" "github.com/shoenig/test/must" ) @@ -24,7 +25,7 @@ var testSchedulerConfig = &structs.SchedulerConfiguration{ } func TestFeasibleRankIterator(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) var nodes []*structs.Node for i := 0; i < 10; i++ { nodes = append(nodes, mock.Node()) @@ -40,13 +41,13 @@ func TestFeasibleRankIterator(t *testing.T) { } var ( - legacyCpuResources1024, processorResources1024 = cpuResources(1024) - legacyCpuResources2048, processorResources2048 = cpuResources(2048) - legacyCpuResources4096, processorResources4096 = cpuResources(4096) + legacyCpuResources1024, processorResources1024 = tests.CpuResources(1024) + legacyCpuResources2048, processorResources2048 = tests.CpuResources(2048) + legacyCpuResources4096, processorResources4096 = tests.CpuResources(4096) ) func TestBinPackIterator_NoExistingAlloc(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { @@ -150,11 +151,11 @@ func TestBinPackIterator_NoExistingAlloc(t *testing.T) { // reserved resources are scored equivalent to as if they had a lower amount of // resources. func TestBinPackIterator_NoExistingAlloc_MixedReserve(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) - legacyCpuResources900, processorResources900 := cpuResources(900) - legacyCpuResources1100, processorResources1100 := cpuResources(1100) - legacyCpuResources2000, processorResources2000 := cpuResources(2000) + legacyCpuResources900, processorResources900 := tests.CpuResources(900) + legacyCpuResources1100, processorResources1100 := tests.CpuResources(1100) + legacyCpuResources2000, processorResources2000 := tests.CpuResources(2000) nodes := []*RankedNode{ { @@ -267,7 +268,7 @@ func TestBinPackIterator_NoExistingAlloc_MixedReserve(t *testing.T) { // Tests bin packing iterator with network resources at task and task group level func TestBinPackIterator_Network_Success(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -393,7 +394,7 @@ func TestBinPackIterator_Network_Success(t *testing.T) { func TestBinPackIterator_Network_Failure(t *testing.T) { // Bandwidth tracking is deprecated t.Skip() - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -505,7 +506,7 @@ func TestBinPackIterator_Network_Failure(t *testing.T) { } func TestBinPackIterator_Network_NoCollision_Node(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) eventsCh := make(chan interface{}) ctx.eventsCh = eventsCh @@ -601,7 +602,7 @@ func TestBinPackIterator_Network_NoCollision_Node(t *testing.T) { // This should never happen as it indicates "bad" configuration was either not // caught by validation or caused by bugs in serverside Node handling. func TestBinPackIterator_Network_NodeError(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) eventsCh := make(chan interface{}) ctx.eventsCh = eventsCh @@ -697,7 +698,7 @@ func TestBinPackIterator_Network_NodeError(t *testing.T) { } func TestBinPackIterator_Network_PortCollision_Alloc(t *testing.T) { - state, ctx := testContext(t) + state, ctx := MockContext(t) eventsCh := make(chan interface{}) ctx.eventsCh = eventsCh @@ -823,7 +824,7 @@ func TestBinPackIterator_Network_PortCollision_Alloc(t *testing.T) { // Tests bin packing iterator with host network interpolation of task group level ports configuration func TestBinPackIterator_Network_Interpolation_Success(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -974,7 +975,7 @@ func TestBinPackIterator_Network_Interpolation_Success(t *testing.T) { // Tests that bin packing iterator fails due to absence of meta value // This test has network resources at task group func TestBinPackIterator_Host_Network_Interpolation_Absent_Value(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -1074,7 +1075,7 @@ func TestBinPackIterator_Host_Network_Interpolation_Absent_Value(t *testing.T) { // Tests that bin packing iterator fails due to absence of meta value // This test has network resources at task group func TestBinPackIterator_Host_Network_Interpolation_Interface_Not_Exists(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -1172,7 +1173,7 @@ func TestBinPackIterator_Host_Network_Interpolation_Interface_Not_Exists(t *test } func TestBinPackIterator_PlannedAlloc(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -1274,7 +1275,7 @@ func TestBinPackIterator_PlannedAlloc(t *testing.T) { } func TestBinPackIterator_ReservedCores(t *testing.T) { - state, ctx := testContext(t) + state, ctx := MockContext(t) topology := &numalib.Topology{ Distances: numalib.SLIT{[]numalib.Cost{10}}, @@ -1289,7 +1290,7 @@ func TestBinPackIterator_ReservedCores(t *testing.T) { }}, } topology.SetNodes(idset.From[hw.NodeID]([]hw.NodeID{0})) - legacyCpuResources, processorResources := cpuResourcesFrom(topology) + legacyCpuResources, processorResources := tests.CpuResourcesFrom(topology) nodes := []*RankedNode{ { @@ -1403,7 +1404,7 @@ func TestBinPackIterator_ReservedCores(t *testing.T) { } func TestBinPackIterator_ExistingAlloc(t *testing.T) { - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -1517,7 +1518,7 @@ func TestBinPackIterator_ExistingAlloc(t *testing.T) { } func TestBinPackIterator_ExistingAlloc_PlannedEvict(t *testing.T) { - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -1901,7 +1902,7 @@ func TestBinPackIterator_Devices(t *testing.T) { for _, c := range cases { t.Run(c.Name, func(t *testing.T) { // Setup the context - state, ctx := testContext(t) + state, ctx := MockContext(t) // Canonicalize resources for _, task := range c.TaskGroup.Tasks { @@ -1968,7 +1969,7 @@ func TestBinPackIterator_Devices(t *testing.T) { // Tests that bin packing iterator fails due to overprovisioning of devices // This test has devices at task level func TestBinPackIterator_Device_Failure_With_Eviction(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -2077,7 +2078,7 @@ func TestBinPackIterator_Device_Failure_With_Eviction(t *testing.T) { // Tests that bin packing iterator will not place workloads on nodes // that would go over a designated MaxAlloc value func TestBinPackIterator_MaxAlloc(t *testing.T) { - state, ctx := testContext(t) + state, ctx := MockContext(t) taskGen := func(name string) *structs.Task { return &structs.Task{ @@ -2207,7 +2208,7 @@ func TestBinPackIterator_MaxAlloc(t *testing.T) { } } func TestJobAntiAffinity_PlannedAlloc(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -2287,7 +2288,7 @@ func collectRanked(iter RankIterator) (out []*RankedNode) { } func TestNodeAntiAffinity_PenaltyNodes(t *testing.T) { - _, ctx := testContext(t) + _, ctx := MockContext(t) node1 := &structs.Node{ ID: uuid.Generate(), } @@ -2323,7 +2324,7 @@ func TestNodeAntiAffinity_PenaltyNodes(t *testing.T) { func TestScoreNormalizationIterator(t *testing.T) { // Test normalized scores when there is more than one scorer - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: &structs.Node{ @@ -2387,7 +2388,7 @@ func TestScoreNormalizationIterator(t *testing.T) { func TestNodeAffinityIterator(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) testNodes := func() []*RankedNode { nodes := []*RankedNode{ diff --git a/scheduler/select.go b/scheduler/feasible/select.go similarity index 99% rename from scheduler/select.go rename to scheduler/feasible/select.go index f16731b1a..34c2798d8 100644 --- a/scheduler/select.go +++ b/scheduler/feasible/select.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible // LimitIterator is a RankIterator used to limit the number of options // that are returned before we artificially end the stream. diff --git a/scheduler/select_test.go b/scheduler/feasible/select_test.go similarity index 98% rename from scheduler/select_test.go rename to scheduler/feasible/select_test.go index 5c1727e2e..0e73e189f 100644 --- a/scheduler/select_test.go +++ b/scheduler/feasible/select_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "testing" @@ -15,7 +15,7 @@ import ( func TestLimitIterator(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: mock.Node(), @@ -61,7 +61,7 @@ func TestLimitIterator(t *testing.T) { func TestLimitIterator_ScoreThreshold(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) type testCase struct { desc string nodes []*RankedNode @@ -326,7 +326,7 @@ func TestLimitIterator_ScoreThreshold(t *testing.T) { func TestMaxScoreIterator(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*RankedNode{ { Node: mock.Node(), diff --git a/scheduler/spread.go b/scheduler/feasible/spread.go similarity index 99% rename from scheduler/spread.go rename to scheduler/feasible/spread.go index 400cf9f64..dcafa6ee5 100644 --- a/scheduler/spread.go +++ b/scheduler/feasible/spread.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "github.com/hashicorp/nomad/helper" diff --git a/scheduler/feasible/spread_test.go b/scheduler/feasible/spread_test.go new file mode 100644 index 000000000..57d16c109 --- /dev/null +++ b/scheduler/feasible/spread_test.go @@ -0,0 +1,685 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package feasible + +import ( + "fmt" + "math" + "testing" + + "github.com/shoenig/test" + "github.com/shoenig/test/must" + + "github.com/hashicorp/go-set/v3" + "github.com/hashicorp/nomad/ci" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" +) + +func TestSpreadIterator_SingleAttribute(t *testing.T) { + ci.Parallel(t) + + state, ctx := MockContext(t) + dcs := []string{"dc1", "dc2", "dc1", "dc1"} + var nodes []*RankedNode + + // Add these nodes to the state store + for i, dc := range dcs { + node := mock.Node() + node.Datacenter = dc + if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { + t.Fatalf("failed to upsert node: %v", err) + } + nodes = append(nodes, &RankedNode{Node: node}) + } + + static := NewStaticRankIterator(ctx, nodes) + + job := mock.Job() + tg := job.TaskGroups[0] + job.TaskGroups[0].Count = 10 + // add allocs to nodes in dc1 + upserting := []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + EvalID: uuid.Generate(), + NodeID: nodes[0].Node.ID, + }, + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + EvalID: uuid.Generate(), + NodeID: nodes[2].Node.ID, + }, + } + + if err := state.UpsertAllocs(structs.MsgTypeTestSetup, 1000, upserting); err != nil { + t.Fatalf("failed to UpsertAllocs: %v", err) + } + + // Create spread target of 80% in dc1 + // Implicitly, this means 20% in dc2 + spread := &structs.Spread{ + Weight: 100, + Attribute: "${node.datacenter}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "dc1", + Percent: 80, + }, + }, + } + tg.Spreads = []*structs.Spread{spread} + spreadIter := NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + + scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) + + out := collectRanked(scoreNorm) + + // Expect nodes in dc1 with existing allocs to get a boost + // Boost should be ((desiredCount-actual)/desired)*spreadWeight + // For this test, that becomes dc1 = ((8-3)/8 ) = 0.5, and dc2=(2-1)/2 + expectedScores := map[string]float64{ + "dc1": 0.625, + "dc2": 0.5, + } + for _, rn := range out { + must.Eq(t, expectedScores[rn.Node.Datacenter], rn.FinalScore) + } + + // Update the plan to add more allocs to nodes in dc1 + // After this step there are enough allocs to meet the desired count in dc1 + ctx.plan.NodeAllocation[nodes[0].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[0].Node.ID, + }, + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[0].Node.ID, + }, + // Should be ignored as it is a different job. + { + Namespace: structs.DefaultNamespace, + TaskGroup: "bbb", + JobID: "ignore 2", + Job: job, + ID: uuid.Generate(), + NodeID: nodes[0].Node.ID, + }, + } + ctx.plan.NodeAllocation[nodes[3].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[3].Node.ID, + }, + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[3].Node.ID, + }, + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[3].Node.ID, + }, + } + + // Reset the scores + for _, node := range nodes { + node.Scores = nil + node.FinalScore = 0 + } + static = NewStaticRankIterator(ctx, nodes) + spreadIter = NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) + out = collectRanked(scoreNorm) + + // Expect nodes in dc2 with existing allocs to get a boost + // DC1 nodes are not boosted because there are enough allocs to meet + // the desired count + expectedScores = map[string]float64{ + "dc1": 0, + "dc2": 0.5, + } + for _, rn := range out { + must.Eq(t, expectedScores[rn.Node.Datacenter], rn.FinalScore) + } +} + +func TestSpreadIterator_MultipleAttributes(t *testing.T) { + ci.Parallel(t) + + state, ctx := MockContext(t) + dcs := []string{"dc1", "dc2", "dc1", "dc1"} + rack := []string{"r1", "r1", "r2", "r2"} + var nodes []*RankedNode + + // Add these nodes to the state store + for i, dc := range dcs { + node := mock.Node() + node.Datacenter = dc + node.Meta["rack"] = rack[i] + if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { + t.Fatalf("failed to upsert node: %v", err) + } + nodes = append(nodes, &RankedNode{Node: node}) + } + + static := NewStaticRankIterator(ctx, nodes) + + job := mock.Job() + tg := job.TaskGroups[0] + job.TaskGroups[0].Count = 10 + // add allocs to nodes in dc1 + upserting := []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + EvalID: uuid.Generate(), + NodeID: nodes[0].Node.ID, + }, + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + EvalID: uuid.Generate(), + NodeID: nodes[2].Node.ID, + }, + } + + if err := state.UpsertAllocs(structs.MsgTypeTestSetup, 1000, upserting); err != nil { + t.Fatalf("failed to UpsertAllocs: %v", err) + } + + spread1 := &structs.Spread{ + Weight: 100, + Attribute: "${node.datacenter}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "dc1", + Percent: 60, + }, + { + Value: "dc2", + Percent: 40, + }, + }, + } + + spread2 := &structs.Spread{ + Weight: 50, + Attribute: "${meta.rack}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "r1", + Percent: 40, + }, + { + Value: "r2", + Percent: 60, + }, + }, + } + + tg.Spreads = []*structs.Spread{spread1, spread2} + spreadIter := NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + + scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) + + out := collectRanked(scoreNorm) + + // Score comes from combining two different spread factors + // Second node should have the highest score because it has no allocs and its in dc2/r1 + expectedScores := map[string]float64{ + nodes[0].Node.ID: 0.500, + nodes[1].Node.ID: 0.667, + nodes[2].Node.ID: 0.556, + nodes[3].Node.ID: 0.556, + } + for _, rn := range out { + must.Eq(t, fmt.Sprintf("%.3f", expectedScores[rn.Node.ID]), fmt.Sprintf("%.3f", rn.FinalScore)) + } + +} + +func TestSpreadIterator_EvenSpread(t *testing.T) { + ci.Parallel(t) + + state, ctx := MockContext(t) + dcs := []string{"dc1", "dc2", "dc1", "dc2", "dc1", "dc2", "dc2", "dc1", "dc1", "dc1"} + var nodes []*RankedNode + + // Add these nodes to the state store + for i, dc := range dcs { + node := mock.Node() + node.Datacenter = dc + if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { + t.Fatalf("failed to upsert node: %v", err) + } + nodes = append(nodes, &RankedNode{Node: node}) + } + + static := NewStaticRankIterator(ctx, nodes) + job := mock.Job() + tg := job.TaskGroups[0] + job.TaskGroups[0].Count = 10 + + // Configure even spread across node.datacenter + spread := &structs.Spread{ + Weight: 100, + Attribute: "${node.datacenter}", + } + tg.Spreads = []*structs.Spread{spread} + spreadIter := NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + + scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) + + out := collectRanked(scoreNorm) + + // Nothing placed so both dc nodes get 0 as the score + expectedScores := map[string]float64{ + "dc1": 0, + "dc2": 0, + } + for _, rn := range out { + must.Eq(t, fmt.Sprintf("%.3f", expectedScores[rn.Node.Datacenter]), fmt.Sprintf("%.3f", rn.FinalScore)) + + } + + // Update the plan to add allocs to nodes in dc1 + // After this step dc2 nodes should get boosted + ctx.plan.NodeAllocation[nodes[0].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[0].Node.ID, + }, + } + ctx.plan.NodeAllocation[nodes[2].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[2].Node.ID, + }, + } + + // Reset the scores + for _, node := range nodes { + node.Scores = nil + node.FinalScore = 0 + } + static = NewStaticRankIterator(ctx, nodes) + spreadIter = NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) + out = collectRanked(scoreNorm) + + // Expect nodes in dc2 with existing allocs to get a boost + // dc1 nodes are penalized because they have allocs + expectedScores = map[string]float64{ + "dc1": -1, + "dc2": 1, + } + for _, rn := range out { + must.Eq(t, expectedScores[rn.Node.Datacenter], rn.FinalScore) + } + + // Update the plan to add more allocs to nodes in dc2 + // After this step dc1 nodes should get boosted + ctx.plan.NodeAllocation[nodes[1].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[1].Node.ID, + }, + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[1].Node.ID, + }, + } + ctx.plan.NodeAllocation[nodes[3].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[3].Node.ID, + }, + } + + // Reset the scores + for _, node := range nodes { + node.Scores = nil + node.FinalScore = 0 + } + static = NewStaticRankIterator(ctx, nodes) + spreadIter = NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) + out = collectRanked(scoreNorm) + + // Expect nodes in dc2 to be penalized because there are 3 allocs there now + // dc1 nodes are boosted because that has 2 allocs + expectedScores = map[string]float64{ + "dc1": 0.5, + "dc2": -0.5, + } + for _, rn := range out { + must.Eq(t, fmt.Sprintf("%3.3f", expectedScores[rn.Node.Datacenter]), fmt.Sprintf("%3.3f", rn.FinalScore)) + } + + // Add another node in dc3 + node := mock.Node() + node.Datacenter = "dc3" + if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(1111), node); err != nil { + t.Fatalf("failed to upsert node: %v", err) + } + nodes = append(nodes, &RankedNode{Node: node}) + + // Add another alloc to dc1, now its count matches dc2 + ctx.plan.NodeAllocation[nodes[4].Node.ID] = []*structs.Allocation{ + { + Namespace: structs.DefaultNamespace, + TaskGroup: tg.Name, + JobID: job.ID, + Job: job, + ID: uuid.Generate(), + NodeID: nodes[4].Node.ID, + }, + } + + // Reset scores + for _, node := range nodes { + node.Scores = nil + node.FinalScore = 0 + } + static = NewStaticRankIterator(ctx, nodes) + spreadIter = NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) + out = collectRanked(scoreNorm) + + // Expect dc1 and dc2 to be penalized because they have 3 allocs + // dc3 should get a boost because it has 0 allocs + expectedScores = map[string]float64{ + "dc1": -1, + "dc2": -1, + "dc3": 1, + } + for _, rn := range out { + must.Eq(t, fmt.Sprintf("%.3f", expectedScores[rn.Node.Datacenter]), fmt.Sprintf("%.3f", rn.FinalScore)) + } + +} + +// Test scenarios where the spread iterator sets maximum penalty (-1.0) +func TestSpreadIterator_MaxPenalty(t *testing.T) { + ci.Parallel(t) + + state, ctx := MockContext(t) + var nodes []*RankedNode + + // Add nodes in dc3 to the state store + for i := 0; i < 5; i++ { + node := mock.Node() + node.Datacenter = "dc3" + if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { + t.Fatalf("failed to upsert node: %v", err) + } + nodes = append(nodes, &RankedNode{Node: node}) + } + + static := NewStaticRankIterator(ctx, nodes) + + job := mock.Job() + tg := job.TaskGroups[0] + job.TaskGroups[0].Count = 5 + + // Create spread target of 80% in dc1 + // and 20% in dc2 + spread := &structs.Spread{ + Weight: 100, + Attribute: "${node.datacenter}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "dc1", + Percent: 80, + }, + { + Value: "dc2", + Percent: 20, + }, + }, + } + tg.Spreads = []*structs.Spread{spread} + spreadIter := NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + + scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) + + out := collectRanked(scoreNorm) + + // All nodes are in dc3 so score should be -1 + for _, rn := range out { + must.Eq(t, -1.0, rn.FinalScore) + } + + // Reset scores + for _, node := range nodes { + node.Scores = nil + node.FinalScore = 0 + } + + // Create spread on attribute that doesn't exist on any nodes + spread = &structs.Spread{ + Weight: 100, + Attribute: "${meta.foo}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "bar", + Percent: 80, + }, + { + Value: "baz", + Percent: 20, + }, + }, + } + + tg.Spreads = []*structs.Spread{spread} + static = NewStaticRankIterator(ctx, nodes) + spreadIter = NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) + out = collectRanked(scoreNorm) + + // All nodes don't have the spread attribute so score should be -1 + for _, rn := range out { + must.Eq(t, -1.0, rn.FinalScore) + } + +} + +func TestSpreadIterator_NoInfinity(t *testing.T) { + ci.Parallel(t) + + store, ctx := MockContext(t) + var nodes []*RankedNode + + // Add 3 nodes in different DCs to the state store + for i := 1; i < 4; i++ { + node := mock.Node() + node.Datacenter = fmt.Sprintf("dc%d", i) + must.NoError(t, store.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node)) + nodes = append(nodes, &RankedNode{Node: node}) + } + + static := NewStaticRankIterator(ctx, nodes) + + job := mock.Job() + tg := job.TaskGroups[0] + job.TaskGroups[0].Count = 8 + + // Create spread target of 50% in dc1, 50% in dc2, and 0% in the implicit target + spread := &structs.Spread{ + Weight: 100, + Attribute: "${node.datacenter}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "dc1", + Percent: 50, + }, + { + Value: "dc2", + Percent: 50, + }, + { + Value: "*", + Percent: 0, + }, + }, + } + tg.Spreads = []*structs.Spread{spread} + spreadIter := NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + + scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) + + out := collectRanked(scoreNorm) + + // Scores should be even between dc1 and dc2 nodes, without an -Inf on dc3 + must.Len(t, 3, out) + test.Eq(t, 0.75, out[0].FinalScore) + test.Eq(t, 0.75, out[1].FinalScore) + test.Eq(t, -1, out[2].FinalScore) + + // Reset scores + for _, node := range nodes { + node.Scores = nil + node.FinalScore = 0 + } + + // Create very unbalanced spread target to force large negative scores + spread = &structs.Spread{ + Weight: 100, + Attribute: "${node.datacenter}", + SpreadTarget: []*structs.SpreadTarget{ + { + Value: "dc1", + Percent: 99, + }, + { + Value: "dc2", + Percent: 1, + }, + { + Value: "*", + Percent: 0, + }, + }, + } + tg.Spreads = []*structs.Spread{spread} + static = NewStaticRankIterator(ctx, nodes) + spreadIter = NewSpreadIterator(ctx, static) + spreadIter.SetJob(job) + spreadIter.SetTaskGroup(tg) + + scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) + + out = collectRanked(scoreNorm) + + // Scores should heavily favor dc1, with an -Inf on dc3 + must.Len(t, 3, out) + desired := 8 * 0.99 // 8 allocs * 99% + test.Eq(t, (desired-1)/desired, out[0].FinalScore) + test.Eq(t, -11.5, out[1].FinalScore) + test.LessEq(t, out[1].FinalScore, out[2].FinalScore, + test.Sprintf("expected implicit dc3 to be <= dc2")) +} + +func Test_evenSpreadScoreBoost(t *testing.T) { + ci.Parallel(t) + + pset := &propertySet{ + existingValues: map[string]uint64{}, + proposedValues: map[string]uint64{ + "dc2": 1, + "dc1": 1, + "dc3": 1, + }, + clearedValues: map[string]uint64{ + "dc2": 1, + "dc3": 1, + }, + targetAttribute: "${node.datacenter}", + targetValues: &set.Set[string]{}, + } + + opt := &structs.Node{ + Datacenter: "dc2", + } + boost := evenSpreadScoreBoost(pset, opt) + must.False(t, math.IsInf(boost, 1)) + must.Eq(t, 1.0, boost) +} diff --git a/scheduler/stack.go b/scheduler/feasible/stack.go similarity index 93% rename from scheduler/stack.go rename to scheduler/feasible/stack.go index de8501b87..1dd0b17a8 100644 --- a/scheduler/stack.go +++ b/scheduler/feasible/stack.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "math" @@ -77,7 +77,7 @@ type GenericStack struct { func (s *GenericStack) SetNodes(baseNodes []*structs.Node) { // Shuffle base nodes idx, _ := s.ctx.State().LatestIndex() - shuffleNodes(s.ctx.Plan(), idx, baseNodes) + ShuffleNodes(s.ctx.Plan(), idx, baseNodes) // Update the set of base nodes s.source.SetNodes(baseNodes) @@ -153,11 +153,11 @@ func (s *GenericStack) Select(tg *structs.TaskGroup, options *SelectOptions) *Ra start := time.Now() // Get the task groups constraints. - tgConstr := taskGroupConstraints(tg) + tgConstr := TaskGroupConstraints(tg) // Update the parameters of iterators - s.taskGroupDrivers.SetDrivers(tgConstr.drivers) - s.taskGroupConstraint.SetConstraints(tgConstr.constraints) + s.taskGroupDrivers.SetDrivers(tgConstr.Drivers) + s.taskGroupConstraint.SetConstraints(tgConstr.Constraints) s.taskGroupDevices.SetTaskGroup(tg) s.taskGroupHostVolumes.SetVolumes(options.AllocName, s.jobNamespace, s.jobID, tg.Name, tg.Volumes) s.taskGroupCSIVolumes.SetVolumes(options.AllocName, tg.Volumes) @@ -348,11 +348,11 @@ func (s *SystemStack) Select(tg *structs.TaskGroup, options *SelectOptions) *Ran start := time.Now() // Get the task groups constraints. - tgConstr := taskGroupConstraints(tg) + tgConstr := TaskGroupConstraints(tg) // Update the parameters of iterators - s.taskGroupDrivers.SetDrivers(tgConstr.drivers) - s.taskGroupConstraint.SetConstraints(tgConstr.constraints) + s.taskGroupDrivers.SetDrivers(tgConstr.Drivers) + s.taskGroupConstraint.SetConstraints(tgConstr.Constraints) s.taskGroupDevices.SetTaskGroup(tg) s.taskGroupHostVolumes.SetVolumes(options.AllocName, s.jobNamespace, s.jobID, tg.Name, tg.Volumes) s.taskGroupCSIVolumes.SetVolumes(options.AllocName, tg.Volumes) @@ -474,28 +474,28 @@ func NewGenericStack(batch bool, ctx Context) *GenericStack { return s } -// taskGroupConstraints collects the constraints, drivers and resources required by each +// TaskGroupConstraints collects the constraints, drivers and resources required by each // sub-task to aggregate the TaskGroup totals -func taskGroupConstraints(tg *structs.TaskGroup) tgConstrainTuple { - c := tgConstrainTuple{ - constraints: make([]*structs.Constraint, 0, len(tg.Constraints)), - drivers: make(map[string]struct{}), +func TaskGroupConstraints(tg *structs.TaskGroup) TgConstrainTuple { + c := TgConstrainTuple{ + Constraints: make([]*structs.Constraint, 0, len(tg.Constraints)), + Drivers: make(map[string]struct{}), } - c.constraints = append(c.constraints, tg.Constraints...) + c.Constraints = append(c.Constraints, tg.Constraints...) for _, task := range tg.Tasks { - c.drivers[task.Driver] = struct{}{} - c.constraints = append(c.constraints, task.Constraints...) + c.Drivers[task.Driver] = struct{}{} + c.Constraints = append(c.Constraints, task.Constraints...) } return c } -// tgConstrainTuple is used to store the total constraints of a task group. -type tgConstrainTuple struct { - // Holds the combined constraints of the task group and all it's sub-tasks. - constraints []*structs.Constraint +// TgConstrainTuple is used to store the total constraints of a task group. +type TgConstrainTuple struct { + // Holds the combined Constraints of the task group and all it's sub-tasks. + Constraints []*structs.Constraint - // The set of required drivers within the task group. - drivers map[string]struct{} + // The set of required Drivers within the task group. + Drivers map[string]struct{} } diff --git a/scheduler/stack_ce.go b/scheduler/feasible/stack_ce.go similarity index 91% rename from scheduler/stack_ce.go rename to scheduler/feasible/stack_ce.go index e57b1436c..47665abdb 100644 --- a/scheduler/stack_ce.go +++ b/scheduler/feasible/stack_ce.go @@ -4,7 +4,7 @@ //go:build !ent // +build !ent -package scheduler +package feasible func NewQuotaIterator(_ Context, source FeasibleIterator) FeasibleIterator { return source diff --git a/scheduler/stack_test.go b/scheduler/feasible/stack_test.go similarity index 97% rename from scheduler/stack_test.go rename to scheduler/feasible/stack_test.go index 36df061c9..6f736ac3b 100644 --- a/scheduler/stack_test.go +++ b/scheduler/feasible/stack_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package feasible import ( "fmt" @@ -29,7 +29,7 @@ func BenchmarkServiceStack_WithOut_ComputedClass(b *testing.B) { // benchmarks the stack by selecting a job that constrains against one of the // partitions. func benchmarkServiceStack_MetaKeyConstraint(b *testing.B, key string, numNodes, nodePartitions int) { - _, ctx := testContext(b) + _, ctx := MockContext(b) stack := NewGenericStack(false, ctx) // Create 4 classes of nodes. @@ -60,7 +60,7 @@ func benchmarkServiceStack_MetaKeyConstraint(b *testing.B, key string, numNodes, func TestServiceStack_SetNodes(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) stack := NewGenericStack(false, ctx) nodes := []*structs.Node{ @@ -88,7 +88,7 @@ func TestServiceStack_SetNodes(t *testing.T) { func TestServiceStack_SetJob(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) stack := NewGenericStack(false, ctx) job := mock.Job() @@ -103,7 +103,7 @@ func TestServiceStack_SetJob(t *testing.T) { func TestServiceStack_Select_Size(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), } @@ -130,7 +130,7 @@ func TestServiceStack_Select_Size(t *testing.T) { func TestServiceStack_Select_PreferringNodes(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), } @@ -167,7 +167,7 @@ func TestServiceStack_Select_PreferringNodes(t *testing.T) { func TestServiceStack_Select_MetricsReset(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -197,7 +197,7 @@ func TestServiceStack_Select_MetricsReset(t *testing.T) { func TestServiceStack_Select_DriverFilter(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -223,7 +223,7 @@ func TestServiceStack_Select_DriverFilter(t *testing.T) { func TestServiceStack_Select_HostVolume(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) // Create nodes with host volumes and one without. node0 := mock.Node() @@ -317,7 +317,7 @@ func TestServiceStack_Select_HostVolume(t *testing.T) { func TestServiceStack_Select_CSI(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -389,7 +389,7 @@ func TestServiceStack_Select_CSI(t *testing.T) { func TestServiceStack_Select_ConstraintFilter(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -421,7 +421,7 @@ func TestServiceStack_Select_ConstraintFilter(t *testing.T) { func TestServiceStack_Select_BinPack_Overflow(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -455,7 +455,7 @@ func TestServiceStack_Select_BinPack_Overflow(t *testing.T) { func TestSystemStack_SetNodes(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) stack := NewSystemStack(false, ctx) nodes := []*structs.Node{ @@ -477,7 +477,7 @@ func TestSystemStack_SetNodes(t *testing.T) { func TestSystemStack_SetJob(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) stack := NewSystemStack(false, ctx) job := mock.Job() @@ -490,7 +490,7 @@ func TestSystemStack_SetJob(t *testing.T) { func TestSystemStack_Select_Size(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{mock.Node()} stack := NewSystemStack(false, ctx) stack.SetNodes(nodes) @@ -513,7 +513,7 @@ func TestSystemStack_Select_Size(t *testing.T) { func TestSystemStack_Select_MetricsReset(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -542,7 +542,7 @@ func TestSystemStack_Select_MetricsReset(t *testing.T) { func TestSystemStack_Select_DriverFilter(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), } @@ -574,7 +574,7 @@ func TestSystemStack_Select_DriverFilter(t *testing.T) { func TestSystemStack_Select_ConstraintFilter(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), @@ -606,7 +606,7 @@ func TestSystemStack_Select_ConstraintFilter(t *testing.T) { func TestSystemStack_Select_BinPack_Overflow(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := MockContext(t) nodes := []*structs.Node{ mock.Node(), mock.Node(), diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index e1fc9e3a5..8e42c9570 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -16,6 +16,7 @@ import ( "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/feasible" "github.com/hashicorp/nomad/scheduler/reconcile" sstructs "github.com/hashicorp/nomad/scheduler/structs" ) @@ -55,16 +56,16 @@ func (s *SetStatusError) Error() string { type GenericScheduler struct { logger log.Logger eventsCh chan<- interface{} - state State - planner Planner + state sstructs.State + planner sstructs.Planner batch bool eval *structs.Evaluation job *structs.Job plan *structs.Plan planResult *structs.PlanResult - ctx *EvalContext - stack *GenericStack + ctx *feasible.EvalContext + stack *feasible.GenericStack // followUpEvals are evals with WaitUntil set, which are delayed until that time // before being rescheduled @@ -78,7 +79,7 @@ type GenericScheduler struct { } // NewServiceScheduler is a factory function to instantiate a new service scheduler -func NewServiceScheduler(logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) Scheduler { +func NewServiceScheduler(logger log.Logger, eventsCh chan<- interface{}, state sstructs.State, planner sstructs.Planner) sstructs.Scheduler { s := &GenericScheduler{ logger: logger.Named("service_sched"), eventsCh: eventsCh, @@ -90,7 +91,7 @@ func NewServiceScheduler(logger log.Logger, eventsCh chan<- interface{}, state S } // NewBatchScheduler is a factory function to instantiate a new batch scheduler -func NewBatchScheduler(logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) Scheduler { +func NewBatchScheduler(logger log.Logger, eventsCh chan<- interface{}, state sstructs.State, planner sstructs.Planner) sstructs.Scheduler { s := &GenericScheduler{ logger: logger.Named("batch_sched"), eventsCh: eventsCh, @@ -233,10 +234,10 @@ func (s *GenericScheduler) process() (bool, error) { s.failedTGAllocs = nil // Create an evaluation context - s.ctx = NewEvalContext(s.eventsCh, s.state, s.plan, s.logger) + s.ctx = feasible.NewEvalContext(s.eventsCh, s.state, s.plan, s.logger) // Construct the placement stack - s.stack = NewGenericStack(s.batch, s.ctx) + s.stack = feasible.NewGenericStack(s.batch, s.ctx) if !s.job.Stopped() { s.setJob(s.job) } @@ -647,7 +648,7 @@ func (s *GenericScheduler) computePlacements( original := prevAllocation prevAllocation = prevAllocation.Copy() missing.SetPreviousAllocation(prevAllocation) - updateRescheduleTracker(alloc, prevAllocation, now) + UpdateRescheduleTracker(alloc, prevAllocation, now) swapAllocInPlan(s.plan, original, prevAllocation) } } @@ -761,8 +762,8 @@ func needsToSetNodes(a, b *structs.Job) bool { } // getSelectOptions sets up preferred nodes and penalty nodes -func getSelectOptions(prevAllocation *structs.Allocation, preferredNode *structs.Node) *SelectOptions { - selectOptions := &SelectOptions{} +func getSelectOptions(prevAllocation *structs.Allocation, preferredNode *structs.Node) *feasible.SelectOptions { + selectOptions := &feasible.SelectOptions{} if prevAllocation != nil { penaltyNodes := make(map[string]struct{}) @@ -793,11 +794,11 @@ func annotateRescheduleTracker(prev *structs.Allocation, note structs.Reschedule prev.RescheduleTracker.LastReschedule = note } -// updateRescheduleTracker carries over previous restart attempts and adds the +// UpdateRescheduleTracker carries over previous restart attempts and adds the // most recent restart. This mutates both allocations; "alloc" is a new // allocation so this is safe, but "prev" is coming from the state store and // must be copied first. -func updateRescheduleTracker(alloc *structs.Allocation, prev *structs.Allocation, now time.Time) { +func UpdateRescheduleTracker(alloc *structs.Allocation, prev *structs.Allocation, now time.Time) { reschedPolicy := prev.ReschedulePolicy() var rescheduleEvents []*structs.RescheduleEvent if prev.RescheduleTracker != nil { @@ -859,7 +860,7 @@ func (s *GenericScheduler) findPreferredNode(place reconcile.PlacementResult) (* } // selectNextOption calls the stack to get a node for placement -func (s *GenericScheduler) selectNextOption(tg *structs.TaskGroup, selectOptions *SelectOptions) *RankedNode { +func (s *GenericScheduler) selectNextOption(tg *structs.TaskGroup, selectOptions *feasible.SelectOptions) *feasible.RankedNode { option := s.stack.Select(tg, selectOptions) _, schedConfig, _ := s.ctx.State().SchedulerConfig() @@ -886,7 +887,7 @@ func (s *GenericScheduler) selectNextOption(tg *structs.TaskGroup, selectOptions } // handlePreemptions sets relevant preeemption related fields. -func (s *GenericScheduler) handlePreemptions(option *RankedNode, alloc *structs.Allocation, missing reconcile.PlacementResult) { +func (s *GenericScheduler) handlePreemptions(option *feasible.RankedNode, alloc *structs.Allocation, missing reconcile.PlacementResult) { if option.PreemptedAllocs == nil { return } diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 1984c6bda..c558132c8 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -19,6 +19,8 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/scheduler/reconcile" + sstructs "github.com/hashicorp/nomad/scheduler/structs" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test" "github.com/shoenig/test/must" ) @@ -26,7 +28,7 @@ import ( func TestServiceSched_JobRegister(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for range 10 { @@ -126,7 +128,7 @@ func TestServiceSched_JobRegister(t *testing.T) { func TestServiceSched_JobRegister_StickyAllocs(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -182,7 +184,7 @@ func TestServiceSched_JobRegister_StickyAllocs(t *testing.T) { Status: structs.EvalStatusPending, } must.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Evaluation{eval})) - h1 := NewHarnessWithState(t, h.State) + h1 := tests.NewHarnessWithState(t, h.State) if err := h1.Process(NewServiceScheduler, eval); err != nil { t.Fatalf("err: %v", err) } @@ -220,7 +222,7 @@ func TestServiceSched_JobRegister_StickyAllocs(t *testing.T) { func TestServiceSched_JobRegister_StickyHostVolumes(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) nodes := []*structs.Node{ mock.Node(), @@ -329,7 +331,7 @@ func TestServiceSched_JobRegister_StickyHostVolumes(t *testing.T) { func TestServiceSched_JobRegister_DiskConstraints(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -405,7 +407,7 @@ func TestServiceSched_JobRegister_DiskConstraints(t *testing.T) { func TestServiceSched_JobRegister_DistinctHosts(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -489,7 +491,7 @@ func TestServiceSched_JobRegister_DistinctHosts(t *testing.T) { func TestServiceSched_JobRegister_DistinctProperty(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -588,7 +590,7 @@ func TestServiceSched_JobRegister_DistinctProperty(t *testing.T) { func TestServiceSched_JobRegister_DistinctProperty_TaskGroup(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 2; i++ { @@ -669,7 +671,7 @@ func TestServiceSched_JobRegister_DistinctProperty_TaskGroup(t *testing.T) { func TestServiceSched_JobRegister_DistinctProperty_TaskGroup_Incr(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a job that uses distinct property over the node-id job := mock.Job() @@ -758,7 +760,7 @@ func TestServiceSched_Spread(t *testing.T) { for i := 0; i < 10; i++ { name := fmt.Sprintf("%d%% in dc1", start) t.Run(name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) remaining := uint8(100 - start) // Create a job that uses spread over data center job := mock.Job() @@ -853,7 +855,7 @@ func TestServiceSched_Spread(t *testing.T) { func TestServiceSched_JobRegister_Datacenter_Downgrade(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create 5 nodes in each datacenter. // Use two loops so nodes are separated by datacenter. @@ -963,7 +965,7 @@ func TestServiceSched_JobRegister_Datacenter_Downgrade(t *testing.T) { func TestServiceSched_JobRegister_NodePool_Downgrade(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Set global scheduler configuration. h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{ @@ -1094,7 +1096,7 @@ func TestServiceSched_JobRegister_NodePool_Downgrade(t *testing.T) { func TestServiceSched_EvenSpread(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a job that uses even spread over data center job := mock.Job() job.Datacenters = []string{"dc1", "dc2"} @@ -1167,7 +1169,7 @@ func TestServiceSched_EvenSpread(t *testing.T) { func TestServiceSched_JobRegister_Annotate(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -1247,7 +1249,7 @@ func TestServiceSched_JobRegister_Annotate(t *testing.T) { func TestServiceSched_JobRegister_CountZero(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -1298,7 +1300,7 @@ func TestServiceSched_JobRegister_CountZero(t *testing.T) { func TestServiceSched_JobRegister_AllocFail(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create NO nodes // Create a job @@ -1376,7 +1378,7 @@ func TestServiceSched_JobRegister_AllocFail(t *testing.T) { func TestServiceSched_JobRegister_CreateBlockedEval(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a full node node := mock.Node() @@ -1478,7 +1480,7 @@ func TestServiceSched_JobRegister_CreateBlockedEval(t *testing.T) { func TestServiceSched_JobRegister_FeasibleAndInfeasibleTG(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create one node node := mock.Node() @@ -1626,7 +1628,7 @@ func TestServiceSched_JobRegister_SchedulerAlgorithm(t *testing.T) { for _, jobType := range jobTypes { for _, tc := range testCases { t.Run(fmt.Sprintf("%s/%s", jobType, tc.name), func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) // Create node pools. nodePools := []*structs.NodePool{ @@ -1678,14 +1680,14 @@ func TestServiceSched_JobRegister_SchedulerAlgorithm(t *testing.T) { } must.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Evaluation{eval})) - var scheduler Factory + var sched sstructs.Factory switch jobType { case "batch": - scheduler = NewBatchScheduler + sched = NewBatchScheduler case "service": - scheduler = NewServiceScheduler + sched = NewServiceScheduler } - err := h.Process(scheduler, eval) + err := h.Process(sched, eval) must.NoError(t, err) must.Len(t, 1, h.Plans) @@ -1703,7 +1705,7 @@ func TestServiceSched_JobRegister_SchedulerAlgorithm(t *testing.T) { Status: structs.EvalStatusPending, } must.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Evaluation{eval})) - err = h.Process(scheduler, eval) + err = h.Process(sched, eval) must.NoError(t, err) must.Len(t, 2, h.Plans) @@ -1736,7 +1738,7 @@ func TestServiceSched_JobRegister_SchedulerAlgorithm(t *testing.T) { func TestServiceSched_EvaluateMaxPlanEval(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a job and set the task group count to zero. job := mock.Job() @@ -1773,10 +1775,10 @@ func TestServiceSched_EvaluateMaxPlanEval(t *testing.T) { func TestServiceSched_Plan_Partial_Progress(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node of limited resources - legacyCpuResources4000, processorResources4000 := cpuResources(4000) + legacyCpuResources4000, processorResources4000 := tests.CpuResources(4000) node := mock.Node() node.NodeResources.Processors = processorResources4000 node.NodeResources.Cpu = legacyCpuResources4000 @@ -1836,7 +1838,7 @@ func TestServiceSched_Plan_Partial_Progress(t *testing.T) { func TestServiceSched_EvaluateBlockedEval(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a job job := mock.Job() @@ -1883,7 +1885,7 @@ func TestServiceSched_EvaluateBlockedEval(t *testing.T) { func TestServiceSched_EvaluateBlockedEval_Finished(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -1970,7 +1972,7 @@ func TestServiceSched_EvaluateBlockedEval_Finished(t *testing.T) { func TestServiceSched_JobModify(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -2075,7 +2077,7 @@ func TestServiceSched_JobModify(t *testing.T) { func TestServiceSched_JobModify_ExistingDuplicateAllocIndex(t *testing.T) { ci.Parallel(t) - testHarness := NewHarness(t) + testHarness := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -2152,7 +2154,7 @@ func TestServiceSched_JobModify_ExistingDuplicateAllocIndex(t *testing.T) { func TestServiceSched_JobModify_ProposedDuplicateAllocIndex(t *testing.T) { ci.Parallel(t) - testHarness := NewHarness(t) + testHarness := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -2284,7 +2286,7 @@ func TestServiceSched_JobModify_ProposedDuplicateAllocIndex(t *testing.T) { func TestServiceSched_JobModify_ExistingDuplicateAllocIndexNonDestructive(t *testing.T) { ci.Parallel(t) - testHarness := NewHarness(t) + testHarness := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -2374,7 +2376,7 @@ func TestServiceSched_JobModify_ExistingDuplicateAllocIndexNonDestructive(t *tes func TestServiceSched_JobModify_Datacenters(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes in 3 DCs var nodes []*structs.Node @@ -2454,7 +2456,7 @@ func TestServiceSched_JobModify_Datacenters(t *testing.T) { func TestServiceSched_JobModify_IncrCount_NodeLimit(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create one node node := mock.Node() @@ -2548,7 +2550,7 @@ func TestServiceSched_JobModify_IncrCount_NodeLimit(t *testing.T) { func TestServiceSched_JobModify_CountZero(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -2650,7 +2652,7 @@ func TestServiceSched_JobModify_CountZero(t *testing.T) { func TestServiceSched_JobModify_Rolling(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -2758,7 +2760,7 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { func TestServiceSched_JobModify_Rolling_FullNode(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node and clear the reserved resources node := mock.Node() @@ -2880,7 +2882,7 @@ func TestServiceSched_JobModify_Rolling_FullNode(t *testing.T) { func TestServiceSched_JobModify_Canaries(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -3004,7 +3006,7 @@ func TestServiceSched_JobModify_Canaries(t *testing.T) { func TestServiceSched_JobModify_InPlace(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -3154,7 +3156,7 @@ func TestServiceSched_JobModify_InPlace(t *testing.T) { func TestServiceSched_JobModify_InPlace08(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create node node := mock.Node() @@ -3233,7 +3235,7 @@ func TestServiceSched_JobModify_InPlace08(t *testing.T) { func TestServiceSched_JobModify_DistinctProperty(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -3347,7 +3349,7 @@ func TestServiceSched_JobModify_DistinctProperty(t *testing.T) { func TestServiceSched_JobModify_NodeReschedulePenalty(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -3475,7 +3477,7 @@ func TestServiceSched_JobModify_NodeReschedulePenalty(t *testing.T) { func TestServiceSched_JobDeregister_Purged(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Generate a fake job with allocations job := mock.Job() @@ -3544,7 +3546,7 @@ func TestServiceSched_JobDeregister_Purged(t *testing.T) { func TestServiceSched_JobDeregister_Stopped(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Generate a fake job with allocations job := mock.Job() @@ -3671,7 +3673,7 @@ func TestServiceSched_NodeDown(t *testing.T) { for i, tc := range cases { t.Run(tc.name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) // Register a node node := mock.Node() @@ -3789,7 +3791,7 @@ func TestServiceSched_StopOnClientAfter(t *testing.T) { for i, tc := range cases { t.Run(tc.name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) // Node, which is down node := mock.Node() @@ -3906,7 +3908,7 @@ func TestServiceSched_StopOnClientAfter(t *testing.T) { func TestServiceSched_NodeUpdate(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a node node := mock.Node() @@ -3962,7 +3964,7 @@ func TestServiceSched_NodeUpdate(t *testing.T) { func TestServiceSched_NodeDrain(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -4045,7 +4047,7 @@ func TestServiceSched_NodeDrain(t *testing.T) { func TestServiceSched_NodeDrain_Down(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -4158,7 +4160,7 @@ func TestServiceSched_NodeDrain_Down(t *testing.T) { func TestServiceSched_NodeDrain_Canaries(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) n1 := mock.Node() n2 := mock.DrainNode() @@ -4270,7 +4272,7 @@ func TestServiceSched_NodeDrain_Canaries(t *testing.T) { func TestServiceSched_NodeDrain_Queued_Allocations(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.Node() @@ -4323,8 +4325,8 @@ func TestServiceSched_NodeDrain_Queued_Allocations(t *testing.T) { func TestServiceSched_RetryLimit(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) - h.Planner = &RejectPlan{h} + h := tests.NewHarness(t) + h.Planner = &tests.RejectPlan{h} // Create some nodes for i := 0; i < 10; i++ { @@ -4375,7 +4377,7 @@ func TestServiceSched_RetryLimit(t *testing.T) { func TestServiceSched_Reschedule_OnceNow(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -4488,7 +4490,7 @@ func TestServiceSched_Reschedule_OnceNow(t *testing.T) { func TestServiceSched_Reschedule_Later(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node for i := 0; i < 10; i++ { @@ -4577,7 +4579,7 @@ func TestServiceSched_Reschedule_Later(t *testing.T) { func TestServiceSched_Reschedule_MultipleNow(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -4718,7 +4720,7 @@ func TestServiceSched_Reschedule_MultipleNow(t *testing.T) { func TestServiceSched_BlockedReschedule(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) node := mock.Node() must.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node)) @@ -4915,7 +4917,7 @@ func TestServiceSched_BlockedReschedule(t *testing.T) { func TestServiceSched_Reschedule_PruneEvents(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -5049,7 +5051,7 @@ func TestDeployment_FailedAllocs_Reschedule(t *testing.T) { for _, failedDeployment := range []bool{false, true} { t.Run(fmt.Sprintf("Failed Deployment: %v", failedDeployment), func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node for i := 0; i < 10; i++ { @@ -5134,7 +5136,7 @@ func TestDeployment_FailedAllocs_Reschedule(t *testing.T) { func TestBatchSched_Run_CompleteAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -5193,7 +5195,7 @@ func TestBatchSched_Run_CompleteAlloc(t *testing.T) { func TestBatchSched_Run_FailedAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -5265,7 +5267,7 @@ func TestBatchSched_Run_FailedAlloc(t *testing.T) { func TestBatchSched_Run_LostAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -5354,7 +5356,7 @@ func TestBatchSched_Run_LostAlloc(t *testing.T) { func TestBatchSched_Run_FailedAllocQueuedAllocations(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) node := mock.DrainNode() must.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node)) @@ -5408,7 +5410,7 @@ func TestBatchSched_Run_FailedAllocQueuedAllocations(t *testing.T) { func TestBatchSched_ReRun_SuccessfullyFinishedAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one that is drained and has a successfully finished // alloc and a fresh undrained one @@ -5483,7 +5485,7 @@ func TestBatchSched_ReRun_SuccessfullyFinishedAlloc(t *testing.T) { func TestBatchSched_JobModify_InPlace_Terminal(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -5537,7 +5539,7 @@ func TestBatchSched_JobModify_InPlace_Terminal(t *testing.T) { func TestBatchSched_JobModify_Destructive_Terminal(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes var nodes []*structs.Node @@ -5623,7 +5625,7 @@ func TestBatchSched_JobModify_Destructive_Terminal(t *testing.T) { func TestBatchSched_NodeDrain_Running_OldJob(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one that is drained and has a successfully finished // alloc and a fresh undrained one @@ -5696,7 +5698,7 @@ func TestBatchSched_NodeDrain_Running_OldJob(t *testing.T) { func TestBatchSched_NodeDrain_Complete(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one that is drained and has a successfully finished // alloc and a fresh undrained one @@ -5762,7 +5764,7 @@ func TestBatchSched_NodeDrain_Complete(t *testing.T) { func TestBatchSched_ScaleDown_SameName(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -5916,9 +5918,9 @@ func TestGenericSched_AllocFit_Lifecycle(t *testing.T) { } for _, testCase := range testCases { t.Run(testCase.Name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) - legacyCpuResources, processorResources := cpuResources(testCase.NodeCpu) + legacyCpuResources, processorResources := tests.CpuResources(testCase.NodeCpu) node := mock.Node() node.NodeResources.Processors = processorResources node.NodeResources.Cpu = legacyCpuResources @@ -5967,7 +5969,7 @@ func TestGenericSched_AllocFit_Lifecycle(t *testing.T) { func TestGenericSched_AllocFit_MemoryOversubscription(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) node := mock.Node() node.NodeResources.Cpu.CpuShares = 10000 node.NodeResources.Memory.MemoryMB = 1224 @@ -6014,7 +6016,7 @@ func TestGenericSched_AllocFit_MemoryOversubscription(t *testing.T) { func TestGenericSched_ChainedAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -6050,7 +6052,7 @@ func TestGenericSched_ChainedAlloc(t *testing.T) { sort.Strings(allocIDs) // Create a new harness to invoke the scheduler again - h1 := NewHarnessWithState(t, h.State) + h1 := tests.NewHarnessWithState(t, h.State) job1 := mock.Job() job1.ID = job.ID job1.TaskGroups[0].Tasks[0].Env["foo"] = "bar" @@ -6105,7 +6107,7 @@ func TestGenericSched_ChainedAlloc(t *testing.T) { func TestServiceSched_NodeDrain_Sticky(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -6160,7 +6162,7 @@ func TestServiceSched_NodeDrain_Sticky(t *testing.T) { func TestServiceSched_CancelDeployment_Stopped(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Generate a fake job job := mock.Job() @@ -6236,7 +6238,7 @@ func TestServiceSched_CancelDeployment_Stopped(t *testing.T) { func TestServiceSched_CancelDeployment_NewerJob(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Generate a fake job job := mock.Job() @@ -6534,7 +6536,7 @@ func Test_updateRescheduleTracker(t *testing.T) { t.Run(tc.desc, func(t *testing.T) { prevAlloc.RescheduleTracker = &structs.RescheduleTracker{Events: tc.prevAllocEvents} prevAlloc.Job.LookupTaskGroup(prevAlloc.TaskGroup).ReschedulePolicy = tc.reschedPolicy - updateRescheduleTracker(alloc, prevAlloc, tc.reschedTime) + UpdateRescheduleTracker(alloc, prevAlloc, tc.reschedTime) must.Eq(t, tc.expectedRescheduleEvents, alloc.RescheduleTracker.Events) }) } @@ -6544,9 +6546,9 @@ func Test_updateRescheduleTracker(t *testing.T) { func TestServiceSched_Preemption(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) - legacyCpuResources, processorResources := cpuResources(1000) + legacyCpuResources, processorResources := tests.CpuResources(1000) // Create a node node := mock.Node() @@ -6707,7 +6709,7 @@ func TestServiceSched_Preemption(t *testing.T) { func TestServiceSched_Migrate_NonCanary(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) node1 := mock.Node() must.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node1)) @@ -6779,7 +6781,7 @@ func TestServiceSched_Migrate_NonCanary(t *testing.T) { func TestServiceSched_Migrate_CanaryStatus(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) node1 := mock.Node() must.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node1)) @@ -6948,7 +6950,7 @@ func TestServiceSched_Migrate_CanaryStatus(t *testing.T) { func TestDowngradedJobForPlacement_PicksTheLatest(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // This test tests downgradedJobForPlacement directly to ease testing many different scenarios // without invoking the full machinary of scheduling and updating deployment state tracking. @@ -7066,7 +7068,7 @@ func TestDowngradedJobForPlacement_PicksTheLatest(t *testing.T) { func TestServiceSched_RunningWithNextAllocation(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) node1 := mock.Node() must.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node1)) @@ -7140,7 +7142,7 @@ func TestServiceSched_RunningWithNextAllocation(t *testing.T) { func TestServiceSched_CSIVolumesPerAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes, each running the CSI plugin for i := 0; i < 5; i++ { @@ -7307,7 +7309,7 @@ func TestServiceSched_CSIVolumesPerAlloc(t *testing.T) { func TestServiceSched_CSITopology(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) zones := []string{"zone-0", "zone-1", "zone-2", "zone-3"} @@ -7414,7 +7416,7 @@ func TestServiceSched_Client_Disconnect_Creates_Updates_and_Evals(t *testing.T) for _, version := range jobVersions { t.Run(version.name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) count := 1 maxClientDisconnect := 10 * time.Minute @@ -7492,7 +7494,7 @@ func TestServiceSched_Client_Disconnect_Creates_Updates_and_Evals(t *testing.T) func TestServiceSched_ReservedCores_InPlace(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -7585,7 +7587,7 @@ func TestServiceSched_ReservedCores_InPlace(t *testing.T) { h.AssertEvalStatus(t, structs.EvalStatusComplete) } -func initNodeAndAllocs(t *testing.T, h *Harness, job *structs.Job, +func initNodeAndAllocs(t *testing.T, h *tests.Harness, job *structs.Job, nodeStatus, clientStatus string) (*structs.Node, *structs.Job, []*structs.Allocation) { // Node, which is ready node := mock.Node() diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go index a83ed85ae..35bf3a9fd 100644 --- a/scheduler/preemption_test.go +++ b/scheduler/preemption_test.go @@ -5,7 +5,6 @@ package scheduler import ( "fmt" - "strconv" "testing" "github.com/hashicorp/nomad/ci" @@ -13,1359 +12,10 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" psstructs "github.com/hashicorp/nomad/plugins/shared/structs" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test/must" ) -func TestResourceDistance(t *testing.T) { - ci.Parallel(t) - - resourceAsk := &structs.ComparableResources{ - Flattened: structs.AllocatedTaskResources{ - Cpu: structs.AllocatedCpuResources{ - CpuShares: 2048, - }, - Memory: structs.AllocatedMemoryResources{ - MemoryMB: 512, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, - }, - }, - }, - Shared: structs.AllocatedSharedResources{ - DiskMB: 4096, - }, - } - - type testCase struct { - allocResource *structs.ComparableResources - expectedDistance string - } - - testCases := []*testCase{ - { - &structs.ComparableResources{ - Flattened: structs.AllocatedTaskResources{ - Cpu: structs.AllocatedCpuResources{ - CpuShares: 2048, - }, - Memory: structs.AllocatedMemoryResources{ - MemoryMB: 512, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, - }, - }, - }, - Shared: structs.AllocatedSharedResources{ - DiskMB: 4096, - }, - }, - "0.000", - }, - { - &structs.ComparableResources{ - Flattened: structs.AllocatedTaskResources{ - Cpu: structs.AllocatedCpuResources{ - CpuShares: 1024, - }, - Memory: structs.AllocatedMemoryResources{ - MemoryMB: 400, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, - }, - }, - }, - Shared: structs.AllocatedSharedResources{ - DiskMB: 1024, - }, - }, - "0.928", - }, - { - &structs.ComparableResources{ - Flattened: structs.AllocatedTaskResources{ - Cpu: structs.AllocatedCpuResources{ - CpuShares: 8192, - }, - Memory: structs.AllocatedMemoryResources{ - MemoryMB: 200, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 512, - }, - }, - }, - Shared: structs.AllocatedSharedResources{ - DiskMB: 1024, - }, - }, - "3.152", - }, - { - &structs.ComparableResources{ - Flattened: structs.AllocatedTaskResources{ - Cpu: structs.AllocatedCpuResources{ - CpuShares: 2048, - }, - Memory: structs.AllocatedMemoryResources{ - MemoryMB: 500, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, - }, - }, - }, - Shared: structs.AllocatedSharedResources{ - DiskMB: 4096, - }, - }, - "0.023", - }, - } - - for _, tc := range testCases { - t.Run("", func(t *testing.T) { - actualDistance := fmt.Sprintf("%3.3f", basicResourceDistance(resourceAsk, tc.allocResource)) - must.Eq(t, tc.expectedDistance, actualDistance) - }) - - } - -} - -func makeDeviceInstance(instanceID, busID string) *structs.NodeDevice { - return &structs.NodeDevice{ - ID: instanceID, - Healthy: true, - Locality: &structs.NodeDeviceLocality{ - PciBusID: busID, - }, - } -} - -func TestPreemption_Normal(t *testing.T) { - ci.Parallel(t) - - type testCase struct { - desc string - currentAllocations []*structs.Allocation - nodeReservedCapacity *structs.NodeReservedResources - nodeCapacity *structs.NodeResources - resourceAsk *structs.Resources - jobPriority int - currentPreemptions []*structs.Allocation - preemptedAllocIDs map[string]struct{} - } - - highPrioJob := mock.Job() - highPrioJob.Priority = 100 - - lowPrioJob := mock.Job() - lowPrioJob.Priority = 30 - - lowPrioJob2 := mock.Job() - lowPrioJob2.Priority = 40 - - // Create some persistent alloc ids to use in test cases - allocIDs := []string{uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate()} - - var deviceIDs []string - for i := 0; i < 10; i++ { - deviceIDs = append(deviceIDs, "dev"+strconv.Itoa(i)) - } - - legacyCpuResources, processorResources := cpuResources(4000) - - defaultNodeResources := &structs.NodeResources{ - Processors: processorResources, - Cpu: legacyCpuResources, - - Memory: structs.NodeMemoryResources{ - MemoryMB: 8192, - }, - Disk: structs.NodeDiskResources{ - DiskMB: 100 * 1024, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - CIDR: "192.168.0.100/32", - MBits: 1000, - }, - }, - Devices: []*structs.NodeDeviceResource{ - { - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - Attributes: map[string]*psstructs.Attribute{ - "memory": psstructs.NewIntAttribute(11, psstructs.UnitGiB), - "cuda_cores": psstructs.NewIntAttribute(3584, ""), - "graphics_clock": psstructs.NewIntAttribute(1480, psstructs.UnitMHz), - "memory_bandwidth": psstructs.NewIntAttribute(11, psstructs.UnitGBPerS), - }, - Instances: []*structs.NodeDevice{ - makeDeviceInstance(deviceIDs[0], "0000:00:00.0"), - makeDeviceInstance(deviceIDs[1], "0000:00:01.0"), - makeDeviceInstance(deviceIDs[2], "0000:00:02.0"), - makeDeviceInstance(deviceIDs[3], "0000:00:03.0"), - }, - }, - { - Type: "gpu", - Vendor: "nvidia", - Name: "2080ti", - Attributes: map[string]*psstructs.Attribute{ - "memory": psstructs.NewIntAttribute(11, psstructs.UnitGiB), - "cuda_cores": psstructs.NewIntAttribute(3584, ""), - "graphics_clock": psstructs.NewIntAttribute(1480, psstructs.UnitMHz), - "memory_bandwidth": psstructs.NewIntAttribute(11, psstructs.UnitGBPerS), - }, - Instances: []*structs.NodeDevice{ - makeDeviceInstance(deviceIDs[4], "0000:00:04.0"), - makeDeviceInstance(deviceIDs[5], "0000:00:05.0"), - makeDeviceInstance(deviceIDs[6], "0000:00:06.0"), - makeDeviceInstance(deviceIDs[7], "0000:00:07.0"), - makeDeviceInstance(deviceIDs[8], "0000:00:08.0"), - }, - }, - { - Type: "fpga", - Vendor: "intel", - Name: "F100", - Attributes: map[string]*psstructs.Attribute{ - "memory": psstructs.NewIntAttribute(4, psstructs.UnitGiB), - }, - Instances: []*structs.NodeDevice{ - makeDeviceInstance("fpga1", "0000:01:00.0"), - makeDeviceInstance("fpga2", "0000:02:01.0"), - }, - }, - }, - } - - reservedNodeResources := &structs.NodeReservedResources{ - Cpu: structs.NodeReservedCpuResources{ - CpuShares: 100, - }, - Memory: structs.NodeReservedMemoryResources{ - MemoryMB: 256, - }, - Disk: structs.NodeReservedDiskResources{ - DiskMB: 4 * 1024, - }, - } - - testCases := []testCase{ - { - desc: "No preemption because existing allocs are not low priority", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 3200, - MemoryMB: 7256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 50, - }, - }, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 2000, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - ReservedPorts: []structs.Port{{Label: "ssh", Value: 22}}, - MBits: 1, - }, - }, - }, - }, - { - desc: "Preempting low priority allocs not enough to meet resource ask", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 3200, - MemoryMB: 7256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 50, - }, - }, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 4000, - MemoryMB: 8192, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - ReservedPorts: []structs.Port{{Label: "ssh", Value: 22}}, - MBits: 1, - }, - }, - }, - }, - { - desc: "preemption impossible - static port needed is used by higher priority alloc", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1200, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], highPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 600, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 600, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 700, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }, - }, - { - desc: "preempt only from device that has allocation with unused reserved port", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1200, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], highPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth1", - IP: "192.168.0.200", - MBits: 600, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 600, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - // This test sets up a node with two NICs - - nodeCapacity: &structs.NodeResources{ - Processors: processorResources, - Cpu: legacyCpuResources, - Memory: structs.NodeMemoryResources{ - MemoryMB: 8192, - }, - Disk: structs.NodeDiskResources{ - DiskMB: 100 * 1024, - }, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - CIDR: "192.168.0.100/32", - MBits: 1000, - }, - { - Device: "eth1", - CIDR: "192.168.1.100/32", - MBits: 1000, - }, - }, - }, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 600, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - IP: "192.168.0.100", - MBits: 700, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[2]: {}, - }, - }, - { - desc: "Combination of high/low priority allocs, without static ports", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 2800, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAllocWithTaskgroupNetwork(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 200, - }, - }, - }, &structs.NetworkResource{ - Device: "eth0", - IP: "192.168.0.201", - MBits: 300, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 300, - }, - }, - }), - createAlloc(allocIDs[3], lowPrioJob, &structs.Resources{ - CPU: 700, - MemoryMB: 256, - DiskMB: 4 * 1024, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1100, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 840, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - allocIDs[2]: {}, - allocIDs[3]: {}, - }, - }, - { - desc: "preempt allocs with network devices", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 2800, - MemoryMB: 2256, - DiskMB: 4 * 1024, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 800, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1100, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 840, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - }, - }, - { - desc: "ignore allocs with close enough priority for network devices", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 2800, - MemoryMB: 2256, - DiskMB: 4 * 1024, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 800, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: lowPrioJob.Priority + 5, - resourceAsk: &structs.Resources{ - CPU: 1100, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 840, - }, - }, - }, - preemptedAllocIDs: nil, - }, - { - desc: "Preemption needed for all resources except network", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 2800, - MemoryMB: 2256, - DiskMB: 40 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 50, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 512, - DiskMB: 25 * 1024, - }), - createAlloc(allocIDs[3], lowPrioJob, &structs.Resources{ - CPU: 700, - MemoryMB: 276, - DiskMB: 20 * 1024, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 3000, - DiskMB: 50 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 50, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - allocIDs[2]: {}, - allocIDs[3]: {}, - }, - }, - { - desc: "Only one low priority alloc needs to be preempted", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1200, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 500, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 320, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 300, - MemoryMB: 500, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 320, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[2]: {}, - }, - }, - { - desc: "one alloc meets static port need, another meets remaining mbits needed", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1200, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 500, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 200, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 2700, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 800, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - allocIDs[2]: {}, - }, - }, - { - desc: "alloc that meets static port need also meets other needs", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1200, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 600, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 100, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 600, - MemoryMB: 1000, - DiskMB: 25 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 700, - ReservedPorts: []structs.Port{ - { - Label: "db", - Value: 88, - }, - }, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - }, - }, - { - desc: "alloc from job that has existing evictions not chosen for preemption", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1200, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 500, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob2, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 300, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 300, - MemoryMB: 500, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 320, - }, - }, - }, - currentPreemptions: []*structs.Allocation{ - createAlloc(allocIDs[4], lowPrioJob2, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 300, - }, - }, - }), - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - }, - }, - { - desc: "Preemption with one device instance per alloc", - // Add allocations that use two device instances - currentAllocations: []*structs.Allocation{ - createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 500, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[0]}, - }), - createAllocWithDevice(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[1]}, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 512, - DiskMB: 4 * 1024, - Devices: []*structs.RequestedDevice{ - { - Name: "nvidia/gpu/1080ti", - Count: 4, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[0]: {}, - allocIDs[1]: {}, - }, - }, - { - desc: "Preemption multiple devices used", - currentAllocations: []*structs.Allocation{ - createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 500, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[0], deviceIDs[1], deviceIDs[2], deviceIDs[3]}, - }), - createAllocWithDevice(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "fpga", - Vendor: "intel", - Name: "F100", - DeviceIDs: []string{"fpga1"}, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 512, - DiskMB: 4 * 1024, - Devices: []*structs.RequestedDevice{ - { - Name: "nvidia/gpu/1080ti", - Count: 4, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[0]: {}, - }, - }, - { - // This test cases creates allocations across two GPUs - // Both GPUs are eligible for the task, but only allocs sharing the - // same device should be chosen for preemption - desc: "Preemption with allocs across multiple devices that match", - currentAllocations: []*structs.Allocation{ - createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 500, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[0], deviceIDs[1]}, - }), - createAllocWithDevice(allocIDs[1], highPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 100, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[2]}, - }), - createAllocWithDevice(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "2080ti", - DeviceIDs: []string{deviceIDs[4], deviceIDs[5]}, - }), - createAllocWithDevice(allocIDs[3], lowPrioJob, &structs.Resources{ - CPU: 100, - MemoryMB: 256, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "2080ti", - DeviceIDs: []string{deviceIDs[6], deviceIDs[7]}, - }), - createAllocWithDevice(allocIDs[4], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "fpga", - Vendor: "intel", - Name: "F100", - DeviceIDs: []string{"fpga1"}, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 512, - DiskMB: 4 * 1024, - Devices: []*structs.RequestedDevice{ - { - Name: "gpu", - Count: 4, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[2]: {}, - allocIDs[3]: {}, - }, - }, - { - // This test cases creates allocations across two GPUs - // Both GPUs are eligible for the task, but only allocs with the lower - // priority are chosen - desc: "Preemption with lower/higher priority combinations", - currentAllocations: []*structs.Allocation{ - createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 500, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[0], deviceIDs[1]}, - }), - createAllocWithDevice(allocIDs[1], lowPrioJob2, &structs.Resources{ - CPU: 200, - MemoryMB: 100, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[2], deviceIDs[3]}, - }), - createAllocWithDevice(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 256, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "2080ti", - DeviceIDs: []string{deviceIDs[4], deviceIDs[5]}, - }), - createAllocWithDevice(allocIDs[3], lowPrioJob, &structs.Resources{ - CPU: 100, - MemoryMB: 256, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "2080ti", - DeviceIDs: []string{deviceIDs[6], deviceIDs[7]}, - }), - createAllocWithDevice(allocIDs[4], lowPrioJob, &structs.Resources{ - CPU: 100, - MemoryMB: 256, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "2080ti", - DeviceIDs: []string{deviceIDs[8]}, - }), - createAllocWithDevice(allocIDs[5], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "fpga", - Vendor: "intel", - Name: "F100", - DeviceIDs: []string{"fpga1"}, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 512, - DiskMB: 4 * 1024, - Devices: []*structs.RequestedDevice{ - { - Name: "gpu", - Count: 4, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[2]: {}, - allocIDs[3]: {}, - }, - }, - { - desc: "Device preemption not possible due to more instances needed than available", - currentAllocations: []*structs.Allocation{ - createAllocWithDevice(allocIDs[0], lowPrioJob, &structs.Resources{ - CPU: 500, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "gpu", - Vendor: "nvidia", - Name: "1080ti", - DeviceIDs: []string{deviceIDs[0], deviceIDs[1], deviceIDs[2], deviceIDs[3]}, - }), - createAllocWithDevice(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 200, - MemoryMB: 512, - DiskMB: 4 * 1024, - }, &structs.AllocatedDeviceResource{ - Type: "fpga", - Vendor: "intel", - Name: "F100", - DeviceIDs: []string{"fpga1"}, - })}, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 512, - DiskMB: 4 * 1024, - Devices: []*structs.RequestedDevice{ - { - Name: "gpu", - Count: 6, - }, - }, - }, - }, - // This test case exercises the code path for a final filtering step that tries to - // minimize the number of preemptible allocations - { - desc: "Filter out allocs whose resource usage superset is also in the preemption list", - currentAllocations: []*structs.Allocation{ - createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ - CPU: 1800, - MemoryMB: 2256, - DiskMB: 4 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 150, - }, - }, - }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ - CPU: 1500, - MemoryMB: 256, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 100, - }, - }, - }), - createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ - CPU: 600, - MemoryMB: 256, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.200", - MBits: 300, - }, - }, - }), - }, - nodeReservedCapacity: reservedNodeResources, - nodeCapacity: defaultNodeResources, - jobPriority: 100, - resourceAsk: &structs.Resources{ - CPU: 1000, - MemoryMB: 256, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - IP: "192.168.0.100", - MBits: 50, - }, - }, - }, - preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, - }, - }, - } - - for _, tc := range testCases { - t.Run(tc.desc, func(t *testing.T) { - node := mock.Node() - node.NodeResources = tc.nodeCapacity - node.ReservedResources = tc.nodeReservedCapacity - - state, ctx := testContext(t) - - nodes := []*RankedNode{ - { - Node: node, - }, - } - state.UpsertNode(structs.MsgTypeTestSetup, 1000, node) - for _, alloc := range tc.currentAllocations { - alloc.NodeID = node.ID - } - err := state.UpsertAllocs(structs.MsgTypeTestSetup, 1001, tc.currentAllocations) - - must.NoError(t, err) - if tc.currentPreemptions != nil { - ctx.plan.NodePreemptions[node.ID] = tc.currentPreemptions - } - static := NewStaticRankIterator(ctx, nodes) - binPackIter := NewBinPackIterator(ctx, static, true, tc.jobPriority) - job := mock.Job() - job.Priority = tc.jobPriority - binPackIter.SetJob(job) - binPackIter.SetSchedulerConfiguration(testSchedulerConfig) - - taskGroup := &structs.TaskGroup{ - EphemeralDisk: &structs.EphemeralDisk{}, - Tasks: []*structs.Task{ - { - Name: "web", - Resources: tc.resourceAsk, - }, - }, - } - - binPackIter.SetTaskGroup(taskGroup) - option := binPackIter.Next() - if tc.preemptedAllocIDs == nil { - must.Nil(t, option) - } else { - must.NotNil(t, option) - preemptedAllocs := option.PreemptedAllocs - must.Eq(t, len(tc.preemptedAllocIDs), len(preemptedAllocs)) - for _, alloc := range preemptedAllocs { - _, ok := tc.preemptedAllocIDs[alloc.ID] - must.True(t, ok, must.Sprintf("alloc %s was preempted unexpectedly", alloc.ID)) - } - } - }) - } -} - // TestPreemptionMultiple tests evicting multiple allocations in the same time func TestPreemptionMultiple(t *testing.T) { ci.Parallel(t) @@ -1377,9 +27,9 @@ func TestPreemptionMultiple(t *testing.T) { // Then schedule a high priority job needing 2 allocs, using 2 GPUs each. // Expectation: // All low priority allocs should preempted to accomodate the high priority job - h := NewHarness(t) + h := tests.NewHarness(t) - legacyCpuResources, processorResources := cpuResources(4000) + legacyCpuResources, processorResources := tests.CpuResources(4000) // node with 4 GPUs node := mock.Node() @@ -1499,16 +149,6 @@ func TestPreemptionMultiple(t *testing.T) { must.Eq(t, allocIDs, preempted) } -// helper method to create allocations with given jobs and resources -func createAlloc(id string, job *structs.Job, resource *structs.Resources) *structs.Allocation { - return createAllocInner(id, job, resource, nil, nil) -} - -// helper method to create allocation with network at the task group level -func createAllocWithTaskgroupNetwork(id string, job *structs.Job, resource *structs.Resources, tgNet *structs.NetworkResource) *structs.Allocation { - return createAllocInner(id, job, resource, nil, tgNet) -} - func createAllocWithDevice(id string, job *structs.Job, resource *structs.Resources, allocatedDevices *structs.AllocatedDeviceResource) *structs.Allocation { return createAllocInner(id, job, resource, allocatedDevices, nil) } diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go index cb24ad2b3..3a06222ee 100644 --- a/scheduler/scheduler.go +++ b/scheduler/scheduler.go @@ -7,11 +7,7 @@ import ( "fmt" log "github.com/hashicorp/go-hclog" - - memdb "github.com/hashicorp/go-memdb" - "github.com/hashicorp/go-version" - "github.com/hashicorp/nomad/nomad/state" - "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/structs" ) const ( @@ -24,7 +20,7 @@ const ( // BuiltinSchedulers contains the built in registered schedulers // which are available -var BuiltinSchedulers = map[string]Factory{ +var BuiltinSchedulers = map[string]structs.Factory{ "service": NewServiceScheduler, "batch": NewBatchScheduler, "system": NewSystemScheduler, @@ -33,7 +29,9 @@ var BuiltinSchedulers = map[string]Factory{ // NewScheduler is used to instantiate and return a new scheduler // given the scheduler name, initial state, and planner. -func NewScheduler(name string, logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) (Scheduler, error) { +func NewScheduler( + name string, logger log.Logger, eventsCh chan<- interface{}, state structs.State, planner structs.Planner, +) (structs.Scheduler, error) { // Lookup the factory function factory, ok := BuiltinSchedulers[name] if !ok { @@ -44,119 +42,3 @@ func NewScheduler(name string, logger log.Logger, eventsCh chan<- interface{}, s sched := factory(logger, eventsCh, state, planner) return sched, nil } - -// Factory is used to instantiate a new Scheduler -type Factory func(log.Logger, chan<- interface{}, State, Planner) Scheduler - -// Scheduler is the top level instance for a scheduler. A scheduler is -// meant to only encapsulate business logic, pushing the various plumbing -// into Nomad itself. They are invoked to process a single evaluation at -// a time. The evaluation may result in task allocations which are computed -// optimistically, as there are many concurrent evaluations being processed. -// The task allocations are submitted as a plan, and the current leader will -// coordinate the commits to prevent oversubscription or improper allocations -// based on stale state. -type Scheduler interface { - // Process is used to handle a new evaluation. The scheduler is free to - // apply any logic necessary to make the task placements. The state and - // planner will be provided prior to any invocations of process. - Process(*structs.Evaluation) error -} - -// State is an immutable view of the global state. This allows schedulers -// to make intelligent decisions based on allocations of other schedulers -// and to enforce complex constraints that require more information than -// is available to a local state scheduler. -type State interface { - // Config returns the configuration of the state store - Config() *state.StateStoreConfig - - // Nodes returns an iterator over all the nodes. - // The type of each result is *structs.Node - Nodes(ws memdb.WatchSet) (memdb.ResultIterator, error) - - // NodesByNodePool returns an iterator over all nodes in the node pool - NodesByNodePool(ws memdb.WatchSet, poolName string) (memdb.ResultIterator, error) - - // NodePoolByName is used to lookup a node by ID. - NodePoolByName(ws memdb.WatchSet, poolName string) (*structs.NodePool, error) - - // AllocsByJob returns the allocations by JobID - AllocsByJob(ws memdb.WatchSet, namespace, jobID string, all bool) ([]*structs.Allocation, error) - - // AllocsByNode returns all the allocations by node - AllocsByNode(ws memdb.WatchSet, node string) ([]*structs.Allocation, error) - - // AllocByID returns the allocation - AllocByID(ws memdb.WatchSet, allocID string) (*structs.Allocation, error) - - // AllocsByNodeTerminal returns all the allocations by node filtering by terminal status - AllocsByNodeTerminal(ws memdb.WatchSet, node string, terminal bool) ([]*structs.Allocation, error) - - // NodeByID is used to lookup a node by ID - NodeByID(ws memdb.WatchSet, nodeID string) (*structs.Node, error) - - // GetJobByID is used to lookup a job by ID - JobByID(ws memdb.WatchSet, namespace, id string) (*structs.Job, error) - - // DeploymentsByJobID returns the deployments associated with the job - DeploymentsByJobID(ws memdb.WatchSet, namespace, jobID string, all bool) ([]*structs.Deployment, error) - - // JobByIDAndVersion returns the job associated with id and specific version - JobByIDAndVersion(ws memdb.WatchSet, namespace, id string, version uint64) (*structs.Job, error) - - // LatestDeploymentByJobID returns the latest deployment matching the given - // job ID - LatestDeploymentByJobID(ws memdb.WatchSet, namespace, jobID string) (*structs.Deployment, error) - - // SchedulerConfig returns config options for the scheduler - SchedulerConfig() (uint64, *structs.SchedulerConfiguration, error) - - // CSIVolumeByID fetch CSI volumes, containing controller jobs - CSIVolumeByID(memdb.WatchSet, string, string) (*structs.CSIVolume, error) - - // CSIVolumeByID fetch CSI volumes, containing controller jobs - CSIVolumesByNodeID(memdb.WatchSet, string, string) (memdb.ResultIterator, error) - - // HostVolumeByID fetches host volume by its ID - HostVolumeByID(memdb.WatchSet, string, string, bool) (*structs.HostVolume, error) - - // HostVolumesByNodeID gets an iterator with all the volumes attached to a - // given node - HostVolumesByNodeID(memdb.WatchSet, string, state.SortOption) (memdb.ResultIterator, error) - - // TaskGroupHostVolumeClaimsByFields gets all host volume claims for - // a given namespace, job ID and task group name - TaskGroupHostVolumeClaimsByFields(memdb.WatchSet, state.TgvcSearchableFields) (memdb.ResultIterator, error) - - // LatestIndex returns the greatest index value for all indexes. - LatestIndex() (uint64, error) -} - -// Planner interface is used to submit a task allocation plan. -type Planner interface { - // SubmitPlan is used to submit a plan for consideration. - // This will return a PlanResult or an error. It is possible - // that this will result in a state refresh as well. - SubmitPlan(*structs.Plan) (*structs.PlanResult, State, error) - - // UpdateEval is used to update an evaluation. This should update - // a copy of the input evaluation since that should be immutable. - UpdateEval(*structs.Evaluation) error - - // CreateEval is used to create an evaluation. This should set the - // PreviousEval to that of the current evaluation. - CreateEval(*structs.Evaluation) error - - // ReblockEval takes a blocked evaluation and re-inserts it into the blocked - // evaluation tracker. This update occurs only in-memory on the leader. The - // evaluation must exist in a blocked state prior to this being called such - // that on leader changes, the evaluation will be reblocked properly. - ReblockEval(*structs.Evaluation) error - - // ServersMeetMinimumVersion returns whether the Nomad servers in the - // worker's region are at least on the given Nomad version. The - // checkFailedServers parameter specifies whether version for the failed - // servers should be verified. - ServersMeetMinimumVersion(minVersion *version.Version, checkFailedServers bool) bool -} diff --git a/scheduler/scheduler_sysbatch_test.go b/scheduler/scheduler_sysbatch_test.go index d1862a687..d6d69c400 100644 --- a/scheduler/scheduler_sysbatch_test.go +++ b/scheduler/scheduler_sysbatch_test.go @@ -15,6 +15,7 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/kr/pretty" "github.com/shoenig/test/must" ) @@ -22,7 +23,7 @@ import ( func TestSysBatch_JobRegister(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes _ = createNodes(t, h, 10) @@ -95,7 +96,7 @@ func TestSysBatch_JobRegister(t *testing.T) { func TestSysBatch_JobRegister_AddNode_Running(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -172,7 +173,7 @@ func TestSysBatch_JobRegister_AddNode_Running(t *testing.T) { func TestSysBatch_JobRegister_AddNode_Dead(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -250,7 +251,7 @@ func TestSysBatch_JobRegister_AddNode_Dead(t *testing.T) { func TestSysBatch_JobModify(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -340,7 +341,7 @@ func TestSysBatch_JobModify(t *testing.T) { func TestSysBatch_JobModify_InPlace(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -414,7 +415,7 @@ func TestSysBatch_JobModify_InPlace(t *testing.T) { func TestSysBatch_JobDeregister_Purged(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -475,7 +476,7 @@ func TestSysBatch_JobDeregister_Purged(t *testing.T) { func TestSysBatch_JobDeregister_Stopped(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -538,7 +539,7 @@ func TestSysBatch_JobDeregister_Stopped(t *testing.T) { func TestSysBatch_NodeDown(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a down node node := mock.Node() @@ -599,7 +600,7 @@ func TestSysBatch_NodeDown(t *testing.T) { func TestSysBatch_NodeDrain_Down(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -653,7 +654,7 @@ func TestSysBatch_NodeDrain_Down(t *testing.T) { func TestSysBatch_NodeDrain(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -710,7 +711,7 @@ func TestSysBatch_NodeDrain(t *testing.T) { func TestSysBatch_NodeUpdate(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a node node := mock.Node() @@ -754,8 +755,8 @@ func TestSysBatch_NodeUpdate(t *testing.T) { func TestSysBatch_RetryLimit(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) - h.Planner = &RejectPlan{h} + h := tests.NewHarness(t) + h.Planner = &tests.RejectPlan{h} // Create some nodes _ = createNodes(t, h, 10) @@ -800,7 +801,7 @@ func TestSysBatch_RetryLimit(t *testing.T) { func TestSysBatch_Queued_With_Constraints(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) nodes := createNodes(t, h, 3) @@ -847,7 +848,7 @@ func TestSysBatch_Queued_With_Constraints(t *testing.T) { func TestSysBatch_Queued_With_Constraints_PartialMatch(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // linux machines linux := createNodes(t, h, 3) @@ -897,7 +898,7 @@ func TestSysBatch_Queued_With_Constraints_PartialMatch(t *testing.T) { func TestSysBatch_JobConstraint_AddNode(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes var node *structs.Node @@ -1037,7 +1038,7 @@ func TestSysBatch_JobConstraint_AddNode(t *testing.T) { func TestSysBatch_JobConstraint_AllFiltered(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one with a custom class node := mock.Node() @@ -1083,7 +1084,7 @@ func TestSysBatch_JobConstraint_AllFiltered(t *testing.T) { func TestSysBatch_JobConstraint_RunMultiple(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one with a custom class fooNode := mock.Node() @@ -1153,7 +1154,7 @@ func TestSysBatch_JobConstraint_RunMultiple(t *testing.T) { func TestSysBatch_ExistingAllocNoNodes(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) var node *structs.Node // Create a node @@ -1234,7 +1235,7 @@ func TestSysBatch_ExistingAllocNoNodes(t *testing.T) { func TestSysBatch_ConstraintErrors(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) var node *structs.Node // Register some nodes @@ -1309,7 +1310,7 @@ func TestSysBatch_ConstraintErrors(t *testing.T) { func TestSysBatch_ChainedAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes _ = createNodes(t, h, 10) @@ -1342,7 +1343,7 @@ func TestSysBatch_ChainedAlloc(t *testing.T) { sort.Strings(allocIDs) // Create a new harness to invoke the scheduler again - h1 := NewHarnessWithState(t, h.State) + h1 := tests.NewHarnessWithState(t, h.State) job1 := mock.SystemBatchJob() job1.ID = job.ID job1.TaskGroups[0].Tasks[0].Env = make(map[string]string) @@ -1398,7 +1399,7 @@ func TestSysBatch_ChainedAlloc(t *testing.T) { func TestSysBatch_PlanWithDrainedNode(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register two nodes with two different classes node := mock.DrainNode() @@ -1480,7 +1481,7 @@ func TestSysBatch_PlanWithDrainedNode(t *testing.T) { func TestSysBatch_QueuedAllocsMultTG(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register two nodes with two different classes node := mock.Node() @@ -1538,9 +1539,9 @@ func TestSysBatch_QueuedAllocsMultTG(t *testing.T) { func TestSysBatch_Preemption(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) - legacyCpuResources, processorResources := cpuResources(3072) + legacyCpuResources, processorResources := tests.CpuResources(3072) // Create nodes nodes := make([]*structs.Node, 0) @@ -1838,7 +1839,7 @@ func TestSysBatch_canHandle(t *testing.T) { must.True(t, s.canHandle(structs.EvalTriggerPeriodicJob)) }) } -func createNodes(t *testing.T, h *Harness, n int) []*structs.Node { +func createNodes(t *testing.T, h *tests.Harness, n int) []*structs.Node { nodes := make([]*structs.Node, n) for i := 0; i < n; i++ { node := mock.Node() diff --git a/scheduler/scheduler_system.go b/scheduler/scheduler_system.go index 96528a6c8..437f19518 100644 --- a/scheduler/scheduler_system.go +++ b/scheduler/scheduler_system.go @@ -11,6 +11,7 @@ import ( "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/feasible" "github.com/hashicorp/nomad/scheduler/reconcile" sstructs "github.com/hashicorp/nomad/scheduler/structs" ) @@ -33,16 +34,16 @@ const ( type SystemScheduler struct { logger log.Logger eventsCh chan<- interface{} - state State - planner Planner + state sstructs.State + planner sstructs.Planner sysbatch bool eval *structs.Evaluation job *structs.Job plan *structs.Plan planResult *structs.PlanResult - ctx *EvalContext - stack *SystemStack + ctx *feasible.EvalContext + stack *feasible.SystemStack nodes []*structs.Node notReadyNodes map[string]struct{} @@ -57,7 +58,7 @@ type SystemScheduler struct { // NewSystemScheduler is a factory function to instantiate a new system // scheduler. -func NewSystemScheduler(logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) Scheduler { +func NewSystemScheduler(logger log.Logger, eventsCh chan<- interface{}, state sstructs.State, planner sstructs.Planner) sstructs.Scheduler { return &SystemScheduler{ logger: logger.Named("system_sched"), eventsCh: eventsCh, @@ -67,7 +68,7 @@ func NewSystemScheduler(logger log.Logger, eventsCh chan<- interface{}, state St } } -func NewSysBatchScheduler(logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) Scheduler { +func NewSysBatchScheduler(logger log.Logger, eventsCh chan<- interface{}, state sstructs.State, planner sstructs.Planner) sstructs.Scheduler { return &SystemScheduler{ logger: logger.Named("sysbatch_sched"), eventsCh: eventsCh, @@ -153,10 +154,10 @@ func (s *SystemScheduler) process() (bool, error) { s.failedTGAllocs = nil // Create an evaluation context - s.ctx = NewEvalContext(s.eventsCh, s.state, s.plan, s.logger) + s.ctx = feasible.NewEvalContext(s.eventsCh, s.state, s.plan, s.logger) // Construct the placement stack - s.stack = NewSystemStack(s.sysbatch, s.ctx) + s.stack = feasible.NewSystemStack(s.sysbatch, s.ctx) if !s.job.Stopped() { s.setJob(s.job) } @@ -382,7 +383,7 @@ func (s *SystemScheduler) computePlacements(place []reconcile.AllocTuple, existi s.stack.SetNodes(nodes) // Attempt to match the task group - option := s.stack.Select(missing.TaskGroup, &SelectOptions{AllocName: missing.Name}) + option := s.stack.Select(missing.TaskGroup, &feasible.SelectOptions{AllocName: missing.Name}) if option == nil { // If the task can't be placed on this node, update reporting data @@ -391,7 +392,7 @@ func (s *SystemScheduler) computePlacements(place []reconcile.AllocTuple, existi // If this node was filtered because of constraint // mismatches and we couldn't create an allocation then // decrement queuedAllocs for that task group. - if s.ctx.metrics.NodesFiltered > 0 { + if s.ctx.Metrics().NodesFiltered > 0 { queued := s.queuedAllocs[tgName] - 1 s.queuedAllocs[tgName] = queued @@ -572,7 +573,7 @@ func (s *SystemScheduler) canHandle(trigger string) bool { // evictAndPlace is used to mark allocations for evicts and add them to the // placement queue. evictAndPlace modifies both the diffResult and the // limit. It returns true if the limit has been reached. -func evictAndPlace(ctx Context, diff *reconcile.NodeReconcileResult, allocs []reconcile.AllocTuple, desc string, limit *int) bool { +func evictAndPlace(ctx feasible.Context, diff *reconcile.NodeReconcileResult, allocs []reconcile.AllocTuple, desc string, limit *int) bool { n := len(allocs) for i := 0; i < n && i < *limit; i++ { a := allocs[i] diff --git a/scheduler/scheduler_system_test.go b/scheduler/scheduler_system_test.go index 1d8e809f1..d41969eef 100644 --- a/scheduler/scheduler_system_test.go +++ b/scheduler/scheduler_system_test.go @@ -16,14 +16,16 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/feasible" "github.com/hashicorp/nomad/scheduler/reconcile" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test/must" ) func TestSystemSched_JobRegister(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes _ = createNodes(t, h, 10) @@ -96,7 +98,7 @@ func TestSystemSched_JobRegister(t *testing.T) { func TestSystemSched_JobRegister_StickyAllocs(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes _ = createNodes(t, h, 10) @@ -147,7 +149,7 @@ func TestSystemSched_JobRegister_StickyAllocs(t *testing.T) { Status: structs.EvalStatusPending, } must.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Evaluation{eval})) - h1 := NewHarnessWithState(t, h.State) + h1 := tests.NewHarnessWithState(t, h.State) if err := h1.Process(NewSystemScheduler, eval); err != nil { t.Fatalf("err: %v", err) } @@ -171,7 +173,7 @@ func TestSystemSched_JobRegister_StickyAllocs(t *testing.T) { func TestSystemSched_JobRegister_EphemeralDiskConstraint(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -215,7 +217,7 @@ func TestSystemSched_JobRegister_EphemeralDiskConstraint(t *testing.T) { } // Create a new harness to test the scheduling result for the second job - h1 := NewHarnessWithState(t, h.State) + h1 := tests.NewHarnessWithState(t, h.State) // Create a mock evaluation to register the job eval1 := &structs.Evaluation{ Namespace: structs.DefaultNamespace, @@ -242,7 +244,7 @@ func TestSystemSched_JobRegister_EphemeralDiskConstraint(t *testing.T) { func TestSystemSched_ExhaustResources(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create a node node := mock.Node() @@ -320,7 +322,7 @@ func TestSystemSched_ExhaustResources(t *testing.T) { func TestSystemSched_JobRegister_Annotate(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes for i := 0; i < 10; i++ { @@ -419,7 +421,7 @@ func TestSystemSched_JobRegister_Annotate(t *testing.T) { func TestSystemSched_JobRegister_AddNode(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -499,7 +501,7 @@ func TestSystemSched_JobRegister_AddNode(t *testing.T) { func TestSystemSched_JobRegister_AllocFail(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create NO nodes // Create a job @@ -533,7 +535,7 @@ func TestSystemSched_JobRegister_AllocFail(t *testing.T) { func TestSystemSched_JobModify(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -622,7 +624,7 @@ func TestSystemSched_JobModify(t *testing.T) { func TestSystemSched_JobModify_Rolling(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -722,7 +724,7 @@ func TestSystemSched_JobModify_Rolling(t *testing.T) { func TestSystemSched_JobModify_InPlace(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -804,7 +806,7 @@ func TestSystemSched_JobModify_InPlace(t *testing.T) { func TestSystemSched_JobModify_RemoveDC(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes node1 := mock.Node() @@ -889,7 +891,7 @@ func TestSystemSched_JobModify_RemoveDC(t *testing.T) { func TestSystemSched_JobDeregister_Purged(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -950,7 +952,7 @@ func TestSystemSched_JobDeregister_Purged(t *testing.T) { func TestSystemSched_JobDeregister_Stopped(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes nodes := createNodes(t, h, 10) @@ -1013,7 +1015,7 @@ func TestSystemSched_JobDeregister_Stopped(t *testing.T) { func TestSystemSched_NodeDown(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a down node node := mock.Node() @@ -1074,7 +1076,7 @@ func TestSystemSched_NodeDown(t *testing.T) { func TestSystemSched_NodeDrain_Down(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -1128,7 +1130,7 @@ func TestSystemSched_NodeDrain_Down(t *testing.T) { func TestSystemSched_NodeDrain(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a draining node node := mock.DrainNode() @@ -1185,7 +1187,7 @@ func TestSystemSched_NodeDrain(t *testing.T) { func TestSystemSched_NodeUpdate(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a node node := mock.Node() @@ -1229,8 +1231,8 @@ func TestSystemSched_NodeUpdate(t *testing.T) { func TestSystemSched_RetryLimit(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) - h.Planner = &RejectPlan{h} + h := tests.NewHarness(t) + h.Planner = &tests.RejectPlan{h} // Create some nodes _ = createNodes(t, h, 10) @@ -1275,7 +1277,7 @@ func TestSystemSched_RetryLimit(t *testing.T) { func TestSystemSched_Queued_With_Constraints(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register a node node := mock.Node() @@ -1316,7 +1318,7 @@ func TestSystemSched_Queued_With_Constraints(t *testing.T) { func TestSystemSched_JobConstraint_AddNode(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes var node *structs.Node @@ -1458,7 +1460,7 @@ func TestSystemSched_JobConstraint_AddNode(t *testing.T) { func TestSystemSched_JobConstraint_AllFiltered(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one with a custom class node := mock.Node() @@ -1505,7 +1507,7 @@ func TestSystemSched_JobConstraint_AllFiltered(t *testing.T) { // subsequent runs, and report the outcome appropriately func TestSystemSched_JobConstraint_RunMultipleTimes(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create two nodes, one with a custom name fooNode := mock.Node() @@ -1545,14 +1547,14 @@ func TestSystemSched_JobConstraint_RunMultipleTimes(t *testing.T) { testCases := []struct { desc string job func() *structs.Job - check func(*testing.T, *Harness) + check func(*testing.T, *tests.Harness) }{ { desc: "Rerunning the job with constraint shouldn't report failed allocations", job: func() *structs.Job { return job }, - check: func(t *testing.T, h *Harness) { + check: func(t *testing.T, h *tests.Harness) { // Ensure a plan is not added, because no action should be taken must.Len(t, 1, h.Plans) @@ -1574,7 +1576,7 @@ func TestSystemSched_JobConstraint_RunMultipleTimes(t *testing.T) { must.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job)) return job }, - check: func(t *testing.T, h *Harness) { + check: func(t *testing.T, h *tests.Harness) { // Ensure another plan is added for the updated alloc must.Len(t, 2, h.Plans) @@ -1601,7 +1603,7 @@ func TestSystemSched_JobConstraint_RunMultipleTimes(t *testing.T) { must.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job)) return job }, - check: func(t *testing.T, h *Harness) { + check: func(t *testing.T, h *tests.Harness) { // Ensure another plan is added for the updated alloc must.Len(t, 3, h.Plans) @@ -1625,7 +1627,7 @@ func TestSystemSched_JobConstraint_RunMultipleTimes(t *testing.T) { must.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job)) return job }, - check: func(t *testing.T, h *Harness) { + check: func(t *testing.T, h *tests.Harness) { // Ensure another Plan for the updated alloc must.Len(t, 4, h.Plans) @@ -1661,7 +1663,7 @@ func TestSystemSched_JobConstraint_RunMultipleTimes(t *testing.T) { func TestSystemSched_ExistingAllocNoNodes(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) var node *structs.Node // Create a node @@ -1742,7 +1744,7 @@ func TestSystemSched_ExistingAllocNoNodes(t *testing.T) { func TestSystemSched_ConstraintErrors(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) var node *structs.Node // Register some nodes @@ -1815,7 +1817,7 @@ func TestSystemSched_ConstraintErrors(t *testing.T) { func TestSystemSched_ChainedAlloc(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Create some nodes _ = createNodes(t, h, 10) @@ -1848,7 +1850,7 @@ func TestSystemSched_ChainedAlloc(t *testing.T) { sort.Strings(allocIDs) // Create a new harness to invoke the scheduler again - h1 := NewHarnessWithState(t, h.State) + h1 := tests.NewHarnessWithState(t, h.State) job1 := mock.SystemJob() job1.ID = job.ID job1.TaskGroups[0].Tasks[0].Env = make(map[string]string) @@ -1905,7 +1907,7 @@ func TestSystemSched_ChainedAlloc(t *testing.T) { func TestSystemSched_PlanWithDrainedNode(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register two nodes with two different classes node := mock.DrainNode() @@ -1987,7 +1989,7 @@ func TestSystemSched_PlanWithDrainedNode(t *testing.T) { func TestSystemSched_QueuedAllocsMultTG(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) // Register two nodes with two different classes node := mock.Node() @@ -2045,9 +2047,9 @@ func TestSystemSched_QueuedAllocsMultTG(t *testing.T) { func TestSystemSched_Preemption(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) - legacyCpuResources, processorResources := cpuResources(3072) + legacyCpuResources, processorResources := tests.CpuResources(3072) // Create nodes nodes := make([]*structs.Node, 0) @@ -3058,7 +3060,7 @@ func TestSystemSched_NodeDisconnected(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) // Register a node node := mock.Node() @@ -3203,7 +3205,7 @@ func TestSystemSched_NodeDisconnected(t *testing.T) { func TestSystemSched_CSITopology(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) zones := []string{"zone-0", "zone-1", "zone-2", "zone-3"} @@ -3291,7 +3293,7 @@ func TestSystemSched_CSITopology(t *testing.T) { func TestEvictAndPlace_LimitLessThanAllocs(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := feasible.MockContext(t) allocs := []reconcile.AllocTuple{ {Alloc: &structs.Allocation{ID: uuid.Generate()}}, {Alloc: &structs.Allocation{ID: uuid.Generate()}}, @@ -3312,7 +3314,7 @@ func TestEvictAndPlace_LimitLessThanAllocs(t *testing.T) { func TestEvictAndPlace_LimitEqualToAllocs(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := feasible.MockContext(t) allocs := []reconcile.AllocTuple{ {Alloc: &structs.Allocation{ID: uuid.Generate()}}, {Alloc: &structs.Allocation{ID: uuid.Generate()}}, @@ -3332,7 +3334,7 @@ func TestEvictAndPlace_LimitEqualToAllocs(t *testing.T) { func TestEvictAndPlace_LimitGreaterThanAllocs(t *testing.T) { ci.Parallel(t) - _, ctx := testContext(t) + _, ctx := feasible.MockContext(t) allocs := []reconcile.AllocTuple{ {Alloc: &structs.Allocation{ID: uuid.Generate()}}, {Alloc: &structs.Allocation{ID: uuid.Generate()}}, diff --git a/scheduler/scheduler_test.go b/scheduler/scheduler_test.go index a90881e55..f88368ce1 100644 --- a/scheduler/scheduler_test.go +++ b/scheduler/scheduler_test.go @@ -13,6 +13,8 @@ import ( "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + sstructs "github.com/hashicorp/nomad/scheduler/structs" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test/must" ) @@ -114,7 +116,7 @@ func TestScheduler_JobRegister_MemoryMaxHonored(t *testing.T) { for _, jobType := range jobTypes { for _, c := range cases { t.Run(fmt.Sprintf("%s/%s", jobType, c.name), func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) // Create node pools. nodePools := []*structs.NodePool{ @@ -172,7 +174,7 @@ func TestScheduler_JobRegister_MemoryMaxHonored(t *testing.T) { must.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Evaluation{eval})) // Process the evaluation - var scheduler Factory + var scheduler sstructs.Factory switch jobType { case "batch": scheduler = NewBatchScheduler diff --git a/scheduler/spread_test.go b/scheduler/spread_test.go index 70725d630..50d12db76 100644 --- a/scheduler/spread_test.go +++ b/scheduler/spread_test.go @@ -5,689 +5,22 @@ package scheduler import ( "fmt" - "math" "math/rand" "sort" "testing" "time" - "github.com/shoenig/test" - "github.com/shoenig/test/must" - - "github.com/hashicorp/go-set/v3" "github.com/hashicorp/nomad/ci" "github.com/hashicorp/nomad/helper/pointer" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/feasible" + "github.com/hashicorp/nomad/scheduler/tests" + "github.com/shoenig/test" + "github.com/shoenig/test/must" ) -func TestSpreadIterator_SingleAttribute(t *testing.T) { - ci.Parallel(t) - - state, ctx := testContext(t) - dcs := []string{"dc1", "dc2", "dc1", "dc1"} - var nodes []*RankedNode - - // Add these nodes to the state store - for i, dc := range dcs { - node := mock.Node() - node.Datacenter = dc - if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { - t.Fatalf("failed to upsert node: %v", err) - } - nodes = append(nodes, &RankedNode{Node: node}) - } - - static := NewStaticRankIterator(ctx, nodes) - - job := mock.Job() - tg := job.TaskGroups[0] - job.TaskGroups[0].Count = 10 - // add allocs to nodes in dc1 - upserting := []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - EvalID: uuid.Generate(), - NodeID: nodes[0].Node.ID, - }, - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - EvalID: uuid.Generate(), - NodeID: nodes[2].Node.ID, - }, - } - - if err := state.UpsertAllocs(structs.MsgTypeTestSetup, 1000, upserting); err != nil { - t.Fatalf("failed to UpsertAllocs: %v", err) - } - - // Create spread target of 80% in dc1 - // Implicitly, this means 20% in dc2 - spread := &structs.Spread{ - Weight: 100, - Attribute: "${node.datacenter}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "dc1", - Percent: 80, - }, - }, - } - tg.Spreads = []*structs.Spread{spread} - spreadIter := NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - - scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) - - out := collectRanked(scoreNorm) - - // Expect nodes in dc1 with existing allocs to get a boost - // Boost should be ((desiredCount-actual)/desired)*spreadWeight - // For this test, that becomes dc1 = ((8-3)/8 ) = 0.5, and dc2=(2-1)/2 - expectedScores := map[string]float64{ - "dc1": 0.625, - "dc2": 0.5, - } - for _, rn := range out { - must.Eq(t, expectedScores[rn.Node.Datacenter], rn.FinalScore) - } - - // Update the plan to add more allocs to nodes in dc1 - // After this step there are enough allocs to meet the desired count in dc1 - ctx.plan.NodeAllocation[nodes[0].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[0].Node.ID, - }, - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[0].Node.ID, - }, - // Should be ignored as it is a different job. - { - Namespace: structs.DefaultNamespace, - TaskGroup: "bbb", - JobID: "ignore 2", - Job: job, - ID: uuid.Generate(), - NodeID: nodes[0].Node.ID, - }, - } - ctx.plan.NodeAllocation[nodes[3].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[3].Node.ID, - }, - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[3].Node.ID, - }, - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[3].Node.ID, - }, - } - - // Reset the scores - for _, node := range nodes { - node.Scores = nil - node.FinalScore = 0 - } - static = NewStaticRankIterator(ctx, nodes) - spreadIter = NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) - out = collectRanked(scoreNorm) - - // Expect nodes in dc2 with existing allocs to get a boost - // DC1 nodes are not boosted because there are enough allocs to meet - // the desired count - expectedScores = map[string]float64{ - "dc1": 0, - "dc2": 0.5, - } - for _, rn := range out { - must.Eq(t, expectedScores[rn.Node.Datacenter], rn.FinalScore) - } -} - -func TestSpreadIterator_MultipleAttributes(t *testing.T) { - ci.Parallel(t) - - state, ctx := testContext(t) - dcs := []string{"dc1", "dc2", "dc1", "dc1"} - rack := []string{"r1", "r1", "r2", "r2"} - var nodes []*RankedNode - - // Add these nodes to the state store - for i, dc := range dcs { - node := mock.Node() - node.Datacenter = dc - node.Meta["rack"] = rack[i] - if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { - t.Fatalf("failed to upsert node: %v", err) - } - nodes = append(nodes, &RankedNode{Node: node}) - } - - static := NewStaticRankIterator(ctx, nodes) - - job := mock.Job() - tg := job.TaskGroups[0] - job.TaskGroups[0].Count = 10 - // add allocs to nodes in dc1 - upserting := []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - EvalID: uuid.Generate(), - NodeID: nodes[0].Node.ID, - }, - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - EvalID: uuid.Generate(), - NodeID: nodes[2].Node.ID, - }, - } - - if err := state.UpsertAllocs(structs.MsgTypeTestSetup, 1000, upserting); err != nil { - t.Fatalf("failed to UpsertAllocs: %v", err) - } - - spread1 := &structs.Spread{ - Weight: 100, - Attribute: "${node.datacenter}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "dc1", - Percent: 60, - }, - { - Value: "dc2", - Percent: 40, - }, - }, - } - - spread2 := &structs.Spread{ - Weight: 50, - Attribute: "${meta.rack}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "r1", - Percent: 40, - }, - { - Value: "r2", - Percent: 60, - }, - }, - } - - tg.Spreads = []*structs.Spread{spread1, spread2} - spreadIter := NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - - scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) - - out := collectRanked(scoreNorm) - - // Score comes from combining two different spread factors - // Second node should have the highest score because it has no allocs and its in dc2/r1 - expectedScores := map[string]float64{ - nodes[0].Node.ID: 0.500, - nodes[1].Node.ID: 0.667, - nodes[2].Node.ID: 0.556, - nodes[3].Node.ID: 0.556, - } - for _, rn := range out { - must.Eq(t, fmt.Sprintf("%.3f", expectedScores[rn.Node.ID]), fmt.Sprintf("%.3f", rn.FinalScore)) - } - -} - -func TestSpreadIterator_EvenSpread(t *testing.T) { - ci.Parallel(t) - - state, ctx := testContext(t) - dcs := []string{"dc1", "dc2", "dc1", "dc2", "dc1", "dc2", "dc2", "dc1", "dc1", "dc1"} - var nodes []*RankedNode - - // Add these nodes to the state store - for i, dc := range dcs { - node := mock.Node() - node.Datacenter = dc - if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { - t.Fatalf("failed to upsert node: %v", err) - } - nodes = append(nodes, &RankedNode{Node: node}) - } - - static := NewStaticRankIterator(ctx, nodes) - job := mock.Job() - tg := job.TaskGroups[0] - job.TaskGroups[0].Count = 10 - - // Configure even spread across node.datacenter - spread := &structs.Spread{ - Weight: 100, - Attribute: "${node.datacenter}", - } - tg.Spreads = []*structs.Spread{spread} - spreadIter := NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - - scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) - - out := collectRanked(scoreNorm) - - // Nothing placed so both dc nodes get 0 as the score - expectedScores := map[string]float64{ - "dc1": 0, - "dc2": 0, - } - for _, rn := range out { - must.Eq(t, fmt.Sprintf("%.3f", expectedScores[rn.Node.Datacenter]), fmt.Sprintf("%.3f", rn.FinalScore)) - - } - - // Update the plan to add allocs to nodes in dc1 - // After this step dc2 nodes should get boosted - ctx.plan.NodeAllocation[nodes[0].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[0].Node.ID, - }, - } - ctx.plan.NodeAllocation[nodes[2].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[2].Node.ID, - }, - } - - // Reset the scores - for _, node := range nodes { - node.Scores = nil - node.FinalScore = 0 - } - static = NewStaticRankIterator(ctx, nodes) - spreadIter = NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) - out = collectRanked(scoreNorm) - - // Expect nodes in dc2 with existing allocs to get a boost - // dc1 nodes are penalized because they have allocs - expectedScores = map[string]float64{ - "dc1": -1, - "dc2": 1, - } - for _, rn := range out { - must.Eq(t, expectedScores[rn.Node.Datacenter], rn.FinalScore) - } - - // Update the plan to add more allocs to nodes in dc2 - // After this step dc1 nodes should get boosted - ctx.plan.NodeAllocation[nodes[1].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[1].Node.ID, - }, - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[1].Node.ID, - }, - } - ctx.plan.NodeAllocation[nodes[3].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[3].Node.ID, - }, - } - - // Reset the scores - for _, node := range nodes { - node.Scores = nil - node.FinalScore = 0 - } - static = NewStaticRankIterator(ctx, nodes) - spreadIter = NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) - out = collectRanked(scoreNorm) - - // Expect nodes in dc2 to be penalized because there are 3 allocs there now - // dc1 nodes are boosted because that has 2 allocs - expectedScores = map[string]float64{ - "dc1": 0.5, - "dc2": -0.5, - } - for _, rn := range out { - must.Eq(t, fmt.Sprintf("%3.3f", expectedScores[rn.Node.Datacenter]), fmt.Sprintf("%3.3f", rn.FinalScore)) - } - - // Add another node in dc3 - node := mock.Node() - node.Datacenter = "dc3" - if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(1111), node); err != nil { - t.Fatalf("failed to upsert node: %v", err) - } - nodes = append(nodes, &RankedNode{Node: node}) - - // Add another alloc to dc1, now its count matches dc2 - ctx.plan.NodeAllocation[nodes[4].Node.ID] = []*structs.Allocation{ - { - Namespace: structs.DefaultNamespace, - TaskGroup: tg.Name, - JobID: job.ID, - Job: job, - ID: uuid.Generate(), - NodeID: nodes[4].Node.ID, - }, - } - - // Reset scores - for _, node := range nodes { - node.Scores = nil - node.FinalScore = 0 - } - static = NewStaticRankIterator(ctx, nodes) - spreadIter = NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) - out = collectRanked(scoreNorm) - - // Expect dc1 and dc2 to be penalized because they have 3 allocs - // dc3 should get a boost because it has 0 allocs - expectedScores = map[string]float64{ - "dc1": -1, - "dc2": -1, - "dc3": 1, - } - for _, rn := range out { - must.Eq(t, fmt.Sprintf("%.3f", expectedScores[rn.Node.Datacenter]), fmt.Sprintf("%.3f", rn.FinalScore)) - } - -} - -// Test scenarios where the spread iterator sets maximum penalty (-1.0) -func TestSpreadIterator_MaxPenalty(t *testing.T) { - ci.Parallel(t) - - state, ctx := testContext(t) - var nodes []*RankedNode - - // Add nodes in dc3 to the state store - for i := 0; i < 5; i++ { - node := mock.Node() - node.Datacenter = "dc3" - if err := state.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node); err != nil { - t.Fatalf("failed to upsert node: %v", err) - } - nodes = append(nodes, &RankedNode{Node: node}) - } - - static := NewStaticRankIterator(ctx, nodes) - - job := mock.Job() - tg := job.TaskGroups[0] - job.TaskGroups[0].Count = 5 - - // Create spread target of 80% in dc1 - // and 20% in dc2 - spread := &structs.Spread{ - Weight: 100, - Attribute: "${node.datacenter}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "dc1", - Percent: 80, - }, - { - Value: "dc2", - Percent: 20, - }, - }, - } - tg.Spreads = []*structs.Spread{spread} - spreadIter := NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - - scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) - - out := collectRanked(scoreNorm) - - // All nodes are in dc3 so score should be -1 - for _, rn := range out { - must.Eq(t, -1.0, rn.FinalScore) - } - - // Reset scores - for _, node := range nodes { - node.Scores = nil - node.FinalScore = 0 - } - - // Create spread on attribute that doesn't exist on any nodes - spread = &structs.Spread{ - Weight: 100, - Attribute: "${meta.foo}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "bar", - Percent: 80, - }, - { - Value: "baz", - Percent: 20, - }, - }, - } - - tg.Spreads = []*structs.Spread{spread} - static = NewStaticRankIterator(ctx, nodes) - spreadIter = NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) - out = collectRanked(scoreNorm) - - // All nodes don't have the spread attribute so score should be -1 - for _, rn := range out { - must.Eq(t, -1.0, rn.FinalScore) - } - -} - -func TestSpreadIterator_NoInfinity(t *testing.T) { - ci.Parallel(t) - - store, ctx := testContext(t) - var nodes []*RankedNode - - // Add 3 nodes in different DCs to the state store - for i := 1; i < 4; i++ { - node := mock.Node() - node.Datacenter = fmt.Sprintf("dc%d", i) - must.NoError(t, store.UpsertNode(structs.MsgTypeTestSetup, uint64(100+i), node)) - nodes = append(nodes, &RankedNode{Node: node}) - } - - static := NewStaticRankIterator(ctx, nodes) - - job := mock.Job() - tg := job.TaskGroups[0] - job.TaskGroups[0].Count = 8 - - // Create spread target of 50% in dc1, 50% in dc2, and 0% in the implicit target - spread := &structs.Spread{ - Weight: 100, - Attribute: "${node.datacenter}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "dc1", - Percent: 50, - }, - { - Value: "dc2", - Percent: 50, - }, - { - Value: "*", - Percent: 0, - }, - }, - } - tg.Spreads = []*structs.Spread{spread} - spreadIter := NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - - scoreNorm := NewScoreNormalizationIterator(ctx, spreadIter) - - out := collectRanked(scoreNorm) - - // Scores should be even between dc1 and dc2 nodes, without an -Inf on dc3 - must.Len(t, 3, out) - test.Eq(t, 0.75, out[0].FinalScore) - test.Eq(t, 0.75, out[1].FinalScore) - test.Eq(t, -1, out[2].FinalScore) - - // Reset scores - for _, node := range nodes { - node.Scores = nil - node.FinalScore = 0 - } - - // Create very unbalanced spread target to force large negative scores - spread = &structs.Spread{ - Weight: 100, - Attribute: "${node.datacenter}", - SpreadTarget: []*structs.SpreadTarget{ - { - Value: "dc1", - Percent: 99, - }, - { - Value: "dc2", - Percent: 1, - }, - { - Value: "*", - Percent: 0, - }, - }, - } - tg.Spreads = []*structs.Spread{spread} - static = NewStaticRankIterator(ctx, nodes) - spreadIter = NewSpreadIterator(ctx, static) - spreadIter.SetJob(job) - spreadIter.SetTaskGroup(tg) - - scoreNorm = NewScoreNormalizationIterator(ctx, spreadIter) - - out = collectRanked(scoreNorm) - - // Scores should heavily favor dc1, with an -Inf on dc3 - must.Len(t, 3, out) - desired := 8 * 0.99 // 8 allocs * 99% - test.Eq(t, (desired-1)/desired, out[0].FinalScore) - test.Eq(t, -11.5, out[1].FinalScore) - test.LessEq(t, out[1].FinalScore, out[2].FinalScore, - test.Sprintf("expected implicit dc3 to be <= dc2")) -} - -func Test_evenSpreadScoreBoost(t *testing.T) { - ci.Parallel(t) - - pset := &propertySet{ - existingValues: map[string]uint64{}, - proposedValues: map[string]uint64{ - "dc2": 1, - "dc1": 1, - "dc3": 1, - }, - clearedValues: map[string]uint64{ - "dc2": 1, - "dc3": 1, - }, - targetAttribute: "${node.datacenter}", - targetValues: &set.Set[string]{}, - } - - opt := &structs.Node{ - Datacenter: "dc2", - } - boost := evenSpreadScoreBoost(pset, opt) - must.False(t, math.IsInf(boost, 1)) - must.Eq(t, 1.0, boost) -} - // TestSpreadOnLargeCluster exercises potentially quadratic // performance cases with spread scheduling when we have a large // number of eligible nodes unless we limit the number that each @@ -756,7 +89,7 @@ func TestSpreadOnLargeCluster(t *testing.T) { tc := cases[i] t.Run(tc.name, func(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) err := upsertNodes(h, tc.nodeCount, tc.racks) must.NoError(t, err) job := generateJob(tc.allocs) @@ -810,7 +143,7 @@ func generateUnevenRacks(t *testing.T, nodes int, rackCount int) map[string]int // upsertNodes creates a collection of Nodes in the state store, // distributed among the racks -func upsertNodes(h *Harness, count int, racks map[string]int) error { +func upsertNodes(h *tests.Harness, count int, racks map[string]int) error { datacenters := []string{"dc-1", "dc-2"} rackAssignments := []string{} @@ -850,7 +183,7 @@ func generateJob(jobSize int) *structs.Job { return job } -func upsertJob(h *Harness, job *structs.Job) (*structs.Evaluation, error) { +func upsertJob(h *tests.Harness, job *structs.Job) (*structs.Evaluation, error) { err := h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job) if err != nil { return nil, err @@ -874,7 +207,7 @@ func upsertJob(h *Harness, job *structs.Job) (*structs.Evaluation, error) { // validateEqualSpread compares the resulting plan to the node // metadata to verify that each group of spread targets has an equal // distribution. -func validateEqualSpread(h *Harness) error { +func validateEqualSpread(h *tests.Harness) error { iter, err := h.State.Nodes(nil) if err != nil { @@ -931,7 +264,7 @@ func validateEqualSpread(h *Harness) error { func TestSpreadPanicDowngrade(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) nodes := []*structs.Node{} for i := 0; i < 5; i++ { @@ -1028,9 +361,9 @@ func TestSpread_ImplicitTargets(t *testing.T) { dcs := []string{"dc1", "dc2", "dc3"} - setupNodes := func(h *Harness) map[string]string { + setupNodes := func(h *tests.Harness) map[string]string { nodesToDcs := map[string]string{} - var nodes []*RankedNode + var nodes []*feasible.RankedNode for i, dc := range dcs { for n := 0; n < 4; n++ { @@ -1038,14 +371,14 @@ func TestSpread_ImplicitTargets(t *testing.T) { node.Datacenter = dc must.NoError(t, h.State.UpsertNode( structs.MsgTypeTestSetup, uint64(100+i), node)) - nodes = append(nodes, &RankedNode{Node: node}) + nodes = append(nodes, &feasible.RankedNode{Node: node}) nodesToDcs[node.ID] = node.Datacenter } } return nodesToDcs } - setupJob := func(h *Harness, testCaseSpread *structs.Spread) *structs.Evaluation { + setupJob := func(h *tests.Harness, testCaseSpread *structs.Spread) *structs.Evaluation { job := mock.MinJob() job.Datacenters = dcs job.TaskGroups[0].Count = 12 @@ -1131,7 +464,7 @@ func TestSpread_ImplicitTargets(t *testing.T) { } for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - h := NewHarness(t) + h := tests.NewHarness(t) nodesToDcs := setupNodes(h) eval := setupJob(h, tc.spread) must.NoError(t, h.Process(NewServiceScheduler, eval)) diff --git a/scheduler/structs/interfaces.go b/scheduler/structs/interfaces.go new file mode 100644 index 000000000..7c755f823 --- /dev/null +++ b/scheduler/structs/interfaces.go @@ -0,0 +1,133 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package structs + +import ( + log "github.com/hashicorp/go-hclog" + + "github.com/hashicorp/go-memdb" + "github.com/hashicorp/go-version" + "github.com/hashicorp/nomad/nomad/state" + "github.com/hashicorp/nomad/nomad/structs" +) + +/* + * This package contains top-level interfaces used throughout the scheduler. + */ + +// Factory is used to instantiate a new Scheduler +type Factory func(log.Logger, chan<- interface{}, State, Planner) Scheduler + +// Scheduler is the top level instance for a scheduler. A scheduler is +// meant to only encapsulate business logic, pushing the various plumbing +// into Nomad itself. They are invoked to process a single evaluation at +// a time. The evaluation may result in task allocations which are computed +// optimistically, as there are many concurrent evaluations being processed. +// The task allocations are submitted as a plan, and the current leader will +// coordinate the commits to prevent oversubscription or improper allocations +// based on stale state. +type Scheduler interface { + // Process is used to handle a new evaluation. The scheduler is free to + // apply any logic necessary to make the task placements. The state and + // planner will be provided prior to any invocations of process. + Process(*structs.Evaluation) error +} + +// State is an immutable view of the global state. This allows schedulers +// to make intelligent decisions based on allocations of other schedulers +// and to enforce complex constraints that require more information than +// is available to a local state scheduler. +type State interface { + // Config returns the configuration of the state store + Config() *state.StateStoreConfig + + // Nodes returns an iterator over all the nodes. + // The type of each result is *structs.Node + Nodes(ws memdb.WatchSet) (memdb.ResultIterator, error) + + // NodesByNodePool returns an iterator over all nodes in the node pool + NodesByNodePool(ws memdb.WatchSet, poolName string) (memdb.ResultIterator, error) + + // NodePoolByName is used to lookup a node by ID. + NodePoolByName(ws memdb.WatchSet, poolName string) (*structs.NodePool, error) + + // AllocsByJob returns the allocations by JobID + AllocsByJob(ws memdb.WatchSet, namespace, jobID string, all bool) ([]*structs.Allocation, error) + + // AllocsByNode returns all the allocations by node + AllocsByNode(ws memdb.WatchSet, node string) ([]*structs.Allocation, error) + + // AllocByID returns the allocation + AllocByID(ws memdb.WatchSet, allocID string) (*structs.Allocation, error) + + // AllocsByNodeTerminal returns all the allocations by node filtering by terminal status + AllocsByNodeTerminal(ws memdb.WatchSet, node string, terminal bool) ([]*structs.Allocation, error) + + // NodeByID is used to lookup a node by ID + NodeByID(ws memdb.WatchSet, nodeID string) (*structs.Node, error) + + // GetJobByID is used to lookup a job by ID + JobByID(ws memdb.WatchSet, namespace, id string) (*structs.Job, error) + + // DeploymentsByJobID returns the deployments associated with the job + DeploymentsByJobID(ws memdb.WatchSet, namespace, jobID string, all bool) ([]*structs.Deployment, error) + + // JobByIDAndVersion returns the job associated with id and specific version + JobByIDAndVersion(ws memdb.WatchSet, namespace, id string, version uint64) (*structs.Job, error) + + // LatestDeploymentByJobID returns the latest deployment matching the given + // job ID + LatestDeploymentByJobID(ws memdb.WatchSet, namespace, jobID string) (*structs.Deployment, error) + + // SchedulerConfig returns config options for the scheduler + SchedulerConfig() (uint64, *structs.SchedulerConfiguration, error) + + // CSIVolumeByID fetch CSI volumes, containing controller jobs + CSIVolumeByID(memdb.WatchSet, string, string) (*structs.CSIVolume, error) + + // CSIVolumeByID fetch CSI volumes, containing controller jobs + CSIVolumesByNodeID(memdb.WatchSet, string, string) (memdb.ResultIterator, error) + + // HostVolumeByID fetches host volume by its ID + HostVolumeByID(memdb.WatchSet, string, string, bool) (*structs.HostVolume, error) + + // HostVolumesByNodeID gets an iterator with all the volumes attached to a + // given node + HostVolumesByNodeID(memdb.WatchSet, string, state.SortOption) (memdb.ResultIterator, error) + + // TaskGroupHostVolumeClaimsByFields gets all host volume claims for + // a given namespace, job ID and task group name + TaskGroupHostVolumeClaimsByFields(memdb.WatchSet, state.TgvcSearchableFields) (memdb.ResultIterator, error) + + // LatestIndex returns the greatest index value for all indexes. + LatestIndex() (uint64, error) +} + +// Planner interface is used to submit a task allocation plan. +type Planner interface { + // SubmitPlan is used to submit a plan for consideration. + // This will return a PlanResult or an error. It is possible + // that this will result in a state refresh as well. + SubmitPlan(*structs.Plan) (*structs.PlanResult, State, error) + + // UpdateEval is used to update an evaluation. This should update + // a copy of the input evaluation since that should be immutable. + UpdateEval(*structs.Evaluation) error + + // CreateEval is used to create an evaluation. This should set the + // PreviousEval to that of the current evaluation. + CreateEval(*structs.Evaluation) error + + // ReblockEval takes a blocked evaluation and re-inserts it into the blocked + // evaluation tracker. This update occurs only in-memory on the leader. The + // evaluation must exist in a blocked state prior to this being called such + // that on leader changes, the evaluation will be reblocked properly. + ReblockEval(*structs.Evaluation) error + + // ServersMeetMinimumVersion returns whether the Nomad servers in the + // worker's region are at least on the given Nomad version. The + // checkFailedServers parameter specifies whether version for the failed + // servers should be verified. + ServersMeetMinimumVersion(minVersion *version.Version, checkFailedServers bool) bool +} diff --git a/scheduler/structs/structs.go b/scheduler/structs/structs.go new file mode 100644 index 000000000..662d84a9e --- /dev/null +++ b/scheduler/structs/structs.go @@ -0,0 +1,53 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package structs + +import "github.com/hashicorp/nomad/nomad/structs" + +// PortCollisionEvent is an event that can happen during scheduling when +// an unexpected port collision is detected. +type PortCollisionEvent struct { + Reason string + Node *structs.Node + Allocations []*structs.Allocation + + // TODO: this is a large struct, but may be required to debug unexpected + // port collisions. Re-evaluate its need in the future if the bug is fixed + // or not caused by this field. + NetIndex *structs.NetworkIndex +} + +func (ev *PortCollisionEvent) Copy() *PortCollisionEvent { + if ev == nil { + return nil + } + c := new(PortCollisionEvent) + *c = *ev + c.Node = ev.Node.Copy() + if len(ev.Allocations) > 0 { + for i, a := range ev.Allocations { + c.Allocations[i] = a.Copy() + } + + } + c.NetIndex = ev.NetIndex.Copy() + return c +} + +func (ev *PortCollisionEvent) Sanitize() *PortCollisionEvent { + if ev == nil { + return nil + } + clean := ev.Copy() + + clean.Node = ev.Node.Sanitize() + clean.Node.Meta = make(map[string]string) + + for i, alloc := range ev.Allocations { + clean.Allocations[i] = alloc.CopySkipJob() + clean.Allocations[i].Job = nil + } + + return clean +} diff --git a/scheduler/tests/numa_ce.go b/scheduler/tests/numa_ce.go new file mode 100644 index 000000000..603f9190c --- /dev/null +++ b/scheduler/tests/numa_ce.go @@ -0,0 +1,53 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package tests + +import ( + "github.com/hashicorp/nomad/client/lib/idset" + "github.com/hashicorp/nomad/client/lib/numalib" + "github.com/hashicorp/nomad/client/lib/numalib/hw" + "github.com/hashicorp/nomad/nomad/structs" +) + +// CpuResources creates both the legacy and modern structs concerning cpu +// metrics used for resource accounting +// +// only creates a trivial single node, single core system for the sake of +// compatibility with existing tests +func CpuResources(shares int) (structs.LegacyNodeCpuResources, structs.NodeProcessorResources) { + n := &structs.NodeResources{ + Processors: structs.NodeProcessorResources{ + Topology: &numalib.Topology{ + Distances: numalib.SLIT{[]numalib.Cost{10}}, + Cores: []numalib.Core{{ + SocketID: 0, + NodeID: 0, + ID: 0, + Grade: numalib.Performance, + Disable: false, + BaseSpeed: hw.MHz(shares), + }}, + }, + }, + } + n.Processors.Topology.SetNodes(idset.From[hw.NodeID]([]hw.NodeID{0})) + + // polyfill the legacy struct + n.Compatibility() + + return n.Cpu, n.Processors +} + +func CpuResourcesFrom(top *numalib.Topology) (structs.LegacyNodeCpuResources, structs.NodeProcessorResources) { + n := &structs.NodeResources{ + Processors: structs.NodeProcessorResources{ + Topology: top, + }, + } + + // polyfill the legacy struct + n.Compatibility() + + return n.Cpu, n.Processors +} diff --git a/scheduler/testing.go b/scheduler/tests/testing.go similarity index 95% rename from scheduler/testing.go rename to scheduler/tests/testing.go index 2500bcf85..b46bedf6f 100644 --- a/scheduler/testing.go +++ b/scheduler/tests/testing.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: BUSL-1.1 -package scheduler +package tests import ( "fmt" @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/nomad/helper/testlog" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" + sstructs "github.com/hashicorp/nomad/scheduler/structs" "github.com/shoenig/test/must" ) @@ -26,7 +27,7 @@ func (r *RejectPlan) ServersMeetMinimumVersion(minVersion *version.Version, chec return r.Harness.serversMeetMinimumVersion } -func (r *RejectPlan) SubmitPlan(*structs.Plan) (*structs.PlanResult, State, error) { +func (r *RejectPlan) SubmitPlan(*structs.Plan) (*structs.PlanResult, sstructs.State, error) { result := new(structs.PlanResult) result.RefreshIndex = r.Harness.NextIndex() return result, r.Harness.State, nil @@ -51,7 +52,7 @@ type Harness struct { t testing.TB State *state.StateStore - Planner Planner + Planner sstructs.Planner planLock sync.Mutex Plans []*structs.Plan @@ -92,7 +93,7 @@ func NewHarnessWithState(t testing.TB, state *state.StateStore) *Harness { } // SubmitPlan is used to handle plan submission -func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, error) { +func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, sstructs.State, error) { // Ensure sequential plan application h.planLock.Lock() defer h.planLock.Unlock() @@ -273,14 +274,14 @@ func (h *Harness) NextIndex() uint64 { } // Snapshot is used to snapshot the current state -func (h *Harness) Snapshot() State { +func (h *Harness) Snapshot() sstructs.State { snap, _ := h.State.Snapshot() return snap } // Scheduler is used to return a new scheduler from // a snapshot of current state using the harness for planning. -func (h *Harness) Scheduler(factory Factory) Scheduler { +func (h *Harness) Scheduler(factory sstructs.Factory) sstructs.Scheduler { logger := testlog.HCLogger(h.t) eventsCh := make(chan interface{}) @@ -288,7 +289,7 @@ func (h *Harness) Scheduler(factory Factory) Scheduler { go func() { for e := range eventsCh { switch event := e.(type) { - case *PortCollisionEvent: + case *sstructs.PortCollisionEvent: h.t.Errorf("unexpected worker eval event: %v", event.Reason) } } @@ -299,7 +300,7 @@ func (h *Harness) Scheduler(factory Factory) Scheduler { // Process is used to process an evaluation given a factory // function to create the scheduler -func (h *Harness) Process(factory Factory, eval *structs.Evaluation) error { +func (h *Harness) Process(factory sstructs.Factory, eval *structs.Evaluation) error { sched := h.Scheduler(factory) return sched.Process(eval) } diff --git a/scheduler/util.go b/scheduler/util.go index d3f33f222..555774ea6 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -4,10 +4,8 @@ package scheduler import ( - "encoding/binary" "fmt" "maps" - "math/rand" "slices" log "github.com/hashicorp/go-hclog" @@ -15,13 +13,14 @@ import ( "github.com/hashicorp/go-set/v3" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/feasible" "github.com/hashicorp/nomad/scheduler/reconcile" sstructs "github.com/hashicorp/nomad/scheduler/structs" ) // readyNodesInDCsAndPool returns all the ready nodes in the given datacenters // and pool, and a mapping of each data center to the count of ready nodes. -func readyNodesInDCsAndPool(state State, dcs []string, pool string) ([]*structs.Node, map[string]struct{}, map[string]int, error) { +func readyNodesInDCsAndPool(state sstructs.State, dcs []string, pool string) ([]*structs.Node, map[string]struct{}, map[string]int, error) { // Index the DCs dcMap := make(map[string]int) @@ -101,7 +100,7 @@ func progressMade(result *structs.PlanResult) bool { // underlying nodes are tainted, and should force a migration of the allocation, // or if the underlying nodes are disconnected, and should be used to calculate // the reconnect timeout of its allocations. All the nodes returned in the map are tainted. -func taintedNodes(state State, allocs []*structs.Allocation) (map[string]*structs.Node, error) { +func taintedNodes(state sstructs.State, allocs []*structs.Allocation) (map[string]*structs.Node, error) { out := make(map[string]*structs.Node) for _, alloc := range allocs { if _, ok := out[alloc.NodeID]; ok { @@ -134,29 +133,6 @@ func taintedNodes(state State, allocs []*structs.Allocation) (map[string]*struct return out, nil } -// shuffleNodes randomizes the slice order with the Fisher-Yates -// algorithm. We seed the random source with the eval ID (which is -// random) to aid in postmortem debugging of specific evaluations and -// state snapshots. -func shuffleNodes(plan *structs.Plan, index uint64, nodes []*structs.Node) { - - // use the last 4 bytes because those are the random bits - // if we have sortable IDs - buf := []byte(plan.EvalID) - seed := binary.BigEndian.Uint64(buf[len(buf)-8:]) - - // for retried plans the index is the plan result's RefreshIndex - // so that we don't retry with the exact same shuffle - seed ^= index - r := rand.New(rand.NewSource(int64(seed >> 2))) - - n := len(nodes) - for i := n - 1; i > 0; i-- { - j := r.Intn(i + 1) - nodes[i], nodes[j] = nodes[j], nodes[i] - } -} - // comparison records the _first_ detected difference between two groups during // a comparison in tasksUpdated // @@ -562,7 +538,7 @@ func renderTemplatesUpdated(a, b *structs.RestartPolicy, msg string) comparison } // setStatus is used to update the status of the evaluation -func setStatus(logger log.Logger, planner Planner, +func setStatus(logger log.Logger, planner sstructs.Planner, eval, nextEval, spawnedBlocked *structs.Evaluation, tgMetrics map[string]*structs.AllocMetric, status, desc string, queuedAllocs map[string]int, deploymentID string) error { @@ -588,8 +564,8 @@ func setStatus(logger log.Logger, planner Planner, // inplaceUpdate attempts to update allocations in-place where possible. It // returns the allocs that couldn't be done inplace and then those that could. -func inplaceUpdate(ctx Context, eval *structs.Evaluation, job *structs.Job, - stack Stack, updates []reconcile.AllocTuple) (destructive, inplace []reconcile.AllocTuple) { +func inplaceUpdate(ctx feasible.Context, eval *structs.Evaluation, job *structs.Job, + stack feasible.Stack, updates []reconcile.AllocTuple) (destructive, inplace []reconcile.AllocTuple) { // doInplace manipulates the updates map to make the current allocation // an inplace update. @@ -653,7 +629,7 @@ func inplaceUpdate(ctx Context, eval *structs.Evaluation, job *structs.Job, // Attempt to match the task group option := stack.Select(update.TaskGroup, - &SelectOptions{AllocName: update.Alloc.Name}) + &feasible.SelectOptions{AllocName: update.Alloc.Name}) // Pop the allocation ctx.Plan().PopUpdate(update.Alloc) @@ -849,7 +825,7 @@ func updateNonTerminalAllocsToLost(plan *structs.Plan, tainted map[string]*struc // by the reconciler to make decisions about how to update an allocation. The // factory allows the reconciler to be unaware of how to determine the type of // update necessary and can minimize the set of objects it is exposed to. -func genericAllocUpdateFn(ctx Context, stack Stack, evalID string) reconcile.AllocUpdateType { +func genericAllocUpdateFn(ctx feasible.Context, stack feasible.Stack, evalID string) reconcile.AllocUpdateType { return func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) { // Same index, so nothing to do if existing.Job.JobModifyIndex == newJob.JobModifyIndex { @@ -899,7 +875,7 @@ func genericAllocUpdateFn(ctx Context, stack Stack, evalID string) reconcile.All ctx.Plan().AppendStoppedAlloc(existing, sstructs.StatusAllocInPlace, "", "") // Attempt to match the task group - option := stack.Select(newTG, &SelectOptions{AllocName: existing.Name}) + option := stack.Select(newTG, &feasible.SelectOptions{AllocName: existing.Name}) // Pop the allocation ctx.Plan().PopUpdate(existing) diff --git a/scheduler/util_test.go b/scheduler/util_test.go index ab2466b52..b7134cf22 100644 --- a/scheduler/util_test.go +++ b/scheduler/util_test.go @@ -14,7 +14,9 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/scheduler/feasible" "github.com/hashicorp/nomad/scheduler/reconcile" + "github.com/hashicorp/nomad/scheduler/tests" "github.com/shoenig/test/must" ) @@ -215,12 +217,12 @@ func TestShuffleNodes(t *testing.T) { copy(orig, nodes) eval := mock.Eval() // will have random EvalID plan := eval.MakePlan(mock.Job()) - shuffleNodes(plan, 1000, nodes) + feasible.ShuffleNodes(plan, 1000, nodes) must.NotEq(t, nodes, orig) nodes2 := make([]*structs.Node, len(nodes)) copy(nodes2, orig) - shuffleNodes(plan, 1000, nodes2) + feasible.ShuffleNodes(plan, 1000, nodes2) must.Eq(t, nodes, nodes2) @@ -590,7 +592,7 @@ func TestNetworkUpdated(t *testing.T) { func TestSetStatus(t *testing.T) { ci.Parallel(t) - h := NewHarness(t) + h := tests.NewHarness(t) logger := testlog.HCLogger(t) eval := mock.Eval() status := "a" @@ -603,7 +605,7 @@ func TestSetStatus(t *testing.T) { must.Sprintf("setStatus() submited invalid eval: %v", newEval)) // Test next evals - h = NewHarness(t) + h = tests.NewHarness(t) next := mock.Eval() must.NoError(t, setStatus(logger, h, eval, next, nil, nil, status, desc, nil, "")) must.Eq(t, 1, len(h.Evals), must.Sprintf("setStatus() didn't update plan: %v", h.Evals)) @@ -612,7 +614,7 @@ func TestSetStatus(t *testing.T) { must.Eq(t, next.ID, newEval.NextEval, must.Sprintf("setStatus() didn't set nextEval correctly: %v", newEval)) // Test blocked evals - h = NewHarness(t) + h = tests.NewHarness(t) blocked := mock.Eval() must.NoError(t, setStatus(logger, h, eval, nil, blocked, nil, status, desc, nil, "")) must.Eq(t, 1, len(h.Evals), must.Sprintf("setStatus() didn't update plan: %v", h.Evals)) @@ -621,7 +623,7 @@ func TestSetStatus(t *testing.T) { must.Eq(t, blocked.ID, newEval.BlockedEval, must.Sprintf("setStatus() didn't set BlockedEval correctly: %v", newEval)) // Test metrics - h = NewHarness(t) + h = tests.NewHarness(t) metrics := map[string]*structs.AllocMetric{"foo": nil} must.NoError(t, setStatus(logger, h, eval, nil, nil, metrics, status, desc, nil, "")) must.Eq(t, 1, len(h.Evals), must.Sprintf("setStatus() didn't update plan: %v", h.Evals)) @@ -631,7 +633,7 @@ func TestSetStatus(t *testing.T) { must.Sprintf("setStatus() didn't set failed task group metrics correctly: %v", newEval)) // Test queued allocations - h = NewHarness(t) + h = tests.NewHarness(t) queuedAllocs := map[string]int{"web": 1} must.NoError(t, setStatus(logger, h, eval, nil, nil, metrics, status, desc, queuedAllocs, "")) @@ -640,7 +642,7 @@ func TestSetStatus(t *testing.T) { newEval = h.Evals[0] must.Eq(t, newEval.QueuedAllocations, queuedAllocs, must.Sprintf("setStatus() didn't set failed task group metrics correctly: %v", newEval)) - h = NewHarness(t) + h = tests.NewHarness(t) dID := uuid.Generate() must.NoError(t, setStatus(logger, h, eval, nil, nil, metrics, status, desc, queuedAllocs, dID)) must.Eq(t, 1, len(h.Evals), must.Sprintf("setStatus() didn't update plan: %v", h.Evals)) @@ -652,7 +654,7 @@ func TestSetStatus(t *testing.T) { func TestInplaceUpdate_ChangedTaskGroup(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := feasible.MockContext(t) eval := mock.Eval() job := mock.Job() @@ -697,19 +699,19 @@ func TestInplaceUpdate_ChangedTaskGroup(t *testing.T) { tg.Tasks = append(tg.Tasks, task) updates := []reconcile.AllocTuple{{Alloc: alloc, TaskGroup: tg}} - stack := NewGenericStack(false, ctx) + stack := feasible.NewGenericStack(false, ctx) // Do the inplace update. unplaced, inplace := inplaceUpdate(ctx, eval, job, stack, updates) must.True(t, len(unplaced) == 1 && len(inplace) == 0, must.Sprint("inplaceUpdate incorrectly did an inplace update")) - must.MapEmpty(t, ctx.plan.NodeAllocation, must.Sprint("inplaceUpdate incorrectly did an inplace update")) + must.MapEmpty(t, ctx.Plan().NodeAllocation, must.Sprint("inplaceUpdate incorrectly did an inplace update")) } func TestInplaceUpdate_AllocatedResources(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := feasible.MockContext(t) eval := mock.Eval() job := mock.Job() @@ -751,16 +753,16 @@ func TestInplaceUpdate_AllocatedResources(t *testing.T) { } updates := []reconcile.AllocTuple{{Alloc: alloc, TaskGroup: tg}} - stack := NewGenericStack(false, ctx) + stack := feasible.NewGenericStack(false, ctx) // Do the inplace update. unplaced, inplace := inplaceUpdate(ctx, eval, job, stack, updates) must.True(t, len(unplaced) == 0 && len(inplace) == 1, must.Sprint("inplaceUpdate incorrectly did not perform an inplace update")) - must.MapNotEmpty(t, ctx.plan.NodeAllocation, must.Sprint("inplaceUpdate incorrectly did an inplace update")) - must.SliceNotEmpty(t, ctx.plan.NodeAllocation[node.ID][0].AllocatedResources.Shared.Ports) + must.MapNotEmpty(t, ctx.Plan().NodeAllocation, must.Sprint("inplaceUpdate incorrectly did an inplace update")) + must.SliceNotEmpty(t, ctx.Plan().NodeAllocation[node.ID][0].AllocatedResources.Shared.Ports) - port, ok := ctx.plan.NodeAllocation[node.ID][0].AllocatedResources.Shared.Ports.Get("api-port") + port, ok := ctx.Plan().NodeAllocation[node.ID][0].AllocatedResources.Shared.Ports.Get("api-port") must.True(t, ok) must.Eq(t, 19910, port.Value) } @@ -768,7 +770,7 @@ func TestInplaceUpdate_AllocatedResources(t *testing.T) { func TestInplaceUpdate_NoMatch(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := feasible.MockContext(t) eval := mock.Eval() job := mock.Job() @@ -809,19 +811,19 @@ func TestInplaceUpdate_NoMatch(t *testing.T) { tg.Tasks[0].Resources = resource updates := []reconcile.AllocTuple{{Alloc: alloc, TaskGroup: tg}} - stack := NewGenericStack(false, ctx) + stack := feasible.NewGenericStack(false, ctx) // Do the inplace update. unplaced, inplace := inplaceUpdate(ctx, eval, job, stack, updates) must.True(t, len(unplaced) == 1 && len(inplace) == 0, must.Sprint("inplaceUpdate incorrectly did an inplace update")) - must.MapEmpty(t, ctx.plan.NodeAllocation, must.Sprint("inplaceUpdate incorrectly did an inplace update")) + must.MapEmpty(t, ctx.Plan().NodeAllocation, must.Sprint("inplaceUpdate incorrectly did an inplace update")) } func TestInplaceUpdate_Success(t *testing.T) { ci.Parallel(t) - state, ctx := testContext(t) + state, ctx := feasible.MockContext(t) eval := mock.Eval() job := mock.Job() @@ -878,14 +880,14 @@ func TestInplaceUpdate_Success(t *testing.T) { tg.Tasks[0].Services = append(tg.Tasks[0].Services, newServices...) updates := []reconcile.AllocTuple{{Alloc: alloc, TaskGroup: tg}} - stack := NewGenericStack(false, ctx) + stack := feasible.NewGenericStack(false, ctx) stack.SetJob(job) // Do the inplace update. unplaced, inplace := inplaceUpdate(ctx, eval, job, stack, updates) must.True(t, len(unplaced) == 0 && len(inplace) == 1, must.Sprint("inplaceUpdate did not do an inplace update")) - must.Eq(t, 1, len(ctx.plan.NodeAllocation), must.Sprint("inplaceUpdate did not do an inplace update")) + must.Eq(t, 1, len(ctx.Plan().NodeAllocation), must.Sprint("inplaceUpdate did not do an inplace update")) must.Eq(t, alloc.ID, inplace[0].Alloc.ID, must.Sprintf("inplaceUpdate returned the wrong, inplace updated alloc: %#v", inplace)) // Get the alloc we inserted. @@ -912,7 +914,7 @@ func TestInplaceUpdate_Success(t *testing.T) { func TestInplaceUpdate_WildcardDatacenters(t *testing.T) { ci.Parallel(t) - store, ctx := testContext(t) + store, ctx := feasible.MockContext(t) eval := mock.Eval() job := mock.Job() job.Datacenters = []string{"*"} @@ -928,20 +930,20 @@ func TestInplaceUpdate_WildcardDatacenters(t *testing.T) { must.NoError(t, store.UpsertAllocs(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{alloc})) updates := []reconcile.AllocTuple{{Alloc: alloc, TaskGroup: job.TaskGroups[0]}} - stack := NewGenericStack(false, ctx) + stack := feasible.NewGenericStack(false, ctx) unplaced, inplace := inplaceUpdate(ctx, eval, job, stack, updates) must.Len(t, 1, inplace, must.Sprintf("inplaceUpdate should have an inplace update")) must.Len(t, 0, unplaced) - must.MapNotEmpty(t, ctx.plan.NodeAllocation, + must.MapNotEmpty(t, ctx.Plan().NodeAllocation, must.Sprintf("inplaceUpdate should have an inplace update")) } func TestInplaceUpdate_NodePools(t *testing.T) { ci.Parallel(t) - store, ctx := testContext(t) + store, ctx := feasible.MockContext(t) eval := mock.Eval() job := mock.Job() job.Datacenters = []string{"*"} @@ -973,12 +975,12 @@ func TestInplaceUpdate_NodePools(t *testing.T) { {Alloc: alloc1, TaskGroup: job.TaskGroups[0]}, {Alloc: alloc2, TaskGroup: job.TaskGroups[0]}, } - stack := NewGenericStack(false, ctx) + stack := feasible.NewGenericStack(false, ctx) destructive, inplace := inplaceUpdate(ctx, eval, job, stack, updates) must.Len(t, 1, inplace, must.Sprint("should have an inplace update")) must.Eq(t, alloc1.ID, inplace[0].Alloc.ID) - must.Len(t, 1, ctx.plan.NodeAllocation[node1.ID], + must.Len(t, 1, ctx.Plan().NodeAllocation[node1.ID], must.Sprint("NodeAllocation should have an inplace update for node1")) // note that NodeUpdate with the new alloc won't be populated here yet @@ -1136,11 +1138,11 @@ func TestTaskGroupConstraints(t *testing.T) { expConstr := []*structs.Constraint{constr, constr2, constr3} expDrivers := map[string]struct{}{"exec": {}, "docker": {}} - actConstrains := taskGroupConstraints(tg) - must.Eq(t, actConstrains.constraints, expConstr, must.Sprintf( - "taskGroupConstraints(%v) returned %v; want %v", tg, actConstrains.constraints, expConstr)) - must.Eq(t, actConstrains.drivers, expDrivers, must.Sprintf( - "taskGroupConstraints(%v) returned %v; want %v", tg, actConstrains.drivers, expDrivers)) + actConstrains := feasible.TaskGroupConstraints(tg) + must.Eq(t, actConstrains.Constraints, expConstr, must.Sprintf( + "taskGroupConstraints(%v) returned %v; want %v", tg, actConstrains.Constraints, expConstr)) + must.Eq(t, actConstrains.Drivers, expDrivers, must.Sprintf( + "taskGroupConstraints(%v) returned %v; want %v", tg, actConstrains.Drivers, expDrivers)) } func TestProgressMade(t *testing.T) {