From 5f27e0010dfb50f60e7a8dd1a0dbb29cb4d6c60b Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Mon, 10 Sep 2018 12:38:36 -0500 Subject: [PATCH 01/38] structs and API changes to plan and alloc structs needed for preemption --- api/allocations.go | 62 ++++++++++++----------- api/jobs.go | 1 + nomad/structs/structs.go | 94 +++++++++++++++++++++++++++++++++-- nomad/structs/structs_test.go | 52 +++++++++++++++++++ 4 files changed, 174 insertions(+), 35 deletions(-) diff --git a/api/allocations.go b/api/allocations.go index cdeb3e8a095..6d860362bbd 100644 --- a/api/allocations.go +++ b/api/allocations.go @@ -67,36 +67,38 @@ func (a *Allocations) GC(alloc *Allocation, q *QueryOptions) error { // Allocation is used for serialization of allocations. type Allocation struct { - ID string - Namespace string - EvalID string - Name string - NodeID string - JobID string - Job *Job - TaskGroup string - Resources *Resources - TaskResources map[string]*Resources - AllocatedResources *AllocatedResources - Services map[string]string - Metrics *AllocationMetric - DesiredStatus string - DesiredDescription string - DesiredTransition DesiredTransition - ClientStatus string - ClientDescription string - TaskStates map[string]*TaskState - DeploymentID string - DeploymentStatus *AllocDeploymentStatus - FollowupEvalID string - PreviousAllocation string - NextAllocation string - RescheduleTracker *RescheduleTracker - CreateIndex uint64 - ModifyIndex uint64 - AllocModifyIndex uint64 - CreateTime int64 - ModifyTime int64 + ID string + Namespace string + EvalID string + Name string + NodeID string + JobID string + Job *Job + TaskGroup string + Resources *Resources + TaskResources map[string]*Resources + AllocatedResources *AllocatedResources + Services map[string]string + Metrics *AllocationMetric + DesiredStatus string + DesiredDescription string + DesiredTransition DesiredTransition + ClientStatus string + ClientDescription string + TaskStates map[string]*TaskState + DeploymentID string + DeploymentStatus *AllocDeploymentStatus + FollowupEvalID string + PreviousAllocation string + NextAllocation string + RescheduleTracker *RescheduleTracker + PreemptedAllocations []string + PreemptedByAllocation string + CreateIndex uint64 + ModifyIndex uint64 + AllocModifyIndex uint64 + CreateTime int64 + ModifyTime int64 } // AllocationMetric is used to deserialize allocation metrics. diff --git a/api/jobs.go b/api/jobs.go index b89d2e05053..148916c492b 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -1013,6 +1013,7 @@ type ObjectDiff struct { type PlanAnnotations struct { DesiredTGUpdates map[string]*DesiredUpdates + PreemptedAllocs []*AllocationListStub } type DesiredUpdates struct { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index f26df1d5c35..143e1cde991 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -643,6 +643,15 @@ type ApplyPlanResultsRequest struct { // processed many times, potentially making state updates, without the state of // the evaluation itself being updated. EvalID string + + // NodePreemptions is a map from node id to a set of allocations from other + // lower priority jobs that are preempted. Preempted allocations are marked + // as stopped. + NodePreemptions []*Allocation + + // PreemptionEvals is a slice of follow up evals for jobs whose allocations + // have been preempted to place allocs in this plan + PreemptionEvals []*Evaluation } // AllocUpdateRequest is used to submit changes to allocations, either @@ -1876,6 +1885,29 @@ func (r *Resources) Add(delta *Resources) error { return nil } +// Subtract removes the resources of the delta to this, potentially +// returning an error if not possible. +func (r *Resources) Subtract(delta *Resources) error { + if delta == nil { + return nil + } + r.CPU -= delta.CPU + r.MemoryMB -= delta.MemoryMB + r.DiskMB -= delta.DiskMB + r.IOPS -= delta.IOPS + + for _, n := range delta.Networks { + // Find the matching interface by IP or CIDR + idx := r.NetIndex(n) + if idx == -1 { + r.Networks = append(r.Networks, n.Copy()) + } else { + r.Networks[idx].MBits -= delta.Networks[idx].MBits + } + } + return nil +} + func (r *Resources) GoString() string { return fmt.Sprintf("*%#v", *r) } @@ -7040,6 +7072,14 @@ type Allocation struct { // that can be rescheduled in the future FollowupEvalID string + // PreemptedAllocations captures IDs of any allocations that were preempted + // in order to place this allocation + PreemptedAllocations []string + + // PreemptedByAllocation tracks the alloc ID of the allocation that caused this allocation + // to stop running because it got preempted + PreemptedByAllocation string + // Raft Indexes CreateIndex uint64 ModifyIndex uint64 @@ -7114,6 +7154,7 @@ func (a *Allocation) copyImpl(job bool) *Allocation { } na.RescheduleTracker = a.RescheduleTracker.Copy() + na.PreemptedAllocations = helper.CopySliceString(a.PreemptedAllocations) return na } @@ -7758,6 +7799,7 @@ const ( EvalTriggerMaxPlans = "max-plan-attempts" EvalTriggerRetryFailedAlloc = "alloc-failure" EvalTriggerQueuedAllocs = "queued-allocs" + EvalTriggerPreemption = "preempted" ) const ( @@ -7983,11 +8025,12 @@ func (e *Evaluation) ShouldBlock() bool { // for a given Job func (e *Evaluation) MakePlan(j *Job) *Plan { p := &Plan{ - EvalID: e.ID, - Priority: e.Priority, - Job: j, - NodeUpdate: make(map[string][]*Allocation), - NodeAllocation: make(map[string][]*Allocation), + EvalID: e.ID, + Priority: e.Priority, + Job: j, + NodeUpdate: make(map[string][]*Allocation), + NodeAllocation: make(map[string][]*Allocation), + NodePreemptions: make(map[string][]*Allocation), } if j != nil { p.AllAtOnce = j.AllAtOnce @@ -8100,6 +8143,11 @@ type Plan struct { // deployments. This allows the scheduler to cancel any unneeded deployment // because the job is stopped or the update block is removed. DeploymentUpdates []*DeploymentStatusUpdate + + // NodePreemptions is a map from node id to a set of allocations from other + // lower priority jobs that are preempted. Preempted allocations are marked + // as evicted. + NodePreemptions map[string][]*Allocation } // AppendUpdate marks the allocation for eviction. The clientStatus of the @@ -8132,6 +8180,27 @@ func (p *Plan) AppendUpdate(alloc *Allocation, desiredStatus, desiredDesc, clien p.NodeUpdate[node] = append(existing, newAlloc) } +func (p *Plan) AppendPreemptedAlloc(alloc *Allocation, desiredStatus, preemptingAllocID string) { + newAlloc := new(Allocation) + *newAlloc = *alloc + // Normalize the job + newAlloc.Job = nil + + // Strip the resources as it can be rebuilt. + newAlloc.Resources = nil + + newAlloc.DesiredStatus = desiredStatus + newAlloc.PreemptedByAllocation = preemptingAllocID + + desiredDesc := fmt.Sprintf("Preempted by alloc ID %v", preemptingAllocID) + newAlloc.DesiredDescription = desiredDesc + + node := alloc.NodeID + // Append this alloc to slice for this node + existing := p.NodePreemptions[node] + p.NodePreemptions[node] = append(existing, newAlloc) +} + func (p *Plan) PopUpdate(alloc *Allocation) { existing := p.NodeUpdate[alloc.NodeID] n := len(existing) @@ -8163,6 +8232,14 @@ func (p *Plan) IsNoOp() bool { len(p.DeploymentUpdates) == 0 } +// PreemptedAllocs is used to store information about a set of allocations +// for the same job that get preempted as part of placing allocations for the +// job in the plan. + +// Preempted allocs represents a map from jobid to allocations +// to be preempted +type PreemptedAllocs map[*NamespacedID][]*Allocation + // PlanResult is the result of a plan submitted to the leader. type PlanResult struct { // NodeUpdate contains all the updates that were committed. @@ -8177,6 +8254,11 @@ type PlanResult struct { // DeploymentUpdates is the set of deployment updates that were committed. DeploymentUpdates []*DeploymentStatusUpdate + // NodePreemptions is a map from node id to a set of allocations from other + // lower priority jobs that are preempted. Preempted allocations are marked + // as stopped. + NodePreemptions map[string][]*Allocation + // RefreshIndex is the index the worker should refresh state up to. // This allows all evictions and allocations to be materialized. // If any allocations were rejected due to stale data (node state, @@ -8213,6 +8295,8 @@ func (p *PlanResult) FullCommit(plan *Plan) (bool, int, int) { type PlanAnnotations struct { // DesiredTGUpdates is the set of desired updates per task group. DesiredTGUpdates map[string]*DesiredUpdates + // PreemptedAllocs is the set of allocations to be preempted to make the placement successful. + PreemptedAllocs []*AllocListStub } // DesiredUpdates is the set of changes the scheduler would like to make given diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index 88bebcbe880..dd1c25342f0 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -1869,6 +1869,58 @@ func TestResource_Add_Network(t *testing.T) { } } +func TestResource_Subtract(t *testing.T) { + r1 := &Resources{ + CPU: 2000, + MemoryMB: 2048, + DiskMB: 10000, + IOPS: 100, + Networks: []*NetworkResource{ + { + CIDR: "10.0.0.0/8", + MBits: 100, + ReservedPorts: []Port{{"ssh", 22}}, + }, + }, + } + r2 := &Resources{ + CPU: 1000, + MemoryMB: 1024, + DiskMB: 5000, + IOPS: 50, + Networks: []*NetworkResource{ + { + IP: "10.0.0.1", + MBits: 20, + ReservedPorts: []Port{{"web", 80}}, + }, + }, + } + + err := r1.Subtract(r2) + if err != nil { + t.Fatalf("Err: %v", err) + } + + expect := &Resources{ + CPU: 1000, + MemoryMB: 1024, + DiskMB: 5000, + IOPS: 50, + Networks: []*NetworkResource{ + { + CIDR: "10.0.0.0/8", + MBits: 80, + ReservedPorts: []Port{{"ssh", 22}}, + }, + }, + } + + if !reflect.DeepEqual(expect.Networks, r1.Networks) { + t.Fatalf("bad: %#v %#v", expect, r1) + } +} + func TestEncodeDecode(t *testing.T) { type FooRequest struct { Foo string From 80041603043fd1a09aa05f835c60b1ace14b558c Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Mon, 10 Sep 2018 13:13:10 -0500 Subject: [PATCH 02/38] REview feedback --- nomad/structs/structs.go | 18 ++++-------------- 1 file changed, 4 insertions(+), 14 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 143e1cde991..563a4534d2c 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -644,9 +644,8 @@ type ApplyPlanResultsRequest struct { // the evaluation itself being updated. EvalID string - // NodePreemptions is a map from node id to a set of allocations from other - // lower priority jobs that are preempted. Preempted allocations are marked - // as stopped. + // NodePreemptions is a slice of allocations from other lower priority jobs + // that are preempted. Preempted allocations are marked as evicted. NodePreemptions []*Allocation // PreemptionEvals is a slice of follow up evals for jobs whose allocations @@ -1899,9 +1898,7 @@ func (r *Resources) Subtract(delta *Resources) error { for _, n := range delta.Networks { // Find the matching interface by IP or CIDR idx := r.NetIndex(n) - if idx == -1 { - r.Networks = append(r.Networks, n.Copy()) - } else { + if idx != -1 { r.Networks[idx].MBits -= delta.Networks[idx].MBits } } @@ -8232,14 +8229,6 @@ func (p *Plan) IsNoOp() bool { len(p.DeploymentUpdates) == 0 } -// PreemptedAllocs is used to store information about a set of allocations -// for the same job that get preempted as part of placing allocations for the -// job in the plan. - -// Preempted allocs represents a map from jobid to allocations -// to be preempted -type PreemptedAllocs map[*NamespacedID][]*Allocation - // PlanResult is the result of a plan submitted to the leader. type PlanResult struct { // NodeUpdate contains all the updates that were committed. @@ -8295,6 +8284,7 @@ func (p *PlanResult) FullCommit(plan *Plan) (bool, int, int) { type PlanAnnotations struct { // DesiredTGUpdates is the set of desired updates per task group. DesiredTGUpdates map[string]*DesiredUpdates + // PreemptedAllocs is the set of allocations to be preempted to make the placement successful. PreemptedAllocs []*AllocListStub } From bf7192c58d8503ac4123660f33a6f989e0c28242 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 11 Sep 2018 11:36:15 -0500 Subject: [PATCH 03/38] Add number of evictions to DesiredUpdates struct to use in CLI/API --- api/jobs.go | 1 + nomad/structs/structs.go | 1 + 2 files changed, 2 insertions(+) diff --git a/api/jobs.go b/api/jobs.go index 148916c492b..a064b318424 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -1024,6 +1024,7 @@ type DesiredUpdates struct { InPlaceUpdate uint64 DestructiveUpdate uint64 Canary uint64 + Evict uint64 } type JobDispatchRequest struct { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 563a4534d2c..5523299fd26 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -8299,6 +8299,7 @@ type DesiredUpdates struct { InPlaceUpdate uint64 DestructiveUpdate uint64 Canary uint64 + Evict uint64 } func (d *DesiredUpdates) GoString() string { From fc266f79872d6e7bb1eb8c985f2f57f863600a74 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Mon, 10 Sep 2018 12:38:36 -0500 Subject: [PATCH 04/38] structs and API changes to plan and alloc structs needed for preemption --- nomad/structs/structs.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 5523299fd26..be625dcd73b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -8229,6 +8229,14 @@ func (p *Plan) IsNoOp() bool { len(p.DeploymentUpdates) == 0 } +// PreemptedAllocs is used to store information about a set of allocations +// for the same job that get preempted as part of placing allocations for the +// job in the plan. + +// Preempted allocs represents a map from jobid to allocations +// to be preempted +type PreemptedAllocs map[*NamespacedID][]*Allocation + // PlanResult is the result of a plan submitted to the leader. type PlanResult struct { // NodeUpdate contains all the updates that were committed. From 715d869499514c4fe1b0df7719f11adcb77ca40e Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Fri, 21 Sep 2018 16:05:00 -0500 Subject: [PATCH 05/38] Implement preemption for system jobs. This commit implements an allocation selection algorithm for finding allocations to preempt. It currently special cases network resource asks from others (cpu/memory/disk/iops). --- nomad/fsm.go | 2 +- nomad/plan_apply.go | 70 +++ nomad/state/state_store.go | 37 ++ nomad/structs/structs.go | 20 +- scheduler/context_test.go | 5 +- scheduler/generic_sched.go | 3 +- scheduler/preemption.go | 437 ++++++++++++++++++ scheduler/preemption_test.go | 789 +++++++++++++++++++++++++++++++++ scheduler/rank.go | 81 +++- scheduler/system_sched.go | 20 +- scheduler/system_sched_test.go | 346 ++++++++++++++- scheduler/testing.go | 14 + 12 files changed, 1794 insertions(+), 30 deletions(-) create mode 100644 scheduler/preemption.go create mode 100644 scheduler/preemption_test.go diff --git a/nomad/fsm.go b/nomad/fsm.go index 0111f0c35ca..ba02bf06396 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -813,7 +813,7 @@ func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { n.logger.Error("ApplyPlan failed", "error", err) return err } - + n.handleUpsertedEvals(req.PreemptionEvals) return nil } diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 12f4dc02a2c..66fd64adda6 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -10,6 +10,7 @@ import ( "github.com/armon/go-metrics" "github.com/hashicorp/go-multierror" + "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/raft" @@ -163,6 +164,7 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap Deployment: result.Deployment, DeploymentUpdates: result.DeploymentUpdates, EvalID: plan.EvalID, + NodePreemptions: make([]*structs.Allocation, 0, len(result.NodePreemptions)), } for _, updateList := range result.NodeUpdate { req.Alloc = append(req.Alloc, updateList...) @@ -171,6 +173,10 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap req.Alloc = append(req.Alloc, allocList...) } + for _, preemptions := range result.NodePreemptions { + req.NodePreemptions = append(req.NodePreemptions, preemptions...) + } + // Set the time the alloc was applied for the first time. This can be used // to approximate the scheduling time. now := time.Now().UTC().UnixNano() @@ -181,6 +187,41 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap alloc.ModifyTime = now } + // Set create and modify time for preempted allocs if any + // Also gather jobids to create follow up evals + preemptedJobIDs := make(map[structs.NamespacedID]struct{}) + for _, alloc := range req.NodePreemptions { + if alloc.CreateTime == 0 { + alloc.CreateTime = now + } + alloc.ModifyTime = now + id := structs.NamespacedID{Namespace: alloc.Namespace, ID: alloc.JobID} + _, ok := preemptedJobIDs[id] + if !ok { + preemptedJobIDs[id] = struct{}{} + } + } + + var evals []*structs.Evaluation + for preemptedJobID, _ := range preemptedJobIDs { + job, _ := p.State().JobByID(nil, preemptedJobID.Namespace, preemptedJobID.ID) // TODO Fix me + if job != nil { + //TODO(preetha): This eval is missing class eligibility related fields + // need to figure out how to set them per job + eval := &structs.Evaluation{ + ID: uuid.Generate(), + Namespace: job.Namespace, + TriggeredBy: structs.EvalTriggerPreemption, + JobID: job.ID, + Type: job.Type, + Priority: job.Priority, + Status: structs.EvalStatusPending, + } + evals = append(evals, eval) + } + } + req.PreemptionEvals = evals + // Dispatch the Raft transaction future, err := p.raftApplyFuture(structs.ApplyPlanResultsRequestType, &req) if err != nil { @@ -259,6 +300,7 @@ func evaluatePlanPlacements(pool *EvaluatePool, snap *state.StateSnapshot, plan NodeAllocation: make(map[string][]*structs.Allocation), Deployment: plan.Deployment.Copy(), DeploymentUpdates: plan.DeploymentUpdates, + NodePreemptions: make(map[string][]*structs.Allocation), } // Collect all the nodeIDs @@ -304,6 +346,7 @@ func evaluatePlanPlacements(pool *EvaluatePool, snap *state.StateSnapshot, plan result.NodeAllocation = nil result.DeploymentUpdates = nil result.Deployment = nil + result.NodePreemptions = nil return true } @@ -318,6 +361,25 @@ func evaluatePlanPlacements(pool *EvaluatePool, snap *state.StateSnapshot, plan if nodeAlloc := plan.NodeAllocation[nodeID]; len(nodeAlloc) > 0 { result.NodeAllocation[nodeID] = nodeAlloc } + + if nodePreemptions := plan.NodePreemptions[nodeID]; nodePreemptions != nil { + var filteredNodePreemptions []*structs.Allocation + // Do a pass over preempted allocs in the plan to check + // whether the alloc is already in a terminal state + for _, preemptedAlloc := range nodePreemptions { + alloc, err := snap.AllocByID(nil, preemptedAlloc.ID) + if err != nil { + mErr.Errors = append(mErr.Errors, err) + } + if alloc != nil { + if !alloc.TerminalStatus() { + filteredNodePreemptions = append(filteredNodePreemptions, preemptedAlloc) + } + } + } + result.NodePreemptions[nodeID] = filteredNodePreemptions + } + return } @@ -461,6 +523,14 @@ func evaluateNodePlan(snap *state.StateSnapshot, plan *structs.Plan, nodeID stri if update := plan.NodeUpdate[nodeID]; len(update) > 0 { remove = append(remove, update...) } + + // Remove any preempted allocs + if preempted := plan.NodePreemptions[nodeID]; len(preempted) > 0 { + for _, allocs := range preempted { + remove = append(remove, allocs) + } + } + if updated := plan.NodeAllocation[nodeID]; len(updated) > 0 { for _, alloc := range updated { remove = append(remove, alloc) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 8885ebab420..d212b508be5 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -218,6 +218,43 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR } } + // TODO(preetha) do a pass to group by jobid + // Prepare preempted allocs in the plan results for update + for _, preemptedAlloc := range results.NodePreemptions { + // Look for existing alloc + existing, err := txn.First("allocs", "id", preemptedAlloc.ID) + if err != nil { + return fmt.Errorf("alloc lookup failed: %v", err) + } + + // Nothing to do if this does not exist + if existing == nil { + return nil + } + exist := existing.(*structs.Allocation) + + // Copy everything from the existing allocation + copyAlloc := exist.Copy() + + // Only update the fields set by the scheduler + copyAlloc.DesiredStatus = preemptedAlloc.DesiredStatus + copyAlloc.PreemptedByAllocation = preemptedAlloc.PreemptedByAllocation + copyAlloc.DesiredDescription = preemptedAlloc.DesiredDescription + + // Upsert the preempted allocations + if err := s.upsertAllocsImpl(index, []*structs.Allocation{copyAlloc}, txn); err != nil { + return err + } + } + + // Upsert followup evals for allocs that were preempted + + for _, eval := range results.PreemptionEvals { + if err := s.nestedUpsertEval(txn, index, eval); err != nil { + return err + } + } + txn.Commit() return nil } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index be625dcd73b..868514cf95e 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -7796,7 +7796,7 @@ const ( EvalTriggerMaxPlans = "max-plan-attempts" EvalTriggerRetryFailedAlloc = "alloc-failure" EvalTriggerQueuedAllocs = "queued-allocs" - EvalTriggerPreemption = "preempted" + EvalTriggerPreemption = "preemption" ) const ( @@ -8177,21 +8177,23 @@ func (p *Plan) AppendUpdate(alloc *Allocation, desiredStatus, desiredDesc, clien p.NodeUpdate[node] = append(existing, newAlloc) } +// AppendPreemptedAlloc is used to append an allocation that's being preempted to the plan. +// To minimize the size of the plan, this only sets a minimal set of fields in the allocation func (p *Plan) AppendPreemptedAlloc(alloc *Allocation, desiredStatus, preemptingAllocID string) { - newAlloc := new(Allocation) - *newAlloc = *alloc - // Normalize the job - newAlloc.Job = nil - - // Strip the resources as it can be rebuilt. - newAlloc.Resources = nil - + newAlloc := &Allocation{} + newAlloc.ID = alloc.ID + newAlloc.JobID = alloc.JobID + newAlloc.Namespace = alloc.Namespace newAlloc.DesiredStatus = desiredStatus newAlloc.PreemptedByAllocation = preemptingAllocID desiredDesc := fmt.Sprintf("Preempted by alloc ID %v", preemptingAllocID) newAlloc.DesiredDescription = desiredDesc + // TaskResources are needed by the plan applier to check if allocations fit + // after removing preempted allocations + newAlloc.TaskResources = alloc.TaskResources + node := alloc.NodeID // Append this alloc to slice for this node existing := p.NodePreemptions[node] diff --git a/scheduler/context_test.go b/scheduler/context_test.go index 28416b04773..8eb7792c89b 100644 --- a/scheduler/context_test.go +++ b/scheduler/context_test.go @@ -14,8 +14,9 @@ import ( func testContext(t testing.TB) (*state.StateStore, *EvalContext) { state := state.TestStateStore(t) plan := &structs.Plan{ - NodeUpdate: make(map[string][]*structs.Allocation), - NodeAllocation: make(map[string][]*structs.Allocation), + NodeUpdate: make(map[string][]*structs.Allocation), + NodeAllocation: make(map[string][]*structs.Allocation), + NodePreemptions: make(map[string][]*structs.Allocation), } logger := testlog.HCLogger(t) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index cf6ce977af8..541678fd5db 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -130,7 +130,8 @@ func (s *GenericScheduler) Process(eval *structs.Evaluation) error { structs.EvalTriggerRollingUpdate, structs.EvalTriggerQueuedAllocs, structs.EvalTriggerPeriodicJob, structs.EvalTriggerMaxPlans, structs.EvalTriggerDeploymentWatcher, structs.EvalTriggerRetryFailedAlloc, - structs.EvalTriggerFailedFollowUp: + structs.EvalTriggerFailedFollowUp, + structs.EvalTriggerPreemption: default: desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", eval.TriggeredBy) diff --git a/scheduler/preemption.go b/scheduler/preemption.go new file mode 100644 index 00000000000..f3a80d272ac --- /dev/null +++ b/scheduler/preemption.go @@ -0,0 +1,437 @@ +package scheduler + +import ( + "fmt" + "math" + "sort" + + "github.com/hashicorp/nomad/nomad/structs" +) + +// maxParallelPenalty is a score penalty applied to allocations to mitigate against +// too many allocations of the same job being preempted. This penalty is applied after the +// number of allocations being preempted exceeds max_parallel value in the job's migrate stanza +const maxParallelPenalty = 50.0 + +type PreemptionType uint8 + +const ( + NetworkResource PreemptionType = iota + CPUMemoryDiskIOPS +) + +// resourceDistance returns how close the resource is to the resource being asked for +// It is calculated by first computing a relative fraction and then measuring how close +// that is to the origin coordinate. Lower values are better +func resourceDistance(resource *structs.Resources, resourceAsk *structs.Resources) float64 { + memoryCoord, cpuCoord, iopsCoord, diskMBCoord := 0.0, 0.0, 0.0, 0.0 + if resourceAsk.MemoryMB > 0 { + memoryCoord = float64(resourceAsk.MemoryMB-resource.MemoryMB) / float64(resourceAsk.MemoryMB) + } + if resourceAsk.CPU > 0 { + cpuCoord = float64(resourceAsk.CPU-resource.CPU) / float64(resourceAsk.CPU) + } + if resourceAsk.IOPS > 0 { + iopsCoord = float64(resourceAsk.IOPS-resource.IOPS) / float64(resourceAsk.IOPS) + } + if resourceAsk.DiskMB > 0 { + diskMBCoord = float64(resourceAsk.DiskMB-resource.DiskMB) / float64(resourceAsk.DiskMB) + } + + originDist := math.Sqrt( + math.Pow(memoryCoord, 2) + + math.Pow(cpuCoord, 2) + + math.Pow(iopsCoord, 2) + + math.Pow(diskMBCoord, 2)) + return originDist +} + +// networkResourceDistance returns distance based on network megabits +func networkResourceDistance(resource *structs.Resources, resourceAsk *structs.Resources) float64 { + networkCoord := math.MaxFloat64 + if len(resourceAsk.Networks) > 0 && resourceAsk.Networks[0].MBits > 0 { + networkCoord = float64(resourceAsk.Networks[0].MBits-resource.Networks[0].MBits) / float64(resourceAsk.Networks[0].MBits) + } + + originDist := math.Sqrt( + math.Pow(networkCoord, 2)) + return originDist +} + +// getPreemptionScore is used to calculate a score (lower is better) based on the distance between +// the needed resource and requirements. A penalty is added when the choice already has some existing +// allocations in the plan that are being preempted. +func getPreemptionScore(resource *structs.Resources, resourceAsk *structs.Resources, preemptionType PreemptionType, maxParallel int, numPreemptedAllocs int) float64 { + maxParallelScorePenalty := 0.0 + if maxParallel > 0 && numPreemptedAllocs >= maxParallel { + maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty + } + switch preemptionType { + case NetworkResource: + return networkResourceDistance(resource, resourceAsk) + maxParallelScorePenalty + case CPUMemoryDiskIOPS: + return resourceDistance(resource, resourceAsk) + maxParallelScorePenalty + } + panic(fmt.Errorf("Unknown preemption type:%v", preemptionType)) +} + +// findPreemptibleAllocationsForTaskGroup computes a list of allocations to preempt to accommodate +// the resources asked for. Only allocs with a job priority < 10 of jobPriority are considered +// This method is used after network resource needs have already been met. +func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs.Allocation, resourceAsk *structs.Resources, node *structs.Node, currentPreemptions []*structs.Allocation) []*structs.Allocation { + resourcesNeeded := resourceAsk.Copy() + allocsByPriority := filterAndGroupPreemptibleAllocs(jobPriority, current) + var bestAllocs []*structs.Allocation + allRequirementsMet := false + var preemptedResources *structs.Resources + + //TODO(preetha): should add some debug logging + + nodeRemainingResources := node.Resources.Copy() + + // Initialize nodeRemainingResources with the remaining resources + // after accounting for reserved resources and all allocations + + // Subtract the reserved resources of the node + if node.Reserved != nil { + nodeRemainingResources.Subtract(node.Reserved) + } + + // Subtract current allocations + for _, alloc := range current { + nodeRemainingResources.Subtract(alloc.Resources) + } + + // Iterate over allocations grouped by priority to find preemptible allocations + for _, allocGrp := range allocsByPriority { + for len(allocGrp.allocs) > 0 && !allRequirementsMet { + closestAllocIndex := -1 + bestDistance := math.MaxFloat64 + // find the alloc with the closest distance + for index, alloc := range allocGrp.allocs { + currentPreemptionCount := computeCurrentPreemptions(alloc, currentPreemptions) + maxParallel := 0 + tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) + if tg != nil && tg.Migrate != nil { + maxParallel = tg.Migrate.MaxParallel + } + distance := getPreemptionScore(alloc.Resources, resourcesNeeded, CPUMemoryDiskIOPS, maxParallel, currentPreemptionCount) + if distance < bestDistance { + bestDistance = distance + closestAllocIndex = index + } + } + closestAlloc := allocGrp.allocs[closestAllocIndex] + + if preemptedResources == nil { + preemptedResources = closestAlloc.Resources.Copy() + } else { + preemptedResources.Add(closestAlloc.Resources) + } + availableResources := preemptedResources.Copy() + availableResources.Add(nodeRemainingResources) + + allRequirementsMet = meetsNonNetworkRequirements(availableResources, resourceAsk) + bestAllocs = append(bestAllocs, closestAlloc) + + allocGrp.allocs[closestAllocIndex] = allocGrp.allocs[len(allocGrp.allocs)-1] + allocGrp.allocs = allocGrp.allocs[:len(allocGrp.allocs)-1] + resourcesNeeded.Subtract(closestAlloc.Resources) + } + if allRequirementsMet { + break + } + } + + // Early return if all allocs examined and requirements were not met + if !allRequirementsMet { + return nil + } + + // We do another pass to eliminate unnecessary preemptions + // This filters out allocs whose resources are already covered by another alloc + filteredBestAllocs := eliminateSuperSetAllocations(bestAllocs, resourceAsk, nodeRemainingResources, resourceDistance, meetsNonNetworkRequirements) + return filteredBestAllocs + +} + +// computeCurrentPreemptions counts the number of other allocations being preempted that match the job and task group of +// the alloc under consideration. This is used as a scoring factor to minimize too many allocs of the same job being preempted at once +func computeCurrentPreemptions(currentAlloc *structs.Allocation, currentPreemptions []*structs.Allocation) int { + numCurrentPreemptionsForJob := 0 + for _, alloc := range currentPreemptions { + if alloc.JobID == currentAlloc.JobID && alloc.Namespace == currentAlloc.Namespace && alloc.TaskGroup == currentAlloc.TaskGroup { + numCurrentPreemptionsForJob++ + } + } + return numCurrentPreemptionsForJob +} + +// meetsNonNetworkRequirements checks if the first resource meets or exceeds the second resource's requirements +// This intentionally ignores network requirements, those are handled by meetsNetworkRequirements +func meetsNonNetworkRequirements(first *structs.Resources, second *structs.Resources) bool { + if first.CPU < second.CPU { + return false + } + if first.MemoryMB < second.MemoryMB { + return false + } + if first.DiskMB < second.DiskMB { + return false + } + if first.IOPS < second.IOPS { + return false + } + return true +} + +// meetsNetworkRequirements checks if the first resource meets or exceeds the second resource's network MBits requirements +func meetsNetworkRequirements(first *structs.Resources, second *structs.Resources) bool { + if len(first.Networks) == 0 || len(second.Networks) == 0 { + return false + } + return first.Networks[0].MBits >= second.Networks[0].MBits +} + +type groupedAllocs struct { + priority int + allocs []*structs.Allocation +} + +func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocation) []*groupedAllocs { + allocsByPriority := make(map[int][]*structs.Allocation) + for _, alloc := range current { + // Why is alloc.Job even nil though? + if alloc.Job == nil { + continue + } + + // Skip allocs whose priority is within a delta of 10 + // This also skips any allocs of the current job + // for which we are attempting preemption + if jobPriority-alloc.Job.Priority < 10 { + continue + } + grpAllocs, ok := allocsByPriority[alloc.Job.Priority] + if !ok { + grpAllocs = make([]*structs.Allocation, 0) + } + grpAllocs = append(grpAllocs, alloc) + allocsByPriority[alloc.Job.Priority] = grpAllocs + } + + var groupedSortedAllocs []*groupedAllocs + for priority, allocs := range allocsByPriority { + groupedSortedAllocs = append(groupedSortedAllocs, &groupedAllocs{ + priority: priority, + allocs: allocs}) + } + + // Sort by priority + sort.Slice(groupedSortedAllocs, func(i, j int) bool { + return groupedSortedAllocs[i].priority < groupedSortedAllocs[j].priority + }) + + return groupedSortedAllocs +} + +type distanceFn func(first *structs.Resources, second *structs.Resources) float64 + +type meetsRequirementsFn func(first *structs.Resources, second *structs.Resources) bool + +func eliminateSuperSetAllocations(bestAllocs []*structs.Allocation, resourceAsk *structs.Resources, + nodeRemainingResources *structs.Resources, distanceFunc distanceFn, reqFunc meetsRequirementsFn) []*structs.Allocation { + // Sort by distance reversed to surface any superset allocs first + sort.Slice(bestAllocs, func(i, j int) bool { + distance1 := distanceFunc(bestAllocs[i].Resources, resourceAsk) + distance2 := distanceFunc(bestAllocs[j].Resources, resourceAsk) + return distance1 > distance2 + }) + + var preemptedResources *structs.Resources + var filteredBestAllocs []*structs.Allocation + + // Do another pass to eliminate allocations that are a superset of other allocations + // in the preemption set + for _, alloc := range bestAllocs { + if preemptedResources == nil { + preemptedResources = alloc.Resources + } else { + preemptedResources.Add(alloc.Resources) + } + filteredBestAllocs = append(filteredBestAllocs, alloc) + availableResources := preemptedResources.Copy() + availableResources.Add(nodeRemainingResources) + + requirementsMet := reqFunc(availableResources, resourceAsk) + if requirementsMet { + break + } + } + return filteredBestAllocs +} + +// preemptForNetworkResourceAsk tries to find allocations to preempt to meet network resources. +// this needs to consider network resources at the task level and for the same task it should +// only preempt allocations that share the same network device +func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allocation, resourceAsk *structs.Resources, + netIdx *structs.NetworkIndex, currentPreemptions []*structs.Allocation) []*structs.Allocation { + + // Early return if there are no current allocs + if len(currentAllocs) == 0 { + return nil + } + + networkResourceAsk := resourceAsk.Networks[0] + deviceToAllocs := make(map[string][]*structs.Allocation) + MbitsNeeded := networkResourceAsk.MBits + reservedPortsNeeded := networkResourceAsk.ReservedPorts + + // Create a map from each device to allocs + // We do this because to place a task we have to be able to + // preempt allocations that are using the same device. + // + // This step also filters out high priority allocations and allocations + // that are not using any network resources + for _, alloc := range currentAllocs { + if alloc.Job == nil { + continue + } + + if jobPriority-alloc.Job.Priority < 10 { + continue + } + if len(alloc.Resources.Networks) > 0 { + device := alloc.Resources.Networks[0].Device + allocsForDevice := deviceToAllocs[device] + allocsForDevice = append(allocsForDevice, alloc) + deviceToAllocs[device] = allocsForDevice + } + } + + // If no existing allocations use network resources, return early + if len(deviceToAllocs) == 0 { + return nil + } + + var allocsToPreempt []*structs.Allocation + + met := false + freeBandwidth := 0 + + for device, currentAllocs := range deviceToAllocs { + totalBandwidth := netIdx.AvailBandwidth[device] + // If the device doesn't have enough total available bandwidth, skip + if totalBandwidth < MbitsNeeded { + continue + } + + // Track how much existing free bandwidth we have before preemption + freeBandwidth = totalBandwidth - netIdx.UsedBandwidth[device] + + preemptedBandwidth := 0 + + // Reset allocsToPreempt since we don't want to preempt across devices for the same task + allocsToPreempt = nil + + // Build map from used reserved ports to allocation + usedPortToAlloc := make(map[int]*structs.Allocation) + + // First try to satisfy needed reserved ports + if len(reservedPortsNeeded) > 0 { + for _, alloc := range currentAllocs { + for _, tr := range alloc.TaskResources { + reservedPorts := tr.Networks[0].ReservedPorts + for _, p := range reservedPorts { + usedPortToAlloc[p.Value] = alloc + } + } + } + + // Look for allocs that are using reserved ports needed + for _, port := range reservedPortsNeeded { + alloc, ok := usedPortToAlloc[port.Value] + if ok { + preemptedBandwidth += alloc.Resources.Networks[0].MBits + allocsToPreempt = append(allocsToPreempt, alloc) + } + } + + // Remove allocs that were preempted to satisfy reserved ports + currentAllocs = structs.RemoveAllocs(currentAllocs, allocsToPreempt) + } + + // If bandwidth requirements have been met, stop + if preemptedBandwidth+freeBandwidth >= MbitsNeeded { + met = true + break + } + + // Split by priority + allocsByPriority := filterAndGroupPreemptibleAllocs(jobPriority, currentAllocs) + + for _, allocsGrp := range allocsByPriority { + allocs := allocsGrp.allocs + + // Sort by distance function that takes into account needed MBits + // as well as penalty for preempting an allocation + // whose task group already has existing preemptions + sort.Slice(allocs, func(i, j int) bool { + firstAlloc := allocs[i] + currentPreemptionCount1 := computeCurrentPreemptions(firstAlloc, currentPreemptions) + + // Look up configured maxParallel value for these allocation's task groups + var maxParallel1, maxParallel2 int + tg1 := allocs[i].Job.LookupTaskGroup(allocs[i].TaskGroup) + if tg1 != nil && tg1.Migrate != nil { + maxParallel1 = tg1.Migrate.MaxParallel + } + distance1 := getPreemptionScore(allocs[i].Resources, resourceAsk, NetworkResource, maxParallel1, currentPreemptionCount1) + + secondAlloc := allocs[j] + currentPreemptionCount2 := computeCurrentPreemptions(secondAlloc, currentPreemptions) + tg2 := secondAlloc.Job.LookupTaskGroup(secondAlloc.TaskGroup) + if tg2 != nil && tg2.Migrate != nil { + maxParallel2 = tg2.Migrate.MaxParallel + } + distance2 := getPreemptionScore(secondAlloc.Resources, resourceAsk, NetworkResource, maxParallel2, currentPreemptionCount2) + + return distance1 < distance2 + }) + + for _, alloc := range allocs { + preemptedBandwidth += alloc.Resources.Networks[0].MBits + allocsToPreempt = append(allocsToPreempt, alloc) + if preemptedBandwidth+freeBandwidth >= MbitsNeeded { + met = true + break + } + } + if met { + break + } + } + if met { + break + } + } + if len(allocsToPreempt) == 0 { + return nil + } + + // Build a resource object with just the network Mbits filled in + // Its safe to use the first preempted allocation's network resource + // here because all allocations preempted will be from the same device + nodeRemainingResources := &structs.Resources{ + Networks: []*structs.NetworkResource{ + { + Device: allocsToPreempt[0].Resources.Networks[0].Device, + MBits: freeBandwidth, + }, + }, + } + + // Do a final pass to eliminate any superset allocations + filteredBestAllocs := eliminateSuperSetAllocations(allocsToPreempt, resourceAsk, nodeRemainingResources, networkResourceDistance, meetsNetworkRequirements) + return filteredBestAllocs +} diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go new file mode 100644 index 00000000000..cacef52bd93 --- /dev/null +++ b/scheduler/preemption_test.go @@ -0,0 +1,789 @@ +package scheduler + +import ( + "testing" + + "fmt" + + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +func TestResourceDistance(t *testing.T) { + resourceAsk := &structs.Resources{ + CPU: 2048, + MemoryMB: 512, + IOPS: 300, + DiskMB: 4096, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + } + + type testCase struct { + allocResource *structs.Resources + expectedDistance string + } + + testCases := []*testCase{ + { + &structs.Resources{ + CPU: 2048, + MemoryMB: 512, + IOPS: 300, + DiskMB: 4096, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + "0.000", + }, + { + &structs.Resources{ + CPU: 1024, + MemoryMB: 400, + IOPS: 200, + DiskMB: 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + "0.986", + }, + { + &structs.Resources{ + CPU: 1024, + MemoryMB: 200, + IOPS: 200, + DiskMB: 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 512, + }, + }, + }, + "1.138", + }, + { + &structs.Resources{ + CPU: 8192, + MemoryMB: 200, + IOPS: 200, + DiskMB: 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 512, + }, + }, + }, + "3.169", + }, + { + &structs.Resources{ + CPU: 2048, + MemoryMB: 500, + IOPS: 300, + DiskMB: 4096, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, + }, + }, + "0.023", + }, + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + require := require.New(t) + require.Equal(tc.expectedDistance, fmt.Sprintf("%3.3f", resourceDistance(tc.allocResource, resourceAsk))) + }) + + } + +} + +func TestPreemption(t *testing.T) { + type testCase struct { + desc string + currentAllocations []*structs.Allocation + nodeReservedCapacity *structs.Resources + nodeCapacity *structs.Resources + 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 = 30 + + // Create some persistent alloc ids to use in test cases + allocIDs := []string{uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate()} + + nodeResources := &structs.Resources{ + CPU: 4000, + MemoryMB: 8192, + DiskMB: 100 * 1024, + IOPS: 150, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + CIDR: "192.168.0.100/32", + MBits: 1000, + }, + }, + } + reservedNodeResources := &structs.Resources{ + CPU: 100, + MemoryMB: 256, + DiskMB: 4 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "ssh", Value: 22}}, + MBits: 1, + }, + }, + } + + 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, + IOPS: 150, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 50, + }, + }, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 100, + 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, + IOPS: 150, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 50, + }, + }, + })}, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + jobPriority: 100, + resourceAsk: &structs.Resources{ + CPU: 4000, + MemoryMB: 8192, + DiskMB: 4 * 1024, + IOPS: 300, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "ssh", Value: 22}}, + MBits: 1, + }, + }, + }, + }, + { + 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, + IOPS: 150, + 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, + IOPS: 50, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 500, + }, + }, + }), + createAlloc(allocIDs[2], lowPrioJob, &structs.Resources{ + CPU: 200, + MemoryMB: 256, + DiskMB: 4 * 1024, + IOPS: 50, + 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: nodeResources, + 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: "Preemption needed for all resources except network", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 2800, + MemoryMB: 2256, + DiskMB: 40 * 1024, + IOPS: 100, + 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, + IOPS: 50, + 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: nodeResources, + 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, + IOPS: 50, + 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, + IOPS: 10, + 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, + IOPS: 10, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 300, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + 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, + IOPS: 150, + 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, + IOPS: 50, + 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, + IOPS: 50, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 200, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + 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 needds", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + IOPS: 50, + 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, + IOPS: 50, + 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, + IOPS: 50, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 100, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + 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, + IOPS: 50, + 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, + IOPS: 10, + 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, + IOPS: 10, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 300, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + 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, + IOPS: 10, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 300, + }, + }, + }), + }, + preemptedAllocIDs: map[string]struct{}{ + allocIDs[1]: {}, + }, + }, + // 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, + IOPS: 50, + 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: nodeResources, + 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.Resources = tc.nodeCapacity + node.Reserved = tc.nodeReservedCapacity + + state, ctx := testContext(t) + nodes := []*RankedNode{ + { + Node: node, + }, + } + state.UpsertNode(1000, node) + for _, alloc := range tc.currentAllocations { + alloc.NodeID = node.ID + } + require := require.New(t) + err := state.UpsertAllocs(1001, tc.currentAllocations) + require.Nil(err) + if tc.currentPreemptions != nil { + ctx.plan.NodePreemptions[node.ID] = tc.currentPreemptions + } + static := NewStaticRankIterator(ctx, nodes) + binPackIter := NewBinPackIterator(ctx, static, true, tc.jobPriority) + + taskGroup := &structs.TaskGroup{ + EphemeralDisk: &structs.EphemeralDisk{}, + Tasks: []*structs.Task{ + { + Name: "web", + Resources: tc.resourceAsk, + }, + }, + } + binPackIter.SetTaskGroup(taskGroup) + option := binPackIter.Next() + if tc.preemptedAllocIDs == nil { + require.Nil(option) + } else { + require.NotNil(option) + preemptedAllocs := option.PreemptedAllocs + require.Equal(len(tc.preemptedAllocIDs), len(preemptedAllocs)) + for _, alloc := range preemptedAllocs { + _, ok := tc.preemptedAllocIDs[alloc.ID] + require.True(ok) + } + } + }) + } +} + +// helper method to create allocations with given jobs and resources +func createAlloc(id string, job *structs.Job, resource *structs.Resources) *structs.Allocation { + alloc := &structs.Allocation{ + ID: id, + Job: job, + JobID: job.ID, + TaskResources: map[string]*structs.Resources{ + "web": resource, + }, + Resources: resource, + Namespace: structs.DefaultNamespace, + EvalID: uuid.Generate(), + DesiredStatus: structs.AllocDesiredStatusRun, + ClientStatus: structs.AllocClientStatusRunning, + TaskGroup: "web", + } + return alloc +} diff --git a/scheduler/rank.go b/scheduler/rank.go index d2e70a4e680..3c7adceaa37 100644 --- a/scheduler/rank.go +++ b/scheduler/rank.go @@ -25,6 +25,10 @@ type RankedNode struct { // Allocs is used to cache the proposed allocations on the // node. This can be shared between iterators that require it. Proposed []*structs.Allocation + + // PreemptedAllocs is used by the BinpackIterator to identify allocs + // that should be preempted in order to make the placement + PreemptedAllocs []*structs.Allocation } func (r *RankedNode) GoString() string { @@ -195,6 +199,16 @@ OUTER: DiskMB: int64(iter.taskGroup.EphemeralDisk.SizeMB), }, } + + var allocsToPreempt []*structs.Allocation + + // Count the number of existing preemptions + allPreemptions := iter.ctx.Plan().NodePreemptions + var currentPreemptions []*structs.Allocation + for _, allocs := range allPreemptions { + currentPreemptions = append(currentPreemptions, allocs...) + } + for _, task := range iter.taskGroup.Tasks { // Allocate the resources taskResources := &structs.AllocatedTaskResources{ @@ -211,10 +225,40 @@ OUTER: ask := task.Resources.Networks[0].Copy() offer, err := netIdx.AssignNetwork(ask) if offer == nil { - iter.ctx.Metrics().ExhaustedNode(option.Node, - fmt.Sprintf("network: %s", err)) + // If eviction is not enabled, mark this node as exhausted and continue + if !iter.evict { + iter.ctx.Metrics().ExhaustedNode(option.Node, + fmt.Sprintf("network: %s", err)) + netIdx.Release() + continue OUTER + } + + // Look for preemptible allocations to satisfy the network resource for this task + preemptedAllocsForTaskNetwork := preemptForNetworkResourceAsk(iter.priority, proposed, taskResources, netIdx, currentPreemptions) + if preemptedAllocsForTaskNetwork == nil { + iter.ctx.Metrics().ExhaustedNode(option.Node, + fmt.Sprintf("unable to meet network resource %v after preemption", ask)) + netIdx.Release() + continue OUTER + } + allocsToPreempt = append(allocsToPreempt, preemptedAllocsForTaskNetwork...) + + // First subtract out preempted allocations + proposed = structs.RemoveAllocs(proposed, preemptedAllocsForTaskNetwork) + + // Reset the network index and try the offer again netIdx.Release() - continue OUTER + netIdx = structs.NewNetworkIndex() + netIdx.SetNode(option.Node) + netIdx.AddAllocs(proposed) + + offer, err = netIdx.AssignNetwork(ask) + if offer == nil { + iter.ctx.Metrics().ExhaustedNode(option.Node, + fmt.Sprintf("unexecpted error, unable to create offer after preempting:%v", err)) + netIdx.Release() + continue OUTER + } } // Reserve this to prevent another task from colliding @@ -234,19 +278,32 @@ OUTER: // Add the resources we are trying to fit proposed = append(proposed, &structs.Allocation{AllocatedResources: total}) - // Check if these allocations fit, if they do not, simply skip this node + // Check if these allocations fit fit, dim, util, _ := structs.AllocsFit(option.Node, proposed, netIdx) netIdx.Release() if !fit { - iter.ctx.Metrics().ExhaustedNode(option.Node, dim) - continue + // Skip the node if evictions are not enabled + if !iter.evict { + iter.ctx.Metrics().ExhaustedNode(option.Node, dim) + continue + } + // If eviction is enabled and the node doesn't fit the alloc, check if + // any allocs can be preempted + + // Remove the last element containing the current placement from proposed allocs + current := proposed[:len(proposed)-1] + preemptForTaskGroup := findPreemptibleAllocationsForTaskGroup(iter.priority, current, total, option.Node, currentPreemptions) + allocsToPreempt = append(allocsToPreempt, preemptForTaskGroup...) + // If we were unable to find preempted allocs to meet these requirements + // mark as exhausted and continue + if len(preemptForTaskGroup) == 0 { + iter.ctx.Metrics().ExhaustedNode(option.Node, dim) + continue + } + } + if len(allocsToPreempt) > 0 { + option.PreemptedAllocs = allocsToPreempt } - - // XXX: For now we completely ignore evictions. We should use that flag - // to determine if its possible to evict other lower priority allocations - // to make room. This explodes the search space, so it must be done - // carefully. - // Score the fit normally otherwise fitness := structs.ScoreFit(option.Node, util) normalizedFit := fitness / binPackingMaxFitScore diff --git a/scheduler/system_sched.go b/scheduler/system_sched.go index 60202eb687c..912440aefef 100644 --- a/scheduler/system_sched.go +++ b/scheduler/system_sched.go @@ -60,7 +60,7 @@ func (s *SystemScheduler) Process(eval *structs.Evaluation) error { // Verify the evaluation trigger reason is understood switch eval.TriggeredBy { case structs.EvalTriggerJobRegister, structs.EvalTriggerNodeUpdate, structs.EvalTriggerFailedFollowUp, - structs.EvalTriggerJobDeregister, structs.EvalTriggerRollingUpdate, + structs.EvalTriggerJobDeregister, structs.EvalTriggerRollingUpdate, structs.EvalTriggerPreemption, structs.EvalTriggerDeploymentWatcher, structs.EvalTriggerNodeDrain: default: desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", @@ -347,6 +347,24 @@ func (s *SystemScheduler) computePlacements(place []allocTuple) error { alloc.PreviousAllocation = missing.Alloc.ID } + // If this placement involves preemption, set DesiredState to stop for those allocations + if option.PreemptedAllocs != nil { + var preemptedAllocIDs []string + for _, stop := range option.PreemptedAllocs { + s.plan.AppendPreemptedAlloc(stop, structs.AllocDesiredStatusEvict, alloc.ID) + + preemptedAllocIDs = append(preemptedAllocIDs, stop.ID) + if s.eval.AnnotatePlan && s.plan.Annotations != nil { + s.plan.Annotations.PreemptedAllocs = append(s.plan.Annotations.PreemptedAllocs, stop.Stub()) + if s.plan.Annotations.DesiredTGUpdates != nil { + desired := s.plan.Annotations.DesiredTGUpdates[missing.TaskGroup.Name] + desired.Evict += 1 + } + } + } + alloc.PreemptedAllocations = preemptedAllocIDs + } + s.plan.AppendAlloc(alloc) } else { // Lazy initialize the failed map diff --git a/scheduler/system_sched_test.go b/scheduler/system_sched_test.go index 3d78b706136..d2cf1f5f8d2 100644 --- a/scheduler/system_sched_test.go +++ b/scheduler/system_sched_test.go @@ -6,11 +6,14 @@ import ( "testing" "time" + "fmt" + memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" ) func TestSystemSched_JobRegister(t *testing.T) { @@ -218,7 +221,7 @@ func TestSystemSched_JobRegister_EphemeralDiskConstraint(t *testing.T) { JobID: job1.ID, Status: structs.EvalStatusPending, } - noErr(t, h.State.UpsertEvals(h.NextIndex(), []*structs.Evaluation{eval})) + noErr(t, h.State.UpsertEvals(h.NextIndex(), []*structs.Evaluation{eval1})) // Process the evaluation if err := h1.Process(NewSystemScheduler, eval1); err != nil { @@ -274,15 +277,31 @@ func TestSystemSched_ExhaustResources(t *testing.T) { JobID: job.ID, Status: structs.EvalStatusPending, } - noErr(t, h.State.UpsertEvals(h.NextIndex(), []*structs.Evaluation{eval})) + noErr(t, h.State.UpsertEvals(h.NextIndex(), []*structs.Evaluation{eval1})) // Process the evaluation if err := h.Process(NewSystemScheduler, eval1); err != nil { t.Fatalf("err: %v", err) } - // Ensure that we have one allocation queued from the system job eval + // System scheduler will preempt the service job and would have placed eval1 + require := require.New(t) + + newPlan := h.Plans[1] + require.Len(newPlan.NodeAllocation, 1) + require.Len(newPlan.NodePreemptions, 1) + + for _, allocList := range newPlan.NodeAllocation { + require.Len(allocList, 1) + require.Equal(job.ID, allocList[0].JobID) + } + + for _, allocList := range newPlan.NodePreemptions { + require.Len(allocList, 1) + require.Equal(svcJob.ID, allocList[0].JobID) + } + // Ensure that we have no queued allocations on the second eval queued := h.Evals[1].QueuedAllocations["web"] - if queued != 1 { + if queued != 0 { t.Fatalf("expected: %v, actual: %v", 1, queued) } } @@ -1529,3 +1548,322 @@ func TestSystemSched_QueuedAllocsMultTG(t *testing.T) { h.AssertEvalStatus(t, structs.EvalStatusComplete) } + +func TestSystemSched_Preemption(t *testing.T) { + h := NewHarness(t) + + // Create nodes + var nodes []*structs.Node + for i := 0; i < 2; i++ { + node := mock.Node() + node.Resources = &structs.Resources{ + CPU: 3072, + MemoryMB: 5034, + DiskMB: 20 * 1024, + IOPS: 150, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + CIDR: "192.168.0.100/32", + MBits: 1000, + }, + }, + } + noErr(t, h.State.UpsertNode(h.NextIndex(), node)) + nodes = append(nodes, node) + } + + // Create some low priority batch jobs and allocations for them + // One job uses a reserved port + job1 := mock.BatchJob() + job1.Type = structs.JobTypeBatch + job1.Priority = 20 + job1.TaskGroups[0].Tasks[0].Resources = &structs.Resources{ + CPU: 512, + MemoryMB: 1024, + Networks: []*structs.NetworkResource{ + { + MBits: 200, + ReservedPorts: []structs.Port{ + { + Label: "web", + Value: 80, + }, + }, + }, + }, + } + + alloc1 := mock.Alloc() + alloc1.Job = job1 + alloc1.JobID = job1.ID + alloc1.NodeID = nodes[0].ID + alloc1.Name = "my-job[0]" + alloc1.TaskGroup = job1.TaskGroups[0].Name + alloc1.Resources = &structs.Resources{ + CPU: 512, + MemoryMB: 1024, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 200, + ReservedPorts: []structs.Port{ + { + Label: "web", + Value: 80, + }, + }, + }, + }, + } + alloc1.TaskResources = map[string]*structs.Resources{ + "web": { + CPU: 512, + MemoryMB: 1024, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 200, + ReservedPorts: []structs.Port{ + { + Label: "web", + Value: 80, + }, + }, + }, + }, + }, + } + noErr(t, h.State.UpsertJob(h.NextIndex(), job1)) + + job2 := mock.BatchJob() + job2.Type = structs.JobTypeBatch + job2.Priority = 20 + job2.TaskGroups[0].Tasks[0].Resources = &structs.Resources{ + CPU: 512, + MemoryMB: 1024, + Networks: []*structs.NetworkResource{ + { + MBits: 200, + }, + }, + } + + alloc2 := mock.Alloc() + alloc2.Job = job2 + alloc2.JobID = job2.ID + alloc2.NodeID = nodes[0].ID + alloc2.Name = "my-job[2]" + alloc2.TaskGroup = job2.TaskGroups[0].Name + alloc2.Resources = &structs.Resources{ + CPU: 512, + MemoryMB: 1024, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 200, + }, + }, + } + alloc2.TaskResources = map[string]*structs.Resources{ + "web": { + CPU: 512, + MemoryMB: 1024, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 200, + }, + }, + }, + } + noErr(t, h.State.UpsertJob(h.NextIndex(), job2)) + + job3 := mock.Job() + job3.Type = structs.JobTypeBatch + job3.Priority = 40 + job3.TaskGroups[0].Tasks[0].Resources = &structs.Resources{ + CPU: 1024, + MemoryMB: 2048, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 400, + }, + }, + } + + alloc3 := mock.Alloc() + alloc3.Job = job3 + alloc3.JobID = job3.ID + alloc3.NodeID = nodes[0].ID + alloc3.Name = "my-job[0]" + alloc3.TaskGroup = job3.TaskGroups[0].Name + alloc3.Resources = &structs.Resources{ + CPU: 1024, + MemoryMB: 25, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 400, + }, + }, + } + alloc3.TaskResources = map[string]*structs.Resources{ + "web": { + CPU: 1024, + MemoryMB: 25, + DiskMB: 5 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 400, + }, + }, + }, + } + noErr(t, h.State.UpsertAllocs(h.NextIndex(), []*structs.Allocation{alloc1, alloc2, alloc3})) + + // Create a high priority job and allocs for it + // These allocs should not be preempted + + job4 := mock.BatchJob() + job4.Type = structs.JobTypeBatch + job4.Priority = 100 + job4.TaskGroups[0].Tasks[0].Resources = &structs.Resources{ + CPU: 1024, + MemoryMB: 2048, + Networks: []*structs.NetworkResource{ + { + MBits: 100, + }, + }, + } + + alloc4 := mock.Alloc() + alloc4.Job = job4 + alloc4.JobID = job4.ID + alloc4.NodeID = nodes[0].ID + alloc4.Name = "my-job4[0]" + alloc4.TaskGroup = job4.TaskGroups[0].Name + alloc4.Resources = &structs.Resources{ + CPU: 1024, + MemoryMB: 2048, + DiskMB: 2 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 100, + }, + }, + } + alloc4.TaskResources = map[string]*structs.Resources{ + "web": { + CPU: 1024, + MemoryMB: 2048, + DiskMB: 2 * 1024, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 100, + }, + }, + }, + } + noErr(t, h.State.UpsertJob(h.NextIndex(), job4)) + noErr(t, h.State.UpsertAllocs(h.NextIndex(), []*structs.Allocation{alloc4})) + + // Create a system job such that it would need to preempt both allocs to succeed + job := mock.SystemJob() + job.TaskGroups[0].Tasks[0].Resources = &structs.Resources{ + CPU: 1948, + MemoryMB: 256, + Networks: []*structs.NetworkResource{ + { + MBits: 800, + DynamicPorts: []structs.Port{{Label: "http"}}, + }, + }, + } + noErr(t, h.State.UpsertJob(h.NextIndex(), job)) + + // Create a mock evaluation to register the job + eval := &structs.Evaluation{ + Namespace: structs.DefaultNamespace, + ID: uuid.Generate(), + Priority: job.Priority, + TriggeredBy: structs.EvalTriggerJobRegister, + JobID: job.ID, + Status: structs.EvalStatusPending, + } + noErr(t, h.State.UpsertEvals(h.NextIndex(), []*structs.Evaluation{eval})) + + // Process the evaluation + err := h.Process(NewSystemScheduler, eval) + require := require.New(t) + require.Nil(err) + + // Ensure a single plan + require.Equal(1, len(h.Plans)) + plan := h.Plans[0] + + // Ensure the plan doesn't have annotations. + require.Nil(plan.Annotations) + + // Ensure the plan allocated on both nodes + var planned []*structs.Allocation + preemptingAllocId := "" + require.Equal(2, len(plan.NodeAllocation)) + + // The alloc that got placed on node 1 is the preemptor + for _, allocList := range plan.NodeAllocation { + planned = append(planned, allocList...) + for _, alloc := range allocList { + if alloc.NodeID == nodes[0].ID { + preemptingAllocId = alloc.ID + } + } + } + + // Lookup the allocations by JobID + ws := memdb.NewWatchSet() + out, err := h.State.AllocsByJob(ws, job.Namespace, job.ID, false) + noErr(t, err) + + // Ensure all allocations placed + require.Equal(2, len(out)) + + // Verify that one node has preempted allocs + require.NotNil(plan.NodePreemptions[nodes[0].ID]) + preemptedAllocs := plan.NodePreemptions[nodes[0].ID] + + // Verify that three jobs have preempted allocs + require.Equal(3, len(preemptedAllocs)) + + expectedPreemptedJobIDs := []string{job1.ID, job2.ID, job3.ID} + + // We expect job1, job2 and job3 to have preempted allocations + // job4 should not have any allocs preempted + for _, alloc := range preemptedAllocs { + require.Contains(expectedPreemptedJobIDs, alloc.JobID) + } + // Look up the preempted allocs by job ID + ws = memdb.NewWatchSet() + + for _, jobId := range expectedPreemptedJobIDs { + out, err = h.State.AllocsByJob(ws, structs.DefaultNamespace, jobId, false) + noErr(t, err) + for _, alloc := range out { + require.Equal(structs.AllocDesiredStatusEvict, alloc.DesiredStatus) + require.Equal(fmt.Sprintf("Preempted by alloc ID %v", preemptingAllocId), alloc.DesiredDescription) + } + } + + h.AssertEvalStatus(t, structs.EvalStatusComplete) + +} diff --git a/scheduler/testing.go b/scheduler/testing.go index 0410b1190d7..0a527ede973 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -96,6 +96,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er result := new(structs.PlanResult) result.NodeUpdate = plan.NodeUpdate result.NodeAllocation = plan.NodeAllocation + result.NodePreemptions = plan.NodePreemptions result.AllocIndex = index // Flatten evicts and allocs @@ -116,6 +117,18 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er } } + // Set create and modify time for preempted allocs and flatten them + var preemptedAllocs []*structs.Allocation + for _, preemptions := range result.NodePreemptions { + for _, alloc := range preemptions { + if alloc.CreateTime == 0 { + alloc.CreateTime = now + } + alloc.ModifyTime = now + preemptedAllocs = append(preemptedAllocs, alloc) + } + } + // Setup the update request req := structs.ApplyPlanResultsRequest{ AllocUpdateRequest: structs.AllocUpdateRequest{ @@ -125,6 +138,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er Deployment: plan.Deployment, DeploymentUpdates: plan.DeploymentUpdates, EvalID: plan.EvalID, + NodePreemptions: preemptedAllocs, } // Apply the full plan From b5cbd73924f0434c3817d9885d2e2a604ab58d9a Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Sun, 23 Sep 2018 20:09:14 -0500 Subject: [PATCH 06/38] Fix comment --- nomad/plan_apply.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 66fd64adda6..e909dbe8edd 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -204,10 +204,8 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap var evals []*structs.Evaluation for preemptedJobID, _ := range preemptedJobIDs { - job, _ := p.State().JobByID(nil, preemptedJobID.Namespace, preemptedJobID.ID) // TODO Fix me + job, _ := p.State().JobByID(nil, preemptedJobID.Namespace, preemptedJobID.ID) if job != nil { - //TODO(preetha): This eval is missing class eligibility related fields - // need to figure out how to set them per job eval := &structs.Evaluation{ ID: uuid.Generate(), Namespace: job.Namespace, From 13e314cb5650e84f30d920456acddaae1a604db7 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Sun, 23 Sep 2018 20:42:03 -0500 Subject: [PATCH 07/38] Fix logic bug, unit test for plan apply method in state store --- nomad/state/state_store.go | 4 +- nomad/state/state_store_test.go | 79 +++++++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 3 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index d212b508be5..71db85d5745 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -218,7 +218,6 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR } } - // TODO(preetha) do a pass to group by jobid // Prepare preempted allocs in the plan results for update for _, preemptedAlloc := range results.NodePreemptions { // Look for existing alloc @@ -229,7 +228,7 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR // Nothing to do if this does not exist if existing == nil { - return nil + continue } exist := existing.(*structs.Allocation) @@ -248,7 +247,6 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR } // Upsert followup evals for allocs that were preempted - for _, eval := range results.PreemptionEvals { if err := s.nestedUpsertEval(txn, index, eval); err != nil { return err diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 79cd8559f98..194eaedac28 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -244,6 +244,85 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { assert.EqualValues(1001, evalOut.ModifyIndex) } +// This test checks that: +// 1) Preempted allocations in plan results are updated +// 2) Evals are inserted for preempted jobs +func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { + state := testStateStore(t) + alloc := mock.Alloc() + job := alloc.Job + alloc.Job = nil + + require := require.New(t) + + err := state.UpsertJob(999, job) + require.Nil(err) + + eval := mock.Eval() + eval.JobID = job.ID + + // Create an eval + err = state.UpsertEvals(1, []*structs.Evaluation{eval}) + require.Nil(err) + + // Insert alloc that'll be preempted in the plan + preemptedAlloc := mock.Alloc() + err = state.UpsertAllocs(2, []*structs.Allocation{preemptedAlloc}) + require.Nil(err) + + minimalPreemptedAlloc := &structs.Allocation{ + ID: preemptedAlloc.ID, + Namespace: preemptedAlloc.Namespace, + DesiredStatus: structs.AllocDesiredStatusEvict, + DesiredDescription: fmt.Sprintf("Preempted by allocation %v", alloc.ID), + } + + eval2 := mock.Eval() + eval2.JobID = preemptedAlloc.JobID + + // Create a plan result + res := structs.ApplyPlanResultsRequest{ + AllocUpdateRequest: structs.AllocUpdateRequest{ + Alloc: []*structs.Allocation{alloc}, + Job: job, + }, + EvalID: eval.ID, + NodePreemptions: []*structs.Allocation{minimalPreemptedAlloc}, + PreemptionEvals: []*structs.Evaluation{eval2}, + } + + err = state.UpsertPlanResults(1000, &res) + require.Nil(err) + + ws := memdb.NewWatchSet() + + // Verify alloc and eval created by plan + out, err := state.AllocByID(ws, alloc.ID) + require.Nil(err) + require.Equal(alloc, out) + + index, err := state.Index("allocs") + require.Nil(err) + require.EqualValues(1000, index) + + evalOut, err := state.EvalByID(ws, eval.ID) + require.Nil(err) + require.NotNil(evalOut) + require.EqualValues(1000, evalOut.ModifyIndex) + + // Verify preempted alloc and eval for preempted job + preempted, err := state.AllocByID(ws, preemptedAlloc.ID) + require.Nil(err) + require.Equal(preempted.DesiredStatus, structs.AllocDesiredStatusEvict) + require.Equal(preempted.DesiredDescription, fmt.Sprintf("Preempted by allocation %v", alloc.ID)) + + preemptedJobEval, err := state.EvalByID(ws, eval2.ID) + require.Nil(err) + require.NotNil(preemptedJobEval) + require.EqualValues(1000, preemptedJobEval.ModifyIndex) + +} + // This test checks that deployment updates are applied correctly func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) { state := testStateStore(t) From fd6bff23329f47f8360d6b2c5aece10982674b8b Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Sun, 23 Sep 2018 21:26:24 -0500 Subject: [PATCH 08/38] Fix linting and better comments --- scheduler/preemption.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index f3a80d272ac..1eb2d2f3218 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -363,7 +363,6 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo // If bandwidth requirements have been met, stop if preemptedBandwidth+freeBandwidth >= MbitsNeeded { - met = true break } @@ -407,10 +406,12 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo break } } + // If we met bandwidth needs we can break out of loop that's iterating by priority within a device if met { break } } + // If we met bandwidth needs we can break out of loop that's iterating by allocs sharing the same network device if met { break } From 51c5bae393d530be41346734219dc24db3f92445 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 27 Sep 2018 15:10:15 -0500 Subject: [PATCH 09/38] Fix linting --- nomad/plan_apply.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index e909dbe8edd..f48d7c4fbe2 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -203,7 +203,7 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap } var evals []*structs.Evaluation - for preemptedJobID, _ := range preemptedJobIDs { + for preemptedJobID := range preemptedJobIDs { job, _ := p.State().JobByID(nil, preemptedJobID.Namespace, preemptedJobID.ID) if job != nil { eval := &structs.Evaluation{ From 784b96c104373bbf0005ed541d37e1411becb4ea Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 27 Sep 2018 23:27:38 -0500 Subject: [PATCH 10/38] Support for new scheduler config API, first use case is to disable preemption --- api/operator.go | 45 ++++++++++++ api/operator_test.go | 66 ++++++++++++++++++ command/agent/http.go | 2 + command/agent/operator_endpoint.go | 64 +++++++++++++++++ command/agent/operator_endpoint_test.go | 92 +++++++++++++++++++++++++ nomad/fsm.go | 19 +++++ nomad/fsm_test.go | 44 ++++++++++++ nomad/leader.go | 25 +++++++ nomad/operator_endpoint.go | 61 ++++++++++++++++ nomad/state/schema.go | 19 +++++ nomad/state/state_store.go | 78 +++++++++++++++++++++ nomad/structs/operator.go | 23 +++++++ nomad/structs/structs.go | 1 + 13 files changed, 539 insertions(+) diff --git a/api/operator.go b/api/operator.go index be2ba800569..85a28fe8d01 100644 --- a/api/operator.go +++ b/api/operator.go @@ -1,5 +1,7 @@ package api +import "strconv" + // Operator can be used to perform low-level operator tasks for Nomad. type Operator struct { c *Client @@ -106,3 +108,46 @@ func (op *Operator) RaftRemovePeerByID(id string, q *WriteOptions) error { resp.Body.Close() return nil } + +type SchedulerConfiguration struct { + // EnablePreemption specifies whether to enable eviction of lower + // priority jobs to place higher priority jobs. + EnablePreemption bool + + // CreateIndex/ModifyIndex store the create/modify indexes of this configuration. + CreateIndex uint64 + ModifyIndex uint64 +} + +// SchedulerGetConfiguration is used to query the current Scheduler configuration. +func (op *Operator) SchedulerGetConfiguration(q *QueryOptions) (*SchedulerConfiguration, *QueryMeta, error) { + var resp SchedulerConfiguration + qm, err := op.c.query("/v1/operator/scheduler/config", &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, qm, nil +} + +// SchedulerSetConfiguration is used to set the current Scheduler configuration. +func (op *Operator) SchedulerSetConfiguration(conf *SchedulerConfiguration, q *WriteOptions) (*WriteMeta, error) { + var out bool + wm, err := op.c.write("/v1/operator/scheduler/config", conf, &out, q) + if err != nil { + return nil, err + } + return wm, nil +} + +// SchedulerCASConfiguration is used to perform a Check-And-Set update on the +// Scheduler configuration. The ModifyIndex value will be respected. Returns +// true on success or false on failures. +func (op *Operator) SchedulerCASConfiguration(conf *SchedulerConfiguration, q *WriteOptions) (bool, *WriteMeta, error) { + var out bool + wm, err := op.c.write("/v1/operator/scheduler/config?cas="+strconv.FormatUint(conf.ModifyIndex, 10), conf, &out, q) + if err != nil { + return false, nil, err + } + + return out, wm, nil +} diff --git a/api/operator_test.go b/api/operator_test.go index 5b13fc66c5b..277d85afe5f 100644 --- a/api/operator_test.go +++ b/api/operator_test.go @@ -3,6 +3,9 @@ package api import ( "strings" "testing" + + "github.com/hashicorp/consul/testutil/retry" + "github.com/stretchr/testify/require" ) func TestOperator_RaftGetConfiguration(t *testing.T) { @@ -51,3 +54,66 @@ func TestOperator_RaftRemovePeerByID(t *testing.T) { t.Fatalf("err: %v", err) } } + +func TestAPI_OperatorSchedulerGetSetConfiguration(t *testing.T) { + t.Parallel() + require := require.New(t) + c, s := makeClient(t, nil, nil) + defer s.Stop() + + operator := c.Operator() + var config *SchedulerConfiguration + retry.Run(t, func(r *retry.R) { + var err error + config, _, err = operator.SchedulerGetConfiguration(nil) + r.Check(err) + }) + require.False(config.EnablePreemption) + + // Change a config setting + newConf := &SchedulerConfiguration{EnablePreemption: true} + _, err := operator.SchedulerSetConfiguration(newConf, nil) + require.Nil(err) + + config, _, err = operator.SchedulerGetConfiguration(nil) + require.Nil(err) + require.True(config.EnablePreemption) +} + +func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { + t.Parallel() + require := require.New(t) + c, s := makeClient(t, nil, nil) + defer s.Stop() + + operator := c.Operator() + var config *SchedulerConfiguration + retry.Run(t, func(r *retry.R) { + var err error + config, _, err = operator.SchedulerGetConfiguration(nil) + r.Check(err) + }) + require.False(config.EnablePreemption) + + // Pass an invalid ModifyIndex + { + newConf := &SchedulerConfiguration{ + EnablePreemption: true, + ModifyIndex: config.ModifyIndex - 1, + } + resp, _, err := operator.SchedulerCASConfiguration(newConf, nil) + require.Nil(err) + require.False(resp) + } + + // Pass a valid ModifyIndex + { + newConf := &SchedulerConfiguration{ + EnablePreemption: true, + ModifyIndex: config.ModifyIndex, + } + resp, _, err := operator.SchedulerCASConfiguration(newConf, nil) + require.Nil(err) + require.True(resp) + } +} diff --git a/command/agent/http.go b/command/agent/http.go index 857f5f8bcde..d1bb0e74a45 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -194,6 +194,8 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/system/gc", s.wrap(s.GarbageCollectRequest)) s.mux.HandleFunc("/v1/system/reconcile/summaries", s.wrap(s.ReconcileJobSummaries)) + s.mux.HandleFunc("/v1/operator/scheduler/config", s.wrap(s.OperatorSchedulerConfiguration)) + if uiEnabled { s.mux.Handle("/ui/", http.StripPrefix("/ui/", handleUI(http.FileServer(&UIAssetWrapper{FileSystem: assetFS()})))) } else { diff --git a/command/agent/operator_endpoint.go b/command/agent/operator_endpoint.go index 02b25c2187e..9e9ebe9b34f 100644 --- a/command/agent/operator_endpoint.go +++ b/command/agent/operator_endpoint.go @@ -208,3 +208,67 @@ func (s *HTTPServer) OperatorServerHealth(resp http.ResponseWriter, req *http.Re return out, nil } + +// OperatorSchedulerConfiguration is used to inspect the current Scheduler configuration. +// This supports the stale query mode in case the cluster doesn't have a leader. +func (s *HTTPServer) OperatorSchedulerConfiguration(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + // Switch on the method + switch req.Method { + case "GET": + var args structs.GenericRequest + if done := s.parse(resp, req, &args.Region, &args.QueryOptions); done { + return nil, nil + } + + var reply structs.SchedulerConfiguration + if err := s.agent.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { + return nil, err + } + + out := api.SchedulerConfiguration{ + EnablePreemption: reply.EnablePreemption, + CreateIndex: reply.CreateIndex, + ModifyIndex: reply.ModifyIndex, + } + + return out, nil + + case "PUT": + var args structs.SchedulerSetConfigRequest + s.parseWriteRequest(req, &args.WriteRequest) + + var conf api.SchedulerConfiguration + if err := decodeBody(req, &conf); err != nil { + return nil, CodedError(http.StatusBadRequest, fmt.Sprintf("Error parsing autopilot config: %v", err)) + } + + args.Config = structs.SchedulerConfiguration{ + EnablePreemption: conf.EnablePreemption, + } + + // Check for cas value + params := req.URL.Query() + if _, ok := params["cas"]; ok { + casVal, err := strconv.ParseUint(params.Get("cas"), 10, 64) + if err != nil { + return nil, CodedError(http.StatusBadRequest, fmt.Sprintf("Error parsing cas value: %v", err)) + } + args.Config.ModifyIndex = casVal + args.CAS = true + } + + var reply bool + if err := s.agent.RPC("Operator.SchedulerSetConfiguration", &args, &reply); err != nil { + return nil, err + } + + // Only use the out value if this was a CAS + if !args.CAS { + return true, nil + } + return reply, nil + + default: + return nil, CodedError(404, ErrInvalidMethod) + } +} diff --git a/command/agent/operator_endpoint_test.go b/command/agent/operator_endpoint_test.go index 2d848676511..2bf82d79009 100644 --- a/command/agent/operator_endpoint_test.go +++ b/command/agent/operator_endpoint_test.go @@ -13,6 +13,7 @@ import ( "github.com/hashicorp/nomad/api" "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestHTTP_OperatorRaftConfiguration(t *testing.T) { @@ -257,3 +258,94 @@ func TestOperator_ServerHealth_Unhealthy(t *testing.T) { }) }) } + +func TestOperator_SchedulerGetConfiguration(t *testing.T) { + t.Parallel() + httpTest(t, nil, func(s *TestAgent) { + require := require.New(t) + body := bytes.NewBuffer(nil) + req, _ := http.NewRequest("GET", "/v1/operator/scheduler/config", body) + resp := httptest.NewRecorder() + obj, err := s.Server.OperatorSchedulerConfiguration(resp, req) + require.Nil(err) + require.Equal(200, resp.Code) + out, ok := obj.(api.SchedulerConfiguration) + require.True(ok) + require.False(out.EnablePreemption) + }) +} + +func TestOperator_SchedulerSetConfiguration(t *testing.T) { + t.Parallel() + httpTest(t, nil, func(s *TestAgent) { + require := require.New(t) + body := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + req, _ := http.NewRequest("PUT", "/v1/operator/scheduler/config", body) + resp := httptest.NewRecorder() + _, err := s.Server.OperatorSchedulerConfiguration(resp, req) + require.Nil(err) + require.Equal(200, resp.Code) + + args := structs.GenericRequest{ + QueryOptions: structs.QueryOptions{ + Region: s.Config.Region, + }, + } + + var reply structs.SchedulerConfiguration + err = s.RPC("Operator.SchedulerGetConfiguration", &args, &reply) + require.Nil(err) + require.True(reply.EnablePreemption) + }) +} + +func TestOperator_SchedulerCASConfiguration(t *testing.T) { + t.Parallel() + httpTest(t, nil, func(s *TestAgent) { + require := require.New(t) + body := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + req, _ := http.NewRequest("PUT", "/v1/operator/scheduler/config", body) + resp := httptest.NewRecorder() + _, err := s.Server.OperatorSchedulerConfiguration(resp, req) + require.Nil(err) + require.Equal(200, resp.Code) + + args := structs.GenericRequest{ + QueryOptions: structs.QueryOptions{ + Region: s.Config.Region, + }, + } + + var reply structs.SchedulerConfiguration + if err := s.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { + t.Fatalf("err: %v", err) + } + require.True(reply.EnablePreemption) + + // Create a CAS request, bad index + { + buf := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + req, _ := http.NewRequest("PUT", fmt.Sprintf("/v1/operator/scheduler/config?cas=%d", reply.ModifyIndex-1), buf) + resp := httptest.NewRecorder() + obj, err := s.Server.OperatorSchedulerConfiguration(resp, req) + require.Nil(err) + require.False(obj.(bool)) + } + + // Create a CAS request, good index + { + buf := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + req, _ := http.NewRequest("PUT", fmt.Sprintf("/v1/operator/scheduler/config?cas=%d", reply.ModifyIndex), buf) + resp := httptest.NewRecorder() + obj, err := s.Server.OperatorSchedulerConfiguration(resp, req) + require.Nil(err) + require.True(obj.(bool)) + } + + // Verify the update + if err := s.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { + t.Fatalf("err: %v", err) + } + require.True(reply.EnablePreemption) + }) +} diff --git a/nomad/fsm.go b/nomad/fsm.go index ba02bf06396..bba9d0e723d 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -247,6 +247,8 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { return n.applyNodeEligibilityUpdate(buf[1:], log.Index) case structs.BatchNodeUpdateDrainRequestType: return n.applyBatchDrainUpdate(buf[1:], log.Index) + case structs.SchedulerConfigRequestType: + return n.applySchedulerConfigUpdate(buf[1:], log.Index) } // Check enterprise only message types. @@ -1833,6 +1835,23 @@ func (s *nomadSnapshot) persistACLTokens(sink raft.SnapshotSink, return nil } +func (n *nomadFSM) applySchedulerConfigUpdate(buf []byte, index uint64) interface{} { + var req structs.SchedulerSetConfigRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + defer metrics.MeasureSince([]string{"nomad", "fsm", "scheduler-config"}, time.Now()) + + if req.CAS { + act, err := n.state.SchedulerCASConfig(index, req.Config.ModifyIndex, &req.Config) + if err != nil { + return err + } + return act + } + return n.state.SchedulerSetConfig(index, &req.Config) +} + // Release is a no-op, as we just need to GC the pointer // to the state store snapshot. There is nothing to explicitly // cleanup. diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index c8802bf9bd3..48f50271a54 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -2825,3 +2825,47 @@ func TestFSM_Autopilot(t *testing.T) { t.Fatalf("bad: %v", config.CleanupDeadServers) } } + +func TestFSM_SchedulerConfig(t *testing.T) { + t.Parallel() + fsm := testFSM(t) + + require := require.New(t) + + // Set the autopilot config using a request. + req := structs.SchedulerSetConfigRequest{ + Config: structs.SchedulerConfiguration{ + EnablePreemption: true, + }, + } + buf, err := structs.Encode(structs.SchedulerConfigRequestType, req) + require.Nil(err) + + resp := fsm.Apply(makeLog(buf)) + if _, ok := resp.(error); ok { + t.Fatalf("bad: %v", resp) + } + + // Verify key is set directly in the state store. + _, config, err := fsm.state.SchedulerConfig() + require.Nil(err) + + require.Equal(config.EnablePreemption, req.Config.EnablePreemption) + + // Now use CAS and provide an old index + req.CAS = true + req.Config.EnablePreemption = false + req.Config.ModifyIndex = config.ModifyIndex - 1 + buf, err = structs.Encode(structs.SchedulerConfigRequestType, req) + require.Nil(err) + + resp = fsm.Apply(makeLog(buf)) + if _, ok := resp.(error); ok { + t.Fatalf("bad: %v", resp) + } + + _, config, err = fsm.state.SchedulerConfig() + require.Nil(err) + // Verify that preemption is still enabled + require.True(config.EnablePreemption) +} diff --git a/nomad/leader.go b/nomad/leader.go index ef0e622cf9b..f7e8cd98929 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -187,6 +187,9 @@ func (s *Server) establishLeadership(stopCh chan struct{}) error { s.getOrCreateAutopilotConfig() s.autopilot.Start() + // Initialize scheduler configuration + s.getOrCreateSchedulerConfig() + // Enable the plan queue, since we are now the leader s.planQueue.SetEnabled(true) @@ -1230,3 +1233,25 @@ func (s *Server) getOrCreateAutopilotConfig() *structs.AutopilotConfig { return config } + +// getOrCreateSchedulerConfig is used to get the scheduler config, initializing it if necessary +func (s *Server) getOrCreateSchedulerConfig() *structs.SchedulerConfiguration { + state := s.fsm.State() + _, config, err := state.SchedulerConfig() + if err != nil { + s.logger.Named("core").Error("failed to get scheduler config", "error", err) + return nil + } + if config != nil { + return config + } + + config = &structs.SchedulerConfiguration{EnablePreemption: false} + req := structs.SchedulerSetConfigRequest{Config: *config} + if _, _, err = s.raftApply(structs.SchedulerConfigRequestType, req); err != nil { + s.logger.Named("core").Error("failed to initialize config", "error", err) + return nil + } + + return config +} diff --git a/nomad/operator_endpoint.go b/nomad/operator_endpoint.go index 39615b3fa89..a04f1284337 100644 --- a/nomad/operator_endpoint.go +++ b/nomad/operator_endpoint.go @@ -284,3 +284,64 @@ func (op *Operator) ServerHealth(args *structs.GenericRequest, reply *autopilot. return nil } + +// SchedulerSetConfiguration is used to set the current Scheduler configuration. +func (op *Operator) SchedulerSetConfiguration(args *structs.SchedulerSetConfigRequest, reply *bool) error { + if done, err := op.srv.forward("Operator.SchedulerSetConfiguration", args, args, reply); done { + return err + } + + // This action requires operator write access. + rule, err := op.srv.ResolveToken(args.AuthToken) + if err != nil { + return err + } + if rule != nil && !rule.AllowOperatorWrite() { + return structs.ErrPermissionDenied + } + + // Apply the update + resp, _, err := op.srv.raftApply(structs.SchedulerConfigRequestType, args) + if err != nil { + op.logger.Error("failed applying Scheduler configuration", "error", err) + return err + } + if respErr, ok := resp.(error); ok { + return respErr + } + + // Check if the return type is a bool. + if respBool, ok := resp.(bool); ok { + *reply = respBool + } + return nil +} + +// SchedulerGetConfiguration is used to retrieve the current Scheduler configuration. +func (op *Operator) SchedulerGetConfiguration(args *structs.GenericRequest, reply *structs.SchedulerConfiguration) error { + if done, err := op.srv.forward("Operator.SchedulerGetConfiguration", args, args, reply); done { + return err + } + + // This action requires operator read access. + rule, err := op.srv.ResolveToken(args.AuthToken) + if err != nil { + return err + } + if rule != nil && !rule.AllowOperatorRead() { + return structs.ErrPermissionDenied + } + + state := op.srv.fsm.State() + _, config, err := state.SchedulerConfig() + if err != nil { + return err + } + if config == nil { + return fmt.Errorf("scheduler config not initialized yet") + } + + *reply = *config + + return nil +} diff --git a/nomad/state/schema.go b/nomad/state/schema.go index 8b77de60357..7c76c035920 100644 --- a/nomad/state/schema.go +++ b/nomad/state/schema.go @@ -44,6 +44,7 @@ func init() { aclPolicyTableSchema, aclTokenTableSchema, autopilotConfigTableSchema, + schedulerConfigTableSchema, }...) } @@ -599,3 +600,21 @@ func aclTokenTableSchema() *memdb.TableSchema { }, } } + +// schedulerConfigTableSchema returns the MemDB schema for the scheduler config table. +// This table is used to store configuration options for the scheduler +func schedulerConfigTableSchema() *memdb.TableSchema { + return &memdb.TableSchema{ + Name: "scheduler_config", + Indexes: map[string]*memdb.IndexSchema{ + "id": { + Name: "id", + AllowMissing: true, + Unique: true, + Indexer: &memdb.ConditionalIndex{ + Conditional: func(obj interface{}) (bool, error) { return true, nil }, + }, + }, + }, + } +} diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 71db85d5745..b228cfe10d9 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -3991,3 +3991,81 @@ func (r *StateRestore) addEphemeralDiskToTaskGroups(job *structs.Job) { } } } + +// SchedulerConfig is used to get the current Scheduler configuration. +func (s *StateStore) SchedulerConfig() (uint64, *structs.SchedulerConfiguration, error) { + tx := s.db.Txn(false) + defer tx.Abort() + + // Get the scheduler config + c, err := tx.First("scheduler_config", "id") + if err != nil { + return 0, nil, fmt.Errorf("failed scheduler config lookup: %s", err) + } + + config, ok := c.(*structs.SchedulerConfiguration) + if !ok { + return 0, nil, nil + } + + return config.ModifyIndex, config, nil +} + +// SchedulerSetConfig is used to set the current Scheduler configuration. +func (s *StateStore) SchedulerSetConfig(idx uint64, config *structs.SchedulerConfiguration) error { + tx := s.db.Txn(true) + defer tx.Abort() + + s.schedulerSetConfigTxn(idx, tx, config) + + tx.Commit() + return nil +} + +// SchedulerCASConfig is used to try updating the scheduler configuration with a +// given Raft index. If the CAS index specified is not equal to the last observed index +// for the config, then the call is a noop, +func (s *StateStore) SchedulerCASConfig(idx, cidx uint64, config *structs.SchedulerConfiguration) (bool, error) { + tx := s.db.Txn(true) + defer tx.Abort() + + // Check for an existing config + existing, err := tx.First("scheduler_config", "id") + if err != nil { + return false, fmt.Errorf("failed scheduler config lookup: %s", err) + } + + // If the existing index does not match the provided CAS + // index arg, then we shouldn't update anything and can safely + // return early here. + e, ok := existing.(*structs.SchedulerConfiguration) + if !ok || e.ModifyIndex != cidx { + return false, nil + } + + s.schedulerSetConfigTxn(idx, tx, config) + + tx.Commit() + return true, nil +} + +func (s *StateStore) schedulerSetConfigTxn(idx uint64, tx *memdb.Txn, config *structs.SchedulerConfiguration) error { + // Check for an existing config + existing, err := tx.First("scheduler_config", "id") + if err != nil { + return fmt.Errorf("failed scheduler config lookup: %s", err) + } + + // Set the indexes. + if existing != nil { + config.CreateIndex = existing.(*structs.SchedulerConfiguration).CreateIndex + } else { + config.CreateIndex = idx + } + config.ModifyIndex = idx + + if err := tx.Insert("scheduler_config", config); err != nil { + return fmt.Errorf("failed updating scheduler config: %s", err) + } + return nil +} diff --git a/nomad/structs/operator.go b/nomad/structs/operator.go index ecd7f97d411..23d797916d5 100644 --- a/nomad/structs/operator.go +++ b/nomad/structs/operator.go @@ -119,3 +119,26 @@ type AutopilotConfig struct { CreateIndex uint64 ModifyIndex uint64 } + +type SchedulerConfiguration struct { + // EnablePreemption specifies whether to enable eviction of lower + // priority jobs to place higher priority jobs. + EnablePreemption bool + + // CreateIndex/ModifyIndex store the create/modify indexes of this configuration. + CreateIndex uint64 + ModifyIndex uint64 +} + +// SchedulerSetConfigRequest is used by the Operator endpoint to update the +// current Scheduler configuration of the cluster. +type SchedulerSetConfigRequest struct { + // Config is the new Scheduler configuration to use. + Config SchedulerConfiguration + + // CAS controls whether to use check-and-set semantics for this request. + CAS bool + + // WriteRequest holds the ACL token to go along with this request. + WriteRequest +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 868514cf95e..b1c241dee4b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -81,6 +81,7 @@ const ( AllocUpdateDesiredTransitionRequestType NodeUpdateEligibilityRequestType BatchNodeUpdateDrainRequestType + SchedulerConfigRequestType ) const ( From 2143fa2ab7b7ec6d56aa8eaf1b0a90b96bea7f0a Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 27 Sep 2018 23:44:01 -0500 Subject: [PATCH 11/38] Use scheduler config from state store to enable/disable preemption --- scheduler/scheduler.go | 3 +++ scheduler/stack.go | 14 +++++++++----- scheduler/system_sched_test.go | 6 ++++++ 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go index b9f0f5e3aa7..bd921a788d1 100644 --- a/scheduler/scheduler.go +++ b/scheduler/scheduler.go @@ -88,6 +88,9 @@ type State interface { // 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) } // Planner interface is used to submit a task allocation plan. diff --git a/scheduler/stack.go b/scheduler/stack.go index 6be27c5e203..d9ae982b495 100644 --- a/scheduler/stack.go +++ b/scheduler/stack.go @@ -109,10 +109,9 @@ func NewGenericStack(batch bool, ctx Context) *GenericStack { rankSource := NewFeasibleRankIterator(ctx, s.distinctPropertyConstraint) // Apply the bin packing, this depends on the resources needed - // by a particular task group. Only enable eviction for the service - // scheduler as that logic is expensive. - evict := !batch - s.binPack = NewBinPackIterator(ctx, rankSource, evict, 0) + // by a particular task group. + + s.binPack = NewBinPackIterator(ctx, rankSource, false, 0) // Apply the job anti-affinity iterator. This is to avoid placing // multiple allocations on the same node for this job. @@ -287,7 +286,12 @@ func NewSystemStack(ctx Context) *SystemStack { // Apply the bin packing, this depends on the resources needed // by a particular task group. Enable eviction as system jobs are high // priority. - s.binPack = NewBinPackIterator(ctx, rankSource, true, 0) + _, schedConfig, _ := s.ctx.State().SchedulerConfig() + enablePreemption := false + if schedConfig != nil { + enablePreemption = schedConfig.EnablePreemption + } + s.binPack = NewBinPackIterator(ctx, rankSource, enablePreemption, 0) // Apply score normalization s.scoreNorm = NewScoreNormalizationIterator(ctx, s.binPack) diff --git a/scheduler/system_sched_test.go b/scheduler/system_sched_test.go index d2cf1f5f8d2..a190d113a7d 100644 --- a/scheduler/system_sched_test.go +++ b/scheduler/system_sched_test.go @@ -242,6 +242,9 @@ func TestSystemSched_ExhaustResources(t *testing.T) { node := mock.Node() noErr(t, h.State.UpsertNode(h.NextIndex(), node)) + // Enable Preemption + h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{EnablePreemption: true}) + // Create a service job which consumes most of the system resources svcJob := mock.Job() svcJob.TaskGroups[0].Count = 1 @@ -1573,6 +1576,9 @@ func TestSystemSched_Preemption(t *testing.T) { nodes = append(nodes, node) } + // Enable Preemption + h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{EnablePreemption: true}) + // Create some low priority batch jobs and allocations for them // One job uses a reserved port job1 := mock.BatchJob() From 6966e3c3e8636455a5ef65c43afe63869ab13a41 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Mon, 1 Oct 2018 09:26:52 -0500 Subject: [PATCH 12/38] Make preemption config a struct to allow for enabling based on scheduler type --- api/operator.go | 9 +++++++-- api/operator_test.go | 12 ++++++------ command/agent/operator_endpoint.go | 6 +++--- command/agent/operator_endpoint_test.go | 24 ++++++++++++++++-------- nomad/fsm_test.go | 10 ++++++---- nomad/leader.go | 2 +- nomad/structs/operator.go | 10 ++++++++-- scheduler/stack.go | 4 ++-- scheduler/system_sched_test.go | 12 ++++++++++-- 9 files changed, 59 insertions(+), 30 deletions(-) diff --git a/api/operator.go b/api/operator.go index 85a28fe8d01..298c8bf21f9 100644 --- a/api/operator.go +++ b/api/operator.go @@ -110,15 +110,20 @@ func (op *Operator) RaftRemovePeerByID(id string, q *WriteOptions) error { } type SchedulerConfiguration struct { - // EnablePreemption specifies whether to enable eviction of lower + // PreemptionConfig specifies whether to enable eviction of lower // priority jobs to place higher priority jobs. - EnablePreemption bool + PreemptionConfig PreemptionConfig // CreateIndex/ModifyIndex store the create/modify indexes of this configuration. CreateIndex uint64 ModifyIndex uint64 } +// PreemptionConfig specifies whether preemption is enabled based on scheduler type +type PreemptionConfig struct { + SystemSchedulerEnabled bool +} + // SchedulerGetConfiguration is used to query the current Scheduler configuration. func (op *Operator) SchedulerGetConfiguration(q *QueryOptions) (*SchedulerConfiguration, *QueryMeta, error) { var resp SchedulerConfiguration diff --git a/api/operator_test.go b/api/operator_test.go index 277d85afe5f..9c95228af71 100644 --- a/api/operator_test.go +++ b/api/operator_test.go @@ -68,16 +68,16 @@ func TestAPI_OperatorSchedulerGetSetConfiguration(t *testing.T) { config, _, err = operator.SchedulerGetConfiguration(nil) r.Check(err) }) - require.False(config.EnablePreemption) + require.True(config.PreemptionConfig.SystemSchedulerEnabled) // Change a config setting - newConf := &SchedulerConfiguration{EnablePreemption: true} + newConf := &SchedulerConfiguration{PreemptionConfig: PreemptionConfig{SystemSchedulerEnabled: false}} _, err := operator.SchedulerSetConfiguration(newConf, nil) require.Nil(err) config, _, err = operator.SchedulerGetConfiguration(nil) require.Nil(err) - require.True(config.EnablePreemption) + require.True(config.PreemptionConfig.SystemSchedulerEnabled) } func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { @@ -93,12 +93,12 @@ func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { config, _, err = operator.SchedulerGetConfiguration(nil) r.Check(err) }) - require.False(config.EnablePreemption) + require.True(config.PreemptionConfig.SystemSchedulerEnabled) // Pass an invalid ModifyIndex { newConf := &SchedulerConfiguration{ - EnablePreemption: true, + PreemptionConfig: PreemptionConfig{SystemSchedulerEnabled: false}, ModifyIndex: config.ModifyIndex - 1, } resp, _, err := operator.SchedulerCASConfiguration(newConf, nil) @@ -109,7 +109,7 @@ func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { // Pass a valid ModifyIndex { newConf := &SchedulerConfiguration{ - EnablePreemption: true, + PreemptionConfig: PreemptionConfig{SystemSchedulerEnabled: false}, ModifyIndex: config.ModifyIndex, } resp, _, err := operator.SchedulerCASConfiguration(newConf, nil) diff --git a/command/agent/operator_endpoint.go b/command/agent/operator_endpoint.go index 9e9ebe9b34f..af62f06d206 100644 --- a/command/agent/operator_endpoint.go +++ b/command/agent/operator_endpoint.go @@ -226,7 +226,7 @@ func (s *HTTPServer) OperatorSchedulerConfiguration(resp http.ResponseWriter, re } out := api.SchedulerConfiguration{ - EnablePreemption: reply.EnablePreemption, + PreemptionConfig: api.PreemptionConfig{SystemSchedulerEnabled: reply.PreemptionConfig.SystemSchedulerEnabled}, CreateIndex: reply.CreateIndex, ModifyIndex: reply.ModifyIndex, } @@ -239,11 +239,11 @@ func (s *HTTPServer) OperatorSchedulerConfiguration(resp http.ResponseWriter, re var conf api.SchedulerConfiguration if err := decodeBody(req, &conf); err != nil { - return nil, CodedError(http.StatusBadRequest, fmt.Sprintf("Error parsing autopilot config: %v", err)) + return nil, CodedError(http.StatusBadRequest, fmt.Sprintf("Error parsing scheduler config: %v", err)) } args.Config = structs.SchedulerConfiguration{ - EnablePreemption: conf.EnablePreemption, + PreemptionConfig: structs.PreemptionConfig{SystemSchedulerEnabled: conf.PreemptionConfig.SystemSchedulerEnabled}, } // Check for cas value diff --git a/command/agent/operator_endpoint_test.go b/command/agent/operator_endpoint_test.go index 2bf82d79009..234aec05891 100644 --- a/command/agent/operator_endpoint_test.go +++ b/command/agent/operator_endpoint_test.go @@ -271,7 +271,7 @@ func TestOperator_SchedulerGetConfiguration(t *testing.T) { require.Equal(200, resp.Code) out, ok := obj.(api.SchedulerConfiguration) require.True(ok) - require.False(out.EnablePreemption) + require.True(out.PreemptionConfig.SystemSchedulerEnabled) }) } @@ -279,7 +279,9 @@ func TestOperator_SchedulerSetConfiguration(t *testing.T) { t.Parallel() httpTest(t, nil, func(s *TestAgent) { require := require.New(t) - body := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + body := bytes.NewBuffer([]byte(`{"PreemptionConfig": { + "SystemSchedulerEnabled": true + }}`)) req, _ := http.NewRequest("PUT", "/v1/operator/scheduler/config", body) resp := httptest.NewRecorder() _, err := s.Server.OperatorSchedulerConfiguration(resp, req) @@ -295,7 +297,7 @@ func TestOperator_SchedulerSetConfiguration(t *testing.T) { var reply structs.SchedulerConfiguration err = s.RPC("Operator.SchedulerGetConfiguration", &args, &reply) require.Nil(err) - require.True(reply.EnablePreemption) + require.True(reply.PreemptionConfig.SystemSchedulerEnabled) }) } @@ -303,7 +305,9 @@ func TestOperator_SchedulerCASConfiguration(t *testing.T) { t.Parallel() httpTest(t, nil, func(s *TestAgent) { require := require.New(t) - body := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + body := bytes.NewBuffer([]byte(`{"PreemptionConfig": { + "SystemSchedulerEnabled": true + }}`)) req, _ := http.NewRequest("PUT", "/v1/operator/scheduler/config", body) resp := httptest.NewRecorder() _, err := s.Server.OperatorSchedulerConfiguration(resp, req) @@ -320,11 +324,13 @@ func TestOperator_SchedulerCASConfiguration(t *testing.T) { if err := s.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { t.Fatalf("err: %v", err) } - require.True(reply.EnablePreemption) + require.True(reply.PreemptionConfig.SystemSchedulerEnabled) // Create a CAS request, bad index { - buf := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + buf := bytes.NewBuffer([]byte(`{"PreemptionConfig": { + "SystemSchedulerEnabled": false + }}`)) req, _ := http.NewRequest("PUT", fmt.Sprintf("/v1/operator/scheduler/config?cas=%d", reply.ModifyIndex-1), buf) resp := httptest.NewRecorder() obj, err := s.Server.OperatorSchedulerConfiguration(resp, req) @@ -334,7 +340,9 @@ func TestOperator_SchedulerCASConfiguration(t *testing.T) { // Create a CAS request, good index { - buf := bytes.NewBuffer([]byte(`{"EnablePreemption": true}`)) + buf := bytes.NewBuffer([]byte(`{"PreemptionConfig": { + "SystemSchedulerEnabled": false + }}`)) req, _ := http.NewRequest("PUT", fmt.Sprintf("/v1/operator/scheduler/config?cas=%d", reply.ModifyIndex), buf) resp := httptest.NewRecorder() obj, err := s.Server.OperatorSchedulerConfiguration(resp, req) @@ -346,6 +354,6 @@ func TestOperator_SchedulerCASConfiguration(t *testing.T) { if err := s.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { t.Fatalf("err: %v", err) } - require.True(reply.EnablePreemption) + require.False(reply.PreemptionConfig.SystemSchedulerEnabled) }) } diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 48f50271a54..99c65382c71 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -2835,7 +2835,9 @@ func TestFSM_SchedulerConfig(t *testing.T) { // Set the autopilot config using a request. req := structs.SchedulerSetConfigRequest{ Config: structs.SchedulerConfiguration{ - EnablePreemption: true, + PreemptionConfig: structs.PreemptionConfig{ + SystemSchedulerEnabled: true, + }, }, } buf, err := structs.Encode(structs.SchedulerConfigRequestType, req) @@ -2850,11 +2852,11 @@ func TestFSM_SchedulerConfig(t *testing.T) { _, config, err := fsm.state.SchedulerConfig() require.Nil(err) - require.Equal(config.EnablePreemption, req.Config.EnablePreemption) + require.Equal(config.PreemptionConfig.SystemSchedulerEnabled, req.Config.PreemptionConfig.SystemSchedulerEnabled) // Now use CAS and provide an old index req.CAS = true - req.Config.EnablePreemption = false + req.Config.PreemptionConfig = structs.PreemptionConfig{SystemSchedulerEnabled: false} req.Config.ModifyIndex = config.ModifyIndex - 1 buf, err = structs.Encode(structs.SchedulerConfigRequestType, req) require.Nil(err) @@ -2867,5 +2869,5 @@ func TestFSM_SchedulerConfig(t *testing.T) { _, config, err = fsm.state.SchedulerConfig() require.Nil(err) // Verify that preemption is still enabled - require.True(config.EnablePreemption) + require.True(config.PreemptionConfig.SystemSchedulerEnabled) } diff --git a/nomad/leader.go b/nomad/leader.go index f7e8cd98929..e43edc67003 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -1246,7 +1246,7 @@ func (s *Server) getOrCreateSchedulerConfig() *structs.SchedulerConfiguration { return config } - config = &structs.SchedulerConfiguration{EnablePreemption: false} + config = &structs.SchedulerConfiguration{PreemptionConfig: structs.PreemptionConfig{SystemSchedulerEnabled: true}} req := structs.SchedulerSetConfigRequest{Config: *config} if _, _, err = s.raftApply(structs.SchedulerConfigRequestType, req); err != nil { s.logger.Named("core").Error("failed to initialize config", "error", err) diff --git a/nomad/structs/operator.go b/nomad/structs/operator.go index 23d797916d5..f6143a053d4 100644 --- a/nomad/structs/operator.go +++ b/nomad/structs/operator.go @@ -121,15 +121,21 @@ type AutopilotConfig struct { } type SchedulerConfiguration struct { - // EnablePreemption specifies whether to enable eviction of lower + // PreemptionConfig specifies whether to enable eviction of lower // priority jobs to place higher priority jobs. - EnablePreemption bool + PreemptionConfig PreemptionConfig // CreateIndex/ModifyIndex store the create/modify indexes of this configuration. CreateIndex uint64 ModifyIndex uint64 } +// PreemptionConfig specifies whether preemption is enabled based on scheduler type +type PreemptionConfig struct { + // SystemSchedulerEnabled specifies if preemption is enabled for system jobs + SystemSchedulerEnabled bool +} + // SchedulerSetConfigRequest is used by the Operator endpoint to update the // current Scheduler configuration of the cluster. type SchedulerSetConfigRequest struct { diff --git a/scheduler/stack.go b/scheduler/stack.go index d9ae982b495..48f89f81fdb 100644 --- a/scheduler/stack.go +++ b/scheduler/stack.go @@ -287,9 +287,9 @@ func NewSystemStack(ctx Context) *SystemStack { // by a particular task group. Enable eviction as system jobs are high // priority. _, schedConfig, _ := s.ctx.State().SchedulerConfig() - enablePreemption := false + enablePreemption := true if schedConfig != nil { - enablePreemption = schedConfig.EnablePreemption + enablePreemption = schedConfig.PreemptionConfig.SystemSchedulerEnabled } s.binPack = NewBinPackIterator(ctx, rankSource, enablePreemption, 0) diff --git a/scheduler/system_sched_test.go b/scheduler/system_sched_test.go index a190d113a7d..10cb83ae3c6 100644 --- a/scheduler/system_sched_test.go +++ b/scheduler/system_sched_test.go @@ -243,7 +243,11 @@ func TestSystemSched_ExhaustResources(t *testing.T) { noErr(t, h.State.UpsertNode(h.NextIndex(), node)) // Enable Preemption - h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{EnablePreemption: true}) + h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{ + PreemptionConfig: structs.PreemptionConfig{ + SystemSchedulerEnabled: true, + }, + }) // Create a service job which consumes most of the system resources svcJob := mock.Job() @@ -1577,7 +1581,11 @@ func TestSystemSched_Preemption(t *testing.T) { } // Enable Preemption - h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{EnablePreemption: true}) + h.State.SchedulerSetConfig(h.NextIndex(), &structs.SchedulerConfiguration{ + PreemptionConfig: structs.PreemptionConfig{ + SystemSchedulerEnabled: true, + }, + }) // Create some low priority batch jobs and allocations for them // One job uses a reserved port From 24b393483aaf9b1d95ecfd0e6779d4b84e18f7ee Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 16 Oct 2018 18:23:21 -0500 Subject: [PATCH 13/38] Modify preemption code to use new style of resource structs --- nomad/structs/structs.go | 31 ++++ scheduler/preemption.go | 278 +++++++++++++++++++++-------------- scheduler/preemption_test.go | 159 +++++++++++--------- scheduler/rank.go | 2 +- 4 files changed, 291 insertions(+), 179 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index b1c241dee4b..0040a680201 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -7431,6 +7431,37 @@ func (a *Allocation) ComparableResources() *ComparableResources { } } +// COMPAT(0.11): Remove in 0.11 +// CompatibleNetworkResources returns network resources on the allocation +// by reading AllocatedResources which are populated starting in 0.9 and +// falling back to pre 0.9 fields (Resources/TaskResources) if set +func (a *Allocation) CompatibleNetworkResources() []*NetworkResource { + var ret []*NetworkResource + // Alloc already has 0.9+ behavior + if a.AllocatedResources != nil { + var comparableResources *ComparableResources + for _, taskResource := range a.AllocatedResources.Tasks { + if comparableResources == nil { + comparableResources = taskResource.Comparable() + } else { + comparableResources.Add(taskResource.Comparable()) + } + } + ret = comparableResources.Flattened.Networks + } else if a.Resources != nil { + // Alloc has pre 0.9 total resources + ret = a.Resources.Networks + } else if a.TaskResources != nil { + // Alloc has pre 0.9 task resources + resources := new(Resources) + for _, taskResource := range a.TaskResources { + resources.Add(taskResource) + } + ret = resources.Networks + } + return ret +} + // LookupTask by name from the Allocation. Returns nil if the Job is not set, the // TaskGroup does not exist, or the task name cannot be found. func (a *Allocation) LookupTask(name string) *Task { diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 1eb2d2f3218..053b81ef0d9 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -1,7 +1,6 @@ package scheduler import ( - "fmt" "math" "sort" @@ -13,93 +12,84 @@ import ( // number of allocations being preempted exceeds max_parallel value in the job's migrate stanza const maxParallelPenalty = 50.0 -type PreemptionType uint8 - -const ( - NetworkResource PreemptionType = iota - CPUMemoryDiskIOPS -) - -// resourceDistance returns how close the resource is to the resource being asked for -// It is calculated by first computing a relative fraction and then measuring how close -// that is to the origin coordinate. Lower values are better -func resourceDistance(resource *structs.Resources, resourceAsk *structs.Resources) float64 { - memoryCoord, cpuCoord, iopsCoord, diskMBCoord := 0.0, 0.0, 0.0, 0.0 - if resourceAsk.MemoryMB > 0 { - memoryCoord = float64(resourceAsk.MemoryMB-resource.MemoryMB) / float64(resourceAsk.MemoryMB) +func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources) float64 { + memoryCoord, cpuCoord, diskMBCoord := 0.0, 0.0, 0.0 + if resourceAsk.Flattened.Memory.MemoryMB > 0 { + memoryCoord = (float64(resourceAsk.Flattened.Memory.MemoryMB) - float64(resourceUsed.Flattened.Memory.MemoryMB)) / float64(resourceAsk.Flattened.Memory.MemoryMB) } - if resourceAsk.CPU > 0 { - cpuCoord = float64(resourceAsk.CPU-resource.CPU) / float64(resourceAsk.CPU) + if resourceAsk.Flattened.Cpu.CpuShares > 0 { + cpuCoord = (float64(resourceAsk.Flattened.Cpu.CpuShares) - float64(resourceUsed.Flattened.Cpu.CpuShares)) / float64(resourceAsk.Flattened.Cpu.CpuShares) } - if resourceAsk.IOPS > 0 { - iopsCoord = float64(resourceAsk.IOPS-resource.IOPS) / float64(resourceAsk.IOPS) + if resourceAsk.Shared.DiskMB > 0 { + diskMBCoord = (float64(resourceAsk.Shared.DiskMB) - float64(resourceUsed.Shared.DiskMB)) / float64(resourceAsk.Shared.DiskMB) } - if resourceAsk.DiskMB > 0 { - diskMBCoord = float64(resourceAsk.DiskMB-resource.DiskMB) / float64(resourceAsk.DiskMB) - } - originDist := math.Sqrt( math.Pow(memoryCoord, 2) + math.Pow(cpuCoord, 2) + - math.Pow(iopsCoord, 2) + math.Pow(diskMBCoord, 2)) return originDist } -// networkResourceDistance returns distance based on network megabits -func networkResourceDistance(resource *structs.Resources, resourceAsk *structs.Resources) float64 { - networkCoord := math.MaxFloat64 - if len(resourceAsk.Networks) > 0 && resourceAsk.Networks[0].MBits > 0 { - networkCoord = float64(resourceAsk.Networks[0].MBits-resource.Networks[0].MBits) / float64(resourceAsk.Networks[0].MBits) +// getPreemptionScoreForTaskGroupResources is used to calculate a score (lower is better) based on the distance between +// the needed resource and requirements. A penalty is added when the choice already has some existing +// allocations in the plan that are being preempted. +func getPreemptionScoreForTaskGroupResources(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources, maxParallel int, numPreemptedAllocs int) float64 { + maxParallelScorePenalty := 0.0 + if maxParallel > 0 && numPreemptedAllocs >= maxParallel { + maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty } - - originDist := math.Sqrt( - math.Pow(networkCoord, 2)) - return originDist + return basicResourceDistance(resourceAsk, resourceUsed) + maxParallelScorePenalty } -// getPreemptionScore is used to calculate a score (lower is better) based on the distance between -// the needed resource and requirements. A penalty is added when the choice already has some existing -// allocations in the plan that are being preempted. -func getPreemptionScore(resource *structs.Resources, resourceAsk *structs.Resources, preemptionType PreemptionType, maxParallel int, numPreemptedAllocs int) float64 { +// getPreemptionScoreForNetwork is similar to getPreemptionScoreForTaskGroupResources but only uses network mbits to calculate a preemption score +func getPreemptionScoreForNetwork(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource, maxParallel int, numPreemptedAllocs int) float64 { + if resourceUsed == nil || resourceNeeded == nil { + return math.MaxFloat64 + } maxParallelScorePenalty := 0.0 if maxParallel > 0 && numPreemptedAllocs >= maxParallel { maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty } - switch preemptionType { - case NetworkResource: - return networkResourceDistance(resource, resourceAsk) + maxParallelScorePenalty - case CPUMemoryDiskIOPS: - return resourceDistance(resource, resourceAsk) + maxParallelScorePenalty + return networkResourceDistance(resourceUsed, resourceNeeded) + maxParallelScorePenalty +} + +// networkResourceDistance returns distance based on network megabits +func networkResourceDistance(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource) float64 { + networkCoord := math.MaxFloat64 + if resourceUsed != nil && resourceNeeded != nil { + networkCoord = float64(resourceNeeded.MBits-resourceUsed.MBits) / float64(resourceNeeded.MBits) } - panic(fmt.Errorf("Unknown preemption type:%v", preemptionType)) + + originDist := math.Sqrt( + math.Pow(networkCoord, 2)) + return originDist } // findPreemptibleAllocationsForTaskGroup computes a list of allocations to preempt to accommodate // the resources asked for. Only allocs with a job priority < 10 of jobPriority are considered // This method is used after network resource needs have already been met. -func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs.Allocation, resourceAsk *structs.Resources, node *structs.Node, currentPreemptions []*structs.Allocation) []*structs.Allocation { - resourcesNeeded := resourceAsk.Copy() +func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs.Allocation, resourceAsk *structs.AllocatedResources, node *structs.Node, currentPreemptions []*structs.Allocation) []*structs.Allocation { + resourcesNeeded := resourceAsk.Comparable() allocsByPriority := filterAndGroupPreemptibleAllocs(jobPriority, current) var bestAllocs []*structs.Allocation allRequirementsMet := false - var preemptedResources *structs.Resources + var preemptedResources *structs.ComparableResources //TODO(preetha): should add some debug logging - nodeRemainingResources := node.Resources.Copy() + nodeRemainingResources := node.ComparableResources() // Initialize nodeRemainingResources with the remaining resources // after accounting for reserved resources and all allocations // Subtract the reserved resources of the node - if node.Reserved != nil { - nodeRemainingResources.Subtract(node.Reserved) + if node.ComparableReservedResources() != nil { + nodeRemainingResources.Subtract(node.ComparableReservedResources()) } // Subtract current allocations for _, alloc := range current { - nodeRemainingResources.Subtract(alloc.Resources) + nodeRemainingResources.Subtract(alloc.ComparableResources()) } // Iterate over allocations grouped by priority to find preemptible allocations @@ -115,7 +105,7 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. if tg != nil && tg.Migrate != nil { maxParallel = tg.Migrate.MaxParallel } - distance := getPreemptionScore(alloc.Resources, resourcesNeeded, CPUMemoryDiskIOPS, maxParallel, currentPreemptionCount) + distance := getPreemptionScoreForTaskGroupResources(resourcesNeeded, alloc.ComparableResources(), maxParallel, currentPreemptionCount) if distance < bestDistance { bestDistance = distance closestAllocIndex = index @@ -124,19 +114,22 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. closestAlloc := allocGrp.allocs[closestAllocIndex] if preemptedResources == nil { - preemptedResources = closestAlloc.Resources.Copy() + preemptedResources = closestAlloc.ComparableResources() } else { - preemptedResources.Add(closestAlloc.Resources) + preemptedResources.Add(closestAlloc.ComparableResources()) } availableResources := preemptedResources.Copy() availableResources.Add(nodeRemainingResources) - allRequirementsMet = meetsNonNetworkRequirements(availableResources, resourceAsk) + // This step needs the original resources asked for as the second arg, can't use the running total + allRequirementsMet = meetsNonNetworkRequirements(availableResources, resourceAsk.Comparable()) bestAllocs = append(bestAllocs, closestAlloc) allocGrp.allocs[closestAllocIndex] = allocGrp.allocs[len(allocGrp.allocs)-1] allocGrp.allocs = allocGrp.allocs[:len(allocGrp.allocs)-1] - resourcesNeeded.Subtract(closestAlloc.Resources) + + // this is the remaining total of resources needed + resourcesNeeded.Subtract(closestAlloc.ComparableResources()) } if allRequirementsMet { break @@ -148,9 +141,18 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. return nil } + resourcesNeeded = resourceAsk.Comparable() // We do another pass to eliminate unnecessary preemptions // This filters out allocs whose resources are already covered by another alloc - filteredBestAllocs := eliminateSuperSetAllocations(bestAllocs, resourceAsk, nodeRemainingResources, resourceDistance, meetsNonNetworkRequirements) + + // Sort bestAllocs by distance descending (without penalty) + sort.Slice(bestAllocs, func(i, j int) bool { + distance1 := basicResourceDistance(resourcesNeeded, bestAllocs[i].ComparableResources()) + distance2 := basicResourceDistance(resourcesNeeded, bestAllocs[j].ComparableResources()) + return distance1 > distance2 + }) + + filteredBestAllocs := eliminateSuperSetAllocationsForTaskGroup(bestAllocs, nodeRemainingResources, resourcesNeeded) return filteredBestAllocs } @@ -169,28 +171,25 @@ func computeCurrentPreemptions(currentAlloc *structs.Allocation, currentPreempti // meetsNonNetworkRequirements checks if the first resource meets or exceeds the second resource's requirements // This intentionally ignores network requirements, those are handled by meetsNetworkRequirements -func meetsNonNetworkRequirements(first *structs.Resources, second *structs.Resources) bool { - if first.CPU < second.CPU { - return false - } - if first.MemoryMB < second.MemoryMB { +func meetsNonNetworkRequirements(first *structs.ComparableResources, second *structs.ComparableResources) bool { + if first.Flattened.Cpu.CpuShares < second.Flattened.Cpu.CpuShares { return false } - if first.DiskMB < second.DiskMB { + if first.Flattened.Memory.MemoryMB < second.Flattened.Memory.MemoryMB { return false } - if first.IOPS < second.IOPS { + if first.Shared.DiskMB < second.Shared.DiskMB { return false } return true } // meetsNetworkRequirements checks if the first resource meets or exceeds the second resource's network MBits requirements -func meetsNetworkRequirements(first *structs.Resources, second *structs.Resources) bool { - if len(first.Networks) == 0 || len(second.Networks) == 0 { +func meetsNetworkRequirements(firstMbits int, secondMbits int) bool { + if firstMbits == 0 || secondMbits == 0 { return false } - return first.Networks[0].MBits >= second.Networks[0].MBits + return firstMbits >= secondMbits } type groupedAllocs struct { @@ -235,35 +234,52 @@ func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocat return groupedSortedAllocs } -type distanceFn func(first *structs.Resources, second *structs.Resources) float64 +func eliminateSuperSetAllocationsForTaskGroup(bestAllocs []*structs.Allocation, + nodeRemainingResources *structs.ComparableResources, + resourceAsk *structs.ComparableResources) []*structs.Allocation { -type meetsRequirementsFn func(first *structs.Resources, second *structs.Resources) bool + var preemptedResources *structs.ComparableResources + var filteredBestAllocs []*structs.Allocation -func eliminateSuperSetAllocations(bestAllocs []*structs.Allocation, resourceAsk *structs.Resources, - nodeRemainingResources *structs.Resources, distanceFunc distanceFn, reqFunc meetsRequirementsFn) []*structs.Allocation { - // Sort by distance reversed to surface any superset allocs first - sort.Slice(bestAllocs, func(i, j int) bool { - distance1 := distanceFunc(bestAllocs[i].Resources, resourceAsk) - distance2 := distanceFunc(bestAllocs[j].Resources, resourceAsk) - return distance1 > distance2 - }) + // Do another pass to eliminate allocations that are a superset of other allocations + // in the preemption set + for _, alloc := range bestAllocs { + if preemptedResources == nil { + preemptedResources = alloc.ComparableResources().Copy() + } else { + preemptedResources.Add(alloc.ComparableResources().Copy()) + } + filteredBestAllocs = append(filteredBestAllocs, alloc) + availableResources := preemptedResources.Copy() + availableResources.Add(nodeRemainingResources) - var preemptedResources *structs.Resources + requirementsMet, _ := availableResources.Superset(resourceAsk) + if requirementsMet { + break + } + } + return filteredBestAllocs +} + +func eliminateSuperSetAllocationsForNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk *structs.NetworkResource, + nodeRemainingResources *structs.ComparableResources) []*structs.Allocation { + + var preemptedResources *structs.ComparableResources var filteredBestAllocs []*structs.Allocation // Do another pass to eliminate allocations that are a superset of other allocations // in the preemption set for _, alloc := range bestAllocs { if preemptedResources == nil { - preemptedResources = alloc.Resources + preemptedResources = alloc.ComparableResources().Copy() } else { - preemptedResources.Add(alloc.Resources) + preemptedResources.Add(alloc.ComparableResources().Copy()) } filteredBestAllocs = append(filteredBestAllocs, alloc) availableResources := preemptedResources.Copy() availableResources.Add(nodeRemainingResources) - requirementsMet := reqFunc(availableResources, resourceAsk) + requirementsMet := meetsNetworkRequirements(availableResources.Flattened.Networks[0].MBits, networkResourcesAsk.MBits) if requirementsMet { break } @@ -274,7 +290,7 @@ func eliminateSuperSetAllocations(bestAllocs []*structs.Allocation, resourceAsk // preemptForNetworkResourceAsk tries to find allocations to preempt to meet network resources. // this needs to consider network resources at the task level and for the same task it should // only preempt allocations that share the same network device -func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allocation, resourceAsk *structs.Resources, +func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allocation, networkResourceAsk *structs.NetworkResource, netIdx *structs.NetworkIndex, currentPreemptions []*structs.Allocation) []*structs.Allocation { // Early return if there are no current allocs @@ -282,7 +298,6 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo return nil } - networkResourceAsk := resourceAsk.Networks[0] deviceToAllocs := make(map[string][]*structs.Allocation) MbitsNeeded := networkResourceAsk.MBits reservedPortsNeeded := networkResourceAsk.ReservedPorts @@ -301,8 +316,9 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo if jobPriority-alloc.Job.Priority < 10 { continue } - if len(alloc.Resources.Networks) > 0 { - device := alloc.Resources.Networks[0].Device + networks := alloc.CompatibleNetworkResources() + if len(networks) > 0 { + device := networks[0].Device allocsForDevice := deviceToAllocs[device] allocsForDevice = append(allocsForDevice, alloc) deviceToAllocs[device] = allocsForDevice @@ -376,26 +392,7 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo // as well as penalty for preempting an allocation // whose task group already has existing preemptions sort.Slice(allocs, func(i, j int) bool { - firstAlloc := allocs[i] - currentPreemptionCount1 := computeCurrentPreemptions(firstAlloc, currentPreemptions) - - // Look up configured maxParallel value for these allocation's task groups - var maxParallel1, maxParallel2 int - tg1 := allocs[i].Job.LookupTaskGroup(allocs[i].TaskGroup) - if tg1 != nil && tg1.Migrate != nil { - maxParallel1 = tg1.Migrate.MaxParallel - } - distance1 := getPreemptionScore(allocs[i].Resources, resourceAsk, NetworkResource, maxParallel1, currentPreemptionCount1) - - secondAlloc := allocs[j] - currentPreemptionCount2 := computeCurrentPreemptions(secondAlloc, currentPreemptions) - tg2 := secondAlloc.Job.LookupTaskGroup(secondAlloc.TaskGroup) - if tg2 != nil && tg2.Migrate != nil { - maxParallel2 = tg2.Migrate.MaxParallel - } - distance2 := getPreemptionScore(secondAlloc.Resources, resourceAsk, NetworkResource, maxParallel2, currentPreemptionCount2) - - return distance1 < distance2 + return distanceComparatorForNetwork(allocs, currentPreemptions, networkResourceAsk, i, j) }) for _, alloc := range allocs { @@ -423,16 +420,77 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo // Build a resource object with just the network Mbits filled in // Its safe to use the first preempted allocation's network resource // here because all allocations preempted will be from the same device - nodeRemainingResources := &structs.Resources{ - Networks: []*structs.NetworkResource{ - { - Device: allocsToPreempt[0].Resources.Networks[0].Device, - MBits: freeBandwidth, + nodeRemainingResources := &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Networks: []*structs.NetworkResource{ + { + Device: allocsToPreempt[0].Resources.Networks[0].Device, + MBits: freeBandwidth, + }, }, }, } + // Sort by distance reversed to surface any superset allocs first + // This sort only looks at mbits because we should still not prefer + // allocs that have a maxParallel penalty + sort.Slice(allocsToPreempt, func(i, j int) bool { + firstAlloc := allocsToPreempt[i] + secondAlloc := allocsToPreempt[j] + + firstAllocNetworks := firstAlloc.ComparableResources().Flattened.Networks + var firstAllocNetResourceUsed *structs.NetworkResource + if len(firstAllocNetworks) > 0 { + firstAllocNetResourceUsed = firstAllocNetworks[0] + } + + secondAllocNetworks := secondAlloc.ComparableResources().Flattened.Networks + var secondAllocNetResourceUsed *structs.NetworkResource + if len(secondAllocNetworks) > 0 { + secondAllocNetResourceUsed = secondAllocNetworks[0] + } + + distance1 := networkResourceDistance(firstAllocNetResourceUsed, networkResourceAsk) + distance2 := networkResourceDistance(secondAllocNetResourceUsed, networkResourceAsk) + return distance1 > distance2 + }) + // Do a final pass to eliminate any superset allocations - filteredBestAllocs := eliminateSuperSetAllocations(allocsToPreempt, resourceAsk, nodeRemainingResources, networkResourceDistance, meetsNetworkRequirements) + filteredBestAllocs := eliminateSuperSetAllocationsForNetwork(allocsToPreempt, networkResourceAsk, nodeRemainingResources) return filteredBestAllocs } + +func distanceComparatorForNetwork(allocs []*structs.Allocation, currentPreemptions []*structs.Allocation, networkResourceAsk *structs.NetworkResource, i int, j int) bool { + firstAlloc := allocs[i] + currentPreemptionCount1 := computeCurrentPreemptions(firstAlloc, currentPreemptions) + // Look up configured maxParallel value for these allocation's task groups + var maxParallel1, maxParallel2 int + tg1 := allocs[i].Job.LookupTaskGroup(firstAlloc.TaskGroup) + if tg1 != nil && tg1.Migrate != nil { + maxParallel1 = tg1.Migrate.MaxParallel + } + // Dereference network usage on first alloc if its there + firstAllocNetworks := firstAlloc.CompatibleNetworkResources() + var firstAllocNetResourceUsed *structs.NetworkResource + if len(firstAllocNetworks) > 0 { + firstAllocNetResourceUsed = firstAllocNetworks[0] + } + + distance1 := getPreemptionScoreForNetwork(firstAllocNetResourceUsed, networkResourceAsk, maxParallel1, currentPreemptionCount1) + + secondAlloc := allocs[j] + currentPreemptionCount2 := computeCurrentPreemptions(secondAlloc, currentPreemptions) + tg2 := secondAlloc.Job.LookupTaskGroup(secondAlloc.TaskGroup) + if tg2 != nil && tg2.Migrate != nil { + maxParallel2 = tg2.Migrate.MaxParallel + } + // Dereference network usage on first alloc if its there + secondAllocNetworks := secondAlloc.CompatibleNetworkResources() + var secondAllocNetResourceUsed *structs.NetworkResource + if len(secondAllocNetworks) > 0 { + secondAllocNetResourceUsed = secondAllocNetworks[0] + } + + distance2 := getPreemptionScoreForNetwork(secondAllocNetResourceUsed, networkResourceAsk, maxParallel2, currentPreemptionCount2) + return distance1 < distance2 +} diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go index cacef52bd93..b5f3ce61585 100644 --- a/scheduler/preemption_test.go +++ b/scheduler/preemption_test.go @@ -12,97 +12,117 @@ import ( ) func TestResourceDistance(t *testing.T) { - resourceAsk := &structs.Resources{ - CPU: 2048, - MemoryMB: 512, - IOPS: 300, - DiskMB: 4096, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, + 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.Resources + allocResource *structs.ComparableResources expectedDistance string } testCases := []*testCase{ { - &structs.Resources{ - CPU: 2048, - MemoryMB: 512, - IOPS: 300, - DiskMB: 4096, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, + &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.Resources{ - CPU: 1024, - MemoryMB: 400, - IOPS: 200, - DiskMB: 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, + &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Cpu: structs.AllocatedCpuResources{ + CpuShares: 1024, }, - }, - }, - "0.986", - }, - { - &structs.Resources{ - CPU: 1024, - MemoryMB: 200, - IOPS: 200, - DiskMB: 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 512, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 400, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + MBits: 1024, + }, }, }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 1024, + }, }, - "1.138", + "0.928", }, { - &structs.Resources{ - CPU: 8192, - MemoryMB: 200, - IOPS: 200, - DiskMB: 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 512, + &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.169", + "3.152", }, { - &structs.Resources{ - CPU: 2048, - MemoryMB: 500, - IOPS: 300, - DiskMB: 4096, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 1024, + &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", }, @@ -111,7 +131,8 @@ func TestResourceDistance(t *testing.T) { for _, tc := range testCases { t.Run("", func(t *testing.T) { require := require.New(t) - require.Equal(tc.expectedDistance, fmt.Sprintf("%3.3f", resourceDistance(tc.allocResource, resourceAsk))) + actualDistance := fmt.Sprintf("%3.3f", basicResourceDistance(resourceAsk, tc.allocResource)) + require.Equal(tc.expectedDistance, actualDistance) }) } @@ -190,7 +211,7 @@ func TestPreemption(t *testing.T) { nodeCapacity: nodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ - CPU: 100, + CPU: 2000, MemoryMB: 256, DiskMB: 4 * 1024, Networks: []*structs.NetworkResource{ @@ -373,7 +394,6 @@ func TestPreemption(t *testing.T) { CPU: 1200, MemoryMB: 2256, DiskMB: 4 * 1024, - IOPS: 50, Networks: []*structs.NetworkResource{ { Device: "eth0", @@ -386,7 +406,6 @@ func TestPreemption(t *testing.T) { CPU: 200, MemoryMB: 256, DiskMB: 4 * 1024, - IOPS: 10, Networks: []*structs.NetworkResource{ { Device: "eth0", @@ -399,12 +418,11 @@ func TestPreemption(t *testing.T) { CPU: 200, MemoryMB: 256, DiskMB: 4 * 1024, - IOPS: 10, Networks: []*structs.NetworkResource{ { Device: "eth0", IP: "192.168.0.200", - MBits: 300, + MBits: 320, }, }, }), @@ -736,6 +754,9 @@ func TestPreemption(t *testing.T) { } require := require.New(t) err := state.UpsertAllocs(1001, tc.currentAllocations) + for _, alloc := range tc.currentAllocations { + fmt.Println(alloc.ID) + } require.Nil(err) if tc.currentPreemptions != nil { ctx.plan.NodePreemptions[node.ID] = tc.currentPreemptions @@ -752,6 +773,7 @@ func TestPreemption(t *testing.T) { }, }, } + binPackIter.SetTaskGroup(taskGroup) option := binPackIter.Next() if tc.preemptedAllocIDs == nil { @@ -760,6 +782,7 @@ func TestPreemption(t *testing.T) { require.NotNil(option) preemptedAllocs := option.PreemptedAllocs require.Equal(len(tc.preemptedAllocIDs), len(preemptedAllocs)) + fmt.Println(preemptedAllocs[0].ID) for _, alloc := range preemptedAllocs { _, ok := tc.preemptedAllocIDs[alloc.ID] require.True(ok) diff --git a/scheduler/rank.go b/scheduler/rank.go index 3c7adceaa37..7e8ba45e576 100644 --- a/scheduler/rank.go +++ b/scheduler/rank.go @@ -234,7 +234,7 @@ OUTER: } // Look for preemptible allocations to satisfy the network resource for this task - preemptedAllocsForTaskNetwork := preemptForNetworkResourceAsk(iter.priority, proposed, taskResources, netIdx, currentPreemptions) + preemptedAllocsForTaskNetwork := preemptForNetworkResourceAsk(iter.priority, proposed, ask, netIdx, currentPreemptions) if preemptedAllocsForTaskNetwork == nil { iter.ctx.Metrics().ExhaustedNode(option.Node, fmt.Sprintf("unable to meet network resource %v after preemption", ask)) From 9f35923db07a660d1f6af5679c4e78d69c8a930c Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 16 Oct 2018 20:46:44 -0500 Subject: [PATCH 14/38] fix end to end scheduler test to use new resource structs correctly --- scheduler/preemption.go | 18 +--- scheduler/preemption_test.go | 5 +- scheduler/system_sched_test.go | 181 +++++++++++++++++---------------- 3 files changed, 95 insertions(+), 109 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 053b81ef0d9..b40b3681d6e 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -122,7 +122,8 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. availableResources.Add(nodeRemainingResources) // This step needs the original resources asked for as the second arg, can't use the running total - allRequirementsMet = meetsNonNetworkRequirements(availableResources, resourceAsk.Comparable()) + allRequirementsMet, _ = availableResources.Superset(resourceAsk.Comparable()) + bestAllocs = append(bestAllocs, closestAlloc) allocGrp.allocs[closestAllocIndex] = allocGrp.allocs[len(allocGrp.allocs)-1] @@ -169,21 +170,6 @@ func computeCurrentPreemptions(currentAlloc *structs.Allocation, currentPreempti return numCurrentPreemptionsForJob } -// meetsNonNetworkRequirements checks if the first resource meets or exceeds the second resource's requirements -// This intentionally ignores network requirements, those are handled by meetsNetworkRequirements -func meetsNonNetworkRequirements(first *structs.ComparableResources, second *structs.ComparableResources) bool { - if first.Flattened.Cpu.CpuShares < second.Flattened.Cpu.CpuShares { - return false - } - if first.Flattened.Memory.MemoryMB < second.Flattened.Memory.MemoryMB { - return false - } - if first.Shared.DiskMB < second.Shared.DiskMB { - return false - } - return true -} - // meetsNetworkRequirements checks if the first resource meets or exceeds the second resource's network MBits requirements func meetsNetworkRequirements(firstMbits int, secondMbits int) bool { if firstMbits == 0 || secondMbits == 0 { diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go index b5f3ce61585..e9ca02ecb6f 100644 --- a/scheduler/preemption_test.go +++ b/scheduler/preemption_test.go @@ -754,9 +754,7 @@ func TestPreemption(t *testing.T) { } require := require.New(t) err := state.UpsertAllocs(1001, tc.currentAllocations) - for _, alloc := range tc.currentAllocations { - fmt.Println(alloc.ID) - } + require.Nil(err) if tc.currentPreemptions != nil { ctx.plan.NodePreemptions[node.ID] = tc.currentPreemptions @@ -782,7 +780,6 @@ func TestPreemption(t *testing.T) { require.NotNil(option) preemptedAllocs := option.PreemptedAllocs require.Equal(len(tc.preemptedAllocIDs), len(preemptedAllocs)) - fmt.Println(preemptedAllocs[0].ID) for _, alloc := range preemptedAllocs { _, ok := tc.preemptedAllocIDs[alloc.ID] require.True(ok) diff --git a/scheduler/system_sched_test.go b/scheduler/system_sched_test.go index 10cb83ae3c6..27d99b20547 100644 --- a/scheduler/system_sched_test.go +++ b/scheduler/system_sched_test.go @@ -1563,6 +1563,7 @@ func TestSystemSched_Preemption(t *testing.T) { var nodes []*structs.Node for i := 0; i < 2; i++ { node := mock.Node() + //TODO(preetha): remove in 0.11 node.Resources = &structs.Resources{ CPU: 3072, MemoryMB: 5034, @@ -1576,6 +1577,24 @@ func TestSystemSched_Preemption(t *testing.T) { }, }, } + node.NodeResources = &structs.NodeResources{ + Cpu: structs.NodeCpuResources{ + CpuShares: 3072, + }, + Memory: structs.NodeMemoryResources{ + MemoryMB: 5034, + }, + Disk: structs.NodeDiskResources{ + DiskMB: 20 * 1024, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + CIDR: "192.168.0.100/32", + MBits: 1000, + }, + }, + } noErr(t, h.State.UpsertNode(h.NextIndex(), node)) nodes = append(nodes, node) } @@ -1614,42 +1633,30 @@ func TestSystemSched_Preemption(t *testing.T) { alloc1.NodeID = nodes[0].ID alloc1.Name = "my-job[0]" alloc1.TaskGroup = job1.TaskGroups[0].Name - alloc1.Resources = &structs.Resources{ - CPU: 512, - MemoryMB: 1024, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 200, - ReservedPorts: []structs.Port{ + alloc1.AllocatedResources = &structs.AllocatedResources{ + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: 512, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 1024, + }, + Networks: []*structs.NetworkResource{ { - Label: "web", - Value: 80, + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "web", Value: 80}}, + MBits: 200, }, }, }, }, - } - alloc1.TaskResources = map[string]*structs.Resources{ - "web": { - CPU: 512, - MemoryMB: 1024, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 200, - ReservedPorts: []structs.Port{ - { - Label: "web", - Value: 80, - }, - }, - }, - }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 5 * 1024, }, } + noErr(t, h.State.UpsertJob(h.NextIndex(), job1)) job2 := mock.BatchJob() @@ -1671,29 +1678,27 @@ func TestSystemSched_Preemption(t *testing.T) { alloc2.NodeID = nodes[0].ID alloc2.Name = "my-job[2]" alloc2.TaskGroup = job2.TaskGroups[0].Name - alloc2.Resources = &structs.Resources{ - CPU: 512, - MemoryMB: 1024, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 200, - }, - }, - } - alloc2.TaskResources = map[string]*structs.Resources{ - "web": { - CPU: 512, - MemoryMB: 1024, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 200, + alloc2.AllocatedResources = &structs.AllocatedResources{ + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: 512, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 1024, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + MBits: 200, + }, }, }, }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 5 * 1024, + }, } noErr(t, h.State.UpsertJob(h.NextIndex(), job2)) @@ -1717,29 +1722,28 @@ func TestSystemSched_Preemption(t *testing.T) { alloc3.NodeID = nodes[0].ID alloc3.Name = "my-job[0]" alloc3.TaskGroup = job3.TaskGroups[0].Name - alloc3.Resources = &structs.Resources{ - CPU: 1024, - MemoryMB: 25, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 400, - }, - }, - } - alloc3.TaskResources = map[string]*structs.Resources{ - "web": { - CPU: 1024, - MemoryMB: 25, - DiskMB: 5 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 400, + alloc3.AllocatedResources = &structs.AllocatedResources{ + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: 1024, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 25, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "web", Value: 80}}, + MBits: 400, + }, }, }, }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 5 * 1024, + }, } noErr(t, h.State.UpsertAllocs(h.NextIndex(), []*structs.Allocation{alloc1, alloc2, alloc3})) @@ -1765,29 +1769,28 @@ func TestSystemSched_Preemption(t *testing.T) { alloc4.NodeID = nodes[0].ID alloc4.Name = "my-job4[0]" alloc4.TaskGroup = job4.TaskGroups[0].Name - alloc4.Resources = &structs.Resources{ - CPU: 1024, - MemoryMB: 2048, - DiskMB: 2 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 100, - }, - }, - } - alloc4.TaskResources = map[string]*structs.Resources{ - "web": { - CPU: 1024, - MemoryMB: 2048, - DiskMB: 2 * 1024, - Networks: []*structs.NetworkResource{ - { - Device: "eth0", - MBits: 100, + alloc4.AllocatedResources = &structs.AllocatedResources{ + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: 1024, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 2048, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "web", Value: 80}}, + MBits: 100, + }, }, }, }, + Shared: structs.AllocatedSharedResources{ + DiskMB: 2 * 1024, + }, } noErr(t, h.State.UpsertJob(h.NextIndex(), job4)) noErr(t, h.State.UpsertAllocs(h.NextIndex(), []*structs.Allocation{alloc4})) From a960cced8c3ead7a4f97279d69207f78edbd1cff Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 16 Oct 2018 21:21:55 -0500 Subject: [PATCH 15/38] comments --- scheduler/preemption.go | 45 ++++++++++++++++++++++++++--------------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index b40b3681d6e..d8c79e10914 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -12,6 +12,8 @@ import ( // number of allocations being preempted exceeds max_parallel value in the job's migrate stanza const maxParallelPenalty = 50.0 +// basicResourceDistance computes a distance using a coordinate system. It compares resource fields like CPU/Memory and Disk. +// Values emitted are in the range [0, maxFloat] func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources) float64 { memoryCoord, cpuCoord, diskMBCoord := 0.0, 0.0, 0.0 if resourceAsk.Flattened.Memory.MemoryMB > 0 { @@ -30,6 +32,18 @@ func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUse return originDist } +// networkResourceDistance returns a distance based only on network megabits +func networkResourceDistance(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource) float64 { + networkCoord := math.MaxFloat64 + if resourceUsed != nil && resourceNeeded != nil { + networkCoord = float64(resourceNeeded.MBits-resourceUsed.MBits) / float64(resourceNeeded.MBits) + } + + originDist := math.Sqrt( + math.Pow(networkCoord, 2)) + return originDist +} + // getPreemptionScoreForTaskGroupResources is used to calculate a score (lower is better) based on the distance between // the needed resource and requirements. A penalty is added when the choice already has some existing // allocations in the plan that are being preempted. @@ -41,7 +55,8 @@ func getPreemptionScoreForTaskGroupResources(resourceAsk *structs.ComparableReso return basicResourceDistance(resourceAsk, resourceUsed) + maxParallelScorePenalty } -// getPreemptionScoreForNetwork is similar to getPreemptionScoreForTaskGroupResources but only uses network mbits to calculate a preemption score +// getPreemptionScoreForNetwork is similar to getPreemptionScoreForTaskGroupResources +// but only uses network Mbits to calculate a preemption score func getPreemptionScoreForNetwork(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource, maxParallel int, numPreemptedAllocs int) float64 { if resourceUsed == nil || resourceNeeded == nil { return math.MaxFloat64 @@ -53,21 +68,9 @@ func getPreemptionScoreForNetwork(resourceUsed *structs.NetworkResource, resourc return networkResourceDistance(resourceUsed, resourceNeeded) + maxParallelScorePenalty } -// networkResourceDistance returns distance based on network megabits -func networkResourceDistance(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource) float64 { - networkCoord := math.MaxFloat64 - if resourceUsed != nil && resourceNeeded != nil { - networkCoord = float64(resourceNeeded.MBits-resourceUsed.MBits) / float64(resourceNeeded.MBits) - } - - originDist := math.Sqrt( - math.Pow(networkCoord, 2)) - return originDist -} - // findPreemptibleAllocationsForTaskGroup computes a list of allocations to preempt to accommodate // the resources asked for. Only allocs with a job priority < 10 of jobPriority are considered -// This method is used after network resource needs have already been met. +// This method is meant only for finding preemptible allocations based on CPU/Memory/Disk func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs.Allocation, resourceAsk *structs.AllocatedResources, node *structs.Node, currentPreemptions []*structs.Allocation) []*structs.Allocation { resourcesNeeded := resourceAsk.Comparable() allocsByPriority := filterAndGroupPreemptibleAllocs(jobPriority, current) @@ -183,6 +186,8 @@ type groupedAllocs struct { allocs []*structs.Allocation } +// filterAndGroupPreemptibleAllocs groups allocations by priority after removing any from jobs of +// a higher priority than jobPriority func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocation) []*groupedAllocs { allocsByPriority := make(map[int][]*structs.Allocation) for _, alloc := range current { @@ -220,6 +225,9 @@ func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocat return groupedSortedAllocs } +// eliminateSuperSetAllocationsForTaskGroup is used as a final step to remove +// any allocations that meet a superset of requirements from the set of allocations +// to preempt func eliminateSuperSetAllocationsForTaskGroup(bestAllocs []*structs.Allocation, nodeRemainingResources *structs.ComparableResources, resourceAsk *structs.ComparableResources) []*structs.Allocation { @@ -247,6 +255,8 @@ func eliminateSuperSetAllocationsForTaskGroup(bestAllocs []*structs.Allocation, return filteredBestAllocs } +// eliminateSuperSetAllocationsForNetwork is similar to eliminateSuperSetAllocationsForTaskGroup but only +// considers network Mbits func eliminateSuperSetAllocationsForNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk *structs.NetworkResource, nodeRemainingResources *structs.ComparableResources) []*structs.Allocation { @@ -274,8 +284,8 @@ func eliminateSuperSetAllocationsForNetwork(bestAllocs []*structs.Allocation, ne } // preemptForNetworkResourceAsk tries to find allocations to preempt to meet network resources. -// this needs to consider network resources at the task level and for the same task it should -// only preempt allocations that share the same network device +// This is called once per task when assigning a network to the task. While finding allocations +// to preempt, this only considers allocations that share the same network device func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allocation, networkResourceAsk *structs.NetworkResource, netIdx *structs.NetworkIndex, currentPreemptions []*structs.Allocation) []*structs.Allocation { @@ -446,6 +456,9 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo return filteredBestAllocs } +// distanceComparatorForNetwork is used as the sorting function when finding allocations to preempt. It uses +// both a coordinayte distance function based on Mbits needed, and a penalty if the allocation under consideration +// belongs to a job that already has more preempted allocations func distanceComparatorForNetwork(allocs []*structs.Allocation, currentPreemptions []*structs.Allocation, networkResourceAsk *structs.NetworkResource, i int, j int) bool { firstAlloc := allocs[i] currentPreemptionCount1 := computeCurrentPreemptions(firstAlloc, currentPreemptions) From 655689a743d10f8c43b1f734dcbff4ad3a08e085 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 16 Oct 2018 21:26:42 -0500 Subject: [PATCH 16/38] Preempted allocations should be removed from proposed allocations --- scheduler/context.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/scheduler/context.go b/scheduler/context.go index 031a3b45a51..1da0776913e 100644 --- a/scheduler/context.go +++ b/scheduler/context.go @@ -123,6 +123,12 @@ func (e *EvalContext) ProposedAllocs(nodeID string) ([]*structs.Allocation, erro proposed = structs.RemoveAllocs(existingAlloc, update) } + // Remove any allocs that are being preempted + nodePreemptedAllocs := e.plan.NodePreemptions[nodeID] + if len(nodePreemptedAllocs) > 0 { + proposed = structs.RemoveAllocs(existingAlloc, nodePreemptedAllocs) + } + // We create an index of the existing allocations so that if an inplace // update occurs, we do not double count and we override the old allocation. proposedIDs := make(map[string]*structs.Allocation, len(proposed)) From c4e0e66c5b2d62b202ac8843f25e95f9f268c5a9 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Wed, 17 Oct 2018 21:40:14 -0500 Subject: [PATCH 17/38] Restore/Snapshot plus unit tests for scheduler configuration --- nomad/fsm.go | 57 +++++++++++++++++++++++++-------- nomad/fsm_test.go | 23 +++++++++++++ nomad/state/state_store.go | 7 ++++ nomad/state/state_store_test.go | 27 ++++++++++++++++ 4 files changed, 101 insertions(+), 13 deletions(-) diff --git a/nomad/fsm.go b/nomad/fsm.go index bba9d0e723d..dee8420ffb1 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -45,6 +45,7 @@ const ( DeploymentSnapshot ACLPolicySnapshot ACLTokenSnapshot + SchedulerConfigSnapshot ) // LogApplier is the definition of a function that can apply a Raft log @@ -815,6 +816,8 @@ func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { n.logger.Error("ApplyPlan failed", "error", err) return err } + + // Add evals for jobs that were preempted n.handleUpsertedEvals(req.PreemptionEvals) return nil } @@ -996,6 +999,23 @@ func (n *nomadFSM) applyAutopilotUpdate(buf []byte, index uint64) interface{} { return n.state.AutopilotSetConfig(index, &req.Config) } +func (n *nomadFSM) applySchedulerConfigUpdate(buf []byte, index uint64) interface{} { + var req structs.SchedulerSetConfigRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + defer metrics.MeasureSince([]string{"nomad", "fsm", "scheduler-config"}, time.Now()) + + if req.CAS { + act, err := n.state.SchedulerCASConfig(index, req.Config.ModifyIndex, &req.Config) + if err != nil { + return err + } + return act + } + return n.state.SchedulerSetConfig(index, &req.Config) +} + func (n *nomadFSM) Snapshot() (raft.FSMSnapshot, error) { // Create a new snapshot snap, err := n.state.Snapshot() @@ -1216,6 +1236,15 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { return err } + case SchedulerConfigSnapshot: + schedConfig := new(structs.SchedulerConfiguration) + if err := dec.Decode(schedConfig); err != nil { + return err + } + if err := restore.SchedulerConfigRestore(schedConfig); err != nil { + return err + } + default: // Check if this is an enterprise only object being restored restorer, ok := n.enterpriseRestorers[snapType] @@ -1503,6 +1532,10 @@ func (s *nomadSnapshot) Persist(sink raft.SnapshotSink) error { sink.Cancel() return err } + if err := s.persistSchedulerConfig(sink, encoder); err != nil { + sink.Cancel() + return err + } return nil } @@ -1835,21 +1868,19 @@ func (s *nomadSnapshot) persistACLTokens(sink raft.SnapshotSink, return nil } -func (n *nomadFSM) applySchedulerConfigUpdate(buf []byte, index uint64) interface{} { - var req structs.SchedulerSetConfigRequest - if err := structs.Decode(buf, &req); err != nil { - panic(fmt.Errorf("failed to decode request: %v", err)) +func (s *nomadSnapshot) persistSchedulerConfig(sink raft.SnapshotSink, + encoder *codec.Encoder) error { + // Get scheduler config + _, schedConfig, err := s.snap.SchedulerConfig() + if err != nil { + return err } - defer metrics.MeasureSince([]string{"nomad", "fsm", "scheduler-config"}, time.Now()) - - if req.CAS { - act, err := n.state.SchedulerCASConfig(index, req.Config.ModifyIndex, &req.Config) - if err != nil { - return err - } - return act + // Write out scheduler config + sink.Write([]byte{byte(SchedulerConfigSnapshot)}) + if err := encoder.Encode(schedConfig); err != nil { + return err } - return n.state.SchedulerSetConfig(index, &req.Config) + return nil } // Release is a no-op, as we just need to GC the pointer diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 99c65382c71..dcb3b917006 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -2634,6 +2634,29 @@ func TestFSM_SnapshotRestore_ACLTokens(t *testing.T) { assert.Equal(t, tk2, out2) } +func TestFSM_SnapshotRestore_SchedulerConfiguration(t *testing.T) { + t.Parallel() + // Add some state + fsm := testFSM(t) + state := fsm.State() + schedConfig := &structs.SchedulerConfiguration{ + PreemptionConfig: structs.PreemptionConfig{ + SystemSchedulerEnabled: true, + }, + } + state.SchedulerSetConfig(1000, schedConfig) + + // Verify the contents + require := require.New(t) + fsm2 := testSnapshotRestore(t, fsm) + state2 := fsm2.State() + index, out, err := state2.SchedulerConfig() + require.Nil(err) + require.EqualValues(1000, index) + require.Equal(schedConfig, out) + +} + func TestFSM_SnapshotRestore_AddMissingSummary(t *testing.T) { t.Parallel() // Add some state diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index b228cfe10d9..09b742750a1 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -3973,6 +3973,13 @@ func (r *StateRestore) ACLTokenRestore(token *structs.ACLToken) error { return nil } +func (r *StateRestore) SchedulerConfigRestore(schedConfig *structs.SchedulerConfiguration) error { + if err := r.txn.Insert("scheduler_config", schedConfig); err != nil { + return fmt.Errorf("inserting scheduler config failed: %s", err) + } + return nil +} + // addEphemeralDiskToTaskGroups adds missing EphemeralDisk objects to TaskGroups func (r *StateRestore) addEphemeralDiskToTaskGroups(job *structs.Job) { for _, tg := range job.TaskGroups { diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 194eaedac28..a750a8fa2af 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -6766,6 +6766,33 @@ func TestStateStore_RestoreACLToken(t *testing.T) { assert.Equal(t, token, out) } +func TestStateStore_SchedulerConfig(t *testing.T) { + state := testStateStore(t) + schedConfig := &structs.SchedulerConfiguration{ + PreemptionConfig: structs.PreemptionConfig{ + SystemSchedulerEnabled: false, + }, + CreateIndex: 100, + ModifyIndex: 200, + } + + require := require.New(t) + restore, err := state.Restore() + + require.Nil(err) + + err = restore.SchedulerConfigRestore(schedConfig) + require.Nil(err) + + restore.Commit() + + modIndex, out, err := state.SchedulerConfig() + require.Nil(err) + require.Equal(schedConfig.ModifyIndex, modIndex) + + require.Equal(schedConfig, out) +} + func TestStateStore_Abandon(t *testing.T) { s := testStateStore(t) abandonCh := s.AbandonCh() From c3b8e4f5737cd40ff140d55fa72a42b106c154dc Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Wed, 17 Oct 2018 22:29:29 -0500 Subject: [PATCH 18/38] Add fsm layer tests --- nomad/fsm.go | 6 +++--- nomad/fsm_test.go | 49 ++++++++++++++++++++++++++++++++++++++++++++--- 2 files changed, 49 insertions(+), 6 deletions(-) diff --git a/nomad/fsm.go b/nomad/fsm.go index dee8420ffb1..71b4c07bdab 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -1004,14 +1004,14 @@ func (n *nomadFSM) applySchedulerConfigUpdate(buf []byte, index uint64) interfac if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - defer metrics.MeasureSince([]string{"nomad", "fsm", "scheduler-config"}, time.Now()) + defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_scheduler_config"}, time.Now()) if req.CAS { - act, err := n.state.SchedulerCASConfig(index, req.Config.ModifyIndex, &req.Config) + applied, err := n.state.SchedulerCASConfig(index, req.Config.ModifyIndex, &req.Config) if err != nil { return err } - return act + return applied } return n.state.SchedulerSetConfig(index, &req.Config) } diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index dcb3b917006..ebe62189d78 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1664,7 +1664,7 @@ func TestFSM_DeregisterVaultAccessor(t *testing.T) { func TestFSM_ApplyPlanResults(t *testing.T) { t.Parallel() fsm := testFSM(t) - + fsm.evalBroker.SetEnabled(true) // Create the request and create a deployment alloc := mock.Alloc() alloc.Resources = &structs.Resources{} // COMPAT(0.11): Remove in 0.11, used to bypass resource creation in state store @@ -1683,13 +1683,39 @@ func TestFSM_ApplyPlanResults(t *testing.T) { fsm.State().UpsertEvals(1, []*structs.Evaluation{eval}) fsm.State().UpsertJobSummary(1, mock.JobSummary(alloc.JobID)) + + // set up preempted jobs and allocs + job1 := mock.Job() + job2 := mock.Job() + + alloc1 := mock.Alloc() + alloc1.Job = job1 + alloc1.JobID = job1.ID + alloc1.PreemptedByAllocation = alloc.ID + + alloc2 := mock.Alloc() + alloc2.Job = job2 + alloc2.JobID = job2.ID + alloc2.PreemptedByAllocation = alloc.ID + + fsm.State().UpsertAllocs(1, []*structs.Allocation{alloc1, alloc2}) + + // evals for preempted jobs + eval1 := mock.Eval() + eval1.JobID = job1.ID + + eval2 := mock.Eval() + eval2.JobID = job2.ID + req := structs.ApplyPlanResultsRequest{ AllocUpdateRequest: structs.AllocUpdateRequest{ Job: job, Alloc: []*structs.Allocation{alloc}, }, - Deployment: d, - EvalID: eval.ID, + Deployment: d, + EvalID: eval.ID, + NodePreemptions: []*structs.Allocation{alloc1, alloc2}, + PreemptionEvals: []*structs.Evaluation{eval1, eval2}, } buf, err := structs.Encode(structs.ApplyPlanResultsRequestType, req) if err != nil { @@ -1714,6 +1740,23 @@ func TestFSM_ApplyPlanResults(t *testing.T) { alloc.Job = job assert.Equal(alloc, out) + // Verify that evals for preempted jobs have been created + e1, err := fsm.State().EvalByID(ws, eval1.ID) + require := require.New(t) + require.Nil(err) + require.NotNil(e1) + + e2, err := fsm.State().EvalByID(ws, eval2.ID) + require.Nil(err) + require.NotNil(e2) + + // Verify that eval broker has both evals + _, ok := fsm.evalBroker.evals[e1.ID] + require.True(ok) + + _, ok = fsm.evalBroker.evals[e1.ID] + require.True(ok) + dout, err := fsm.State().DeploymentByID(ws, d.ID) assert.Nil(err) tg, ok := dout.TaskGroups[alloc.TaskGroup] From 12af2aed49d4f7649bdcbccd30897fc3a472ed00 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Wed, 17 Oct 2018 22:38:05 -0500 Subject: [PATCH 19/38] make default config a variable --- nomad/leader.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/nomad/leader.go b/nomad/leader.go index e43edc67003..bf3f350f152 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -42,6 +42,9 @@ const ( var minAutopilotVersion = version.Must(version.NewVersion("0.8.0")) +// Default configuration for scheduler with preemption emabled for system jobs +var defaultSchedulerConfig = &structs.SchedulerConfiguration{PreemptionConfig: structs.PreemptionConfig{SystemSchedulerEnabled: true}} + // monitorLeadership is used to monitor if we acquire or lose our role // as the leader in the Raft cluster. There is some work the leader is // expected to do, so we must react to changes @@ -1246,8 +1249,7 @@ func (s *Server) getOrCreateSchedulerConfig() *structs.SchedulerConfiguration { return config } - config = &structs.SchedulerConfiguration{PreemptionConfig: structs.PreemptionConfig{SystemSchedulerEnabled: true}} - req := structs.SchedulerSetConfigRequest{Config: *config} + req := structs.SchedulerSetConfigRequest{Config: *defaultSchedulerConfig} if _, _, err = s.raftApply(structs.SchedulerConfigRequestType, req); err != nil { s.logger.Named("core").Error("failed to initialize config", "error", err) return nil From c4a04eb1b209356b9f613a803458223f84ff70d4 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Wed, 17 Oct 2018 22:45:40 -0500 Subject: [PATCH 20/38] style fixes --- nomad/operator_endpoint.go | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/nomad/operator_endpoint.go b/nomad/operator_endpoint.go index a04f1284337..01aed37416d 100644 --- a/nomad/operator_endpoint.go +++ b/nomad/operator_endpoint.go @@ -295,8 +295,7 @@ func (op *Operator) SchedulerSetConfiguration(args *structs.SchedulerSetConfigRe rule, err := op.srv.ResolveToken(args.AuthToken) if err != nil { return err - } - if rule != nil && !rule.AllowOperatorWrite() { + } else if rule != nil && !rule.AllowOperatorWrite() { return structs.ErrPermissionDenied } @@ -305,8 +304,7 @@ func (op *Operator) SchedulerSetConfiguration(args *structs.SchedulerSetConfigRe if err != nil { op.logger.Error("failed applying Scheduler configuration", "error", err) return err - } - if respErr, ok := resp.(error); ok { + } else if respErr, ok := resp.(error); ok { return respErr } @@ -327,8 +325,7 @@ func (op *Operator) SchedulerGetConfiguration(args *structs.GenericRequest, repl rule, err := op.srv.ResolveToken(args.AuthToken) if err != nil { return err - } - if rule != nil && !rule.AllowOperatorRead() { + } else if rule != nil && !rule.AllowOperatorRead() { return structs.ErrPermissionDenied } @@ -336,8 +333,7 @@ func (op *Operator) SchedulerGetConfiguration(args *structs.GenericRequest, repl _, config, err := state.SchedulerConfig() if err != nil { return err - } - if config == nil { + } else if config == nil { return fmt.Errorf("scheduler config not initialized yet") } From 191b8626d25c0558b168ceac58a278167b7ea0e1 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Wed, 17 Oct 2018 23:06:53 -0500 Subject: [PATCH 21/38] More review comments --- api/operator_test.go | 2 +- nomad/plan_apply.go | 11 +-- nomad/state/state_store.go | 168 +++++++++++++++++++------------------ 3 files changed, 91 insertions(+), 90 deletions(-) diff --git a/api/operator_test.go b/api/operator_test.go index 9c95228af71..25dcf44a8ee 100644 --- a/api/operator_test.go +++ b/api/operator_test.go @@ -77,7 +77,7 @@ func TestAPI_OperatorSchedulerGetSetConfiguration(t *testing.T) { config, _, err = operator.SchedulerGetConfiguration(nil) require.Nil(err) - require.True(config.PreemptionConfig.SystemSchedulerEnabled) + require.False(config.PreemptionConfig.SystemSchedulerEnabled) } func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index f48d7c4fbe2..205609ce5e1 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -191,9 +191,6 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap // Also gather jobids to create follow up evals preemptedJobIDs := make(map[structs.NamespacedID]struct{}) for _, alloc := range req.NodePreemptions { - if alloc.CreateTime == 0 { - alloc.CreateTime = now - } alloc.ModifyTime = now id := structs.NamespacedID{Namespace: alloc.Namespace, ID: alloc.JobID} _, ok := preemptedJobIDs[id] @@ -368,13 +365,13 @@ func evaluatePlanPlacements(pool *EvaluatePool, snap *state.StateSnapshot, plan alloc, err := snap.AllocByID(nil, preemptedAlloc.ID) if err != nil { mErr.Errors = append(mErr.Errors, err) + continue } - if alloc != nil { - if !alloc.TerminalStatus() { - filteredNodePreemptions = append(filteredNodePreemptions, preemptedAlloc) - } + if alloc != nil && !alloc.TerminalStatus() { + filteredNodePreemptions = append(filteredNodePreemptions, preemptedAlloc) } } + result.NodePreemptions[nodeID] = filteredNodePreemptions } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 09b742750a1..793d01e8309 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -219,6 +219,7 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR } // Prepare preempted allocs in the plan results for update + var preemptedAllocs []*structs.Allocation for _, preemptedAlloc := range results.NodePreemptions { // Look for existing alloc existing, err := txn.First("allocs", "id", preemptedAlloc.ID) @@ -239,11 +240,14 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR copyAlloc.DesiredStatus = preemptedAlloc.DesiredStatus copyAlloc.PreemptedByAllocation = preemptedAlloc.PreemptedByAllocation copyAlloc.DesiredDescription = preemptedAlloc.DesiredDescription + copyAlloc.ModifyTime = preemptedAlloc.ModifyTime + preemptedAllocs = append(preemptedAllocs, copyAlloc) - // Upsert the preempted allocations - if err := s.upsertAllocsImpl(index, []*structs.Allocation{copyAlloc}, txn); err != nil { - return err - } + } + + // Upsert the preempted allocations + if err := s.upsertAllocsImpl(index, preemptedAllocs, txn); err != nil { + return err } // Upsert followup evals for allocs that were preempted @@ -3855,6 +3859,84 @@ func (s *StateStore) BootstrapACLTokens(index, resetIndex uint64, token *structs return nil } +// SchedulerConfig is used to get the current Scheduler configuration. +func (s *StateStore) SchedulerConfig() (uint64, *structs.SchedulerConfiguration, error) { + tx := s.db.Txn(false) + defer tx.Abort() + + // Get the scheduler config + c, err := tx.First("scheduler_config", "id") + if err != nil { + return 0, nil, fmt.Errorf("failed scheduler config lookup: %s", err) + } + + config, ok := c.(*structs.SchedulerConfiguration) + if !ok { + return 0, nil, nil + } + + return config.ModifyIndex, config, nil +} + +// SchedulerSetConfig is used to set the current Scheduler configuration. +func (s *StateStore) SchedulerSetConfig(idx uint64, config *structs.SchedulerConfiguration) error { + tx := s.db.Txn(true) + defer tx.Abort() + + s.schedulerSetConfigTxn(idx, tx, config) + + tx.Commit() + return nil +} + +// SchedulerCASConfig is used to try updating the scheduler configuration with a +// given Raft index. If the CAS index specified is not equal to the last observed index +// for the config, then the call is a noop, +func (s *StateStore) SchedulerCASConfig(idx, cidx uint64, config *structs.SchedulerConfiguration) (bool, error) { + tx := s.db.Txn(true) + defer tx.Abort() + + // Check for an existing config + existing, err := tx.First("scheduler_config", "id") + if err != nil { + return false, fmt.Errorf("failed scheduler config lookup: %s", err) + } + + // If the existing index does not match the provided CAS + // index arg, then we shouldn't update anything and can safely + // return early here. + e, ok := existing.(*structs.SchedulerConfiguration) + if !ok || e.ModifyIndex != cidx { + return false, nil + } + + s.schedulerSetConfigTxn(idx, tx, config) + + tx.Commit() + return true, nil +} + +func (s *StateStore) schedulerSetConfigTxn(idx uint64, tx *memdb.Txn, config *structs.SchedulerConfiguration) error { + // Check for an existing config + existing, err := tx.First("scheduler_config", "id") + if err != nil { + return fmt.Errorf("failed scheduler config lookup: %s", err) + } + + // Set the indexes. + if existing != nil { + config.CreateIndex = existing.(*structs.SchedulerConfiguration).CreateIndex + } else { + config.CreateIndex = idx + } + config.ModifyIndex = idx + + if err := tx.Insert("scheduler_config", config); err != nil { + return fmt.Errorf("failed updating scheduler config: %s", err) + } + return nil +} + // StateSnapshot is used to provide a point-in-time snapshot type StateSnapshot struct { StateStore @@ -3998,81 +4080,3 @@ func (r *StateRestore) addEphemeralDiskToTaskGroups(job *structs.Job) { } } } - -// SchedulerConfig is used to get the current Scheduler configuration. -func (s *StateStore) SchedulerConfig() (uint64, *structs.SchedulerConfiguration, error) { - tx := s.db.Txn(false) - defer tx.Abort() - - // Get the scheduler config - c, err := tx.First("scheduler_config", "id") - if err != nil { - return 0, nil, fmt.Errorf("failed scheduler config lookup: %s", err) - } - - config, ok := c.(*structs.SchedulerConfiguration) - if !ok { - return 0, nil, nil - } - - return config.ModifyIndex, config, nil -} - -// SchedulerSetConfig is used to set the current Scheduler configuration. -func (s *StateStore) SchedulerSetConfig(idx uint64, config *structs.SchedulerConfiguration) error { - tx := s.db.Txn(true) - defer tx.Abort() - - s.schedulerSetConfigTxn(idx, tx, config) - - tx.Commit() - return nil -} - -// SchedulerCASConfig is used to try updating the scheduler configuration with a -// given Raft index. If the CAS index specified is not equal to the last observed index -// for the config, then the call is a noop, -func (s *StateStore) SchedulerCASConfig(idx, cidx uint64, config *structs.SchedulerConfiguration) (bool, error) { - tx := s.db.Txn(true) - defer tx.Abort() - - // Check for an existing config - existing, err := tx.First("scheduler_config", "id") - if err != nil { - return false, fmt.Errorf("failed scheduler config lookup: %s", err) - } - - // If the existing index does not match the provided CAS - // index arg, then we shouldn't update anything and can safely - // return early here. - e, ok := existing.(*structs.SchedulerConfiguration) - if !ok || e.ModifyIndex != cidx { - return false, nil - } - - s.schedulerSetConfigTxn(idx, tx, config) - - tx.Commit() - return true, nil -} - -func (s *StateStore) schedulerSetConfigTxn(idx uint64, tx *memdb.Txn, config *structs.SchedulerConfiguration) error { - // Check for an existing config - existing, err := tx.First("scheduler_config", "id") - if err != nil { - return fmt.Errorf("failed scheduler config lookup: %s", err) - } - - // Set the indexes. - if existing != nil { - config.CreateIndex = existing.(*structs.SchedulerConfiguration).CreateIndex - } else { - config.CreateIndex = idx - } - config.ModifyIndex = idx - - if err := tx.Insert("scheduler_config", config); err != nil { - return fmt.Errorf("failed updating scheduler config: %s", err) - } - return nil -} From 21432d69dc12cef32c7dc1612fb7b143e527bf85 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Wed, 17 Oct 2018 23:49:37 -0500 Subject: [PATCH 22/38] More style and readablity fixes from review --- nomad/state/state_store_test.go | 32 +++++------ nomad/structs/operator.go | 1 + nomad/structs/structs.go | 40 +++----------- nomad/structs/structs_test.go | 95 +++++++++++++++++++-------------- scheduler/generic_sched.go | 3 +- scheduler/rank.go | 18 ++++--- scheduler/system_sched.go | 4 +- scheduler/testing.go | 3 -- 8 files changed, 95 insertions(+), 101 deletions(-) diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index a750a8fa2af..3fa78d14e0c 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -248,27 +248,27 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { // 1) Preempted allocations in plan results are updated // 2) Evals are inserted for preempted jobs func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { + require := require.New(t) + state := testStateStore(t) alloc := mock.Alloc() job := alloc.Job alloc.Job = nil - require := require.New(t) - + // Insert job err := state.UpsertJob(999, job) - require.Nil(err) + require.NoError(err) + // Create an eval eval := mock.Eval() eval.JobID = job.ID - - // Create an eval err = state.UpsertEvals(1, []*structs.Evaluation{eval}) - require.Nil(err) + require.NoError(err) - // Insert alloc that'll be preempted in the plan + // Insert alloc that will be preempted in the plan preemptedAlloc := mock.Alloc() err = state.UpsertAllocs(2, []*structs.Allocation{preemptedAlloc}) - require.Nil(err) + require.NoError(err) minimalPreemptedAlloc := &structs.Allocation{ ID: preemptedAlloc.ID, @@ -277,6 +277,7 @@ func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { DesiredDescription: fmt.Sprintf("Preempted by allocation %v", alloc.ID), } + // Create eval for preempted job eval2 := mock.Eval() eval2.JobID = preemptedAlloc.JobID @@ -292,32 +293,33 @@ func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { } err = state.UpsertPlanResults(1000, &res) - require.Nil(err) + require.NoError(err) ws := memdb.NewWatchSet() // Verify alloc and eval created by plan out, err := state.AllocByID(ws, alloc.ID) - require.Nil(err) + require.NoError(err) require.Equal(alloc, out) index, err := state.Index("allocs") - require.Nil(err) + require.NoError(err) require.EqualValues(1000, index) evalOut, err := state.EvalByID(ws, eval.ID) - require.Nil(err) + require.NoError(err) require.NotNil(evalOut) require.EqualValues(1000, evalOut.ModifyIndex) - // Verify preempted alloc and eval for preempted job + // Verify preempted alloc preempted, err := state.AllocByID(ws, preemptedAlloc.ID) - require.Nil(err) + require.NoError(err) require.Equal(preempted.DesiredStatus, structs.AllocDesiredStatusEvict) require.Equal(preempted.DesiredDescription, fmt.Sprintf("Preempted by allocation %v", alloc.ID)) + // Verify eval for preempted job preemptedJobEval, err := state.EvalByID(ws, eval2.ID) - require.Nil(err) + require.NoError(err) require.NotNil(preemptedJobEval) require.EqualValues(1000, preemptedJobEval.ModifyIndex) diff --git a/nomad/structs/operator.go b/nomad/structs/operator.go index f6143a053d4..97809726eee 100644 --- a/nomad/structs/operator.go +++ b/nomad/structs/operator.go @@ -120,6 +120,7 @@ type AutopilotConfig struct { ModifyIndex uint64 } +// SchedulerConfiguration is the config for controlling scheduler behavior type SchedulerConfiguration struct { // PreemptionConfig specifies whether to enable eviction of lower // priority jobs to place higher priority jobs. diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 0040a680201..4e7269410f1 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1885,27 +1885,6 @@ func (r *Resources) Add(delta *Resources) error { return nil } -// Subtract removes the resources of the delta to this, potentially -// returning an error if not possible. -func (r *Resources) Subtract(delta *Resources) error { - if delta == nil { - return nil - } - r.CPU -= delta.CPU - r.MemoryMB -= delta.MemoryMB - r.DiskMB -= delta.DiskMB - r.IOPS -= delta.IOPS - - for _, n := range delta.Networks { - // Find the matching interface by IP or CIDR - idx := r.NetIndex(n) - if idx != -1 { - r.Networks[idx].MBits -= delta.Networks[idx].MBits - } - } - return nil -} - func (r *Resources) GoString() string { return fmt.Sprintf("*%#v", *r) } @@ -8224,10 +8203,15 @@ func (p *Plan) AppendPreemptedAlloc(alloc *Allocation, desiredStatus, preempting // TaskResources are needed by the plan applier to check if allocations fit // after removing preempted allocations - newAlloc.TaskResources = alloc.TaskResources + if alloc.AllocatedResources != nil { + newAlloc.AllocatedResources = alloc.AllocatedResources + } else { + newAlloc.TaskResources = alloc.TaskResources + newAlloc.SharedResources = alloc.SharedResources + } - node := alloc.NodeID // Append this alloc to slice for this node + node := alloc.NodeID existing := p.NodePreemptions[node] p.NodePreemptions[node] = append(existing, newAlloc) } @@ -8263,14 +8247,6 @@ func (p *Plan) IsNoOp() bool { len(p.DeploymentUpdates) == 0 } -// PreemptedAllocs is used to store information about a set of allocations -// for the same job that get preempted as part of placing allocations for the -// job in the plan. - -// Preempted allocs represents a map from jobid to allocations -// to be preempted -type PreemptedAllocs map[*NamespacedID][]*Allocation - // PlanResult is the result of a plan submitted to the leader. type PlanResult struct { // NodeUpdate contains all the updates that were committed. @@ -8341,7 +8317,7 @@ type DesiredUpdates struct { InPlaceUpdate uint64 DestructiveUpdate uint64 Canary uint64 - Evict uint64 + Preemptions uint64 } func (d *DesiredUpdates) GoString() string { diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index dd1c25342f0..a2338300ee2 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -1869,56 +1869,73 @@ func TestResource_Add_Network(t *testing.T) { } } -func TestResource_Subtract(t *testing.T) { - r1 := &Resources{ - CPU: 2000, - MemoryMB: 2048, - DiskMB: 10000, - IOPS: 100, - Networks: []*NetworkResource{ - { - CIDR: "10.0.0.0/8", - MBits: 100, - ReservedPorts: []Port{{"ssh", 22}}, +func TestComparableResources_Subtract(t *testing.T) { + r1 := &ComparableResources{ + Flattened: AllocatedTaskResources{ + Cpu: AllocatedCpuResources{ + CpuShares: 2000, }, - }, - } - r2 := &Resources{ - CPU: 1000, - MemoryMB: 1024, - DiskMB: 5000, - IOPS: 50, - Networks: []*NetworkResource{ - { - IP: "10.0.0.1", - MBits: 20, - ReservedPorts: []Port{{"web", 80}}, + Memory: AllocatedMemoryResources{ + MemoryMB: 2048, + }, + Networks: []*NetworkResource{ + { + CIDR: "10.0.0.0/8", + MBits: 100, + ReservedPorts: []Port{{"ssh", 22}}, + }, }, }, + Shared: AllocatedSharedResources{ + DiskMB: 10000, + }, } - err := r1.Subtract(r2) - if err != nil { - t.Fatalf("Err: %v", err) + r2 := &ComparableResources{ + Flattened: AllocatedTaskResources{ + Cpu: AllocatedCpuResources{ + CpuShares: 1000, + }, + Memory: AllocatedMemoryResources{ + MemoryMB: 1024, + }, + Networks: []*NetworkResource{ + { + CIDR: "10.0.0.0/8", + MBits: 20, + ReservedPorts: []Port{{"ssh", 22}}, + }, + }, + }, + Shared: AllocatedSharedResources{ + DiskMB: 5000, + }, } + r1.Subtract(r2) - expect := &Resources{ - CPU: 1000, - MemoryMB: 1024, - DiskMB: 5000, - IOPS: 50, - Networks: []*NetworkResource{ - { - CIDR: "10.0.0.0/8", - MBits: 80, - ReservedPorts: []Port{{"ssh", 22}}, + expect := &ComparableResources{ + Flattened: AllocatedTaskResources{ + Cpu: AllocatedCpuResources{ + CpuShares: 1000, + }, + Memory: AllocatedMemoryResources{ + MemoryMB: 1024, }, + Networks: []*NetworkResource{ + { + CIDR: "10.0.0.0/8", + MBits: 80, + ReservedPorts: []Port{{"ssh", 22}}, + }, + }, + }, + Shared: AllocatedSharedResources{ + DiskMB: 5000, }, } - if !reflect.DeepEqual(expect.Networks, r1.Networks) { - t.Fatalf("bad: %#v %#v", expect, r1) - } + require := require.New(t) + require.Equal(expect, r1) } func TestEncodeDecode(t *testing.T) { diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 541678fd5db..93f982ede7d 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -130,8 +130,7 @@ func (s *GenericScheduler) Process(eval *structs.Evaluation) error { structs.EvalTriggerRollingUpdate, structs.EvalTriggerQueuedAllocs, structs.EvalTriggerPeriodicJob, structs.EvalTriggerMaxPlans, structs.EvalTriggerDeploymentWatcher, structs.EvalTriggerRetryFailedAlloc, - structs.EvalTriggerFailedFollowUp, - structs.EvalTriggerPreemption: + structs.EvalTriggerFailedFollowUp, structs.EvalTriggerPreemption: default: desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", eval.TriggeredBy) diff --git a/scheduler/rank.go b/scheduler/rank.go index 7e8ba45e576..5286fe3022f 100644 --- a/scheduler/rank.go +++ b/scheduler/rank.go @@ -234,17 +234,17 @@ OUTER: } // Look for preemptible allocations to satisfy the network resource for this task - preemptedAllocsForTaskNetwork := preemptForNetworkResourceAsk(iter.priority, proposed, ask, netIdx, currentPreemptions) - if preemptedAllocsForTaskNetwork == nil { + netPreemptions := preemptForNetworkResourceAsk(iter.priority, proposed, ask, netIdx, currentPreemptions) + if netPreemptions == nil { iter.ctx.Metrics().ExhaustedNode(option.Node, fmt.Sprintf("unable to meet network resource %v after preemption", ask)) netIdx.Release() continue OUTER } - allocsToPreempt = append(allocsToPreempt, preemptedAllocsForTaskNetwork...) + allocsToPreempt = append(allocsToPreempt, netPreemptions...) // First subtract out preempted allocations - proposed = structs.RemoveAllocs(proposed, preemptedAllocsForTaskNetwork) + proposed = structs.RemoveAllocs(proposed, netPreemptions) // Reset the network index and try the offer again netIdx.Release() @@ -254,8 +254,7 @@ OUTER: offer, err = netIdx.AssignNetwork(ask) if offer == nil { - iter.ctx.Metrics().ExhaustedNode(option.Node, - fmt.Sprintf("unexecpted error, unable to create offer after preempting:%v", err)) + iter.ctx.Logger().Error(fmt.Sprintf("unexpected error, unable to create offer after preempting:%v", err)) netIdx.Release() continue OUTER } @@ -275,6 +274,9 @@ OUTER: total.Tasks[task.Name] = taskResources } + // Store current set of running allocs before adding resources for the task group + current := proposed + // Add the resources we are trying to fit proposed = append(proposed, &structs.Allocation{AllocatedResources: total}) @@ -290,10 +292,9 @@ OUTER: // If eviction is enabled and the node doesn't fit the alloc, check if // any allocs can be preempted - // Remove the last element containing the current placement from proposed allocs - current := proposed[:len(proposed)-1] preemptForTaskGroup := findPreemptibleAllocationsForTaskGroup(iter.priority, current, total, option.Node, currentPreemptions) allocsToPreempt = append(allocsToPreempt, preemptForTaskGroup...) + // If we were unable to find preempted allocs to meet these requirements // mark as exhausted and continue if len(preemptForTaskGroup) == 0 { @@ -304,6 +305,7 @@ OUTER: if len(allocsToPreempt) > 0 { option.PreemptedAllocs = allocsToPreempt } + // Score the fit normally otherwise fitness := structs.ScoreFit(option.Node, util) normalizedFit := fitness / binPackingMaxFitScore diff --git a/scheduler/system_sched.go b/scheduler/system_sched.go index 912440aefef..e3f4015fbd1 100644 --- a/scheduler/system_sched.go +++ b/scheduler/system_sched.go @@ -347,7 +347,7 @@ func (s *SystemScheduler) computePlacements(place []allocTuple) error { alloc.PreviousAllocation = missing.Alloc.ID } - // If this placement involves preemption, set DesiredState to stop for those allocations + // If this placement involves preemption, set DesiredState to evict for those allocations if option.PreemptedAllocs != nil { var preemptedAllocIDs []string for _, stop := range option.PreemptedAllocs { @@ -358,7 +358,7 @@ func (s *SystemScheduler) computePlacements(place []allocTuple) error { s.plan.Annotations.PreemptedAllocs = append(s.plan.Annotations.PreemptedAllocs, stop.Stub()) if s.plan.Annotations.DesiredTGUpdates != nil { desired := s.plan.Annotations.DesiredTGUpdates[missing.TaskGroup.Name] - desired.Evict += 1 + desired.Preemptions += 1 } } } diff --git a/scheduler/testing.go b/scheduler/testing.go index 0a527ede973..d701436c5b5 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -121,9 +121,6 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er var preemptedAllocs []*structs.Allocation for _, preemptions := range result.NodePreemptions { for _, alloc := range preemptions { - if alloc.CreateTime == 0 { - alloc.CreateTime = now - } alloc.ModifyTime = now preemptedAllocs = append(preemptedAllocs, alloc) } From 4cc21fbcfa10063452c2721ebdaf0a7d811811f2 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 18 Oct 2018 00:07:56 -0500 Subject: [PATCH 23/38] more minor cleanup --- nomad/state/state_store_test.go | 1 + scheduler/rank.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 3fa78d14e0c..f0aa37aa1f7 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -274,6 +274,7 @@ func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { ID: preemptedAlloc.ID, Namespace: preemptedAlloc.Namespace, DesiredStatus: structs.AllocDesiredStatusEvict, + ModifyTime: time.Now().Unix(), DesiredDescription: fmt.Sprintf("Preempted by allocation %v", alloc.ID), } diff --git a/scheduler/rank.go b/scheduler/rank.go index 5286fe3022f..72a164d7da7 100644 --- a/scheduler/rank.go +++ b/scheduler/rank.go @@ -289,9 +289,9 @@ OUTER: iter.ctx.Metrics().ExhaustedNode(option.Node, dim) continue } + // If eviction is enabled and the node doesn't fit the alloc, check if // any allocs can be preempted - preemptForTaskGroup := findPreemptibleAllocationsForTaskGroup(iter.priority, current, total, option.Node, currentPreemptions) allocsToPreempt = append(allocsToPreempt, preemptForTaskGroup...) From 35635ba39a26ce3f01118f334ce31392933644a5 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Mon, 29 Oct 2018 12:34:03 -0500 Subject: [PATCH 24/38] refactor preemption code to use method recievers and setters for common fields --- scheduler/preemption.go | 455 ++++++++++++++++++++++------------------ scheduler/rank.go | 19 +- 2 files changed, 267 insertions(+), 207 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index d8c79e10914..bdd6428f0fe 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -12,89 +12,100 @@ import ( // number of allocations being preempted exceeds max_parallel value in the job's migrate stanza const maxParallelPenalty = 50.0 -// basicResourceDistance computes a distance using a coordinate system. It compares resource fields like CPU/Memory and Disk. -// Values emitted are in the range [0, maxFloat] -func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources) float64 { - memoryCoord, cpuCoord, diskMBCoord := 0.0, 0.0, 0.0 - if resourceAsk.Flattened.Memory.MemoryMB > 0 { - memoryCoord = (float64(resourceAsk.Flattened.Memory.MemoryMB) - float64(resourceUsed.Flattened.Memory.MemoryMB)) / float64(resourceAsk.Flattened.Memory.MemoryMB) - } - if resourceAsk.Flattened.Cpu.CpuShares > 0 { - cpuCoord = (float64(resourceAsk.Flattened.Cpu.CpuShares) - float64(resourceUsed.Flattened.Cpu.CpuShares)) / float64(resourceAsk.Flattened.Cpu.CpuShares) - } - if resourceAsk.Shared.DiskMB > 0 { - diskMBCoord = (float64(resourceAsk.Shared.DiskMB) - float64(resourceUsed.Shared.DiskMB)) / float64(resourceAsk.Shared.DiskMB) +type groupedAllocs struct { + priority int + allocs []*structs.Allocation +} + +// Preemptor is used to track existing allocations +// and find suitable allocations to preempt +type Preemptor struct { + // currentPreemptions is a map computed when SetPreemptions is called + // it tracks the number of preempted allocations per job/taskgroup + currentPreemptions map[structs.NamespacedID]map[string]int + + // jobPriority is the priority of the job being preempted + jobPriority int + + // nodeRemainingResources tracks remaining available resources on the node + nodeRemainingResources *structs.ComparableResources + + // currentAllocs is the candidate set used to find preemptible allocations + currentAllocs []*structs.Allocation +} + +func NewPreemptor(jobPriority int) *Preemptor { + return &Preemptor{ + currentPreemptions: make(map[structs.NamespacedID]map[string]int), + jobPriority: jobPriority, } - originDist := math.Sqrt( - math.Pow(memoryCoord, 2) + - math.Pow(cpuCoord, 2) + - math.Pow(diskMBCoord, 2)) - return originDist } -// networkResourceDistance returns a distance based only on network megabits -func networkResourceDistance(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource) float64 { - networkCoord := math.MaxFloat64 - if resourceUsed != nil && resourceNeeded != nil { - networkCoord = float64(resourceNeeded.MBits-resourceUsed.MBits) / float64(resourceNeeded.MBits) +// SetNode sets the node +func (p *Preemptor) SetNode(node *structs.Node) { + nodeRemainingResources := node.ComparableResources() + // Subtract the reserved resources of the node + if node.ComparableReservedResources() != nil { + nodeRemainingResources.Subtract(node.ComparableReservedResources()) } + p.nodeRemainingResources = nodeRemainingResources +} - originDist := math.Sqrt( - math.Pow(networkCoord, 2)) - return originDist +// SetCandidates initializes the candidate set from which preemptions are chosen +func (p *Preemptor) SetCandidates(allocs []*structs.Allocation) { + p.currentAllocs = allocs } -// getPreemptionScoreForTaskGroupResources is used to calculate a score (lower is better) based on the distance between -// the needed resource and requirements. A penalty is added when the choice already has some existing -// allocations in the plan that are being preempted. -func getPreemptionScoreForTaskGroupResources(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources, maxParallel int, numPreemptedAllocs int) float64 { - maxParallelScorePenalty := 0.0 - if maxParallel > 0 && numPreemptedAllocs >= maxParallel { - maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty +// SetPreemptions initializes a map tracking existing counts of preempted allocations +// per job/task group. This is used while scoring preemption options +func (p *Preemptor) SetPreemptions(allocs []*structs.Allocation) { + // Clear out existing values + for k := range p.currentPreemptions { + delete(p.currentPreemptions, k) } - return basicResourceDistance(resourceAsk, resourceUsed) + maxParallelScorePenalty -} -// getPreemptionScoreForNetwork is similar to getPreemptionScoreForTaskGroupResources -// but only uses network Mbits to calculate a preemption score -func getPreemptionScoreForNetwork(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource, maxParallel int, numPreemptedAllocs int) float64 { - if resourceUsed == nil || resourceNeeded == nil { - return math.MaxFloat64 + // Initialize counts + for _, alloc := range allocs { + id := structs.NamespacedID{alloc.JobID, alloc.Namespace} + countMap, ok := p.currentPreemptions[id] + if !ok { + countMap = make(map[string]int) + } + c := countMap[alloc.TaskGroup] + countMap[alloc.TaskGroup] = c + 1 + p.currentPreemptions[id] = countMap } - maxParallelScorePenalty := 0.0 - if maxParallel > 0 && numPreemptedAllocs >= maxParallel { - maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty +} + +// getNumPreemptions counts the number of other allocations being preempted that match the job and task group of +// the alloc under consideration. This is used as a scoring factor to minimize too many allocs of the same job being preempted at once +func (p *Preemptor) getNumPreemptions(alloc *structs.Allocation) int { + numCurrentPreemptionsForJob := 0 + countMap := p.currentPreemptions[structs.NamespacedID{alloc.JobID, alloc.Namespace}] + if countMap != nil { + numCurrentPreemptionsForJob = countMap[alloc.TaskGroup] } - return networkResourceDistance(resourceUsed, resourceNeeded) + maxParallelScorePenalty + return numCurrentPreemptionsForJob } -// findPreemptibleAllocationsForTaskGroup computes a list of allocations to preempt to accommodate +// preemptForTaskGroup computes a list of allocations to preempt to accommodate // the resources asked for. Only allocs with a job priority < 10 of jobPriority are considered // This method is meant only for finding preemptible allocations based on CPU/Memory/Disk -func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs.Allocation, resourceAsk *structs.AllocatedResources, node *structs.Node, currentPreemptions []*structs.Allocation) []*structs.Allocation { +func (p *Preemptor) preemptForTaskGroup(resourceAsk *structs.AllocatedResources) []*structs.Allocation { resourcesNeeded := resourceAsk.Comparable() - allocsByPriority := filterAndGroupPreemptibleAllocs(jobPriority, current) - var bestAllocs []*structs.Allocation - allRequirementsMet := false - var preemptedResources *structs.ComparableResources - - //TODO(preetha): should add some debug logging - - nodeRemainingResources := node.ComparableResources() - - // Initialize nodeRemainingResources with the remaining resources - // after accounting for reserved resources and all allocations - - // Subtract the reserved resources of the node - if node.ComparableReservedResources() != nil { - nodeRemainingResources.Subtract(node.ComparableReservedResources()) - } // Subtract current allocations - for _, alloc := range current { - nodeRemainingResources.Subtract(alloc.ComparableResources()) + for _, alloc := range p.currentAllocs { + p.nodeRemainingResources.Subtract(alloc.ComparableResources()) } + // Group candidates by priority, filter out ineligible allocs + allocsByPriority := filterAndGroupPreemptibleAllocs(p.jobPriority, p.currentAllocs) + + var bestAllocs []*structs.Allocation + allRequirementsMet := false + var preemptedResources *structs.ComparableResources + // Iterate over allocations grouped by priority to find preemptible allocations for _, allocGrp := range allocsByPriority { for len(allocGrp.allocs) > 0 && !allRequirementsMet { @@ -102,13 +113,13 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. bestDistance := math.MaxFloat64 // find the alloc with the closest distance for index, alloc := range allocGrp.allocs { - currentPreemptionCount := computeCurrentPreemptions(alloc, currentPreemptions) + currentPreemptionCount := p.getNumPreemptions(alloc) maxParallel := 0 tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) if tg != nil && tg.Migrate != nil { maxParallel = tg.Migrate.MaxParallel } - distance := getPreemptionScoreForTaskGroupResources(resourcesNeeded, alloc.ComparableResources(), maxParallel, currentPreemptionCount) + distance := scoreForTaskGroup(resourcesNeeded, alloc.ComparableResources(), maxParallel, currentPreemptionCount) if distance < bestDistance { bestDistance = distance closestAllocIndex = index @@ -122,7 +133,7 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. preemptedResources.Add(closestAlloc.ComparableResources()) } availableResources := preemptedResources.Copy() - availableResources.Add(nodeRemainingResources) + availableResources.Add(p.nodeRemainingResources) // This step needs the original resources asked for as the second arg, can't use the running total allRequirementsMet, _ = availableResources.Superset(resourceAsk.Comparable()) @@ -156,141 +167,18 @@ func findPreemptibleAllocationsForTaskGroup(jobPriority int, current []*structs. return distance1 > distance2 }) - filteredBestAllocs := eliminateSuperSetAllocationsForTaskGroup(bestAllocs, nodeRemainingResources, resourcesNeeded) + filteredBestAllocs := filterSupersetTaskGroup(bestAllocs, p.nodeRemainingResources, resourcesNeeded) return filteredBestAllocs } -// computeCurrentPreemptions counts the number of other allocations being preempted that match the job and task group of -// the alloc under consideration. This is used as a scoring factor to minimize too many allocs of the same job being preempted at once -func computeCurrentPreemptions(currentAlloc *structs.Allocation, currentPreemptions []*structs.Allocation) int { - numCurrentPreemptionsForJob := 0 - for _, alloc := range currentPreemptions { - if alloc.JobID == currentAlloc.JobID && alloc.Namespace == currentAlloc.Namespace && alloc.TaskGroup == currentAlloc.TaskGroup { - numCurrentPreemptionsForJob++ - } - } - return numCurrentPreemptionsForJob -} - -// meetsNetworkRequirements checks if the first resource meets or exceeds the second resource's network MBits requirements -func meetsNetworkRequirements(firstMbits int, secondMbits int) bool { - if firstMbits == 0 || secondMbits == 0 { - return false - } - return firstMbits >= secondMbits -} - -type groupedAllocs struct { - priority int - allocs []*structs.Allocation -} - -// filterAndGroupPreemptibleAllocs groups allocations by priority after removing any from jobs of -// a higher priority than jobPriority -func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocation) []*groupedAllocs { - allocsByPriority := make(map[int][]*structs.Allocation) - for _, alloc := range current { - // Why is alloc.Job even nil though? - if alloc.Job == nil { - continue - } - - // Skip allocs whose priority is within a delta of 10 - // This also skips any allocs of the current job - // for which we are attempting preemption - if jobPriority-alloc.Job.Priority < 10 { - continue - } - grpAllocs, ok := allocsByPriority[alloc.Job.Priority] - if !ok { - grpAllocs = make([]*structs.Allocation, 0) - } - grpAllocs = append(grpAllocs, alloc) - allocsByPriority[alloc.Job.Priority] = grpAllocs - } - - var groupedSortedAllocs []*groupedAllocs - for priority, allocs := range allocsByPriority { - groupedSortedAllocs = append(groupedSortedAllocs, &groupedAllocs{ - priority: priority, - allocs: allocs}) - } - - // Sort by priority - sort.Slice(groupedSortedAllocs, func(i, j int) bool { - return groupedSortedAllocs[i].priority < groupedSortedAllocs[j].priority - }) - - return groupedSortedAllocs -} - -// eliminateSuperSetAllocationsForTaskGroup is used as a final step to remove -// any allocations that meet a superset of requirements from the set of allocations -// to preempt -func eliminateSuperSetAllocationsForTaskGroup(bestAllocs []*structs.Allocation, - nodeRemainingResources *structs.ComparableResources, - resourceAsk *structs.ComparableResources) []*structs.Allocation { - - var preemptedResources *structs.ComparableResources - var filteredBestAllocs []*structs.Allocation - - // Do another pass to eliminate allocations that are a superset of other allocations - // in the preemption set - for _, alloc := range bestAllocs { - if preemptedResources == nil { - preemptedResources = alloc.ComparableResources().Copy() - } else { - preemptedResources.Add(alloc.ComparableResources().Copy()) - } - filteredBestAllocs = append(filteredBestAllocs, alloc) - availableResources := preemptedResources.Copy() - availableResources.Add(nodeRemainingResources) - - requirementsMet, _ := availableResources.Superset(resourceAsk) - if requirementsMet { - break - } - } - return filteredBestAllocs -} - -// eliminateSuperSetAllocationsForNetwork is similar to eliminateSuperSetAllocationsForTaskGroup but only -// considers network Mbits -func eliminateSuperSetAllocationsForNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk *structs.NetworkResource, - nodeRemainingResources *structs.ComparableResources) []*structs.Allocation { - - var preemptedResources *structs.ComparableResources - var filteredBestAllocs []*structs.Allocation - - // Do another pass to eliminate allocations that are a superset of other allocations - // in the preemption set - for _, alloc := range bestAllocs { - if preemptedResources == nil { - preemptedResources = alloc.ComparableResources().Copy() - } else { - preemptedResources.Add(alloc.ComparableResources().Copy()) - } - filteredBestAllocs = append(filteredBestAllocs, alloc) - availableResources := preemptedResources.Copy() - availableResources.Add(nodeRemainingResources) - - requirementsMet := meetsNetworkRequirements(availableResources.Flattened.Networks[0].MBits, networkResourcesAsk.MBits) - if requirementsMet { - break - } - } - return filteredBestAllocs -} - -// preemptForNetworkResourceAsk tries to find allocations to preempt to meet network resources. +// preemptForNetwork tries to find allocations to preempt to meet network resources. // This is called once per task when assigning a network to the task. While finding allocations // to preempt, this only considers allocations that share the same network device -func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allocation, networkResourceAsk *structs.NetworkResource, - netIdx *structs.NetworkIndex, currentPreemptions []*structs.Allocation) []*structs.Allocation { +func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResource, netIdx *structs.NetworkIndex) []*structs.Allocation { // Early return if there are no current allocs - if len(currentAllocs) == 0 { + if len(p.currentAllocs) == 0 { return nil } @@ -304,12 +192,12 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo // // This step also filters out high priority allocations and allocations // that are not using any network resources - for _, alloc := range currentAllocs { + for _, alloc := range p.currentAllocs { if alloc.Job == nil { continue } - if jobPriority-alloc.Job.Priority < 10 { + if p.jobPriority-alloc.Job.Priority < 10 { continue } networks := alloc.CompatibleNetworkResources() @@ -379,7 +267,7 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo } // Split by priority - allocsByPriority := filterAndGroupPreemptibleAllocs(jobPriority, currentAllocs) + allocsByPriority := filterAndGroupPreemptibleAllocs(p.jobPriority, currentAllocs) for _, allocsGrp := range allocsByPriority { allocs := allocsGrp.allocs @@ -388,7 +276,7 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo // as well as penalty for preempting an allocation // whose task group already has existing preemptions sort.Slice(allocs, func(i, j int) bool { - return distanceComparatorForNetwork(allocs, currentPreemptions, networkResourceAsk, i, j) + return p.distanceComparatorForNetwork(allocs, networkResourceAsk, i, j) }) for _, alloc := range allocs { @@ -452,16 +340,177 @@ func preemptForNetworkResourceAsk(jobPriority int, currentAllocs []*structs.Allo }) // Do a final pass to eliminate any superset allocations - filteredBestAllocs := eliminateSuperSetAllocationsForNetwork(allocsToPreempt, networkResourceAsk, nodeRemainingResources) + filteredBestAllocs := filterSupersetNetwork(allocsToPreempt, networkResourceAsk, nodeRemainingResources) + return filteredBestAllocs +} + +// basicResourceDistance computes a distance using a coordinate system. It compares resource fields like CPU/Memory and Disk. +// Values emitted are in the range [0, maxFloat] +func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources) float64 { + memoryCoord, cpuCoord, diskMBCoord := 0.0, 0.0, 0.0 + if resourceAsk.Flattened.Memory.MemoryMB > 0 { + memoryCoord = (float64(resourceAsk.Flattened.Memory.MemoryMB) - float64(resourceUsed.Flattened.Memory.MemoryMB)) / float64(resourceAsk.Flattened.Memory.MemoryMB) + } + if resourceAsk.Flattened.Cpu.CpuShares > 0 { + cpuCoord = (float64(resourceAsk.Flattened.Cpu.CpuShares) - float64(resourceUsed.Flattened.Cpu.CpuShares)) / float64(resourceAsk.Flattened.Cpu.CpuShares) + } + if resourceAsk.Shared.DiskMB > 0 { + diskMBCoord = (float64(resourceAsk.Shared.DiskMB) - float64(resourceUsed.Shared.DiskMB)) / float64(resourceAsk.Shared.DiskMB) + } + originDist := math.Sqrt( + math.Pow(memoryCoord, 2) + + math.Pow(cpuCoord, 2) + + math.Pow(diskMBCoord, 2)) + return originDist +} + +// networkResourceDistance returns a distance based only on network megabits +func networkResourceDistance(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource) float64 { + networkCoord := math.MaxFloat64 + if resourceUsed != nil && resourceNeeded != nil { + networkCoord = float64(resourceNeeded.MBits-resourceUsed.MBits) / float64(resourceNeeded.MBits) + } + + originDist := math.Sqrt( + math.Pow(networkCoord, 2)) + return originDist +} + +// scoreForTaskGroup is used to calculate a score (lower is better) based on the distance between +// the needed resource and requirements. A penalty is added when the choice already has some existing +// allocations in the plan that are being preempted. +func scoreForTaskGroup(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources, maxParallel int, numPreemptedAllocs int) float64 { + maxParallelScorePenalty := 0.0 + if maxParallel > 0 && numPreemptedAllocs >= maxParallel { + maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty + } + return basicResourceDistance(resourceAsk, resourceUsed) + maxParallelScorePenalty +} + +// scoreForNetwork is similar to scoreForTaskGroup +// but only uses network Mbits to calculate a preemption score +func scoreForNetwork(resourceUsed *structs.NetworkResource, resourceNeeded *structs.NetworkResource, maxParallel int, numPreemptedAllocs int) float64 { + if resourceUsed == nil || resourceNeeded == nil { + return math.MaxFloat64 + } + maxParallelScorePenalty := 0.0 + if maxParallel > 0 && numPreemptedAllocs >= maxParallel { + maxParallelScorePenalty = float64((numPreemptedAllocs+1)-maxParallel) * maxParallelPenalty + } + return networkResourceDistance(resourceUsed, resourceNeeded) + maxParallelScorePenalty +} + +// meetsNetworkRequirements checks if the first resource meets or exceeds the second resource's network MBits requirements +func meetsNetworkRequirements(firstMbits int, secondMbits int) bool { + if firstMbits == 0 || secondMbits == 0 { + return false + } + return firstMbits >= secondMbits +} + +// filterAndGroupPreemptibleAllocs groups allocations by priority after removing any from jobs of +// a higher priority than jobPriority +func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocation) []*groupedAllocs { + allocsByPriority := make(map[int][]*structs.Allocation) + for _, alloc := range current { + // Why is alloc.Job even nil though? + if alloc.Job == nil { + continue + } + + // Skip allocs whose priority is within a delta of 10 + // This also skips any allocs of the current job + // for which we are attempting preemption + if jobPriority-alloc.Job.Priority < 10 { + continue + } + grpAllocs, ok := allocsByPriority[alloc.Job.Priority] + if !ok { + grpAllocs = make([]*structs.Allocation, 0) + } + grpAllocs = append(grpAllocs, alloc) + allocsByPriority[alloc.Job.Priority] = grpAllocs + } + + var groupedSortedAllocs []*groupedAllocs + for priority, allocs := range allocsByPriority { + groupedSortedAllocs = append(groupedSortedAllocs, &groupedAllocs{ + priority: priority, + allocs: allocs}) + } + + // Sort by priority + sort.Slice(groupedSortedAllocs, func(i, j int) bool { + return groupedSortedAllocs[i].priority < groupedSortedAllocs[j].priority + }) + + return groupedSortedAllocs +} + +// filterSupersetTaskGroup is used as a final step to remove +// any allocations that meet a superset of requirements from +// the set of allocations to preempt +func filterSupersetTaskGroup(bestAllocs []*structs.Allocation, + nodeRemainingResources *structs.ComparableResources, + resourceAsk *structs.ComparableResources) []*structs.Allocation { + + var preemptedResources *structs.ComparableResources + var filteredBestAllocs []*structs.Allocation + + // Do another pass to eliminate allocations that are a superset of other allocations + // in the preemption set + for _, alloc := range bestAllocs { + if preemptedResources == nil { + preemptedResources = alloc.ComparableResources().Copy() + } else { + preemptedResources.Add(alloc.ComparableResources().Copy()) + } + filteredBestAllocs = append(filteredBestAllocs, alloc) + availableResources := preemptedResources.Copy() + availableResources.Add(nodeRemainingResources) + + requirementsMet, _ := availableResources.Superset(resourceAsk) + if requirementsMet { + break + } + } + return filteredBestAllocs +} + +// filterSupersetNetwork is similar to filterSupersetTaskGroup but only +// considers network Mbits +func filterSupersetNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk *structs.NetworkResource, + nodeRemainingResources *structs.ComparableResources) []*structs.Allocation { + + var preemptedResources *structs.ComparableResources + var filteredBestAllocs []*structs.Allocation + + // Do another pass to eliminate allocations that are a superset of other allocations + // in the preemption set + for _, alloc := range bestAllocs { + if preemptedResources == nil { + preemptedResources = alloc.ComparableResources().Copy() + } else { + preemptedResources.Add(alloc.ComparableResources().Copy()) + } + filteredBestAllocs = append(filteredBestAllocs, alloc) + availableResources := preemptedResources.Copy() + availableResources.Add(nodeRemainingResources) + + requirementsMet := meetsNetworkRequirements(availableResources.Flattened.Networks[0].MBits, networkResourcesAsk.MBits) + if requirementsMet { + break + } + } return filteredBestAllocs } // distanceComparatorForNetwork is used as the sorting function when finding allocations to preempt. It uses // both a coordinayte distance function based on Mbits needed, and a penalty if the allocation under consideration // belongs to a job that already has more preempted allocations -func distanceComparatorForNetwork(allocs []*structs.Allocation, currentPreemptions []*structs.Allocation, networkResourceAsk *structs.NetworkResource, i int, j int) bool { +func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, networkResourceAsk *structs.NetworkResource, i int, j int) bool { firstAlloc := allocs[i] - currentPreemptionCount1 := computeCurrentPreemptions(firstAlloc, currentPreemptions) + currentPreemptionCount1 := p.getNumPreemptions(firstAlloc) // Look up configured maxParallel value for these allocation's task groups var maxParallel1, maxParallel2 int tg1 := allocs[i].Job.LookupTaskGroup(firstAlloc.TaskGroup) @@ -475,10 +524,10 @@ func distanceComparatorForNetwork(allocs []*structs.Allocation, currentPreemptio firstAllocNetResourceUsed = firstAllocNetworks[0] } - distance1 := getPreemptionScoreForNetwork(firstAllocNetResourceUsed, networkResourceAsk, maxParallel1, currentPreemptionCount1) + distance1 := scoreForNetwork(firstAllocNetResourceUsed, networkResourceAsk, maxParallel1, currentPreemptionCount1) secondAlloc := allocs[j] - currentPreemptionCount2 := computeCurrentPreemptions(secondAlloc, currentPreemptions) + currentPreemptionCount2 := p.getNumPreemptions(secondAlloc) tg2 := secondAlloc.Job.LookupTaskGroup(secondAlloc.TaskGroup) if tg2 != nil && tg2.Migrate != nil { maxParallel2 = tg2.Migrate.MaxParallel @@ -490,6 +539,6 @@ func distanceComparatorForNetwork(allocs []*structs.Allocation, currentPreemptio secondAllocNetResourceUsed = secondAllocNetworks[0] } - distance2 := getPreemptionScoreForNetwork(secondAllocNetResourceUsed, networkResourceAsk, maxParallel2, currentPreemptionCount2) + distance2 := scoreForNetwork(secondAllocNetResourceUsed, networkResourceAsk, maxParallel2, currentPreemptionCount2) return distance1 < distance2 } diff --git a/scheduler/rank.go b/scheduler/rank.go index 72a164d7da7..47cb8a89c8b 100644 --- a/scheduler/rank.go +++ b/scheduler/rank.go @@ -202,12 +202,17 @@ OUTER: var allocsToPreempt []*structs.Allocation + // Initialize preemptor with node + preemptor := NewPreemptor(iter.priority) + preemptor.SetNode(option.Node) + // Count the number of existing preemptions allPreemptions := iter.ctx.Plan().NodePreemptions var currentPreemptions []*structs.Allocation for _, allocs := range allPreemptions { currentPreemptions = append(currentPreemptions, allocs...) } + preemptor.SetPreemptions(currentPreemptions) for _, task := range iter.taskGroup.Tasks { // Allocate the resources @@ -234,7 +239,9 @@ OUTER: } // Look for preemptible allocations to satisfy the network resource for this task - netPreemptions := preemptForNetworkResourceAsk(iter.priority, proposed, ask, netIdx, currentPreemptions) + preemptor.SetCandidates(proposed) + + netPreemptions := preemptor.preemptForNetwork(ask, netIdx) if netPreemptions == nil { iter.ctx.Metrics().ExhaustedNode(option.Node, fmt.Sprintf("unable to meet network resource %v after preemption", ask)) @@ -292,12 +299,16 @@ OUTER: // If eviction is enabled and the node doesn't fit the alloc, check if // any allocs can be preempted - preemptForTaskGroup := findPreemptibleAllocationsForTaskGroup(iter.priority, current, total, option.Node, currentPreemptions) - allocsToPreempt = append(allocsToPreempt, preemptForTaskGroup...) + + // Initialize preemptor with candidate set + preemptor.SetCandidates(current) + + preemptedAllocs := preemptor.preemptForTaskGroup(total) + allocsToPreempt = append(allocsToPreempt, preemptedAllocs...) // If we were unable to find preempted allocs to meet these requirements // mark as exhausted and continue - if len(preemptForTaskGroup) == 0 { + if len(preemptedAllocs) == 0 { iter.ctx.Metrics().ExhaustedNode(option.Node, dim) continue } From 88005852e3c2b0985f6b74bba40fcd062d131996 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Mon, 29 Oct 2018 13:10:43 -0500 Subject: [PATCH 25/38] Introduce a response object for scheduler configuration --- api/operator.go | 14 ++++++++++++-- api/operator_test.go | 10 +++++----- command/agent/operator_endpoint.go | 12 +++++++++--- command/agent/operator_endpoint_test.go | 4 ++-- nomad/operator_endpoint.go | 9 +++++++-- nomad/state/schema.go | 1 + nomad/structs/operator.go | 10 ++++++++++ 7 files changed, 46 insertions(+), 14 deletions(-) diff --git a/api/operator.go b/api/operator.go index 298c8bf21f9..cdef67cf811 100644 --- a/api/operator.go +++ b/api/operator.go @@ -119,14 +119,24 @@ type SchedulerConfiguration struct { ModifyIndex uint64 } +// SchedulerConfigurationResponse is the response object that wraps SchedulerConfiguration +type SchedulerConfigurationResponse struct { + // SchedulerConfig contains scheduler config options + SchedulerConfig SchedulerConfiguration + + // CreateIndex/ModifyIndex store the create/modify indexes of this configuration. + CreateIndex uint64 + ModifyIndex uint64 +} + // PreemptionConfig specifies whether preemption is enabled based on scheduler type type PreemptionConfig struct { SystemSchedulerEnabled bool } // SchedulerGetConfiguration is used to query the current Scheduler configuration. -func (op *Operator) SchedulerGetConfiguration(q *QueryOptions) (*SchedulerConfiguration, *QueryMeta, error) { - var resp SchedulerConfiguration +func (op *Operator) SchedulerGetConfiguration(q *QueryOptions) (*SchedulerConfigurationResponse, *QueryMeta, error) { + var resp SchedulerConfigurationResponse qm, err := op.c.query("/v1/operator/scheduler/config", &resp, q) if err != nil { return nil, nil, err diff --git a/api/operator_test.go b/api/operator_test.go index 25dcf44a8ee..c21cafd7d66 100644 --- a/api/operator_test.go +++ b/api/operator_test.go @@ -62,13 +62,13 @@ func TestAPI_OperatorSchedulerGetSetConfiguration(t *testing.T) { defer s.Stop() operator := c.Operator() - var config *SchedulerConfiguration + var config *SchedulerConfigurationResponse retry.Run(t, func(r *retry.R) { var err error config, _, err = operator.SchedulerGetConfiguration(nil) r.Check(err) }) - require.True(config.PreemptionConfig.SystemSchedulerEnabled) + require.True(config.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) // Change a config setting newConf := &SchedulerConfiguration{PreemptionConfig: PreemptionConfig{SystemSchedulerEnabled: false}} @@ -77,7 +77,7 @@ func TestAPI_OperatorSchedulerGetSetConfiguration(t *testing.T) { config, _, err = operator.SchedulerGetConfiguration(nil) require.Nil(err) - require.False(config.PreemptionConfig.SystemSchedulerEnabled) + require.False(config.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) } func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { @@ -87,13 +87,13 @@ func TestAPI_OperatorSchedulerCASConfiguration(t *testing.T) { defer s.Stop() operator := c.Operator() - var config *SchedulerConfiguration + var config *SchedulerConfigurationResponse retry.Run(t, func(r *retry.R) { var err error config, _, err = operator.SchedulerGetConfiguration(nil) r.Check(err) }) - require.True(config.PreemptionConfig.SystemSchedulerEnabled) + require.True(config.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) // Pass an invalid ModifyIndex { diff --git a/command/agent/operator_endpoint.go b/command/agent/operator_endpoint.go index af62f06d206..ec0ca85910a 100644 --- a/command/agent/operator_endpoint.go +++ b/command/agent/operator_endpoint.go @@ -220,18 +220,24 @@ func (s *HTTPServer) OperatorSchedulerConfiguration(resp http.ResponseWriter, re return nil, nil } - var reply structs.SchedulerConfiguration + var reply structs.SchedulerConfigurationResponse if err := s.agent.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { return nil, err } out := api.SchedulerConfiguration{ - PreemptionConfig: api.PreemptionConfig{SystemSchedulerEnabled: reply.PreemptionConfig.SystemSchedulerEnabled}, + PreemptionConfig: api.PreemptionConfig{SystemSchedulerEnabled: reply.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled}, CreateIndex: reply.CreateIndex, ModifyIndex: reply.ModifyIndex, } - return out, nil + resp := api.SchedulerConfigurationResponse{ + SchedulerConfig: out, + CreateIndex: out.CreateIndex, + ModifyIndex: out.ModifyIndex, + } + + return resp, nil case "PUT": var args structs.SchedulerSetConfigRequest diff --git a/command/agent/operator_endpoint_test.go b/command/agent/operator_endpoint_test.go index 234aec05891..ee424fb9f87 100644 --- a/command/agent/operator_endpoint_test.go +++ b/command/agent/operator_endpoint_test.go @@ -269,9 +269,9 @@ func TestOperator_SchedulerGetConfiguration(t *testing.T) { obj, err := s.Server.OperatorSchedulerConfiguration(resp, req) require.Nil(err) require.Equal(200, resp.Code) - out, ok := obj.(api.SchedulerConfiguration) + out, ok := obj.(api.SchedulerConfigurationResponse) require.True(ok) - require.True(out.PreemptionConfig.SystemSchedulerEnabled) + require.True(out.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) }) } diff --git a/nomad/operator_endpoint.go b/nomad/operator_endpoint.go index 01aed37416d..504bcfcaca3 100644 --- a/nomad/operator_endpoint.go +++ b/nomad/operator_endpoint.go @@ -316,7 +316,7 @@ func (op *Operator) SchedulerSetConfiguration(args *structs.SchedulerSetConfigRe } // SchedulerGetConfiguration is used to retrieve the current Scheduler configuration. -func (op *Operator) SchedulerGetConfiguration(args *structs.GenericRequest, reply *structs.SchedulerConfiguration) error { +func (op *Operator) SchedulerGetConfiguration(args *structs.GenericRequest, reply *structs.SchedulerConfigurationResponse) error { if done, err := op.srv.forward("Operator.SchedulerGetConfiguration", args, args, reply); done { return err } @@ -337,7 +337,12 @@ func (op *Operator) SchedulerGetConfiguration(args *structs.GenericRequest, repl return fmt.Errorf("scheduler config not initialized yet") } - *reply = *config + resp := &structs.SchedulerConfigurationResponse{ + SchedulerConfig: *config, + CreateIndex: config.CreateIndex, + ModifyIndex: config.ModifyIndex, + } + *reply = *resp return nil } diff --git a/nomad/state/schema.go b/nomad/state/schema.go index 7c76c035920..53df292dc22 100644 --- a/nomad/state/schema.go +++ b/nomad/state/schema.go @@ -611,6 +611,7 @@ func schedulerConfigTableSchema() *memdb.TableSchema { Name: "id", AllowMissing: true, Unique: true, + // This indexer ensures that this table is a singleton Indexer: &memdb.ConditionalIndex{ Conditional: func(obj interface{}) (bool, error) { return true, nil }, }, diff --git a/nomad/structs/operator.go b/nomad/structs/operator.go index 97809726eee..059399d9184 100644 --- a/nomad/structs/operator.go +++ b/nomad/structs/operator.go @@ -131,6 +131,16 @@ type SchedulerConfiguration struct { ModifyIndex uint64 } +// SchedulerConfigurationResponse is the response object that wraps SchedulerConfiguration +type SchedulerConfigurationResponse struct { + // SchedulerConfig contains scheduler config options + SchedulerConfig SchedulerConfiguration + + // CreateIndex/ModifyIndex store the create/modify indexes of this configuration. + CreateIndex uint64 + ModifyIndex uint64 +} + // PreemptionConfig specifies whether preemption is enabled based on scheduler type type PreemptionConfig struct { // SystemSchedulerEnabled specifies if preemption is enabled for system jobs From 17344a7f3a4eb525f96ffa193cadd2d16d1aad8c Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 30 Oct 2018 09:14:56 -0500 Subject: [PATCH 26/38] Introduce interface with multiple implementations for resource distance --- nomad/structs/structs.go | 1 + scheduler/preemption.go | 208 ++++++++++++++++++++++----------------- scheduler/rank.go | 4 +- 3 files changed, 121 insertions(+), 92 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 4e7269410f1..8e665103bea 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -7403,6 +7403,7 @@ func (a *Allocation) ComparableResources() *ComparableResources { Memory: AllocatedMemoryResources{ MemoryMB: int64(resources.MemoryMB), }, + Networks: resources.Networks, }, Shared: AllocatedSharedResources{ DiskMB: int64(resources.DiskMB), diff --git a/scheduler/preemption.go b/scheduler/preemption.go index bdd6428f0fe..6ecf49cd828 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -17,6 +17,82 @@ type groupedAllocs struct { allocs []*structs.Allocation } +// PreemptionResource interface is implemented by different +// types of resources. +type PreemptionResource interface { + // MeetsRequirements returns true if the available resources match needed resources + MeetsRequirements() bool + + // Distance returns values in the range [0, MaxFloat], lower is better + Distance() float64 +} + +// NetworkPreemptionResource implements PreemptionResource for network assignments +// It only looks at MBits needed +type NetworkPreemptionResource struct { + availableResources *structs.NetworkResource + resourceNeeded *structs.NetworkResource +} + +func (n *NetworkPreemptionResource) MeetsRequirements() bool { + mbitsAvailable := n.availableResources.MBits + mbitsNeeded := n.resourceNeeded.MBits + if mbitsAvailable == 0 || mbitsNeeded == 0 { + return false + } + return mbitsAvailable >= mbitsNeeded +} + +func (n *NetworkPreemptionResource) Distance() float64 { + networkCoord := math.MaxFloat64 + if n.availableResources != nil && n.resourceNeeded != nil { + networkCoord = float64(n.resourceNeeded.MBits-n.availableResources.MBits) / float64(n.resourceNeeded.MBits) + } + + originDist := math.Sqrt( + math.Pow(networkCoord, 2)) + return originDist +} + +// BasePreemptionResource implements PreemptionResource for CPU/Memory/Disk +type BasePreemptionResource struct { + availableResources *structs.ComparableResources + resourceNeeded *structs.ComparableResources +} + +func (b *BasePreemptionResource) MeetsRequirements() bool { + super, _ := b.availableResources.Superset(b.resourceNeeded) + return super +} + +func (b *BasePreemptionResource) Distance() float64 { + return basicResourceDistance(b.resourceNeeded, b.availableResources) +} + +// PreemptionResourceFactory returns a new PreemptionResource +type PreemptionResourceFactory func(availableResources *structs.ComparableResources, resourceAsk *structs.ComparableResources) PreemptionResource + +// GetNetworkPreemptionResourceFactory returns a preemption resource factory for network assignments +func GetNetworkPreemptionResourceFactory() PreemptionResourceFactory { + return func(availableResources *structs.ComparableResources, resourceNeeded *structs.ComparableResources) PreemptionResource { + available := availableResources.Flattened.Networks[0] + return &NetworkPreemptionResource{ + availableResources: available, + resourceNeeded: resourceNeeded.Flattened.Networks[0], + } + } +} + +// GetBasePreemptionResourceFactory returns a preemption resource factory for CPU/Memory/Disk +func GetBasePreemptionResourceFactory() PreemptionResourceFactory { + return func(availableResources *structs.ComparableResources, resourceNeeded *structs.ComparableResources) PreemptionResource { + return &BasePreemptionResource{ + availableResources: availableResources, + resourceNeeded: resourceNeeded, + } + } +} + // Preemptor is used to track existing allocations // and find suitable allocations to preempt type Preemptor struct { @@ -59,7 +135,7 @@ func (p *Preemptor) SetCandidates(allocs []*structs.Allocation) { // SetPreemptions initializes a map tracking existing counts of preempted allocations // per job/task group. This is used while scoring preemption options func (p *Preemptor) SetPreemptions(allocs []*structs.Allocation) { - // Clear out existing values + // Clear out existing values since this can be called more than once for k := range p.currentPreemptions { delete(p.currentPreemptions, k) } @@ -88,10 +164,10 @@ func (p *Preemptor) getNumPreemptions(alloc *structs.Allocation) int { return numCurrentPreemptionsForJob } -// preemptForTaskGroup computes a list of allocations to preempt to accommodate +// PreemptForTaskGroup computes a list of allocations to preempt to accommodate // the resources asked for. Only allocs with a job priority < 10 of jobPriority are considered // This method is meant only for finding preemptible allocations based on CPU/Memory/Disk -func (p *Preemptor) preemptForTaskGroup(resourceAsk *structs.AllocatedResources) []*structs.Allocation { +func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) []*structs.Allocation { resourcesNeeded := resourceAsk.Comparable() // Subtract current allocations @@ -111,7 +187,7 @@ func (p *Preemptor) preemptForTaskGroup(resourceAsk *structs.AllocatedResources) for len(allocGrp.allocs) > 0 && !allRequirementsMet { closestAllocIndex := -1 bestDistance := math.MaxFloat64 - // find the alloc with the closest distance + // Find the alloc with the closest distance for index, alloc := range allocGrp.allocs { currentPreemptionCount := p.getNumPreemptions(alloc) maxParallel := 0 @@ -143,7 +219,7 @@ func (p *Preemptor) preemptForTaskGroup(resourceAsk *structs.AllocatedResources) allocGrp.allocs[closestAllocIndex] = allocGrp.allocs[len(allocGrp.allocs)-1] allocGrp.allocs = allocGrp.allocs[:len(allocGrp.allocs)-1] - // this is the remaining total of resources needed + // This is the remaining total of resources needed resourcesNeeded.Subtract(closestAlloc.ComparableResources()) } if allRequirementsMet { @@ -159,23 +235,17 @@ func (p *Preemptor) preemptForTaskGroup(resourceAsk *structs.AllocatedResources) resourcesNeeded = resourceAsk.Comparable() // We do another pass to eliminate unnecessary preemptions // This filters out allocs whose resources are already covered by another alloc + basePreemptionResource := GetBasePreemptionResourceFactory() - // Sort bestAllocs by distance descending (without penalty) - sort.Slice(bestAllocs, func(i, j int) bool { - distance1 := basicResourceDistance(resourcesNeeded, bestAllocs[i].ComparableResources()) - distance2 := basicResourceDistance(resourcesNeeded, bestAllocs[j].ComparableResources()) - return distance1 > distance2 - }) - - filteredBestAllocs := filterSupersetTaskGroup(bestAllocs, p.nodeRemainingResources, resourcesNeeded) + filteredBestAllocs := filterSuperset(bestAllocs, p.nodeRemainingResources, resourcesNeeded, basePreemptionResource) return filteredBestAllocs } -// preemptForNetwork tries to find allocations to preempt to meet network resources. +// PreemptForNetwork tries to find allocations to preempt to meet network resources. // This is called once per task when assigning a network to the task. While finding allocations // to preempt, this only considers allocations that share the same network device -func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResource, netIdx *structs.NetworkIndex) []*structs.Allocation { +func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResource, netIdx *structs.NetworkIndex) []*structs.Allocation { // Early return if there are no current allocs if len(p.currentAllocs) == 0 { @@ -189,7 +259,7 @@ func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResourc // Create a map from each device to allocs // We do this because to place a task we have to be able to // preempt allocations that are using the same device. - // + // This step also filters out high priority allocations and allocations // that are not using any network resources for _, alloc := range p.currentAllocs { @@ -200,7 +270,9 @@ func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResourc if p.jobPriority-alloc.Job.Priority < 10 { continue } - networks := alloc.CompatibleNetworkResources() + networks := alloc.ComparableResources().Flattened.Networks + + // Only include if the alloc has a network device if len(networks) > 0 { device := networks[0].Device allocsForDevice := deviceToAllocs[device] @@ -221,6 +293,7 @@ func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResourc for device, currentAllocs := range deviceToAllocs { totalBandwidth := netIdx.AvailBandwidth[device] + // If the device doesn't have enough total available bandwidth, skip if totalBandwidth < MbitsNeeded { continue @@ -272,13 +345,12 @@ func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResourc for _, allocsGrp := range allocsByPriority { allocs := allocsGrp.allocs - // Sort by distance function that takes into account needed MBits - // as well as penalty for preempting an allocation - // whose task group already has existing preemptions + // Sort by distance function sort.Slice(allocs, func(i, j int) bool { return p.distanceComparatorForNetwork(allocs, networkResourceAsk, i, j) }) + // Iterate over allocs until end of if requirements have been met for _, alloc := range allocs { preemptedBandwidth += alloc.Resources.Networks[0].MBits allocsToPreempt = append(allocsToPreempt, alloc) @@ -287,12 +359,13 @@ func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResourc break } } - // If we met bandwidth needs we can break out of loop that's iterating by priority within a device + + // If we met bandwidth needs we can break out of iterating by priority within a device if met { break } } - // If we met bandwidth needs we can break out of loop that's iterating by allocs sharing the same network device + // If we met bandwidth needs we don't need to examine the next network device if met { break } @@ -315,32 +388,14 @@ func (p *Preemptor) preemptForNetwork(networkResourceAsk *structs.NetworkResourc }, } - // Sort by distance reversed to surface any superset allocs first - // This sort only looks at mbits because we should still not prefer - // allocs that have a maxParallel penalty - sort.Slice(allocsToPreempt, func(i, j int) bool { - firstAlloc := allocsToPreempt[i] - secondAlloc := allocsToPreempt[j] - - firstAllocNetworks := firstAlloc.ComparableResources().Flattened.Networks - var firstAllocNetResourceUsed *structs.NetworkResource - if len(firstAllocNetworks) > 0 { - firstAllocNetResourceUsed = firstAllocNetworks[0] - } - - secondAllocNetworks := secondAlloc.ComparableResources().Flattened.Networks - var secondAllocNetResourceUsed *structs.NetworkResource - if len(secondAllocNetworks) > 0 { - secondAllocNetResourceUsed = secondAllocNetworks[0] - } - - distance1 := networkResourceDistance(firstAllocNetResourceUsed, networkResourceAsk) - distance2 := networkResourceDistance(secondAllocNetResourceUsed, networkResourceAsk) - return distance1 > distance2 - }) - // Do a final pass to eliminate any superset allocations - filteredBestAllocs := filterSupersetNetwork(allocsToPreempt, networkResourceAsk, nodeRemainingResources) + preemptionResourceFactory := GetNetworkPreemptionResourceFactory() + resourcesNeeded := &structs.ComparableResources{ + Flattened: structs.AllocatedTaskResources{ + Networks: []*structs.NetworkResource{networkResourceAsk}, + }, + } + filteredBestAllocs := filterSuperset(allocsToPreempt, nodeRemainingResources, resourcesNeeded, preemptionResourceFactory) return filteredBestAllocs } @@ -400,14 +455,6 @@ func scoreForNetwork(resourceUsed *structs.NetworkResource, resourceNeeded *stru return networkResourceDistance(resourceUsed, resourceNeeded) + maxParallelScorePenalty } -// meetsNetworkRequirements checks if the first resource meets or exceeds the second resource's network MBits requirements -func meetsNetworkRequirements(firstMbits int, secondMbits int) bool { - if firstMbits == 0 || secondMbits == 0 { - return false - } - return firstMbits >= secondMbits -} - // filterAndGroupPreemptibleAllocs groups allocations by priority after removing any from jobs of // a higher priority than jobPriority func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocation) []*groupedAllocs { @@ -447,40 +494,20 @@ func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocat return groupedSortedAllocs } -// filterSupersetTaskGroup is used as a final step to remove +// filterSuperset is used as a final step to remove // any allocations that meet a superset of requirements from // the set of allocations to preempt -func filterSupersetTaskGroup(bestAllocs []*structs.Allocation, +func filterSuperset(bestAllocs []*structs.Allocation, nodeRemainingResources *structs.ComparableResources, - resourceAsk *structs.ComparableResources) []*structs.Allocation { - - var preemptedResources *structs.ComparableResources - var filteredBestAllocs []*structs.Allocation + resourceAsk *structs.ComparableResources, + preemptionResourceFactory PreemptionResourceFactory) []*structs.Allocation { - // Do another pass to eliminate allocations that are a superset of other allocations - // in the preemption set - for _, alloc := range bestAllocs { - if preemptedResources == nil { - preemptedResources = alloc.ComparableResources().Copy() - } else { - preemptedResources.Add(alloc.ComparableResources().Copy()) - } - filteredBestAllocs = append(filteredBestAllocs, alloc) - availableResources := preemptedResources.Copy() - availableResources.Add(nodeRemainingResources) - - requirementsMet, _ := availableResources.Superset(resourceAsk) - if requirementsMet { - break - } - } - return filteredBestAllocs -} - -// filterSupersetNetwork is similar to filterSupersetTaskGroup but only -// considers network Mbits -func filterSupersetNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk *structs.NetworkResource, - nodeRemainingResources *structs.ComparableResources) []*structs.Allocation { + // Sort bestAllocs by distance descending (without penalty) + sort.Slice(bestAllocs, func(i, j int) bool { + distance1 := preemptionResourceFactory(bestAllocs[i].ComparableResources(), resourceAsk).Distance() + distance2 := preemptionResourceFactory(bestAllocs[j].ComparableResources(), resourceAsk).Distance() + return distance1 > distance2 + }) var preemptedResources *structs.ComparableResources var filteredBestAllocs []*structs.Allocation @@ -497,7 +524,8 @@ func filterSupersetNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk availableResources := preemptedResources.Copy() availableResources.Add(nodeRemainingResources) - requirementsMet := meetsNetworkRequirements(availableResources.Flattened.Networks[0].MBits, networkResourcesAsk.MBits) + premptionResource := preemptionResourceFactory(availableResources, resourceAsk) + requirementsMet := premptionResource.MeetsRequirements() if requirementsMet { break } @@ -506,7 +534,7 @@ func filterSupersetNetwork(bestAllocs []*structs.Allocation, networkResourcesAsk } // distanceComparatorForNetwork is used as the sorting function when finding allocations to preempt. It uses -// both a coordinayte distance function based on Mbits needed, and a penalty if the allocation under consideration +// both a coordinate distance function based on Mbits needed, and a penalty if the allocation under consideration // belongs to a job that already has more preempted allocations func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, networkResourceAsk *structs.NetworkResource, i int, j int) bool { firstAlloc := allocs[i] @@ -518,7 +546,7 @@ func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, n maxParallel1 = tg1.Migrate.MaxParallel } // Dereference network usage on first alloc if its there - firstAllocNetworks := firstAlloc.CompatibleNetworkResources() + firstAllocNetworks := firstAlloc.ComparableResources().Flattened.Networks var firstAllocNetResourceUsed *structs.NetworkResource if len(firstAllocNetworks) > 0 { firstAllocNetResourceUsed = firstAllocNetworks[0] @@ -532,8 +560,8 @@ func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, n if tg2 != nil && tg2.Migrate != nil { maxParallel2 = tg2.Migrate.MaxParallel } - // Dereference network usage on first alloc if its there - secondAllocNetworks := secondAlloc.CompatibleNetworkResources() + // Dereference network usage on second alloc if its there + secondAllocNetworks := secondAlloc.ComparableResources().Flattened.Networks var secondAllocNetResourceUsed *structs.NetworkResource if len(secondAllocNetworks) > 0 { secondAllocNetResourceUsed = secondAllocNetworks[0] diff --git a/scheduler/rank.go b/scheduler/rank.go index 47cb8a89c8b..402dd9c7081 100644 --- a/scheduler/rank.go +++ b/scheduler/rank.go @@ -241,7 +241,7 @@ OUTER: // Look for preemptible allocations to satisfy the network resource for this task preemptor.SetCandidates(proposed) - netPreemptions := preemptor.preemptForNetwork(ask, netIdx) + netPreemptions := preemptor.PreemptForNetwork(ask, netIdx) if netPreemptions == nil { iter.ctx.Metrics().ExhaustedNode(option.Node, fmt.Sprintf("unable to meet network resource %v after preemption", ask)) @@ -303,7 +303,7 @@ OUTER: // Initialize preemptor with candidate set preemptor.SetCandidates(current) - preemptedAllocs := preemptor.preemptForTaskGroup(total) + preemptedAllocs := preemptor.PreemptForTaskGroup(total) allocsToPreempt = append(allocsToPreempt, preemptedAllocs...) // If we were unable to find preempted allocs to meet these requirements From f2b027797bce74087ac893ff073c3a292755e404 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Tue, 30 Oct 2018 11:10:46 -0500 Subject: [PATCH 27/38] Fix return type in tests after refactor --- command/agent/operator_endpoint_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/command/agent/operator_endpoint_test.go b/command/agent/operator_endpoint_test.go index ee424fb9f87..6de2e2ee8da 100644 --- a/command/agent/operator_endpoint_test.go +++ b/command/agent/operator_endpoint_test.go @@ -294,10 +294,10 @@ func TestOperator_SchedulerSetConfiguration(t *testing.T) { }, } - var reply structs.SchedulerConfiguration + var reply structs.SchedulerConfigurationResponse err = s.RPC("Operator.SchedulerGetConfiguration", &args, &reply) require.Nil(err) - require.True(reply.PreemptionConfig.SystemSchedulerEnabled) + require.True(reply.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) }) } @@ -320,11 +320,11 @@ func TestOperator_SchedulerCASConfiguration(t *testing.T) { }, } - var reply structs.SchedulerConfiguration + var reply structs.SchedulerConfigurationResponse if err := s.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { t.Fatalf("err: %v", err) } - require.True(reply.PreemptionConfig.SystemSchedulerEnabled) + require.True(reply.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) // Create a CAS request, bad index { @@ -354,6 +354,6 @@ func TestOperator_SchedulerCASConfiguration(t *testing.T) { if err := s.RPC("Operator.SchedulerGetConfiguration", &args, &reply); err != nil { t.Fatalf("err: %v", err) } - require.False(reply.PreemptionConfig.SystemSchedulerEnabled) + require.False(reply.SchedulerConfig.PreemptionConfig.SystemSchedulerEnabled) }) } From 22d156f5a031ff750b0038c1ca8ac05240fdbc7c Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 1 Nov 2018 12:01:59 -0500 Subject: [PATCH 28/38] review comments --- scheduler/preemption.go | 98 ++++++++++++++++++++--------------------- 1 file changed, 48 insertions(+), 50 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 6ecf49cd828..905ce1dd44d 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -17,6 +17,11 @@ type groupedAllocs struct { allocs []*structs.Allocation } +type allocInfo struct { + maxParallel int + resources *structs.ComparableResources +} + // PreemptionResource interface is implemented by different // types of resources. type PreemptionResource interface { @@ -44,14 +49,7 @@ func (n *NetworkPreemptionResource) MeetsRequirements() bool { } func (n *NetworkPreemptionResource) Distance() float64 { - networkCoord := math.MaxFloat64 - if n.availableResources != nil && n.resourceNeeded != nil { - networkCoord = float64(n.resourceNeeded.MBits-n.availableResources.MBits) / float64(n.resourceNeeded.MBits) - } - - originDist := math.Sqrt( - math.Pow(networkCoord, 2)) - return originDist + return networkResourceDistance(n.availableResources, n.resourceNeeded) } // BasePreemptionResource implements PreemptionResource for CPU/Memory/Disk @@ -100,10 +98,16 @@ type Preemptor struct { // it tracks the number of preempted allocations per job/taskgroup currentPreemptions map[structs.NamespacedID]map[string]int + // allocDetails is a map computed when SetCandidates is called + // it stores some precomputed details about the allocation needed + // when scoring it for preemption + allocDetails map[string]*allocInfo + // jobPriority is the priority of the job being preempted jobPriority int - // nodeRemainingResources tracks remaining available resources on the node + // nodeRemainingResources tracks available resources on the node after + // accounting for running allocations nodeRemainingResources *structs.ComparableResources // currentAllocs is the candidate set used to find preemptible allocations @@ -114,6 +118,7 @@ func NewPreemptor(jobPriority int) *Preemptor { return &Preemptor{ currentPreemptions: make(map[structs.NamespacedID]map[string]int), jobPriority: jobPriority, + allocDetails: make(map[string]*allocInfo), } } @@ -121,8 +126,8 @@ func NewPreemptor(jobPriority int) *Preemptor { func (p *Preemptor) SetNode(node *structs.Node) { nodeRemainingResources := node.ComparableResources() // Subtract the reserved resources of the node - if node.ComparableReservedResources() != nil { - nodeRemainingResources.Subtract(node.ComparableReservedResources()) + if c := node.ComparableReservedResources(); c != nil { + nodeRemainingResources.Subtract(c) } p.nodeRemainingResources = nodeRemainingResources } @@ -130,15 +135,21 @@ func (p *Preemptor) SetNode(node *structs.Node) { // SetCandidates initializes the candidate set from which preemptions are chosen func (p *Preemptor) SetCandidates(allocs []*structs.Allocation) { p.currentAllocs = allocs + for _, alloc := range allocs { + maxParallel := 0 + tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) + if tg != nil && tg.Migrate != nil { + maxParallel = tg.Migrate.MaxParallel + } + p.allocDetails[alloc.ID] = &allocInfo{maxParallel, alloc.ComparableResources()} + } } // SetPreemptions initializes a map tracking existing counts of preempted allocations // per job/task group. This is used while scoring preemption options func (p *Preemptor) SetPreemptions(allocs []*structs.Allocation) { // Clear out existing values since this can be called more than once - for k := range p.currentPreemptions { - delete(p.currentPreemptions, k) - } + p.currentPreemptions = make(map[structs.NamespacedID]map[string]int) // Initialize counts for _, alloc := range allocs { @@ -146,22 +157,20 @@ func (p *Preemptor) SetPreemptions(allocs []*structs.Allocation) { countMap, ok := p.currentPreemptions[id] if !ok { countMap = make(map[string]int) + p.currentPreemptions[id] = countMap } - c := countMap[alloc.TaskGroup] - countMap[alloc.TaskGroup] = c + 1 - p.currentPreemptions[id] = countMap + countMap[alloc.TaskGroup]++ } } // getNumPreemptions counts the number of other allocations being preempted that match the job and task group of // the alloc under consideration. This is used as a scoring factor to minimize too many allocs of the same job being preempted at once func (p *Preemptor) getNumPreemptions(alloc *structs.Allocation) int { - numCurrentPreemptionsForJob := 0 - countMap := p.currentPreemptions[structs.NamespacedID{alloc.JobID, alloc.Namespace}] - if countMap != nil { - numCurrentPreemptionsForJob = countMap[alloc.TaskGroup] + c, ok := p.currentPreemptions[structs.NamespacedID{alloc.JobID, alloc.Namespace}][alloc.TaskGroup] + if !ok { + return 0 } - return numCurrentPreemptionsForJob + return c } // PreemptForTaskGroup computes a list of allocations to preempt to accommodate @@ -180,7 +189,9 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) var bestAllocs []*structs.Allocation allRequirementsMet := false - var preemptedResources *structs.ComparableResources + + // Initialize variable to track resources as they become available from preemption + availableResources := p.nodeRemainingResources.Copy() // Iterate over allocations grouped by priority to find preemptible allocations for _, allocGrp := range allocsByPriority { @@ -190,26 +201,16 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) // Find the alloc with the closest distance for index, alloc := range allocGrp.allocs { currentPreemptionCount := p.getNumPreemptions(alloc) - maxParallel := 0 - tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) - if tg != nil && tg.Migrate != nil { - maxParallel = tg.Migrate.MaxParallel - } - distance := scoreForTaskGroup(resourcesNeeded, alloc.ComparableResources(), maxParallel, currentPreemptionCount) + allocDetails := p.allocDetails[alloc.ID] + maxParallel := allocDetails.maxParallel + distance := scoreForTaskGroup(resourcesNeeded, allocDetails.resources, maxParallel, currentPreemptionCount) if distance < bestDistance { bestDistance = distance closestAllocIndex = index } } closestAlloc := allocGrp.allocs[closestAllocIndex] - - if preemptedResources == nil { - preemptedResources = closestAlloc.ComparableResources() - } else { - preemptedResources.Add(closestAlloc.ComparableResources()) - } - availableResources := preemptedResources.Copy() - availableResources.Add(p.nodeRemainingResources) + availableResources.Add(closestAlloc.ComparableResources()) // This step needs the original resources asked for as the second arg, can't use the running total allRequirementsMet, _ = availableResources.Superset(resourceAsk.Comparable()) @@ -232,11 +233,10 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) return nil } - resourcesNeeded = resourceAsk.Comparable() // We do another pass to eliminate unnecessary preemptions // This filters out allocs whose resources are already covered by another alloc basePreemptionResource := GetBasePreemptionResourceFactory() - + resourcesNeeded = resourceAsk.Comparable() filteredBestAllocs := filterSuperset(bestAllocs, p.nodeRemainingResources, resourcesNeeded, basePreemptionResource) return filteredBestAllocs @@ -257,20 +257,19 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc reservedPortsNeeded := networkResourceAsk.ReservedPorts // Create a map from each device to allocs - // We do this because to place a task we have to be able to - // preempt allocations that are using the same device. - - // This step also filters out high priority allocations and allocations - // that are not using any network resources + // We can only preempt within allocations that + // are using the same device for _, alloc := range p.currentAllocs { if alloc.Job == nil { continue } + // Filter out alloc that's ineligible due to priority if p.jobPriority-alloc.Job.Priority < 10 { continue } - networks := alloc.ComparableResources().Flattened.Networks + allocResources := p.allocDetails[alloc.ID].resources + networks := allocResources.Flattened.Networks // Only include if the alloc has a network device if len(networks) > 0 { @@ -287,7 +286,6 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc } var allocsToPreempt []*structs.Allocation - met := false freeBandwidth := 0 @@ -313,8 +311,9 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // First try to satisfy needed reserved ports if len(reservedPortsNeeded) > 0 { for _, alloc := range currentAllocs { - for _, tr := range alloc.TaskResources { - reservedPorts := tr.Networks[0].ReservedPorts + allocResources := p.allocDetails[alloc.ID].resources + for _, n := range allocResources.Flattened.Networks { + reservedPorts := n.ReservedPorts for _, p := range reservedPorts { usedPortToAlloc[p.Value] = alloc } @@ -426,8 +425,7 @@ func networkResourceDistance(resourceUsed *structs.NetworkResource, resourceNeed networkCoord = float64(resourceNeeded.MBits-resourceUsed.MBits) / float64(resourceNeeded.MBits) } - originDist := math.Sqrt( - math.Pow(networkCoord, 2)) + originDist := math.Abs(networkCoord) return originDist } From 993b6a272a66c6c46b1a3e9ca46fae7916728bb5 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 1 Nov 2018 16:15:58 -0500 Subject: [PATCH 29/38] Cleaner way to exit early, and fixed a couple more places reading from alloc.Resources --- scheduler/preemption.go | 48 +++++++++++++++++++-------- scheduler/preemption_test.go | 63 ++++++++++++++++++++++++++++++++++-- 2 files changed, 96 insertions(+), 15 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 905ce1dd44d..99eb9d9224e 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -255,6 +255,10 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc deviceToAllocs := make(map[string][]*structs.Allocation) MbitsNeeded := networkResourceAsk.MBits reservedPortsNeeded := networkResourceAsk.ReservedPorts + reservedPorts := make(map[int]interface{}) + for _, port := range reservedPortsNeeded { + reservedPorts[port.Value] = struct{}{} + } // Create a map from each device to allocs // We can only preempt within allocations that @@ -266,6 +270,12 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // Filter out alloc that's ineligible due to priority if p.jobPriority-alloc.Job.Priority < 10 { + // If this allocation uses a needed reserved port + // preemption is impossible so we return early + networks := alloc.ComparableResources().Flattened.Networks + if len(networks) > 0 && usedReservedPorts(networks[0], reservedPorts) { + return nil + } continue } allocResources := p.allocDetails[alloc.ID].resources @@ -289,6 +299,7 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc met := false freeBandwidth := 0 +OUTER: for device, currentAllocs := range deviceToAllocs { totalBandwidth := netIdx.AvailBandwidth[device] @@ -324,7 +335,8 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc for _, port := range reservedPortsNeeded { alloc, ok := usedPortToAlloc[port.Value] if ok { - preemptedBandwidth += alloc.Resources.Networks[0].MBits + allocResources := p.allocDetails[alloc.ID].resources + preemptedBandwidth += allocResources.Flattened.Networks[0].MBits allocsToPreempt = append(allocsToPreempt, alloc) } } @@ -335,7 +347,8 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // If bandwidth requirements have been met, stop if preemptedBandwidth+freeBandwidth >= MbitsNeeded { - break + met = true + break OUTER } // Split by priority @@ -351,25 +364,21 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // Iterate over allocs until end of if requirements have been met for _, alloc := range allocs { - preemptedBandwidth += alloc.Resources.Networks[0].MBits + allocResources := p.allocDetails[alloc.ID].resources + preemptedBandwidth += allocResources.Flattened.Networks[0].MBits allocsToPreempt = append(allocsToPreempt, alloc) if preemptedBandwidth+freeBandwidth >= MbitsNeeded { met = true - break + break OUTER } } - // If we met bandwidth needs we can break out of iterating by priority within a device - if met { - break - } - } - // If we met bandwidth needs we don't need to examine the next network device - if met { - break } + } - if len(allocsToPreempt) == 0 { + + // Early return if we could not meet resource needs after examining allocs + if !met { return nil } @@ -398,6 +407,19 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc return filteredBestAllocs } +func usedReservedPorts(net *structs.NetworkResource, portMap map[int]interface{}) bool { + ports := net.ReservedPorts + if len(ports) > 0 { + for _, p := range ports { + _, ok := portMap[p.Value] + if ok { + return true + } + } + } + return false +} + // basicResourceDistance computes a distance using a coordinate system. It compares resource fields like CPU/Memory and Disk. // Values emitted are in the range [0, maxFloat] func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources) float64 { diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go index e9ca02ecb6f..285d41db4cd 100644 --- a/scheduler/preemption_test.go +++ b/scheduler/preemption_test.go @@ -258,6 +258,64 @@ func TestPreemption(t *testing.T) { }, }, }, + { + 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, + IOPS: 50, + 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, + IOPS: 50, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.200", + MBits: 600, + ReservedPorts: []structs.Port{ + { + Label: "db", + Value: 88, + }, + }, + }, + }, + }), + }, + nodeReservedCapacity: reservedNodeResources, + nodeCapacity: nodeResources, + 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: "Combination of high/low priority allocs, without static ports", currentAllocations: []*structs.Allocation{ @@ -326,7 +384,8 @@ func TestPreemption(t *testing.T) { allocIDs[2]: {}, allocIDs[3]: {}, }, - }, { + }, + { desc: "Preemption needed for all resources except network", currentAllocations: []*structs.Allocation{ createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ @@ -522,7 +581,7 @@ func TestPreemption(t *testing.T) { }, }, { - desc: "alloc that meets static port need also meets other needds", + desc: "alloc that meets static port need also meets other needs", currentAllocations: []*structs.Allocation{ createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ CPU: 1200, From 3ad7b3fb5b9f7fd76fdc0b04fa1cc1aefb6b96b3 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 1 Nov 2018 16:36:11 -0500 Subject: [PATCH 30/38] More review comments --- scheduler/preemption.go | 29 +++++++++++++++-------------- scheduler/testing.go | 2 +- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 99eb9d9224e..26e5af6a430 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -94,6 +94,7 @@ func GetBasePreemptionResourceFactory() PreemptionResourceFactory { // Preemptor is used to track existing allocations // and find suitable allocations to preempt type Preemptor struct { + // currentPreemptions is a map computed when SetPreemptions is called // it tracks the number of preempted allocations per job/taskgroup currentPreemptions map[structs.NamespacedID]map[string]int @@ -125,6 +126,7 @@ func NewPreemptor(jobPriority int) *Preemptor { // SetNode sets the node func (p *Preemptor) SetNode(node *structs.Node) { nodeRemainingResources := node.ComparableResources() + // Subtract the reserved resources of the node if c := node.ComparableReservedResources(); c != nil { nodeRemainingResources.Subtract(c) @@ -148,6 +150,7 @@ func (p *Preemptor) SetCandidates(allocs []*structs.Allocation) { // SetPreemptions initializes a map tracking existing counts of preempted allocations // per job/task group. This is used while scoring preemption options func (p *Preemptor) SetPreemptions(allocs []*structs.Allocation) { + // Clear out existing values since this can be called more than once p.currentPreemptions = make(map[structs.NamespacedID]map[string]int) @@ -237,7 +240,7 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) // This filters out allocs whose resources are already covered by another alloc basePreemptionResource := GetBasePreemptionResourceFactory() resourcesNeeded = resourceAsk.Comparable() - filteredBestAllocs := filterSuperset(bestAllocs, p.nodeRemainingResources, resourcesNeeded, basePreemptionResource) + filteredBestAllocs := p.filterSuperset(bestAllocs, p.nodeRemainingResources, resourcesNeeded, basePreemptionResource) return filteredBestAllocs } @@ -270,6 +273,7 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // Filter out alloc that's ineligible due to priority if p.jobPriority-alloc.Job.Priority < 10 { + // If this allocation uses a needed reserved port // preemption is impossible so we return early networks := alloc.ComparableResources().Flattened.Networks @@ -403,7 +407,7 @@ OUTER: Networks: []*structs.NetworkResource{networkResourceAsk}, }, } - filteredBestAllocs := filterSuperset(allocsToPreempt, nodeRemainingResources, resourcesNeeded, preemptionResourceFactory) + filteredBestAllocs := p.filterSuperset(allocsToPreempt, nodeRemainingResources, resourcesNeeded, preemptionResourceFactory) return filteredBestAllocs } @@ -475,12 +479,11 @@ func scoreForNetwork(resourceUsed *structs.NetworkResource, resourceNeeded *stru return networkResourceDistance(resourceUsed, resourceNeeded) + maxParallelScorePenalty } -// filterAndGroupPreemptibleAllocs groups allocations by priority after removing any from jobs of -// a higher priority than jobPriority +// filterAndGroupPreemptibleAllocs groups allocations by priority after filtering allocs +// that are not preemptible based on the jobPriority arg func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocation) []*groupedAllocs { allocsByPriority := make(map[int][]*structs.Allocation) for _, alloc := range current { - // Why is alloc.Job even nil though? if alloc.Job == nil { continue } @@ -517,7 +520,7 @@ func filterAndGroupPreemptibleAllocs(jobPriority int, current []*structs.Allocat // filterSuperset is used as a final step to remove // any allocations that meet a superset of requirements from // the set of allocations to preempt -func filterSuperset(bestAllocs []*structs.Allocation, +func (p *Preemptor) filterSuperset(bestAllocs []*structs.Allocation, nodeRemainingResources *structs.ComparableResources, resourceAsk *structs.ComparableResources, preemptionResourceFactory PreemptionResourceFactory) []*structs.Allocation { @@ -529,20 +532,15 @@ func filterSuperset(bestAllocs []*structs.Allocation, return distance1 > distance2 }) - var preemptedResources *structs.ComparableResources + availableResources := nodeRemainingResources.Copy() var filteredBestAllocs []*structs.Allocation // Do another pass to eliminate allocations that are a superset of other allocations // in the preemption set for _, alloc := range bestAllocs { - if preemptedResources == nil { - preemptedResources = alloc.ComparableResources().Copy() - } else { - preemptedResources.Add(alloc.ComparableResources().Copy()) - } filteredBestAllocs = append(filteredBestAllocs, alloc) - availableResources := preemptedResources.Copy() - availableResources.Add(nodeRemainingResources) + allocResources := p.allocDetails[alloc.ID].resources + availableResources.Add(allocResources) premptionResource := preemptionResourceFactory(availableResources, resourceAsk) requirementsMet := premptionResource.MeetsRequirements() @@ -559,12 +557,14 @@ func filterSuperset(bestAllocs []*structs.Allocation, func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, networkResourceAsk *structs.NetworkResource, i int, j int) bool { firstAlloc := allocs[i] currentPreemptionCount1 := p.getNumPreemptions(firstAlloc) + // Look up configured maxParallel value for these allocation's task groups var maxParallel1, maxParallel2 int tg1 := allocs[i].Job.LookupTaskGroup(firstAlloc.TaskGroup) if tg1 != nil && tg1.Migrate != nil { maxParallel1 = tg1.Migrate.MaxParallel } + // Dereference network usage on first alloc if its there firstAllocNetworks := firstAlloc.ComparableResources().Flattened.Networks var firstAllocNetResourceUsed *structs.NetworkResource @@ -580,6 +580,7 @@ func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, n if tg2 != nil && tg2.Migrate != nil { maxParallel2 = tg2.Migrate.MaxParallel } + // Dereference network usage on second alloc if its there secondAllocNetworks := secondAlloc.ComparableResources().Flattened.Networks var secondAllocNetResourceUsed *structs.NetworkResource diff --git a/scheduler/testing.go b/scheduler/testing.go index d701436c5b5..31fc9ab91af 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -117,7 +117,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er } } - // Set create and modify time for preempted allocs and flatten them + // Set modify time for preempted allocs and flatten them var preemptedAllocs []*structs.Allocation for _, preemptions := range result.NodePreemptions { for _, alloc := range preemptions { From 06ad182026ee3f14bfaf03f52ea7595893897389 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 1 Nov 2018 16:44:43 -0500 Subject: [PATCH 31/38] Plumb alloc resource cache in a few more places. also removed now unused method --- nomad/structs/structs.go | 31 ------------------------------- scheduler/preemption.go | 23 +++++++++++++++-------- 2 files changed, 15 insertions(+), 39 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 8e665103bea..cffd31a1f91 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -7411,37 +7411,6 @@ func (a *Allocation) ComparableResources() *ComparableResources { } } -// COMPAT(0.11): Remove in 0.11 -// CompatibleNetworkResources returns network resources on the allocation -// by reading AllocatedResources which are populated starting in 0.9 and -// falling back to pre 0.9 fields (Resources/TaskResources) if set -func (a *Allocation) CompatibleNetworkResources() []*NetworkResource { - var ret []*NetworkResource - // Alloc already has 0.9+ behavior - if a.AllocatedResources != nil { - var comparableResources *ComparableResources - for _, taskResource := range a.AllocatedResources.Tasks { - if comparableResources == nil { - comparableResources = taskResource.Comparable() - } else { - comparableResources.Add(taskResource.Comparable()) - } - } - ret = comparableResources.Flattened.Networks - } else if a.Resources != nil { - // Alloc has pre 0.9 total resources - ret = a.Resources.Networks - } else if a.TaskResources != nil { - // Alloc has pre 0.9 task resources - resources := new(Resources) - for _, taskResource := range a.TaskResources { - resources.Add(taskResource) - } - ret = resources.Networks - } - return ret -} - // LookupTask by name from the Allocation. Returns nil if the Job is not set, the // TaskGroup does not exist, or the task name cannot be found. func (a *Allocation) LookupTask(name string) *Task { diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 26e5af6a430..1c6fc821851 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -184,7 +184,8 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) // Subtract current allocations for _, alloc := range p.currentAllocs { - p.nodeRemainingResources.Subtract(alloc.ComparableResources()) + allocResources := p.allocDetails[alloc.ID].resources + p.nodeRemainingResources.Subtract(allocResources) } // Group candidates by priority, filter out ineligible allocs @@ -213,7 +214,8 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) } } closestAlloc := allocGrp.allocs[closestAllocIndex] - availableResources.Add(closestAlloc.ComparableResources()) + closestResources := p.allocDetails[closestAlloc.ID].resources + availableResources.Add(closestResources) // This step needs the original resources asked for as the second arg, can't use the running total allRequirementsMet, _ = availableResources.Superset(resourceAsk.Comparable()) @@ -224,7 +226,7 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) allocGrp.allocs = allocGrp.allocs[:len(allocGrp.allocs)-1] // This is the remaining total of resources needed - resourcesNeeded.Subtract(closestAlloc.ComparableResources()) + resourcesNeeded.Subtract(closestResources) } if allRequirementsMet { break @@ -276,7 +278,8 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // If this allocation uses a needed reserved port // preemption is impossible so we return early - networks := alloc.ComparableResources().Flattened.Networks + allocResources := p.allocDetails[alloc.ID].resources + networks := allocResources.Flattened.Networks if len(networks) > 0 && usedReservedPorts(networks[0], reservedPorts) { return nil } @@ -527,8 +530,10 @@ func (p *Preemptor) filterSuperset(bestAllocs []*structs.Allocation, // Sort bestAllocs by distance descending (without penalty) sort.Slice(bestAllocs, func(i, j int) bool { - distance1 := preemptionResourceFactory(bestAllocs[i].ComparableResources(), resourceAsk).Distance() - distance2 := preemptionResourceFactory(bestAllocs[j].ComparableResources(), resourceAsk).Distance() + a1Resources := p.allocDetails[bestAllocs[i].ID].resources + a2Resources := p.allocDetails[bestAllocs[j].ID].resources + distance1 := preemptionResourceFactory(a1Resources, resourceAsk).Distance() + distance2 := preemptionResourceFactory(a2Resources, resourceAsk).Distance() return distance1 > distance2 }) @@ -566,7 +571,8 @@ func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, n } // Dereference network usage on first alloc if its there - firstAllocNetworks := firstAlloc.ComparableResources().Flattened.Networks + firstAllocResources := p.allocDetails[firstAlloc.ID].resources + firstAllocNetworks := firstAllocResources.Flattened.Networks var firstAllocNetResourceUsed *structs.NetworkResource if len(firstAllocNetworks) > 0 { firstAllocNetResourceUsed = firstAllocNetworks[0] @@ -582,7 +588,8 @@ func (p *Preemptor) distanceComparatorForNetwork(allocs []*structs.Allocation, n } // Dereference network usage on second alloc if its there - secondAllocNetworks := secondAlloc.ComparableResources().Flattened.Networks + secondAllocResources := p.allocDetails[secondAlloc.ID].resources + secondAllocNetworks := secondAllocResources.Flattened.Networks var secondAllocNetResourceUsed *structs.NetworkResource if len(secondAllocNetworks) > 0 { secondAllocNetResourceUsed = secondAllocNetworks[0] From 35d31f8be6eab0da942f68aae2f0e86014779c65 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 1 Nov 2018 17:05:17 -0500 Subject: [PATCH 32/38] more minor review feedback --- api/jobs.go | 2 +- nomad/leader.go | 11 ++++++++--- nomad/state/state_store.go | 4 ++-- nomad/structs/structs.go | 1 + 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/api/jobs.go b/api/jobs.go index a064b318424..0c8fc4608c5 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -1024,7 +1024,7 @@ type DesiredUpdates struct { InPlaceUpdate uint64 DestructiveUpdate uint64 Canary uint64 - Evict uint64 + Preemptions uint64 } type JobDispatchRequest struct { diff --git a/nomad/leader.go b/nomad/leader.go index bf3f350f152..3d6759d6971 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -42,8 +42,12 @@ const ( var minAutopilotVersion = version.Must(version.NewVersion("0.8.0")) -// Default configuration for scheduler with preemption emabled for system jobs -var defaultSchedulerConfig = &structs.SchedulerConfiguration{PreemptionConfig: structs.PreemptionConfig{SystemSchedulerEnabled: true}} +// Default configuration for scheduler with preemption enabled for system jobs +var defaultSchedulerConfig = &structs.SchedulerConfiguration{ + PreemptionConfig: structs.PreemptionConfig{ + SystemSchedulerEnabled: true, + }, +} // monitorLeadership is used to monitor if we acquire or lose our role // as the leader in the Raft cluster. There is some work the leader is @@ -1237,7 +1241,8 @@ func (s *Server) getOrCreateAutopilotConfig() *structs.AutopilotConfig { return config } -// getOrCreateSchedulerConfig is used to get the scheduler config, initializing it if necessary +// getOrCreateSchedulerConfig is used to get the scheduler config. We create a default +// config if it doesn't already exist for bootstrapping an empty cluster func (s *Server) getOrCreateSchedulerConfig() *structs.SchedulerConfiguration { state := s.fsm.State() _, config, err := state.SchedulerConfig() diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 793d01e8309..c2fcc4162fe 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -3889,9 +3889,9 @@ func (s *StateStore) SchedulerSetConfig(idx uint64, config *structs.SchedulerCon return nil } -// SchedulerCASConfig is used to try updating the scheduler configuration with a +// SchedulerCASConfig is used to update the scheduler configuration with a // given Raft index. If the CAS index specified is not equal to the last observed index -// for the config, then the call is a noop, +// for the config, then the call is a noop. func (s *StateStore) SchedulerCASConfig(idx, cidx uint64, config *structs.SchedulerConfiguration) (bool, error) { tx := s.db.Txn(true) defer tx.Abort() diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index cffd31a1f91..014ee78ada0 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -8176,6 +8176,7 @@ func (p *Plan) AppendPreemptedAlloc(alloc *Allocation, desiredStatus, preempting if alloc.AllocatedResources != nil { newAlloc.AllocatedResources = alloc.AllocatedResources } else { + // COMPAT Remove in version 0.11 newAlloc.TaskResources = alloc.TaskResources newAlloc.SharedResources = alloc.SharedResources } From b3738a0c226654f8ff3eaef0324cf9aa0e29ff49 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Thu, 1 Nov 2018 20:06:32 -0500 Subject: [PATCH 33/38] unit test plan apply with preemptions --- nomad/plan_apply.go | 5 +- nomad/plan_apply_test.go | 115 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 118 insertions(+), 2 deletions(-) diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 205609ce5e1..95dd007f0bc 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -187,7 +187,7 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap alloc.ModifyTime = now } - // Set create and modify time for preempted allocs if any + // Set modify time for preempted allocs if any // Also gather jobids to create follow up evals preemptedJobIDs := make(map[structs.NamespacedID]struct{}) for _, alloc := range req.NodePreemptions { @@ -358,9 +358,10 @@ func evaluatePlanPlacements(pool *EvaluatePool, snap *state.StateSnapshot, plan } if nodePreemptions := plan.NodePreemptions[nodeID]; nodePreemptions != nil { - var filteredNodePreemptions []*structs.Allocation + // Do a pass over preempted allocs in the plan to check // whether the alloc is already in a terminal state + var filteredNodePreemptions []*structs.Allocation for _, preemptedAlloc := range nodePreemptions { alloc, err := snap.AllocByID(nil, preemptedAlloc.ID) if err != nil { diff --git a/nomad/plan_apply_test.go b/nomad/plan_apply_test.go index 5f2e735caa0..1b5142aa919 100644 --- a/nomad/plan_apply_test.go +++ b/nomad/plan_apply_test.go @@ -12,6 +12,7 @@ import ( "github.com/hashicorp/nomad/testutil" "github.com/hashicorp/raft" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) const ( @@ -271,6 +272,120 @@ func TestPlanApply_EvalPlan_Simple(t *testing.T) { } } +func TestPlanApply_EvalPlan_Preemption(t *testing.T) { + t.Parallel() + state := testStateStore(t) + node := mock.Node() + node.NodeResources = &structs.NodeResources{ + Cpu: structs.NodeCpuResources{ + CpuShares: 2000, + }, + Memory: structs.NodeMemoryResources{ + MemoryMB: 4192, + }, + Disk: structs.NodeDiskResources{ + DiskMB: 30 * 1024, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + CIDR: "192.168.0.100/32", + MBits: 1000, + }, + }, + } + state.UpsertNode(1000, node) + + preemptedAlloc := mock.Alloc() + preemptedAlloc.NodeID = node.ID + preemptedAlloc.AllocatedResources = &structs.AllocatedResources{ + Shared: structs.AllocatedSharedResources{ + DiskMB: 25 * 1024, + }, + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: 1500, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 4000, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "admin", Value: 5000}}, + MBits: 800, + DynamicPorts: []structs.Port{{Label: "http", Value: 9876}}, + }, + }, + }, + }, + } + + // Insert a preempted alloc such that the alloc will fit only after preemption + state.UpsertAllocs(1001, []*structs.Allocation{preemptedAlloc}) + + alloc := mock.Alloc() + alloc.AllocatedResources = &structs.AllocatedResources{ + Shared: structs.AllocatedSharedResources{ + DiskMB: 24 * 1024, + }, + Tasks: map[string]*structs.AllocatedTaskResources{ + "web": { + Cpu: structs.AllocatedCpuResources{ + CpuShares: 1500, + }, + Memory: structs.AllocatedMemoryResources{ + MemoryMB: 3200, + }, + Networks: []*structs.NetworkResource{ + { + Device: "eth0", + IP: "192.168.0.100", + ReservedPorts: []structs.Port{{Label: "admin", Value: 5000}}, + MBits: 800, + DynamicPorts: []structs.Port{{Label: "http", Value: 9876}}, + }, + }, + }, + }, + } + plan := &structs.Plan{ + Job: alloc.Job, + NodeAllocation: map[string][]*structs.Allocation{ + node.ID: {alloc}, + }, + NodePreemptions: map[string][]*structs.Allocation{ + node.ID: {preemptedAlloc}, + }, + Deployment: mock.Deployment(), + DeploymentUpdates: []*structs.DeploymentStatusUpdate{ + { + DeploymentID: uuid.Generate(), + Status: "foo", + StatusDescription: "bar", + }, + }, + } + snap, _ := state.Snapshot() + + pool := NewEvaluatePool(workerPoolSize, workerPoolBufferSize) + defer pool.Shutdown() + + result, err := evaluatePlan(pool, snap, plan, testlog.HCLogger(t)) + + require := require.New(t) + require.NoError(err) + require.NotNil(result) + + require.Equal(result.NodeAllocation, plan.NodeAllocation) + require.Equal(result.Deployment, plan.Deployment) + require.Equal(result.DeploymentUpdates, plan.DeploymentUpdates) + require.Equal(result.NodePreemptions, plan.NodePreemptions) + +} + func TestPlanApply_EvalPlan_Partial(t *testing.T) { t.Parallel() state := testStateStore(t) From 0015095b57f8db80f5dbf62d66118dfecd6a1718 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Fri, 2 Nov 2018 09:09:50 -0500 Subject: [PATCH 34/38] Handle static port preemption when there are multiple devices Also added test case --- scheduler/preemption.go | 42 ++++++++----- scheduler/preemption_test.go | 115 +++++++++++++++++++++++++++++++---- 2 files changed, 132 insertions(+), 25 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 1c6fc821851..e27be15c82d 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -260,11 +260,15 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc deviceToAllocs := make(map[string][]*structs.Allocation) MbitsNeeded := networkResourceAsk.MBits reservedPortsNeeded := networkResourceAsk.ReservedPorts + + // Build map of reserved ports needed for fast access reservedPorts := make(map[int]interface{}) for _, port := range reservedPortsNeeded { reservedPorts[port.Value] = struct{}{} } + usedPortToAlloc := make(map[int]*structs.Allocation) + // Create a map from each device to allocs // We can only preempt within allocations that // are using the same device @@ -294,6 +298,14 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc allocsForDevice := deviceToAllocs[device] allocsForDevice = append(allocsForDevice, alloc) deviceToAllocs[device] = allocsForDevice + + // Populate map from used reserved ports to allocation + for _, n := range allocResources.Flattened.Networks { + reservedPorts := n.ReservedPorts + for _, p := range reservedPorts { + usedPortToAlloc[p.Value] = alloc + } + } } } @@ -323,29 +335,31 @@ OUTER: // Reset allocsToPreempt since we don't want to preempt across devices for the same task allocsToPreempt = nil - // Build map from used reserved ports to allocation - usedPortToAlloc := make(map[int]*structs.Allocation) - // First try to satisfy needed reserved ports if len(reservedPortsNeeded) > 0 { - for _, alloc := range currentAllocs { - allocResources := p.allocDetails[alloc.ID].resources - for _, n := range allocResources.Flattened.Networks { - reservedPorts := n.ReservedPorts - for _, p := range reservedPorts { - usedPortToAlloc[p.Value] = alloc - } - } - } // Look for allocs that are using reserved ports needed for _, port := range reservedPortsNeeded { alloc, ok := usedPortToAlloc[port.Value] + if ok { + // If the reserved port is in the map, we need + // to check if the device matches. allocResources := p.allocDetails[alloc.ID].resources - preemptedBandwidth += allocResources.Flattened.Networks[0].MBits - allocsToPreempt = append(allocsToPreempt, alloc) + allocDevice := allocResources.Flattened.Networks[0].Device + if allocDevice == device { + preemptedBandwidth += allocResources.Flattened.Networks[0].MBits + allocsToPreempt = append(allocsToPreempt, alloc) + } else { + // If its on a different device we continue to the + // outer loop because the current device cannot meet + // a needed, used reserved port + continue OUTER + } } + + // If the reserved port is not in the usedPortToAlloc map it means + // no alloc is using it. } // Remove allocs that were preempted to satisfy reserved ports diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go index 285d41db4cd..cdca7153cf1 100644 --- a/scheduler/preemption_test.go +++ b/scheduler/preemption_test.go @@ -163,7 +163,8 @@ func TestPreemption(t *testing.T) { // Create some persistent alloc ids to use in test cases allocIDs := []string{uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate(), uuid.Generate()} - nodeResources := &structs.Resources{ + // TODO(preetha): Switch to using NodeResources and NodeReservedResources + defaultNodeResources := &structs.Resources{ CPU: 4000, MemoryMB: 8192, DiskMB: 100 * 1024, @@ -208,7 +209,7 @@ func TestPreemption(t *testing.T) { }, })}, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 2000, @@ -241,7 +242,7 @@ func TestPreemption(t *testing.T) { }, })}, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 4000, @@ -295,7 +296,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 600, @@ -316,6 +317,97 @@ func TestPreemption(t *testing.T) { }, }, }, + { + desc: "preempt only from device that has allocation with used reserved port", + currentAllocations: []*structs.Allocation{ + createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ + CPU: 1200, + MemoryMB: 2256, + DiskMB: 4 * 1024, + IOPS: 50, + 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, + IOPS: 50, + 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, + IOPS: 50, + 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.Resources{ + CPU: 4000, + MemoryMB: 8192, + DiskMB: 100 * 1024, + IOPS: 150, + 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[1]: {}, + }, + }, { desc: "Combination of high/low priority allocs, without static ports", currentAllocations: []*structs.Allocation{ @@ -365,7 +457,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 1100, @@ -426,7 +518,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 1000, @@ -487,7 +579,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 300, @@ -555,7 +647,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 2700, @@ -630,7 +722,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 600, @@ -698,7 +790,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 300, @@ -775,7 +867,7 @@ func TestPreemption(t *testing.T) { }), }, nodeReservedCapacity: reservedNodeResources, - nodeCapacity: nodeResources, + nodeCapacity: defaultNodeResources, jobPriority: 100, resourceAsk: &structs.Resources{ CPU: 1000, @@ -800,6 +892,7 @@ func TestPreemption(t *testing.T) { node := mock.Node() node.Resources = tc.nodeCapacity node.Reserved = tc.nodeReservedCapacity + node.NodeResources = nil state, ctx := testContext(t) nodes := []*RankedNode{ From 82359191589d65b994cef40a49b1bb3ef1d1fc87 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Fri, 2 Nov 2018 13:07:24 -0500 Subject: [PATCH 35/38] Fix static port preemption to be device aware --- scheduler/preemption.go | 56 ++++++++++++++++++++++------------------- 1 file changed, 30 insertions(+), 26 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index e27be15c82d..5fe974fc628 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -267,7 +267,7 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc reservedPorts[port.Value] = struct{}{} } - usedPortToAlloc := make(map[int]*structs.Allocation) + filteredReservedPorts := make(map[string]map[int]struct{}) // Create a map from each device to allocs // We can only preempt within allocations that @@ -284,8 +284,14 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // preemption is impossible so we return early allocResources := p.allocDetails[alloc.ID].resources networks := allocResources.Flattened.Networks - if len(networks) > 0 && usedReservedPorts(networks[0], reservedPorts) { - return nil + net := networks[0] + for _, port := range net.ReservedPorts { + portMap, ok := filteredReservedPorts[net.Device] + if !ok { + portMap = make(map[int]struct{}) + filteredReservedPorts[net.Device] = portMap + } + portMap[port.Value] = struct{}{} } continue } @@ -298,14 +304,6 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc allocsForDevice := deviceToAllocs[device] allocsForDevice = append(allocsForDevice, alloc) deviceToAllocs[device] = allocsForDevice - - // Populate map from used reserved ports to allocation - for _, n := range allocResources.Flattened.Networks { - reservedPorts := n.ReservedPorts - for _, p := range reservedPorts { - usedPortToAlloc[p.Value] = alloc - } - } } } @@ -335,31 +333,37 @@ OUTER: // Reset allocsToPreempt since we don't want to preempt across devices for the same task allocsToPreempt = nil + usedPortToAlloc := make(map[int]*structs.Allocation) + // First try to satisfy needed reserved ports if len(reservedPortsNeeded) > 0 { + // Populate usedPort map + for _, alloc := range currentAllocs { + allocResources := p.allocDetails[alloc.ID].resources + for _, n := range allocResources.Flattened.Networks { + reservedPorts := n.ReservedPorts + for _, p := range reservedPorts { + usedPortToAlloc[p.Value] = alloc + } + } + } // Look for allocs that are using reserved ports needed for _, port := range reservedPortsNeeded { alloc, ok := usedPortToAlloc[port.Value] - + allocResources := p.allocDetails[alloc.ID].resources + allocDevice := allocResources.Flattened.Networks[0].Device if ok { - // If the reserved port is in the map, we need - // to check if the device matches. - allocResources := p.allocDetails[alloc.ID].resources - allocDevice := allocResources.Flattened.Networks[0].Device - if allocDevice == device { - preemptedBandwidth += allocResources.Flattened.Networks[0].MBits - allocsToPreempt = append(allocsToPreempt, alloc) - } else { - // If its on a different device we continue to the - // outer loop because the current device cannot meet - // a needed, used reserved port + preemptedBandwidth += allocResources.Flattened.Networks[0].MBits + allocsToPreempt = append(allocsToPreempt, alloc) + } else { + // Check if a higher priority allocation is using this port + // It cant be preempted so we skip to the next device + _, ok := filteredReservedPorts[allocDevice][port.Value] + if ok { continue OUTER } } - - // If the reserved port is not in the usedPortToAlloc map it means - // no alloc is using it. } // Remove allocs that were preempted to satisfy reserved ports From 1380acbf564e6ae7e94f1f84a1a0b3bda36e20e8 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Fri, 2 Nov 2018 15:58:59 -0500 Subject: [PATCH 36/38] dereference safely --- scheduler/preemption.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 5fe974fc628..58d8d3a5a85 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -351,15 +351,14 @@ OUTER: // Look for allocs that are using reserved ports needed for _, port := range reservedPortsNeeded { alloc, ok := usedPortToAlloc[port.Value] - allocResources := p.allocDetails[alloc.ID].resources - allocDevice := allocResources.Flattened.Networks[0].Device if ok { + allocResources := p.allocDetails[alloc.ID].resources preemptedBandwidth += allocResources.Flattened.Networks[0].MBits allocsToPreempt = append(allocsToPreempt, alloc) } else { // Check if a higher priority allocation is using this port // It cant be preempted so we skip to the next device - _, ok := filteredReservedPorts[allocDevice][port.Value] + _, ok := filteredReservedPorts[device][port.Value] if ok { continue OUTER } From c49a3e20c5214e12436c79d5cb2906cfb6bb9e57 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Fri, 2 Nov 2018 16:06:25 -0500 Subject: [PATCH 37/38] Fix test setup --- scheduler/preemption_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scheduler/preemption_test.go b/scheduler/preemption_test.go index cdca7153cf1..cf6f1ef26c8 100644 --- a/scheduler/preemption_test.go +++ b/scheduler/preemption_test.go @@ -318,7 +318,7 @@ func TestPreemption(t *testing.T) { }, }, { - desc: "preempt only from device that has allocation with used reserved port", + desc: "preempt only from device that has allocation with unused reserved port", currentAllocations: []*structs.Allocation{ createAlloc(allocIDs[0], highPrioJob, &structs.Resources{ CPU: 1200, @@ -333,7 +333,7 @@ func TestPreemption(t *testing.T) { }, }, }), - createAlloc(allocIDs[1], lowPrioJob, &structs.Resources{ + createAlloc(allocIDs[1], highPrioJob, &structs.Resources{ CPU: 200, MemoryMB: 256, DiskMB: 4 * 1024, @@ -405,7 +405,7 @@ func TestPreemption(t *testing.T) { }, }, preemptedAllocIDs: map[string]struct{}{ - allocIDs[1]: {}, + allocIDs[2]: {}, }, }, { From 97cf4e1d7a058449813611c0924df0e4bdc86266 Mon Sep 17 00:00:00 2001 From: Preetha Appan Date: Fri, 2 Nov 2018 16:26:34 -0500 Subject: [PATCH 38/38] Address more minor code review feedback --- scheduler/preemption.go | 29 ++++++++++------------------- 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/scheduler/preemption.go b/scheduler/preemption.go index 58d8d3a5a85..fd082cdada3 100644 --- a/scheduler/preemption.go +++ b/scheduler/preemption.go @@ -143,7 +143,7 @@ func (p *Preemptor) SetCandidates(allocs []*structs.Allocation) { if tg != nil && tg.Migrate != nil { maxParallel = tg.Migrate.MaxParallel } - p.allocDetails[alloc.ID] = &allocInfo{maxParallel, alloc.ComparableResources()} + p.allocDetails[alloc.ID] = &allocInfo{maxParallel: maxParallel, resources: alloc.ComparableResources()} } } @@ -197,6 +197,7 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) // Initialize variable to track resources as they become available from preemption availableResources := p.nodeRemainingResources.Copy() + resourcesAsked := resourceAsk.Comparable() // Iterate over allocations grouped by priority to find preemptible allocations for _, allocGrp := range allocsByPriority { for len(allocGrp.allocs) > 0 && !allRequirementsMet { @@ -218,7 +219,7 @@ func (p *Preemptor) PreemptForTaskGroup(resourceAsk *structs.AllocatedResources) availableResources.Add(closestResources) // This step needs the original resources asked for as the second arg, can't use the running total - allRequirementsMet, _ = availableResources.Superset(resourceAsk.Comparable()) + allRequirementsMet, _ = availableResources.Superset(resourcesAsked) bestAllocs = append(bestAllocs, closestAlloc) @@ -262,11 +263,14 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc reservedPortsNeeded := networkResourceAsk.ReservedPorts // Build map of reserved ports needed for fast access - reservedPorts := make(map[int]interface{}) + reservedPorts := make(map[int]struct{}) for _, port := range reservedPortsNeeded { reservedPorts[port.Value] = struct{}{} } + // filteredReservedPorts tracks reserved ports that are + // currently used by higher priority allocations that can't + // be preempted filteredReservedPorts := make(map[string]map[int]struct{}) // Create a map from each device to allocs @@ -279,9 +283,8 @@ func (p *Preemptor) PreemptForNetwork(networkResourceAsk *structs.NetworkResourc // Filter out alloc that's ineligible due to priority if p.jobPriority-alloc.Job.Priority < 10 { - - // If this allocation uses a needed reserved port - // preemption is impossible so we return early + // Populate any reserved ports used by + // this allocation that cannot be preempted allocResources := p.allocDetails[alloc.ID].resources networks := allocResources.Flattened.Networks net := networks[0] @@ -333,6 +336,7 @@ OUTER: // Reset allocsToPreempt since we don't want to preempt across devices for the same task allocsToPreempt = nil + // usedPortToAlloc tracks used ports by allocs in this device usedPortToAlloc := make(map[int]*structs.Allocation) // First try to satisfy needed reserved ports @@ -431,19 +435,6 @@ OUTER: return filteredBestAllocs } -func usedReservedPorts(net *structs.NetworkResource, portMap map[int]interface{}) bool { - ports := net.ReservedPorts - if len(ports) > 0 { - for _, p := range ports { - _, ok := portMap[p.Value] - if ok { - return true - } - } - } - return false -} - // basicResourceDistance computes a distance using a coordinate system. It compares resource fields like CPU/Memory and Disk. // Values emitted are in the range [0, maxFloat] func basicResourceDistance(resourceAsk *structs.ComparableResources, resourceUsed *structs.ComparableResources) float64 {