2015-08-07 00:25:14 +00:00
|
|
|
package scheduler
|
|
|
|
|
|
|
|
import (
|
2015-08-07 00:46:14 +00:00
|
|
|
"fmt"
|
2022-11-17 23:59:33 +00:00
|
|
|
"runtime/debug"
|
2020-08-13 13:35:09 +00:00
|
|
|
"sort"
|
2017-07-07 18:42:51 +00:00
|
|
|
"time"
|
2015-08-07 00:25:14 +00:00
|
|
|
|
2018-09-15 23:23:13 +00:00
|
|
|
log "github.com/hashicorp/go-hclog"
|
2019-03-08 12:48:12 +00:00
|
|
|
"github.com/hashicorp/go-memdb"
|
|
|
|
"github.com/hashicorp/go-multierror"
|
2022-03-07 18:40:57 +00:00
|
|
|
"github.com/hashicorp/go-version"
|
2017-09-29 16:58:48 +00:00
|
|
|
"github.com/hashicorp/nomad/helper/uuid"
|
2015-08-07 00:25:14 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/structs"
|
|
|
|
)
|
|
|
|
|
2015-08-13 22:17:24 +00:00
|
|
|
const (
|
2015-08-14 05:35:48 +00:00
|
|
|
// maxServiceScheduleAttempts is used to limit the number of times
|
|
|
|
// we will attempt to schedule if we continue to hit conflicts for services.
|
|
|
|
maxServiceScheduleAttempts = 5
|
|
|
|
|
|
|
|
// maxBatchScheduleAttempts is used to limit the number of times
|
|
|
|
// we will attempt to schedule if we continue to hit conflicts for batch.
|
|
|
|
maxBatchScheduleAttempts = 2
|
2015-08-26 00:06:06 +00:00
|
|
|
|
|
|
|
// allocNotNeeded is the status used when a job no longer requires an allocation
|
|
|
|
allocNotNeeded = "alloc not needed due to job update"
|
|
|
|
|
2022-03-14 16:41:15 +00:00
|
|
|
// allocReconnected is the status to use when a replacement allocation is stopped
|
|
|
|
// because a disconnected node reconnects.
|
|
|
|
allocReconnected = "alloc not needed due to disconnected client reconnect"
|
|
|
|
|
2015-08-26 00:06:06 +00:00
|
|
|
// allocMigrating is the status used when we must migrate an allocation
|
|
|
|
allocMigrating = "alloc is being migrated"
|
|
|
|
|
|
|
|
// allocUpdating is the status used when a job requires an update
|
|
|
|
allocUpdating = "alloc is being updated due to job update"
|
2015-09-07 19:27:12 +00:00
|
|
|
|
2016-08-03 22:45:42 +00:00
|
|
|
// allocLost is the status used when an allocation is lost
|
|
|
|
allocLost = "alloc is lost since its node is down"
|
|
|
|
|
2022-02-16 18:50:20 +00:00
|
|
|
// allocUnknown is the status used when an allocation is unknown
|
|
|
|
allocUnknown = "alloc is unknown since its node is disconnected"
|
|
|
|
|
2015-09-07 19:27:12 +00:00
|
|
|
// allocInPlace is the status used when speculating on an in-place update
|
|
|
|
allocInPlace = "alloc updating in-place"
|
2016-05-25 21:11:14 +00:00
|
|
|
|
2020-10-09 21:31:38 +00:00
|
|
|
// allocNodeTainted is the status used when stopping an alloc because its
|
2018-03-27 00:54:46 +00:00
|
|
|
// node is tainted.
|
|
|
|
allocNodeTainted = "alloc not needed as node is tainted"
|
|
|
|
|
2019-06-06 19:04:32 +00:00
|
|
|
// allocRescheduled is the status used when an allocation failed and was rescheduled
|
|
|
|
allocRescheduled = "alloc was rescheduled because it failed"
|
|
|
|
|
2016-05-25 21:11:14 +00:00
|
|
|
// blockedEvalMaxPlanDesc is the description used for blocked evals that are
|
|
|
|
// a result of hitting the max number of plan attempts
|
|
|
|
blockedEvalMaxPlanDesc = "created due to placement conflicts"
|
|
|
|
|
|
|
|
// blockedEvalFailedPlacements is the description used for blocked evals
|
|
|
|
// that are a result of failing to place all allocations.
|
|
|
|
blockedEvalFailedPlacements = "created to place remaining allocations"
|
2018-03-08 00:44:54 +00:00
|
|
|
|
2018-04-10 20:30:15 +00:00
|
|
|
// reschedulingFollowupEvalDesc is the description used when creating follow
|
|
|
|
// up evals for delayed rescheduling
|
|
|
|
reschedulingFollowupEvalDesc = "created for delayed rescheduling"
|
|
|
|
|
2022-02-16 18:50:20 +00:00
|
|
|
// disconnectTimeoutFollowupEvalDesc is the description used when creating follow
|
|
|
|
// up evals for allocations that be should be stopped after its disconnect
|
|
|
|
// timeout has passed.
|
|
|
|
disconnectTimeoutFollowupEvalDesc = "created for delayed disconnect timeout"
|
|
|
|
|
2018-03-08 00:44:54 +00:00
|
|
|
// maxPastRescheduleEvents is the maximum number of past reschedule event
|
|
|
|
// that we track when unlimited rescheduling is enabled
|
|
|
|
maxPastRescheduleEvents = 5
|
2015-08-13 22:17:24 +00:00
|
|
|
)
|
|
|
|
|
2022-03-07 18:40:57 +00:00
|
|
|
// minVersionMaxClientDisconnect is the minimum version that supports max_client_disconnect.
|
2022-04-08 20:48:37 +00:00
|
|
|
var minVersionMaxClientDisconnect = version.Must(version.NewVersion("1.3.0"))
|
2022-03-07 18:40:57 +00:00
|
|
|
|
2015-08-15 21:47:13 +00:00
|
|
|
// SetStatusError is used to set the status of the evaluation to the given error
|
|
|
|
type SetStatusError struct {
|
|
|
|
Err error
|
|
|
|
EvalStatus string
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SetStatusError) Error() string {
|
|
|
|
return s.Err.Error()
|
|
|
|
}
|
|
|
|
|
2015-08-14 05:28:37 +00:00
|
|
|
// GenericScheduler is used for 'service' and 'batch' type jobs. This scheduler is
|
2017-12-13 17:36:03 +00:00
|
|
|
// designed for long-lived services, and as such spends more time attempting
|
2015-08-07 00:25:14 +00:00
|
|
|
// to make a high quality placement. This is the primary scheduler for
|
2015-08-14 05:28:37 +00:00
|
|
|
// most workloads. It also supports a 'batch' mode to optimize for fast decision
|
|
|
|
// making at the cost of quality.
|
|
|
|
type GenericScheduler struct {
|
2022-01-15 01:09:14 +00:00
|
|
|
logger log.Logger
|
|
|
|
eventsCh chan<- interface{}
|
|
|
|
state State
|
|
|
|
planner Planner
|
|
|
|
batch bool
|
2015-08-14 00:11:20 +00:00
|
|
|
|
2016-02-10 05:24:47 +00:00
|
|
|
eval *structs.Evaluation
|
|
|
|
job *structs.Job
|
|
|
|
plan *structs.Plan
|
|
|
|
planResult *structs.PlanResult
|
|
|
|
ctx *EvalContext
|
|
|
|
stack *GenericStack
|
2015-09-07 22:17:39 +00:00
|
|
|
|
2020-06-03 13:48:38 +00:00
|
|
|
// followUpEvals are evals with WaitUntil set, which are delayed until that time
|
|
|
|
// before being rescheduled
|
2018-03-12 20:44:33 +00:00
|
|
|
followUpEvals []*structs.Evaluation
|
2016-01-28 21:43:48 +00:00
|
|
|
|
2017-05-18 19:36:04 +00:00
|
|
|
deployment *structs.Deployment
|
|
|
|
|
2016-05-27 18:26:14 +00:00
|
|
|
blocked *structs.Evaluation
|
|
|
|
failedTGAllocs map[string]*structs.AllocMetric
|
2016-07-18 22:04:05 +00:00
|
|
|
queuedAllocs map[string]int
|
2015-08-07 00:25:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// NewServiceScheduler is a factory function to instantiate a new service scheduler
|
2022-01-15 01:09:14 +00:00
|
|
|
func NewServiceScheduler(logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) Scheduler {
|
2015-08-14 05:28:37 +00:00
|
|
|
s := &GenericScheduler{
|
2022-01-15 01:09:14 +00:00
|
|
|
logger: logger.Named("service_sched"),
|
|
|
|
eventsCh: eventsCh,
|
|
|
|
state: state,
|
|
|
|
planner: planner,
|
|
|
|
batch: false,
|
2015-08-14 05:28:37 +00:00
|
|
|
}
|
|
|
|
return s
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewBatchScheduler is a factory function to instantiate a new batch scheduler
|
2022-01-15 01:09:14 +00:00
|
|
|
func NewBatchScheduler(logger log.Logger, eventsCh chan<- interface{}, state State, planner Planner) Scheduler {
|
2015-08-14 05:28:37 +00:00
|
|
|
s := &GenericScheduler{
|
2022-01-15 01:09:14 +00:00
|
|
|
logger: logger.Named("batch_sched"),
|
|
|
|
eventsCh: eventsCh,
|
|
|
|
state: state,
|
|
|
|
planner: planner,
|
|
|
|
batch: true,
|
2015-08-07 00:25:14 +00:00
|
|
|
}
|
|
|
|
return s
|
|
|
|
}
|
|
|
|
|
|
|
|
// Process is used to handle a single evaluation
|
2022-02-07 16:47:53 +00:00
|
|
|
func (s *GenericScheduler) Process(eval *structs.Evaluation) (err error) {
|
|
|
|
|
|
|
|
defer func() {
|
|
|
|
if r := recover(); r != nil {
|
2022-11-17 23:59:33 +00:00
|
|
|
s.logger.Error("processing eval panicked scheduler - please report this as a bug!", "eval_id", eval.ID, "error", r, "stack_trace", string(debug.Stack()))
|
|
|
|
err = fmt.Errorf("failed to process eval: %v", r)
|
2022-02-07 16:47:53 +00:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2015-09-23 04:45:20 +00:00
|
|
|
// Store the evaluation
|
|
|
|
s.eval = eval
|
|
|
|
|
2018-09-15 23:23:13 +00:00
|
|
|
// Update our logger with the eval's information
|
|
|
|
s.logger = s.logger.With("eval_id", eval.ID, "job_id", eval.JobID, "namespace", eval.Namespace)
|
|
|
|
|
2015-08-14 00:40:23 +00:00
|
|
|
// Verify the evaluation trigger reason is understood
|
2015-08-07 00:46:14 +00:00
|
|
|
switch eval.TriggeredBy {
|
2018-01-24 00:47:00 +00:00
|
|
|
case structs.EvalTriggerJobRegister, structs.EvalTriggerJobDeregister,
|
|
|
|
structs.EvalTriggerNodeDrain, structs.EvalTriggerNodeUpdate,
|
2019-04-01 14:21:03 +00:00
|
|
|
structs.EvalTriggerAllocStop,
|
2018-09-24 21:47:49 +00:00
|
|
|
structs.EvalTriggerRollingUpdate, structs.EvalTriggerQueuedAllocs,
|
2017-07-04 19:24:27 +00:00
|
|
|
structs.EvalTriggerPeriodicJob, structs.EvalTriggerMaxPlans,
|
2018-09-21 22:28:18 +00:00
|
|
|
structs.EvalTriggerDeploymentWatcher, structs.EvalTriggerRetryFailedAlloc,
|
2020-01-17 16:51:35 +00:00
|
|
|
structs.EvalTriggerFailedFollowUp, structs.EvalTriggerPreemption,
|
2022-03-31 15:32:18 +00:00
|
|
|
structs.EvalTriggerScaling, structs.EvalTriggerMaxDisconnectTimeout, structs.EvalTriggerReconnect:
|
2015-08-07 00:46:14 +00:00
|
|
|
default:
|
2015-08-15 21:47:13 +00:00
|
|
|
desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason",
|
2015-08-07 00:46:14 +00:00
|
|
|
eval.TriggeredBy)
|
2018-03-12 20:44:33 +00:00
|
|
|
return setStatus(s.logger, s.planner, s.eval, nil, s.blocked,
|
2017-07-06 00:13:45 +00:00
|
|
|
s.failedTGAllocs, structs.EvalStatusFailed, desc, s.queuedAllocs,
|
|
|
|
s.deployment.GetID())
|
2015-08-07 00:46:14 +00:00
|
|
|
}
|
2015-08-14 00:40:23 +00:00
|
|
|
|
2016-02-10 05:24:47 +00:00
|
|
|
// Retry up to the maxScheduleAttempts and reset if progress is made.
|
|
|
|
progress := func() bool { return progressMade(s.planResult) }
|
2015-08-14 05:35:48 +00:00
|
|
|
limit := maxServiceScheduleAttempts
|
|
|
|
if s.batch {
|
|
|
|
limit = maxBatchScheduleAttempts
|
|
|
|
}
|
2016-02-10 05:24:47 +00:00
|
|
|
if err := retryMax(limit, s.process, progress); err != nil {
|
2015-08-15 21:47:13 +00:00
|
|
|
if statusErr, ok := err.(*SetStatusError); ok {
|
2016-02-10 05:24:47 +00:00
|
|
|
// Scheduling was tried but made no forward progress so create a
|
|
|
|
// blocked eval to retry once resources become available.
|
|
|
|
var mErr multierror.Error
|
2016-05-23 23:27:26 +00:00
|
|
|
if err := s.createBlockedEval(true); err != nil {
|
2016-02-10 05:24:47 +00:00
|
|
|
mErr.Errors = append(mErr.Errors, err)
|
|
|
|
}
|
2018-03-12 20:44:33 +00:00
|
|
|
if err := setStatus(s.logger, s.planner, s.eval, nil, s.blocked,
|
2016-07-18 22:04:05 +00:00
|
|
|
s.failedTGAllocs, statusErr.EvalStatus, err.Error(),
|
2017-07-06 00:13:45 +00:00
|
|
|
s.queuedAllocs, s.deployment.GetID()); err != nil {
|
2016-02-10 05:24:47 +00:00
|
|
|
mErr.Errors = append(mErr.Errors, err)
|
|
|
|
}
|
|
|
|
return mErr.ErrorOrNil()
|
2015-08-15 21:47:13 +00:00
|
|
|
}
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2016-05-20 23:03:53 +00:00
|
|
|
// If the current evaluation is a blocked evaluation and we didn't place
|
|
|
|
// everything, do not update the status to complete.
|
2016-05-27 18:26:14 +00:00
|
|
|
if s.eval.Status == structs.EvalStatusBlocked && len(s.failedTGAllocs) != 0 {
|
2016-06-10 22:24:06 +00:00
|
|
|
e := s.ctx.Eligibility()
|
|
|
|
newEval := s.eval.Copy()
|
|
|
|
newEval.EscapedComputedClass = e.HasEscaped()
|
|
|
|
newEval.ClassEligibility = e.GetClasses()
|
2017-10-13 21:36:02 +00:00
|
|
|
newEval.QuotaLimitReached = e.QuotaLimitReached()
|
2016-06-10 22:24:06 +00:00
|
|
|
return s.planner.ReblockEval(newEval)
|
2016-05-20 23:03:53 +00:00
|
|
|
}
|
|
|
|
|
2015-08-15 21:47:13 +00:00
|
|
|
// Update the status to complete
|
2018-03-12 20:44:33 +00:00
|
|
|
return setStatus(s.logger, s.planner, s.eval, nil, s.blocked,
|
2017-07-06 00:13:45 +00:00
|
|
|
s.failedTGAllocs, structs.EvalStatusComplete, "", s.queuedAllocs,
|
|
|
|
s.deployment.GetID())
|
2015-08-07 00:46:14 +00:00
|
|
|
}
|
|
|
|
|
2016-05-23 23:27:26 +00:00
|
|
|
// createBlockedEval creates a blocked eval and submits it to the planner. If
|
|
|
|
// failure is set to true, the eval's trigger reason reflects that.
|
|
|
|
func (s *GenericScheduler) createBlockedEval(planFailure bool) error {
|
2016-02-11 17:45:27 +00:00
|
|
|
e := s.ctx.Eligibility()
|
|
|
|
escaped := e.HasEscaped()
|
|
|
|
|
|
|
|
// Only store the eligible classes if the eval hasn't escaped.
|
|
|
|
var classEligibility map[string]bool
|
|
|
|
if !escaped {
|
|
|
|
classEligibility = e.GetClasses()
|
2016-02-10 05:24:47 +00:00
|
|
|
}
|
|
|
|
|
2021-04-29 19:03:45 +00:00
|
|
|
s.blocked = s.eval.CreateBlockedEval(classEligibility, escaped, e.QuotaLimitReached(), s.failedTGAllocs)
|
2016-05-23 23:27:26 +00:00
|
|
|
if planFailure {
|
|
|
|
s.blocked.TriggeredBy = structs.EvalTriggerMaxPlans
|
2016-05-25 21:11:14 +00:00
|
|
|
s.blocked.StatusDescription = blockedEvalMaxPlanDesc
|
|
|
|
} else {
|
|
|
|
s.blocked.StatusDescription = blockedEvalFailedPlacements
|
2016-05-23 23:27:26 +00:00
|
|
|
}
|
|
|
|
|
2016-02-10 05:24:47 +00:00
|
|
|
return s.planner.CreateEval(s.blocked)
|
|
|
|
}
|
|
|
|
|
2015-08-14 00:40:23 +00:00
|
|
|
// process is wrapped in retryMax to iteratively run the handler until we have no
|
2015-08-14 00:11:20 +00:00
|
|
|
// further work or we've made the maximum number of attempts.
|
2015-08-14 05:28:37 +00:00
|
|
|
func (s *GenericScheduler) process() (bool, error) {
|
2015-08-11 23:41:48 +00:00
|
|
|
// Lookup the Job by ID
|
2015-09-07 19:27:12 +00:00
|
|
|
var err error
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
2017-09-07 23:56:15 +00:00
|
|
|
s.job, err = s.state.JobByID(ws, s.eval.Namespace, s.eval.JobID)
|
2015-08-11 23:41:48 +00:00
|
|
|
if err != nil {
|
2017-05-18 19:36:04 +00:00
|
|
|
return false, fmt.Errorf("failed to get job %q: %v", s.eval.JobID, err)
|
2015-08-14 00:11:20 +00:00
|
|
|
}
|
2017-05-18 19:36:04 +00:00
|
|
|
|
2016-07-26 05:11:11 +00:00
|
|
|
numTaskGroups := 0
|
2017-05-18 19:36:04 +00:00
|
|
|
stopped := s.job.Stopped()
|
|
|
|
if !stopped {
|
2016-07-26 05:11:11 +00:00
|
|
|
numTaskGroups = len(s.job.TaskGroups)
|
|
|
|
}
|
|
|
|
s.queuedAllocs = make(map[string]int, numTaskGroups)
|
2018-03-08 00:44:54 +00:00
|
|
|
s.followUpEvals = nil
|
2015-08-14 00:11:20 +00:00
|
|
|
|
|
|
|
// Create a plan
|
2019-03-08 12:48:12 +00:00
|
|
|
s.plan = s.eval.MakePlan(s.job)
|
2015-08-14 00:11:20 +00:00
|
|
|
|
2017-05-18 19:36:04 +00:00
|
|
|
if !s.batch {
|
|
|
|
// Get any existing deployment
|
2017-09-07 23:56:15 +00:00
|
|
|
s.deployment, err = s.state.LatestDeploymentByJobID(ws, s.eval.Namespace, s.eval.JobID)
|
2017-05-18 19:36:04 +00:00
|
|
|
if err != nil {
|
|
|
|
return false, fmt.Errorf("failed to get job deployment %q: %v", s.eval.JobID, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-05-19 01:11:40 +00:00
|
|
|
// Reset the failed allocations
|
2016-05-27 18:26:14 +00:00
|
|
|
s.failedTGAllocs = nil
|
2016-05-19 01:11:40 +00:00
|
|
|
|
2015-09-07 18:34:59 +00:00
|
|
|
// Create an evaluation context
|
2022-01-15 01:09:14 +00:00
|
|
|
s.ctx = NewEvalContext(s.eventsCh, s.state, s.plan, s.logger)
|
2015-09-07 18:34:59 +00:00
|
|
|
|
|
|
|
// Construct the placement stack
|
2015-10-17 00:05:23 +00:00
|
|
|
s.stack = NewGenericStack(s.batch, s.ctx)
|
2017-04-19 17:54:03 +00:00
|
|
|
if !s.job.Stopped() {
|
2015-09-07 19:27:12 +00:00
|
|
|
s.stack.SetJob(s.job)
|
|
|
|
}
|
2015-09-07 18:34:59 +00:00
|
|
|
|
2015-08-14 00:40:23 +00:00
|
|
|
// Compute the target job allocations
|
2017-05-23 23:39:15 +00:00
|
|
|
if err := s.computeJobAllocs(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to compute job allocations", "error", err)
|
2015-08-14 00:40:23 +00:00
|
|
|
return false, err
|
2015-08-11 23:41:48 +00:00
|
|
|
}
|
|
|
|
|
2016-05-19 01:11:40 +00:00
|
|
|
// If there are failed allocations, we need to create a blocked evaluation
|
2016-05-20 23:03:53 +00:00
|
|
|
// to place the failed allocations when resources become available. If the
|
2020-06-03 13:48:38 +00:00
|
|
|
// current evaluation is already a blocked eval, we reuse it. If not, submit
|
|
|
|
// a new eval to the planner in createBlockedEval. If rescheduling should
|
|
|
|
// be delayed, do that instead.
|
|
|
|
delayInstead := len(s.followUpEvals) > 0 && s.eval.WaitUntil.IsZero()
|
|
|
|
|
2020-05-13 20:39:04 +00:00
|
|
|
if s.eval.Status != structs.EvalStatusBlocked && len(s.failedTGAllocs) != 0 && s.blocked == nil &&
|
2020-06-03 13:48:38 +00:00
|
|
|
!delayInstead {
|
2016-05-23 23:27:26 +00:00
|
|
|
if err := s.createBlockedEval(false); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to make blocked eval", "error", err)
|
2016-05-19 01:11:40 +00:00
|
|
|
return false, err
|
|
|
|
}
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Debug("failed to place all allocations, blocked eval created", "blocked_eval_id", s.blocked.ID)
|
2016-05-19 01:11:40 +00:00
|
|
|
}
|
|
|
|
|
2016-05-05 18:21:58 +00:00
|
|
|
// If the plan is a no-op, we can bail. If AnnotatePlan is set submit the plan
|
|
|
|
// anyways to get the annotations.
|
|
|
|
if s.plan.IsNoOp() && !s.eval.AnnotatePlan {
|
2015-08-14 01:16:32 +00:00
|
|
|
return true, nil
|
|
|
|
}
|
|
|
|
|
2020-06-03 13:48:38 +00:00
|
|
|
// Create follow up evals for any delayed reschedule eligible allocations, except in
|
|
|
|
// the case that this evaluation was already delayed.
|
|
|
|
if delayInstead {
|
2018-03-02 00:23:44 +00:00
|
|
|
for _, eval := range s.followUpEvals {
|
|
|
|
eval.PreviousEval = s.eval.ID
|
|
|
|
// TODO(preetha) this should be batching evals before inserting them
|
|
|
|
if err := s.planner.CreateEval(eval); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to make next eval for rescheduling", "error", err)
|
2018-03-02 00:23:44 +00:00
|
|
|
return false, err
|
|
|
|
}
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Debug("found reschedulable allocs, followup eval created", "followup_eval_id", eval.ID)
|
2018-03-02 00:23:44 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-02-10 05:24:47 +00:00
|
|
|
// Submit the plan and store the results.
|
2015-08-14 00:11:20 +00:00
|
|
|
result, newState, err := s.planner.SubmitPlan(s.plan)
|
2016-02-10 05:24:47 +00:00
|
|
|
s.planResult = result
|
2015-08-14 00:11:20 +00:00
|
|
|
if err != nil {
|
2015-08-14 00:40:23 +00:00
|
|
|
return false, err
|
2015-08-14 00:11:20 +00:00
|
|
|
}
|
2016-07-22 06:13:07 +00:00
|
|
|
|
|
|
|
// Decrement the number of allocations pending per task group based on the
|
|
|
|
// number of allocations successfully placed
|
2016-07-22 21:53:49 +00:00
|
|
|
adjustQueuedAllocations(s.logger, result, s.queuedAllocs)
|
2015-08-14 00:11:20 +00:00
|
|
|
|
|
|
|
// If we got a state refresh, try again since we have stale data
|
|
|
|
if newState != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Debug("refresh forced")
|
2015-08-14 00:11:20 +00:00
|
|
|
s.state = newState
|
2015-08-14 00:40:23 +00:00
|
|
|
return false, nil
|
2015-08-14 00:11:20 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Try again if the plan was not fully committed, potential conflict
|
|
|
|
fullCommit, expected, actual := result.FullCommit(s.plan)
|
|
|
|
if !fullCommit {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Debug("plan didn't fully commit", "attempted", expected, "placed", actual)
|
2016-02-22 20:15:40 +00:00
|
|
|
if newState == nil {
|
|
|
|
return false, fmt.Errorf("missing state refresh after partial commit")
|
|
|
|
}
|
2015-08-14 00:40:23 +00:00
|
|
|
return false, nil
|
2015-08-14 00:11:20 +00:00
|
|
|
}
|
2015-08-14 00:40:23 +00:00
|
|
|
|
|
|
|
// Success!
|
|
|
|
return true, nil
|
2015-08-14 00:11:20 +00:00
|
|
|
}
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
// computeJobAllocs is used to reconcile differences between the job,
|
|
|
|
// existing allocations and node status to update the allocations.
|
2017-05-23 23:39:15 +00:00
|
|
|
func (s *GenericScheduler) computeJobAllocs() error {
|
2017-05-22 17:58:34 +00:00
|
|
|
// Lookup the allocations by JobID
|
|
|
|
ws := memdb.NewWatchSet()
|
2017-09-07 23:56:15 +00:00
|
|
|
allocs, err := s.state.AllocsByJob(ws, s.eval.Namespace, s.eval.JobID, true)
|
2017-05-22 17:58:34 +00:00
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to get allocs for job '%s': %v",
|
|
|
|
s.eval.JobID, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Determine the tainted nodes containing job allocs
|
|
|
|
tainted, err := taintedNodes(s.state, allocs)
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to get tainted nodes for job '%s': %v",
|
|
|
|
s.eval.JobID, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the allocations which are in pending/running state on tainted
|
2020-05-13 20:39:04 +00:00
|
|
|
// nodes to lost, but only if the scheduler has already marked them
|
2017-05-22 17:58:34 +00:00
|
|
|
updateNonTerminalAllocsToLost(s.plan, tainted, allocs)
|
|
|
|
|
2018-09-15 23:23:13 +00:00
|
|
|
reconciler := NewAllocReconciler(s.logger,
|
2017-06-21 20:26:45 +00:00
|
|
|
genericAllocUpdateFn(s.ctx, s.stack, s.eval.ID),
|
2022-03-07 18:40:57 +00:00
|
|
|
s.batch, s.eval.JobID, s.job, s.deployment, allocs, tainted, s.eval.ID,
|
|
|
|
s.eval.Priority, s.planner.ServersMeetMinimumVersion(minVersionMaxClientDisconnect, true))
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
results := reconciler.Compute()
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Debug("reconciled current state with desired state", "results", log.Fmt("%#v", results))
|
2017-05-22 17:58:34 +00:00
|
|
|
|
2017-05-23 20:02:47 +00:00
|
|
|
if s.eval.AnnotatePlan {
|
|
|
|
s.plan.Annotations = &structs.PlanAnnotations{
|
|
|
|
DesiredTGUpdates: results.desiredTGUpdates,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
// Add the deployment changes to the plan
|
2017-07-04 20:31:01 +00:00
|
|
|
s.plan.Deployment = results.deployment
|
2017-05-22 17:58:34 +00:00
|
|
|
s.plan.DeploymentUpdates = results.deploymentUpdates
|
|
|
|
|
2018-03-02 00:23:44 +00:00
|
|
|
// Store all the follow up evaluations from rescheduled allocations
|
|
|
|
if len(results.desiredFollowupEvals) > 0 {
|
|
|
|
for _, evals := range results.desiredFollowupEvals {
|
|
|
|
s.followUpEvals = append(s.followUpEvals, evals...)
|
|
|
|
}
|
|
|
|
}
|
2018-03-12 20:44:33 +00:00
|
|
|
|
2017-05-23 00:02:20 +00:00
|
|
|
// Update the stored deployment
|
2017-07-04 20:31:01 +00:00
|
|
|
if results.deployment != nil {
|
|
|
|
s.deployment = results.deployment
|
2017-05-23 00:02:20 +00:00
|
|
|
}
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
// Handle the stop
|
|
|
|
for _, stop := range results.stop {
|
2020-06-09 21:13:53 +00:00
|
|
|
s.plan.AppendStoppedAlloc(stop.alloc, stop.statusDescription, stop.clientStatus, stop.followupEvalID)
|
2017-05-22 17:58:34 +00:00
|
|
|
}
|
|
|
|
|
2022-02-16 18:50:20 +00:00
|
|
|
// Handle disconnect updates
|
|
|
|
for _, update := range results.disconnectUpdates {
|
|
|
|
s.plan.AppendUnknownAlloc(update)
|
|
|
|
}
|
|
|
|
|
Update alloc after reconnect and enforece client heartbeat order (#15068)
* scheduler: allow updates after alloc reconnects
When an allocation reconnects to a cluster the scheduler needs to run
special logic to handle the reconnection, check if a replacement was
create and stop one of them.
If the allocation kept running while the node was disconnected, it will
be reconnected with `ClientStatus: running` and the node will have
`Status: ready`. This combination is the same as the normal steady state
of allocation, where everything is running as expected.
In order to differentiate between the two states (an allocation that is
reconnecting and one that is just running) the scheduler needs an extra
piece of state.
The current implementation uses the presence of a
`TaskClientReconnected` task event to detect when the allocation has
reconnected and thus must go through the reconnection process. But this
event remains even after the allocation is reconnected, causing all
future evals to consider the allocation as still reconnecting.
This commit changes the reconnect logic to use an `AllocState` to
register when the allocation was reconnected. This provides the
following benefits:
- Only a limited number of task states are kept, and they are used for
many other events. It's possible that, upon reconnecting, several
actions are triggered that could cause the `TaskClientReconnected`
event to be dropped.
- Task events are set by clients and so their timestamps are subject
to time skew from servers. This prevents using time to determine if
an allocation reconnected after a disconnect event.
- Disconnect events are already stored as `AllocState` and so storing
reconnects there as well makes it the only source of information
required.
With the new logic, the reconnection logic is only triggered if the
last `AllocState` is a disconnect event, meaning that the allocation has
not been reconnected yet. After the reconnection is handled, the new
`ClientStatus` is store in `AllocState` allowing future evals to skip
the reconnection logic.
* scheduler: prevent spurious placement on reconnect
When a client reconnects it makes two independent RPC calls:
- `Node.UpdateStatus` to heartbeat and set its status as `ready`.
- `Node.UpdateAlloc` to update the status of its allocations.
These two calls can happen in any order, and in case the allocations are
updated before a heartbeat it causes the state to be the same as a node
being disconnected: the node status will still be `disconnected` while
the allocation `ClientStatus` is set to `running`.
The current implementation did not handle this order of events properly,
and the scheduler would create an unnecessary placement since it
considered the allocation was being disconnected. This extra allocation
would then be quickly stopped by the heartbeat eval.
This commit adds a new code path to handle this order of events. If the
node is `disconnected` and the allocation `ClientStatus` is `running`
the scheduler will check if the allocation is actually reconnecting
using its `AllocState` events.
* rpc: only allow alloc updates from `ready` nodes
Clients interact with servers using three main RPC methods:
- `Node.GetAllocs` reads allocation data from the server and writes it
to the client.
- `Node.UpdateAlloc` reads allocation from from the client and writes
them to the server.
- `Node.UpdateStatus` writes the client status to the server and is
used as the heartbeat mechanism.
These three methods are called periodically by the clients and are done
so independently from each other, meaning that there can't be any
assumptions in their ordering.
This can generate scenarios that are hard to reason about and to code
for. For example, when a client misses too many heartbeats it will be
considered `down` or `disconnected` and the allocations it was running
are set to `lost` or `unknown`.
When connectivity is restored the to rest of the cluster, the natural
mental model is to think that the client will heartbeat first and then
update its allocations status into the servers.
But since there's no inherit order in these calls the reverse is just as
possible: the client updates the alloc status and then heartbeats. This
results in a state where allocs are, for example, `running` while the
client is still `disconnected`.
This commit adds a new verification to the `Node.UpdateAlloc` method to
reject updates from nodes that are not `ready`, forcing clients to
heartbeat first. Since this check is done server-side there is no need
to coordinate operations client-side: they can continue sending these
requests independently and alloc update will succeed after the heartbeat
is done.
* chagelog: add entry for #15068
* code review
* client: skip terminal allocations on reconnect
When the client reconnects with the server it synchronizes the state of
its allocations by sending data using the `Node.UpdateAlloc` RPC and
fetching data using the `Node.GetClientAllocs` RPC.
If the data fetch happens before the data write, `unknown` allocations
will still be in this state and would trigger the
`allocRunner.Reconnect` flow.
But when the server `DesiredStatus` for the allocation is `stop` the
client should not reconnect the allocation.
* apply more code review changes
* scheduler: persist changes to reconnected allocs
Reconnected allocs have a new AllocState entry that must be persisted by
the plan applier.
* rpc: read node ID from allocs in UpdateAlloc
The AllocUpdateRequest struct is used in three disjoint use cases:
1. Stripped allocs from clients Node.UpdateAlloc RPC using the Allocs,
and WriteRequest fields
2. Raft log message using the Allocs, Evals, and WriteRequest fields
3. Plan updates using the AllocsStopped, AllocsUpdated, and Job fields
Adding a new field that would only be used in one these cases (1) made
things more confusing and error prone. While in theory an
AllocUpdateRequest could send allocations from different nodes, in
practice this never actually happens since only clients call this method
with their own allocations.
* scheduler: remove logic to handle exceptional case
This condition could only be hit if, somehow, the allocation status was
set to "running" while the client was "unknown". This was addressed by
enforcing an order in "Node.UpdateStatus" and "Node.UpdateAlloc" RPC
calls, so this scenario is not expected to happen.
Adding unnecessary code to the scheduler makes it harder to read and
reason about it.
* more code review
* remove another unused test
2022-11-04 20:25:11 +00:00
|
|
|
// Handle reconnect updates.
|
|
|
|
// Reconnected allocs have a new AllocState entry.
|
|
|
|
for _, update := range results.reconnectUpdates {
|
|
|
|
s.ctx.Plan().AppendAlloc(update, nil)
|
|
|
|
}
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
// Handle the in-place updates
|
|
|
|
for _, update := range results.inplaceUpdate {
|
2017-07-06 00:13:45 +00:00
|
|
|
if update.DeploymentID != s.deployment.GetID() {
|
|
|
|
update.DeploymentID = s.deployment.GetID()
|
2017-07-05 19:50:40 +00:00
|
|
|
update.DeploymentStatus = nil
|
|
|
|
}
|
2020-08-25 21:09:21 +00:00
|
|
|
s.ctx.Plan().AppendAlloc(update, nil)
|
2017-05-22 17:58:34 +00:00
|
|
|
}
|
|
|
|
|
2018-03-23 23:55:21 +00:00
|
|
|
// Handle the annotation updates
|
2018-03-26 18:06:21 +00:00
|
|
|
for _, update := range results.attributeUpdates {
|
2020-08-25 21:09:21 +00:00
|
|
|
s.ctx.Plan().AppendAlloc(update, nil)
|
2018-03-23 23:55:21 +00:00
|
|
|
}
|
|
|
|
|
2017-05-23 20:02:47 +00:00
|
|
|
// Nothing remaining to do if placement is not required
|
2017-07-15 23:31:33 +00:00
|
|
|
if len(results.place)+len(results.destructiveUpdate) == 0 {
|
2018-06-13 17:46:39 +00:00
|
|
|
// If the job has been purged we don't have access to the job. Otherwise
|
|
|
|
// set the queued allocs to zero. This is true if the job is being
|
|
|
|
// stopped as well.
|
|
|
|
if s.job != nil {
|
2017-05-23 20:02:47 +00:00
|
|
|
for _, tg := range s.job.TaskGroups {
|
|
|
|
s.queuedAllocs[tg.Name] = 0
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
// Compute the placements
|
2017-07-20 19:23:46 +00:00
|
|
|
place := make([]placementResult, 0, len(results.place))
|
2017-07-15 23:31:33 +00:00
|
|
|
for _, p := range results.place {
|
2022-02-10 21:24:51 +00:00
|
|
|
s.queuedAllocs[p.taskGroup.Name] += 1
|
2017-07-15 23:31:33 +00:00
|
|
|
place = append(place, p)
|
|
|
|
}
|
2017-07-20 19:23:46 +00:00
|
|
|
|
|
|
|
destructive := make([]placementResult, 0, len(results.destructiveUpdate))
|
2017-07-15 23:31:33 +00:00
|
|
|
for _, p := range results.destructiveUpdate {
|
2022-02-10 21:24:51 +00:00
|
|
|
s.queuedAllocs[p.placeTaskGroup.Name] += 1
|
2017-07-20 19:23:46 +00:00
|
|
|
destructive = append(destructive, p)
|
2017-07-15 23:31:33 +00:00
|
|
|
}
|
2017-07-20 19:23:46 +00:00
|
|
|
return s.computePlacements(destructive, place)
|
2015-08-14 01:16:32 +00:00
|
|
|
}
|
|
|
|
|
2020-08-13 13:35:09 +00:00
|
|
|
// downgradedJobForPlacement returns the job appropriate for non-canary placement replacement
|
|
|
|
func (s *GenericScheduler) downgradedJobForPlacement(p placementResult) (string, *structs.Job, error) {
|
|
|
|
ns, jobID := s.job.Namespace, s.job.ID
|
|
|
|
tgName := p.TaskGroup().Name
|
|
|
|
|
|
|
|
// find deployments and use the latest promoted or canaried version
|
|
|
|
deployments, err := s.state.DeploymentsByJobID(nil, ns, jobID, false)
|
|
|
|
if err != nil {
|
|
|
|
return "", nil, fmt.Errorf("failed to lookup job deployments: %v", err)
|
|
|
|
}
|
2020-12-08 17:57:47 +00:00
|
|
|
|
|
|
|
sort.Slice(deployments, func(i, j int) bool {
|
|
|
|
return deployments[i].JobVersion > deployments[j].JobVersion
|
|
|
|
})
|
|
|
|
|
2020-08-13 13:35:09 +00:00
|
|
|
for _, d := range deployments {
|
|
|
|
// It's unexpected to have a recent deployment that doesn't contain the TaskGroup; as all allocations
|
|
|
|
// should be destroyed. In such cases, attempt to find the deployment for that TaskGroup and hopefully
|
|
|
|
// we will kill it soon. This is a defensive measure, have not seen it in practice
|
|
|
|
//
|
|
|
|
// Zero dstate.DesiredCanaries indicates that the TaskGroup allocates were updated in-place without using canaries.
|
|
|
|
if dstate := d.TaskGroups[tgName]; dstate != nil && (dstate.Promoted || dstate.DesiredCanaries == 0) {
|
|
|
|
job, err := s.state.JobByIDAndVersion(nil, ns, jobID, d.JobVersion)
|
|
|
|
return d.ID, job, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-08-26 14:36:34 +00:00
|
|
|
// check if the non-promoted version is a job without update stanza. This version should be the latest "stable" version,
|
|
|
|
// as all subsequent versions must be canaried deployments. Otherwise, we would have found a deployment above,
|
|
|
|
// or the alloc would have been replaced already by a newer non-deployment job.
|
|
|
|
if job, err := s.state.JobByIDAndVersion(nil, ns, jobID, p.MinJobVersion()); err == nil && job != nil && job.Update.IsEmpty() {
|
|
|
|
return "", job, err
|
|
|
|
}
|
|
|
|
|
2020-08-13 13:35:09 +00:00
|
|
|
return "", nil, nil
|
|
|
|
}
|
|
|
|
|
2017-07-20 19:31:08 +00:00
|
|
|
// computePlacements computes placements for allocations. It is given the set of
|
|
|
|
// destructive updates to place and the set of new placements to place.
|
2017-07-20 19:23:46 +00:00
|
|
|
func (s *GenericScheduler) computePlacements(destructive, place []placementResult) error {
|
2017-05-22 17:58:34 +00:00
|
|
|
// Get the base nodes
|
2021-10-27 14:04:13 +00:00
|
|
|
nodes, _, byDC, err := readyNodesInDCs(s.state, s.job.Datacenters)
|
2017-05-22 17:58:34 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var deploymentID string
|
2018-04-05 22:40:49 +00:00
|
|
|
if s.deployment != nil && s.deployment.Active() {
|
2017-05-22 17:58:34 +00:00
|
|
|
deploymentID = s.deployment.ID
|
|
|
|
}
|
|
|
|
|
2017-12-13 17:36:03 +00:00
|
|
|
// Update the set of placement nodes
|
2017-05-22 17:58:34 +00:00
|
|
|
s.stack.SetNodes(nodes)
|
|
|
|
|
2018-03-12 19:32:43 +00:00
|
|
|
// Capture current time to use as the start time for any rescheduled allocations
|
|
|
|
now := time.Now()
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Have to handle destructive changes first as we need to discount their
|
|
|
|
// resources. To understand this imagine the resources were reduced and the
|
|
|
|
// count was scaled up.
|
|
|
|
for _, results := range [][]placementResult{destructive, place} {
|
|
|
|
for _, missing := range results {
|
|
|
|
// Get the task group
|
|
|
|
tg := missing.TaskGroup()
|
|
|
|
|
2020-08-13 13:35:09 +00:00
|
|
|
var downgradedJob *structs.Job
|
|
|
|
|
|
|
|
if missing.DowngradeNonCanary() {
|
|
|
|
jobDeploymentID, job, err := s.downgradedJobForPlacement(missing)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Defensive check - if there is no appropriate deployment for this job, use the latest
|
|
|
|
if job != nil && job.Version >= missing.MinJobVersion() && job.LookupTaskGroup(tg.Name) != nil {
|
|
|
|
tg = job.LookupTaskGroup(tg.Name)
|
|
|
|
downgradedJob = job
|
|
|
|
deploymentID = jobDeploymentID
|
|
|
|
} else {
|
|
|
|
jobVersion := -1
|
|
|
|
if job != nil {
|
|
|
|
jobVersion = int(job.Version)
|
|
|
|
}
|
2020-08-25 21:23:20 +00:00
|
|
|
s.logger.Debug("failed to find appropriate job; using the latest", "expected_version", missing.MinJobVersion, "found_version", jobVersion)
|
2020-08-13 13:35:09 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Check if this task group has already failed
|
|
|
|
if metric, ok := s.failedTGAllocs[tg.Name]; ok {
|
|
|
|
metric.CoalescedFailures += 1
|
2021-04-29 19:03:45 +00:00
|
|
|
metric.ExhaustResources(tg)
|
2017-07-20 19:23:46 +00:00
|
|
|
continue
|
|
|
|
}
|
2017-07-18 00:18:12 +00:00
|
|
|
|
2020-08-25 21:23:20 +00:00
|
|
|
// Use downgraded job in scheduling stack to honor
|
|
|
|
// old job resources and constraints
|
|
|
|
if downgradedJob != nil {
|
|
|
|
s.stack.SetJob(downgradedJob)
|
|
|
|
}
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Find the preferred node
|
|
|
|
preferredNode, err := s.findPreferredNode(missing)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-05-22 17:58:34 +00:00
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Check if we should stop the previous allocation upon successful
|
|
|
|
// placement of its replacement. This allow atomic placements/stops. We
|
|
|
|
// stop the allocation before trying to find a replacement because this
|
|
|
|
// frees the resources currently used by the previous allocation.
|
|
|
|
stopPrevAlloc, stopPrevAllocDesc := missing.StopPreviousAlloc()
|
2018-01-14 22:47:21 +00:00
|
|
|
prevAllocation := missing.PreviousAllocation()
|
2017-07-20 19:23:46 +00:00
|
|
|
if stopPrevAlloc {
|
2020-06-09 21:13:53 +00:00
|
|
|
s.plan.AppendStoppedAlloc(prevAllocation, stopPrevAllocDesc, "", "")
|
2018-01-14 22:47:21 +00:00
|
|
|
}
|
|
|
|
|
2018-01-16 14:01:36 +00:00
|
|
|
// Compute penalty nodes for rescheduled allocs
|
2018-01-19 14:41:53 +00:00
|
|
|
selectOptions := getSelectOptions(prevAllocation, preferredNode)
|
2021-03-18 19:35:11 +00:00
|
|
|
selectOptions.AllocName = missing.Name()
|
2019-04-11 01:20:22 +00:00
|
|
|
option := s.selectNextOption(tg, selectOptions)
|
2017-05-22 17:58:34 +00:00
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Store the available nodes by datacenter
|
|
|
|
s.ctx.Metrics().NodesAvailable = byDC
|
|
|
|
|
2018-08-08 14:41:56 +00:00
|
|
|
// Compute top K scoring node metadata
|
|
|
|
s.ctx.Metrics().PopulateScoreMetaData()
|
|
|
|
|
2020-08-25 21:23:20 +00:00
|
|
|
// Restore stack job now that placement is done, to use plan job version
|
2020-08-13 13:35:09 +00:00
|
|
|
if downgradedJob != nil {
|
|
|
|
s.stack.SetJob(s.job)
|
|
|
|
}
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Set fields based on if we found an allocation option
|
|
|
|
if option != nil {
|
2018-10-02 20:36:04 +00:00
|
|
|
resources := &structs.AllocatedResources{
|
2019-12-16 20:34:58 +00:00
|
|
|
Tasks: option.TaskResources,
|
|
|
|
TaskLifecycles: option.TaskLifecycles,
|
2019-06-18 17:12:23 +00:00
|
|
|
Shared: structs.AllocatedSharedResources{
|
|
|
|
DiskMB: int64(tg.EphemeralDisk.SizeMB),
|
|
|
|
},
|
2019-06-18 04:55:43 +00:00
|
|
|
}
|
|
|
|
if option.AllocResources != nil {
|
2019-06-18 17:12:23 +00:00
|
|
|
resources.Shared.Networks = option.AllocResources.Networks
|
2020-06-16 15:53:10 +00:00
|
|
|
resources.Shared.Ports = option.AllocResources.Ports
|
2018-10-02 20:36:04 +00:00
|
|
|
}
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Create an allocation for this
|
|
|
|
alloc := &structs.Allocation{
|
2018-10-02 20:36:04 +00:00
|
|
|
ID: uuid.Generate(),
|
|
|
|
Namespace: s.job.Namespace,
|
|
|
|
EvalID: s.eval.ID,
|
|
|
|
Name: missing.Name(),
|
|
|
|
JobID: s.job.ID,
|
|
|
|
TaskGroup: tg.Name,
|
|
|
|
Metrics: s.ctx.Metrics(),
|
|
|
|
NodeID: option.Node.ID,
|
2018-07-17 10:03:13 +00:00
|
|
|
NodeName: option.Node.Name,
|
2018-10-02 20:36:04 +00:00
|
|
|
DeploymentID: deploymentID,
|
|
|
|
TaskResources: resources.OldTaskResources(),
|
|
|
|
AllocatedResources: resources,
|
|
|
|
DesiredStatus: structs.AllocDesiredStatusRun,
|
|
|
|
ClientStatus: structs.AllocClientStatusPending,
|
2019-09-05 23:37:24 +00:00
|
|
|
// SharedResources is considered deprecated, will be removed in 0.11.
|
|
|
|
// It is only set for compat reasons.
|
2017-07-20 19:23:46 +00:00
|
|
|
SharedResources: &structs.Resources{
|
2019-06-11 04:29:12 +00:00
|
|
|
DiskMB: tg.EphemeralDisk.SizeMB,
|
2019-09-05 23:37:24 +00:00
|
|
|
Networks: resources.Shared.Networks,
|
2017-07-20 19:23:46 +00:00
|
|
|
},
|
2017-07-05 19:50:40 +00:00
|
|
|
}
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// If the new allocation is replacing an older allocation then we
|
|
|
|
// set the record the older allocation id so that they are chained
|
2018-01-19 14:41:53 +00:00
|
|
|
if prevAllocation != nil {
|
|
|
|
alloc.PreviousAllocation = prevAllocation.ID
|
2018-01-19 21:20:00 +00:00
|
|
|
if missing.IsRescheduling() {
|
2018-03-08 15:36:01 +00:00
|
|
|
updateRescheduleTracker(alloc, prevAllocation, now)
|
2018-01-14 22:47:21 +00:00
|
|
|
}
|
2020-12-17 23:21:46 +00:00
|
|
|
|
|
|
|
// If the allocation has task handles,
|
|
|
|
// copy them to the new allocation
|
|
|
|
propagateTaskState(alloc, prevAllocation, missing.PreviousLost())
|
2017-07-20 19:23:46 +00:00
|
|
|
}
|
2017-07-15 23:31:33 +00:00
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// If we are placing a canary and we found a match, add the canary
|
2018-04-19 20:58:06 +00:00
|
|
|
// to the deployment state object and mark it as a canary.
|
2019-01-25 04:57:07 +00:00
|
|
|
if missing.Canary() && s.deployment != nil {
|
2018-04-19 20:58:06 +00:00
|
|
|
alloc.DeploymentStatus = &structs.AllocDeploymentStatus{
|
|
|
|
Canary: true,
|
|
|
|
}
|
2017-07-20 19:23:46 +00:00
|
|
|
}
|
2017-05-22 17:58:34 +00:00
|
|
|
|
2019-04-11 01:20:22 +00:00
|
|
|
s.handlePreemptions(option, alloc, missing)
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Track the placement
|
2020-08-25 21:09:21 +00:00
|
|
|
s.plan.AppendAlloc(alloc, downgradedJob)
|
2017-07-18 00:18:12 +00:00
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
} else {
|
|
|
|
// Lazy initialize the failed map
|
|
|
|
if s.failedTGAllocs == nil {
|
|
|
|
s.failedTGAllocs = make(map[string]*structs.AllocMetric)
|
|
|
|
}
|
|
|
|
|
2021-04-29 19:03:45 +00:00
|
|
|
// Update metrics with the resources requested by the task group.
|
|
|
|
s.ctx.Metrics().ExhaustResources(tg)
|
|
|
|
|
2017-07-20 19:23:46 +00:00
|
|
|
// Track the fact that we didn't find a placement
|
|
|
|
s.failedTGAllocs[tg.Name] = s.ctx.Metrics()
|
|
|
|
|
|
|
|
// If we weren't able to find a replacement for the allocation, back
|
|
|
|
// out the fact that we asked to stop the allocation.
|
|
|
|
if stopPrevAlloc {
|
2018-01-14 22:47:21 +00:00
|
|
|
s.plan.PopUpdate(prevAllocation)
|
2017-07-20 19:23:46 +00:00
|
|
|
}
|
2017-07-18 00:18:12 +00:00
|
|
|
}
|
2018-01-19 14:41:53 +00:00
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-12-17 23:21:46 +00:00
|
|
|
// propagateTaskState copies task handles from previous allocations to
|
|
|
|
// replacement allocations when the previous allocation is being drained or was
|
|
|
|
// lost. Remote task drivers rely on this to reconnect to remote tasks when the
|
|
|
|
// allocation managing them changes due to a down or draining node.
|
|
|
|
//
|
2021-04-29 22:02:12 +00:00
|
|
|
// The previous allocation will be marked as lost after task state has been
|
|
|
|
// propagated (when the plan is applied), so its ClientStatus is not yet marked
|
|
|
|
// as lost. Instead, we use the `prevLost` flag to track whether the previous
|
|
|
|
// allocation will be marked lost.
|
2020-12-17 23:21:46 +00:00
|
|
|
func propagateTaskState(newAlloc, prev *structs.Allocation, prevLost bool) {
|
|
|
|
// Don't transfer state from client terminal allocs
|
|
|
|
if prev.ClientTerminalStatus() {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// If previous allocation is not lost and not draining, do not copy
|
|
|
|
// task handles.
|
|
|
|
if !prevLost && !prev.DesiredTransition.ShouldMigrate() {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
newAlloc.TaskStates = make(map[string]*structs.TaskState, len(newAlloc.AllocatedResources.Tasks))
|
|
|
|
for taskName, prevState := range prev.TaskStates {
|
|
|
|
if prevState.TaskHandle == nil {
|
|
|
|
// No task handle, skip
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
if _, ok := newAlloc.AllocatedResources.Tasks[taskName]; !ok {
|
|
|
|
// Task dropped in update, skip
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// Copy state
|
|
|
|
newState := structs.NewTaskState()
|
|
|
|
newState.TaskHandle = prevState.TaskHandle.Copy()
|
|
|
|
newAlloc.TaskStates[taskName] = newState
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-01-19 14:41:53 +00:00
|
|
|
// getSelectOptions sets up preferred nodes and penalty nodes
|
|
|
|
func getSelectOptions(prevAllocation *structs.Allocation, preferredNode *structs.Node) *SelectOptions {
|
|
|
|
selectOptions := &SelectOptions{}
|
|
|
|
if prevAllocation != nil {
|
|
|
|
penaltyNodes := make(map[string]struct{})
|
2019-12-03 14:14:49 +00:00
|
|
|
|
|
|
|
// If alloc failed, penalize the node it failed on to encourage
|
|
|
|
// rescheduling on a new node.
|
|
|
|
if prevAllocation.ClientStatus == structs.AllocClientStatusFailed {
|
|
|
|
penaltyNodes[prevAllocation.NodeID] = struct{}{}
|
|
|
|
}
|
2018-01-19 14:41:53 +00:00
|
|
|
if prevAllocation.RescheduleTracker != nil {
|
|
|
|
for _, reschedEvent := range prevAllocation.RescheduleTracker.Events {
|
|
|
|
penaltyNodes[reschedEvent.PrevNodeID] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
selectOptions.PenaltyNodeIDs = penaltyNodes
|
|
|
|
}
|
|
|
|
if preferredNode != nil {
|
|
|
|
selectOptions.PreferredNodes = []*structs.Node{preferredNode}
|
|
|
|
}
|
|
|
|
return selectOptions
|
|
|
|
}
|
|
|
|
|
2018-01-19 21:20:00 +00:00
|
|
|
// updateRescheduleTracker carries over previous restart attempts and adds the most recent restart
|
2018-03-08 15:36:01 +00:00
|
|
|
func updateRescheduleTracker(alloc *structs.Allocation, prev *structs.Allocation, now time.Time) {
|
|
|
|
reschedPolicy := prev.ReschedulePolicy()
|
2018-01-19 14:41:53 +00:00
|
|
|
var rescheduleEvents []*structs.RescheduleEvent
|
|
|
|
if prev.RescheduleTracker != nil {
|
2018-03-02 00:23:44 +00:00
|
|
|
var interval time.Duration
|
|
|
|
if reschedPolicy != nil {
|
|
|
|
interval = reschedPolicy.Interval
|
|
|
|
}
|
2018-03-08 00:44:54 +00:00
|
|
|
// If attempts is set copy all events in the interval range
|
2018-03-02 00:23:44 +00:00
|
|
|
if reschedPolicy.Attempts > 0 {
|
|
|
|
for _, reschedEvent := range prev.RescheduleTracker.Events {
|
2018-03-08 00:44:54 +00:00
|
|
|
timeDiff := now.UnixNano() - reschedEvent.RescheduleTime
|
2018-03-02 00:23:44 +00:00
|
|
|
// Only copy over events that are within restart interval
|
|
|
|
// This keeps the list of events small in cases where there's a long chain of old restart events
|
|
|
|
if interval > 0 && timeDiff <= interval.Nanoseconds() {
|
|
|
|
rescheduleEvents = append(rescheduleEvents, reschedEvent.Copy())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} else {
|
2018-03-08 00:44:54 +00:00
|
|
|
// Only copy the last n if unlimited is set
|
|
|
|
start := 0
|
|
|
|
if len(prev.RescheduleTracker.Events) > maxPastRescheduleEvents {
|
|
|
|
start = len(prev.RescheduleTracker.Events) - maxPastRescheduleEvents
|
|
|
|
}
|
|
|
|
for i := start; i < len(prev.RescheduleTracker.Events); i++ {
|
2018-03-02 00:23:44 +00:00
|
|
|
reschedEvent := prev.RescheduleTracker.Events[i]
|
|
|
|
rescheduleEvents = append(rescheduleEvents, reschedEvent.Copy())
|
|
|
|
}
|
2018-01-19 14:41:53 +00:00
|
|
|
}
|
|
|
|
}
|
2018-03-08 15:36:01 +00:00
|
|
|
nextDelay := prev.NextDelay()
|
2018-03-02 00:23:44 +00:00
|
|
|
rescheduleEvent := structs.NewRescheduleEvent(now.UnixNano(), prev.ID, prev.NodeID, nextDelay)
|
2018-01-19 21:20:00 +00:00
|
|
|
rescheduleEvents = append(rescheduleEvents, rescheduleEvent)
|
2018-01-19 14:41:53 +00:00
|
|
|
alloc.RescheduleTracker = &structs.RescheduleTracker{Events: rescheduleEvents}
|
|
|
|
}
|
|
|
|
|
2017-05-22 17:58:34 +00:00
|
|
|
// findPreferredNode finds the preferred node for an allocation
|
2018-07-05 15:00:03 +00:00
|
|
|
func (s *GenericScheduler) findPreferredNode(place placementResult) (*structs.Node, error) {
|
2020-12-09 19:05:18 +00:00
|
|
|
if prev := place.PreviousAllocation(); prev != nil && place.TaskGroup().EphemeralDisk.Sticky {
|
2017-06-06 21:08:46 +00:00
|
|
|
var preferredNode *structs.Node
|
|
|
|
ws := memdb.NewWatchSet()
|
2018-07-11 18:27:10 +00:00
|
|
|
preferredNode, err := s.state.NodeByID(ws, prev.NodeID)
|
2018-07-05 15:00:03 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2018-07-02 20:05:38 +00:00
|
|
|
if preferredNode != nil && preferredNode.Ready() {
|
2018-07-05 15:00:03 +00:00
|
|
|
return preferredNode, nil
|
2017-05-22 17:58:34 +00:00
|
|
|
}
|
|
|
|
}
|
2018-07-05 15:00:03 +00:00
|
|
|
return nil, nil
|
2017-05-22 17:58:34 +00:00
|
|
|
}
|
2020-05-27 19:02:01 +00:00
|
|
|
|
|
|
|
// selectNextOption calls the stack to get a node for placement
|
|
|
|
func (s *GenericScheduler) selectNextOption(tg *structs.TaskGroup, selectOptions *SelectOptions) *RankedNode {
|
|
|
|
option := s.stack.Select(tg, selectOptions)
|
|
|
|
_, schedConfig, _ := s.ctx.State().SchedulerConfig()
|
|
|
|
|
|
|
|
// Check if preemption is enabled, defaults to true
|
|
|
|
enablePreemption := true
|
|
|
|
if schedConfig != nil {
|
|
|
|
if s.job.Type == structs.JobTypeBatch {
|
|
|
|
enablePreemption = schedConfig.PreemptionConfig.BatchSchedulerEnabled
|
|
|
|
} else {
|
|
|
|
enablePreemption = schedConfig.PreemptionConfig.ServiceSchedulerEnabled
|
|
|
|
}
|
|
|
|
}
|
|
|
|
// Run stack again with preemption enabled
|
|
|
|
if option == nil && enablePreemption {
|
|
|
|
selectOptions.Preempt = true
|
|
|
|
option = s.stack.Select(tg, selectOptions)
|
|
|
|
}
|
|
|
|
return option
|
|
|
|
}
|
|
|
|
|
2020-06-22 14:28:45 +00:00
|
|
|
// handlePreemptions sets relevant preeemption related fields.
|
2020-05-27 19:02:01 +00:00
|
|
|
func (s *GenericScheduler) handlePreemptions(option *RankedNode, alloc *structs.Allocation, missing placementResult) {
|
|
|
|
if option.PreemptedAllocs == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// If this placement involves preemption, set DesiredState to evict for those allocations
|
|
|
|
var preemptedAllocIDs []string
|
|
|
|
for _, stop := range option.PreemptedAllocs {
|
|
|
|
s.plan.AppendPreemptedAlloc(stop, alloc.ID)
|
|
|
|
preemptedAllocIDs = append(preemptedAllocIDs, stop.ID)
|
|
|
|
|
|
|
|
if s.eval.AnnotatePlan && s.plan.Annotations != nil {
|
2020-10-09 05:21:41 +00:00
|
|
|
s.plan.Annotations.PreemptedAllocs = append(s.plan.Annotations.PreemptedAllocs, stop.Stub(nil))
|
2020-05-27 19:02:01 +00:00
|
|
|
if s.plan.Annotations.DesiredTGUpdates != nil {
|
|
|
|
desired := s.plan.Annotations.DesiredTGUpdates[missing.TaskGroup().Name]
|
|
|
|
desired.Preemptions += 1
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
alloc.PreemptedAllocations = preemptedAllocIDs
|
|
|
|
}
|