From bb6524151a7d11187179a284297793ba77f3370a Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 18 May 2017 12:36:04 -0700 Subject: [PATCH 001/105] cancel deployments --- nomad/plan_apply.go | 5 +- nomad/structs/structs.go | 21 ++++- scheduler/generic_sched.go | 34 +++++++- scheduler/generic_sched_test.go | 137 ++++++++++++++++++++++++++++++++ scheduler/scheduler.go | 4 + scheduler/testing.go | 27 ++++--- 6 files changed, 211 insertions(+), 17 deletions(-) diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index a3f7e74212e9..96b6edca9347 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -126,13 +126,10 @@ func (s *Server) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap minUpdates := len(result.NodeUpdate) minUpdates += len(result.NodeAllocation) - // Grab the job - job := plan.Job - // Setup the update request req := structs.ApplyPlanResultsRequest{ AllocUpdateRequest: structs.AllocUpdateRequest{ - Job: job, + Job: plan.Job, Alloc: make([]*structs.Allocation, 0, minUpdates), }, CreatedDeployment: plan.CreatedDeployment, diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 13a2c5015d97..a595f9860466 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3710,6 +3710,11 @@ const ( DeploymentStatusSuccessful = "successful" DeploymentStatusCancelled = "cancelled" DeploymentStatusPaused = "paused" + + // DeploymentStatusDescriptions are the various descriptions of the states a + // deployment can be in. + DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" + DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" ) // Deployment is the object that represents a job deployment which is used to @@ -4591,9 +4596,23 @@ func (p *Plan) AppendAlloc(alloc *Allocation) { p.NodeAllocation[node] = append(existing, alloc) } +// AppendDeploymentUpdate attaches an deployment update to the plan for the +// given deployment ID. +func (p *Plan) AppendDeploymentUpdate(id, status, description string) { + update := &DeploymentStatusUpdate{ + DeploymentID: id, + Status: status, + StatusDescription: description, + } + p.DeploymentUpdates = append(p.DeploymentUpdates, update) +} + // IsNoOp checks if this plan would do nothing func (p *Plan) IsNoOp() bool { - return len(p.NodeUpdate) == 0 && len(p.NodeAllocation) == 0 + return len(p.NodeUpdate) == 0 && + len(p.NodeAllocation) == 0 && + p.CreatedDeployment == nil && + len(p.DeploymentUpdates) == 0 } // PlanResult is the result of a plan submitted to the leader. diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 43f38f15eeba..a12cd1a8d6e0 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -73,6 +73,8 @@ type GenericScheduler struct { limitReached bool nextEval *structs.Evaluation + deployment *structs.Deployment + blocked *structs.Evaluation failedTGAllocs map[string]*structs.AllocMetric queuedAllocs map[string]int @@ -187,11 +189,12 @@ func (s *GenericScheduler) process() (bool, error) { ws := memdb.NewWatchSet() s.job, err = s.state.JobByID(ws, s.eval.JobID) if err != nil { - return false, fmt.Errorf("failed to get job '%s': %v", - s.eval.JobID, err) + return false, fmt.Errorf("failed to get job %q: %v", s.eval.JobID, err) } + numTaskGroups := 0 - if !s.job.Stopped() { + stopped := s.job.Stopped() + if !stopped { numTaskGroups = len(s.job.TaskGroups) } s.queuedAllocs = make(map[string]int, numTaskGroups) @@ -199,6 +202,30 @@ func (s *GenericScheduler) process() (bool, error) { // Create a plan s.plan = s.eval.MakePlan(s.job) + if !s.batch { + // Get any existing deployment + s.deployment, err = s.state.LatestDeploymentByJobID(ws, s.eval.JobID) + if err != nil { + return false, fmt.Errorf("failed to get job deployment %q: %v", s.eval.JobID, err) + } + + // If there is an existing deployment, it should be cancelled under the + // following scenarios: + // 1) The current job is stopped. + // 2) The deployment is for an older version of the job. + if d := s.deployment; d != nil { + if stopped { + s.plan.AppendDeploymentUpdate(d.ID, structs.DeploymentStatusCancelled, + structs.DeploymentStatusDescriptionStoppedJob) + s.deployment = nil + } else if d.JobCreateIndex != s.job.CreateIndex || d.JobModifyIndex != s.job.JobModifyIndex { + s.plan.AppendDeploymentUpdate(d.ID, structs.DeploymentStatusCancelled, + structs.DeploymentStatusDescriptionNewerJob) + s.deployment = nil + } + } + } + // Reset the failed allocations s.failedTGAllocs = nil @@ -234,6 +261,7 @@ func (s *GenericScheduler) process() (bool, error) { return true, nil } + // XXX Don't need a next rolling update eval // If the limit of placements was reached we need to create an evaluation // to pickup from here after the stagger period. if s.limitReached && s.nextEval == nil { diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index afb0e48d4cb9..e1616b22e004 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -2845,3 +2845,140 @@ func TestServiceSched_NodeDrain_Sticky(t *testing.T) { h.AssertEvalStatus(t, structs.EvalStatusComplete) } + +// This test ensures that when a job is stopped, the scheduler properly cancels +// an outstanding deployment. +func TestServiceSched_CancelDeployment_Stopped(t *testing.T) { + h := NewHarness(t) + + // Generate a fake job + job := mock.Job() + job.JobModifyIndex = job.CreateIndex + 1 + job.ModifyIndex = job.CreateIndex + 1 + job.Stop = true + noErr(t, h.State.UpsertJob(h.NextIndex(), job)) + + // Create a deployment + d := mock.Deployment() + d.JobID = job.ID + d.JobCreateIndex = job.CreateIndex + d.JobModifyIndex = job.JobModifyIndex - 1 + noErr(t, h.State.UpsertDeployment(h.NextIndex(), d, false)) + + // Create a mock evaluation to deregister the job + eval := &structs.Evaluation{ + ID: structs.GenerateUUID(), + Priority: 50, + TriggeredBy: structs.EvalTriggerJobDeregister, + JobID: job.ID, + } + + // Process the evaluation + err := h.Process(NewServiceScheduler, eval) + if err != nil { + t.Fatalf("err: %v", err) + } + + // Ensure a single plan + if len(h.Plans) != 1 { + t.Fatalf("bad: %#v", h.Plans) + } + plan := h.Plans[0] + + // Ensure the plan cancelled the existing deployment + ws := memdb.NewWatchSet() + out, err := h.State.LatestDeploymentByJobID(ws, job.ID) + noErr(t, err) + + if out == nil { + t.Fatalf("No deployment for job") + } + if out.ID != d.ID { + t.Fatalf("Latest deployment for job is different than original deployment") + } + if out.Status != structs.DeploymentStatusCancelled { + t.Fatalf("Deployment status is %q, want %q", out.Status, structs.DeploymentStatusCancelled) + } + if out.StatusDescription != structs.DeploymentStatusDescriptionStoppedJob { + t.Fatalf("Deployment status description is %q, want %q", + out.StatusDescription, structs.DeploymentStatusDescriptionStoppedJob) + } + + // Ensure the plan didn't allocate anything + var planned []*structs.Allocation + for _, allocList := range plan.NodeAllocation { + planned = append(planned, allocList...) + } + if len(planned) != 0 { + t.Fatalf("bad: %#v", plan) + } + + h.AssertEvalStatus(t, structs.EvalStatusComplete) +} + +// This test ensures that when a job is updated and had an old deployment, the scheduler properly cancels +// the deployment. +func TestServiceSched_CancelDeployment_NewerJob(t *testing.T) { + h := NewHarness(t) + + // Generate a fake job + job := mock.Job() + noErr(t, h.State.UpsertJob(h.NextIndex(), job)) + + // Create a deployment for an old version of the job + d := mock.Deployment() + d.JobID = job.ID + noErr(t, h.State.UpsertDeployment(h.NextIndex(), d, false)) + + // Upsert again to bump job version + noErr(t, h.State.UpsertJob(h.NextIndex(), job)) + + // Create a mock evaluation to kick the job + eval := &structs.Evaluation{ + ID: structs.GenerateUUID(), + Priority: 50, + TriggeredBy: structs.EvalTriggerJobRegister, + JobID: job.ID, + } + + // Process the evaluation + err := h.Process(NewServiceScheduler, eval) + if err != nil { + t.Fatalf("err: %v", err) + } + + // Ensure a single plan + if len(h.Plans) != 1 { + t.Fatalf("bad: %#v", h.Plans) + } + plan := h.Plans[0] + + // Ensure the plan cancelled the existing deployment + ws := memdb.NewWatchSet() + out, err := h.State.LatestDeploymentByJobID(ws, job.ID) + noErr(t, err) + + if out == nil { + t.Fatalf("No deployment for job") + } + if out.ID != d.ID { + t.Fatalf("Latest deployment for job is different than original deployment") + } + if out.Status != structs.DeploymentStatusCancelled { + t.Fatalf("Deployment status is %q, want %q", out.Status, structs.DeploymentStatusCancelled) + } + if out.StatusDescription != structs.DeploymentStatusDescriptionNewerJob { + t.Fatalf("Deployment status description is %q, want %q", + out.StatusDescription, structs.DeploymentStatusDescriptionNewerJob) + } + // Ensure the plan didn't allocate anything + var planned []*structs.Allocation + for _, allocList := range plan.NodeAllocation { + planned = append(planned, allocList...) + } + if len(planned) != 0 { + t.Fatalf("bad: %#v", plan) + } + + h.AssertEvalStatus(t, structs.EvalStatusComplete) +} diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go index ddbf855c4ca5..995b3affdab8 100644 --- a/scheduler/scheduler.go +++ b/scheduler/scheduler.go @@ -79,6 +79,10 @@ type State interface { // GetJobByID is used to lookup a job by ID JobByID(ws memdb.WatchSet, id string) (*structs.Job, error) + + // LatestDeploymentByJobID returns the latest deployment matching the given + // job ID + LatestDeploymentByJobID(ws memdb.WatchSet, jobID string) (*structs.Deployment, error) } // Planner interface is used to submit a task allocation plan. diff --git a/scheduler/testing.go b/scheduler/testing.go index 74c01c4865db..af1411db1040 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -6,6 +6,7 @@ import ( "os" "sync" "testing" + "time" memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/nomad/state" @@ -108,19 +109,27 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er allocs = append(allocs, allocList...) } - // Attach the plan to all the allocations. It is pulled out in the - // payload to avoid the redundancy of encoding, but should be denormalized - // prior to being inserted into MemDB. - if j := plan.Job; j != nil { - for _, alloc := range allocs { - if alloc.Job == nil { - alloc.Job = j - } + // 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() + for _, alloc := range allocs { + if alloc.CreateTime == 0 { + alloc.CreateTime = now } } + // Setup the update request + req := structs.ApplyPlanResultsRequest{ + AllocUpdateRequest: structs.AllocUpdateRequest{ + Job: plan.Job, + Alloc: allocs, + }, + CreatedDeployment: plan.CreatedDeployment, + DeploymentUpdates: plan.DeploymentUpdates, + } + // Apply the full plan - err := h.State.UpsertAllocs(index, allocs) + err := h.State.UpsertPlanResults(index, &req) return result, nil, err } From df40bd8a5f2e81dd72002f8b4de17f6e781e0c1e Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 22 May 2017 10:58:34 -0700 Subject: [PATCH 002/105] initial reconciler --- helper/funcs.go | 7 + nomad/state/state_store.go | 2 + nomad/structs/structs.go | 32 +- scheduler/generic_sched.go | 161 ++++++++-- scheduler/reconcile.go | 588 +++++++++++++++++++++++++++++++++++++ 5 files changed, 763 insertions(+), 27 deletions(-) create mode 100644 scheduler/reconcile.go diff --git a/helper/funcs.go b/helper/funcs.go index 44a3e57c4ddb..f461669a9846 100644 --- a/helper/funcs.go +++ b/helper/funcs.go @@ -64,6 +64,13 @@ func TimeToPtr(t time.Duration) *time.Duration { return &t } +func IntMin(a, b int) int { + if a < b { + return a + } + return b +} + // MapStringStringSliceValueSet returns the set of values in a map[string][]string func MapStringStringSliceValueSet(m map[string][]string) []string { set := make(map[string]struct{}) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 91b4598447fa..d060c550997c 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -90,6 +90,8 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR txn := s.db.Txn(true) defer txn.Abort() + //s.logger.Printf("ALEX: INSERTING %# v", pretty.Formatter(results)) + // Upsert the newly created deployment if results.CreatedDeployment != nil { if err := s.upsertDeploymentImpl(index, results.CreatedDeployment, true, txn); err != nil { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index a595f9860466..9be6a33415e9 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1515,6 +1515,16 @@ func (j *Job) Stopped() bool { return j == nil || j.Stop } +func (j *Job) HasUpdateStrategy() bool { + for _, tg := range j.TaskGroups { + if tg.Update != nil { + return true + } + } + + return false +} + // Stub is used to return a summary of the job func (j *Job) Stub(summary *JobSummary) *JobListStub { return &JobListStub{ @@ -3752,6 +3762,17 @@ type Deployment struct { ModifyIndex uint64 } +func NewDeployment(job *Job) *Deployment { + return &Deployment{ + ID: GenerateUUID(), + JobID: job.ID, + JobVersion: job.Version, + JobModifyIndex: job.ModifyIndex, + JobCreateIndex: job.CreateIndex, + Status: DeploymentStatusRunning, + } +} + func (d *Deployment) Copy() *Deployment { c := &Deployment{} *c = *d @@ -3779,18 +3800,9 @@ func (d *Deployment) Active() bool { // DeploymentState tracks the state of a deployment for a given task group. type DeploymentState struct { - // Promoted marks whether the canaries have been. Promotion by - // task group is not allowed since that doesn’t allow a single - // job to transition into the “stable” state. + // Promoted marks whether the canaries have been promoted Promoted bool - // RequiresPromotion marks whether the deployment is expecting - // a promotion. This is computable by checking if the job has canaries - // specified, but is stored in the deployment to make it so that consumers - // do not need to query job history and deployments to know whether a - // promotion is needed. - RequiresPromotion bool - // DesiredCanaries is the number of canaries that should be created. DesiredCanaries int diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index a12cd1a8d6e0..c2ae5fb59b3b 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -208,22 +208,6 @@ func (s *GenericScheduler) process() (bool, error) { if err != nil { return false, fmt.Errorf("failed to get job deployment %q: %v", s.eval.JobID, err) } - - // If there is an existing deployment, it should be cancelled under the - // following scenarios: - // 1) The current job is stopped. - // 2) The deployment is for an older version of the job. - if d := s.deployment; d != nil { - if stopped { - s.plan.AppendDeploymentUpdate(d.ID, structs.DeploymentStatusCancelled, - structs.DeploymentStatusDescriptionStoppedJob) - s.deployment = nil - } else if d.JobCreateIndex != s.job.CreateIndex || d.JobModifyIndex != s.job.JobModifyIndex { - s.plan.AppendDeploymentUpdate(d.ID, structs.DeploymentStatusCancelled, - structs.DeploymentStatusDescriptionNewerJob) - s.deployment = nil - } - } } // Reset the failed allocations @@ -239,7 +223,7 @@ func (s *GenericScheduler) process() (bool, error) { } // Compute the target job allocations - if err := s.computeJobAllocs(); err != nil { + if err := s.computeJobAllocs2(); err != nil { s.logger.Printf("[ERR] sched: %#v: %v", s.eval, err) return false, err } @@ -374,6 +358,52 @@ func (s *GenericScheduler) filterCompleteAllocs(allocs []*structs.Allocation) ([ return filtered, terminalAllocsByName } +// computeJobAllocs is used to reconcile differences between the job, +// existing allocations and node status to update the allocations. +func (s *GenericScheduler) computeJobAllocs2() error { + // Lookup the allocations by JobID + ws := memdb.NewWatchSet() + allocs, err := s.state.AllocsByJob(ws, s.eval.JobID, true) + 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 + // nodes to lost + updateNonTerminalAllocsToLost(s.plan, tainted, allocs) + + // Filter out the allocations in a terminal state + allocs, _ = s.filterCompleteAllocs(allocs) + + reconciler := NewAllocReconciler(s.ctx, s.stack, s.batch, s.eval, s.job, s.deployment, allocs, tainted) + results := reconciler.Compute() + + // Add the deployment changes to the plan + s.plan.CreatedDeployment = results.createDeployment + s.plan.DeploymentUpdates = results.deploymentUpdates + + // Handle the stop + for _, stop := range results.stop { + s.plan.AppendUpdate(stop.alloc, structs.AllocDesiredStatusStop, stop.statusDescription, stop.clientStatus) + } + + // Handle the in-place updates + for _, update := range results.inplaceUpdate { + s.ctx.Plan().AppendAlloc(update) + } + + // Compute the placements + return s.computePlacements2(results.place) +} + // computeJobAllocs is used to reconcile differences between the job, // existing allocations and node status to update the allocations. func (s *GenericScheduler) computeJobAllocs() error { @@ -459,6 +489,88 @@ func (s *GenericScheduler) computeJobAllocs() error { return s.computePlacements(diff.place) } +// computePlacements computes placements for allocations +func (s *GenericScheduler) computePlacements2(place []allocPlaceResult) error { + // Get the base nodes + nodes, byDC, err := readyNodesInDCs(s.state, s.job.Datacenters) + if err != nil { + return err + } + + var deploymentID string + if s.deployment != nil { + deploymentID = s.deployment.ID + } + + // Update the set of placement ndoes + s.stack.SetNodes(nodes) + + for _, missing := range place { + // Check if this task group has already failed + if metric, ok := s.failedTGAllocs[missing.taskGroup.Name]; ok { + metric.CoalescedFailures += 1 + continue + } + + // Find the preferred node + preferredNode, err := s.findPreferredNode2(&missing) + if err != nil { + return err + } + + // Attempt to match the task group + var option *RankedNode + if preferredNode != nil { + option, _ = s.stack.SelectPreferringNodes(missing.taskGroup, []*structs.Node{preferredNode}) + } else { + option, _ = s.stack.Select(missing.taskGroup) + } + + // Store the available nodes by datacenter + s.ctx.Metrics().NodesAvailable = byDC + + // Set fields based on if we found an allocation option + if option != nil { + // Create an allocation for this + alloc := &structs.Allocation{ + ID: structs.GenerateUUID(), + EvalID: s.eval.ID, + Name: missing.name, + JobID: s.job.ID, + TaskGroup: missing.taskGroup.Name, + Metrics: s.ctx.Metrics(), + NodeID: option.Node.ID, + DeploymentID: deploymentID, + Canary: missing.canary, + TaskResources: option.TaskResources, + DesiredStatus: structs.AllocDesiredStatusRun, + ClientStatus: structs.AllocClientStatusPending, + + SharedResources: &structs.Resources{ + DiskMB: missing.taskGroup.EphemeralDisk.SizeMB, + }, + } + + // If the new allocation is replacing an older allocation then we + // set the record the older allocation id so that they are chained + if missing.previousAlloc != nil { + alloc.PreviousAllocation = missing.previousAlloc.ID + } + + s.plan.AppendAlloc(alloc) + } else { + // Lazy initialize the failed map + if s.failedTGAllocs == nil { + s.failedTGAllocs = make(map[string]*structs.AllocMetric) + } + + s.failedTGAllocs[missing.taskGroup.Name] = s.ctx.Metrics() + } + } + + return nil +} + // computePlacements computes placements for allocations func (s *GenericScheduler) computePlacements(place []allocTuple) error { // Get the base nodes @@ -534,6 +646,21 @@ func (s *GenericScheduler) computePlacements(place []allocTuple) error { return nil } +// findPreferredNode finds the preferred node for an allocation +func (s *GenericScheduler) findPreferredNode2(place *allocPlaceResult) (node *structs.Node, err error) { + if place.previousAlloc != nil { + if place.taskGroup.EphemeralDisk.Sticky == true { + var preferredNode *structs.Node + ws := memdb.NewWatchSet() + preferredNode, err = s.state.NodeByID(ws, place.previousAlloc.NodeID) + if preferredNode.Ready() { + node = preferredNode + } + } + } + return +} + // findPreferredNode finds the preferred node for an allocation func (s *GenericScheduler) findPreferredNode(allocTuple *allocTuple) (node *structs.Node, err error) { if allocTuple.Alloc != nil { diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go new file mode 100644 index 000000000000..269233626dcf --- /dev/null +++ b/scheduler/reconcile.go @@ -0,0 +1,588 @@ +package scheduler + +import ( + "fmt" + "sort" + + memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/nomad/structs" +) + +type allocReconciler struct { + // ctx gives access to the state store and logger + ctx Context + + // stack allows checking for the ability to do an in-place update + stack Stack + + // batch marks whether the job is a batch job + batch bool + + // eval is the evaluation triggering the scheduling event + eval *structs.Evaluation + + // job is the job being operated on, it may be nil if the job is being + // stopped via a purge + job *structs.Job + + // deployment is the current deployment for the job + deployment *structs.Deployment + + // deploymentPaused marks whether the deployment is paused + deploymentPaused bool + + // groupUpdateStrategy maps task groups to their update strategy + groupUpdateStrategy map[string]*structs.UpdateStrategy + + // taintedNodes contains a map of nodes that are tainted + taintedNodes map[string]*structs.Node + + // existingAllocs is non-terminal existing allocations + existingAllocs []*structs.Allocation + + // result is the results of the reconcile. During computation it can be + // used to store intermediate state + result *reconcileResults +} + +type reconcileResults struct { + createDeployment *structs.Deployment + deploymentUpdates []*structs.DeploymentStatusUpdate + + place []allocPlaceResult + inplaceUpdate []*structs.Allocation + stop []allocStopResult + + // TODO track the desired of the deployment +} + +type allocPlaceResult struct { + name string + canary bool + taskGroup *structs.TaskGroup + previousAlloc *structs.Allocation +} + +type allocStopResult struct { + alloc *structs.Allocation + clientStatus string + statusDescription string +} + +func NewAllocReconciler(ctx Context, stack Stack, batch bool, + eval *structs.Evaluation, job *structs.Job, deployment *structs.Deployment, + existingAllocs []*structs.Allocation, taintedNodes map[string]*structs.Node) *allocReconciler { + + a := &allocReconciler{ + ctx: ctx, + stack: stack, + eval: eval, + batch: batch, + job: job, + deployment: deployment, + existingAllocs: existingAllocs, + taintedNodes: taintedNodes, + result: new(reconcileResults), + } + + if deployment != nil { + a.deploymentPaused = deployment.Status == structs.DeploymentStatusPaused + } + + // Determine the update strategy for each group + if job != nil { + a.groupUpdateStrategy = make(map[string]*structs.UpdateStrategy) + for _, tg := range job.TaskGroups { + if u := tg.Update; u != nil { + a.groupUpdateStrategy[tg.Name] = u + } + } + } + + return a +} + +func (a *allocReconciler) Compute() *reconcileResults { + // If we are just stopping a job we do not need to do anything more than + // stopping all running allocs + if a.job == nil || a.job.Stop { + a.handleStop() + + // Cancel the deployment since it is not needed + if a.deployment != nil { + a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ + DeploymentID: a.deployment.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionStoppedJob, + }) + } + + return a.result + } + + m := newAllocMatrix(a.job, a.existingAllocs) + for group, as := range m { + a.computeGroup(group, as) + } + + return a.result +} + +// handleStop marks all allocations to be stopped, handling the lost case +func (a *allocReconciler) handleStop() { + as := newAllocSet(a.existingAllocs) + untainted, migrate, lost := as.filterByTainted(a.taintedNodes) + a.markStop(untainted, "", allocNotNeeded) + a.markStop(migrate, "", allocNotNeeded) + a.markStop(lost, structs.AllocClientStatusLost, allocLost) +} + +func (a *allocReconciler) markStop(allocs allocSet, clientStatus, statusDescription string) { + for _, alloc := range allocs { + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + clientStatus: clientStatus, + statusDescription: statusDescription, + }) + } +} + +func (a *allocReconciler) computeGroup(group string, as allocSet) { + // Get the task group. The task group may be nil if the job was updates such + // that the task group no longer exists + tg := a.job.LookupTaskGroup(group) + + // Determine what set of alloations are on tainted nodes + untainted, migrate, lost := as.filterByTainted(a.taintedNodes) + + a.ctx.Logger().Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) + a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) + + // If the task group is nil, then the task group has been removed so all we + // need to do is stop everything + if tg == nil { + a.ctx.Logger().Printf("RECONCILER -- STOPPING ALL") + a.markStop(untainted, "", allocNotNeeded) + a.markStop(migrate, "", allocNotNeeded) + a.markStop(lost, structs.AllocClientStatusLost, allocLost) + return + } + + // Get the deployment state for the group + var dstate *structs.DeploymentState + if a.deployment != nil { + dstate = a.deployment.TaskGroups[group] + } + + // Mark all lost allocations for stop. Previous allocation doesn't matter + // here since it is on a lost node + for _, alloc := range lost { + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + clientStatus: structs.AllocClientStatusLost, + statusDescription: allocLost, + }) + } + + // Get any existing canaries + canaries := untainted.filterByCanary() + + // Cancel any canary from a prior deployment + if len(canaries) != 0 { + if a.deployment != nil { + current, older := canaries.filterByDeployment(a.deployment.ID) + a.markStop(older, "", allocNotNeeded) + + a.ctx.Logger().Printf("RECONCILER -- older canaries %#v", older) + a.ctx.Logger().Printf("RECONCILER -- current canaries %#v", current) + + untainted = untainted.difference(older, current) + canaries = current + a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) + } else { + // We don't need any of those canaries since there no longer is a + // deployment + a.markStop(canaries, "", allocNotNeeded) + untainted = untainted.difference(canaries) + canaries = nil + a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) + } + } + + // Stop any unneeded allocations and update the untainted set to not + // included stopped allocations + keep, stop := a.computeStop(tg, untainted) + a.markStop(stop, "", allocNotNeeded) + untainted = keep + + a.ctx.Logger().Printf("RECONCILER -- Stopping (%d); Untainted (%d)", len(stop), len(keep)) + a.ctx.Logger().Printf("RECONCILER -- stopping %#v", stop) + a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) + + // Do inplace upgrades where possible and capture the set of upgrades that + // need to be done destructively. + _, inplace, destructive := a.computeUpdates(tg, untainted) + a.ctx.Logger().Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) + + // XXX Not clear if this is needed + // Update untainted so that it contains all existing allocations that have + // been inplace updated or do not have to be updated and does not include + // any canaries. + //untainted = untainted.difference(destructive) + //a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) + + // Get the update strategy of the group + strategy, update := a.groupUpdateStrategy[group] + + // XXX need a structure for picking names + + // The fact that we have destructive updates and have less canaries than is + // desired means we need to create canaries + requireCanary := len(destructive) != 0 && update && strategy.Canary != 0 && len(canaries) < strategy.Canary + if requireCanary && !a.deploymentPaused { + a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", strategy.Canary-len(canaries)) + for i := len(canaries); i < strategy.Canary; i++ { + a.result.place = append(a.result.place, allocPlaceResult{ + // XXX Pick better name + name: structs.GenerateUUID(), + canary: true, + taskGroup: tg, + }) + } + } + + // Determine how many we can place + limit := tg.Count + if update { + // XXX This is wrong. Need to detect health first. Probably only within + // the truly untainted set + limit = strategy.MaxParallel + } + a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) + + // Place if: + // * The deployment is not paused + // * Not placing any canaries + // * If there are any canaries that they have been promoted + existingCanariesPromoted := dstate == nil || dstate.DesiredCanaries == 0 || dstate.Promoted + canPlace := !a.deploymentPaused && !requireCanary && existingCanariesPromoted + a.ctx.Logger().Printf("RECONCILER -- CAN PLACE %v", canPlace) + if canPlace { + // Place all new allocations + place := a.computePlacements(tg, untainted, destructive) + a.ctx.Logger().Printf("RECONCILER -- Placing (%d)", len(place)) + for _, p := range place { + a.result.place = append(a.result.place, p) + } + + // Do all destructive updates + min := helper.IntMin(len(destructive), limit) + i := 0 + a.ctx.Logger().Printf("RECONCILER -- Destructive Updating (%d)", min) + for _, alloc := range destructive { + if i == min { + break + } + i++ + + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + statusDescription: allocUpdating, + }) + a.result.place = append(a.result.place, allocPlaceResult{ + name: alloc.Name, + taskGroup: tg, + previousAlloc: alloc, + }) + } + limit -= min + } + + // Migrations should be done under the rolling update strategy, however we + // do not abide by the paused state of the deployment since this could block + // node draining. + min := helper.IntMin(len(migrate), limit) + i := 0 + a.ctx.Logger().Printf("RECONCILER -- Migrating (%d)", min) + for _, alloc := range migrate { + if i == min { + break + } + i++ + + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + statusDescription: allocMigrating, + }) + a.result.place = append(a.result.place, allocPlaceResult{ + name: alloc.Name, + canary: false, + taskGroup: tg, + previousAlloc: alloc, + }) + } +} + +func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted, destructiveUpdates allocSet) []allocPlaceResult { + // Hot path the nothing to do case + existing := len(untainted) + len(destructiveUpdates) + if existing == group.Count { + return nil + } + + // XXX need to pick better names + var place []allocPlaceResult + for i := existing; i < group.Count; i++ { + place = append(place, allocPlaceResult{ + name: structs.GenerateUUID(), + taskGroup: group, + }) + } + + return place +} + +func (a *allocReconciler) computeStop(group *structs.TaskGroup, untainted allocSet) (keep, stop allocSet) { + // Hot path the nothing to do case + if len(untainted) <= group.Count { + return untainted, nil + } + + // XXX Sort doesn't actually do the right thing "foo.bar[11]" < "foo.bar[3]" + // TODO make name tree + names := make([]string, 0, len(untainted)) + for name := range untainted { + names = append(names, name) + } + sort.Strings(names) + + keep = make(map[string]*structs.Allocation) + stop = make(map[string]*structs.Allocation) + + for i, name := range names { + a := untainted[name] + if i < group.Count { + keep[a.Name] = a + } else { + stop[a.Name] = a + } + } + + return +} + +func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted allocSet) (ignore, inplace, destructive allocSet) { + // Determine the set of allocations that need to be updated + ignore = make(map[string]*structs.Allocation) + inplace = make(map[string]*structs.Allocation) + destructive = make(map[string]*structs.Allocation) + + ws := memdb.NewWatchSet() + for _, alloc := range untainted { + if alloc.Job.JobModifyIndex == a.job.JobModifyIndex { + ignore[alloc.ID] = alloc + continue + } + + // Check if the task drivers or config has changed, requires + // a destructive upgrade since that cannot be done in-place. + if tasksUpdated(a.job, alloc.Job, group.Name) { + destructive[alloc.ID] = alloc + continue + } + + // Terminal batch allocations are not filtered when they are completed + // successfully. We should avoid adding the allocation to the plan in + // the case that it is an in-place update to avoid both additional data + // in the plan and work for the clients. + if alloc.TerminalStatus() { + ignore[alloc.ID] = alloc + continue + } + + // Get the existing node + node, err := a.ctx.State().NodeByID(ws, alloc.NodeID) + if err != nil { + a.ctx.Logger().Printf("[ERR] sched: %#v failed to get node '%s': %v", a.eval, alloc.NodeID, err) + continue + } + if node == nil { + destructive[alloc.ID] = alloc + continue + } + + // Set the existing node as the base set + a.stack.SetNodes([]*structs.Node{node}) + + // Stage an eviction of the current allocation. This is done so that + // the current allocation is discounted when checking for feasability. + // Otherwise we would be trying to fit the tasks current resources and + // updated resources. After select is called we can remove the evict. + a.ctx.Plan().AppendUpdate(alloc, structs.AllocDesiredStatusStop, allocInPlace, "") + + // Attempt to match the task group + option, _ := a.stack.Select(group) + + // Pop the allocation + a.ctx.Plan().PopUpdate(alloc) + + // Skip if we could not do an in-place update + if option == nil { + destructive[alloc.ID] = alloc + continue + } + + // Restore the network offers from the existing allocation. + // We do not allow network resources (reserved/dynamic ports) + // to be updated. This is guarded in taskUpdated, so we can + // safely restore those here. + for task, resources := range option.TaskResources { + existing := alloc.TaskResources[task] + resources.Networks = existing.Networks + } + + // Create a shallow copy + newAlloc := new(structs.Allocation) + *newAlloc = *alloc + + // Update the allocation + newAlloc.EvalID = a.eval.ID + newAlloc.Job = nil // Use the Job in the Plan + newAlloc.Resources = nil // Computed in Plan Apply + newAlloc.TaskResources = option.TaskResources + newAlloc.Metrics = a.ctx.Metrics() + + // Add this to the result and the tracking allocSet + inplace[alloc.ID] = alloc + a.result.inplaceUpdate = append(a.result.inplaceUpdate, newAlloc) + } + + return +} + +type allocMatrix map[string]allocSet + +func newAllocMatrix(job *structs.Job, allocs []*structs.Allocation) allocMatrix { + m := allocMatrix(make(map[string]allocSet)) + for _, a := range allocs { + s, ok := m[a.TaskGroup] + if !ok { + s = make(map[string]*structs.Allocation) + m[a.TaskGroup] = s + } + s[a.ID] = a + } + for _, tg := range job.TaskGroups { + s, ok := m[tg.Name] + if !ok { + s = make(map[string]*structs.Allocation) + m[tg.Name] = s + } + } + return m +} + +type allocSet map[string]*structs.Allocation + +func (a allocSet) GoString() string { + if len(a) == 0 { + return "[]" + } + + start := fmt.Sprintf("len(%d) [\n", len(a)) + for k := range a { + start += k + ",\n" + } + return start + "]" +} + +func newAllocSet(allocs []*structs.Allocation) allocSet { + s := make(map[string]*structs.Allocation, len(allocs)) + for _, a := range allocs { + s[a.ID] = a + } + return s +} + +func (a allocSet) difference(others ...allocSet) allocSet { + diff := make(map[string]*structs.Allocation) +OUTER: + for k, v := range a { + for _, other := range others { + if _, ok := other[k]; ok { + continue OUTER + } + } + diff[k] = v + } + return diff +} + +func (a allocSet) filterByTainted(nodes map[string]*structs.Node) (untainted, migrate, lost allocSet) { + untainted = make(map[string]*structs.Allocation) + migrate = make(map[string]*structs.Allocation) + lost = make(map[string]*structs.Allocation) + for _, alloc := range a { + n, ok := nodes[alloc.NodeID] + switch { + case !ok: + untainted[alloc.ID] = alloc + case n == nil || n.TerminalStatus(): + lost[alloc.ID] = alloc + default: + migrate[alloc.ID] = alloc + } + } + return +} + +func (a allocSet) filterByCanary() allocSet { + canaries := make(map[string]*structs.Allocation) + for _, alloc := range a { + if alloc.Canary { + canaries[alloc.ID] = alloc + } + } + return canaries +} + +func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { + match = make(map[string]*structs.Allocation) + nonmatch = make(map[string]*structs.Allocation) + for _, alloc := range a { + if alloc.DeploymentID == id { + match[alloc.ID] = alloc + } else { + nonmatch[alloc.ID] = alloc + } + } + return +} + +func (a allocSet) filterByDeploymentHealthy() allocSet { + healthy := make(map[string]*structs.Allocation) + for _, alloc := range a { + if alloc.DeploymentStatus != nil && + alloc.DeploymentStatus.Healthy != nil && + *alloc.DeploymentStatus.Healthy { + healthy[alloc.ID] = alloc + } + } + return healthy +} + +func (a allocSet) filterByUpdateType(ctx Context, stack Stack, eval *structs.Evaluation, + job *structs.Job) (ignore, inplace, destructive allocSet) { + ignore = make(map[string]*structs.Allocation) + inplace = make(map[string]*structs.Allocation) + destructive = make(map[string]*structs.Allocation) + + for _, alloc := range a { + if alloc.Job.JobModifyIndex == job.JobModifyIndex { + ignore[alloc.ID] = alloc + continue + } + } + return +} From c16195a2ea2b72953dc1cc16e52d05dff5a191a7 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 22 May 2017 17:02:20 -0700 Subject: [PATCH 003/105] Only upsert a job if the spec changes and push deployment creation into reconciler --- nomad/job_endpoint.go | 23 +++++---- nomad/job_endpoint_test.go | 26 ++++++++++ nomad/structs/structs.go | 41 ++++++++++++++++ nomad/structs/structs_test.go | 42 ++++++++++++++++ scheduler/generic_sched.go | 5 ++ scheduler/reconcile.go | 91 ++++++++++++++++++++++++----------- 6 files changed, 192 insertions(+), 36 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 2d6703fb9acd..0a1cf719b4fd 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -142,15 +142,20 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis // Clear the Vault token args.Job.VaultToken = "" - // Commit this update via Raft - _, index, err := j.srv.raftApply(structs.JobRegisterRequestType, args) - if err != nil { - j.srv.logger.Printf("[ERR] nomad.job: Register failed: %v", err) - return err - } + // Check if the job has changed at all + if currentJob == nil || currentJob.SpecChanged(args.Job) { + // Commit this update via Raft + _, index, err := j.srv.raftApply(structs.JobRegisterRequestType, args) + if err != nil { + j.srv.logger.Printf("[ERR] nomad.job: Register failed: %v", err) + return err + } - // Populate the reply with job information - reply.JobModifyIndex = index + // Populate the reply with job information + reply.JobModifyIndex = index + } else { + reply.JobModifyIndex = currentJob.JobModifyIndex + } // If the job is periodic or parameterized, we don't create an eval. if args.Job.IsPeriodic() || args.Job.IsParameterized() { @@ -164,7 +169,7 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis Type: args.Job.Type, TriggeredBy: structs.EvalTriggerJobRegister, JobID: args.Job.ID, - JobModifyIndex: index, + JobModifyIndex: reply.JobModifyIndex, Status: structs.EvalStatusPending, } update := &structs.EvalUpdateRequest{ diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 782b8b837040..efb923cfd6ee 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -13,6 +13,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" + "github.com/kr/pretty" ) func TestJobEndpoint_Register(t *testing.T) { @@ -232,6 +233,9 @@ func TestJobEndpoint_Register_Existing(t *testing.T) { if out.Priority != 100 { t.Fatalf("expected update") } + if out.Version != 1 { + t.Fatalf("expected update") + } // Lookup the evaluation eval, err := state.EvalByID(ws, resp.EvalID) @@ -263,6 +267,28 @@ func TestJobEndpoint_Register_Existing(t *testing.T) { if eval.Status != structs.EvalStatusPending { t.Fatalf("bad: %#v", eval) } + + if err := msgpackrpc.CallWithCodec(codec, "Job.Register", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + if resp.Index == 0 { + t.Fatalf("bad index: %d", resp.Index) + } + + // Check to ensure the job version didn't get bumped becasue we submitted + // the same job + state = s1.fsm.State() + ws = memdb.NewWatchSet() + out, err = state.JobByID(ws, job.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + if out == nil { + t.Fatalf("expected job") + } + if out.Version != 1 { + t.Fatalf("expected no update; got %v; diff %v", out.Version, pretty.Diff(job2, out)) + } } func TestJobEndpoint_Register_Periodic(t *testing.T) { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 9be6a33415e9..bfbb2952dce0 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1627,6 +1627,47 @@ func (j *Job) RequiredSignals() map[string]map[string][]string { return signals } +// SpecChanged determines if the functional specification has changed between +// two job versions. The new job will be mutated but returned to its original +// value before returning, thus concurrent access to the job should be blocked. +func (j *Job) SpecChanged(new *Job) bool { + if j == nil { + return new != nil + } + + // Capture the original mutable values so they can be restored. + oStatus := new.Status + oStatusDescription := new.StatusDescription + oStable := new.Stable + oVersion := new.Version + oCreateIndex := new.CreateIndex + oModifyIndex := new.ModifyIndex + oJobModifyIndex := new.JobModifyIndex + + // Update the new job so we can do a reflect + new.Status = j.Status + new.StatusDescription = j.StatusDescription + new.Stable = j.Stable + new.Version = j.Version + new.CreateIndex = j.CreateIndex + new.ModifyIndex = j.ModifyIndex + new.JobModifyIndex = j.JobModifyIndex + + // Deep equals the jobs + equal := reflect.DeepEqual(j, new) + + // Restore the new jobs values + new.Status = oStatus + new.StatusDescription = oStatusDescription + new.Stable = oStable + new.Version = oVersion + new.CreateIndex = oCreateIndex + new.ModifyIndex = oModifyIndex + new.JobModifyIndex = oJobModifyIndex + + return !equal +} + // JobListStub is used to return a subset of job information // for the job list type JobListStub struct { diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index 70c20f4ea0e9..3942d47a3110 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -295,6 +295,48 @@ func TestJob_Canonicalize_Update(t *testing.T) { } } +func TestJob_SpecChanged(t *testing.T) { + // Get a base test job + base := testJob() + + // Only modify the indexes/mutable state of the job + mutatedBase := base.Copy() + mutatedBase.Status = "foo" + mutatedBase.ModifyIndex = base.ModifyIndex + 100 + + // changed contains a spec change that should be detected + change := base.Copy() + change.Priority = 99 + + cases := []struct { + Name string + Original *Job + New *Job + Changed bool + }{ + { + Name: "Same job except mutable indexes", + Changed: false, + Original: base, + New: mutatedBase, + }, + { + Name: "Different", + Changed: true, + Original: base, + New: change, + }, + } + + for _, c := range cases { + t.Run(c.Name, func(t *testing.T) { + if actual := c.Original.SpecChanged(c.New); actual != c.Changed { + t.Fatalf("SpecChanged() returned %v; want %v", actual, c.Changed) + } + }) + } +} + func testJob() *Job { return &Job{ Region: "global", diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index c2ae5fb59b3b..19415f67e545 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -390,6 +390,11 @@ func (s *GenericScheduler) computeJobAllocs2() error { s.plan.CreatedDeployment = results.createDeployment s.plan.DeploymentUpdates = results.deploymentUpdates + // Update the stored deployment + if results.createDeployment != nil { + s.deployment = results.createDeployment + } + // Handle the stop for _, stop := range results.stop { s.plan.AppendUpdate(stop.alloc, structs.AllocDesiredStatusStop, stop.statusDescription, stop.clientStatus) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 269233626dcf..a3eb17dd094a 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -106,7 +106,8 @@ func NewAllocReconciler(ctx Context, stack Stack, batch bool, func (a *allocReconciler) Compute() *reconcileResults { // If we are just stopping a job we do not need to do anything more than // stopping all running allocs - if a.job == nil || a.job.Stop { + stopped := a.job == nil || a.job.Stop + if stopped { a.handleStop() // Cancel the deployment since it is not needed @@ -121,6 +122,29 @@ func (a *allocReconciler) Compute() *reconcileResults { return a.result } + // Check if the deployment is referencing an older job + if d := a.deployment; d != nil { + if d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex { + a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ + DeploymentID: a.deployment.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionNewerJob, + }) + a.deployment = nil + } + } + + // Create a new deployment + if a.deployment == nil && !stopped && a.job.HasUpdateStrategy() { + a.deployment = structs.NewDeployment(a.job) + a.result.createDeployment = a.deployment + a.ctx.Logger().Printf("ALEX: MADE DEPLOYMENT %q", a.deployment.ID) + } + + if a.deployment != nil { + a.ctx.Logger().Printf("ALEX: CURRENT DEPLOYMENT %q", a.deployment.ID) + } + m := newAllocMatrix(a.job, a.existingAllocs) for group, as := range m { a.computeGroup(group, as) @@ -225,13 +249,6 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { _, inplace, destructive := a.computeUpdates(tg, untainted) a.ctx.Logger().Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) - // XXX Not clear if this is needed - // Update untainted so that it contains all existing allocations that have - // been inplace updated or do not have to be updated and does not include - // any canaries. - //untainted = untainted.difference(destructive) - //a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) - // Get the update strategy of the group strategy, update := a.groupUpdateStrategy[group] @@ -239,8 +256,9 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // The fact that we have destructive updates and have less canaries than is // desired means we need to create canaries - requireCanary := len(destructive) != 0 && update && strategy.Canary != 0 && len(canaries) < strategy.Canary - if requireCanary && !a.deploymentPaused { + requireCanary := len(destructive) != 0 && update && len(canaries) < strategy.Canary + placeCanaries := requireCanary && !a.deploymentPaused + if placeCanaries { a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", strategy.Canary-len(canaries)) for i := len(canaries); i < strategy.Canary; i++ { a.result.place = append(a.result.place, allocPlaceResult{ @@ -253,12 +271,8 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } // Determine how many we can place - limit := tg.Count - if update { - // XXX This is wrong. Need to detect health first. Probably only within - // the truly untainted set - limit = strategy.MaxParallel - } + haveCanaries := len(canaries) != 0 || placeCanaries + limit := a.computeLimit(tg, strategy, untainted, haveCanaries) a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) // Place if: @@ -299,18 +313,9 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { limit -= min } - // Migrations should be done under the rolling update strategy, however we - // do not abide by the paused state of the deployment since this could block - // node draining. - min := helper.IntMin(len(migrate), limit) - i := 0 - a.ctx.Logger().Printf("RECONCILER -- Migrating (%d)", min) + // TODO Migrations should be done using a stagger and max_parallel. + a.ctx.Logger().Printf("RECONCILER -- Migrating (%d)", len(migrate)) for _, alloc := range migrate { - if i == min { - break - } - i++ - a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, statusDescription: allocMigrating, @@ -324,6 +329,38 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } } +func (a *allocReconciler) computeLimit(group *structs.TaskGroup, strategy *structs.UpdateStrategy, untainted allocSet, canaries bool) int { + // If there is no update stategy or deployment for the group we can deploy + // as many as the group has + if strategy == nil || a.deployment == nil { + return group.Count + } else if a.deploymentPaused { + // If the deployment is paused, do not create anything else + return 0 + } + + // Get the state of the deployment for the group + deploymentState := a.deployment.TaskGroups[group.Name] + + // If we have canaries and they have not been promoted the limit is 0 + if canaries && (deploymentState == nil || !deploymentState.Promoted) { + return 0 + } + + // If we have been promoted or there are no canaries, the limit is the + // configured MaxParallel - any outstanding non-healthy alloc for the + // deployment + limit := strategy.MaxParallel + partOf, _ := untainted.filterByDeployment(a.deployment.ID) + for _, alloc := range partOf { + if alloc.DeploymentStatus == nil || alloc.DeploymentStatus.Healthy == nil { + limit-- + } + } + + return limit +} + func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted, destructiveUpdates allocSet) []allocPlaceResult { // Hot path the nothing to do case existing := len(untainted) + len(destructiveUpdates) From 0d3c836c6942ba29a827517dd7aeb7ac43ed9cfa Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 22 May 2017 17:06:46 -0700 Subject: [PATCH 004/105] Small cleanup --- nomad/structs/structs.go | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index bfbb2952dce0..b2da436566c6 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1515,6 +1515,7 @@ func (j *Job) Stopped() bool { return j == nil || j.Stop } +// HasUpdateStrategy returns if any task group in the job has an update strategy func (j *Job) HasUpdateStrategy() bool { for _, tg := range j.TaskGroups { if tg.Update != nil { @@ -3803,6 +3804,7 @@ type Deployment struct { ModifyIndex uint64 } +// NewDeployment creates a new deployment given the job. func NewDeployment(job *Job) *Deployment { return &Deployment{ ID: GenerateUUID(), @@ -4649,17 +4651,6 @@ func (p *Plan) AppendAlloc(alloc *Allocation) { p.NodeAllocation[node] = append(existing, alloc) } -// AppendDeploymentUpdate attaches an deployment update to the plan for the -// given deployment ID. -func (p *Plan) AppendDeploymentUpdate(id, status, description string) { - update := &DeploymentStatusUpdate{ - DeploymentID: id, - Status: status, - StatusDescription: description, - } - p.DeploymentUpdates = append(p.DeploymentUpdates, update) -} - // IsNoOp checks if this plan would do nothing func (p *Plan) IsNoOp() bool { return len(p.NodeUpdate) == 0 && From 4bbf24a8751d02b1a1b885cb6e48f208a8912b0e Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 22 May 2017 17:14:38 -0700 Subject: [PATCH 005/105] Split reconcile file --- scheduler/reconcile.go | 127 ------------------------------------ scheduler/reconcile_util.go | 123 ++++++++++++++++++++++++++++++++++ 2 files changed, 123 insertions(+), 127 deletions(-) create mode 100644 scheduler/reconcile_util.go diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index a3eb17dd094a..8d9e822592b2 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -1,7 +1,6 @@ package scheduler import ( - "fmt" "sort" memdb "github.com/hashicorp/go-memdb" @@ -497,129 +496,3 @@ func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted all return } - -type allocMatrix map[string]allocSet - -func newAllocMatrix(job *structs.Job, allocs []*structs.Allocation) allocMatrix { - m := allocMatrix(make(map[string]allocSet)) - for _, a := range allocs { - s, ok := m[a.TaskGroup] - if !ok { - s = make(map[string]*structs.Allocation) - m[a.TaskGroup] = s - } - s[a.ID] = a - } - for _, tg := range job.TaskGroups { - s, ok := m[tg.Name] - if !ok { - s = make(map[string]*structs.Allocation) - m[tg.Name] = s - } - } - return m -} - -type allocSet map[string]*structs.Allocation - -func (a allocSet) GoString() string { - if len(a) == 0 { - return "[]" - } - - start := fmt.Sprintf("len(%d) [\n", len(a)) - for k := range a { - start += k + ",\n" - } - return start + "]" -} - -func newAllocSet(allocs []*structs.Allocation) allocSet { - s := make(map[string]*structs.Allocation, len(allocs)) - for _, a := range allocs { - s[a.ID] = a - } - return s -} - -func (a allocSet) difference(others ...allocSet) allocSet { - diff := make(map[string]*structs.Allocation) -OUTER: - for k, v := range a { - for _, other := range others { - if _, ok := other[k]; ok { - continue OUTER - } - } - diff[k] = v - } - return diff -} - -func (a allocSet) filterByTainted(nodes map[string]*structs.Node) (untainted, migrate, lost allocSet) { - untainted = make(map[string]*structs.Allocation) - migrate = make(map[string]*structs.Allocation) - lost = make(map[string]*structs.Allocation) - for _, alloc := range a { - n, ok := nodes[alloc.NodeID] - switch { - case !ok: - untainted[alloc.ID] = alloc - case n == nil || n.TerminalStatus(): - lost[alloc.ID] = alloc - default: - migrate[alloc.ID] = alloc - } - } - return -} - -func (a allocSet) filterByCanary() allocSet { - canaries := make(map[string]*structs.Allocation) - for _, alloc := range a { - if alloc.Canary { - canaries[alloc.ID] = alloc - } - } - return canaries -} - -func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { - match = make(map[string]*structs.Allocation) - nonmatch = make(map[string]*structs.Allocation) - for _, alloc := range a { - if alloc.DeploymentID == id { - match[alloc.ID] = alloc - } else { - nonmatch[alloc.ID] = alloc - } - } - return -} - -func (a allocSet) filterByDeploymentHealthy() allocSet { - healthy := make(map[string]*structs.Allocation) - for _, alloc := range a { - if alloc.DeploymentStatus != nil && - alloc.DeploymentStatus.Healthy != nil && - *alloc.DeploymentStatus.Healthy { - healthy[alloc.ID] = alloc - } - } - return healthy -} - -func (a allocSet) filterByUpdateType(ctx Context, stack Stack, eval *structs.Evaluation, - job *structs.Job) (ignore, inplace, destructive allocSet) { - ignore = make(map[string]*structs.Allocation) - inplace = make(map[string]*structs.Allocation) - destructive = make(map[string]*structs.Allocation) - - for _, alloc := range a { - if alloc.Job.JobModifyIndex == job.JobModifyIndex { - ignore[alloc.ID] = alloc - continue - } - } - return -} diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go new file mode 100644 index 000000000000..c0f08c29db38 --- /dev/null +++ b/scheduler/reconcile_util.go @@ -0,0 +1,123 @@ +package scheduler + +import ( + "fmt" + + "github.com/hashicorp/nomad/nomad/structs" +) + +// allocMatrix is a mapping of task groups to their allocation set. +type allocMatrix map[string]allocSet + +// newAllocMatrix takes a job and the existing allocations for the job and +// creates an allocMatrix +func newAllocMatrix(job *structs.Job, allocs []*structs.Allocation) allocMatrix { + m := allocMatrix(make(map[string]allocSet)) + for _, a := range allocs { + s, ok := m[a.TaskGroup] + if !ok { + s = make(map[string]*structs.Allocation) + m[a.TaskGroup] = s + } + s[a.ID] = a + } + for _, tg := range job.TaskGroups { + s, ok := m[tg.Name] + if !ok { + s = make(map[string]*structs.Allocation) + m[tg.Name] = s + } + } + return m +} + +// allocSet is a set of allocations with a series of helper functions defined +// that help reconcile state. +type allocSet map[string]*structs.Allocation + +// newAllocSet creates an allocation set given a set of allocations +func newAllocSet(allocs []*structs.Allocation) allocSet { + s := make(map[string]*structs.Allocation, len(allocs)) + for _, a := range allocs { + s[a.ID] = a + } + return s +} + +// GoString provides a human readable view of the set +func (a allocSet) GoString() string { + if len(a) == 0 { + return "[]" + } + + start := fmt.Sprintf("len(%d) [\n", len(a)) + for k := range a { + start += k + ",\n" + } + return start + "]" +} + +// difference returns a new allocSet that has all the existing item except those +// contained within the other allocation sets +func (a allocSet) difference(others ...allocSet) allocSet { + diff := make(map[string]*structs.Allocation) +OUTER: + for k, v := range a { + for _, other := range others { + if _, ok := other[k]; ok { + continue OUTER + } + } + diff[k] = v + } + return diff +} + +// fitlerByTainted takes a set of tainted nodes and filters the allocation set +// into three groups: +// 1. Those that exist on untainted nodes +// 2. Those exist on nodes that are draining +// 3. Those that exist on lost nodes +func (a allocSet) filterByTainted(nodes map[string]*structs.Node) (untainted, migrate, lost allocSet) { + untainted = make(map[string]*structs.Allocation) + migrate = make(map[string]*structs.Allocation) + lost = make(map[string]*structs.Allocation) + for _, alloc := range a { + n, ok := nodes[alloc.NodeID] + switch { + case !ok: + untainted[alloc.ID] = alloc + case n == nil || n.TerminalStatus(): + lost[alloc.ID] = alloc + default: + migrate[alloc.ID] = alloc + } + } + return +} + +// filterByCanary returns a new allocation set that contains only canaries +func (a allocSet) filterByCanary() allocSet { + canaries := make(map[string]*structs.Allocation) + for _, alloc := range a { + if alloc.Canary { + canaries[alloc.ID] = alloc + } + } + return canaries +} + +// filterByDeployment filters allocations into two sets, those that match the +// given deployment ID and those that don't +func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { + match = make(map[string]*structs.Allocation) + nonmatch = make(map[string]*structs.Allocation) + for _, alloc := range a { + if alloc.DeploymentID == id { + match[alloc.ID] = alloc + } else { + nonmatch[alloc.ID] = alloc + } + } + return +} From 3ae7abac3a633f7132fcad10581900a5eb8ea2d2 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 22 May 2017 17:42:41 -0700 Subject: [PATCH 006/105] Some comments and cleanup --- scheduler/reconcile.go | 87 +++++++++++++++++++++++++++--------------- 1 file changed, 56 insertions(+), 31 deletions(-) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 8d9e822592b2..3f03c3a21d86 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -8,6 +8,10 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +// allocReconciler is used to determine the set of allocations that require +// placement, inplace updating or stopping given the job specification and +// existing cluster state. The reconciler should only be used for batch and +// service jobs. type allocReconciler struct { // ctx gives access to the state store and logger ctx Context @@ -31,9 +35,6 @@ type allocReconciler struct { // deploymentPaused marks whether the deployment is paused deploymentPaused bool - // groupUpdateStrategy maps task groups to their update strategy - groupUpdateStrategy map[string]*structs.UpdateStrategy - // taintedNodes contains a map of nodes that are tainted taintedNodes map[string]*structs.Node @@ -45,17 +46,31 @@ type allocReconciler struct { result *reconcileResults } +// reconcileResults contains the results of the reconciliation and should be +// applied by the scheduler. type reconcileResults struct { - createDeployment *structs.Deployment + // createDeployment is the deployment that should be created as a result of + // scheduling + createDeployment *structs.Deployment + + // deploymentUpdates contains a set of deployment updates that should be + // applied as a result of scheduling deploymentUpdates []*structs.DeploymentStatusUpdate - place []allocPlaceResult + // place is the set of allocations to place by the scheduler + place []allocPlaceResult + + // inplaceUpdate is the set of allocations to apply an inplace update to inplaceUpdate []*structs.Allocation - stop []allocStopResult + + // stop is the set of allocations to stop + stop []allocStopResult // TODO track the desired of the deployment } +// allocPlaceResult contains the information required to place a single +// allocation type allocPlaceResult struct { name string canary bool @@ -63,12 +78,15 @@ type allocPlaceResult struct { previousAlloc *structs.Allocation } +// allocStopResult contains the information required to stop a single allocation type allocStopResult struct { alloc *structs.Allocation clientStatus string statusDescription string } +// NewAllocReconciler creates a new reconciler that should be used to determine +// the changes required to bring the cluster state inline with the declared jobspec func NewAllocReconciler(ctx Context, stack Stack, batch bool, eval *structs.Evaluation, job *structs.Job, deployment *structs.Deployment, existingAllocs []*structs.Allocation, taintedNodes map[string]*structs.Node) *allocReconciler { @@ -85,23 +103,16 @@ func NewAllocReconciler(ctx Context, stack Stack, batch bool, result: new(reconcileResults), } + // Detect if the deployment is paused if deployment != nil { a.deploymentPaused = deployment.Status == structs.DeploymentStatusPaused } - // Determine the update strategy for each group - if job != nil { - a.groupUpdateStrategy = make(map[string]*structs.UpdateStrategy) - for _, tg := range job.TaskGroups { - if u := tg.Update; u != nil { - a.groupUpdateStrategy[tg.Name] = u - } - } - } - return a } +// Compute reconciles the existing cluster state and returns the set of changes +// required to converge the job spec and state func (a *allocReconciler) Compute() *reconcileResults { // If we are just stopping a job we do not need to do anything more than // stopping all running allocs @@ -121,7 +132,7 @@ func (a *allocReconciler) Compute() *reconcileResults { return a.result } - // Check if the deployment is referencing an older job + // Check if the deployment is referencing an older job and cancel it if d := a.deployment; d != nil { if d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ @@ -133,7 +144,7 @@ func (a *allocReconciler) Compute() *reconcileResults { } } - // Create a new deployment + // Create a new deployment if necessary if a.deployment == nil && !stopped && a.job.HasUpdateStrategy() { a.deployment = structs.NewDeployment(a.job) a.result.createDeployment = a.deployment @@ -161,6 +172,8 @@ func (a *allocReconciler) handleStop() { a.markStop(lost, structs.AllocClientStatusLost, allocLost) } +// markStop is a helper for marking a set of allocation for stop with a +// particular client status and description. func (a *allocReconciler) markStop(allocs allocSet, clientStatus, statusDescription string) { for _, alloc := range allocs { a.result.stop = append(a.result.stop, allocStopResult{ @@ -171,6 +184,7 @@ func (a *allocReconciler) markStop(allocs allocSet, clientStatus, statusDescript } } +// computeGroup reconciles state for a particular task group. func (a *allocReconciler) computeGroup(group string, as allocSet) { // Get the task group. The task group may be nil if the job was updates such // that the task group no longer exists @@ -239,23 +253,21 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.markStop(stop, "", allocNotNeeded) untainted = keep - a.ctx.Logger().Printf("RECONCILER -- Stopping (%d); Untainted (%d)", len(stop), len(keep)) - a.ctx.Logger().Printf("RECONCILER -- stopping %#v", stop) - a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) - // Do inplace upgrades where possible and capture the set of upgrades that // need to be done destructively. _, inplace, destructive := a.computeUpdates(tg, untainted) + + a.ctx.Logger().Printf("RECONCILER -- Stopping (%d); Untainted (%d)", len(stop), len(keep)) a.ctx.Logger().Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) // Get the update strategy of the group - strategy, update := a.groupUpdateStrategy[group] + strategy := tg.Update // XXX need a structure for picking names // The fact that we have destructive updates and have less canaries than is // desired means we need to create canaries - requireCanary := len(destructive) != 0 && update && len(canaries) < strategy.Canary + requireCanary := len(destructive) != 0 && strategy != nil && len(canaries) < strategy.Canary placeCanaries := requireCanary && !a.deploymentPaused if placeCanaries { a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", strategy.Canary-len(canaries)) @@ -271,7 +283,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // Determine how many we can place haveCanaries := len(canaries) != 0 || placeCanaries - limit := a.computeLimit(tg, strategy, untainted, haveCanaries) + limit := a.computeLimit(tg, untainted, haveCanaries) a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) // Place if: @@ -283,7 +295,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.ctx.Logger().Printf("RECONCILER -- CAN PLACE %v", canPlace) if canPlace { // Place all new allocations - place := a.computePlacements(tg, untainted, destructive) + place := a.computePlacements(tg, untainted) a.ctx.Logger().Printf("RECONCILER -- Placing (%d)", len(place)) for _, p := range place { a.result.place = append(a.result.place, p) @@ -328,10 +340,13 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } } -func (a *allocReconciler) computeLimit(group *structs.TaskGroup, strategy *structs.UpdateStrategy, untainted allocSet, canaries bool) int { +// computeLimit returns the placement limit for a particular group. The inputs +// are the group definition, the existing/untainted allocation set and whether +// any canaries exist or are being placed. +func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted allocSet, canaries bool) int { // If there is no update stategy or deployment for the group we can deploy // as many as the group has - if strategy == nil || a.deployment == nil { + if group.Update == nil || a.deployment == nil { return group.Count } else if a.deploymentPaused { // If the deployment is paused, do not create anything else @@ -349,7 +364,7 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, strategy *struc // If we have been promoted or there are no canaries, the limit is the // configured MaxParallel - any outstanding non-healthy alloc for the // deployment - limit := strategy.MaxParallel + limit := group.Update.MaxParallel partOf, _ := untainted.filterByDeployment(a.deployment.ID) for _, alloc := range partOf { if alloc.DeploymentStatus == nil || alloc.DeploymentStatus.Healthy == nil { @@ -360,9 +375,11 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, strategy *struc return limit } -func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted, destructiveUpdates allocSet) []allocPlaceResult { +// computePlacement returns the set of allocations to place given the group +// definiton and the set of untainted/existing allocations for the group. +func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted allocSet) []allocPlaceResult { // Hot path the nothing to do case - existing := len(untainted) + len(destructiveUpdates) + existing := len(untainted) if existing == group.Count { return nil } @@ -379,6 +396,8 @@ func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted, return place } +// computeStop returns the set of allocations to stop given the group definiton +// and the set of untainted/existing allocations for the group. func (a *allocReconciler) computeStop(group *structs.TaskGroup, untainted allocSet) (keep, stop allocSet) { // Hot path the nothing to do case if len(untainted) <= group.Count { @@ -408,6 +427,12 @@ func (a *allocReconciler) computeStop(group *structs.TaskGroup, untainted allocS return } +// computeUpdates determines which allocations for the passed group require +// updates. Three groups are returned: +// 1. Those that require no upgrades +// 2. Those that can be upgraded in-place. These are added to the results +// automatically since the function contains the correct state to do so, +// 3. Those that require destructive updates func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted allocSet) (ignore, inplace, destructive allocSet) { // Determine the set of allocations that need to be updated ignore = make(map[string]*structs.Allocation) From a7df708c143304fe1072beabda199ba3708d3263 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 22 May 2017 17:44:57 -0700 Subject: [PATCH 007/105] Todos --- scheduler/reconcile.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 3f03c3a21d86..1e8fc1e81505 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -8,6 +8,15 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +/* TODO + * 1) We need a structure that yields names in an order that fills the gaps + * between the existing allocations and handles canaries replacing certain + * allocations. + * 2) Need to populate the desired state of a created deployment + * 3) Need to capture the overall desired transformations so that annotated + * plans work. + */ + // allocReconciler is used to determine the set of allocations that require // placement, inplace updating or stopping given the job specification and // existing cluster state. The reconciler should only be used for batch and From 662bc3941b232c8a04b1fd25c7aa082e6bf79f90 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 23 May 2017 13:02:47 -0700 Subject: [PATCH 008/105] handle annotations --- api/jobs.go | 1 + nomad/structs/structs.go | 1 + scheduler/generic_sched.go | 21 ++++++++++++++++ scheduler/reconcile.go | 49 ++++++++++++++++++++++++++------------ 4 files changed, 57 insertions(+), 15 deletions(-) diff --git a/api/jobs.go b/api/jobs.go index 8a8cb515e43a..3766d6b52341 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -791,6 +791,7 @@ type DesiredUpdates struct { Stop uint64 InPlaceUpdate uint64 DestructiveUpdate uint64 + Canary uint64 } type JobDispatchRequest struct { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index b2da436566c6..e0d8a1802dfe 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4713,6 +4713,7 @@ type DesiredUpdates struct { Stop uint64 InPlaceUpdate uint64 DestructiveUpdate uint64 + Canary uint64 } // msgpackHandle is a shared handle for encoding/decoding of structs diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 19415f67e545..676ff4c4e3cb 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -386,6 +386,12 @@ func (s *GenericScheduler) computeJobAllocs2() error { reconciler := NewAllocReconciler(s.ctx, s.stack, s.batch, s.eval, s.job, s.deployment, allocs, tainted) results := reconciler.Compute() + if s.eval.AnnotatePlan { + s.plan.Annotations = &structs.PlanAnnotations{ + DesiredTGUpdates: results.desiredTGUpdates, + } + } + // Add the deployment changes to the plan s.plan.CreatedDeployment = results.createDeployment s.plan.DeploymentUpdates = results.deploymentUpdates @@ -405,6 +411,21 @@ func (s *GenericScheduler) computeJobAllocs2() error { s.ctx.Plan().AppendAlloc(update) } + // Nothing remaining to do if placement is not required + if len(results.place) == 0 { + if !s.job.Stopped() { + for _, tg := range s.job.TaskGroups { + s.queuedAllocs[tg.Name] = 0 + } + } + return nil + } + + // Record the number of allocations that needs to be placed per Task Group + for _, place := range results.place { + s.queuedAllocs[place.taskGroup.Name] += 1 + } + // Compute the placements return s.computePlacements2(results.place) } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 1e8fc1e81505..a15180358c3e 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -13,8 +13,6 @@ import ( * between the existing allocations and handles canaries replacing certain * allocations. * 2) Need to populate the desired state of a created deployment - * 3) Need to capture the overall desired transformations so that annotated - * plans work. */ // allocReconciler is used to determine the set of allocations that require @@ -75,7 +73,9 @@ type reconcileResults struct { // stop is the set of allocations to stop stop []allocStopResult - // TODO track the desired of the deployment + // desiredTGUpdates captures the desired set of changes to make for each + // task group. + desiredTGUpdates map[string]*structs.DesiredUpdates } // allocPlaceResult contains the information required to place a single @@ -109,7 +109,9 @@ func NewAllocReconciler(ctx Context, stack Stack, batch bool, deployment: deployment, existingAllocs: existingAllocs, taintedNodes: taintedNodes, - result: new(reconcileResults), + result: &reconcileResults{ + desiredTGUpdates: make(map[string]*structs.DesiredUpdates), + }, } // Detect if the deployment is paused @@ -195,6 +197,10 @@ func (a *allocReconciler) markStop(allocs allocSet, clientStatus, statusDescript // computeGroup reconciles state for a particular task group. func (a *allocReconciler) computeGroup(group string, as allocSet) { + // Create the desired update object for the group + desiredChanges := new(structs.DesiredUpdates) + a.result.desiredTGUpdates[group] = desiredChanges + // Get the task group. The task group may be nil if the job was updates such // that the task group no longer exists tg := a.job.LookupTaskGroup(group) @@ -202,9 +208,6 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // Determine what set of alloations are on tainted nodes untainted, migrate, lost := as.filterByTainted(a.taintedNodes) - a.ctx.Logger().Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) - a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) - // If the task group is nil, then the task group has been removed so all we // need to do is stop everything if tg == nil { @@ -212,14 +215,15 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.markStop(untainted, "", allocNotNeeded) a.markStop(migrate, "", allocNotNeeded) a.markStop(lost, structs.AllocClientStatusLost, allocLost) + desiredChanges.Stop = uint64(len(untainted) + len(migrate) + len(lost)) return } - // Get the deployment state for the group - var dstate *structs.DeploymentState - if a.deployment != nil { - dstate = a.deployment.TaskGroups[group] - } + // Track the lost and migrating + desiredChanges.Migrate += uint64(len(migrate) + len(lost)) + + a.ctx.Logger().Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) + a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) // Mark all lost allocations for stop. Previous allocation doesn't matter // here since it is on a lost node @@ -239,6 +243,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { if a.deployment != nil { current, older := canaries.filterByDeployment(a.deployment.ID) a.markStop(older, "", allocNotNeeded) + desiredChanges.Stop += uint64(len(older)) a.ctx.Logger().Printf("RECONCILER -- older canaries %#v", older) a.ctx.Logger().Printf("RECONCILER -- current canaries %#v", current) @@ -250,6 +255,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // We don't need any of those canaries since there no longer is a // deployment a.markStop(canaries, "", allocNotNeeded) + desiredChanges.Stop += uint64(len(canaries)) untainted = untainted.difference(canaries) canaries = nil a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) @@ -260,11 +266,15 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // included stopped allocations keep, stop := a.computeStop(tg, untainted) a.markStop(stop, "", allocNotNeeded) + desiredChanges.Stop += uint64(len(stop)) untainted = keep // Do inplace upgrades where possible and capture the set of upgrades that // need to be done destructively. - _, inplace, destructive := a.computeUpdates(tg, untainted) + ignore, inplace, destructive := a.computeUpdates(tg, untainted) + desiredChanges.Ignore += uint64(len(ignore)) + desiredChanges.InPlaceUpdate += uint64(len(inplace)) + desiredChanges.DestructiveUpdate += uint64(len(destructive)) a.ctx.Logger().Printf("RECONCILER -- Stopping (%d); Untainted (%d)", len(stop), len(keep)) a.ctx.Logger().Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) @@ -279,8 +289,10 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { requireCanary := len(destructive) != 0 && strategy != nil && len(canaries) < strategy.Canary placeCanaries := requireCanary && !a.deploymentPaused if placeCanaries { - a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", strategy.Canary-len(canaries)) - for i := len(canaries); i < strategy.Canary; i++ { + number := strategy.Canary - len(canaries) + desiredChanges.Canary += uint64(number) + a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", number) + for i := 0; i < number; i++ { a.result.place = append(a.result.place, allocPlaceResult{ // XXX Pick better name name: structs.GenerateUUID(), @@ -295,6 +307,12 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { limit := a.computeLimit(tg, untainted, haveCanaries) a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) + // Get the deployment state for the group + var dstate *structs.DeploymentState + if a.deployment != nil { + dstate = a.deployment.TaskGroups[group] + } + // Place if: // * The deployment is not paused // * Not placing any canaries @@ -305,6 +323,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { if canPlace { // Place all new allocations place := a.computePlacements(tg, untainted) + desiredChanges.Place += uint64(len(place)) a.ctx.Logger().Printf("RECONCILER -- Placing (%d)", len(place)) for _, p := range place { a.result.place = append(a.result.place, p) From 66789e81f25e703834a24b2d7f4a29bcbcd4f527 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 23 May 2017 15:16:44 -0700 Subject: [PATCH 009/105] Show canaries on plan --- command/plan.go | 4 +++- scheduler/annotate.go | 4 ++++ scheduler/annotate_test.go | 2 ++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/command/plan.go b/command/plan.go index f21fbba85200..f86bd9dca5ae 100644 --- a/command/plan.go +++ b/command/plan.go @@ -151,7 +151,7 @@ func (c *PlanCommand) Run(args []string) int { func getExitCode(resp *api.JobPlanResponse) int { // Check for changes for _, d := range resp.Annotations.DesiredTGUpdates { - if d.Stop+d.Place+d.Migrate+d.DestructiveUpdate > 0 { + if d.Stop+d.Place+d.Migrate+d.DestructiveUpdate+d.Canary > 0 { return 1 } } @@ -288,6 +288,8 @@ func formatTaskGroupDiff(tg *api.TaskGroupDiff, tgPrefix int, verbose bool) stri color = "[cyan]" case scheduler.UpdateTypeDestructiveUpdate: color = "[yellow]" + case scheduler.UpdateTypeCanary: + color = "[light_yellow]" } updates = append(updates, fmt.Sprintf("[reset]%s%d %s", color, count, updateType)) } diff --git a/scheduler/annotate.go b/scheduler/annotate.go index 05f768b41bdd..a2509bb7376a 100644 --- a/scheduler/annotate.go +++ b/scheduler/annotate.go @@ -19,6 +19,7 @@ const ( UpdateTypeCreate = "create" UpdateTypeDestroy = "destroy" UpdateTypeMigrate = "migrate" + UpdateTypeCanary = "canary" UpdateTypeInplaceUpdate = "in-place update" UpdateTypeDestructiveUpdate = "create/destroy update" ) @@ -71,6 +72,9 @@ func annotateTaskGroup(diff *structs.TaskGroupDiff, annotations *structs.PlanAnn if tg.Stop != 0 { diff.Updates[UpdateTypeDestroy] = tg.Stop } + if tg.Canary != 0 { + diff.Updates[UpdateTypeCanary] = tg.Canary + } if tg.InPlaceUpdate != 0 { diff.Updates[UpdateTypeInplaceUpdate] = tg.InPlaceUpdate } diff --git a/scheduler/annotate_test.go b/scheduler/annotate_test.go index ec9d3bc2552b..0836ff3e8934 100644 --- a/scheduler/annotate_test.go +++ b/scheduler/annotate_test.go @@ -17,6 +17,7 @@ func TestAnnotateTaskGroup_Updates(t *testing.T) { Stop: 4, InPlaceUpdate: 5, DestructiveUpdate: 6, + Canary: 7, }, }, } @@ -35,6 +36,7 @@ func TestAnnotateTaskGroup_Updates(t *testing.T) { UpdateTypeDestroy: 4, UpdateTypeInplaceUpdate: 5, UpdateTypeDestructiveUpdate: 6, + UpdateTypeCanary: 7, }, } From 201ad8bf40c51008b3613d16cd39f64f6780e1b1 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 23 May 2017 16:08:35 -0700 Subject: [PATCH 010/105] Populate desired state per tg --- nomad/structs/structs.go | 1 + scheduler/reconcile.go | 46 +++++++++++++++++++++++++++------------- 2 files changed, 32 insertions(+), 15 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index e0d8a1802dfe..e2b4942cc096 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3813,6 +3813,7 @@ func NewDeployment(job *Job) *Deployment { JobModifyIndex: job.ModifyIndex, JobCreateIndex: job.CreateIndex, Status: DeploymentStatusRunning, + TaskGroups: make(map[string]*DeploymentState, len(job.TaskGroups)), } } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index a15180358c3e..44cd166678fd 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -12,7 +12,6 @@ import ( * 1) We need a structure that yields names in an order that fills the gaps * between the existing allocations and handles canaries replacing certain * allocations. - * 2) Need to populate the desired state of a created deployment */ // allocReconciler is used to determine the set of allocations that require @@ -284,13 +283,31 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // XXX need a structure for picking names + // Get the deployment state for the group + creatingDeployment := a.result.createDeployment != nil + var dstate *structs.DeploymentState + if a.deployment != nil { + var ok bool + dstate, ok = a.deployment.TaskGroups[group] + + // We are creating a deployment + if !ok && creatingDeployment { + dstate = &structs.DeploymentState{} + a.deployment.TaskGroups[group] = dstate + } + } + // The fact that we have destructive updates and have less canaries than is // desired means we need to create canaries requireCanary := len(destructive) != 0 && strategy != nil && len(canaries) < strategy.Canary - placeCanaries := requireCanary && !a.deploymentPaused - if placeCanaries { + if requireCanary && !a.deploymentPaused { number := strategy.Canary - len(canaries) desiredChanges.Canary += uint64(number) + if creatingDeployment { + dstate.DesiredCanaries = strategy.Canary + dstate.DesiredTotal += strategy.Canary + } + a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", number) for i := 0; i < number; i++ { a.result.place = append(a.result.place, allocPlaceResult{ @@ -303,27 +320,26 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } // Determine how many we can place - haveCanaries := len(canaries) != 0 || placeCanaries + haveCanaries := dstate != nil && dstate.DesiredCanaries != 0 limit := a.computeLimit(tg, untainted, haveCanaries) a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) - // Get the deployment state for the group - var dstate *structs.DeploymentState - if a.deployment != nil { - dstate = a.deployment.TaskGroups[group] - } - // Place if: // * The deployment is not paused // * Not placing any canaries // * If there are any canaries that they have been promoted + place := a.computePlacements(tg, untainted) + if creatingDeployment { + dstate.DesiredTotal += len(place) + } + existingCanariesPromoted := dstate == nil || dstate.DesiredCanaries == 0 || dstate.Promoted - canPlace := !a.deploymentPaused && !requireCanary && existingCanariesPromoted - a.ctx.Logger().Printf("RECONCILER -- CAN PLACE %v", canPlace) - if canPlace { - // Place all new allocations - place := a.computePlacements(tg, untainted) + if !a.deploymentPaused && existingCanariesPromoted { + // Update the desired changes and if we are creating a deployment update + // the state. desiredChanges.Place += uint64(len(place)) + + // Place all new allocations a.ctx.Logger().Printf("RECONCILER -- Placing (%d)", len(place)) for _, p := range place { a.result.place = append(a.result.place, p) From 2dc0268406c1c3ccccc11f31af819cdf105e3ab9 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 23 May 2017 16:33:55 -0700 Subject: [PATCH 011/105] Plan reuses job where possible --- nomad/job_endpoint.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 0a1cf719b4fd..e7ead135356a 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -768,13 +768,19 @@ func (j *Job) Plan(args *structs.JobPlanRequest, reply *structs.JobPlanResponse) var index uint64 var updatedIndex uint64 - if oldJob != nil { + + // We want to reused deployments where possible, so only insert the job if + // it has changed or the job didn't exist + if oldJob != nil && oldJob.SpecChanged(args.Job) { index = oldJob.JobModifyIndex updatedIndex = oldJob.JobModifyIndex + 1 - } - // Insert the updated Job into the snapshot - snap.UpsertJob(updatedIndex, args.Job) + // Insert the updated Job into the snapshot + snap.UpsertJob(updatedIndex, args.Job) + } else if oldJob == nil { + // Insert the updated Job into the snapshot + snap.UpsertJob(100, args.Job) + } // Create an eval and mark it as requiring annotations and insert that as well eval := &structs.Evaluation{ From 4c9e5c34fa41d2934b87a5b41868fd9db3a3b613 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 23 May 2017 16:39:15 -0700 Subject: [PATCH 012/105] Remove old --- scheduler/generic_sched.go | 192 ++----------------------------------- 1 file changed, 6 insertions(+), 186 deletions(-) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 676ff4c4e3cb..786012e05093 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -223,7 +223,7 @@ func (s *GenericScheduler) process() (bool, error) { } // Compute the target job allocations - if err := s.computeJobAllocs2(); err != nil { + if err := s.computeJobAllocs(); err != nil { s.logger.Printf("[ERR] sched: %#v: %v", s.eval, err) return false, err } @@ -360,7 +360,7 @@ func (s *GenericScheduler) filterCompleteAllocs(allocs []*structs.Allocation) ([ // computeJobAllocs is used to reconcile differences between the job, // existing allocations and node status to update the allocations. -func (s *GenericScheduler) computeJobAllocs2() error { +func (s *GenericScheduler) computeJobAllocs() error { // Lookup the allocations by JobID ws := memdb.NewWatchSet() allocs, err := s.state.AllocsByJob(ws, s.eval.JobID, true) @@ -427,96 +427,11 @@ func (s *GenericScheduler) computeJobAllocs2() error { } // Compute the placements - return s.computePlacements2(results.place) -} - -// computeJobAllocs is used to reconcile differences between the job, -// existing allocations and node status to update the allocations. -func (s *GenericScheduler) computeJobAllocs() error { - // Materialize all the task groups, job could be missing if deregistered - var groups map[string]*structs.TaskGroup - if !s.job.Stopped() { - groups = materializeTaskGroups(s.job) - } - - // Lookup the allocations by JobID - ws := memdb.NewWatchSet() - allocs, err := s.state.AllocsByJob(ws, s.eval.JobID, true) - 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 - // nodes to lost - updateNonTerminalAllocsToLost(s.plan, tainted, allocs) - - // Filter out the allocations in a terminal state - allocs, terminalAllocs := s.filterCompleteAllocs(allocs) - - // Diff the required and existing allocations - diff := diffAllocs(s.job, tainted, groups, allocs, terminalAllocs) - s.logger.Printf("[DEBUG] sched: %#v: %#v", s.eval, diff) - - // Add all the allocs to stop - for _, e := range diff.stop { - s.plan.AppendUpdate(e.Alloc, structs.AllocDesiredStatusStop, allocNotNeeded, "") - } - - // Attempt to do the upgrades in place - destructiveUpdates, inplaceUpdates := inplaceUpdate(s.ctx, s.eval, s.job, s.stack, diff.update) - diff.update = destructiveUpdates - - if s.eval.AnnotatePlan { - s.plan.Annotations = &structs.PlanAnnotations{ - DesiredTGUpdates: desiredUpdates(diff, inplaceUpdates, destructiveUpdates), - } - } - - // Check if a rolling upgrade strategy is being used - limit := len(diff.update) + len(diff.migrate) + len(diff.lost) - if !s.job.Stopped() && s.job.Update.Rolling() { - limit = s.job.Update.MaxParallel - } - - // Treat migrations as an eviction and a new placement. - s.limitReached = evictAndPlace(s.ctx, diff, diff.migrate, allocMigrating, &limit) - - // Treat non in-place updates as an eviction and new placement. - s.limitReached = s.limitReached || evictAndPlace(s.ctx, diff, diff.update, allocUpdating, &limit) - - // Lost allocations should be transistioned to desired status stop and client - // status lost and a new placement should be made - s.limitReached = s.limitReached || markLostAndPlace(s.ctx, diff, diff.lost, allocLost, &limit) - - // Nothing remaining to do if placement is not required - if len(diff.place) == 0 { - if !s.job.Stopped() { - for _, tg := range s.job.TaskGroups { - s.queuedAllocs[tg.Name] = 0 - } - } - return nil - } - - // Record the number of allocations that needs to be placed per Task Group - for _, allocTuple := range diff.place { - s.queuedAllocs[allocTuple.TaskGroup.Name] += 1 - } - - // Compute the placements - return s.computePlacements(diff.place) + return s.computePlacements(results.place) } // computePlacements computes placements for allocations -func (s *GenericScheduler) computePlacements2(place []allocPlaceResult) error { +func (s *GenericScheduler) computePlacements(place []allocPlaceResult) error { // Get the base nodes nodes, byDC, err := readyNodesInDCs(s.state, s.job.Datacenters) if err != nil { @@ -539,7 +454,7 @@ func (s *GenericScheduler) computePlacements2(place []allocPlaceResult) error { } // Find the preferred node - preferredNode, err := s.findPreferredNode2(&missing) + preferredNode, err := s.findPreferredNode(&missing) if err != nil { return err } @@ -597,83 +512,8 @@ func (s *GenericScheduler) computePlacements2(place []allocPlaceResult) error { return nil } -// computePlacements computes placements for allocations -func (s *GenericScheduler) computePlacements(place []allocTuple) error { - // Get the base nodes - nodes, byDC, err := readyNodesInDCs(s.state, s.job.Datacenters) - if err != nil { - return err - } - - // Update the set of placement ndoes - s.stack.SetNodes(nodes) - - for _, missing := range place { - // Check if this task group has already failed - if metric, ok := s.failedTGAllocs[missing.TaskGroup.Name]; ok { - metric.CoalescedFailures += 1 - continue - } - - // Find the preferred node - preferredNode, err := s.findPreferredNode(&missing) - if err != nil { - return err - } - - // Attempt to match the task group - var option *RankedNode - if preferredNode != nil { - option, _ = s.stack.SelectPreferringNodes(missing.TaskGroup, []*structs.Node{preferredNode}) - } else { - option, _ = s.stack.Select(missing.TaskGroup) - } - - // Store the available nodes by datacenter - s.ctx.Metrics().NodesAvailable = byDC - - // Set fields based on if we found an allocation option - if option != nil { - // Create an allocation for this - alloc := &structs.Allocation{ - ID: structs.GenerateUUID(), - EvalID: s.eval.ID, - Name: missing.Name, - JobID: s.job.ID, - TaskGroup: missing.TaskGroup.Name, - Metrics: s.ctx.Metrics(), - NodeID: option.Node.ID, - TaskResources: option.TaskResources, - DesiredStatus: structs.AllocDesiredStatusRun, - ClientStatus: structs.AllocClientStatusPending, - - SharedResources: &structs.Resources{ - DiskMB: missing.TaskGroup.EphemeralDisk.SizeMB, - }, - } - - // If the new allocation is replacing an older allocation then we - // set the record the older allocation id so that they are chained - if missing.Alloc != nil { - alloc.PreviousAllocation = missing.Alloc.ID - } - - s.plan.AppendAlloc(alloc) - } else { - // Lazy initialize the failed map - if s.failedTGAllocs == nil { - s.failedTGAllocs = make(map[string]*structs.AllocMetric) - } - - s.failedTGAllocs[missing.TaskGroup.Name] = s.ctx.Metrics() - } - } - - return nil -} - // findPreferredNode finds the preferred node for an allocation -func (s *GenericScheduler) findPreferredNode2(place *allocPlaceResult) (node *structs.Node, err error) { +func (s *GenericScheduler) findPreferredNode(place *allocPlaceResult) (node *structs.Node, err error) { if place.previousAlloc != nil { if place.taskGroup.EphemeralDisk.Sticky == true { var preferredNode *structs.Node @@ -686,23 +526,3 @@ func (s *GenericScheduler) findPreferredNode2(place *allocPlaceResult) (node *st } return } - -// findPreferredNode finds the preferred node for an allocation -func (s *GenericScheduler) findPreferredNode(allocTuple *allocTuple) (node *structs.Node, err error) { - if allocTuple.Alloc != nil { - taskGroup := allocTuple.Alloc.Job.LookupTaskGroup(allocTuple.Alloc.TaskGroup) - if taskGroup == nil { - err = fmt.Errorf("can't find task group of existing allocation %q", allocTuple.Alloc.ID) - return - } - if taskGroup.EphemeralDisk.Sticky == true { - var preferredNode *structs.Node - ws := memdb.NewWatchSet() - preferredNode, err = s.state.NodeByID(ws, allocTuple.Alloc.NodeID) - if preferredNode.Ready() { - node = preferredNode - } - } - } - return -} From 1dabd206bbbb77539f57b41142e5f1986643b1a1 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 24 May 2017 13:43:01 -0700 Subject: [PATCH 013/105] handle batch filtering --- scheduler/reconcile_util.go | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index c0f08c29db38..c30168a0bacd 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -84,12 +84,24 @@ func (a allocSet) filterByTainted(nodes map[string]*structs.Node) (untainted, mi lost = make(map[string]*structs.Allocation) for _, alloc := range a { n, ok := nodes[alloc.NodeID] - switch { - case !ok: + if !ok { + untainted[alloc.ID] = alloc + continue + } + + // If the job is batch and finished successfully, the fact that the + // node is tainted does not mean it should be migrated or marked as + // lost as the work was already successfully finished. However for + // service/system jobs, tasks should never complete. The check of + // batch type, defends against client bugs. + if alloc.Job.Type == structs.JobTypeBatch && alloc.RanSuccessfully() { untainted[alloc.ID] = alloc - case n == nil || n.TerminalStatus(): + continue + } + + if n == nil || n.TerminalStatus() { lost[alloc.ID] = alloc - default: + } else { migrate[alloc.ID] = alloc } } From 85e0d6fccd5eb2d2b792a1eeb2322e0c1dff7167 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 31 May 2017 11:34:46 -0700 Subject: [PATCH 014/105] assign names --- client/driver/env/env.go | 2 +- nomad/structs/bitmap.go | 9 + nomad/structs/bitmap_test.go | 22 +++ nomad/structs/funcs.go | 5 + nomad/structs/structs.go | 35 ++-- nomad/structs/structs_test.go | 18 +- scheduler/generic_sched_test.go | 4 +- scheduler/reconcile.go | 293 +++++++++++++++++++++++++------- scheduler/reconcile_util.go | 16 ++ 9 files changed, 309 insertions(+), 95 deletions(-) diff --git a/client/driver/env/env.go b/client/driver/env/env.go index 481bd88535c7..71bee9e440b6 100644 --- a/client/driver/env/env.go +++ b/client/driver/env/env.go @@ -380,7 +380,7 @@ func (b *Builder) setTask(task *structs.Task) *Builder { func (b *Builder) setAlloc(alloc *structs.Allocation) *Builder { b.allocId = alloc.ID b.allocName = alloc.Name - b.allocIndex = alloc.Index() + b.allocIndex = int(alloc.Index()) b.jobName = alloc.Job.Name // Set meta diff --git a/nomad/structs/bitmap.go b/nomad/structs/bitmap.go index 78a213a3fa4d..63758a0be67b 100644 --- a/nomad/structs/bitmap.go +++ b/nomad/structs/bitmap.go @@ -40,6 +40,15 @@ func (b Bitmap) Set(idx uint) { b[bucket] |= mask } +// Unset is used to unset the given index of the bitmap +func (b Bitmap) Unset(idx uint) { + bucket := idx >> 3 + // Mask should be all ones minus the idx position + offset := 1 << (idx & 7) + mask := byte(offset ^ 0xff) + b[bucket] &= mask +} + // Check is used to check the given index of the bitmap func (b Bitmap) Check(idx uint) bool { bucket := idx >> 3 diff --git a/nomad/structs/bitmap_test.go b/nomad/structs/bitmap_test.go index 83961697de90..42b2c635e2b3 100644 --- a/nomad/structs/bitmap_test.go +++ b/nomad/structs/bitmap_test.go @@ -96,4 +96,26 @@ func TestBitmap(t *testing.T) { t.Fatalf("bad") } } + + // Set a few bits + b.Set(0) + b.Set(255) + b.Unset(0) + b.Unset(255) + + // Clear the bits + if b[0] != 0 { + t.Fatalf("bad") + } + if b.Check(0) { + t.Fatalf("bad") + } + + // Verify the bytes + if b[len(b)-1] != 0 { + t.Fatalf("bad") + } + if b.Check(255) { + t.Fatalf("bad") + } } diff --git a/nomad/structs/funcs.go b/nomad/structs/funcs.go index ab8a980a2cb9..b6c5ef12cae7 100644 --- a/nomad/structs/funcs.go +++ b/nomad/structs/funcs.go @@ -214,3 +214,8 @@ func DenormalizeAllocationJobs(job *Job, allocs []*Allocation) { } } } + +// AllocName returns the name of the allocation given the input. +func AllocName(job, group string, idx uint) string { + return fmt.Sprintf("%s.%s[%d]", job, group, idx) +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index e2b4942cc096..66c45f65dd7b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3976,6 +3976,20 @@ type Allocation struct { CreateTime int64 } +// Index returns the index of the allocation. If the allocation is from a task +// group with count greater than 1, there will be multiple allocations for it. +func (a *Allocation) Index() uint { + l := len(a.Name) + prefix := len(a.JobID) + len(a.TaskGroup) + 2 + if l <= 3 || l <= prefix { + return uint(0) + } + + strNum := a.Name[prefix : len(a.Name)-1] + num, _ := strconv.Atoi(strNum) + return uint(num) +} + func (a *Allocation) Copy() *Allocation { return a.copyImpl(true) } @@ -4109,27 +4123,6 @@ func (a *Allocation) ShouldMigrate() bool { return true } -var ( - // AllocationIndexRegex is a regular expression to find the allocation index. - AllocationIndexRegex = regexp.MustCompile(".+\\[(\\d+)\\]$") -) - -// Index returns the index of the allocation. If the allocation is from a task -// group with count greater than 1, there will be multiple allocations for it. -func (a *Allocation) Index() int { - matches := AllocationIndexRegex.FindStringSubmatch(a.Name) - if len(matches) != 2 { - return -1 - } - - index, err := strconv.Atoi(matches[1]) - if err != nil { - return -1 - } - - return index -} - // AllocListStub is used to return a subset of alloc information type AllocListStub struct { ID string diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index 3942d47a3110..f6e0328f16c4 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -1658,13 +1658,21 @@ func TestRestartPolicy_Validate(t *testing.T) { } func TestAllocation_Index(t *testing.T) { - a1 := Allocation{Name: "example.cache[0]"} - e1 := 0 - a2 := Allocation{Name: "ex[123]am123ple.c311ac[123]he12[1][77]"} - e2 := 77 + a1 := Allocation{ + Name: "example.cache[1]", + TaskGroup: "cache", + JobID: "example", + Job: &Job{ + ID: "example", + TaskGroups: []*TaskGroup{{Name: "cache"}}}, + } + e1 := uint(1) + a2 := a1.Copy() + a2.Name = "example.cache[713127]" + e2 := uint(713127) if a1.Index() != e1 || a2.Index() != e2 { - t.Fatal() + t.Fatalf("Got %d and %d", a1.Index(), a2.Index()) } } diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index e1616b22e004..9ef7c05830fe 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -1289,7 +1289,7 @@ func TestServiceSched_JobModify_CountZero(t *testing.T) { alloc.Job = job alloc.JobID = job.ID alloc.NodeID = nodes[i].ID - alloc.Name = fmt.Sprintf("my-job.web[%d]", i) + alloc.Name = structs.AllocName(alloc.JobID, alloc.TaskGroup, uint(i)) allocs = append(allocs, alloc) } noErr(t, h.State.UpsertAllocs(h.NextIndex(), allocs)) @@ -1301,7 +1301,7 @@ func TestServiceSched_JobModify_CountZero(t *testing.T) { alloc.Job = job alloc.JobID = job.ID alloc.NodeID = nodes[i].ID - alloc.Name = fmt.Sprintf("my-job.web[%d]", i) + alloc.Name = structs.AllocName(alloc.JobID, alloc.TaskGroup, uint(i)) alloc.DesiredStatus = structs.AllocDesiredStatusStop terminal = append(terminal, alloc) } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 44cd166678fd..7d1be535771e 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -1,19 +1,11 @@ package scheduler import ( - "sort" - memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" ) -/* TODO - * 1) We need a structure that yields names in an order that fills the gaps - * between the existing allocations and handles canaries replacing certain - * allocations. - */ - // allocReconciler is used to determine the set of allocations that require // placement, inplace updating or stopping given the job specification and // existing cluster state. The reconciler should only be used for batch and @@ -218,6 +210,20 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { return } + // Get the deployment state for the group + creatingDeployment := a.result.createDeployment != nil + var dstate *structs.DeploymentState + if a.deployment != nil { + var ok bool + dstate, ok = a.deployment.TaskGroups[group] + + // We are creating a deployment + if !ok && creatingDeployment { + dstate = &structs.DeploymentState{} + a.deployment.TaskGroups[group] = dstate + } + } + // Track the lost and migrating desiredChanges.Migrate += uint64(len(migrate) + len(lost)) @@ -247,9 +253,8 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.ctx.Logger().Printf("RECONCILER -- older canaries %#v", older) a.ctx.Logger().Printf("RECONCILER -- current canaries %#v", current) - untainted = untainted.difference(older, current) + untainted = untainted.difference(older) canaries = current - a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) } else { // We don't need any of those canaries since there no longer is a // deployment @@ -257,16 +262,29 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { desiredChanges.Stop += uint64(len(canaries)) untainted = untainted.difference(canaries) canaries = nil - a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) } + a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) } + // Create a structure for choosing names + nameIndex := newAllocNameIndex(a.eval.JobID, group, tg.Count, untainted) + // Stop any unneeded allocations and update the untainted set to not - // included stopped allocations - keep, stop := a.computeStop(tg, untainted) + // included stopped allocations. We ignore canaries since that can push us + // over the desired count + existingCanariesPromoted := dstate == nil || dstate.DesiredCanaries == 0 || dstate.Promoted + stop := a.computeStop(tg, nameIndex, untainted.difference(canaries), canaries, existingCanariesPromoted) a.markStop(stop, "", allocNotNeeded) desiredChanges.Stop += uint64(len(stop)) - untainted = keep + untainted = untainted.difference(stop) + + // Having stopped un-needed allocations, append the canaries to the existing + // set of untainted because they are promoted. This will cause them to be + // treated like non-canaries + if existingCanariesPromoted { + untainted = untainted.union(canaries) + nameIndex.Add(canaries) + } // Do inplace upgrades where possible and capture the set of upgrades that // need to be done destructively. @@ -275,33 +293,19 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { desiredChanges.InPlaceUpdate += uint64(len(inplace)) desiredChanges.DestructiveUpdate += uint64(len(destructive)) - a.ctx.Logger().Printf("RECONCILER -- Stopping (%d); Untainted (%d)", len(stop), len(keep)) + a.ctx.Logger().Printf("RECONCILER -- Stopping (%d)", len(stop)) a.ctx.Logger().Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) // Get the update strategy of the group strategy := tg.Update - // XXX need a structure for picking names - - // Get the deployment state for the group - creatingDeployment := a.result.createDeployment != nil - var dstate *structs.DeploymentState - if a.deployment != nil { - var ok bool - dstate, ok = a.deployment.TaskGroups[group] - - // We are creating a deployment - if !ok && creatingDeployment { - dstate = &structs.DeploymentState{} - a.deployment.TaskGroups[group] = dstate - } - } - // The fact that we have destructive updates and have less canaries than is // desired means we need to create canaries - requireCanary := len(destructive) != 0 && strategy != nil && len(canaries) < strategy.Canary + numDestructive := len(destructive) + requireCanary := numDestructive != 0 && strategy != nil && len(canaries) < strategy.Canary if requireCanary && !a.deploymentPaused { number := strategy.Canary - len(canaries) + number = helper.IntMin(numDestructive, number) desiredChanges.Canary += uint64(number) if creatingDeployment { dstate.DesiredCanaries = strategy.Canary @@ -309,10 +313,9 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", number) - for i := 0; i < number; i++ { + for _, name := range nameIndex.NextCanaries(uint(number), canaries, destructive) { a.result.place = append(a.result.place, allocPlaceResult{ - // XXX Pick better name - name: structs.GenerateUUID(), + name: name, canary: true, taskGroup: tg, }) @@ -321,19 +324,18 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // Determine how many we can place haveCanaries := dstate != nil && dstate.DesiredCanaries != 0 - limit := a.computeLimit(tg, untainted, haveCanaries) + limit := a.computeLimit(tg, untainted, destructive, haveCanaries) a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) // Place if: // * The deployment is not paused // * Not placing any canaries // * If there are any canaries that they have been promoted - place := a.computePlacements(tg, untainted) + place := a.computePlacements(tg, nameIndex, untainted) if creatingDeployment { dstate.DesiredTotal += len(place) } - existingCanariesPromoted := dstate == nil || dstate.DesiredCanaries == 0 || dstate.Promoted if !a.deploymentPaused && existingCanariesPromoted { // Update the desired changes and if we are creating a deployment update // the state. @@ -387,10 +389,10 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // computeLimit returns the placement limit for a particular group. The inputs // are the group definition, the existing/untainted allocation set and whether // any canaries exist or are being placed. -func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted allocSet, canaries bool) int { +func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, destructive allocSet, canaries bool) int { // If there is no update stategy or deployment for the group we can deploy // as many as the group has - if group.Update == nil || a.deployment == nil { + if group.Update == nil || len(destructive) == 0 { return group.Count } else if a.deploymentPaused { // If the deployment is paused, do not create anything else @@ -421,18 +423,19 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted alloc // computePlacement returns the set of allocations to place given the group // definiton and the set of untainted/existing allocations for the group. -func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted allocSet) []allocPlaceResult { +func (a *allocReconciler) computePlacements(group *structs.TaskGroup, + nameIndex *allocNameIndex, untainted allocSet) []allocPlaceResult { + // Hot path the nothing to do case existing := len(untainted) - if existing == group.Count { + if existing >= group.Count { return nil } - // XXX need to pick better names var place []allocPlaceResult - for i := existing; i < group.Count; i++ { + for _, name := range nameIndex.Next(uint(group.Count - existing)) { place = append(place, allocPlaceResult{ - name: structs.GenerateUUID(), + name: name, taskGroup: group, }) } @@ -441,34 +444,28 @@ func (a *allocReconciler) computePlacements(group *structs.TaskGroup, untainted } // computeStop returns the set of allocations to stop given the group definiton -// and the set of untainted/existing allocations for the group. -func (a *allocReconciler) computeStop(group *structs.TaskGroup, untainted allocSet) (keep, stop allocSet) { +// and the set of untainted and canary allocations for the group. +func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *allocNameIndex, + untainted, canaries allocSet, promoted bool) allocSet { // Hot path the nothing to do case - if len(untainted) <= group.Count { - return untainted, nil + remove := len(untainted) - group.Count + if promoted { + remove += len(canaries) } - - // XXX Sort doesn't actually do the right thing "foo.bar[11]" < "foo.bar[3]" - // TODO make name tree - names := make([]string, 0, len(untainted)) - for name := range untainted { - names = append(names, name) + if remove <= 0 { + return nil } - sort.Strings(names) - - keep = make(map[string]*structs.Allocation) - stop = make(map[string]*structs.Allocation) - for i, name := range names { - a := untainted[name] - if i < group.Count { - keep[a.Name] = a - } else { - stop[a.Name] = a + // nameIndex does not include the canaries + removeNames := nameIndex.Highest(uint(remove)) + stop := make(map[string]*structs.Allocation) + for id, a := range untainted { + if _, remove := removeNames[a.Name]; remove { + stop[id] = a } } - return + return stop } // computeUpdates determines which allocations for the passed group require @@ -565,3 +562,167 @@ func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted all return } + +// allocNameIndex is used to select allocation names for placement or removal +// given an existing set of placed allocations. +type allocNameIndex struct { + job, taskGroup string + count int + b structs.Bitmap +} + +// newAllocNameIndex returns an allocNameIndex for use in selecting names of +// allocations to create or stop. It takes the job and task group name, desired +// count and any existing allocations as input. +func newAllocNameIndex(job, taskGroup string, count int, in allocSet) *allocNameIndex { + return &allocNameIndex{ + count: count, + b: bitmapFrom(in, uint(count)), + job: job, + taskGroup: taskGroup, + } +} + +func bitmapFrom(input allocSet, minSize uint) structs.Bitmap { + var max uint + for _, a := range input { + if num := a.Index(); num > max { + max = num + } + } + + if l := uint(len(input)); minSize < l { + minSize = l + } + if max < minSize { + max = minSize + } + if max == 0 { + max = 8 + } + + // byteAlign the count + if remainder := max % 8; remainder != 0 { + max = max + 8 - remainder + } + + bitmap, err := structs.NewBitmap(max) + if err != nil { + panic(err) + } + + for _, a := range input { + bitmap.Set(a.Index()) + } + + return bitmap +} + +// Add adds the allocations to the name index +func (a *allocNameIndex) Add(set allocSet) { + for _, alloc := range set { + a.b.Set(alloc.Index()) + } +} + +// RemoveHighest removes and returns the hightest n used names. The returned set +// can be less than n if there aren't n names set in the index +func (a *allocNameIndex) Highest(n uint) map[string]struct{} { + h := make(map[string]struct{}, n) + for i := a.b.Size(); i > uint(0) && uint(len(h)) <= n; i-- { + // Use this to avoid wrapping around b/c of the unsigned int + idx := i - 1 + if a.b.Check(idx) { + a.b.Unset(idx) + h[structs.AllocName(a.job, a.taskGroup, idx)] = struct{}{} + } + } + + return h +} + +// NextCanaries returns the next n names for use as canaries and sets them as +// used. The existing canaries and destructive updates are also passed in. +func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) []string { + next := make([]string, 0, n) + + // First select indexes from the allocations that are undergoing destructive + // updates. This way we avoid duplicate names as they will get replaced. + dmap := bitmapFrom(destructive, uint(a.count)) + var remainder uint + for _, idx := range dmap.IndexesInRange(true, uint(0), uint(a.count)-1) { + name := structs.AllocName(a.job, a.taskGroup, uint(idx)) + if _, used := existing[name]; !used { + next = append(next, name) + a.b.Set(uint(idx)) + + // If we have enough, return + remainder := n - uint(len(next)) + if remainder == 0 { + return next + } + } + } + + // Get the set of unset names that can be used + for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { + name := structs.AllocName(a.job, a.taskGroup, uint(idx)) + if _, used := existing[name]; !used { + next = append(next, name) + a.b.Set(uint(idx)) + + // If we have enough, return + remainder = n - uint(len(next)) + if remainder == 0 { + return next + } + } + } + + // We have exhausted the prefered and free set, now just pick overlapping + // indexes + var i uint + for i = 0; i < remainder; i++ { + name := structs.AllocName(a.job, a.taskGroup, i) + if _, used := existing[name]; !used { + next = append(next, name) + a.b.Set(i) + + // If we have enough, return + remainder = n - uint(len(next)) + if remainder == 0 { + return next + } + } + } + + return next +} + +// Next returns the next n names for use as new placements and sets them as +// used. +func (a *allocNameIndex) Next(n uint) []string { + next := make([]string, 0, n) + + // Get the set of unset names that can be used + var remainder uint + for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { + next = append(next, structs.AllocName(a.job, a.taskGroup, uint(idx))) + a.b.Set(uint(idx)) + + // If we have enough, return + remainder := n - uint(len(next)) + if remainder == 0 { + return next + } + } + + // We have exhausted the free set, now just pick overlapping indexes + var i uint + for i = 0; i < remainder; i++ { + next = append(next, structs.AllocName(a.job, a.taskGroup, i)) + a.b.Set(i) + } + + return next +} diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index c30168a0bacd..e039986a578d 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -73,6 +73,22 @@ OUTER: return diff } +// union returns a new allocSet that has the union of the two allocSets. +// Conflicts prefer the last passed allocSet containing the value +func (a allocSet) union(others ...allocSet) allocSet { + union := make(map[string]*structs.Allocation, len(a)) + order := []allocSet{a} + order = append(order, others...) + + for _, set := range order { + for k, v := range set { + union[k] = v + } + } + + return union +} + // fitlerByTainted takes a set of tainted nodes and filters the allocation set // into three groups: // 1. Those that exist on untainted nodes From d72b270a51b711e053114ffe37929bd9a3e6a46d Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 31 May 2017 16:55:40 -0700 Subject: [PATCH 015/105] Pull out in-place updating into a passed in function; reduce inputs to reconciler --- scheduler/generic_sched.go | 4 +- scheduler/reconcile.go | 148 +++++++++++-------------------------- scheduler/util.go | 75 +++++++++++++++++++ 3 files changed, 123 insertions(+), 104 deletions(-) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 786012e05093..b0c1ddcc4c51 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -383,7 +383,9 @@ func (s *GenericScheduler) computeJobAllocs() error { // Filter out the allocations in a terminal state allocs, _ = s.filterCompleteAllocs(allocs) - reconciler := NewAllocReconciler(s.ctx, s.stack, s.batch, s.eval, s.job, s.deployment, allocs, tainted) + reconciler := NewAllocReconciler(s.ctx.Logger(), + newAllocUpdateFn(s.ctx, s.stack, s.eval.ID), + s.batch, s.eval.JobID, s.job, s.deployment, allocs, tainted) results := reconciler.Compute() if s.eval.AnnotatePlan { diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 7d1be535771e..f6983689d829 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -1,32 +1,42 @@ package scheduler import ( - memdb "github.com/hashicorp/go-memdb" + "log" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" ) +// allocUpdateType takes an existing allocation and a new job definition and +// returns whether the allocation can ignore the change, requires a destructive +// update, or can be inplace updated. If it can be inplace updated, an updated +// allocation that has the new resources and alloc metrics attached will be +// returned. +type allocUpdateType func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) + // allocReconciler is used to determine the set of allocations that require // placement, inplace updating or stopping given the job specification and // existing cluster state. The reconciler should only be used for batch and // service jobs. type allocReconciler struct { - // ctx gives access to the state store and logger - ctx Context + // logger is used to log debug information. Logging should be kept at a + // minimal here + logger *log.Logger - // stack allows checking for the ability to do an in-place update - stack Stack + // canInplace is used to check if the allocation can be inplace upgraded + allocUpdateFn allocUpdateType // batch marks whether the job is a batch job batch bool - // eval is the evaluation triggering the scheduling event - eval *structs.Evaluation - // job is the job being operated on, it may be nil if the job is being // stopped via a purge job *structs.Job + // jobID is the ID of the job being operated on. The job may be nil if it is + // being stopped so we require this seperately. + jobID string + // deployment is the current deployment for the job deployment *structs.Deployment @@ -87,15 +97,15 @@ type allocStopResult struct { // NewAllocReconciler creates a new reconciler that should be used to determine // the changes required to bring the cluster state inline with the declared jobspec -func NewAllocReconciler(ctx Context, stack Stack, batch bool, - eval *structs.Evaluation, job *structs.Job, deployment *structs.Deployment, +func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch bool, + jobID string, job *structs.Job, deployment *structs.Deployment, existingAllocs []*structs.Allocation, taintedNodes map[string]*structs.Node) *allocReconciler { a := &allocReconciler{ - ctx: ctx, - stack: stack, - eval: eval, + logger: logger, + allocUpdateFn: allocUpdateFn, batch: batch, + jobID: jobID, job: job, deployment: deployment, existingAllocs: existingAllocs, @@ -150,11 +160,11 @@ func (a *allocReconciler) Compute() *reconcileResults { if a.deployment == nil && !stopped && a.job.HasUpdateStrategy() { a.deployment = structs.NewDeployment(a.job) a.result.createDeployment = a.deployment - a.ctx.Logger().Printf("ALEX: MADE DEPLOYMENT %q", a.deployment.ID) + a.logger.Printf("ALEX: MADE DEPLOYMENT %q", a.deployment.ID) } if a.deployment != nil { - a.ctx.Logger().Printf("ALEX: CURRENT DEPLOYMENT %q", a.deployment.ID) + a.logger.Printf("ALEX: CURRENT DEPLOYMENT %q", a.deployment.ID) } m := newAllocMatrix(a.job, a.existingAllocs) @@ -202,7 +212,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // If the task group is nil, then the task group has been removed so all we // need to do is stop everything if tg == nil { - a.ctx.Logger().Printf("RECONCILER -- STOPPING ALL") + a.logger.Printf("RECONCILER -- STOPPING ALL") a.markStop(untainted, "", allocNotNeeded) a.markStop(migrate, "", allocNotNeeded) a.markStop(lost, structs.AllocClientStatusLost, allocLost) @@ -227,8 +237,8 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // Track the lost and migrating desiredChanges.Migrate += uint64(len(migrate) + len(lost)) - a.ctx.Logger().Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) - a.ctx.Logger().Printf("RECONCILER -- untainted %#v", untainted) + a.logger.Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) + a.logger.Printf("RECONCILER -- untainted %#v", untainted) // Mark all lost allocations for stop. Previous allocation doesn't matter // here since it is on a lost node @@ -250,8 +260,8 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.markStop(older, "", allocNotNeeded) desiredChanges.Stop += uint64(len(older)) - a.ctx.Logger().Printf("RECONCILER -- older canaries %#v", older) - a.ctx.Logger().Printf("RECONCILER -- current canaries %#v", current) + a.logger.Printf("RECONCILER -- older canaries %#v", older) + a.logger.Printf("RECONCILER -- current canaries %#v", current) untainted = untainted.difference(older) canaries = current @@ -263,11 +273,11 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { untainted = untainted.difference(canaries) canaries = nil } - a.ctx.Logger().Printf("RECONCILER -- untainted - remove canaries %#v", untainted) + a.logger.Printf("RECONCILER -- untainted - remove canaries %#v", untainted) } // Create a structure for choosing names - nameIndex := newAllocNameIndex(a.eval.JobID, group, tg.Count, untainted) + nameIndex := newAllocNameIndex(a.jobID, group, tg.Count, untainted) // Stop any unneeded allocations and update the untainted set to not // included stopped allocations. We ignore canaries since that can push us @@ -293,8 +303,8 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { desiredChanges.InPlaceUpdate += uint64(len(inplace)) desiredChanges.DestructiveUpdate += uint64(len(destructive)) - a.ctx.Logger().Printf("RECONCILER -- Stopping (%d)", len(stop)) - a.ctx.Logger().Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) + a.logger.Printf("RECONCILER -- Stopping (%d)", len(stop)) + a.logger.Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) // Get the update strategy of the group strategy := tg.Update @@ -312,7 +322,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { dstate.DesiredTotal += strategy.Canary } - a.ctx.Logger().Printf("RECONCILER -- Canary (%d)", number) + a.logger.Printf("RECONCILER -- Canary (%d)", number) for _, name := range nameIndex.NextCanaries(uint(number), canaries, destructive) { a.result.place = append(a.result.place, allocPlaceResult{ name: name, @@ -325,7 +335,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // Determine how many we can place haveCanaries := dstate != nil && dstate.DesiredCanaries != 0 limit := a.computeLimit(tg, untainted, destructive, haveCanaries) - a.ctx.Logger().Printf("RECONCILER -- LIMIT %v", limit) + a.logger.Printf("RECONCILER -- LIMIT %v", limit) // Place if: // * The deployment is not paused @@ -342,7 +352,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { desiredChanges.Place += uint64(len(place)) // Place all new allocations - a.ctx.Logger().Printf("RECONCILER -- Placing (%d)", len(place)) + a.logger.Printf("RECONCILER -- Placing (%d)", len(place)) for _, p := range place { a.result.place = append(a.result.place, p) } @@ -350,7 +360,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // Do all destructive updates min := helper.IntMin(len(destructive), limit) i := 0 - a.ctx.Logger().Printf("RECONCILER -- Destructive Updating (%d)", min) + a.logger.Printf("RECONCILER -- Destructive Updating (%d)", min) for _, alloc := range destructive { if i == min { break @@ -371,7 +381,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } // TODO Migrations should be done using a stagger and max_parallel. - a.ctx.Logger().Printf("RECONCILER -- Migrating (%d)", len(migrate)) + a.logger.Printf("RECONCILER -- Migrating (%d)", len(migrate)) for _, alloc := range migrate { a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, @@ -480,84 +490,16 @@ func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted all inplace = make(map[string]*structs.Allocation) destructive = make(map[string]*structs.Allocation) - ws := memdb.NewWatchSet() for _, alloc := range untainted { - if alloc.Job.JobModifyIndex == a.job.JobModifyIndex { + ignoreChange, destructiveChange, inplaceAlloc := a.allocUpdateFn(alloc, a.job, group) + if ignoreChange { ignore[alloc.ID] = alloc - continue - } - - // Check if the task drivers or config has changed, requires - // a destructive upgrade since that cannot be done in-place. - if tasksUpdated(a.job, alloc.Job, group.Name) { - destructive[alloc.ID] = alloc - continue - } - - // Terminal batch allocations are not filtered when they are completed - // successfully. We should avoid adding the allocation to the plan in - // the case that it is an in-place update to avoid both additional data - // in the plan and work for the clients. - if alloc.TerminalStatus() { - ignore[alloc.ID] = alloc - continue - } - - // Get the existing node - node, err := a.ctx.State().NodeByID(ws, alloc.NodeID) - if err != nil { - a.ctx.Logger().Printf("[ERR] sched: %#v failed to get node '%s': %v", a.eval, alloc.NodeID, err) - continue - } - if node == nil { + } else if destructiveChange { destructive[alloc.ID] = alloc - continue - } - - // Set the existing node as the base set - a.stack.SetNodes([]*structs.Node{node}) - - // Stage an eviction of the current allocation. This is done so that - // the current allocation is discounted when checking for feasability. - // Otherwise we would be trying to fit the tasks current resources and - // updated resources. After select is called we can remove the evict. - a.ctx.Plan().AppendUpdate(alloc, structs.AllocDesiredStatusStop, allocInPlace, "") - - // Attempt to match the task group - option, _ := a.stack.Select(group) - - // Pop the allocation - a.ctx.Plan().PopUpdate(alloc) - - // Skip if we could not do an in-place update - if option == nil { - destructive[alloc.ID] = alloc - continue - } - - // Restore the network offers from the existing allocation. - // We do not allow network resources (reserved/dynamic ports) - // to be updated. This is guarded in taskUpdated, so we can - // safely restore those here. - for task, resources := range option.TaskResources { - existing := alloc.TaskResources[task] - resources.Networks = existing.Networks + } else { + inplace[alloc.ID] = alloc + a.result.inplaceUpdate = append(a.result.inplaceUpdate, inplaceAlloc) } - - // Create a shallow copy - newAlloc := new(structs.Allocation) - *newAlloc = *alloc - - // Update the allocation - newAlloc.EvalID = a.eval.ID - newAlloc.Job = nil // Use the Job in the Plan - newAlloc.Resources = nil // Computed in Plan Apply - newAlloc.TaskResources = option.TaskResources - newAlloc.Metrics = a.ctx.Metrics() - - // Add this to the result and the tracking allocSet - inplace[alloc.ID] = alloc - a.result.inplaceUpdate = append(a.result.inplaceUpdate, newAlloc) } return diff --git a/scheduler/util.go b/scheduler/util.go index c84283b1c9fc..4f6bd5f92471 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -732,3 +732,78 @@ func updateNonTerminalAllocsToLost(plan *structs.Plan, tainted map[string]*struc } } } + +func newAllocUpdateFn(ctx Context, stack Stack, evalID string) allocUpdateType { + return func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) { + // Same index, so nothing to do + if existing.Job.JobModifyIndex == newJob.JobModifyIndex { + return true, false, nil + } + + // Check if the task drivers or config has changed, requires + // a destructive upgrade since that cannot be done in-place. + if tasksUpdated(newJob, existing.Job, newTG.Name) { + return false, true, nil + } + + // Terminal batch allocations are not filtered when they are completed + // successfully. We should avoid adding the allocation to the plan in + // the case that it is an in-place update to avoid both additional data + // in the plan and work for the clients. + if existing.TerminalStatus() { + return true, false, nil + } + + // Get the existing node + ws := memdb.NewWatchSet() + node, err := ctx.State().NodeByID(ws, existing.NodeID) + if err != nil { + ctx.Logger().Printf("[ERR] sched: %#v failed to get node '%s': %v", evalID, existing.NodeID, err) + return true, false, nil + } + if node == nil { + return false, true, nil + } + + // Set the existing node as the base set + stack.SetNodes([]*structs.Node{node}) + + // Stage an eviction of the current allocation. This is done so that + // the current allocation is discounted when checking for feasability. + // Otherwise we would be trying to fit the tasks current resources and + // updated resources. After select is called we can remove the evict. + ctx.Plan().AppendUpdate(existing, structs.AllocDesiredStatusStop, allocInPlace, "") + + // Attempt to match the task group + option, _ := stack.Select(newTG) + + // Pop the allocation + ctx.Plan().PopUpdate(existing) + + // Require destructive if we could not do an in-place update + if option == nil { + return false, true, nil + } + + // Restore the network offers from the existing allocation. + // We do not allow network resources (reserved/dynamic ports) + // to be updated. This is guarded in taskUpdated, so we can + // safely restore those here. + for task, resources := range option.TaskResources { + existingResources := existing.TaskResources[task] + resources.Networks = existingResources.Networks + } + + // Create a shallow copy + newAlloc := new(structs.Allocation) + *newAlloc = *existing + + // Update the allocation + newAlloc.EvalID = evalID + newAlloc.Job = nil // Use the Job in the Plan + newAlloc.Resources = nil // Computed in Plan Apply + newAlloc.TaskResources = option.TaskResources + newAlloc.Metrics = ctx.Metrics() + return false, false, newAlloc + } +} From f32a9a553933defd9edca4c826b9c08cae8cc2e6 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 1 Jun 2017 15:16:24 -0700 Subject: [PATCH 016/105] Non-Canary/Deployment Tests --- scheduler/reconcile.go | 195 ++++--- scheduler/reconcile_test.go | 1020 +++++++++++++++++++++++++++++++++++ scheduler/reconcile_util.go | 13 +- scheduler/util.go | 2 + 4 files changed, 1153 insertions(+), 77 deletions(-) create mode 100644 scheduler/reconcile_test.go diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index f6983689d829..7c27d12bc1c0 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -12,7 +12,8 @@ import ( // update, or can be inplace updated. If it can be inplace updated, an updated // allocation that has the new resources and alloc metrics attached will be // returned. -type allocUpdateType func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) +type allocUpdateType func(existing *structs.Allocation, newJob *structs.Job, + newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) // allocReconciler is used to determine the set of allocations that require // placement, inplace updating or stopping given the job specification and @@ -126,13 +127,33 @@ func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch // Compute reconciles the existing cluster state and returns the set of changes // required to converge the job spec and state func (a *allocReconciler) Compute() *reconcileResults { + // Create the allocation matrix + m := newAllocMatrix(a.job, a.existingAllocs) + + // Handle creating or stopoing deployments + a.computeDeployments() + // If we are just stopping a job we do not need to do anything more than // stopping all running allocs - stopped := a.job == nil || a.job.Stop - if stopped { - a.handleStop() + if a.job.Stopped() { + a.handleStop(m) + return a.result + } + + // Reconcile each group + for group, as := range m { + a.computeGroup(group, as) + } + + return a.result +} - // Cancel the deployment since it is not needed +// XXX Shouldn't cancel failed deployments +// computeDeployments cancels any deployment that is not needed and creates a +// deployment if it is needed +func (a *allocReconciler) computeDeployments() { + // If the job is stopped and there is a deployment cancel it + if a.job.Stopped() { if a.deployment != nil { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ DeploymentID: a.deployment.ID, @@ -141,7 +162,8 @@ func (a *allocReconciler) Compute() *reconcileResults { }) } - return a.result + // Nothing else to do + return } // Check if the deployment is referencing an older job and cancel it @@ -157,31 +179,24 @@ func (a *allocReconciler) Compute() *reconcileResults { } // Create a new deployment if necessary - if a.deployment == nil && !stopped && a.job.HasUpdateStrategy() { + if a.deployment == nil && !a.job.Stopped() && a.job.HasUpdateStrategy() { a.deployment = structs.NewDeployment(a.job) a.result.createDeployment = a.deployment a.logger.Printf("ALEX: MADE DEPLOYMENT %q", a.deployment.ID) } - - if a.deployment != nil { - a.logger.Printf("ALEX: CURRENT DEPLOYMENT %q", a.deployment.ID) - } - - m := newAllocMatrix(a.job, a.existingAllocs) - for group, as := range m { - a.computeGroup(group, as) - } - - return a.result } // handleStop marks all allocations to be stopped, handling the lost case -func (a *allocReconciler) handleStop() { - as := newAllocSet(a.existingAllocs) - untainted, migrate, lost := as.filterByTainted(a.taintedNodes) - a.markStop(untainted, "", allocNotNeeded) - a.markStop(migrate, "", allocNotNeeded) - a.markStop(lost, structs.AllocClientStatusLost, allocLost) +func (a *allocReconciler) handleStop(m allocMatrix) { + for group, as := range m { + untainted, migrate, lost := as.filterByTainted(a.taintedNodes) + a.markStop(untainted, "", allocNotNeeded) + a.markStop(migrate, "", allocNotNeeded) + a.markStop(lost, structs.AllocClientStatusLost, allocLost) + desiredChanges := new(structs.DesiredUpdates) + desiredChanges.Stop = uint64(len(as)) + a.result.desiredTGUpdates[group] = desiredChanges + } } // markStop is a helper for marking a set of allocation for stop with a @@ -197,7 +212,7 @@ func (a *allocReconciler) markStop(allocs allocSet, clientStatus, statusDescript } // computeGroup reconciles state for a particular task group. -func (a *allocReconciler) computeGroup(group string, as allocSet) { +func (a *allocReconciler) computeGroup(group string, all allocSet) { // Create the desired update object for the group desiredChanges := new(structs.DesiredUpdates) a.result.desiredTGUpdates[group] = desiredChanges @@ -207,7 +222,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { tg := a.job.LookupTaskGroup(group) // Determine what set of alloations are on tainted nodes - untainted, migrate, lost := as.filterByTainted(a.taintedNodes) + untainted, migrate, lost := all.filterByTainted(a.taintedNodes) // If the task group is nil, then the task group has been removed so all we // need to do is stop everything @@ -234,22 +249,9 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } } - // Track the lost and migrating - desiredChanges.Migrate += uint64(len(migrate) + len(lost)) - a.logger.Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) a.logger.Printf("RECONCILER -- untainted %#v", untainted) - // Mark all lost allocations for stop. Previous allocation doesn't matter - // here since it is on a lost node - for _, alloc := range lost { - a.result.stop = append(a.result.stop, allocStopResult{ - alloc: alloc, - clientStatus: structs.AllocClientStatusLost, - statusDescription: allocLost, - }) - } - // Get any existing canaries canaries := untainted.filterByCanary() @@ -276,15 +278,15 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.logger.Printf("RECONCILER -- untainted - remove canaries %#v", untainted) } - // Create a structure for choosing names - nameIndex := newAllocNameIndex(a.jobID, group, tg.Count, untainted) + // Create a structure for choosing names. Seed with the taken names which is + // the union of untainted and migrating nodes + nameIndex := newAllocNameIndex(a.jobID, group, tg.Count, untainted.union(migrate)) // Stop any unneeded allocations and update the untainted set to not // included stopped allocations. We ignore canaries since that can push us // over the desired count existingCanariesPromoted := dstate == nil || dstate.DesiredCanaries == 0 || dstate.Promoted - stop := a.computeStop(tg, nameIndex, untainted.difference(canaries), canaries, existingCanariesPromoted) - a.markStop(stop, "", allocNotNeeded) + stop := a.computeStop(tg, nameIndex, untainted, migrate, lost, canaries, existingCanariesPromoted) desiredChanges.Stop += uint64(len(stop)) untainted = untainted.difference(stop) @@ -293,7 +295,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // treated like non-canaries if existingCanariesPromoted { untainted = untainted.union(canaries) - nameIndex.Add(canaries) + nameIndex.Set(canaries) } // Do inplace upgrades where possible and capture the set of upgrades that @@ -306,12 +308,10 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { a.logger.Printf("RECONCILER -- Stopping (%d)", len(stop)) a.logger.Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) - // Get the update strategy of the group - strategy := tg.Update - // The fact that we have destructive updates and have less canaries than is // desired means we need to create canaries numDestructive := len(destructive) + strategy := tg.Update requireCanary := numDestructive != 0 && strategy != nil && len(canaries) < strategy.Canary if requireCanary && !a.deploymentPaused { number := strategy.Canary - len(canaries) @@ -341,7 +341,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { // * The deployment is not paused // * Not placing any canaries // * If there are any canaries that they have been promoted - place := a.computePlacements(tg, nameIndex, untainted) + place := a.computePlacements(tg, nameIndex, untainted, migrate) if creatingDeployment { dstate.DesiredTotal += len(place) } @@ -381,6 +381,7 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } // TODO Migrations should be done using a stagger and max_parallel. + desiredChanges.Migrate += uint64(len(migrate)) a.logger.Printf("RECONCILER -- Migrating (%d)", len(migrate)) for _, alloc := range migrate { a.result.stop = append(a.result.stop, allocStopResult{ @@ -397,8 +398,8 @@ func (a *allocReconciler) computeGroup(group string, as allocSet) { } // computeLimit returns the placement limit for a particular group. The inputs -// are the group definition, the existing/untainted allocation set and whether -// any canaries exist or are being placed. +// are the group definition, the untainted and destructive allocation set and +// whether any canaries exist or are being placed. func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, destructive allocSet, canaries bool) int { // If there is no update stategy or deployment for the group we can deploy // as many as the group has @@ -432,12 +433,12 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, dest } // computePlacement returns the set of allocations to place given the group -// definiton and the set of untainted/existing allocations for the group. +// definiton, the set of untainted and migrating allocations for the group. func (a *allocReconciler) computePlacements(group *structs.TaskGroup, - nameIndex *allocNameIndex, untainted allocSet) []allocPlaceResult { + nameIndex *allocNameIndex, untainted, migrate allocSet) []allocPlaceResult { // Hot path the nothing to do case - existing := len(untainted) + existing := len(untainted) + len(migrate) if existing >= group.Count { return nil } @@ -456,22 +457,60 @@ func (a *allocReconciler) computePlacements(group *structs.TaskGroup, // computeStop returns the set of allocations to stop given the group definiton // and the set of untainted and canary allocations for the group. func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *allocNameIndex, - untainted, canaries allocSet, promoted bool) allocSet { - // Hot path the nothing to do case - remove := len(untainted) - group.Count - if promoted { - remove += len(canaries) + untainted, migrate, lost, canaries allocSet, promoted bool) allocSet { + + // Mark all lost allocations for stop. Previous allocation doesn't matter + // here since it is on a lost node + var stop allocSet + stop = stop.union(lost) + a.markStop(lost, structs.AllocClientStatusLost, allocLost) + + if !promoted { + // Canaries are in the untainted set and should be discounted. + untainted = untainted.difference(canaries) } + + // Hot path the nothing to do case + remove := len(untainted) + len(migrate) - group.Count if remove <= 0 { return nil } + // Prefer selecting from the migrating set before stopping existing allocs + if len(migrate) != 0 { + mNames := newAllocNameIndex(a.jobID, group.Name, group.Count, migrate) + removeNames := mNames.Highest(uint(remove)) + for id, alloc := range migrate { + if _, match := removeNames[alloc.Name]; !match { + continue + } + a.logger.Printf("ALEX -- STOPPING migrating alloc %q", id) + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + statusDescription: allocNotNeeded, + }) + delete(migrate, id) + stop[id] = alloc + nameIndex.UnsetIndex(alloc.Index()) + + remove-- + if remove == 0 { + return stop + } + } + } + // nameIndex does not include the canaries + a.logger.Printf("ALEX -- ATTEMPTING STOP of %d normal allocs", remove) removeNames := nameIndex.Highest(uint(remove)) - stop := make(map[string]*structs.Allocation) - for id, a := range untainted { - if _, remove := removeNames[a.Name]; remove { - stop[id] = a + for id, alloc := range untainted { + if _, remove := removeNames[alloc.Name]; remove { + a.logger.Printf("ALEX -- STOPPING normal alloc %q", id) + stop[id] = alloc + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + statusDescription: allocNotNeeded, + }) } } @@ -560,18 +599,11 @@ func bitmapFrom(input allocSet, minSize uint) structs.Bitmap { return bitmap } -// Add adds the allocations to the name index -func (a *allocNameIndex) Add(set allocSet) { - for _, alloc := range set { - a.b.Set(alloc.Index()) - } -} - // RemoveHighest removes and returns the hightest n used names. The returned set // can be less than n if there aren't n names set in the index func (a *allocNameIndex) Highest(n uint) map[string]struct{} { h := make(map[string]struct{}, n) - for i := a.b.Size(); i > uint(0) && uint(len(h)) <= n; i-- { + for i := a.b.Size(); i > uint(0) && uint(len(h)) < n; i-- { // Use this to avoid wrapping around b/c of the unsigned int idx := i - 1 if a.b.Check(idx) { @@ -583,6 +615,25 @@ func (a *allocNameIndex) Highest(n uint) map[string]struct{} { return h } +// Set sets the indexes from the passed alloc set as used +func (a *allocNameIndex) Set(set allocSet) { + for _, alloc := range set { + a.b.Set(alloc.Index()) + } +} + +// Unset unsets all indexes of the passed alloc set as being used +func (a *allocNameIndex) Unset(as allocSet) { + for _, alloc := range as { + a.b.Unset(alloc.Index()) + } +} + +// UnsetIndex unsets the index as having its name used +func (a *allocNameIndex) UnsetIndex(idx uint) { + a.b.Unset(idx) +} + // NextCanaries returns the next n names for use as canaries and sets them as // used. The existing canaries and destructive updates are also passed in. func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) []string { @@ -647,13 +698,13 @@ func (a *allocNameIndex) Next(n uint) []string { next := make([]string, 0, n) // Get the set of unset names that can be used - var remainder uint + remainder := n for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { next = append(next, structs.AllocName(a.job, a.taskGroup, uint(idx))) a.b.Set(uint(idx)) // If we have enough, return - remainder := n - uint(len(next)) + remainder = n - uint(len(next)) if remainder == 0 { return next } diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go new file mode 100644 index 000000000000..6fabe1b8f363 --- /dev/null +++ b/scheduler/reconcile_test.go @@ -0,0 +1,1020 @@ +package scheduler + +import ( + "log" + "os" + "reflect" + "regexp" + "strconv" + "testing" + + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/kr/pretty" +) + +/* + +TODO: +Basic Tests: +√ Place when there is nothing in the cluster +√ Place remainder when there is some in the cluster +√ Scale down from n to n-m where n != m +√ Scale down from n to zero +√ Inplace upgrade test +√ Inplace upgrade and scale up test +√ Inplace upgrade and scale down test +√ Destructive upgrade +√ Destructive upgrade and scale up test +√ Destructive upgrade and scale down test +√ Handle lost nodes +√ Handle lost nodes and scale up +√ Handle lost nodes and scale down +√ Handle draining nodes +√ Handle draining nodes and scale up +√ Handle draining nodes and scale down +√ Handle task group being removed +√ Handle job being stopped both as .Stopped and nil +√ Place more that one group + +Canary Tests: +- Stopped job cancels any existing deployment +- JobIndex change cancels any existing deployment +- Creates a deployment if any group has an update strategy +- Paused deployment doesn't create any more canaries +- Paused deployment doesn't do any placements +- Paused deployment doesn't do destructive updates +- Paused deployment does do migrations +- Canary that is on a tainted node +- Stop old canaries +- Create new canaries on job change +- Create new canaries on job change while scaling up +- Create new canaries on job change while scaling down +- Fill canaries if partial placement +- Promote canaries unblocks max_parallel +- Failed deployment should not place anything + +*/ + +func testLogger() *log.Logger { + return log.New(os.Stderr, "", log.LstdFlags) +} + +func allocUpdateFnIgnore(*structs.Allocation, *structs.Job, *structs.TaskGroup) (bool, bool, *structs.Allocation) { + return true, false, nil +} + +func allocUpdateFnDestructive(*structs.Allocation, *structs.Job, *structs.TaskGroup) (bool, bool, *structs.Allocation) { + return false, true, nil +} + +func allocUpdateFnInplace(existing *structs.Allocation, _ *structs.Job, newTG *structs.TaskGroup) (bool, bool, *structs.Allocation) { + // Create a shallow copy + newAlloc := new(structs.Allocation) + *newAlloc = *existing + newAlloc.TaskResources = make(map[string]*structs.Resources) + + // Use the new task resources but keep the network from the old + for _, task := range newTG.Tasks { + r := task.Resources.Copy() + r.Networks = existing.TaskResources[task.Name].Networks + newAlloc.TaskResources[task.Name] = r + } + + return false, false, newAlloc +} + +var ( + // AllocationIndexRegex is a regular expression to find the allocation index. + allocationIndexRegex = regexp.MustCompile(".+\\[(\\d+)\\]$") +) + +// allocNameToIndex returns the index of the allocation. +func allocNameToIndex(name string) uint { + matches := allocationIndexRegex.FindStringSubmatch(name) + if len(matches) != 2 { + return 0 + } + + index, err := strconv.Atoi(matches[1]) + if err != nil { + return 0 + } + + return uint(index) +} + +func assertNamesHaveIndexes(t *testing.T, indexes []int, names []string) { + m := make(map[uint]int) + for _, i := range indexes { + m[uint(i)] += 1 + } + + for _, n := range names { + index := allocNameToIndex(n) + val, contained := m[index] + if !contained { + t.Fatalf("Unexpected index %d from name %s\nAll names: %v", index, n, names) + } + + val-- + if val < 0 { + t.Fatalf("Index %d repeated too many times\nAll names: %v", index, names) + } + m[index] = val + } + + for k, remainder := range m { + if remainder != 0 { + t.Fatalf("Index %d has %d remaining uses expected\nAll names: %v", k, remainder, names) + } + } +} + +func intRange(pairs ...int) []int { + if len(pairs)%2 != 0 { + return nil + } + + var r []int + for i := 0; i < len(pairs); i += 2 { + for j := pairs[i]; j <= pairs[i+1]; j++ { + r = append(r, j) + } + } + return r +} + +func placeResultsToNames(place []allocPlaceResult) []string { + names := make([]string, 0, len(place)) + for _, p := range place { + names = append(names, p.name) + } + return names +} + +func stopResultsToNames(stop []allocStopResult) []string { + names := make([]string, 0, len(stop)) + for _, s := range stop { + names = append(names, s.alloc.Name) + } + return names +} + +func allocsToNames(allocs []*structs.Allocation) []string { + names := make([]string, 0, len(allocs)) + for _, a := range allocs { + names = append(names, a.Name) + } + return names +} + +type resultExpectation struct { + createDeployment *structs.Deployment + deploymentUpdates int + place int + inplace int + stop int + desiredTGUpdates map[string]*structs.DesiredUpdates +} + +func assertResults(t *testing.T, r *reconcileResults, exp *resultExpectation) { + + if exp.createDeployment != nil && r.createDeployment == nil { + t.Fatalf("Expect a created deployment got none") + } else if exp.createDeployment == nil && r.createDeployment != nil { + t.Fatalf("Expect no created deployment; got %v", r.createDeployment) + } else if !reflect.DeepEqual(r.createDeployment, exp.createDeployment) { + t.Fatalf("Unexpected createdDeployment: %v", pretty.Diff(r.createDeployment, exp.createDeployment)) + } + + if l := len(r.deploymentUpdates); l != exp.deploymentUpdates { + t.Fatalf("Expect %d deployment updates; got %v", exp.deploymentUpdates, r.deploymentUpdates) + } + if l := len(r.place); l != exp.place { + t.Fatalf("Expected %d placements; got %d", exp.place, l) + } + if l := len(r.inplaceUpdate); l != exp.inplace { + t.Fatalf("Expected %d inplaceUpdate; got %d", exp.inplace, l) + } + if l := len(r.stop); l != exp.stop { + t.Fatalf("Expected %d stops; got %d", exp.stop, l) + } + if l := len(r.desiredTGUpdates); l != len(exp.desiredTGUpdates) { + t.Fatalf("Expected %d task group desired tg updates annotations; got %d", len(exp.desiredTGUpdates), l) + } + + // Check the desired updates happened + for group, desired := range exp.desiredTGUpdates { + act, ok := r.desiredTGUpdates[group] + if !ok { + t.Fatalf("Expected desired updates for group %q", group) + } + + if !reflect.DeepEqual(act, desired) { + t.Fatalf("Unexpected annotations for group %q: %v", group, pretty.Diff(act, desired)) + } + } +} + +// Tests the reconciler properly handles placements for a job that has no +// existing allocations +func TestReconciler_Place_NoExisting(t *testing.T) { + job := mock.Job() + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, nil, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 10, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles placements for a job that has some +// existing allocations +func TestReconciler_Place_Existing(t *testing.T) { + job := mock.Job() + + // Create 3 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 5; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 5, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 5, + Ignore: 5, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(5, 9), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles stopping allocations for a job that has +// scaled down +func TestReconciler_ScaleDown_Partial(t *testing.T) { + // Has desired 10 + job := mock.Job() + + // Create 20 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 20; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 0, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + Stop: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(10, 19), stopResultsToNames(r.stop)) +} + +// Tests the reconciler properly handles stopping allocations for a job that has +// scaled down to zero desired +func TestReconciler_ScaleDown_Zero(t *testing.T) { + // Set desired 0 + job := mock.Job() + job.TaskGroups[0].Count = 0 + + // Create 20 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 20; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 0, + inplace: 0, + stop: 20, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 20, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 19), stopResultsToNames(r.stop)) +} + +// Tests the reconciler properly handles inplace upgrading allocations +func TestReconciler_Inplace(t *testing.T) { + job := mock.Job() + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnInplace, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 0, + inplace: 10, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + InPlaceUpdate: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), allocsToNames(r.inplaceUpdate)) +} + +// Tests the reconciler properly handles inplace upgrading allocations while +// scaling up +func TestReconciler_Inplace_ScaleUp(t *testing.T) { + // Set desired 15 + job := mock.Job() + job.TaskGroups[0].Count = 15 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnInplace, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 5, + inplace: 10, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 5, + InPlaceUpdate: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), allocsToNames(r.inplaceUpdate)) + assertNamesHaveIndexes(t, intRange(10, 14), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles inplace upgrading allocations while +// scaling down +func TestReconciler_Inplace_ScaleDown(t *testing.T) { + // Set desired 5 + job := mock.Job() + job.TaskGroups[0].Count = 5 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnInplace, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 0, + inplace: 5, + stop: 5, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 5, + InPlaceUpdate: 5, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 4), allocsToNames(r.inplaceUpdate)) + assertNamesHaveIndexes(t, intRange(5, 9), stopResultsToNames(r.stop)) +} + +// Tests the reconciler properly handles destructive upgrading allocations +func TestReconciler_Destructive(t *testing.T) { + job := mock.Job() + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 10, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + DestructiveUpdate: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) +} + +// Tests the reconciler properly handles destructive upgrading allocations while +// scaling up +func TestReconciler_Destructive_ScaleUp(t *testing.T) { + // Set desired 15 + job := mock.Job() + job.TaskGroups[0].Count = 15 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 15, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 5, + DestructiveUpdate: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 14), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles destructive upgrading allocations while +// scaling down +func TestReconciler_Destructive_ScaleDown(t *testing.T) { + // Set desired 5 + job := mock.Job() + job.TaskGroups[0].Count = 5 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 5, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 5, + DestructiveUpdate: 5, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 4), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles lost nodes with allocations +func TestReconciler_LostNode(t *testing.T) { + job := mock.Job() + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Status = structs.NodeStatusDown + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 2, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 2, + Ignore: 8, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles lost nodes with allocations while +// scaling up +func TestReconciler_LostNode_ScaleUp(t *testing.T) { + // Set desired 15 + job := mock.Job() + job.TaskGroups[0].Count = 15 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Status = structs.NodeStatusDown + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 7, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 7, + Ignore: 8, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 1, 10, 14), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles lost nodes with allocations while +// scaling down +func TestReconciler_LostNode_ScaleDown(t *testing.T) { + // Set desired 5 + job := mock.Job() + job.TaskGroups[0].Count = 5 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Status = structs.NodeStatusDown + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 0, + inplace: 0, + stop: 5, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 5, + Ignore: 5, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1, 7, 9), stopResultsToNames(r.stop)) +} + +// Tests the reconciler properly handles draining nodes with allocations +func TestReconciler_DrainNode(t *testing.T) { + job := mock.Job() + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Drain = true + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 2, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Migrate: 2, + Ignore: 8, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles draining nodes with allocations while +// scaling up +func TestReconciler_DrainNode_ScaleUp(t *testing.T) { + // Set desired 15 + job := mock.Job() + job.TaskGroups[0].Count = 15 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Drain = true + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 7, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 5, + Migrate: 2, + Ignore: 8, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 1, 10, 14), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles draining nodes with allocations while +// scaling down +func TestReconciler_DrainNode_ScaleDown(t *testing.T) { + // Set desired 8 + job := mock.Job() + job.TaskGroups[0].Count = 8 + + // Create 10 existing allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 3) + for i := 0; i < 3; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Drain = true + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 1, + inplace: 0, + stop: 3, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Migrate: 1, + Stop: 2, + Ignore: 7, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 2), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 0), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles a task group being removed +func TestReconciler_RemovedTG(t *testing.T) { + job := mock.Job() + + // Create 10 allocations for a tg that no longer exists + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + oldName := job.TaskGroups[0].Name + newName := "different" + job.TaskGroups[0].Name = newName + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 10, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + oldName: { + Stop: 10, + }, + newName: { + Place: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 9), placeResultsToNames(r.place)) +} + +// Tests the reconciler properly handles a job in stopped states +func TestReconciler_JobStopped(t *testing.T) { + job := mock.Job() + job.Stop = true + + cases := []struct { + name string + job *structs.Job + jobID, taskGroup string + }{ + { + name: "stopped job", + job: job, + jobID: job.ID, + taskGroup: job.TaskGroups[0].Name, + }, + { + name: "nil job", + job: nil, + jobID: "foo", + taskGroup: "bar", + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create 10 allocations for a tg that no longer exists + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = c.job + alloc.JobID = c.jobID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(c.jobID, c.taskGroup, uint(i)) + alloc.TaskGroup = c.taskGroup + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, c.jobID, c.job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 0, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + c.taskGroup: { + Stop: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) + }) + } +} + +// Tests the reconciler properly handles jobs with multiple task groups +func TestReconciler_MultiTG(t *testing.T) { + job := mock.Job() + tg2 := job.TaskGroups[0].Copy() + tg2.Name = "foo" + job.TaskGroups = append(job.TaskGroups, tg2) + + // Create 2 existing allocations for the first tg + var allocs []*structs.Allocation + for i := 0; i < 2; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: 0, + place: 18, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 8, + Ignore: 2, + }, + tg2.Name: { + Place: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(2, 9, 0, 9), placeResultsToNames(r.place)) +} diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index e039986a578d..295a40e1a8a2 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -21,11 +21,14 @@ func newAllocMatrix(job *structs.Job, allocs []*structs.Allocation) allocMatrix } s[a.ID] = a } - for _, tg := range job.TaskGroups { - s, ok := m[tg.Name] - if !ok { - s = make(map[string]*structs.Allocation) - m[tg.Name] = s + + if job != nil { + for _, tg := range job.TaskGroups { + s, ok := m[tg.Name] + if !ok { + s = make(map[string]*structs.Allocation) + m[tg.Name] = s + } } } return m diff --git a/scheduler/util.go b/scheduler/util.go index 4f6bd5f92471..e2c632c00cab 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -733,6 +733,8 @@ func updateNonTerminalAllocsToLost(plan *structs.Plan, tainted map[string]*struc } } +// newAllocUpdateFn is a factory for the scheduler to create an allocUpdateType +// function for the reconciler func newAllocUpdateFn(ctx Context, stack Stack, evalID string) allocUpdateType { return func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) { // Same index, so nothing to do From 369a04b135c5680129c2167665afca590db28c6c Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 2 Jun 2017 16:11:29 -0700 Subject: [PATCH 017/105] Deployment tests --- nomad/structs/structs.go | 47 +- scheduler/reconcile.go | 173 ++-- scheduler/reconcile_test.go | 1593 +++++++++++++++++++++++++++++++++-- scheduler/reconcile_util.go | 28 +- 4 files changed, 1689 insertions(+), 152 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 66c45f65dd7b..f98abe86f1ff 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4080,26 +4080,6 @@ func (a *Allocation) RanSuccessfully() bool { return allSuccess } -// Stub returns a list stub for the allocation -func (a *Allocation) Stub() *AllocListStub { - return &AllocListStub{ - ID: a.ID, - EvalID: a.EvalID, - Name: a.Name, - NodeID: a.NodeID, - JobID: a.JobID, - TaskGroup: a.TaskGroup, - DesiredStatus: a.DesiredStatus, - DesiredDescription: a.DesiredDescription, - ClientStatus: a.ClientStatus, - ClientDescription: a.ClientDescription, - TaskStates: a.TaskStates, - CreateIndex: a.CreateIndex, - ModifyIndex: a.ModifyIndex, - CreateTime: a.CreateTime, - } -} - // ShouldMigrate returns if the allocation needs data migration func (a *Allocation) ShouldMigrate() bool { if a.DesiredStatus == AllocDesiredStatusStop || a.DesiredStatus == AllocDesiredStatusEvict { @@ -4123,6 +4103,33 @@ func (a *Allocation) ShouldMigrate() bool { return true } +// DeploymentHealthy returns if the allocation is marked as healthy as part of a +// deployment +func (a *Allocation) DeploymentHealthy() bool { + return a.DeploymentStatus != nil && + a.DeploymentStatus.Healthy != nil && *a.DeploymentStatus.Healthy +} + +// Stub returns a list stub for the allocation +func (a *Allocation) Stub() *AllocListStub { + return &AllocListStub{ + ID: a.ID, + EvalID: a.EvalID, + Name: a.Name, + NodeID: a.NodeID, + JobID: a.JobID, + TaskGroup: a.TaskGroup, + DesiredStatus: a.DesiredStatus, + DesiredDescription: a.DesiredDescription, + ClientStatus: a.ClientStatus, + ClientDescription: a.ClientDescription, + TaskStates: a.TaskStates, + CreateIndex: a.CreateIndex, + ModifyIndex: a.ModifyIndex, + CreateTime: a.CreateTime, + } +} + // AllocListStub is used to return a subset of alloc information type AllocListStub struct { ID string diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 7c27d12bc1c0..186007929c39 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -44,6 +44,9 @@ type allocReconciler struct { // deploymentPaused marks whether the deployment is paused deploymentPaused bool + // deploymentFailed marks whether the deployment is failed + deploymentFailed bool + // taintedNodes contains a map of nodes that are tainted taintedNodes map[string]*structs.Node @@ -119,6 +122,7 @@ func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch // Detect if the deployment is paused if deployment != nil { a.deploymentPaused = deployment.Status == structs.DeploymentStatusPaused + a.deploymentFailed = deployment.Status == structs.DeploymentStatusFailed } return a @@ -152,9 +156,9 @@ func (a *allocReconciler) Compute() *reconcileResults { // computeDeployments cancels any deployment that is not needed and creates a // deployment if it is needed func (a *allocReconciler) computeDeployments() { - // If the job is stopped and there is a deployment cancel it + // If the job is stopped and there is a deployment non-terminal deployment, cancel it if a.job.Stopped() { - if a.deployment != nil { + if a.deployment != nil && a.deployment.Active() { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ DeploymentID: a.deployment.ID, Status: structs.DeploymentStatusCancelled, @@ -168,7 +172,7 @@ func (a *allocReconciler) computeDeployments() { // Check if the deployment is referencing an older job and cancel it if d := a.deployment; d != nil { - if d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex { + if d.Active() && (d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex) { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ DeploymentID: a.deployment.ID, Status: structs.DeploymentStatusCancelled, @@ -178,6 +182,7 @@ func (a *allocReconciler) computeDeployments() { } } + // XXX Should probably do this as needed // Create a new deployment if necessary if a.deployment == nil && !a.job.Stopped() && a.job.HasUpdateStrategy() { a.deployment = structs.NewDeployment(a.job) @@ -221,13 +226,11 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // that the task group no longer exists tg := a.job.LookupTaskGroup(group) - // Determine what set of alloations are on tainted nodes - untainted, migrate, lost := all.filterByTainted(a.taintedNodes) - // If the task group is nil, then the task group has been removed so all we // need to do is stop everything if tg == nil { a.logger.Printf("RECONCILER -- STOPPING ALL") + untainted, migrate, lost := all.filterByTainted(a.taintedNodes) a.markStop(untainted, "", allocNotNeeded) a.markStop(migrate, "", allocNotNeeded) a.markStop(lost, structs.AllocClientStatusLost, allocLost) @@ -249,51 +252,60 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } } - a.logger.Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) - a.logger.Printf("RECONCILER -- untainted %#v", untainted) - // Get any existing canaries - canaries := untainted.filterByCanary() + canaries := all.filterByCanary() // Cancel any canary from a prior deployment if len(canaries) != 0 { if a.deployment != nil { current, older := canaries.filterByDeployment(a.deployment.ID) + + // Stop the older canaries a.markStop(older, "", allocNotNeeded) desiredChanges.Stop += uint64(len(older)) + canaries = current - a.logger.Printf("RECONCILER -- older canaries %#v", older) - a.logger.Printf("RECONCILER -- current canaries %#v", current) + // Handle canaries on migrating/lost nodes here by just stopping + // them + untainted, migrate, lost := canaries.filterByTainted(a.taintedNodes) + a.markStop(migrate, "", allocMigrating) + a.markStop(lost, structs.AllocClientStatusLost, allocLost) + canaries = untainted - untainted = untainted.difference(older) - canaries = current + // Update the all set + all = all.difference(older, migrate, lost) + a.logger.Printf("RECONCILER -- canaries %#v", canaries) } else { + // XXX this is totally wrong they may just be promoted good canaries // We don't need any of those canaries since there no longer is a // deployment a.markStop(canaries, "", allocNotNeeded) desiredChanges.Stop += uint64(len(canaries)) - untainted = untainted.difference(canaries) + all = all.difference(canaries) canaries = nil } - a.logger.Printf("RECONCILER -- untainted - remove canaries %#v", untainted) } + // Determine what set of alloations are on tainted nodes + untainted, migrate, lost := all.filterByTainted(a.taintedNodes) + a.logger.Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) + // Create a structure for choosing names. Seed with the taken names which is - // the union of untainted and migrating nodes + // the union of untainted and migrating nodes (includes canaries) nameIndex := newAllocNameIndex(a.jobID, group, tg.Count, untainted.union(migrate)) // Stop any unneeded allocations and update the untainted set to not // included stopped allocations. We ignore canaries since that can push us // over the desired count - existingCanariesPromoted := dstate == nil || dstate.DesiredCanaries == 0 || dstate.Promoted - stop := a.computeStop(tg, nameIndex, untainted, migrate, lost, canaries, existingCanariesPromoted) + canaryState := dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted + stop := a.computeStop(tg, nameIndex, untainted, migrate, lost, canaries, canaryState) desiredChanges.Stop += uint64(len(stop)) untainted = untainted.difference(stop) // Having stopped un-needed allocations, append the canaries to the existing // set of untainted because they are promoted. This will cause them to be // treated like non-canaries - if existingCanariesPromoted { + if !canaryState { untainted = untainted.union(canaries) nameIndex.Set(canaries) } @@ -303,7 +315,9 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { ignore, inplace, destructive := a.computeUpdates(tg, untainted) desiredChanges.Ignore += uint64(len(ignore)) desiredChanges.InPlaceUpdate += uint64(len(inplace)) - desiredChanges.DestructiveUpdate += uint64(len(destructive)) + if creatingDeployment { + dstate.DesiredTotal += len(destructive) + len(inplace) + } a.logger.Printf("RECONCILER -- Stopping (%d)", len(stop)) a.logger.Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) @@ -313,16 +327,15 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { numDestructive := len(destructive) strategy := tg.Update requireCanary := numDestructive != 0 && strategy != nil && len(canaries) < strategy.Canary - if requireCanary && !a.deploymentPaused { + if requireCanary && !a.deploymentPaused && !a.deploymentFailed { number := strategy.Canary - len(canaries) number = helper.IntMin(numDestructive, number) desiredChanges.Canary += uint64(number) if creatingDeployment { dstate.DesiredCanaries = strategy.Canary - dstate.DesiredTotal += strategy.Canary } - a.logger.Printf("RECONCILER -- Canary (%d)", number) + a.logger.Printf("RECONCILER -- Place Canaries (%d)", number) for _, name := range nameIndex.NextCanaries(uint(number), canaries, destructive) { a.result.place = append(a.result.place, allocPlaceResult{ name: name, @@ -333,12 +346,12 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } // Determine how many we can place - haveCanaries := dstate != nil && dstate.DesiredCanaries != 0 - limit := a.computeLimit(tg, untainted, destructive, haveCanaries) + canaryState = dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted + limit := a.computeLimit(tg, untainted, destructive, canaryState) a.logger.Printf("RECONCILER -- LIMIT %v", limit) // Place if: - // * The deployment is not paused + // * The deployment is not paused or failed // * Not placing any canaries // * If there are any canaries that they have been promoted place := a.computePlacements(tg, nameIndex, untainted, migrate) @@ -346,27 +359,23 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { dstate.DesiredTotal += len(place) } - if !a.deploymentPaused && existingCanariesPromoted { - // Update the desired changes and if we are creating a deployment update - // the state. - desiredChanges.Place += uint64(len(place)) - + if !a.deploymentPaused && !a.deploymentFailed && !canaryState { // Place all new allocations a.logger.Printf("RECONCILER -- Placing (%d)", len(place)) + desiredChanges.Place += uint64(len(place)) for _, p := range place { a.result.place = append(a.result.place, p) } + // XXX Needs to be done in order // Do all destructive updates min := helper.IntMin(len(destructive), limit) - i := 0 + limit -= min + desiredChanges.DestructiveUpdate += uint64(min) + desiredChanges.Ignore += uint64(len(destructive) - min) a.logger.Printf("RECONCILER -- Destructive Updating (%d)", min) - for _, alloc := range destructive { - if i == min { - break - } - i++ - + for _, alloc := range destructive.nameOrder()[:min] { + a.logger.Printf("RECONCILER -- Destructive Updating %q %q", alloc.ID, alloc.Name) a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, statusDescription: allocUpdating, @@ -377,44 +386,53 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { previousAlloc: alloc, }) } - limit -= min + } else { + a.logger.Printf("RECONCILER -- NON PROMOTED CASE") + desiredChanges.Ignore += uint64(len(destructive)) } // TODO Migrations should be done using a stagger and max_parallel. - desiredChanges.Migrate += uint64(len(migrate)) - a.logger.Printf("RECONCILER -- Migrating (%d)", len(migrate)) + if !a.deploymentFailed { + desiredChanges.Migrate += uint64(len(migrate)) + a.logger.Printf("RECONCILER -- Migrating (%d)", len(migrate)) + } else { + desiredChanges.Stop += uint64(len(migrate)) + } + for _, alloc := range migrate { a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, statusDescription: allocMigrating, }) - a.result.place = append(a.result.place, allocPlaceResult{ - name: alloc.Name, - canary: false, - taskGroup: tg, - previousAlloc: alloc, - }) + + // If the deployment is failed, just stop the allocation + if !a.deploymentFailed { + a.result.place = append(a.result.place, allocPlaceResult{ + name: alloc.Name, + canary: false, + taskGroup: tg, + previousAlloc: alloc, + }) + } } + } // computeLimit returns the placement limit for a particular group. The inputs // are the group definition, the untainted and destructive allocation set and -// whether any canaries exist or are being placed. -func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, destructive allocSet, canaries bool) int { +// whether we are in a canary state. +func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, destructive allocSet, canaryState bool) int { // If there is no update stategy or deployment for the group we can deploy // as many as the group has if group.Update == nil || len(destructive) == 0 { return group.Count - } else if a.deploymentPaused { - // If the deployment is paused, do not create anything else + } else if a.deploymentPaused || a.deploymentFailed { + // If the deployment is paused or failed, do not create anything else return 0 } - // Get the state of the deployment for the group - deploymentState := a.deployment.TaskGroups[group.Name] - // If we have canaries and they have not been promoted the limit is 0 - if canaries && (deploymentState == nil || !deploymentState.Promoted) { + if canaryState { return 0 } @@ -424,7 +442,7 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, dest limit := group.Update.MaxParallel partOf, _ := untainted.filterByDeployment(a.deployment.ID) for _, alloc := range partOf { - if alloc.DeploymentStatus == nil || alloc.DeploymentStatus.Healthy == nil { + if !alloc.DeploymentHealthy() { limit-- } } @@ -457,7 +475,7 @@ func (a *allocReconciler) computePlacements(group *structs.TaskGroup, // computeStop returns the set of allocations to stop given the group definiton // and the set of untainted and canary allocations for the group. func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *allocNameIndex, - untainted, migrate, lost, canaries allocSet, promoted bool) allocSet { + untainted, migrate, lost, canaries allocSet, canaryState bool) allocSet { // Mark all lost allocations for stop. Previous allocation doesn't matter // here since it is on a lost node @@ -465,15 +483,37 @@ func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *alloc stop = stop.union(lost) a.markStop(lost, structs.AllocClientStatusLost, allocLost) - if !promoted { - // Canaries are in the untainted set and should be discounted. + // If we are still deploying or creating canaries, don't stop them + if canaryState { untainted = untainted.difference(canaries) } // Hot path the nothing to do case remove := len(untainted) + len(migrate) - group.Count if remove <= 0 { - return nil + return stop + } + + // Prefer stopping any alloc that has the same name as the canaries if we + // are promoted + if !canaryState && len(canaries) != 0 { + canaryNames := canaries.nameSet() + for id, alloc := range untainted.difference(canaries) { + if _, match := canaryNames[alloc.Name]; match { + a.logger.Printf("ALEX -- STOPPING alloc with same name as canary %q %q", id, alloc.Name) + stop[id] = alloc + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + statusDescription: allocNotNeeded, + }) + delete(untainted, id) + + remove-- + if remove == 0 { + return stop + } + } + } } // Prefer selecting from the migrating set before stopping existing allocs @@ -505,7 +545,7 @@ func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *alloc removeNames := nameIndex.Highest(uint(remove)) for id, alloc := range untainted { if _, remove := removeNames[alloc.Name]; remove { - a.logger.Printf("ALEX -- STOPPING normal alloc %q", id) + a.logger.Printf("ALEX -- STOPPING normal alloc %q %q", id, alloc.Name) stop[id] = alloc a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, @@ -639,13 +679,16 @@ func (a *allocNameIndex) UnsetIndex(idx uint) { func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) []string { next := make([]string, 0, n) + // Create a name index + existingNames := existing.nameSet() + // First select indexes from the allocations that are undergoing destructive // updates. This way we avoid duplicate names as they will get replaced. dmap := bitmapFrom(destructive, uint(a.count)) var remainder uint for _, idx := range dmap.IndexesInRange(true, uint(0), uint(a.count)-1) { name := structs.AllocName(a.job, a.taskGroup, uint(idx)) - if _, used := existing[name]; !used { + if _, used := existingNames[name]; !used { next = append(next, name) a.b.Set(uint(idx)) @@ -660,7 +703,7 @@ func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) [] // Get the set of unset names that can be used for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { name := structs.AllocName(a.job, a.taskGroup, uint(idx)) - if _, used := existing[name]; !used { + if _, used := existingNames[name]; !used { next = append(next, name) a.b.Set(uint(idx)) @@ -677,7 +720,7 @@ func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) [] var i uint for i = 0; i < remainder; i++ { name := structs.AllocName(a.job, a.taskGroup, i) - if _, used := existing[name]; !used { + if _, used := existingNames[name]; !used { next = append(next, name) a.b.Set(i) diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 6fabe1b8f363..e92621b4855c 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -1,61 +1,88 @@ package scheduler import ( + "fmt" "log" "os" "reflect" "regexp" "strconv" "testing" + "time" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/kr/pretty" ) /* - -TODO: Basic Tests: -√ Place when there is nothing in the cluster -√ Place remainder when there is some in the cluster -√ Scale down from n to n-m where n != m -√ Scale down from n to zero -√ Inplace upgrade test -√ Inplace upgrade and scale up test -√ Inplace upgrade and scale down test -√ Destructive upgrade -√ Destructive upgrade and scale up test -√ Destructive upgrade and scale down test -√ Handle lost nodes -√ Handle lost nodes and scale up -√ Handle lost nodes and scale down -√ Handle draining nodes -√ Handle draining nodes and scale up -√ Handle draining nodes and scale down -√ Handle task group being removed -√ Handle job being stopped both as .Stopped and nil -√ Place more that one group - -Canary Tests: -- Stopped job cancels any existing deployment -- JobIndex change cancels any existing deployment -- Creates a deployment if any group has an update strategy -- Paused deployment doesn't create any more canaries -- Paused deployment doesn't do any placements -- Paused deployment doesn't do destructive updates -- Paused deployment does do migrations -- Canary that is on a tainted node -- Stop old canaries -- Create new canaries on job change -- Create new canaries on job change while scaling up -- Create new canaries on job change while scaling down -- Fill canaries if partial placement -- Promote canaries unblocks max_parallel -- Failed deployment should not place anything - +√ Place when there is nothing in the cluster +√ Place remainder when there is some in the cluster +√ Scale down from n to n-m where n != m +√ Scale down from n to zero +√ Inplace upgrade test +√ Inplace upgrade and scale up test +√ Inplace upgrade and scale down test +√ Destructive upgrade +√ Destructive upgrade and scale up test +√ Destructive upgrade and scale down test +√ Handle lost nodes +√ Handle lost nodes and scale up +√ Handle lost nodes and scale down +√ Handle draining nodes +√ Handle draining nodes and scale up +√ Handle draining nodes and scale down +√ Handle task group being removed +√ Handle job being stopped both as .Stopped and nil +√ Place more that one group + +Deployment Tests: +√ Stopped job cancels any active deployment +√ Stopped job doesn't cancel terminal deployment +√ JobIndex change cancels any active deployment +√ JobIndex change doens't cancels any terminal deployment +√ Destructive changes create deployment and get rolled out via max_parallelism +- Don't create a deployment if there are no changes +- Deployment created by all inplace updates +√ Paused or failed deployment doesn't create any more canaries +√ Paused or failed deployment doesn't do any placements +√ Paused or failed deployment doesn't do destructive updates +√ Paused does do migrations +√ Failed deployment doesn't do migrations +√ Canary that is on a draining node +√ Canary that is on a lost node +√ Stop old canaries +√ Create new canaries on job change +√ Create new canaries on job change while scaling up +√ Create new canaries on job change while scaling down +√ Fill canaries if partial placement +√ Promote canaries unblocks max_parallel +√ Promote canaries when canaries == count +√ Only place as many as are healthy in deployment +√ Limit calculation accounts for healthy allocs on migrating/lost nodes +√ Failed deployment should not place anything +- Run after canaries have been promoted, new allocs have been rolled out and there is no deployment */ +var ( + canaryUpdate = &structs.UpdateStrategy{ + Canary: 2, + MaxParallel: 2, + HealthCheck: structs.UpdateStrategyHealthCheck_Checks, + MinHealthyTime: 10 * time.Second, + HealthyDeadline: 10 * time.Minute, + } + + noCanaryUpdate = &structs.UpdateStrategy{ + MaxParallel: 4, + HealthCheck: structs.UpdateStrategyHealthCheck_Checks, + MinHealthyTime: 10 * time.Second, + HealthyDeadline: 10 * time.Minute, + } +) + func testLogger() *log.Logger { return log.New(os.Stderr, "", log.LstdFlags) } @@ -84,6 +111,16 @@ func allocUpdateFnInplace(existing *structs.Allocation, _ *structs.Job, newTG *s return false, false, newAlloc } +func allocUpdateFnMock(handled map[string]allocUpdateType, unhandled allocUpdateType) allocUpdateType { + return func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (bool, bool, *structs.Allocation) { + if fn, ok := handled[existing.ID]; ok { + return fn(existing, newJob, newTG) + } + + return unhandled(existing, newJob, newTG) + } +} + var ( // AllocationIndexRegex is a regular expression to find the allocation index. allocationIndexRegex = regexp.MustCompile(".+\\[(\\d+)\\]$") @@ -131,6 +168,38 @@ func assertNamesHaveIndexes(t *testing.T, indexes []int, names []string) { } } +func assertNoCanariesStopped(t *testing.T, stop []allocStopResult) { + for _, s := range stop { + if s.alloc.Canary { + t.Fatalf("Stopping canary alloc %q %q", s.alloc.ID, s.alloc.Name) + } + } +} + +func assertPlaceResultsHavePreviousAllocs(t *testing.T, numPrevious int, place []allocPlaceResult) { + names := make(map[string]struct{}, numPrevious) + + found := 0 + for _, p := range place { + if _, ok := names[p.name]; ok { + t.Fatalf("Name %q already placed", p.name) + } + names[p.name] = struct{}{} + + if p.previousAlloc == nil { + continue + } + + if act := p.previousAlloc.Name; p.name != act { + t.Fatalf("Name mismatch on previous alloc; got %q; want %q", act, p.name) + } + found++ + } + if numPrevious != found { + t.Fatalf("wanted %d; got %d placements with previous allocs", numPrevious, found) + } +} + func intRange(pairs ...int) []int { if len(pairs)%2 != 0 { return nil @@ -171,7 +240,7 @@ func allocsToNames(allocs []*structs.Allocation) []string { type resultExpectation struct { createDeployment *structs.Deployment - deploymentUpdates int + deploymentUpdates []*structs.DeploymentStatusUpdate place int inplace int stop int @@ -184,12 +253,16 @@ func assertResults(t *testing.T, r *reconcileResults, exp *resultExpectation) { t.Fatalf("Expect a created deployment got none") } else if exp.createDeployment == nil && r.createDeployment != nil { t.Fatalf("Expect no created deployment; got %v", r.createDeployment) - } else if !reflect.DeepEqual(r.createDeployment, exp.createDeployment) { - t.Fatalf("Unexpected createdDeployment: %v", pretty.Diff(r.createDeployment, exp.createDeployment)) + } else if exp.createDeployment != nil && r.createDeployment != nil { + // Clear the deployment ID + r.createDeployment.ID, exp.createDeployment.ID = "", "" + if !reflect.DeepEqual(r.createDeployment, exp.createDeployment) { + t.Fatalf("Unexpected createdDeployment: %v", pretty.Diff(r.createDeployment, exp.createDeployment)) + } } - if l := len(r.deploymentUpdates); l != exp.deploymentUpdates { - t.Fatalf("Expect %d deployment updates; got %v", exp.deploymentUpdates, r.deploymentUpdates) + if !reflect.DeepEqual(r.deploymentUpdates, exp.deploymentUpdates) { + t.Fatalf("Unexpected deploymentUpdates: %v", pretty.Diff(r.deploymentUpdates, exp.deploymentUpdates)) } if l := len(r.place); l != exp.place { t.Fatalf("Expected %d placements; got %d", exp.place, l) @@ -227,7 +300,7 @@ func TestReconciler_Place_NoExisting(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 10, inplace: 0, stop: 0, @@ -263,7 +336,7 @@ func TestReconciler_Place_Existing(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 5, inplace: 0, stop: 0, @@ -301,7 +374,7 @@ func TestReconciler_ScaleDown_Partial(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 0, inplace: 0, stop: 10, @@ -340,7 +413,7 @@ func TestReconciler_ScaleDown_Zero(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 0, inplace: 0, stop: 20, @@ -375,7 +448,7 @@ func TestReconciler_Inplace(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 0, inplace: 10, stop: 0, @@ -413,7 +486,7 @@ func TestReconciler_Inplace_ScaleUp(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 5, inplace: 10, stop: 0, @@ -453,7 +526,7 @@ func TestReconciler_Inplace_ScaleDown(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 0, inplace: 5, stop: 5, @@ -490,7 +563,7 @@ func TestReconciler_Destructive(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 10, inplace: 0, stop: 10, @@ -503,6 +576,7 @@ func TestReconciler_Destructive(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 9), placeResultsToNames(r.place)) assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) + assertPlaceResultsHavePreviousAllocs(t, 10, r.place) } // Tests the reconciler properly handles destructive upgrading allocations while @@ -529,7 +603,7 @@ func TestReconciler_Destructive_ScaleUp(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 15, inplace: 0, stop: 10, @@ -543,6 +617,7 @@ func TestReconciler_Destructive_ScaleUp(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) assertNamesHaveIndexes(t, intRange(0, 14), placeResultsToNames(r.place)) + assertPlaceResultsHavePreviousAllocs(t, 10, r.place) } // Tests the reconciler properly handles destructive upgrading allocations while @@ -569,7 +644,7 @@ func TestReconciler_Destructive_ScaleDown(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 5, inplace: 0, stop: 10, @@ -583,6 +658,7 @@ func TestReconciler_Destructive_ScaleDown(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) assertNamesHaveIndexes(t, intRange(0, 4), placeResultsToNames(r.place)) + assertPlaceResultsHavePreviousAllocs(t, 5, r.place) } // Tests the reconciler properly handles lost nodes with allocations @@ -615,13 +691,14 @@ func TestReconciler_LostNode(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 2, inplace: 0, stop: 2, desiredTGUpdates: map[string]*structs.DesiredUpdates{ job.TaskGroups[0].Name: { Place: 2, + Stop: 2, Ignore: 8, }, }, @@ -664,13 +741,14 @@ func TestReconciler_LostNode_ScaleUp(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 7, inplace: 0, stop: 2, desiredTGUpdates: map[string]*structs.DesiredUpdates{ job.TaskGroups[0].Name: { Place: 7, + Stop: 2, Ignore: 8, }, }, @@ -713,7 +791,7 @@ func TestReconciler_LostNode_ScaleDown(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 0, inplace: 0, stop: 5, @@ -758,7 +836,7 @@ func TestReconciler_DrainNode(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 2, inplace: 0, stop: 2, @@ -772,6 +850,7 @@ func TestReconciler_DrainNode(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) + assertPlaceResultsHavePreviousAllocs(t, 2, r.place) } // Tests the reconciler properly handles draining nodes with allocations while @@ -807,7 +886,7 @@ func TestReconciler_DrainNode_ScaleUp(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 7, inplace: 0, stop: 2, @@ -822,6 +901,7 @@ func TestReconciler_DrainNode_ScaleUp(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) assertNamesHaveIndexes(t, intRange(0, 1, 10, 14), placeResultsToNames(r.place)) + assertPlaceResultsHavePreviousAllocs(t, 2, r.place) } // Tests the reconciler properly handles draining nodes with allocations while @@ -857,7 +937,7 @@ func TestReconciler_DrainNode_ScaleDown(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 1, inplace: 0, stop: 3, @@ -872,6 +952,7 @@ func TestReconciler_DrainNode_ScaleDown(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 2), stopResultsToNames(r.stop)) assertNamesHaveIndexes(t, intRange(0, 0), placeResultsToNames(r.place)) + assertPlaceResultsHavePreviousAllocs(t, 1, r.place) } // Tests the reconciler properly handles a task group being removed @@ -899,7 +980,7 @@ func TestReconciler_RemovedTG(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 10, inplace: 0, stop: 10, @@ -943,7 +1024,7 @@ func TestReconciler_JobStopped(t *testing.T) { for _, c := range cases { t.Run(c.name, func(t *testing.T) { - // Create 10 allocations for a tg that no longer exists + // Create 10 allocations var allocs []*structs.Allocation for i := 0; i < 10; i++ { alloc := mock.Alloc() @@ -961,7 +1042,7 @@ func TestReconciler_JobStopped(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 0, inplace: 0, stop: 10, @@ -1001,7 +1082,7 @@ func TestReconciler_MultiTG(t *testing.T) { // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: 0, + deploymentUpdates: nil, place: 18, inplace: 0, stop: 0, @@ -1018,3 +1099,1387 @@ func TestReconciler_MultiTG(t *testing.T) { assertNamesHaveIndexes(t, intRange(2, 9, 0, 9), placeResultsToNames(r.place)) } + +// Tests the reconciler cancels an old deployment when the job is being stopped +func TestReconciler_CancelDeployment_JobStop(t *testing.T) { + job := mock.Job() + job.Stop = true + + running := structs.NewDeployment(job) + failed := structs.NewDeployment(job) + failed.Status = structs.DeploymentStatusFailed + + cases := []struct { + name string + job *structs.Job + jobID, taskGroup string + deployment *structs.Deployment + cancel bool + }{ + { + name: "stopped job, running deployment", + job: job, + jobID: job.ID, + taskGroup: job.TaskGroups[0].Name, + deployment: running, + cancel: true, + }, + { + name: "nil job, running deployment", + job: nil, + jobID: "foo", + taskGroup: "bar", + deployment: running, + cancel: true, + }, + { + name: "stopped job, failed deployment", + job: job, + jobID: job.ID, + taskGroup: job.TaskGroups[0].Name, + deployment: failed, + cancel: false, + }, + { + name: "nil job, failed deployment", + job: nil, + jobID: "foo", + taskGroup: "bar", + deployment: failed, + cancel: false, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create 10 allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = c.job + alloc.JobID = c.jobID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(c.jobID, c.taskGroup, uint(i)) + alloc.TaskGroup = c.taskGroup + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, c.jobID, c.job, c.deployment, allocs, nil) + r := reconciler.Compute() + + var updates []*structs.DeploymentStatusUpdate + if c.cancel { + updates = []*structs.DeploymentStatusUpdate{ + { + DeploymentID: c.deployment.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionStoppedJob, + }, + } + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: updates, + place: 0, + inplace: 0, + stop: 10, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + c.taskGroup: { + Stop: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 9), stopResultsToNames(r.stop)) + }) + } +} + +// Tests the reconciler cancels an old deployment when the job is updated +func TestReconciler_CancelDeployment_JobUpdate(t *testing.T) { + // Create a base job + job := mock.Job() + + // Create two deployments + running := structs.NewDeployment(job) + failed := structs.NewDeployment(job) + failed.Status = structs.DeploymentStatusFailed + + // Make the job newer than the deployment + job.JobModifyIndex += 10 + + cases := []struct { + name string + deployment *structs.Deployment + cancel bool + }{ + { + name: "running deployment", + deployment: running, + cancel: true, + }, + { + name: "failed deployment", + deployment: failed, + cancel: false, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create 10 allocations + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, c.deployment, allocs, nil) + r := reconciler.Compute() + + var updates []*structs.DeploymentStatusUpdate + if c.cancel { + updates = []*structs.DeploymentStatusUpdate{ + { + DeploymentID: c.deployment.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionNewerJob, + }, + } + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: updates, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + }, + }, + }) + }) + } +} + +// Tests the reconciler creates a deployment and does a rolling upgrade +func TestReconciler_CreateDeployment_RollingUpgrade(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredTotal: 10, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: d, + deploymentUpdates: nil, + place: 4, + inplace: 0, + stop: 4, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + DestructiveUpdate: 4, + Ignore: 6, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 3), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(0, 3), stopResultsToNames(r.stop)) +} + +// Tests the reconciler doesn't place any more canaries when the deployment is +// paused or failed +func TestReconciler_PausedOrFailedDeployment_NoMoreCanaries(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + cases := []struct { + name string + deploymentStatus string + }{ + { + name: "paused deployment", + deploymentStatus: structs.DeploymentStatusPaused, + }, + { + name: "failed deployment", + deploymentStatus: structs.DeploymentStatusFailed, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create a deployment that is paused and has placed some canaries + d := structs.NewDeployment(job) + d.Status = c.deploymentStatus + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredCanaries: 2, + DesiredTotal: 10, + PlacedAllocs: 1, + } + + // Create 10 allocations for the original job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, 0) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + allocs = append(allocs, canary) + + mockUpdateFn := allocUpdateFnMock(map[string]allocUpdateType{canary.ID: allocUpdateFnIgnore}, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 11, + }, + }, + }) + }) + } +} + +// Tests the reconciler doesn't place any more allocs when the deployment is +// paused or failed +func TestReconciler_PausedOrFailedDeployment_NoMorePlacements(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + job.TaskGroups[0].Count = 15 + + cases := []struct { + name string + deploymentStatus string + }{ + { + name: "paused deployment", + deploymentStatus: structs.DeploymentStatusPaused, + }, + { + name: "failed deployment", + deploymentStatus: structs.DeploymentStatusFailed, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create a deployment that is paused and has placed some canaries + d := structs.NewDeployment(job) + d.Status = c.deploymentStatus + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 15, + PlacedAllocs: 10, + } + + // Create 10 allocations for the new job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + }, + }, + }) + }) + } +} + +// Tests the reconciler doesn't do any more destructive updates when the +// deployment is paused or failed +func TestReconciler_PausedOrFailedDeployment_NoMoreDestructiveUpdates(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + cases := []struct { + name string + deploymentStatus string + }{ + { + name: "paused deployment", + deploymentStatus: structs.DeploymentStatusPaused, + }, + { + name: "failed deployment", + deploymentStatus: structs.DeploymentStatusFailed, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create a deployment that is paused and has placed some canaries + d := structs.NewDeployment(job) + d.Status = c.deploymentStatus + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + PlacedAllocs: 1, + } + + // Create 9 allocations for the original job + var allocs []*structs.Allocation + for i := 1; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create one for the new job + newAlloc := mock.Alloc() + newAlloc.Job = job + newAlloc.JobID = job.ID + newAlloc.NodeID = structs.GenerateUUID() + newAlloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, 0) + newAlloc.TaskGroup = job.TaskGroups[0].Name + newAlloc.DeploymentID = d.ID + allocs = append(allocs, newAlloc) + + mockUpdateFn := allocUpdateFnMock(map[string]allocUpdateType{newAlloc.ID: allocUpdateFnIgnore}, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + }, + }, + }) + }) + } +} + +// Tests the reconciler handles migrations correctly when a deployment is paused +// or failed +func TestReconciler_PausedOrFailedDeployment_Migrations(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + cases := []struct { + name string + deploymentStatus string + place int + stop int + ignoreAnnotation uint64 + migrateAnnotation uint64 + stopAnnotation uint64 + }{ + { + name: "paused deployment", + deploymentStatus: structs.DeploymentStatusPaused, + place: 3, + stop: 3, + ignoreAnnotation: 5, + migrateAnnotation: 3, + }, + { + name: "failed deployment", + deploymentStatus: structs.DeploymentStatusFailed, + place: 0, + stop: 3, + ignoreAnnotation: 5, + migrateAnnotation: 0, + stopAnnotation: 3, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + // Create a deployment that is paused and has placed some canaries + d := structs.NewDeployment(job) + d.Status = c.deploymentStatus + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + PlacedAllocs: 8, + } + + // Create 8 allocations in the deployment + var allocs []*structs.Allocation + for i := 0; i < 8; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + alloc.DeploymentID = d.ID + allocs = append(allocs, alloc) + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 3) + for i := 0; i < 3; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Drain = true + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, d, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: c.place, + inplace: 0, + stop: c.stop, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Migrate: c.migrateAnnotation, + Ignore: c.ignoreAnnotation, + Stop: c.stopAnnotation, + }, + }, + }) + }) + } +} + +// Tests the reconciler handles migrating a canary correctly on a draining node +func TestReconciler_DrainNode_Canary(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + // Create a deployment that is paused and has placed some canaries + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 2, + } + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create two canaries for the new job + handled := make(map[string]allocUpdateType) + for i := 0; i < 2; i++ { + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + allocs = append(allocs, canary) + handled[canary.ID] = allocUpdateFnIgnore + } + + // Build a map of tainted nodes that contains the last canary + tainted := make(map[string]*structs.Node, 1) + n := mock.Node() + n.ID = allocs[11].NodeID + n.Drain = true + tainted[n.ID] = n + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 1, + inplace: 0, + stop: 1, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 1, + Ignore: 11, + }, + }, + }) + assertNamesHaveIndexes(t, intRange(1, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(1, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler handles migrating a canary correctly on a lost node +func TestReconciler_LostNode_Canary(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + // Create a deployment that is paused and has placed some canaries + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 2, + } + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create two canaries for the new job + handled := make(map[string]allocUpdateType) + for i := 0; i < 2; i++ { + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + allocs = append(allocs, canary) + handled[canary.ID] = allocUpdateFnIgnore + } + + // Build a map of tainted nodes that contains the last canary + tainted := make(map[string]*structs.Node, 1) + n := mock.Node() + n.ID = allocs[11].NodeID + n.Status = structs.NodeStatusDown + tainted[n.ID] = n + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 1, + inplace: 0, + stop: 1, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 1, + Ignore: 11, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(1, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(1, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler handles stopping canaries from older deployments +func TestReconciler_StopOldCanaries(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + // Create an old deployment that has placed some canaries + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 2, + } + + // Update the job + job.JobModifyIndex += 10 + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create canaries + for i := 0; i < 2; i++ { + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + allocs = append(allocs, canary) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + newD := structs.NewDeployment(job) + newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredCanaries: 2, + DesiredTotal: 10, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: newD, + deploymentUpdates: []*structs.DeploymentStatusUpdate{ + { + DeploymentID: d.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionNewerJob, + }, + }, + place: 2, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 2, + Stop: 2, + Ignore: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler creates new canaries when the job changes +func TestReconciler_NewCanaries(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + newD := structs.NewDeployment(job) + newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredCanaries: 2, + DesiredTotal: 10, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: newD, + deploymentUpdates: nil, + place: 2, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 2, + Ignore: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler creates new canaries when the job changes and scales up +func TestReconciler_NewCanaries_ScaleUp(t *testing.T) { + // Scale the job up to 15 + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + job.TaskGroups[0].Count = 15 + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + newD := structs.NewDeployment(job) + newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredCanaries: 2, + DesiredTotal: 15, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: newD, + deploymentUpdates: nil, + place: 2, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 2, + Ignore: 10, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) +} + +// Tests the reconciler creates new canaries when the job changes and scales +// down +func TestReconciler_NewCanaries_ScaleDown(t *testing.T) { + // Scale the job down to 5 + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + job.TaskGroups[0].Count = 5 + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + newD := structs.NewDeployment(job) + newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredCanaries: 2, + DesiredTotal: 5, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: newD, + deploymentUpdates: nil, + place: 2, + inplace: 0, + stop: 5, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 2, + Stop: 5, + Ignore: 5, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(5, 9), stopResultsToNames(r.stop)) +} + +// Tests the reconciler handles filling the names of partially placed canaries +func TestReconciler_NewCanaries_FillNames(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = &structs.UpdateStrategy{ + Canary: 4, + MaxParallel: 2, + HealthCheck: structs.UpdateStrategyHealthCheck_Checks, + MinHealthyTime: 10 * time.Second, + HealthyDeadline: 10 * time.Minute, + } + + // Create an existing deployment that has placed some canaries + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + DesiredCanaries: 4, + PlacedAllocs: 2, + } + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create canaries but pick names at the ends + for i := 0; i < 4; i += 3 { + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + allocs = append(allocs, canary) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 2, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Canary: 2, + Ignore: 12, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(1, 2), placeResultsToNames(r.place)) +} + +// Tests the reconciler handles canary promotion by unblocking max_parallel +func TestReconciler_PromoteCanaries_Unblock(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + // Create an existing deployment that has placed some canaries and mark them + // promoted + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 2, + } + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create the canaries + handled := make(map[string]allocUpdateType) + for i := 0; i < 2; i++ { + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + canary.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, canary) + handled[canary.ID] = allocUpdateFnIgnore + } + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 2, + inplace: 0, + stop: 4, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 2, + DestructiveUpdate: 2, + Ignore: 8, + }, + }, + }) + + assertNoCanariesStopped(t, r.stop) + assertNamesHaveIndexes(t, intRange(2, 3), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(0, 3), stopResultsToNames(r.stop)) +} + +// Tests the reconciler handles canary promotion when the canary count equals +// the total correctly +func TestReconciler_PromoteCanaries_CanariesEqualCount(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + job.TaskGroups[0].Count = 2 + + // Create an existing deployment that has placed some canaries and mark them + // promoted + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 2, + DesiredCanaries: 2, + PlacedAllocs: 2, + } + + // Create 2 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 2; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create the canaries + handled := make(map[string]allocUpdateType) + for i := 0; i < 2; i++ { + // Create one canary + canary := mock.Alloc() + canary.Job = job + canary.JobID = job.ID + canary.NodeID = structs.GenerateUUID() + canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + canary.TaskGroup = job.TaskGroups[0].Name + canary.Canary = true + canary.DeploymentID = d.ID + canary.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, canary) + handled[canary.ID] = allocUpdateFnIgnore + } + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 2, + Ignore: 2, + }, + }, + }) + + assertNoCanariesStopped(t, r.stop) + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) +} + +// Tests the reconciler checks the health of placed allocs to determine the +// limit +func TestReconciler_DeploymentLimit_HealthAccounting(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + cases := []struct { + healthy int + }{ + { + healthy: 0, + }, + { + healthy: 1, + }, + { + healthy: 2, + }, + { + healthy: 3, + }, + { + healthy: 4, + }, + } + + for _, c := range cases { + t.Run(fmt.Sprintf("%d healthy", c.healthy), func(t *testing.T) { + // Create an existing deployment that has placed some canaries and mark them + // promoted + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + PlacedAllocs: 4, + } + + // Create 6 allocations from the old job + var allocs []*structs.Allocation + for i := 4; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create the new allocs + handled := make(map[string]allocUpdateType) + for i := 0; i < 4; i++ { + new := mock.Alloc() + new.Job = job + new.JobID = job.ID + new.NodeID = structs.GenerateUUID() + new.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + new.TaskGroup = job.TaskGroups[0].Name + new.DeploymentID = d.ID + if i < c.healthy { + new.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + } + allocs = append(allocs, new) + handled[new.ID] = allocUpdateFnIgnore + } + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: c.healthy, + inplace: 0, + stop: c.healthy, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + DestructiveUpdate: uint64(c.healthy), + Ignore: uint64(10 - c.healthy), + }, + }, + }) + + if c.healthy != 0 { + assertNamesHaveIndexes(t, intRange(4, 3+c.healthy), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(4, 3+c.healthy), stopResultsToNames(r.stop)) + } + }) + } +} + +// Tests the reconciler handles an alloc on a tainted node during a rolling +// update +func TestReconciler_TaintedNode_RollingUpgrade(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + // Create an existing deployment that has some placed allocs + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + PlacedAllocs: 4, + } + + // Create 6 allocations from the old job + var allocs []*structs.Allocation + for i := 4; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create the healthy replacements + handled := make(map[string]allocUpdateType) + for i := 0; i < 4; i++ { + new := mock.Alloc() + new.Job = job + new.JobID = job.ID + new.NodeID = structs.GenerateUUID() + new.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + new.TaskGroup = job.TaskGroups[0].Name + new.DeploymentID = d.ID + new.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, new) + handled[new.ID] = allocUpdateFnIgnore + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[6+i].NodeID + if i == 0 { + n.Status = structs.NodeStatusDown + } else { + n.Drain = true + } + tainted[n.ID] = n + } + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 6, + inplace: 0, + stop: 6, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 1, // Place the lost + Stop: 1, // Stop the lost + Migrate: 1, // Migrate the tainted + DestructiveUpdate: 4, + Ignore: 4, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1, 4, 7), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(0, 1, 4, 7), stopResultsToNames(r.stop)) +} + +// Tests the reconciler handles a failed deployment and does no placements +func TestReconciler_FailedDeployment_NoPlacements(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + // Create an existing failed deployment that has some placed allocs + d := structs.NewDeployment(job) + d.Status = structs.DeploymentStatusFailed + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + PlacedAllocs: 4, + } + + // Create 6 allocations from the old job + var allocs []*structs.Allocation + for i := 4; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create the healthy replacements + handled := make(map[string]allocUpdateType) + for i := 0; i < 4; i++ { + new := mock.Alloc() + new.Job = job + new.JobID = job.ID + new.NodeID = structs.GenerateUUID() + new.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + new.TaskGroup = job.TaskGroups[0].Name + new.DeploymentID = d.ID + new.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, new) + handled[new.ID] = allocUpdateFnIgnore + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 2) + for i := 0; i < 2; i++ { + n := mock.Node() + n.ID = allocs[6+i].NodeID + if i == 0 { + n.Status = structs.NodeStatusDown + } else { + n.Drain = true + } + tainted[n.ID] = n + } + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Stop: 2, + Ignore: 8, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) +} + +// Tests the reconciler handles a run after a deployment is complete +// successfully. +func TestReconciler_CompleteDeployment(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + + // Create allocations from the old job + dID := structs.GenerateUUID() + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + alloc.DeploymentID = dID + if i < 2 { + alloc.Canary = true + } + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + }, + }, + }) +} diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index 295a40e1a8a2..c278725042b2 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -2,6 +2,8 @@ package scheduler import ( "fmt" + "sort" + "strings" "github.com/hashicorp/nomad/nomad/structs" ) @@ -54,10 +56,30 @@ func (a allocSet) GoString() string { } start := fmt.Sprintf("len(%d) [\n", len(a)) - for k := range a { - start += k + ",\n" + var s []string + for k, v := range a { + s = append(s, fmt.Sprintf("%q: %v", k, v.Name)) + } + return start + strings.Join(s, "\n") + "]" +} + +func (a allocSet) nameSet() map[string]struct{} { + names := make(map[string]struct{}, len(a)) + for _, alloc := range a { + names[alloc.Name] = struct{}{} + } + return names +} + +func (a allocSet) nameOrder() []*structs.Allocation { + allocs := make([]*structs.Allocation, 0, len(a)) + for _, alloc := range a { + allocs = append(allocs, alloc) } - return start + "]" + sort.Slice(allocs, func(i, j int) bool { + return allocs[i].Index() < allocs[j].Index() + }) + return allocs } // difference returns a new allocSet that has all the existing item except those From af7f93b56bfe71f350605f7cf636d80009886417 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 6 Jun 2017 14:08:46 -0700 Subject: [PATCH 018/105] Fix canary handling --- nomad/structs/structs.go | 53 +++++- scheduler/generic_sched.go | 14 +- scheduler/generic_sched_test.go | 79 +++++---- scheduler/reconcile.go | 291 +++++--------------------------- scheduler/reconcile_test.go | 96 ++++++++++- scheduler/reconcile_util.go | 201 ++++++++++++++++++++++ 6 files changed, 429 insertions(+), 305 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index f98abe86f1ff..18fb50d4b01b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3842,6 +3842,14 @@ func (d *Deployment) Active() bool { } } +func (d *Deployment) GoString() string { + base := fmt.Sprintf("Deployment ID %q for job %q has status %q:", d.ID, d.JobID, d.Status) + for group, state := range d.TaskGroups { + base += fmt.Sprintf("\nTask Group %q has state:\n%#v", group, state) + } + return base +} + // DeploymentState tracks the state of a deployment for a given task group. type DeploymentState struct { // Promoted marks whether the canaries have been promoted @@ -3864,6 +3872,20 @@ type DeploymentState struct { UnhealthyAllocs int } +func (d *DeploymentState) GoString() string { + base := fmt.Sprintf("Desired Total: %d", d.DesiredTotal) + if d.DesiredCanaries > 0 { + base += fmt.Sprintf("\nDesired Canaries: %d", d.DesiredCanaries) + base += fmt.Sprintf("\nPromoted: %v", d.Promoted) + } + if d.PlacedAllocs > 0 { + base := fmt.Sprintf("\nPlaced: %d", d.PlacedAllocs) + base += fmt.Sprintf("\nHealthy: %d", d.HealthyAllocs) + base += fmt.Sprintf("\nUnhealthy: %d", d.UnhealthyAllocs) + } + return base +} + func (d *DeploymentState) Copy() *DeploymentState { c := &DeploymentState{} *c = *d @@ -4103,13 +4125,6 @@ func (a *Allocation) ShouldMigrate() bool { return true } -// DeploymentHealthy returns if the allocation is marked as healthy as part of a -// deployment -func (a *Allocation) DeploymentHealthy() bool { - return a.DeploymentStatus != nil && - a.DeploymentStatus.Healthy != nil && *a.DeploymentStatus.Healthy -} - // Stub returns a list stub for the allocation func (a *Allocation) Stub() *AllocListStub { return &AllocListStub{ @@ -4259,6 +4274,29 @@ type AllocDeploymentStatus struct { // as part of a deployment. It can be unset if it has neither been marked // healthy or unhealthy. Healthy *bool + + // Promoted marks whether the allocation is promoted. This field is only + // used if the allocation is a canary. + Promoted bool +} + +// IsHealthy returns if the allocation is marked as healthy as part of a +// deployment +func (a *AllocDeploymentStatus) IsHealthy() bool { + if a == nil { + return false + } + + return a.Healthy != nil && *a.Healthy +} + +// IsPromoted returns if the allocation is promoted as as part of a deployment +func (a *AllocDeploymentStatus) IsPromoted() bool { + if a == nil { + return false + } + + return a.Promoted } func (a *AllocDeploymentStatus) Copy() *AllocDeploymentStatus { @@ -4267,6 +4305,7 @@ func (a *AllocDeploymentStatus) Copy() *AllocDeploymentStatus { } c := new(AllocDeploymentStatus) + *c = *a if a.Healthy != nil { c.Healthy = helper.BoolToPtr(*a.Healthy) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index b0c1ddcc4c51..50671cd792cd 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -516,14 +516,12 @@ func (s *GenericScheduler) computePlacements(place []allocPlaceResult) error { // findPreferredNode finds the preferred node for an allocation func (s *GenericScheduler) findPreferredNode(place *allocPlaceResult) (node *structs.Node, err error) { - if place.previousAlloc != nil { - if place.taskGroup.EphemeralDisk.Sticky == true { - var preferredNode *structs.Node - ws := memdb.NewWatchSet() - preferredNode, err = s.state.NodeByID(ws, place.previousAlloc.NodeID) - if preferredNode.Ready() { - node = preferredNode - } + if place.previousAlloc != nil && place.taskGroup.EphemeralDisk.Sticky == true { + var preferredNode *structs.Node + ws := memdb.NewWatchSet() + preferredNode, err = s.state.NodeByID(ws, place.previousAlloc.NodeID) + if preferredNode.Ready() { + node = preferredNode } } return diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 9ef7c05830fe..e8df6336facf 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -122,18 +122,20 @@ func TestServiceSched_JobRegister_StickyAllocs(t *testing.T) { // Ensure the plan allocated plan := h.Plans[0] - var planned []*structs.Allocation + planned := make(map[string]*structs.Allocation) for _, allocList := range plan.NodeAllocation { - planned = append(planned, allocList...) + for _, alloc := range allocList { + planned[alloc.ID] = alloc + } } if len(planned) != 10 { t.Fatalf("bad: %#v", plan) } - // Get an allocation and mark it as failed - alloc := planned[4].Copy() - alloc.ClientStatus = structs.AllocClientStatusFailed - noErr(t, h.State.UpdateAllocsFromClient(h.NextIndex(), []*structs.Allocation{alloc})) + // Update the job to force a rolling upgrade + updated := job.Copy() + updated.TaskGroups[0].Tasks[0].Resources.CPU += 10 + noErr(t, h.State.UpsertJob(h.NextIndex(), updated)) // Create a mock evaluation to handle the update eval = &structs.Evaluation{ @@ -148,18 +150,32 @@ func TestServiceSched_JobRegister_StickyAllocs(t *testing.T) { } // Ensure we have created only one new allocation + // Ensure a single plan + if len(h1.Plans) != 1 { + t.Fatalf("bad: %#v", h1.Plans) + } plan = h1.Plans[0] var newPlanned []*structs.Allocation for _, allocList := range plan.NodeAllocation { newPlanned = append(newPlanned, allocList...) } - if len(newPlanned) != 1 { + if len(newPlanned) != 10 { t.Fatalf("bad plan: %#v", plan) } - // Ensure that the new allocation was placed on the same node as the older - // one - if newPlanned[0].NodeID != alloc.NodeID || newPlanned[0].PreviousAllocation != alloc.ID { - t.Fatalf("expected: %#v, actual: %#v", alloc, newPlanned[0]) + // Ensure that the new allocations were placed on the same node as the older + // ones + for _, new := range newPlanned { + if new.PreviousAllocation == "" { + t.Fatalf("new alloc %q doesn't have a previous allocation", new.ID) + } + + old, ok := planned[new.PreviousAllocation] + if !ok { + t.Fatalf("new alloc %q previous allocation doesn't match any prior placed alloc (%q)", new.ID, new.PreviousAllocation) + } + if new.NodeID != old.NodeID { + t.Fatalf("new alloc and old alloc node doesn't match; got %q; want %q", new.NodeID, old.NodeID) + } } } @@ -1394,9 +1410,12 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { // Update the job job2 := mock.Job() job2.ID = job.ID - job2.Update = structs.UpdateStrategy{ - Stagger: 30 * time.Second, - MaxParallel: 5, + desiredUpdates := 4 + job2.TaskGroups[0].Update = &structs.UpdateStrategy{ + MaxParallel: desiredUpdates, + HealthCheck: structs.UpdateStrategyHealthCheck_Checks, + MinHealthyTime: 10 * time.Second, + HealthyDeadline: 10 * time.Minute, } // Update the task, such that it cannot be done in-place @@ -1428,8 +1447,8 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { for _, updateList := range plan.NodeUpdate { update = append(update, updateList...) } - if len(update) != job2.Update.MaxParallel { - t.Fatalf("bad: %#v", plan) + if len(update) != desiredUpdates { + t.Fatalf("bad: got %d; want %d: %#v", len(update), desiredUpdates, plan) } // Ensure the plan allocated @@ -1437,32 +1456,22 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { for _, allocList := range plan.NodeAllocation { planned = append(planned, allocList...) } - if len(planned) != job2.Update.MaxParallel { + if len(planned) != desiredUpdates { t.Fatalf("bad: %#v", plan) } h.AssertEvalStatus(t, structs.EvalStatusComplete) - // Ensure a follow up eval was created - eval = h.Evals[0] - if eval.NextEval == "" { - t.Fatalf("missing next eval") - } - - // Check for create - if len(h.CreateEvals) == 0 { - t.Fatalf("missing created eval") - } - create := h.CreateEvals[0] - if eval.NextEval != create.ID { - t.Fatalf("ID mismatch") + // Ensure a deployment was created + if plan.CreatedDeployment == nil { + t.Fatalf("bad: %#v", plan) } - if create.PreviousEval != eval.ID { - t.Fatalf("missing previous eval") + state, ok := plan.CreatedDeployment.TaskGroups[job.TaskGroups[0].Name] + if !ok { + t.Fatalf("bad: %#v", plan) } - - if create.TriggeredBy != structs.EvalTriggerRollingUpdate { - t.Fatalf("bad: %#v", create) + if state.DesiredTotal != 10 && state.DesiredCanaries != 0 { + t.Fatalf("bad: %#v", state) } } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 186007929c39..3c60b281f573 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -134,8 +134,8 @@ func (a *allocReconciler) Compute() *reconcileResults { // Create the allocation matrix m := newAllocMatrix(a.job, a.existingAllocs) - // Handle creating or stopoing deployments - a.computeDeployments() + // Handle stopping unneeded deployments + a.cancelDeployments() // If we are just stopping a job we do not need to do anything more than // stopping all running allocs @@ -152,11 +152,9 @@ func (a *allocReconciler) Compute() *reconcileResults { return a.result } -// XXX Shouldn't cancel failed deployments -// computeDeployments cancels any deployment that is not needed and creates a -// deployment if it is needed -func (a *allocReconciler) computeDeployments() { - // If the job is stopped and there is a deployment non-terminal deployment, cancel it +// cancelDeployments cancels any deployment that is not needed +func (a *allocReconciler) cancelDeployments() { + // If the job is stopped and there is a non-terminal deployment, cancel it if a.job.Stopped() { if a.deployment != nil && a.deployment.Active() { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ @@ -181,14 +179,6 @@ func (a *allocReconciler) computeDeployments() { a.deployment = nil } } - - // XXX Should probably do this as needed - // Create a new deployment if necessary - if a.deployment == nil && !a.job.Stopped() && a.job.HasUpdateStrategy() { - a.deployment = structs.NewDeployment(a.job) - a.result.createDeployment = a.deployment - a.logger.Printf("ALEX: MADE DEPLOYMENT %q", a.deployment.ID) - } } // handleStop marks all allocations to be stopped, handling the lost case @@ -229,7 +219,6 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // If the task group is nil, then the task group has been removed so all we // need to do is stop everything if tg == nil { - a.logger.Printf("RECONCILER -- STOPPING ALL") untainted, migrate, lost := all.filterByTainted(a.taintedNodes) a.markStop(untainted, "", allocNotNeeded) a.markStop(migrate, "", allocNotNeeded) @@ -239,64 +228,53 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } // Get the deployment state for the group - creatingDeployment := a.result.createDeployment != nil var dstate *structs.DeploymentState + var existingDeployment bool if a.deployment != nil { - var ok bool - dstate, ok = a.deployment.TaskGroups[group] - - // We are creating a deployment - if !ok && creatingDeployment { - dstate = &structs.DeploymentState{} - a.deployment.TaskGroups[group] = dstate - } + dstate, existingDeployment = a.deployment.TaskGroups[group] + } + if !existingDeployment { + dstate = &structs.DeploymentState{} } - // Get any existing canaries + // Handle stopping unneeded canaries and tracking placed canaries canaries := all.filterByCanary() - - // Cancel any canary from a prior deployment if len(canaries) != 0 { if a.deployment != nil { + // Stop all non-promoted canaries from older deployments current, older := canaries.filterByDeployment(a.deployment.ID) - - // Stop the older canaries - a.markStop(older, "", allocNotNeeded) - desiredChanges.Stop += uint64(len(older)) - canaries = current + nonPromotedOlder := older.filterByPromoted(false) + a.markStop(nonPromotedOlder, "", allocNotNeeded) + desiredChanges.Stop += uint64(len(nonPromotedOlder)) // Handle canaries on migrating/lost nodes here by just stopping // them - untainted, migrate, lost := canaries.filterByTainted(a.taintedNodes) + untainted, migrate, lost := current.filterByTainted(a.taintedNodes) a.markStop(migrate, "", allocMigrating) a.markStop(lost, structs.AllocClientStatusLost, allocLost) canaries = untainted // Update the all set - all = all.difference(older, migrate, lost) - a.logger.Printf("RECONCILER -- canaries %#v", canaries) + all = all.difference(nonPromotedOlder, migrate, lost) } else { - // XXX this is totally wrong they may just be promoted good canaries - // We don't need any of those canaries since there no longer is a - // deployment - a.markStop(canaries, "", allocNotNeeded) - desiredChanges.Stop += uint64(len(canaries)) - all = all.difference(canaries) + // Stop all non-promoted canaries + nonPromoted := canaries.filterByPromoted(false) + a.markStop(nonPromoted, "", allocNotNeeded) + desiredChanges.Stop += uint64(len(nonPromoted)) + all = all.difference(nonPromoted) canaries = nil } } // Determine what set of alloations are on tainted nodes untainted, migrate, lost := all.filterByTainted(a.taintedNodes) - a.logger.Printf("RECONCILER -- untainted (%d); migrate (%d); lost (%d)", len(untainted), len(migrate), len(lost)) // Create a structure for choosing names. Seed with the taken names which is // the union of untainted and migrating nodes (includes canaries) nameIndex := newAllocNameIndex(a.jobID, group, tg.Count, untainted.union(migrate)) // Stop any unneeded allocations and update the untainted set to not - // included stopped allocations. We ignore canaries since that can push us - // over the desired count + // included stopped allocations. canaryState := dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted stop := a.computeStop(tg, nameIndex, untainted, migrate, lost, canaries, canaryState) desiredChanges.Stop += uint64(len(stop)) @@ -315,13 +293,10 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { ignore, inplace, destructive := a.computeUpdates(tg, untainted) desiredChanges.Ignore += uint64(len(ignore)) desiredChanges.InPlaceUpdate += uint64(len(inplace)) - if creatingDeployment { + if !existingDeployment { dstate.DesiredTotal += len(destructive) + len(inplace) } - a.logger.Printf("RECONCILER -- Stopping (%d)", len(stop)) - a.logger.Printf("RECONCILER -- Inplace (%d); Destructive (%d)", len(inplace), len(destructive)) - // The fact that we have destructive updates and have less canaries than is // desired means we need to create canaries numDestructive := len(destructive) @@ -331,11 +306,10 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { number := strategy.Canary - len(canaries) number = helper.IntMin(numDestructive, number) desiredChanges.Canary += uint64(number) - if creatingDeployment { + if !existingDeployment { dstate.DesiredCanaries = strategy.Canary } - a.logger.Printf("RECONCILER -- Place Canaries (%d)", number) for _, name := range nameIndex.NextCanaries(uint(number), canaries, destructive) { a.result.place = append(a.result.place, allocPlaceResult{ name: name, @@ -348,34 +322,29 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // Determine how many we can place canaryState = dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted limit := a.computeLimit(tg, untainted, destructive, canaryState) - a.logger.Printf("RECONCILER -- LIMIT %v", limit) // Place if: // * The deployment is not paused or failed // * Not placing any canaries // * If there are any canaries that they have been promoted place := a.computePlacements(tg, nameIndex, untainted, migrate) - if creatingDeployment { + if !existingDeployment { dstate.DesiredTotal += len(place) } if !a.deploymentPaused && !a.deploymentFailed && !canaryState { // Place all new allocations - a.logger.Printf("RECONCILER -- Placing (%d)", len(place)) desiredChanges.Place += uint64(len(place)) for _, p := range place { a.result.place = append(a.result.place, p) } - // XXX Needs to be done in order // Do all destructive updates min := helper.IntMin(len(destructive), limit) limit -= min desiredChanges.DestructiveUpdate += uint64(min) desiredChanges.Ignore += uint64(len(destructive) - min) - a.logger.Printf("RECONCILER -- Destructive Updating (%d)", min) for _, alloc := range destructive.nameOrder()[:min] { - a.logger.Printf("RECONCILER -- Destructive Updating %q %q", alloc.ID, alloc.Name) a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, statusDescription: allocUpdating, @@ -387,14 +356,12 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { }) } } else { - a.logger.Printf("RECONCILER -- NON PROMOTED CASE") desiredChanges.Ignore += uint64(len(destructive)) } // TODO Migrations should be done using a stagger and max_parallel. if !a.deploymentFailed { desiredChanges.Migrate += uint64(len(migrate)) - a.logger.Printf("RECONCILER -- Migrating (%d)", len(migrate)) } else { desiredChanges.Stop += uint64(len(migrate)) } @@ -416,6 +383,12 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } } + // Create a new deployment if necessary + if a.deployment == nil && strategy != nil && dstate.DesiredTotal != 0 { + a.deployment = structs.NewDeployment(a.job) + a.result.createDeployment = a.deployment + a.deployment.TaskGroups[group] = dstate + } } // computeLimit returns the placement limit for a particular group. The inputs @@ -437,12 +410,16 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, dest } // If we have been promoted or there are no canaries, the limit is the - // configured MaxParallel - any outstanding non-healthy alloc for the + // configured MaxParallel minus any outstanding non-healthy alloc for the // deployment limit := group.Update.MaxParallel - partOf, _ := untainted.filterByDeployment(a.deployment.ID) + dID := "invalid" + if a.deployment != nil { + dID = a.deployment.ID + } + partOf, _ := untainted.filterByDeployment(dID) for _, alloc := range partOf { - if !alloc.DeploymentHealthy() { + if !alloc.DeploymentStatus.IsHealthy() { limit-- } } @@ -472,8 +449,9 @@ func (a *allocReconciler) computePlacements(group *structs.TaskGroup, return place } -// computeStop returns the set of allocations to stop given the group definiton -// and the set of untainted and canary allocations for the group. +// computeStop returns the set of allocations that are marked for stopping given +// the group definiton, the set of allocations in various states and whether we +// are canarying. func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *allocNameIndex, untainted, migrate, lost, canaries allocSet, canaryState bool) allocSet { @@ -500,7 +478,6 @@ func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *alloc canaryNames := canaries.nameSet() for id, alloc := range untainted.difference(canaries) { if _, match := canaryNames[alloc.Name]; match { - a.logger.Printf("ALEX -- STOPPING alloc with same name as canary %q %q", id, alloc.Name) stop[id] = alloc a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, @@ -524,7 +501,6 @@ func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *alloc if _, match := removeNames[alloc.Name]; !match { continue } - a.logger.Printf("ALEX -- STOPPING migrating alloc %q", id) a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, statusDescription: allocNotNeeded, @@ -540,12 +516,10 @@ func (a *allocReconciler) computeStop(group *structs.TaskGroup, nameIndex *alloc } } - // nameIndex does not include the canaries - a.logger.Printf("ALEX -- ATTEMPTING STOP of %d normal allocs", remove) + // Select the allocs with the highest count to remove removeNames := nameIndex.Highest(uint(remove)) for id, alloc := range untainted { if _, remove := removeNames[alloc.Name]; remove { - a.logger.Printf("ALEX -- STOPPING normal alloc %q %q", id, alloc.Name) stop[id] = alloc a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, @@ -583,182 +557,3 @@ func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted all return } - -// allocNameIndex is used to select allocation names for placement or removal -// given an existing set of placed allocations. -type allocNameIndex struct { - job, taskGroup string - count int - b structs.Bitmap -} - -// newAllocNameIndex returns an allocNameIndex for use in selecting names of -// allocations to create or stop. It takes the job and task group name, desired -// count and any existing allocations as input. -func newAllocNameIndex(job, taskGroup string, count int, in allocSet) *allocNameIndex { - return &allocNameIndex{ - count: count, - b: bitmapFrom(in, uint(count)), - job: job, - taskGroup: taskGroup, - } -} - -func bitmapFrom(input allocSet, minSize uint) structs.Bitmap { - var max uint - for _, a := range input { - if num := a.Index(); num > max { - max = num - } - } - - if l := uint(len(input)); minSize < l { - minSize = l - } - if max < minSize { - max = minSize - } - if max == 0 { - max = 8 - } - - // byteAlign the count - if remainder := max % 8; remainder != 0 { - max = max + 8 - remainder - } - - bitmap, err := structs.NewBitmap(max) - if err != nil { - panic(err) - } - - for _, a := range input { - bitmap.Set(a.Index()) - } - - return bitmap -} - -// RemoveHighest removes and returns the hightest n used names. The returned set -// can be less than n if there aren't n names set in the index -func (a *allocNameIndex) Highest(n uint) map[string]struct{} { - h := make(map[string]struct{}, n) - for i := a.b.Size(); i > uint(0) && uint(len(h)) < n; i-- { - // Use this to avoid wrapping around b/c of the unsigned int - idx := i - 1 - if a.b.Check(idx) { - a.b.Unset(idx) - h[structs.AllocName(a.job, a.taskGroup, idx)] = struct{}{} - } - } - - return h -} - -// Set sets the indexes from the passed alloc set as used -func (a *allocNameIndex) Set(set allocSet) { - for _, alloc := range set { - a.b.Set(alloc.Index()) - } -} - -// Unset unsets all indexes of the passed alloc set as being used -func (a *allocNameIndex) Unset(as allocSet) { - for _, alloc := range as { - a.b.Unset(alloc.Index()) - } -} - -// UnsetIndex unsets the index as having its name used -func (a *allocNameIndex) UnsetIndex(idx uint) { - a.b.Unset(idx) -} - -// NextCanaries returns the next n names for use as canaries and sets them as -// used. The existing canaries and destructive updates are also passed in. -func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) []string { - next := make([]string, 0, n) - - // Create a name index - existingNames := existing.nameSet() - - // First select indexes from the allocations that are undergoing destructive - // updates. This way we avoid duplicate names as they will get replaced. - dmap := bitmapFrom(destructive, uint(a.count)) - var remainder uint - for _, idx := range dmap.IndexesInRange(true, uint(0), uint(a.count)-1) { - name := structs.AllocName(a.job, a.taskGroup, uint(idx)) - if _, used := existingNames[name]; !used { - next = append(next, name) - a.b.Set(uint(idx)) - - // If we have enough, return - remainder := n - uint(len(next)) - if remainder == 0 { - return next - } - } - } - - // Get the set of unset names that can be used - for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { - name := structs.AllocName(a.job, a.taskGroup, uint(idx)) - if _, used := existingNames[name]; !used { - next = append(next, name) - a.b.Set(uint(idx)) - - // If we have enough, return - remainder = n - uint(len(next)) - if remainder == 0 { - return next - } - } - } - - // We have exhausted the prefered and free set, now just pick overlapping - // indexes - var i uint - for i = 0; i < remainder; i++ { - name := structs.AllocName(a.job, a.taskGroup, i) - if _, used := existingNames[name]; !used { - next = append(next, name) - a.b.Set(i) - - // If we have enough, return - remainder = n - uint(len(next)) - if remainder == 0 { - return next - } - } - } - - return next -} - -// Next returns the next n names for use as new placements and sets them as -// used. -func (a *allocNameIndex) Next(n uint) []string { - next := make([]string, 0, n) - - // Get the set of unset names that can be used - remainder := n - for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { - next = append(next, structs.AllocName(a.job, a.taskGroup, uint(idx))) - a.b.Set(uint(idx)) - - // If we have enough, return - remainder = n - uint(len(next)) - if remainder == 0 { - return next - } - } - - // We have exhausted the free set, now just pick overlapping indexes - var i uint - for i = 0; i < remainder; i++ { - next = append(next, structs.AllocName(a.job, a.taskGroup, i)) - a.b.Set(i) - } - - return next -} diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index e92621b4855c..f3da09991163 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -44,8 +44,8 @@ Deployment Tests: √ JobIndex change cancels any active deployment √ JobIndex change doens't cancels any terminal deployment √ Destructive changes create deployment and get rolled out via max_parallelism -- Don't create a deployment if there are no changes -- Deployment created by all inplace updates +√ Don't create a deployment if there are no changes +√ Deployment created by all inplace updates √ Paused or failed deployment doesn't create any more canaries √ Paused or failed deployment doesn't do any placements √ Paused or failed deployment doesn't do destructive updates @@ -63,7 +63,7 @@ Deployment Tests: √ Only place as many as are healthy in deployment √ Limit calculation accounts for healthy allocs on migrating/lost nodes √ Failed deployment should not place anything -- Run after canaries have been promoted, new allocs have been rolled out and there is no deployment +√ Run after canaries have been promoted, new allocs have been rolled out and there is no deployment */ var ( @@ -252,12 +252,13 @@ func assertResults(t *testing.T, r *reconcileResults, exp *resultExpectation) { if exp.createDeployment != nil && r.createDeployment == nil { t.Fatalf("Expect a created deployment got none") } else if exp.createDeployment == nil && r.createDeployment != nil { - t.Fatalf("Expect no created deployment; got %v", r.createDeployment) + t.Fatalf("Expect no created deployment; got %#v", r.createDeployment) } else if exp.createDeployment != nil && r.createDeployment != nil { // Clear the deployment ID r.createDeployment.ID, exp.createDeployment.ID = "", "" if !reflect.DeepEqual(r.createDeployment, exp.createDeployment) { - t.Fatalf("Unexpected createdDeployment: %v", pretty.Diff(r.createDeployment, exp.createDeployment)) + t.Fatalf("Unexpected createdDeployment; got\n %#v\nwant\n%#v\nDiff: %v", + r.createDeployment, exp.createDeployment, pretty.Diff(r.createDeployment, exp.createDeployment)) } } @@ -1272,8 +1273,9 @@ func TestReconciler_CancelDeployment_JobUpdate(t *testing.T) { } } -// Tests the reconciler creates a deployment and does a rolling upgrade -func TestReconciler_CreateDeployment_RollingUpgrade(t *testing.T) { +// Tests the reconciler creates a deployment and does a rolling upgrade with +// destructive changes +func TestReconciler_CreateDeployment_RollingUpgrade_Destructive(t *testing.T) { job := mock.Job() job.TaskGroups[0].Update = noCanaryUpdate @@ -1316,6 +1318,82 @@ func TestReconciler_CreateDeployment_RollingUpgrade(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 3), stopResultsToNames(r.stop)) } +// Tests the reconciler creates a deployment for inplace updates +func TestReconciler_CreateDeployment_RollingUpgrade_Inplace(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + // Create 10 allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnInplace, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredTotal: 10, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: d, + deploymentUpdates: nil, + place: 0, + inplace: 10, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + InPlaceUpdate: 10, + }, + }, + }) +} + +// Tests the reconciler doesn't creates a deployment if there are no changes +func TestReconciler_DontCreateDeployment_NoChanges(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + // Create 10 allocations from the job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + DestructiveUpdate: 0, + Ignore: 10, + }, + }, + }) +} + // Tests the reconciler doesn't place any more canaries when the deployment is // paused or failed func TestReconciler_PausedOrFailedDeployment_NoMoreCanaries(t *testing.T) { @@ -2462,6 +2540,10 @@ func TestReconciler_CompleteDeployment(t *testing.T) { alloc.DeploymentID = dID if i < 2 { alloc.Canary = true + alloc.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + Promoted: true, + } } allocs = append(allocs, alloc) } diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index c278725042b2..27e4933b8f00 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -63,6 +63,7 @@ func (a allocSet) GoString() string { return start + strings.Join(s, "\n") + "]" } +// nameSet returns the set of allocation names func (a allocSet) nameSet() map[string]struct{} { names := make(map[string]struct{}, len(a)) for _, alloc := range a { @@ -71,6 +72,7 @@ func (a allocSet) nameSet() map[string]struct{} { return names } +// nameOrder returns the set of allocation names in sorted order func (a allocSet) nameOrder() []*structs.Allocation { allocs := make([]*structs.Allocation, 0, len(a)) for _, alloc := range a { @@ -160,6 +162,23 @@ func (a allocSet) filterByCanary() allocSet { return canaries } +// filterByPromoted filters the allocset by whether the canaries are promoted or +// not +func (a allocSet) filterByPromoted(p bool) allocSet { + promoted := make(map[string]*structs.Allocation) + for _, alloc := range a { + if !alloc.Canary { + continue + } + if p && alloc.DeploymentStatus.IsPromoted() { + promoted[alloc.ID] = alloc + } else if !p && !alloc.DeploymentStatus.IsPromoted() { + promoted[alloc.ID] = alloc + } + } + return promoted +} + // filterByDeployment filters allocations into two sets, those that match the // given deployment ID and those that don't func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { @@ -174,3 +193,185 @@ func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { } return } + +// allocNameIndex is used to select allocation names for placement or removal +// given an existing set of placed allocations. +type allocNameIndex struct { + job, taskGroup string + count int + b structs.Bitmap +} + +// newAllocNameIndex returns an allocNameIndex for use in selecting names of +// allocations to create or stop. It takes the job and task group name, desired +// count and any existing allocations as input. +func newAllocNameIndex(job, taskGroup string, count int, in allocSet) *allocNameIndex { + return &allocNameIndex{ + count: count, + b: bitmapFrom(in, uint(count)), + job: job, + taskGroup: taskGroup, + } +} + +// bitmapFrom creates a bitmap from the given allocation set and a minimum size +// maybe given. The size of the bitmap is as the larger of the passed minimum +// and t the maximum alloc index of the passed input (byte alligned). +func bitmapFrom(input allocSet, minSize uint) structs.Bitmap { + var max uint + for _, a := range input { + if num := a.Index(); num > max { + max = num + } + } + + if l := uint(len(input)); minSize < l { + minSize = l + } + if max < minSize { + max = minSize + } + if max == 0 { + max = 8 + } + + // byteAlign the count + if remainder := max % 8; remainder != 0 { + max = max + 8 - remainder + } + + bitmap, err := structs.NewBitmap(max) + if err != nil { + panic(err) + } + + for _, a := range input { + bitmap.Set(a.Index()) + } + + return bitmap +} + +// RemoveHighest removes and returns the hightest n used names. The returned set +// can be less than n if there aren't n names set in the index +func (a *allocNameIndex) Highest(n uint) map[string]struct{} { + h := make(map[string]struct{}, n) + for i := a.b.Size(); i > uint(0) && uint(len(h)) < n; i-- { + // Use this to avoid wrapping around b/c of the unsigned int + idx := i - 1 + if a.b.Check(idx) { + a.b.Unset(idx) + h[structs.AllocName(a.job, a.taskGroup, idx)] = struct{}{} + } + } + + return h +} + +// Set sets the indexes from the passed alloc set as used +func (a *allocNameIndex) Set(set allocSet) { + for _, alloc := range set { + a.b.Set(alloc.Index()) + } +} + +// Unset unsets all indexes of the passed alloc set as being used +func (a *allocNameIndex) Unset(as allocSet) { + for _, alloc := range as { + a.b.Unset(alloc.Index()) + } +} + +// UnsetIndex unsets the index as having its name used +func (a *allocNameIndex) UnsetIndex(idx uint) { + a.b.Unset(idx) +} + +// NextCanaries returns the next n names for use as canaries and sets them as +// used. The existing canaries and destructive updates are also passed in. +func (a *allocNameIndex) NextCanaries(n uint, existing, destructive allocSet) []string { + next := make([]string, 0, n) + + // Create a name index + existingNames := existing.nameSet() + + // First select indexes from the allocations that are undergoing destructive + // updates. This way we avoid duplicate names as they will get replaced. + dmap := bitmapFrom(destructive, uint(a.count)) + var remainder uint + for _, idx := range dmap.IndexesInRange(true, uint(0), uint(a.count)-1) { + name := structs.AllocName(a.job, a.taskGroup, uint(idx)) + if _, used := existingNames[name]; !used { + next = append(next, name) + a.b.Set(uint(idx)) + + // If we have enough, return + remainder := n - uint(len(next)) + if remainder == 0 { + return next + } + } + } + + // Get the set of unset names that can be used + for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { + name := structs.AllocName(a.job, a.taskGroup, uint(idx)) + if _, used := existingNames[name]; !used { + next = append(next, name) + a.b.Set(uint(idx)) + + // If we have enough, return + remainder = n - uint(len(next)) + if remainder == 0 { + return next + } + } + } + + // We have exhausted the prefered and free set, now just pick overlapping + // indexes + var i uint + for i = 0; i < remainder; i++ { + name := structs.AllocName(a.job, a.taskGroup, i) + if _, used := existingNames[name]; !used { + next = append(next, name) + a.b.Set(i) + + // If we have enough, return + remainder = n - uint(len(next)) + if remainder == 0 { + return next + } + } + } + + return next +} + +// Next returns the next n names for use as new placements and sets them as +// used. +func (a *allocNameIndex) Next(n uint) []string { + next := make([]string, 0, n) + + // Get the set of unset names that can be used + remainder := n + for _, idx := range a.b.IndexesInRange(false, uint(0), uint(a.count)-1) { + next = append(next, structs.AllocName(a.job, a.taskGroup, uint(idx))) + a.b.Set(uint(idx)) + + // If we have enough, return + remainder = n - uint(len(next)) + if remainder == 0 { + return next + } + } + + // We have exhausted the free set, now just pick overlapping indexes + var i uint + for i = 0; i < remainder; i++ { + next = append(next, structs.AllocName(a.job, a.taskGroup, i)) + a.b.Set(i) + } + + return next +} From ff8c057cef09a9993ea04959807b9c0ab0f18efa Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 21 Jun 2017 13:14:03 -0700 Subject: [PATCH 019/105] SpecChanged doesn't mutate passed job --- nomad/structs/structs.go | 40 +++++++++++----------------------------- 1 file changed, 11 insertions(+), 29 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 18fb50d4b01b..d8f73b02ebd1 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1629,44 +1629,26 @@ func (j *Job) RequiredSignals() map[string]map[string][]string { } // SpecChanged determines if the functional specification has changed between -// two job versions. The new job will be mutated but returned to its original -// value before returning, thus concurrent access to the job should be blocked. +// two job versions. func (j *Job) SpecChanged(new *Job) bool { if j == nil { return new != nil } - // Capture the original mutable values so they can be restored. - oStatus := new.Status - oStatusDescription := new.StatusDescription - oStable := new.Stable - oVersion := new.Version - oCreateIndex := new.CreateIndex - oModifyIndex := new.ModifyIndex - oJobModifyIndex := new.JobModifyIndex + // Create a copy of the new job + c := new.Copy() // Update the new job so we can do a reflect - new.Status = j.Status - new.StatusDescription = j.StatusDescription - new.Stable = j.Stable - new.Version = j.Version - new.CreateIndex = j.CreateIndex - new.ModifyIndex = j.ModifyIndex - new.JobModifyIndex = j.JobModifyIndex + c.Status = j.Status + c.StatusDescription = j.StatusDescription + c.Stable = j.Stable + c.Version = j.Version + c.CreateIndex = j.CreateIndex + c.ModifyIndex = j.ModifyIndex + c.JobModifyIndex = j.JobModifyIndex // Deep equals the jobs - equal := reflect.DeepEqual(j, new) - - // Restore the new jobs values - new.Status = oStatus - new.StatusDescription = oStatusDescription - new.Stable = oStable - new.Version = oVersion - new.CreateIndex = oCreateIndex - new.ModifyIndex = oModifyIndex - new.JobModifyIndex = oJobModifyIndex - - return !equal + return !reflect.DeepEqual(j, c) } // JobListStub is used to return a subset of job information From 0d29972c8c4dbeedc9ec318680d27425b2fe74ef Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 21 Jun 2017 13:20:54 -0700 Subject: [PATCH 020/105] cleanup limit detection --- scheduler/reconcile.go | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 3c60b281f573..7e1e1aa47023 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -413,14 +413,12 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, dest // configured MaxParallel minus any outstanding non-healthy alloc for the // deployment limit := group.Update.MaxParallel - dID := "invalid" if a.deployment != nil { - dID = a.deployment.ID - } - partOf, _ := untainted.filterByDeployment(dID) - for _, alloc := range partOf { - if !alloc.DeploymentStatus.IsHealthy() { - limit-- + partOf, _ := untainted.filterByDeployment(a.deployment.ID) + for _, alloc := range partOf { + if !alloc.DeploymentStatus.IsHealthy() { + limit-- + } } } From 0ec6d743387734ffc300e9e4fb955e3a89f9bf52 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 21 Jun 2017 13:26:45 -0700 Subject: [PATCH 021/105] update description of the alloc update factory function --- scheduler/generic_sched.go | 2 +- scheduler/util.go | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 50671cd792cd..aa541b936582 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -384,7 +384,7 @@ func (s *GenericScheduler) computeJobAllocs() error { allocs, _ = s.filterCompleteAllocs(allocs) reconciler := NewAllocReconciler(s.ctx.Logger(), - newAllocUpdateFn(s.ctx, s.stack, s.eval.ID), + genericAllocUpdateFn(s.ctx, s.stack, s.eval.ID), s.batch, s.eval.JobID, s.job, s.deployment, allocs, tainted) results := reconciler.Compute() diff --git a/scheduler/util.go b/scheduler/util.go index e2c632c00cab..9a99d0d60091 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -733,9 +733,13 @@ func updateNonTerminalAllocsToLost(plan *structs.Plan, tainted map[string]*struc } } -// newAllocUpdateFn is a factory for the scheduler to create an allocUpdateType -// function for the reconciler -func newAllocUpdateFn(ctx Context, stack Stack, evalID string) allocUpdateType { +// genericAllocUpdateFn is a factory for the scheduler to create an allocUpdateType +// function to be passed into the reconciler. The factory takes objects that +// exist only in the scheduler context and returns a function that can be used +// by the reconciler to make decsions about how to update an allocation. The +// factory allows the reconciler to be unaware of how to determine the type of +// update necessary and can minimize the set of objects it is exposed to. +func genericAllocUpdateFn(ctx Context, stack Stack, evalID string) allocUpdateType { return func(existing *structs.Allocation, newJob *structs.Job, newTG *structs.TaskGroup) (ignore, destructive bool, updated *structs.Allocation) { // Same index, so nothing to do if existing.Job.JobModifyIndex == newJob.JobModifyIndex { From 53f4952c56a0e3aa0cb55caac15c4d420785d724 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 26 Jun 2017 14:23:52 -0700 Subject: [PATCH 022/105] initial impl --- api/allocations.go | 11 + api/evaluations.go | 1 + nomad/deployment_endpoint.go | 54 +++ nomad/deployment_endpoint_test.go | 136 ++++++ nomad/deploymentwatcher/batcher.go | 121 +++++ nomad/deploymentwatcher/deployment_watcher.go | 420 ++++++++++++++++++ .../deploymentwatcher/deployments_watcher.go | 250 +++++++++++ nomad/fsm.go | 71 +++ nomad/server.go | 23 +- nomad/state/schema.go | 10 + nomad/state/state_store.go | 283 +++++++++++- nomad/state/state_store_test.go | 55 +++ nomad/structs/structs.go | 118 ++++- scheduler/reconcile.go | 8 +- 14 files changed, 1547 insertions(+), 14 deletions(-) create mode 100644 nomad/deployment_endpoint.go create mode 100644 nomad/deployment_endpoint_test.go create mode 100644 nomad/deploymentwatcher/batcher.go create mode 100644 nomad/deploymentwatcher/deployment_watcher.go create mode 100644 nomad/deploymentwatcher/deployments_watcher.go diff --git a/api/allocations.go b/api/allocations.go index 540fcf5f790d..9dbc9bf6e498 100644 --- a/api/allocations.go +++ b/api/allocations.go @@ -106,6 +106,7 @@ type Allocation struct { ClientStatus string ClientDescription string TaskStates map[string]*TaskState + DeploymentStatus *AllocDeploymentStatus PreviousAllocation string CreateIndex uint64 ModifyIndex uint64 @@ -142,11 +143,21 @@ type AllocationListStub struct { ClientStatus string ClientDescription string TaskStates map[string]*TaskState + DeploymentStatus *AllocDeploymentStatus CreateIndex uint64 ModifyIndex uint64 CreateTime int64 } +// AllocDeploymentStatus captures the status of the allocation as part of the +// deployment. This can include things like if the allocation has been marked as +// heatlhy. +type AllocDeploymentStatus struct { + Healthy *bool + Promoted bool + ModifyIndex uint64 +} + // AllocIndexSort reverse sorts allocs by CreateIndex. type AllocIndexSort []*AllocationListStub diff --git a/api/evaluations.go b/api/evaluations.go index 527f844ead60..0102af233132 100644 --- a/api/evaluations.go +++ b/api/evaluations.go @@ -62,6 +62,7 @@ type Evaluation struct { JobModifyIndex uint64 NodeID string NodeModifyIndex uint64 + DeploymentID string Status string StatusDescription string Wait time.Duration diff --git a/nomad/deployment_endpoint.go b/nomad/deployment_endpoint.go new file mode 100644 index 000000000000..af7d82746950 --- /dev/null +++ b/nomad/deployment_endpoint.go @@ -0,0 +1,54 @@ +package nomad + +import ( + "time" + + metrics "github.com/armon/go-metrics" + memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/nomad/state" + "github.com/hashicorp/nomad/nomad/structs" +) + +// Deployment endpoint is used for manipulating deployments +type Deployment struct { + srv *Server +} + +// TODO http endpoint and api +// Allocations returns the list of allocations that are a part of the deployment +func (d *Deployment) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { + if done, err := d.srv.forward("Deployment.Allocations", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "allocations"}, time.Now()) + + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + run: func(ws memdb.WatchSet, state *state.StateStore) error { + // Capture all the allocations + allocs, err := state.AllocsByDeployment(ws, args.DeploymentID) + if err != nil { + return err + } + + stubs := make([]*structs.AllocListStub, 0, len(allocs)) + for _, alloc := range allocs { + stubs = append(stubs, alloc.Stub()) + } + reply.Allocations = stubs + + // Use the last index that affected the jobs table + index, err := state.Index("allocs") + if err != nil { + return err + } + reply.Index = index + + // Set the query response + d.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return d.srv.blockingRPC(&opts) +} diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go new file mode 100644 index 000000000000..d03c752eb293 --- /dev/null +++ b/nomad/deployment_endpoint_test.go @@ -0,0 +1,136 @@ +package nomad + +import ( + "testing" + "time" + + msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/testutil" +) + +func TestDeploymentEndpoint_Allocations(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the register request + deployment := mock.Deployment() + alloc := mock.Alloc() + alloc.DeploymentID = deployment.ID + summary := mock.JobSummary(alloc.JobID) + state := s1.fsm.State() + + if err := state.UpsertJobSummary(999, summary); err != nil { + t.Fatalf("err: %v", err) + } + if err := state.UpsertDeployment(1000, deployment, false); err != nil { + t.Fatalf("err: %v", err) + } + if err := state.UpsertAllocs(1001, []*structs.Allocation{alloc}); err != nil { + t.Fatalf("err: %v", err) + } + + // Lookup the allocations + get := &structs.DeploymentSpecificRequest{ + DeploymentID: deployment.ID, + QueryOptions: structs.QueryOptions{Region: "global"}, + } + var resp structs.AllocListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + if resp.Index != 1001 { + t.Fatalf("Bad index: %d %d", resp.Index, 1001) + } + + if len(resp.Allocations) != 1 { + t.Fatalf("bad: %#v", resp.Allocations) + } + if resp.Allocations[0].ID != alloc.ID { + t.Fatalf("bad: %#v", resp.Allocations[0]) + } +} + +func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the alloc + deployment := mock.Deployment() + alloc := mock.Alloc() + alloc.DeploymentID = deployment.ID + summary := mock.JobSummary(alloc.JobID) + + if err := state.UpsertDeployment(1, deployment, false); err != nil { + t.Fatalf("err: %v", err) + } + if err := state.UpsertJobSummary(2, summary); err != nil { + t.Fatalf("err: %v", err) + } + + // Upsert alloc triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertAllocs(3, []*structs.Allocation{alloc}); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req := &structs.DeploymentSpecificRequest{ + DeploymentID: deployment.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.AllocListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 3 { + t.Fatalf("Bad index: %d %d", resp.Index, 3) + } + if len(resp.Allocations) != 1 || resp.Allocations[0].ID != alloc.ID { + t.Fatalf("bad: %#v", resp.Allocations) + } + + // Client updates trigger watches + alloc2 := mock.Alloc() + alloc2.ID = alloc.ID + alloc2.DeploymentID = alloc.DeploymentID + alloc2.ClientStatus = structs.AllocClientStatusRunning + time.AfterFunc(100*time.Millisecond, func() { + state.UpsertJobSummary(4, mock.JobSummary(alloc2.JobID)) + if err := state.UpdateAllocsFromClient(5, []*structs.Allocation{alloc2}); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 3 + start = time.Now() + var resp2 structs.AllocListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) + } + if resp2.Index != 5 { + t.Fatalf("Bad index: %d %d", resp2.Index, 5) + } + if len(resp2.Allocations) != 1 || resp.Allocations[0].ID != alloc.ID || + resp2.Allocations[0].ClientStatus != structs.AllocClientStatusRunning { + t.Fatalf("bad: %#v", resp2.Allocations) + } +} diff --git a/nomad/deploymentwatcher/batcher.go b/nomad/deploymentwatcher/batcher.go new file mode 100644 index 000000000000..4dde9bba1294 --- /dev/null +++ b/nomad/deploymentwatcher/batcher.go @@ -0,0 +1,121 @@ +package deploymentwatcher + +import ( + "sync" + "time" + + "github.com/hashicorp/nomad/nomad/structs" +) + +// EvalBatcher is used to batch the creation of evaluations +type EvalBatcher struct { + // raft is used to actually commit the evaluations + raft DeploymentRaftEndpoints + + // future to be returned to callers + f *EvalFuture + + // inCh is used to pass evaluations to the daemon process + inCh chan *structs.Evaluation + + // exitCh is used to exit the daemon batcher + exitCh chan struct{} + + l sync.Mutex +} + +// NewEvalBatcher returns an EvalBatcher that uses the passed raft endpoints to +// create the evaluations and exits the batcher when the passed exit channel is +// closed. +func NewEvalBatcher(raft DeploymentRaftEndpoints, exitCh chan struct{}) *EvalBatcher { + b := &EvalBatcher{ + raft: raft, + exitCh: exitCh, + } + + go b.batcher() + return b +} + +// CreateEval batches the creation of the evaluation and returns a future that +// tracks the evaluations creation. +func (b *EvalBatcher) CreateEval(e *structs.Evaluation) *EvalFuture { + b.l.Lock() + defer b.l.Unlock() + + if b.f == nil { + b.f = NewEvalFuture() + } + + b.inCh <- e + return b.f +} + +// batcher is the long lived batcher goroutine +func (b *EvalBatcher) batcher() { + ticker := time.NewTicker(200 * time.Millisecond) + evals := make(map[string]*structs.Evaluation) + for { + select { + case <-b.exitCh: + ticker.Stop() + return + case e := <-b.inCh: + evals[e.DeploymentID] = e + case <-ticker.C: + if len(evals) == 0 { + continue + } + + // Capture the future + b.l.Lock() + f := b.f + b.l.Unlock() + + // Shouldn't be possible but protect ourselves + if f == nil { + continue + } + + // Capture the evals + all := make([]*structs.Evaluation, 0, len(evals)) + for _, e := range evals { + all = append(all, e) + } + + // Upsert the evals + f.Set(b.raft.UpsertEvals(all)) + + // Reset the evals list + evals = make(map[string]*structs.Evaluation) + } + } +} + +// EvalFuture is a future that can be used to retrieve the index the eval was +// created at or any error in the creation process +type EvalFuture struct { + index uint64 + err error + waitCh chan struct{} +} + +// NewEvalFuture returns a new EvalFuture +func NewEvalFuture() *EvalFuture { + return &EvalFuture{ + waitCh: make(chan struct{}), + } +} + +// Set sets the results of the future, unblocking any client. +func (f *EvalFuture) Set(index uint64, err error) { + f.index = index + f.err = err + close(f.waitCh) +} + +// Results returns the creation index and any error. +func (f *EvalFuture) Results() (uint64, error) { + <-f.waitCh + return f.index, f.err +} diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go new file mode 100644 index 000000000000..494a04e97666 --- /dev/null +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -0,0 +1,420 @@ +package deploymentwatcher + +import ( + "fmt" + "log" + "sync" + "time" + + "github.com/hashicorp/nomad/nomad/structs" +) + +const ( + // evalBatchPeriod is the batching length before creating an evaluation to + // trigger the scheduler when allocations are marked as healthy. + evalBatchPeriod = 1 * time.Second +) + +// deploymentTriggers are the set of functions required to trigger changes on +// behalf of a deployment +type deploymentTriggers interface { + // createEvaluation is used to create an evaluation. + createEvaluation(eval *structs.Evaluation) (uint64, error) + + // upsertJob is used to roll back a job when autoreverting for a deployment + upsertJob(job *structs.Job) (uint64, error) + + // upsertDeploymentStatusUpdate is used to upsert a deployment status update + // and an optional evaluation and job to upsert + upsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdate, eval *structs.Evaluation, job *structs.Job) (uint64, error) + + // upsertDeploymentPromotion is used to promote canaries in a deployment + upsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) + + // upsertDeploymentAllocHealth is used to set the health of allocations in a + // deployment + upsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) +} + +// deploymentWatcher is used to watch a single deployment and trigger the +// scheduler when allocation health transistions. +type deploymentWatcher struct { + // deploymentTriggers holds the methods required to trigger changes on behalf of the + // deployment + deploymentTriggers + + // DeploymentStateWatchers holds the methods required to watch objects for + // changes on behalf of the deployment + DeploymentStateWatchers + + // d is the deployment being watched + d *structs.Deployment + + // j is the job the deployment is for + j *structs.Job + + // autorevert is used to lookup if an task group should autorevert on + // unhealthy allocations + autorevert map[string]bool + + // outstandingBatch marks whether an outstanding function exists to create + // the evaluation. Access should be done through the lock + outstandingBatch bool + + logger *log.Logger + exitCh chan struct{} + l sync.RWMutex +} + +// newDeploymentWatcher returns a deployment watcher that is used to watch +// deployments and trigger the scheduler as needed. +func newDeploymentWatcher( + logger *log.Logger, + watchers DeploymentStateWatchers, + d *structs.Deployment, + j *structs.Job, + triggers deploymentTriggers) *deploymentWatcher { + + w := &deploymentWatcher{ + d: d, + j: j, + autorevert: make(map[string]bool, len(j.TaskGroups)), + DeploymentStateWatchers: watchers, + deploymentTriggers: triggers, + exitCh: make(chan struct{}), + logger: logger, + } + + for _, tg := range j.TaskGroups { + autorevert := false + if tg.Update != nil && tg.Update.AutoRevert { + autorevert = true + } + w.autorevert[tg.Name] = autorevert + } + + go w.watch() + return w +} + +func (w *deploymentWatcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest) ( + *structs.DeploymentUpdateResponse, error) { + + // If we are failing the deployment, update the status and potentially + // rollback + var j *structs.Job + var u *structs.DeploymentStatusUpdate + + // If there are unhealthy allocations we need to mark the deployment as + // failed and check if we should roll back to a stable job. + if l := len(req.UnhealthyAllocationIDs); l != 0 { + unhealthy := make(map[string]struct{}, l) + for _, alloc := range req.UnhealthyAllocationIDs { + unhealthy[alloc] = struct{}{} + } + + // Get the allocations for the deployment + args := &structs.DeploymentSpecificRequest{DeploymentID: req.DeploymentID} + var resp structs.AllocListResponse + if err := w.Allocations(args, &resp); err != nil { + return nil, err + } + + desc := structs.DeploymentStatusDescriptionFailedAllocations + for _, alloc := range resp.Allocations { + // Check that the alloc has been marked unhealthy + if _, ok := unhealthy[alloc.ID]; !ok { + continue + } + + // Check if the group has autorevert set + if !w.autorevert[alloc.TaskGroup] { + continue + } + + var err error + j, err = w.latestStableJob() + if err != nil { + return nil, err + } + + desc = fmt.Sprintf("%s - rolling back to job version %d", desc, j.Version) + break + } + + u = w.getDeploymentStatusUpdate(structs.DeploymentStatusFailed, desc) + } + + // Create the request + areq := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: *req, + Eval: w.getEval(), + DeploymentUpdate: u, + Job: j, + } + + index, err := w.upsertDeploymentAllocHealth(areq) + if err != nil { + return nil, err + } + + return &structs.DeploymentUpdateResponse{ + EvalID: areq.Eval.ID, + EvalCreateIndex: index, + DeploymentModifyIndex: index, + }, nil +} + +func (w *deploymentWatcher) PromoteDeployment(req *structs.DeploymentPromoteRequest) ( + *structs.DeploymentUpdateResponse, error) { + + // Create the request + areq := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: *req, + Eval: w.getEval(), + } + + index, err := w.upsertDeploymentPromotion(areq) + if err != nil { + return nil, err + } + + return &structs.DeploymentUpdateResponse{ + EvalID: areq.Eval.ID, + EvalCreateIndex: index, + DeploymentModifyIndex: index, + }, nil +} + +func (w *deploymentWatcher) PauseDeployment(req *structs.DeploymentPauseRequest) ( + *structs.DeploymentUpdateResponse, error) { + // Determine the status we should transistion to and if we need to create an + // evaluation + status, desc := structs.DeploymentStatusPaused, structs.DeploymentStatusDescriptionPaused + var eval *structs.Evaluation + evalID := "" + if !req.Pause { + status, desc = structs.DeploymentStatusRunning, structs.DeploymentStatusDescriptionRunning + eval := w.getEval() + evalID = eval.ID + } + update := w.getDeploymentStatusUpdate(status, desc) + + // Commit the change + i, err := w.upsertDeploymentStatusUpdate(update, eval, nil) + if err != nil { + return nil, err + } + + return &structs.DeploymentUpdateResponse{ + EvalID: evalID, + EvalCreateIndex: i, + DeploymentModifyIndex: i, + }, nil +} + +// StopWatch stops watching the deployment. This should be called whenever a +// deployment is completed or the watcher is no longer needed. +func (w *deploymentWatcher) StopWatch() { + w.l.Lock() + defer w.l.Unlock() + close(w.exitCh) +} + +// watch is the long running watcher that takes actions upon allocation changes +func (w *deploymentWatcher) watch() { + latestEval := uint64(0) + for { + // Block getting all allocations that are part of the deployment using + // the last evaluation index. This will have us block waiting for + // something to change past what the scheduler has evaluated. + var allocs []*structs.AllocListStub + select { + case <-w.exitCh: + return + case allocs = <-w.getAllocs(latestEval): + } + + // Get the latest evaluation snapshot index + latestEval = w.latestEvalIndex() + + // Create an evaluation trigger if there is any allocation whose + // deployment status has been updated past the latest eval index. + createEval, failDeployment, rollback := false, false, false + for _, alloc := range allocs { + if alloc.DeploymentStatus == nil { + continue + } + + if alloc.DeploymentStatus.ModifyIndex > latestEval { + createEval = true + } + + if alloc.DeploymentStatus.IsUnhealthy() { + // Check if the group has autorevert set + if w.autorevert[alloc.TaskGroup] { + rollback = true + } + + // Since we have an unhealthy allocation, fail the deployment + failDeployment = true + } + + // All conditions have been hit so we can break + if createEval && failDeployment && rollback { + break + } + } + + // Change the deployments status to failed + if failDeployment { + // Default description + desc := structs.DeploymentStatusDescriptionFailedAllocations + + // Rollback to the old job if necessary + var j *structs.Job + if rollback { + var err error + j, err = w.latestStableJob() + if err != nil { + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to lookup latest stable job for %q: %v", w.d.JobID, err) + } + + // Description should include that the job is being rolled back to + // version N + desc = fmt.Sprintf("%s - rolling back to job version %d", desc, j.Version) + } + + // Update the status of the deployment to failed and create an + // evaluation. + e, u := w.getEval(), w.getDeploymentStatusUpdate(structs.DeploymentStatusFailed, desc) + if index, err := w.upsertDeploymentStatusUpdate(u, e, j); err != nil { + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to update deployment %q status: %v", w.d.ID, err) + } else { + latestEval = index + } + } else if createEval { + // Create an eval to push the deployment along + w.createEvalBatched() + } + } +} + +// latestStableJob returns the latest stable job. It may be nil if none exist +func (w *deploymentWatcher) latestStableJob() (*structs.Job, error) { + args := &structs.JobSpecificRequest{JobID: w.d.JobID} + var resp structs.JobVersionsResponse + if err := w.GetJobVersions(args, &resp); err != nil { + return nil, err + } + + var stable *structs.Job + for _, job := range resp.Versions { + if job.Stable { + stable = job + break + } + } + + return stable, nil +} + +// createEval creates an evaluation for the job and commits it to Raft. +func (w *deploymentWatcher) createEval() (evalID string, evalCreateIndex uint64, err error) { + e := w.getEval() + evalCreateIndex, err = w.createEvaluation(e) + return e.ID, evalCreateIndex, err +} + +// createEvalBatched creates an eval but batches calls together +func (w *deploymentWatcher) createEvalBatched() { + w.l.Lock() + defer w.l.Unlock() + + if w.outstandingBatch { + return + } + + go func() { + // Sleep til the batching period is over + time.Sleep(evalBatchPeriod) + + w.l.Lock() + w.outstandingBatch = false + w.l.Unlock() + + if _, _, err := w.createEval(); err != nil { + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to create evaluation for deployment %q: %v", w.d.ID, err) + } + }() +} + +// getEval returns an evaluation suitable for the deployment +func (w *deploymentWatcher) getEval() *structs.Evaluation { + return &structs.Evaluation{ + ID: structs.GenerateUUID(), + Priority: w.j.Priority, + Type: w.j.Type, + TriggeredBy: structs.EvalTriggerRollingUpdate, + JobID: w.j.ID, + DeploymentID: w.d.ID, + Status: structs.EvalStatusPending, + } +} + +// getDeploymentStatusUpdate returns a deployment status update +func (w *deploymentWatcher) getDeploymentStatusUpdate(status, desc string) *structs.DeploymentStatusUpdate { + return &structs.DeploymentStatusUpdate{ + DeploymentID: w.d.ID, + Status: status, + StatusDescription: desc, + } +} + +// getAllocs retrieves the allocations that are part of the deployment blocking +// at the given index. +func (w *deploymentWatcher) getAllocs(index uint64) <-chan []*structs.AllocListStub { + c := make(chan []*structs.AllocListStub, 1) + go func() { + // Build the request + args := &structs.DeploymentSpecificRequest{ + DeploymentID: w.d.ID, + QueryOptions: structs.QueryOptions{ + MinQueryIndex: index, + }, + } + var resp structs.AllocListResponse + + for resp.Index <= index { + if err := w.Allocations(args, &resp); err != nil { + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to retrieve allocations as part of deployment %q: %v", w.d.ID, err) + close(c) + return + } + } + + c <- resp.Allocations + }() + return c +} + +// latestEvalIndex returns the snapshot index of the last evaluation created for +// the job. The index is used to determine if an allocation update requires an +// evaluation to be triggered. +func (w *deploymentWatcher) latestEvalIndex() uint64 { + args := &structs.JobSpecificRequest{ + JobID: w.d.JobID, + } + var resp structs.JobEvaluationsResponse + err := w.Evaluations(args, &resp) + if err != nil { + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to determine last evaluation index for job %q: %v", w.d.JobID, err) + return 0 + } + + if len(resp.Evaluations) == 0 { + return 0 + } + + return resp.Evaluations[0].SnapshotIndex +} diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go new file mode 100644 index 000000000000..15e3fd1f94f6 --- /dev/null +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -0,0 +1,250 @@ +package deploymentwatcher + +import ( + "fmt" + "log" + "sync" + + "github.com/hashicorp/nomad/nomad/structs" +) + +// DeploymentRaftEndpoints exposes the deployment watcher to a set of functions +// to apply data transforms via Raft. +type DeploymentRaftEndpoints interface { + // UpsertEvals is used to upsert a set of evaluations + UpsertEvals([]*structs.Evaluation) (uint64, error) + + // UpsertJob is used to upsert a job + UpsertJob(job *structs.Job) (uint64, error) + + // UpsertDeploymentStatusUpdate is used to upsert a deployment status update + // and potentially create an evaluation. + UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) + + // UpsertDeploymentPromotion is used to promote canaries in a deployment + UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) + + // UpsertDeploymentAllocHealth is used to set the health of allocations in a + // deployment + UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) +} + +// DeploymentStateWatchers are the set of functions required to watch objects on +// behalf of a deployment +type DeploymentStateWatchers interface { + // Evaluations returns the set of evaluations for the given job + Evaluations(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error + + // Allocations returns the set of allocations that are part of the + // deployment. + Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error + + // GetJobVersions is used to lookup the versions of a job. This is used when + // rolling back to find the latest stable job + GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error +} + +// Watcher is used to watch deployments and their allocations created +// by the scheduler and trigger the scheduler when allocation health +// transistions. +type Watcher struct { + enabled bool + logger *log.Logger + + // raft contains the set of Raft endpoints that can be used by the + // deployments watcher + raft DeploymentRaftEndpoints + + // stateWatchers is the set of functions required to watch a deployment for + // state changes + stateWatchers DeploymentStateWatchers + + // watchers is the set of active watchers, one per deployment + watchers map[string]*deploymentWatcher + + // evalBatcher is used to batch the creation of evaluations + evalBatcher *EvalBatcher + + // exitCh is used to exit any goroutines spawned by the watcher + exitCh chan struct{} + + l sync.RWMutex +} + +// NewDeploymentsWatcher returns a deployments watcher that is used to watch +// deployments and trigger the scheduler as needed. +func NewDeploymentsWatcher(logger *log.Logger, w DeploymentStateWatchers, raft DeploymentRaftEndpoints) *Watcher { + exitCh := make(chan struct{}) + return &Watcher{ + stateWatchers: w, + raft: raft, + watchers: make(map[string]*deploymentWatcher, 32), + evalBatcher: NewEvalBatcher(raft, exitCh), + exitCh: exitCh, + logger: logger, + } +} + +// SetEnabled is used to control if the watcher is enabled. The watcher +// should only be enabled on the active leader. +func (w *Watcher) SetEnabled(enabled bool) { + w.l.Lock() + w.enabled = enabled + w.l.Unlock() + if !enabled { + w.Flush() + } +} + +// Flush is used to clear the state of the watcher +func (w *Watcher) Flush() { + w.l.Lock() + defer w.l.Unlock() + + // Stop all the watchers and clear it + for _, watcher := range w.watchers { + watcher.StopWatch() + } + + close(w.exitCh) + + w.watchers = make(map[string]*deploymentWatcher, 32) + w.exitCh = make(chan struct{}) + w.evalBatcher = NewEvalBatcher(w.raft, w.exitCh) +} + +// Watch adds a deployment to the watch list +func (w *Watcher) Watch(d *structs.Deployment, j *structs.Job) { + w.l.Lock() + defer w.l.Unlock() + + // Not enabled so no-op + if !w.enabled { + return + } + + // Already watched so no-op + if _, ok := w.watchers[d.ID]; ok { + return + } + + w.watchers[d.ID] = newDeploymentWatcher(w.logger, w.stateWatchers, d, j, w) +} + +// Unwatch stops watching a deployment. This can be because the deployment is +// complete or being deleted. +func (w *Watcher) Unwatch(d *structs.Deployment) { + w.l.Lock() + defer w.l.Unlock() + + // Not enabled so no-op + if !w.enabled { + return + } + + if watcher, ok := w.watchers[d.ID]; ok { + watcher.StopWatch() + delete(w.watchers, d.ID) + } +} + +// SetAllocHealth is used to set the health of allocations for a deployment. If +// there are any unhealthy allocations, the deployment is updated to be failed. +// Otherwise the allocations are updated and an evaluation is created. +func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest) ( + *structs.DeploymentUpdateResponse, error) { + w.l.Lock() + defer w.l.Unlock() + + // Not enabled so no-op + if !w.enabled { + return nil, nil + } + + watcher, ok := w.watchers[req.DeploymentID] + if !ok { + return nil, fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + } + + return watcher.SetAllocHealth(req) +} + +// PromoteDeployment is used to promote a deployment. If promote is false, +// deployment is marked as failed. Otherwise the deployment is updated and an +// evaluation is created. +func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest) ( + *structs.DeploymentUpdateResponse, error) { + w.l.Lock() + defer w.l.Unlock() + + // Not enabled so no-op + if !w.enabled { + return nil, nil + } + + watcher, ok := w.watchers[req.DeploymentID] + if !ok { + return nil, fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + } + + return watcher.PromoteDeployment(req) +} + +// PauseDeployment is used to toggle the pause state on a deployment. If the +// deployment is being unpaused, an evaluation is created. +func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest) ( + *structs.DeploymentUpdateResponse, error) { + w.l.Lock() + defer w.l.Unlock() + + // Not enabled so no-op + if !w.enabled { + return nil, nil + } + + watcher, ok := w.watchers[req.DeploymentID] + if !ok { + return nil, fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + } + + return watcher.PauseDeployment(req) +} + +// createEvaluation commits the given evaluation to Raft but batches the commit +// with other calls. +func (w *Watcher) createEvaluation(eval *structs.Evaluation) (uint64, error) { + w.l.Lock() + f := w.evalBatcher.CreateEval(eval) + w.l.Unlock() + + return f.Results() +} + +// upsertJob commits the given job to Raft +func (w *Watcher) upsertJob(job *structs.Job) (uint64, error) { + return w.raft.UpsertJob(job) +} + +// upsertDeploymentStatusUpdate commits the given deployment update and optional +// evaluation to Raft +func (w *Watcher) upsertDeploymentStatusUpdate( + u *structs.DeploymentStatusUpdate, + e *structs.Evaluation, + j *structs.Job) (uint64, error) { + return w.raft.UpsertDeploymentStatusUpdate(&structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: u, + Eval: e, + Job: j, + }) +} + +// upsertDeploymentPromotion commits the given deployment promotion to Raft +func (w *Watcher) upsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { + return w.raft.UpsertDeploymentPromotion(req) +} + +// upsertDeploymentAllocHealth commits the given allocation health changes to +// Raft +func (w *Watcher) upsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { + return w.raft.UpsertDeploymentAllocHealth(req) +} diff --git a/nomad/fsm.go b/nomad/fsm.go index 2d8b28107c0e..a157d73203cf 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -157,6 +157,12 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { return n.applyDeregisterVaultAccessor(buf[1:], log.Index) case structs.ApplyPlanResultsRequestType: return n.applyPlanResults(buf[1:], log.Index) + case structs.DeploymentStatusUpdateRequestType: + return n.applyDeploymentStatusUpdate(buf[1:], log.Index) + case structs.DeploymentPromoteRequestType: + return n.applyDeploymentPromotion(buf[1:], log.Index) + case structs.DeploymentAllocHealthRequestType: + return n.applyDeploymentAllocHealth(buf[1:], log.Index) default: if ignoreUnknown { n.logger.Printf("[WARN] nomad.fsm: ignoring unknown message type (%d), upgrade to newer version", msgType) @@ -565,6 +571,71 @@ func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { return nil } +// TODO test +// applyDeploymentStatusUpdate is used to update the status of an existing +// deployment +func (n *nomadFSM) applyDeploymentStatusUpdate(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_status_update"}, time.Now()) + var req structs.DeploymentStatusUpdateRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + + if err := n.state.UpsertDeploymentStatusUpdate(index, &req); err != nil { + n.logger.Printf("[ERR] nomad.fsm: UpsertDeploymentStatusUpdate failed: %v", err) + return err + } + + if req.Eval != nil && req.Eval.ShouldEnqueue() { + n.evalBroker.Enqueue(req.Eval) + } + + return nil +} + +// TODO test +// applyDeploymentPromotion is used to promote canaries in a deployment +func (n *nomadFSM) applyDeploymentPromotion(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_promotion"}, time.Now()) + var req structs.ApplyDeploymentPromoteRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + + if err := n.state.UpsertDeploymentPromotion(index, &req); err != nil { + n.logger.Printf("[ERR] nomad.fsm: UpsertDeploymentPromotion failed: %v", err) + return err + } + + if req.Eval != nil && req.Eval.ShouldEnqueue() { + n.evalBroker.Enqueue(req.Eval) + } + + return nil +} + +// TODO test +// applyDeploymentAllocHealth is used to set the health of allocations as part +// of a deployment +func (n *nomadFSM) applyDeploymentAllocHealth(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_alloc_health"}, time.Now()) + var req structs.ApplyDeploymentAllocHealthRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + + if err := n.state.UpsertDeploymentAllocHealth(index, &req); err != nil { + n.logger.Printf("[ERR] nomad.fsm: UpsertDeploymentAllocHealth failed: %v", err) + return err + } + + if req.Eval != nil && req.Eval.ShouldEnqueue() { + n.evalBroker.Enqueue(req.Eval) + } + + return nil +} + func (n *nomadFSM) Snapshot() (raft.FSMSnapshot, error) { // Create a new snapshot snap, err := n.state.Snapshot() diff --git a/nomad/server.go b/nomad/server.go index 14355a871ce7..70844f07fef6 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -161,16 +161,17 @@ type Server struct { // Holds the RPC endpoints type endpoints struct { - Status *Status - Node *Node - Job *Job - Eval *Eval - Plan *Plan - Alloc *Alloc - Region *Region - Periodic *Periodic - System *System - Operator *Operator + Status *Status + Node *Node + Job *Job + Eval *Eval + Plan *Plan + Alloc *Alloc + Deployment *Deployment + Region *Region + Periodic *Periodic + System *System + Operator *Operator } // NewServer is used to construct a new Nomad server from the @@ -679,6 +680,7 @@ func (s *Server) setupRPC(tlsWrap tlsutil.RegionWrapper) error { s.endpoints.Eval = &Eval{s} s.endpoints.Job = &Job{s} s.endpoints.Node = &Node{srv: s} + s.endpoints.Deployment = &Deployment{srv: s} s.endpoints.Operator = &Operator{s} s.endpoints.Periodic = &Periodic{s} s.endpoints.Plan = &Plan{s} @@ -691,6 +693,7 @@ func (s *Server) setupRPC(tlsWrap tlsutil.RegionWrapper) error { s.rpcServer.Register(s.endpoints.Eval) s.rpcServer.Register(s.endpoints.Job) s.rpcServer.Register(s.endpoints.Node) + s.rpcServer.Register(s.endpoints.Deployment) s.rpcServer.Register(s.endpoints.Operator) s.rpcServer.Register(s.endpoints.Periodic) s.rpcServer.Register(s.endpoints.Plan) diff --git a/nomad/state/schema.go b/nomad/state/schema.go index dfeef6678882..600c551c908c 100644 --- a/nomad/state/schema.go +++ b/nomad/state/schema.go @@ -379,6 +379,16 @@ func allocTableSchema() *memdb.TableSchema { Field: "EvalID", }, }, + + // Deployment index is used to lookup allocations by deployment + "deployment": &memdb.IndexSchema{ + Name: "deployment", + AllowMissing: false, + Unique: false, + Indexer: &memdb.UUIDFieldIndex{ + Field: "DeploymentID", + }, + }, }, } } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index d060c550997c..1f9aa1d4af07 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -6,6 +6,8 @@ import ( "log" "github.com/hashicorp/go-memdb" + multierror "github.com/hashicorp/go-multierror" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" ) @@ -587,7 +589,15 @@ func (s *StateStore) Nodes(ws memdb.WatchSet) (memdb.ResultIterator, error) { func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { txn := s.db.Txn(true) defer txn.Abort() + if err := s.upsertJobImpl(index, job, txn); err != nil { + return err + } + txn.Commit() + return nil +} +// upsertJobImpl is the inplementation for registering a job or updating a job definition +func (s *StateStore) upsertJobImpl(index uint64, job *structs.Job, txn *memdb.Txn) error { // Check if the job already exists existing, err := txn.First("jobs", "id", job.ID) if err != nil { @@ -647,7 +657,6 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() return nil } @@ -1677,6 +1686,29 @@ func (s *StateStore) AllocsByEval(ws memdb.WatchSet, evalID string) ([]*structs. return out, nil } +// AllocsByDeployment returns all the allocations by deployment id +func (s *StateStore) AllocsByDeployment(ws memdb.WatchSet, deploymentID string) ([]*structs.Allocation, error) { + txn := s.db.Txn(false) + + // Get an iterator over the deployments allocations + iter, err := txn.Get("allocs", "deployment", deploymentID) + if err != nil { + return nil, err + } + + ws.Add(iter.WatchCh()) + + var out []*structs.Allocation + for { + raw := iter.Next() + if raw == nil { + break + } + out = append(out, raw.(*structs.Allocation)) + } + return out, nil +} + // Allocs returns an iterator over all the evaluations func (s *StateStore) Allocs(ws memdb.WatchSet) (memdb.ResultIterator, error) { txn := s.db.Txn(false) @@ -1814,6 +1846,255 @@ func (s *StateStore) VaultAccessorsByNode(ws memdb.WatchSet, nodeID string) ([]* return out, nil } +// TODO test +// UpsertDeploymentStatusUpdate is used to upsert deployment status updates and +// potentially make a evaluation +func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.DeploymentStatusUpdateRequest) error { + txn := s.db.Txn(true) + defer txn.Abort() + + if err := s.upsertDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { + return err + } + + // Upsert the job if necessary + if req.Job != nil { + if err := s.upsertJobImpl(index, req.Job, txn); err != nil { + return err + } + } + + // Upsert the optional eval + if req.Eval != nil { + if err := s.nestedUpsertEval(txn, index, req.Eval); err != nil { + return err + } + } + + txn.Commit() + return nil +} + +// upsertDeploymentStatusUpdateImpl is used to upsert deployment status updates +func (s *StateStore) upsertDeploymentStatusUpdateImpl(index uint64, u *structs.DeploymentStatusUpdate, txn *memdb.Txn) error { + raw, err := txn.First("deployment", "id", u.DeploymentID) + if err != nil { + return err + } + if raw == nil { + return fmt.Errorf("Deployment ID %q couldn't be updated as it does not exist", u.DeploymentID) + } + + // Ensure that the deployment is not already terminal + act := raw.(*structs.Deployment) + if !act.Active() { + return fmt.Errorf("Deployment %q has terminal status %q:", u.DeploymentID, act.Status) + } + + // Apply the new status + copy := act.Copy() + copy.Status = u.Status + copy.StatusDescription = u.StatusDescription + copy.ModifyIndex = index + + // Insert the deployment + if err := txn.Insert("deployment", copy); err != nil { + return err + } + + // Update the index + if err := txn.Insert("index", &IndexEntry{"deployment", index}); err != nil { + return fmt.Errorf("index update failed: %v", err) + } + + return nil +} + +// TODO test +// UpsertDeploymentPromotion is used to promote canaries in a deployment and +// potentially make a evaluation +func (s *StateStore) UpsertDeploymentPromotion(index uint64, req *structs.ApplyDeploymentPromoteRequest) error { + txn := s.db.Txn(true) + defer txn.Abort() + + // Retrieve deployment and ensure it is not terminal + rawd, err := txn.First("deployment", "id", req.DeploymentID) + if err != nil { + return err + } else if rawd == nil { + return fmt.Errorf("Deployment ID %q couldn't be updated as it does not exist", req.DeploymentID) + } + + // Ensure that the deployment is not already terminal + deployment := rawd.(*structs.Deployment) + if !deployment.Active() { + return fmt.Errorf("Deployment %q has terminal status %q:", deployment.ID, deployment.Status) + } + + // Retrieve effected allocations + iter, err := txn.Get("allocs", "deployment", req.DeploymentID) + if err != nil { + return err + } + + var unhealthyErr multierror.Error + var canaries []*structs.Allocation + for { + raw := iter.Next() + if raw == nil { + break + } + + alloc := raw.(*structs.Allocation) + if !alloc.Canary { + continue + } + + // Check that the canary is part of a group being promoted + inGroup, ok := req.Groups[alloc.TaskGroup] + if !req.All && (!ok || !inGroup) { + continue + } + + // Ensure the canaries are healthy + if !alloc.DeploymentStatus.IsHealthy() { + multierror.Append(&unhealthyErr, fmt.Errorf("Canary allocation %q for group %q is not healthy", alloc.ID, alloc.TaskGroup)) + continue + } + + canaries = append(canaries, alloc) + } + + if err := unhealthyErr.ErrorOrNil(); err != nil { + return err + } + + // Mark canaries as promoted and upsert them + canariesCopy := make([]*structs.Allocation, 0, len(canaries)) + for _, c := range canaries { + copy := c.Copy() + copy.DeploymentStatus.Promoted = true + copy.DeploymentStatus.ModifyIndex = index + canariesCopy = append(canariesCopy, copy) + } + if err := s.upsertAllocsImpl(index, canariesCopy, txn); err != nil { + return err + } + + // Update deployment + copy := deployment.Copy() + copy.ModifyIndex = index + for tg, status := range copy.TaskGroups { + inGroup, ok := req.Groups[tg] + if !req.All && (!ok || !inGroup) { + continue + } + + status.Promoted = true + } + + // Insert the deployment + if err := txn.Insert("deployment", copy); err != nil { + return err + } + + // Update the index + if err := txn.Insert("index", &IndexEntry{"deployment", index}); err != nil { + return fmt.Errorf("index update failed: %v", err) + } + + // Upsert the optional eval + if req.Eval != nil { + if err := s.nestedUpsertEval(txn, index, req.Eval); err != nil { + return err + } + } + + txn.Commit() + return nil +} + +// TODO test +// UpsertDeploymentAllocHealth is used to update the health of allocations as +// part of the deployment and potentially make a evaluation +func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { + txn := s.db.Txn(true) + defer txn.Abort() + + // Update the health status of each allocation + if total := len(req.HealthyAllocationIDs) + len(req.UnhealthyAllocationIDs); total != 0 { + setAllocHealth := func(id string, healthy bool) error { + existing, err := txn.First("allocs", "id", id) + if err != nil { + return fmt.Errorf("alloc %q lookup failed: %v", id, err) + } + if existing == nil { + return fmt.Errorf("unknown alloc %q", id) + } + + old := existing.(*structs.Allocation) + copy := old.Copy() + + // Set the health + if copy.DeploymentStatus == nil { + copy.DeploymentStatus = &structs.AllocDeploymentStatus{} + } + copy.DeploymentStatus.Healthy = helper.BoolToPtr(true) + copy.DeploymentStatus.ModifyIndex = index + + if err := s.updateDeploymentWithAlloc(index, copy, old, txn); err != nil { + return fmt.Errorf("error updating deployment: %v", err) + } + + if err := txn.Insert("allocs", copy); err != nil { + return fmt.Errorf("alloc insert failed: %v", err) + } + + return nil + } + + for _, id := range req.HealthyAllocationIDs { + if err := setAllocHealth(id, true); err != nil { + return err + } + } + for _, id := range req.UnhealthyAllocationIDs { + if err := setAllocHealth(id, false); err != nil { + return err + } + } + + // Update the indexes + if err := txn.Insert("index", &IndexEntry{"allocs", index}); err != nil { + return fmt.Errorf("index update failed: %v", err) + } + } + + // Update the deployment status as needed. + if req.DeploymentUpdate != nil { + if err := s.upsertDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { + return err + } + } + + // Upsert the job if necessary + if req.Job != nil { + if err := s.upsertJobImpl(index, req.Job, txn); err != nil { + return err + } + } + + // Upsert the optional eval + if req.Eval != nil { + if err := s.nestedUpsertEval(txn, index, req.Eval); err != nil { + return err + } + } + + txn.Commit() + return nil +} + // LastIndex returns the greatest index value for all indexes func (s *StateStore) LatestIndex() (uint64, error) { indexes, err := s.Indexes() diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 1783f1cdbfbd..d21d6cc0e341 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -3252,6 +3252,61 @@ func TestStateStore_UpsertAlloc_Alloc(t *testing.T) { } } +func TestStateStore_UpsertAlloc_Deployment(t *testing.T) { + state := testStateStore(t) + deployment := mock.Deployment() + alloc := mock.Alloc() + alloc.DeploymentID = deployment.ID + + if err := state.UpsertJob(999, alloc.Job); err != nil { + t.Fatalf("err: %v", err) + } + if err := state.UpsertDeployment(1000, deployment, false); err != nil { + t.Fatalf("err: %v", err) + } + + // Create a watch set so we can test that update fires the watch + ws := memdb.NewWatchSet() + if _, err := state.AllocsByDeployment(ws, alloc.DeploymentID); err != nil { + t.Fatalf("bad: %v", err) + } + + err := state.UpsertAllocs(1001, []*structs.Allocation{alloc}) + if err != nil { + t.Fatalf("err: %v", err) + } + + if !watchFired(ws) { + t.Fatalf("watch not fired") + } + + ws = memdb.NewWatchSet() + allocs, err := state.AllocsByDeployment(ws, alloc.DeploymentID) + if err != nil { + t.Fatalf("err: %v", err) + } + + if len(allocs) != 1 { + t.Fatalf("bad: %#v", allocs) + } + + if !reflect.DeepEqual(alloc, allocs[0]) { + t.Fatalf("bad: %#v %#v", alloc, allocs[0]) + } + + index, err := state.Index("allocs") + if err != nil { + t.Fatalf("err: %v", err) + } + if index != 1001 { + t.Fatalf("bad: %d", index) + } + + if watchFired(ws) { + t.Fatalf("bad") + } +} + // Testing to ensure we keep issue // https://github.com/hashicorp/nomad/issues/2583 fixed func TestStateStore_UpsertAlloc_No_Job(t *testing.T) { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index d8f73b02ebd1..81d42cc8fe2b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -54,6 +54,9 @@ const ( VaultAccessorRegisterRequestType VaultAccessorDegisterRequestType ApplyPlanResultsRequestType + DeploymentStatusUpdateRequestType + DeploymentPromoteRequestType + DeploymentAllocHealtRequestType ) const ( @@ -489,6 +492,86 @@ type GenericRequest struct { QueryOptions } +// DeploymentStatusUpdateRequest is used to update the status of a deployment as +// well as optionally creating an evaluation atomically. +type DeploymentStatusUpdateRequest struct { + // Eval, if set, is used to create an evaluation at the same time as + // updating the status of a deployment. + Eval *Evaluation + + // DeploymentUpdate is a status update to apply to the given + // deployment. + DeploymentUpdate *DeploymentStatusUpdate + + // Job is used to optionally upsert a job. This is used when setting the + // allocation health results in a deployment failure and the deployment + // auto-reverts to the latest stable job. + Job *Job +} + +// DeploymentAllocHealthRequest is used to set the health of a set of +// allocations as part of a deployment. +type DeploymentAllocHealthRequest struct { + DeploymentID string + + // Marks these allocations as healthy, allow further allocations + // to be rolled. + HealthyAllocationIDs []string + + // Any unhealthy allocations fail the deployment + UnhealthyAllocationIDs []string +} + +// ApplyDeploymentAllocHealthRequest is used to apply an alloc health request via Raft +type ApplyDeploymentAllocHealthRequest struct { + DeploymentAllocHealthRequest + + // An optional field to update the status of a deployment + DeploymentUpdate *DeploymentStatusUpdate + + // Job is used to optionally upsert a job. This is used when setting the + // allocation health results in a deployment failure and the deployment + // auto-reverts to the latest stable job. + Job *Job + + // An optional evaluation to create after promoting the canaries + Eval *Evaluation +} + +// DeploymentPromoteRequest is used to promote task groups in a deployment +type DeploymentPromoteRequest struct { + DeploymentID string + + // All is to promote all task groups + All bool + + // Groups is used to set the promotion status per task group + Groups map[string]bool +} + +// ApplyDeploymentPromoteRequest is used to apply a promotion request via Raft +type ApplyDeploymentPromoteRequest struct { + DeploymentPromoteRequest + + // An optional evaluation to create after promoting the canaries + Eval *Evaluation +} + +// DeploymentPauseRequest is used to pause a deployment +type DeploymentPauseRequest struct { + DeploymentID string + + // Pause sets the pause status + Pause bool +} + +// DeploymentSpecificRequest is used to make a request specific to a particular +// deployment +type DeploymentSpecificRequest struct { + DeploymentID string + QueryOptions +} + // GenericResponse is used to respond to a request where no // specific response information is needed. type GenericResponse struct { @@ -727,6 +810,15 @@ type PeriodicForceResponse struct { WriteMeta } +// DeploymentUpdateResponse is used to respond to a deployment change. The +// response will include the modify index of the deployment as well as details +// of any triggered evaluation. +type DeploymentUpdateResponse struct { + EvalID string + EvalCreateIndex uint64 + DeploymentModifyIndex uint64 +} + const ( NodeStatusInit = "initializing" NodeStatusReady = "ready" @@ -3747,8 +3839,11 @@ const ( // DeploymentStatusDescriptions are the various descriptions of the states a // deployment can be in. - DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" - DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" + DeploymentStatusDescriptionRunning = "Deployment is running" + DeploymentStatusDescriptionPaused = "Deployment is paused" + DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" + DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" + DeploymentStatusDescriptionFailedAllocations = "Failed due to unhealthy allocations" ) // Deployment is the object that represents a job deployment which is used to @@ -4121,6 +4216,7 @@ func (a *Allocation) Stub() *AllocListStub { ClientStatus: a.ClientStatus, ClientDescription: a.ClientDescription, TaskStates: a.TaskStates, + DeploymentStatus: a.DeploymentStatus, CreateIndex: a.CreateIndex, ModifyIndex: a.ModifyIndex, CreateTime: a.CreateTime, @@ -4140,6 +4236,7 @@ type AllocListStub struct { ClientStatus string ClientDescription string TaskStates map[string]*TaskState + DeploymentStatus *AllocDeploymentStatus CreateIndex uint64 ModifyIndex uint64 CreateTime int64 @@ -4260,6 +4357,10 @@ type AllocDeploymentStatus struct { // Promoted marks whether the allocation is promoted. This field is only // used if the allocation is a canary. Promoted bool + + // ModifyIndex is the raft index in which the deployment status was last + // changed. + ModifyIndex uint64 } // IsHealthy returns if the allocation is marked as healthy as part of a @@ -4272,6 +4373,16 @@ func (a *AllocDeploymentStatus) IsHealthy() bool { return a.Healthy != nil && *a.Healthy } +// IsUnhealthy returns if the allocation is marked as unhealthy as part of a +// deployment +func (a *AllocDeploymentStatus) IsUnhealthy() bool { + if a == nil { + return false + } + + return a.Healthy != nil && !*a.Healthy +} + // IsPromoted returns if the allocation is promoted as as part of a deployment func (a *AllocDeploymentStatus) IsPromoted() bool { if a == nil { @@ -4374,6 +4485,9 @@ type Evaluation struct { // the evaluation was created NodeModifyIndex uint64 + // DeploymentID is the ID of the deployment that triggered the evaluation. + DeploymentID string + // Status of the evaluation Status string diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 7e1e1aa47023..e6765719b74b 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -301,7 +301,8 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // desired means we need to create canaries numDestructive := len(destructive) strategy := tg.Update - requireCanary := numDestructive != 0 && strategy != nil && len(canaries) < strategy.Canary + canariesPromoted := dstate != nil && dstate.Promoted + requireCanary := numDestructive != 0 && strategy != nil && len(canaries) < strategy.Canary && !canariesPromoted if requireCanary && !a.deploymentPaused && !a.deploymentFailed { number := strategy.Canary - len(canaries) number = helper.IntMin(numDestructive, number) @@ -416,6 +417,11 @@ func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, dest if a.deployment != nil { partOf, _ := untainted.filterByDeployment(a.deployment.ID) for _, alloc := range partOf { + // An unhealthy allocation means nothing else should be happen. + if alloc.DeploymentStatus.IsUnhealthy() { + return 0 + } + if !alloc.DeploymentStatus.IsHealthy() { limit-- } From b6277af259212aee90919e72def740fb22b632f4 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 26 Jun 2017 17:25:42 -0700 Subject: [PATCH 023/105] state store tests --- nomad/state/state_store.go | 55 +-- nomad/state/state_store_test.go | 616 ++++++++++++++++++++++++++++++++ 2 files changed, 644 insertions(+), 27 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 1f9aa1d4af07..41668ee6836f 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -1846,7 +1846,6 @@ func (s *StateStore) VaultAccessorsByNode(ws memdb.WatchSet, nodeID string) ([]* return out, nil } -// TODO test // UpsertDeploymentStatusUpdate is used to upsert deployment status updates and // potentially make a evaluation func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.DeploymentStatusUpdateRequest) error { @@ -1877,22 +1876,19 @@ func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.Dep // upsertDeploymentStatusUpdateImpl is used to upsert deployment status updates func (s *StateStore) upsertDeploymentStatusUpdateImpl(index uint64, u *structs.DeploymentStatusUpdate, txn *memdb.Txn) error { - raw, err := txn.First("deployment", "id", u.DeploymentID) + // Retrieve deployment + ws := memdb.NewWatchSet() + deployment, err := s.deploymentByIDImpl(ws, u.DeploymentID, txn) if err != nil { return err - } - if raw == nil { + } else if deployment == nil { return fmt.Errorf("Deployment ID %q couldn't be updated as it does not exist", u.DeploymentID) - } - - // Ensure that the deployment is not already terminal - act := raw.(*structs.Deployment) - if !act.Active() { - return fmt.Errorf("Deployment %q has terminal status %q:", u.DeploymentID, act.Status) + } else if !deployment.Active() { + return fmt.Errorf("Deployment %q has terminal status %q:", deployment.ID, deployment.Status) } // Apply the new status - copy := act.Copy() + copy := deployment.Copy() copy.Status = u.Status copy.StatusDescription = u.StatusDescription copy.ModifyIndex = index @@ -1910,24 +1906,20 @@ func (s *StateStore) upsertDeploymentStatusUpdateImpl(index uint64, u *structs.D return nil } -// TODO test // UpsertDeploymentPromotion is used to promote canaries in a deployment and // potentially make a evaluation func (s *StateStore) UpsertDeploymentPromotion(index uint64, req *structs.ApplyDeploymentPromoteRequest) error { txn := s.db.Txn(true) defer txn.Abort() - // Retrieve deployment and ensure it is not terminal - rawd, err := txn.First("deployment", "id", req.DeploymentID) + // Retrieve deployment and ensure it is not terminal and is active + ws := memdb.NewWatchSet() + deployment, err := s.deploymentByIDImpl(ws, req.DeploymentID, txn) if err != nil { return err - } else if rawd == nil { + } else if deployment == nil { return fmt.Errorf("Deployment ID %q couldn't be updated as it does not exist", req.DeploymentID) - } - - // Ensure that the deployment is not already terminal - deployment := rawd.(*structs.Deployment) - if !deployment.Active() { + } else if !deployment.Active() { return fmt.Errorf("Deployment %q has terminal status %q:", deployment.ID, deployment.Status) } @@ -2021,6 +2013,17 @@ func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.Appl txn := s.db.Txn(true) defer txn.Abort() + // Retrieve deployment and ensure it is not terminal and is active + ws := memdb.NewWatchSet() + deployment, err := s.deploymentByIDImpl(ws, req.DeploymentID, txn) + if err != nil { + return err + } else if deployment == nil { + return fmt.Errorf("Deployment ID %q couldn't be updated as it does not exist", req.DeploymentID) + } else if !deployment.Active() { + return fmt.Errorf("Deployment %q has terminal status %q:", deployment.ID, deployment.Status) + } + // Update the health status of each allocation if total := len(req.HealthyAllocationIDs) + len(req.UnhealthyAllocationIDs); total != 0 { setAllocHealth := func(id string, healthy bool) error { @@ -2033,13 +2036,16 @@ func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.Appl } old := existing.(*structs.Allocation) - copy := old.Copy() + if old.DeploymentID != req.DeploymentID { + return fmt.Errorf("alloc %q is not part of deployment %q", id, req.DeploymentID) + } // Set the health + copy := old.Copy() if copy.DeploymentStatus == nil { copy.DeploymentStatus = &structs.AllocDeploymentStatus{} } - copy.DeploymentStatus.Healthy = helper.BoolToPtr(true) + copy.DeploymentStatus.Healthy = helper.BoolToPtr(healthy) copy.DeploymentStatus.ModifyIndex = index if err := s.updateDeploymentWithAlloc(index, copy, old, txn); err != nil { @@ -2556,11 +2562,6 @@ func (s *StateStore) updateDeploymentWithAlloc(index uint64, alloc, existing *st deploymentCopy := deployment.Copy() deploymentCopy.ModifyIndex = index - if unhealthy != 0 { - deploymentCopy.Status = structs.DeploymentStatusFailed - deploymentCopy.StatusDescription = "Allocation(s) marked as unhealthy" - } - state := deploymentCopy.TaskGroups[alloc.TaskGroup] state.PlacedAllocs += placed state.HealthyAllocs += healthy diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index d21d6cc0e341..aac9f5a65713 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -10,6 +10,7 @@ import ( "time" memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" ) @@ -4756,6 +4757,621 @@ func TestJobSummary_UpdateClientStatus(t *testing.T) { } } +// Test that non-existant deployment can't be updated +func TestStateStore_UpsertDeploymentStatusUpdate_NonExistant(t *testing.T) { + state := testStateStore(t) + + // Update the non-existant deployment + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: structs.GenerateUUID(), + Status: structs.DeploymentStatusRunning, + }, + } + err := state.UpsertDeploymentStatusUpdate(2, req) + if err == nil || !strings.Contains(err.Error(), "does not exist") { + t.Fatalf("expected error updating the status because the deployment doesn't exist") + } +} + +// Test that terminal deployment can't be updated +func TestStateStore_UpsertDeploymentStatusUpdate_Terminal(t *testing.T) { + state := testStateStore(t) + + // Insert a terminal deployment + d := mock.Deployment() + d.Status = structs.DeploymentStatusFailed + + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Update the deployment + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusRunning, + }, + } + err := state.UpsertDeploymentStatusUpdate(2, req) + if err == nil || !strings.Contains(err.Error(), "has terminal status") { + t.Fatalf("expected error updating the status because the deployment is terminal") + } +} + +// Test that a non terminal deployment is updated and that a job and eval are +// created. +func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) { + state := testStateStore(t) + + // Insert a deployment + d := mock.Deployment() + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create an eval and a job + e := mock.Eval() + j := mock.Job() + + // Update the deployment + status, desc := structs.DeploymentStatusFailed, "foo" + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: status, + StatusDescription: desc, + }, + Job: j, + Eval: e, + } + err := state.UpsertDeploymentStatusUpdate(2, req) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the status was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if dout.Status != status || dout.StatusDescription != desc { + t.Fatalf("bad: %#v", dout) + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } + + // Check that the job was created + jout, _ := state.JobByID(ws, j.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } +} + +// Test that non-existant deployment can't be promoted +func TestStateStore_UpsertDeploymentPromotion_NonExistant(t *testing.T) { + state := testStateStore(t) + + // Promote the non-existant deployment + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: structs.GenerateUUID(), + All: true, + }, + } + err := state.UpsertDeploymentPromotion(2, req) + if err == nil || !strings.Contains(err.Error(), "does not exist") { + t.Fatalf("expected error promoting because the deployment doesn't exist") + } +} + +// Test that terminal deployment can't be updated +func TestStateStore_UpsertDeploymentPromotion_Terminal(t *testing.T) { + state := testStateStore(t) + + // Insert a terminal deployment + d := mock.Deployment() + d.Status = structs.DeploymentStatusFailed + + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Promote the deployment + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + } + err := state.UpsertDeploymentPromotion(2, req) + if err == nil || !strings.Contains(err.Error(), "has terminal status") { + t.Fatalf("expected error updating the status because the deployment is terminal: %v", err) + } +} + +// Test promoting unhealthy canaries in a deployment. +func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) { + state := testStateStore(t) + + // Create a job + j := mock.Job() + if err := state.UpsertJob(1, j); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a deployment + d := mock.Deployment() + d.JobID = j.ID + if err := state.UpsertDeployment(2, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a set of allocations + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + c1.Canary = true + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + c2.Canary = true + + if err := state.UpsertAllocs(3, []*structs.Allocation{c1, c2}); err != nil { + t.Fatalf("err: %v", err) + } + + // Promote the canaries + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + } + err := state.UpsertDeploymentPromotion(4, req) + if err == nil { + t.Fatalf("bad: %v", err) + } + if !strings.Contains(err.Error(), c1.ID) { + t.Fatalf("expect canary %q to be listed as unhealth: %v", c1.ID, err) + } + if !strings.Contains(err.Error(), c2.ID) { + t.Fatalf("expect canary %q to be listed as unhealth: %v", c2.ID, err) + } +} + +// Test promoting all canaries in a deployment. +func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { + state := testStateStore(t) + + // Create a job with two task groups + j := mock.Job() + tg1 := j.TaskGroups[0] + tg2 := tg1.Copy() + tg2.Name = "foo" + j.TaskGroups = append(j.TaskGroups, tg2) + if err := state.UpsertJob(1, j); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a deployment + d := mock.Deployment() + d.JobID = j.ID + d.TaskGroups = map[string]*structs.DeploymentState{ + "web": &structs.DeploymentState{ + DesiredTotal: 10, + DesiredCanaries: 1, + }, + "foo": &structs.DeploymentState{ + DesiredTotal: 10, + DesiredCanaries: 1, + }, + } + if err := state.UpsertDeployment(2, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a set of allocations + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + c1.Canary = true + c1.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + c2.Canary = true + c2.TaskGroup = tg2.Name + c2.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + + if err := state.UpsertAllocs(3, []*structs.Allocation{c1, c2}); err != nil { + t.Fatalf("err: %v", err) + } + + // Create an eval + e := mock.Eval() + + // Promote the canaries + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + Eval: e, + } + err := state.UpsertDeploymentPromotion(4, req) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the status per task group was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if len(dout.TaskGroups) != 2 { + t.Fatalf("bad: %#v", dout.TaskGroups) + } + for tg, state := range dout.TaskGroups { + if !state.Promoted { + t.Fatalf("bad: group %q not promoted %#v", tg, state) + } + } + + // Check that the allocs were promoted + out1, err := state.AllocByID(ws, c1.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + out2, err := state.AllocByID(ws, c2.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + for _, alloc := range []*structs.Allocation{out1, out2} { + if alloc.DeploymentStatus == nil { + t.Fatalf("bad: alloc %q has nil deployment status", alloc.ID) + } + if !alloc.DeploymentStatus.Promoted { + t.Fatalf("bad: alloc %q not promoted", alloc.ID) + } + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } +} + +// Test promoting a subset of canaries in a deployment. +func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { + state := testStateStore(t) + + // Create a job with two task groups + j := mock.Job() + tg1 := j.TaskGroups[0] + tg2 := tg1.Copy() + tg2.Name = "foo" + j.TaskGroups = append(j.TaskGroups, tg2) + if err := state.UpsertJob(1, j); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a deployment + d := mock.Deployment() + d.JobID = j.ID + d.TaskGroups = map[string]*structs.DeploymentState{ + "web": &structs.DeploymentState{ + DesiredTotal: 10, + DesiredCanaries: 1, + }, + "foo": &structs.DeploymentState{ + DesiredTotal: 10, + DesiredCanaries: 1, + }, + } + if err := state.UpsertDeployment(2, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a set of allocations + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + c1.Canary = true + c1.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + c2.Canary = true + c2.TaskGroup = tg2.Name + c2.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + + if err := state.UpsertAllocs(3, []*structs.Allocation{c1, c2}); err != nil { + t.Fatalf("err: %v", err) + } + + // Create an eval + e := mock.Eval() + + // Promote the canaries + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + Groups: map[string]bool{ + "web": true, + }, + }, + Eval: e, + } + err := state.UpsertDeploymentPromotion(4, req) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the status per task group was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if len(dout.TaskGroups) != 2 { + t.Fatalf("bad: %#v", dout.TaskGroups) + } + stateout, ok := dout.TaskGroups["web"] + if !ok { + t.Fatalf("bad: no state for task group web") + } + if !stateout.Promoted { + t.Fatalf("bad: task group web not promoted: %#v", stateout) + } + + // Check that the allocs were promoted + out1, err := state.AllocByID(ws, c1.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + out2, err := state.AllocByID(ws, c2.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + if !out1.DeploymentStatus.Promoted { + t.Fatalf("bad: alloc %q not promoted", out1.ID) + } + if out2.DeploymentStatus.Promoted { + t.Fatalf("bad: alloc %q promoted", out2.ID) + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } +} + +// Test that allocation health can't be set against a non-existant deployment +func TestStateStore_UpsertDeploymentAllocHealth_NonExistant(t *testing.T) { + state := testStateStore(t) + + // Set health against the non-existant deployment + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: structs.GenerateUUID(), + HealthyAllocationIDs: []string{structs.GenerateUUID()}, + }, + } + err := state.UpsertDeploymentAllocHealth(2, req) + if err == nil || !strings.Contains(err.Error(), "does not exist") { + t.Fatalf("expected error because the deployment doesn't exist: %v", err) + } +} + +// Test that allocation health can't be set against a terminal deployment +func TestStateStore_UpsertDeploymentAllocHealth_Terminal(t *testing.T) { + state := testStateStore(t) + + // Insert a terminal deployment + d := mock.Deployment() + d.Status = structs.DeploymentStatusFailed + + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Set health against the terminal deployment + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{structs.GenerateUUID()}, + }, + } + err := state.UpsertDeploymentAllocHealth(2, req) + if err == nil || !strings.Contains(err.Error(), "has terminal status") { + t.Fatalf("expected error because the deployment is terminal: %v", err) + } +} + +// Test that allocation health can't be set against a non-existant alloc +func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_NonExistant(t *testing.T) { + state := testStateStore(t) + + // Insert a deployment + d := mock.Deployment() + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Set health against the terminal deployment + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{structs.GenerateUUID()}, + }, + } + err := state.UpsertDeploymentAllocHealth(2, req) + if err == nil || !strings.Contains(err.Error(), "unknown alloc") { + t.Fatalf("expected error because the alloc doesn't exist: %v", err) + } +} + +// Test that allocation health can't be set for an alloc with mismatched +// deployment ids +func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_MismatchDeployment(t *testing.T) { + state := testStateStore(t) + + // Insert two deployment + d1 := mock.Deployment() + d2 := mock.Deployment() + if err := state.UpsertDeployment(1, d1, false); err != nil { + t.Fatalf("bad: %v", err) + } + if err := state.UpsertDeployment(2, d2, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Insert an alloc for a random deployment + a := mock.Alloc() + a.DeploymentID = d1.ID + if err := state.UpsertAllocs(3, []*structs.Allocation{a}); err != nil { + t.Fatalf("bad: %v", err) + } + + // Set health against the terminal deployment + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d2.ID, + HealthyAllocationIDs: []string{a.ID}, + }, + } + err := state.UpsertDeploymentAllocHealth(4, req) + if err == nil || !strings.Contains(err.Error(), "not part of deployment") { + t.Fatalf("expected error because the alloc isn't part of the deployment: %v", err) + } +} + +// Test that allocation health is properly set +func TestStateStore_UpsertDeploymentAllocHealth(t *testing.T) { + state := testStateStore(t) + + // Insert a deployment + d := mock.Deployment() + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Insert two allocations + a1 := mock.Alloc() + a1.DeploymentID = d.ID + a2 := mock.Alloc() + a2.DeploymentID = d.ID + if err := state.UpsertAllocs(2, []*structs.Allocation{a1, a2}); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a job to roll back to + j := mock.Job() + + // Create an eval that should be upserted + e := mock.Eval() + + // Create a status update for the deployment + status, desc := structs.DeploymentStatusFailed, "foo" + u := &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: status, + StatusDescription: desc, + } + + // Set health against the deployment + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a1.ID}, + UnhealthyAllocationIDs: []string{a2.ID}, + }, + Job: j, + Eval: e, + DeploymentUpdate: u, + } + err := state.UpsertDeploymentAllocHealth(3, req) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the status was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if dout.Status != status || dout.StatusDescription != desc { + t.Fatalf("bad: %#v", dout) + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } + + // Check that the job was created + jout, _ := state.JobByID(ws, j.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } + + // Check the status of the allocs + out1, err := state.AllocByID(ws, a1.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + out2, err := state.AllocByID(ws, a2.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + if !out1.DeploymentStatus.IsHealthy() { + t.Fatalf("bad: alloc %q not healthy", out1.ID) + } + if !out2.DeploymentStatus.IsUnhealthy() { + t.Fatalf("bad: alloc %q not unhealthy", out2.ID) + } +} + func TestStateStore_UpsertVaultAccessors(t *testing.T) { state := testStateStore(t) a := mock.VaultAccessor() From 9213bb4f6210a8af4487dcf8dc68de09803c54bf Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 27 Jun 2017 10:31:32 -0700 Subject: [PATCH 024/105] FSM Tests --- nomad/fsm.go | 3 - nomad/fsm_test.go | 484 +++++++++++++++++++++++++++++++-------- nomad/structs/structs.go | 2 +- 3 files changed, 392 insertions(+), 97 deletions(-) diff --git a/nomad/fsm.go b/nomad/fsm.go index a157d73203cf..56151d024585 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -571,7 +571,6 @@ func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { return nil } -// TODO test // applyDeploymentStatusUpdate is used to update the status of an existing // deployment func (n *nomadFSM) applyDeploymentStatusUpdate(buf []byte, index uint64) interface{} { @@ -593,7 +592,6 @@ func (n *nomadFSM) applyDeploymentStatusUpdate(buf []byte, index uint64) interfa return nil } -// TODO test // applyDeploymentPromotion is used to promote canaries in a deployment func (n *nomadFSM) applyDeploymentPromotion(buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_promotion"}, time.Now()) @@ -614,7 +612,6 @@ func (n *nomadFSM) applyDeploymentPromotion(buf []byte, index uint64) interface{ return nil } -// TODO test // applyDeploymentAllocHealth is used to set the health of allocations as part // of a deployment func (n *nomadFSM) applyDeploymentAllocHealth(buf []byte, index uint64) interface{} { diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 2eaf4d079d6e..b706f8135058 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -9,6 +9,7 @@ import ( "time" memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" @@ -1047,6 +1048,396 @@ func TestFSM_DeregisterVaultAccessor(t *testing.T) { } } +func TestFSM_ApplyPlanResults(t *testing.T) { + fsm := testFSM(t) + + // Create the request and create a deployment + alloc := mock.Alloc() + job := alloc.Job + alloc.Job = nil + + d := mock.Deployment() + d.JobID = job.ID + d.JobModifyIndex = job.ModifyIndex + d.JobVersion = job.Version + + alloc.DeploymentID = d.ID + + fsm.State().UpsertJobSummary(1, mock.JobSummary(alloc.JobID)) + req := structs.ApplyPlanResultsRequest{ + AllocUpdateRequest: structs.AllocUpdateRequest{ + Job: job, + Alloc: []*structs.Allocation{alloc}, + }, + CreatedDeployment: d, + } + buf, err := structs.Encode(structs.ApplyPlanResultsRequestType, req) + if err != nil { + t.Fatalf("err: %v", err) + } + + resp := fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Verify the allocation is registered + ws := memdb.NewWatchSet() + out, err := fsm.State().AllocByID(ws, alloc.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + alloc.CreateIndex = out.CreateIndex + alloc.ModifyIndex = out.ModifyIndex + alloc.AllocModifyIndex = out.AllocModifyIndex + + // Job should be re-attached + alloc.Job = job + if !reflect.DeepEqual(alloc, out) { + t.Fatalf("bad: %#v %#v", alloc, out) + } + + dout, err := fsm.State().DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + if tg, ok := dout.TaskGroups[alloc.TaskGroup]; !ok || tg.PlacedAllocs != 1 { + t.Fatalf("err: %v %v", tg, err) + } + + // Ensure that the original job is used + evictAlloc := alloc.Copy() + job = mock.Job() + job.Priority = 123 + + evictAlloc.Job = nil + evictAlloc.DesiredStatus = structs.AllocDesiredStatusEvict + req2 := structs.ApplyPlanResultsRequest{ + AllocUpdateRequest: structs.AllocUpdateRequest{ + Job: job, + Alloc: []*structs.Allocation{evictAlloc}, + }, + } + buf, err = structs.Encode(structs.ApplyPlanResultsRequestType, req2) + if err != nil { + t.Fatalf("err: %v", err) + } + + resp = fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Verify we are evicted + out, err = fsm.State().AllocByID(ws, alloc.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + if out.DesiredStatus != structs.AllocDesiredStatusEvict { + t.Fatalf("alloc found!") + } + if out.Job == nil || out.Job.Priority == 123 { + t.Fatalf("bad job") + } +} + +func TestFSM_DeploymentStatusUpdate(t *testing.T) { + fsm := testFSM(t) + fsm.evalBroker.SetEnabled(true) + state := fsm.State() + + // Upsert a deployment + d := mock.Deployment() + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a request to update the deployment, create an eval and job + e := mock.Eval() + j := mock.Job() + status, desc := structs.DeploymentStatusFailed, "foo" + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: status, + StatusDescription: desc, + }, + Job: j, + Eval: e, + } + buf, err := structs.Encode(structs.DeploymentStatusUpdateRequestType, req) + if err != nil { + t.Fatalf("err: %v", err) + } + resp := fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Check that the status was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if dout.Status != status || dout.StatusDescription != desc { + t.Fatalf("bad: %#v", dout) + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } + + // Check that the job was created + jout, _ := state.JobByID(ws, j.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } + + // Assert the eval was enqueued + stats := fsm.evalBroker.Stats() + if stats.TotalReady != 1 { + t.Fatalf("bad: %#v %#v", stats, e) + } +} + +func TestFSM_DeploymentPromotion(t *testing.T) { + fsm := testFSM(t) + fsm.evalBroker.SetEnabled(true) + state := fsm.State() + + // Create a job with two task groups + j := mock.Job() + tg1 := j.TaskGroups[0] + tg2 := tg1.Copy() + tg2.Name = "foo" + j.TaskGroups = append(j.TaskGroups, tg2) + if err := state.UpsertJob(1, j); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a deployment + d := mock.Deployment() + d.JobID = j.ID + d.TaskGroups = map[string]*structs.DeploymentState{ + "web": &structs.DeploymentState{ + DesiredTotal: 10, + DesiredCanaries: 1, + }, + "foo": &structs.DeploymentState{ + DesiredTotal: 10, + DesiredCanaries: 1, + }, + } + if err := state.UpsertDeployment(2, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a set of allocations + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + c1.Canary = true + c1.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + c2.Canary = true + c2.TaskGroup = tg2.Name + c2.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + + if err := state.UpsertAllocs(3, []*structs.Allocation{c1, c2}); err != nil { + t.Fatalf("err: %v", err) + } + + // Create an eval + e := mock.Eval() + + // Promote the canaries + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + Eval: e, + } + buf, err := structs.Encode(structs.DeploymentPromoteRequestType, req) + if err != nil { + t.Fatalf("err: %v", err) + } + resp := fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Check that the status per task group was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if len(dout.TaskGroups) != 2 { + t.Fatalf("bad: %#v", dout.TaskGroups) + } + for tg, state := range dout.TaskGroups { + if !state.Promoted { + t.Fatalf("bad: group %q not promoted %#v", tg, state) + } + } + + // Check that the allocs were promoted + out1, err := state.AllocByID(ws, c1.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + out2, err := state.AllocByID(ws, c2.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + for _, alloc := range []*structs.Allocation{out1, out2} { + if alloc.DeploymentStatus == nil { + t.Fatalf("bad: alloc %q has nil deployment status", alloc.ID) + } + if !alloc.DeploymentStatus.Promoted { + t.Fatalf("bad: alloc %q not promoted", alloc.ID) + } + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } + + // Assert the eval was enqueued + stats := fsm.evalBroker.Stats() + if stats.TotalReady != 1 { + t.Fatalf("bad: %#v %#v", stats, e) + } +} + +func TestFSM_DeploymentAllocHealth(t *testing.T) { + fsm := testFSM(t) + fsm.evalBroker.SetEnabled(true) + state := fsm.State() + + // Insert a deployment + d := mock.Deployment() + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + // Insert two allocations + a1 := mock.Alloc() + a1.DeploymentID = d.ID + a2 := mock.Alloc() + a2.DeploymentID = d.ID + if err := state.UpsertAllocs(2, []*structs.Allocation{a1, a2}); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a job to roll back to + j := mock.Job() + + // Create an eval that should be upserted + e := mock.Eval() + + // Create a status update for the deployment + status, desc := structs.DeploymentStatusFailed, "foo" + u := &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: status, + StatusDescription: desc, + } + + // Set health against the deployment + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a1.ID}, + UnhealthyAllocationIDs: []string{a2.ID}, + }, + Job: j, + Eval: e, + DeploymentUpdate: u, + } + buf, err := structs.Encode(structs.DeploymentAllocHealthRequestType, req) + if err != nil { + t.Fatalf("err: %v", err) + } + resp := fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Check that the status was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if dout.Status != status || dout.StatusDescription != desc { + t.Fatalf("bad: %#v", dout) + } + + // Check that the evaluation was created + eout, _ := state.EvalByID(ws, e.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if eout == nil { + t.Fatalf("bad: %#v", eout) + } + + // Check that the job was created + jout, _ := state.JobByID(ws, j.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } + + // Check the status of the allocs + out1, err := state.AllocByID(ws, a1.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + out2, err := state.AllocByID(ws, a2.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + if !out1.DeploymentStatus.IsHealthy() { + t.Fatalf("bad: alloc %q not healthy", out1.ID) + } + if !out2.DeploymentStatus.IsUnhealthy() { + t.Fatalf("bad: alloc %q not unhealthy", out2.ID) + } + + // Assert the eval was enqueued + stats := fsm.evalBroker.Stats() + if stats.TotalReady != 1 { + t.Fatalf("bad: %#v %#v", stats, e) + } +} + func testSnapshotRestore(t *testing.T, fsm *nomadFSM) *nomadFSM { // Snapshot snap, err := fsm.Snapshot() @@ -1484,96 +1875,3 @@ func TestFSM_ReconcileSummaries(t *testing.T) { t.Fatalf("expected: %#v, actual: %#v", &expected, out2) } } - -func TestFSM_ApplyPlanResults(t *testing.T) { - fsm := testFSM(t) - - // Create the request and create a deployment - alloc := mock.Alloc() - job := alloc.Job - alloc.Job = nil - - d := mock.Deployment() - d.JobID = job.ID - d.JobModifyIndex = job.ModifyIndex - d.JobVersion = job.Version - - alloc.DeploymentID = d.ID - - fsm.State().UpsertJobSummary(1, mock.JobSummary(alloc.JobID)) - req := structs.ApplyPlanResultsRequest{ - AllocUpdateRequest: structs.AllocUpdateRequest{ - Job: job, - Alloc: []*structs.Allocation{alloc}, - }, - CreatedDeployment: d, - } - buf, err := structs.Encode(structs.ApplyPlanResultsRequestType, req) - if err != nil { - t.Fatalf("err: %v", err) - } - - resp := fsm.Apply(makeLog(buf)) - if resp != nil { - t.Fatalf("resp: %v", resp) - } - - // Verify the allocation is registered - ws := memdb.NewWatchSet() - out, err := fsm.State().AllocByID(ws, alloc.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - alloc.CreateIndex = out.CreateIndex - alloc.ModifyIndex = out.ModifyIndex - alloc.AllocModifyIndex = out.AllocModifyIndex - - // Job should be re-attached - alloc.Job = job - if !reflect.DeepEqual(alloc, out) { - t.Fatalf("bad: %#v %#v", alloc, out) - } - - dout, err := fsm.State().DeploymentByID(ws, d.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - if tg, ok := dout.TaskGroups[alloc.TaskGroup]; !ok || tg.PlacedAllocs != 1 { - t.Fatalf("err: %v %v", tg, err) - } - - // Ensure that the original job is used - evictAlloc := alloc.Copy() - job = mock.Job() - job.Priority = 123 - - evictAlloc.Job = nil - evictAlloc.DesiredStatus = structs.AllocDesiredStatusEvict - req2 := structs.ApplyPlanResultsRequest{ - AllocUpdateRequest: structs.AllocUpdateRequest{ - Job: job, - Alloc: []*structs.Allocation{evictAlloc}, - }, - } - buf, err = structs.Encode(structs.ApplyPlanResultsRequestType, req2) - if err != nil { - t.Fatalf("err: %v", err) - } - - resp = fsm.Apply(makeLog(buf)) - if resp != nil { - t.Fatalf("resp: %v", resp) - } - - // Verify we are evicted - out, err = fsm.State().AllocByID(ws, alloc.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - if out.DesiredStatus != structs.AllocDesiredStatusEvict { - t.Fatalf("alloc found!") - } - if out.Job == nil || out.Job.Priority == 123 { - t.Fatalf("bad job") - } -} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 81d42cc8fe2b..fdde1e2c4a8e 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -56,7 +56,7 @@ const ( ApplyPlanResultsRequestType DeploymentStatusUpdateRequestType DeploymentPromoteRequestType - DeploymentAllocHealtRequestType + DeploymentAllocHealthRequestType ) const ( From 7d9d85b4e5affca80718871dbd1c56fa210005fc Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 27 Jun 2017 11:15:07 -0700 Subject: [PATCH 025/105] Deployments list --- nomad/deployment_endpoint.go | 50 ++++++++++++ nomad/deployment_endpoint_test.go | 121 ++++++++++++++++++++++++++++++ nomad/state/state_store.go | 13 ++++ nomad/state/state_store_test.go | 86 +++++++++++++++++++++ nomad/structs/structs.go | 11 +++ 5 files changed, 281 insertions(+) diff --git a/nomad/deployment_endpoint.go b/nomad/deployment_endpoint.go index af7d82746950..07e985983747 100644 --- a/nomad/deployment_endpoint.go +++ b/nomad/deployment_endpoint.go @@ -14,6 +14,56 @@ type Deployment struct { srv *Server } +// TODO http endpoint and api +// List returns the list of deployments in the system +func (d *Deployment) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { + if done, err := d.srv.forward("Deployment.List", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "List"}, time.Now()) + + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + run: func(ws memdb.WatchSet, state *state.StateStore) error { + // Capture all the deployments + var err error + var iter memdb.ResultIterator + if prefix := args.QueryOptions.Prefix; prefix != "" { + iter, err = state.DeploymentsByIDPrefix(ws, prefix) + } else { + iter, err = state.Deployments(ws) + } + if err != nil { + return err + } + + var deploys []*structs.Deployment + for { + raw := iter.Next() + if raw == nil { + break + } + deploy := raw.(*structs.Deployment) + deploys = append(deploys, deploy) + } + reply.Deployments = deploys + + // Use the last index that affected the jobs table + index, err := state.Index("deployment") + if err != nil { + return err + } + reply.Index = index + + // Set the query response + d.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return d.srv.blockingRPC(&opts) +} + // TODO http endpoint and api // Allocations returns the list of allocations that are a part of the deployment func (d *Deployment) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index d03c752eb293..f249940cc37a 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -10,6 +10,127 @@ import ( "github.com/hashicorp/nomad/testutil" ) +func TestDeploymentEndpoint_List(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the register request + deployment := mock.Deployment() + state := s1.fsm.State() + + if err := state.UpsertDeployment(1000, deployment, false); err != nil { + t.Fatalf("err: %v", err) + } + + // Lookup the deployments + get := &structs.DeploymentListRequest{ + QueryOptions: structs.QueryOptions{Region: "global"}, + } + var resp structs.DeploymentListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + if resp.Index != 1000 { + t.Fatalf("Bad index: %d %d", resp.Index, 1000) + } + + if len(resp.Deployments) != 1 { + t.Fatalf("bad: %#v", resp.Deployments) + } + if resp.Deployments[0].ID != deployment.ID { + t.Fatalf("bad: %#v", resp.Deployments[0]) + } + + // Lookup the deploys by prefix + get = &structs.DeploymentListRequest{ + QueryOptions: structs.QueryOptions{Region: "global", Prefix: deployment.ID[:4]}, + } + + var resp2 structs.DeploymentListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", get, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + if resp2.Index != 1000 { + t.Fatalf("Bad index: %d %d", resp2.Index, 1000) + } + + if len(resp2.Deployments) != 1 { + t.Fatalf("bad: %#v", resp2.Deployments) + } + if resp2.Deployments[0].ID != deployment.ID { + t.Fatalf("bad: %#v", resp2.Deployments[0]) + } +} + +func TestDeploymentEndpoint_List_Blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the deployment + deployment := mock.Deployment() + + // Upsert alloc triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertDeployment(3, deployment, false); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req := &structs.DeploymentListRequest{ + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.DeploymentListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 3 { + t.Fatalf("Bad index: %d %d", resp.Index, 3) + } + if len(resp.Deployments) != 1 || resp.Deployments[0].ID != deployment.ID { + t.Fatalf("bad: %#v", resp.Deployments) + } + + // Deployment updates trigger watches + deployment2 := deployment.Copy() + deployment2.Status = structs.DeploymentStatusPaused + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertDeployment(5, deployment2, false); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 3 + start = time.Now() + var resp2 structs.DeploymentListResponse + if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) + } + if resp2.Index != 5 { + t.Fatalf("Bad index: %d %d", resp2.Index, 5) + } + if len(resp2.Deployments) != 1 || resp.Deployments[0].ID != deployment2.ID || + resp2.Deployments[0].Status != structs.DeploymentStatusPaused { + t.Fatalf("bad: %#v", resp2.Deployments) + } +} + func TestDeploymentEndpoint_Allocations(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 41668ee6836f..fda10de4767c 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -312,6 +312,19 @@ func (s *StateStore) Deployments(ws memdb.WatchSet) (memdb.ResultIterator, error return iter, nil } +func (s *StateStore) DeploymentsByIDPrefix(ws memdb.WatchSet, deploymentID string) (memdb.ResultIterator, error) { + txn := s.db.Txn(false) + + // Walk the entire deployments table + iter, err := txn.Get("deployment", "id_prefix", deploymentID) + if err != nil { + return nil, err + } + + ws.Add(iter.WatchCh()) + return iter, nil +} + func (s *StateStore) DeploymentByID(ws memdb.WatchSet, deploymentID string) (*structs.Deployment, error) { txn := s.db.Txn(false) return s.deploymentByIDImpl(ws, deploymentID, txn) diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index aac9f5a65713..c2134c298020 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -511,6 +511,92 @@ func TestStateStore_Deployments(t *testing.T) { } } +func TestStateStore_DeploymentsByIDPrefix(t *testing.T) { + state := testStateStore(t) + deploy := mock.Deployment() + + deploy.ID = "11111111-662e-d0ab-d1c9-3e434af7bdb4" + err := state.UpsertDeployment(1000, deploy, false) + if err != nil { + t.Fatalf("err: %v", err) + } + + // Create a watchset so we can test that getters don't cause it to fire + ws := memdb.NewWatchSet() + iter, err := state.DeploymentsByIDPrefix(ws, deploy.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + gatherDeploys := func(iter memdb.ResultIterator) []*structs.Deployment { + var deploys []*structs.Deployment + for { + raw := iter.Next() + if raw == nil { + break + } + deploy := raw.(*structs.Deployment) + deploys = append(deploys, deploy) + } + return deploys + } + + deploys := gatherDeploys(iter) + if len(deploys) != 1 { + t.Fatalf("err: %v", err) + } + + if watchFired(ws) { + t.Fatalf("bad") + } + + iter, err = state.DeploymentsByIDPrefix(ws, "11") + if err != nil { + t.Fatalf("err: %v", err) + } + + deploys = gatherDeploys(iter) + if len(deploys) != 1 { + t.Fatalf("err: %v", err) + } + + deploy = mock.Deployment() + deploy.ID = "11222222-662e-d0ab-d1c9-3e434af7bdb4" + err = state.UpsertDeployment(1001, deploy, false) + if err != nil { + t.Fatalf("err: %v", err) + } + + if !watchFired(ws) { + t.Fatalf("bad") + } + + ws = memdb.NewWatchSet() + iter, err = state.DeploymentsByIDPrefix(ws, "11") + if err != nil { + t.Fatalf("err: %v", err) + } + + deploys = gatherDeploys(iter) + if len(deploys) != 2 { + t.Fatalf("err: %v", err) + } + + iter, err = state.DeploymentsByIDPrefix(ws, "1111") + if err != nil { + t.Fatalf("err: %v", err) + } + + deploys = gatherDeploys(iter) + if len(deploys) != 1 { + t.Fatalf("err: %v", err) + } + + if watchFired(ws) { + t.Fatalf("bad") + } +} + func TestStateStore_UpsertNode_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index fdde1e2c4a8e..09e19b2d7e6b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -492,6 +492,11 @@ type GenericRequest struct { QueryOptions } +// DeploymentListRequest is used to list the deployments +type DeploymentListRequest struct { + QueryOptions +} + // DeploymentStatusUpdateRequest is used to update the status of a deployment as // well as optionally creating an evaluation atomically. type DeploymentStatusUpdateRequest struct { @@ -791,6 +796,12 @@ type AllocListResponse struct { QueryMeta } +// DeploymentListResponse is used for a list request +type DeploymentListResponse struct { + Deployments []*Deployment + QueryMeta +} + // EvalListResponse is used for a list request type EvalListResponse struct { Evaluations []*Evaluation From 05c710ddfb2984940f9ca6dafb3b8d3f5efd38fd Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 27 Jun 2017 11:52:14 -0700 Subject: [PATCH 026/105] Watcher --- .../deploymentwatcher/deployments_watcher.go | 97 +++++++++++++++++-- nomad/structs/structs.go | 2 +- 2 files changed, 91 insertions(+), 8 deletions(-) diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index 15e3fd1f94f6..bd911d9f6e46 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -39,9 +39,15 @@ type DeploymentStateWatchers interface { // deployment. Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error + // List is used to list all the deployments in the system + List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error + // GetJobVersions is used to lookup the versions of a job. This is used when // rolling back to find the latest stable job GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error + + // GetJob is used to lookup a particular job. + GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error } // Watcher is used to watch deployments and their allocations created @@ -89,10 +95,14 @@ func NewDeploymentsWatcher(logger *log.Logger, w DeploymentStateWatchers, raft D // should only be enabled on the active leader. func (w *Watcher) SetEnabled(enabled bool) { w.l.Lock() + wasEnabled := w.enabled w.enabled = enabled w.l.Unlock() if !enabled { w.Flush() + } else if !wasEnabled { + // Start the watcher if we are transistioning to an enabled state + go w.watchDeployments() } } @@ -113,27 +123,100 @@ func (w *Watcher) Flush() { w.evalBatcher = NewEvalBatcher(w.raft, w.exitCh) } -// Watch adds a deployment to the watch list -func (w *Watcher) Watch(d *structs.Deployment, j *structs.Job) { +// watchDeployments is the long lived go-routine that watches for deployments to +// add and remove watchers on. +func (w *Watcher) watchDeployments() { + dindex := uint64(0) + for { + // Block getting all deployments using the last deployment index. + var resp *structs.DeploymentListResponse + select { + case <-w.exitCh: + return + case resp = <-w.getDeploys(dindex): + } + + // Guard against npe + if resp == nil { + continue + } + + // Ensure we are tracking the things we should and not tracking what we + // shouldn't be + for _, d := range resp.Deployments { + if d.Active() { + if err := w.add(d); err != nil { + w.logger.Printf("[ERR] nomad.deployments_watcher: failed to track deployment %q: %v", d.ID, err) + } + } else { + w.remove(d) + } + } + + // Update the latest index + dindex = resp.Index + } +} + +// getDeploys retrieves all deployments blocking at the given index. +func (w *Watcher) getDeploys(index uint64) <-chan *structs.DeploymentListResponse { + c := make(chan *structs.DeploymentListResponse, 1) + go func() { + // Build the request + args := &structs.DeploymentListRequest{ + QueryOptions: structs.QueryOptions{ + MinQueryIndex: index, + }, + } + var resp structs.DeploymentListResponse + + for resp.Index <= index { + if err := w.stateWatchers.List(args, &resp); err != nil { + w.logger.Printf("[ERR] nomad.deployments_watcher: failed to retrieve deployments: %v", err) + close(c) + return + } + } + + c <- &resp + }() + return c +} + +// add adds a deployment to the watch list +func (w *Watcher) add(d *structs.Deployment) error { w.l.Lock() defer w.l.Unlock() // Not enabled so no-op if !w.enabled { - return + return nil } // Already watched so no-op if _, ok := w.watchers[d.ID]; ok { - return + return nil + } + + // Get the job the deployment is referencing + args := &structs.JobSpecificRequest{ + JobID: d.JobID, + } + var resp structs.SingleJobResponse + if err := w.stateWatchers.GetJob(args, &resp); err != nil { + return err + } + if resp.Job == nil { + return fmt.Errorf("deployment %q references unknown job %q", d.ID, d.JobID) } - w.watchers[d.ID] = newDeploymentWatcher(w.logger, w.stateWatchers, d, j, w) + w.watchers[d.ID] = newDeploymentWatcher(w.logger, w.stateWatchers, d, resp.Job, w) + return nil } -// Unwatch stops watching a deployment. This can be because the deployment is +// remove stops watching a deployment. This can be because the deployment is // complete or being deleted. -func (w *Watcher) Unwatch(d *structs.Deployment) { +func (w *Watcher) remove(d *structs.Deployment) { w.l.Lock() defer w.l.Unlock() diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 09e19b2d7e6b..7b26b838eb82 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3843,10 +3843,10 @@ func (v *Vault) Validate() error { const ( // DeploymentStatuses are the various states a deployment can be be in DeploymentStatusRunning = "running" + DeploymentStatusPaused = "paused" DeploymentStatusFailed = "failed" DeploymentStatusSuccessful = "successful" DeploymentStatusCancelled = "cancelled" - DeploymentStatusPaused = "paused" // DeploymentStatusDescriptions are the various descriptions of the states a // deployment can be in. From 7e43ed746f819a59acb6ac7e638c7c58c0565ae7 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 27 Jun 2017 21:36:16 -0700 Subject: [PATCH 027/105] Deployment watcher tests --- nomad/deploymentwatcher/batcher.go | 14 +- nomad/deploymentwatcher/deployment_watcher.go | 162 +-- .../deploymentwatcher/deployments_watcher.go | 109 +- .../deployments_watcher_test.go | 1000 +++++++++++++++++ nomad/state/state_store.go | 1 - vendor/vendor.json | 40 +- 6 files changed, 1197 insertions(+), 129 deletions(-) create mode 100644 nomad/deploymentwatcher/deployments_watcher_test.go diff --git a/nomad/deploymentwatcher/batcher.go b/nomad/deploymentwatcher/batcher.go index 4dde9bba1294..29df3af51c7e 100644 --- a/nomad/deploymentwatcher/batcher.go +++ b/nomad/deploymentwatcher/batcher.go @@ -1,6 +1,7 @@ package deploymentwatcher import ( + "context" "sync" "time" @@ -18,8 +19,8 @@ type EvalBatcher struct { // inCh is used to pass evaluations to the daemon process inCh chan *structs.Evaluation - // exitCh is used to exit the daemon batcher - exitCh chan struct{} + // ctx is used to exit the daemon batcher + ctx context.Context l sync.Mutex } @@ -27,10 +28,10 @@ type EvalBatcher struct { // NewEvalBatcher returns an EvalBatcher that uses the passed raft endpoints to // create the evaluations and exits the batcher when the passed exit channel is // closed. -func NewEvalBatcher(raft DeploymentRaftEndpoints, exitCh chan struct{}) *EvalBatcher { +func NewEvalBatcher(raft DeploymentRaftEndpoints, ctx context.Context) *EvalBatcher { b := &EvalBatcher{ - raft: raft, - exitCh: exitCh, + raft: raft, + ctx: ctx, } go b.batcher() @@ -57,7 +58,7 @@ func (b *EvalBatcher) batcher() { evals := make(map[string]*structs.Evaluation) for { select { - case <-b.exitCh: + case <-b.ctx.Done(): ticker.Stop() return case e := <-b.inCh: @@ -70,6 +71,7 @@ func (b *EvalBatcher) batcher() { // Capture the future b.l.Lock() f := b.f + b.f = nil b.l.Unlock() // Shouldn't be possible but protect ourselves diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 494a04e97666..da01a0c48234 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -1,11 +1,14 @@ package deploymentwatcher import ( + "context" "fmt" "log" "sync" "time" + "golang.org/x/time/rate" + "github.com/hashicorp/nomad/nomad/structs" ) @@ -39,6 +42,9 @@ type deploymentTriggers interface { // deploymentWatcher is used to watch a single deployment and trigger the // scheduler when allocation health transistions. type deploymentWatcher struct { + // queryLimiter is used to limit the rate of blocking queries + queryLimiter *rate.Limiter + // deploymentTriggers holds the methods required to trigger changes on behalf of the // deployment deploymentTriggers @@ -62,27 +68,33 @@ type deploymentWatcher struct { outstandingBatch bool logger *log.Logger - exitCh chan struct{} + ctx context.Context + exitFn context.CancelFunc l sync.RWMutex } // newDeploymentWatcher returns a deployment watcher that is used to watch // deployments and trigger the scheduler as needed. func newDeploymentWatcher( + parent context.Context, + queryLimiter *rate.Limiter, logger *log.Logger, watchers DeploymentStateWatchers, d *structs.Deployment, j *structs.Job, triggers deploymentTriggers) *deploymentWatcher { + ctx, exitFn := context.WithCancel(parent) w := &deploymentWatcher{ + queryLimiter: queryLimiter, d: d, j: j, autorevert: make(map[string]bool, len(j.TaskGroups)), DeploymentStateWatchers: watchers, deploymentTriggers: triggers, - exitCh: make(chan struct{}), logger: logger, + ctx: ctx, + exitFn: exitFn, } for _, tg := range j.TaskGroups { @@ -97,8 +109,9 @@ func newDeploymentWatcher( return w } -func (w *deploymentWatcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest) ( - *structs.DeploymentUpdateResponse, error) { +func (w *deploymentWatcher) SetAllocHealth( + req *structs.DeploymentAllocHealthRequest, + resp *structs.DeploymentUpdateResponse) error { // If we are failing the deployment, update the status and potentially // rollback @@ -117,7 +130,7 @@ func (w *deploymentWatcher) SetAllocHealth(req *structs.DeploymentAllocHealthReq args := &structs.DeploymentSpecificRequest{DeploymentID: req.DeploymentID} var resp structs.AllocListResponse if err := w.Allocations(args, &resp); err != nil { - return nil, err + return err } desc := structs.DeploymentStatusDescriptionFailedAllocations @@ -135,7 +148,7 @@ func (w *deploymentWatcher) SetAllocHealth(req *structs.DeploymentAllocHealthReq var err error j, err = w.latestStableJob() if err != nil { - return nil, err + return err } desc = fmt.Sprintf("%s - rolling back to job version %d", desc, j.Version) @@ -155,18 +168,19 @@ func (w *deploymentWatcher) SetAllocHealth(req *structs.DeploymentAllocHealthReq index, err := w.upsertDeploymentAllocHealth(areq) if err != nil { - return nil, err + return err } - return &structs.DeploymentUpdateResponse{ - EvalID: areq.Eval.ID, - EvalCreateIndex: index, - DeploymentModifyIndex: index, - }, nil + // Build the response + resp.EvalID = areq.Eval.ID + resp.EvalCreateIndex = index + resp.DeploymentModifyIndex = index + return nil } -func (w *deploymentWatcher) PromoteDeployment(req *structs.DeploymentPromoteRequest) ( - *structs.DeploymentUpdateResponse, error) { +func (w *deploymentWatcher) PromoteDeployment( + req *structs.DeploymentPromoteRequest, + resp *structs.DeploymentUpdateResponse) error { // Create the request areq := &structs.ApplyDeploymentPromoteRequest{ @@ -176,18 +190,19 @@ func (w *deploymentWatcher) PromoteDeployment(req *structs.DeploymentPromoteRequ index, err := w.upsertDeploymentPromotion(areq) if err != nil { - return nil, err + return err } - return &structs.DeploymentUpdateResponse{ - EvalID: areq.Eval.ID, - EvalCreateIndex: index, - DeploymentModifyIndex: index, - }, nil + // Build the response + resp.EvalID = areq.Eval.ID + resp.EvalCreateIndex = index + resp.DeploymentModifyIndex = index + return nil } -func (w *deploymentWatcher) PauseDeployment(req *structs.DeploymentPauseRequest) ( - *structs.DeploymentUpdateResponse, error) { +func (w *deploymentWatcher) PauseDeployment( + req *structs.DeploymentPauseRequest, + resp *structs.DeploymentUpdateResponse) error { // Determine the status we should transistion to and if we need to create an // evaluation status, desc := structs.DeploymentStatusPaused, structs.DeploymentStatusDescriptionPaused @@ -195,7 +210,7 @@ func (w *deploymentWatcher) PauseDeployment(req *structs.DeploymentPauseRequest) evalID := "" if !req.Pause { status, desc = structs.DeploymentStatusRunning, structs.DeploymentStatusDescriptionRunning - eval := w.getEval() + eval = w.getEval() evalID = eval.ID } update := w.getDeploymentStatusUpdate(status, desc) @@ -203,22 +218,20 @@ func (w *deploymentWatcher) PauseDeployment(req *structs.DeploymentPauseRequest) // Commit the change i, err := w.upsertDeploymentStatusUpdate(update, eval, nil) if err != nil { - return nil, err + return err } - return &structs.DeploymentUpdateResponse{ - EvalID: evalID, - EvalCreateIndex: i, - DeploymentModifyIndex: i, - }, nil + // Build the response + resp.EvalID = evalID + resp.EvalCreateIndex = i + resp.DeploymentModifyIndex = i + return nil } // StopWatch stops watching the deployment. This should be called whenever a // deployment is completed or the watcher is no longer needed. func (w *deploymentWatcher) StopWatch() { - w.l.Lock() - defer w.l.Unlock() - close(w.exitCh) + w.exitFn() } // watch is the long running watcher that takes actions upon allocation changes @@ -228,27 +241,35 @@ func (w *deploymentWatcher) watch() { // Block getting all allocations that are part of the deployment using // the last evaluation index. This will have us block waiting for // something to change past what the scheduler has evaluated. - var allocs []*structs.AllocListStub - select { - case <-w.exitCh: - return - case allocs = <-w.getAllocs(latestEval): + allocs, err := w.getAllocs(latestEval) + if err != nil { + if err == context.Canceled { + return + } + + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to retrieve allocations for deployment %q: %v", w.d.ID, err) } // Get the latest evaluation snapshot index - latestEval = w.latestEvalIndex() + latestEval, err = w.latestEvalIndex() + if err != nil { + if err == context.Canceled { + return + } + + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to determine last evaluation index for job %q: %v", w.d.JobID, err) + } // Create an evaluation trigger if there is any allocation whose // deployment status has been updated past the latest eval index. createEval, failDeployment, rollback := false, false, false for _, alloc := range allocs { - if alloc.DeploymentStatus == nil { + if alloc.DeploymentStatus == nil || alloc.DeploymentStatus.ModifyIndex <= latestEval { continue } - if alloc.DeploymentStatus.ModifyIndex > latestEval { - createEval = true - } + // We need to create an eval + createEval = true if alloc.DeploymentStatus.IsUnhealthy() { // Check if the group has autorevert set @@ -373,48 +394,55 @@ func (w *deploymentWatcher) getDeploymentStatusUpdate(status, desc string) *stru // getAllocs retrieves the allocations that are part of the deployment blocking // at the given index. -func (w *deploymentWatcher) getAllocs(index uint64) <-chan []*structs.AllocListStub { - c := make(chan []*structs.AllocListStub, 1) - go func() { - // Build the request - args := &structs.DeploymentSpecificRequest{ - DeploymentID: w.d.ID, - QueryOptions: structs.QueryOptions{ - MinQueryIndex: index, - }, +func (w *deploymentWatcher) getAllocs(index uint64) ([]*structs.AllocListStub, error) { + // Build the request + args := &structs.DeploymentSpecificRequest{ + DeploymentID: w.d.ID, + QueryOptions: structs.QueryOptions{ + MinQueryIndex: index, + }, + } + var resp structs.AllocListResponse + + for resp.Index <= index { + if err := w.queryLimiter.Wait(w.ctx); err != nil { + return nil, err } - var resp structs.AllocListResponse - for resp.Index <= index { - if err := w.Allocations(args, &resp); err != nil { - w.logger.Printf("[ERR] nomad.deployment_watcher: failed to retrieve allocations as part of deployment %q: %v", w.d.ID, err) - close(c) - return - } + if err := w.Allocations(args, &resp); err != nil { + return nil, err } + } - c <- resp.Allocations - }() - return c + return resp.Allocations, nil } -// latestEvalIndex returns the snapshot index of the last evaluation created for +// latestEvalIndex returns the index of the last evaluation created for // the job. The index is used to determine if an allocation update requires an // evaluation to be triggered. -func (w *deploymentWatcher) latestEvalIndex() uint64 { +func (w *deploymentWatcher) latestEvalIndex() (uint64, error) { + if err := w.queryLimiter.Wait(w.ctx); err != nil { + return 0, err + } + args := &structs.JobSpecificRequest{ JobID: w.d.JobID, } var resp structs.JobEvaluationsResponse err := w.Evaluations(args, &resp) if err != nil { - w.logger.Printf("[ERR] nomad.deployment_watcher: failed to determine last evaluation index for job %q: %v", w.d.JobID, err) - return 0 + return 0, err } if len(resp.Evaluations) == 0 { - return 0 + return resp.Index, nil + } + + // Prefer using the snapshot index. Otherwise use the create index + e := resp.Evaluations[0] + if e.SnapshotIndex != 0 { + return e.SnapshotIndex, nil } - return resp.Evaluations[0].SnapshotIndex + return e.CreateIndex, nil } diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index bd911d9f6e46..fb1880a52e68 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -1,10 +1,13 @@ package deploymentwatcher import ( + "context" "fmt" "log" "sync" + "golang.org/x/time/rate" + "github.com/hashicorp/nomad/nomad/structs" ) @@ -50,6 +53,12 @@ type DeploymentStateWatchers interface { GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error } +const ( + // limitStateQueriesPerSecond is the number of state queries allowed per + // second + limitStateQueriesPerSecond = 10.0 +) + // Watcher is used to watch deployments and their allocations created // by the scheduler and trigger the scheduler when allocation health // transistions. @@ -57,6 +66,9 @@ type Watcher struct { enabled bool logger *log.Logger + // queryLimiter is used to limit the rate of blocking queries + queryLimiter *rate.Limiter + // raft contains the set of Raft endpoints that can be used by the // deployments watcher raft DeploymentRaftEndpoints @@ -71,8 +83,9 @@ type Watcher struct { // evalBatcher is used to batch the creation of evaluations evalBatcher *EvalBatcher - // exitCh is used to exit any goroutines spawned by the watcher - exitCh chan struct{} + // ctx and exitFn are used to cancel the watcher + ctx context.Context + exitFn context.CancelFunc l sync.RWMutex } @@ -80,14 +93,16 @@ type Watcher struct { // NewDeploymentsWatcher returns a deployments watcher that is used to watch // deployments and trigger the scheduler as needed. func NewDeploymentsWatcher(logger *log.Logger, w DeploymentStateWatchers, raft DeploymentRaftEndpoints) *Watcher { - exitCh := make(chan struct{}) + ctx, exitFn := context.WithCancel(context.Background()) return &Watcher{ + queryLimiter: rate.NewLimiter(limitStateQueriesPerSecond, 100), stateWatchers: w, raft: raft, watchers: make(map[string]*deploymentWatcher, 32), - evalBatcher: NewEvalBatcher(raft, exitCh), - exitCh: exitCh, + evalBatcher: NewEvalBatcher(raft, ctx), logger: logger, + ctx: ctx, + exitFn: exitFn, } } @@ -116,11 +131,12 @@ func (w *Watcher) Flush() { watcher.StopWatch() } - close(w.exitCh) + // Kill everything associated with the watcher + w.exitFn() w.watchers = make(map[string]*deploymentWatcher, 32) - w.exitCh = make(chan struct{}) - w.evalBatcher = NewEvalBatcher(w.raft, w.exitCh) + w.ctx, w.exitFn = context.WithCancel(context.Background()) + w.evalBatcher = NewEvalBatcher(w.raft, w.ctx) } // watchDeployments is the long lived go-routine that watches for deployments to @@ -129,11 +145,13 @@ func (w *Watcher) watchDeployments() { dindex := uint64(0) for { // Block getting all deployments using the last deployment index. - var resp *structs.DeploymentListResponse - select { - case <-w.exitCh: - return - case resp = <-w.getDeploys(dindex): + resp, err := w.getDeploys(dindex) + if err != nil { + if err == context.Canceled { + return + } + + w.logger.Printf("[ERR] nomad.deployments_watcher: failed to retrieve deploylements: %v", err) } // Guard against npe @@ -159,28 +177,26 @@ func (w *Watcher) watchDeployments() { } // getDeploys retrieves all deployments blocking at the given index. -func (w *Watcher) getDeploys(index uint64) <-chan *structs.DeploymentListResponse { - c := make(chan *structs.DeploymentListResponse, 1) - go func() { - // Build the request - args := &structs.DeploymentListRequest{ - QueryOptions: structs.QueryOptions{ - MinQueryIndex: index, - }, +func (w *Watcher) getDeploys(index uint64) (*structs.DeploymentListResponse, error) { + // Build the request + args := &structs.DeploymentListRequest{ + QueryOptions: structs.QueryOptions{ + MinQueryIndex: index, + }, + } + var resp structs.DeploymentListResponse + + for resp.Index <= index { + if err := w.queryLimiter.Wait(w.ctx); err != nil { + return nil, err } - var resp structs.DeploymentListResponse - for resp.Index <= index { - if err := w.stateWatchers.List(args, &resp); err != nil { - w.logger.Printf("[ERR] nomad.deployments_watcher: failed to retrieve deployments: %v", err) - close(c) - return - } + if err := w.stateWatchers.List(args, &resp); err != nil { + return nil, err } + } - c <- &resp - }() - return c + return &resp, nil } // add adds a deployment to the watch list @@ -210,7 +226,8 @@ func (w *Watcher) add(d *structs.Deployment) error { return fmt.Errorf("deployment %q references unknown job %q", d.ID, d.JobID) } - w.watchers[d.ID] = newDeploymentWatcher(w.logger, w.stateWatchers, d, resp.Job, w) + w.watchers[d.ID] = newDeploymentWatcher(w.ctx, w.queryLimiter, w.logger, w.stateWatchers, d, resp.Job, w) + w.logger.Printf("[TRACE] nomad.deployments_watcher: tracking deployment %q", d.ID) return nil } @@ -228,69 +245,67 @@ func (w *Watcher) remove(d *structs.Deployment) { if watcher, ok := w.watchers[d.ID]; ok { watcher.StopWatch() delete(w.watchers, d.ID) + w.logger.Printf("[TRACE] nomad.deployments_watcher: untracking deployment %q", d.ID) } } // SetAllocHealth is used to set the health of allocations for a deployment. If // there are any unhealthy allocations, the deployment is updated to be failed. // Otherwise the allocations are updated and an evaluation is created. -func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest) ( - *structs.DeploymentUpdateResponse, error) { +func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest, resp *structs.DeploymentUpdateResponse) error { w.l.Lock() defer w.l.Unlock() // Not enabled so no-op if !w.enabled { - return nil, nil + return nil } watcher, ok := w.watchers[req.DeploymentID] if !ok { - return nil, fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) } - return watcher.SetAllocHealth(req) + return watcher.SetAllocHealth(req, resp) } // PromoteDeployment is used to promote a deployment. If promote is false, // deployment is marked as failed. Otherwise the deployment is updated and an // evaluation is created. -func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest) ( - *structs.DeploymentUpdateResponse, error) { +func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest, resp *structs.DeploymentUpdateResponse) error { w.l.Lock() defer w.l.Unlock() // Not enabled so no-op if !w.enabled { - return nil, nil + return nil } watcher, ok := w.watchers[req.DeploymentID] if !ok { - return nil, fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) } - return watcher.PromoteDeployment(req) + return watcher.PromoteDeployment(req, resp) } // PauseDeployment is used to toggle the pause state on a deployment. If the // deployment is being unpaused, an evaluation is created. -func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest) ( - *structs.DeploymentUpdateResponse, error) { +func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest, resp *structs.DeploymentUpdateResponse) error { w.l.Lock() defer w.l.Unlock() // Not enabled so no-op if !w.enabled { - return nil, nil + return nil } watcher, ok := w.watchers[req.DeploymentID] if !ok { - return nil, fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) } - return watcher.PauseDeployment(req) + return watcher.PauseDeployment(req, resp) } // createEvaluation commits the given evaluation to Raft but batches the commit diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go new file mode 100644 index 000000000000..2988c624ddf7 --- /dev/null +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -0,0 +1,1000 @@ +package deploymentwatcher + +import ( + "log" + "os" + "reflect" + "strings" + "sync" + "testing" + "time" + + memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/state" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/testutil" + "github.com/stretchr/testify/assert" + mocker "github.com/stretchr/testify/mock" +) + +// TODO +// Test evaluations are batched between watchers +// Test allocation watcher +// Test that evaluation due to allocation changes are batched + +func testLogger() *log.Logger { + return log.New(os.Stderr, "", log.LstdFlags) +} + +type mockBackend struct { + mocker.Mock + index uint64 + state *state.StateStore + l sync.Mutex +} + +func newMockBackend(t *testing.T) *mockBackend { + state, err := state.NewStateStore(os.Stderr) + if err != nil { + t.Fatalf("err: %v", err) + } + if state == nil { + t.Fatalf("missing state") + } + return &mockBackend{ + index: 10000, + state: state, + } +} + +func (m *mockBackend) nextIndex() uint64 { + m.l.Lock() + defer m.l.Unlock() + i := m.index + m.index++ + return i +} + +func (m *mockBackend) UpsertEvals(evals []*structs.Evaluation) (uint64, error) { + m.Called(evals) + i := m.nextIndex() + return i, m.state.UpsertEvals(i, evals) +} + +func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) { + m.Called(job) + i := m.nextIndex() + return i, m.state.UpsertJob(i, job) +} + +func (m *mockBackend) UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { + m.Called(u) + i := m.nextIndex() + return i, m.state.UpsertDeploymentStatusUpdate(i, u) +} + +// matchDeploymentStatusUpdateConfig is used to configure the matching +// function +type matchDeploymentStatusUpdateConfig struct { + // DeploymentID is the expected ID + DeploymentID string + + // Status is the desired status + Status string + + // StatusDescription is the desired status description + StatusDescription string + + // JobVersion marks whether we expect a roll back job at the given version + JobVersion *uint64 + + // Eval marks whether we expect an evaluation. + Eval bool +} + +// matchDeploymentStatusUpdateRequest is used to match an update request +func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) func(args *structs.DeploymentStatusUpdateRequest) bool { + return func(args *structs.DeploymentStatusUpdateRequest) bool { + if args.DeploymentUpdate.DeploymentID != c.DeploymentID { + return false + } + + if args.DeploymentUpdate.Status != c.Status && args.DeploymentUpdate.StatusDescription != c.StatusDescription { + return false + } + + if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { + return false + } + + if (c.JobVersion != nil && (args.Job == nil || args.Job.Version != *c.JobVersion)) || c.JobVersion == nil && args.Job != nil { + return false + } + + return true + } +} + +func (m *mockBackend) UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { + m.Called(req) + i := m.nextIndex() + return i, m.state.UpsertDeploymentPromotion(i, req) +} + +// matchDeploymentPromoteRequestConfig is used to configure the matching +// function +type matchDeploymentPromoteRequestConfig struct { + // Promotion holds the expected promote request + Promotion *structs.DeploymentPromoteRequest + + // Eval marks whether we expect an evaluation. + Eval bool +} + +// matchDeploymentPromoteRequest is used to match a promote request +func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func(args *structs.ApplyDeploymentPromoteRequest) bool { + return func(args *structs.ApplyDeploymentPromoteRequest) bool { + if !reflect.DeepEqual(*c.Promotion, args.DeploymentPromoteRequest) { + return false + } + + if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { + return false + } + + return true + } +} +func (m *mockBackend) UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { + m.Called(req) + i := m.nextIndex() + return i, m.state.UpsertDeploymentAllocHealth(i, req) +} + +// matchDeploymentAllocHealthRequestConfig is used to configure the matching +// function +type matchDeploymentAllocHealthRequestConfig struct { + // DeploymentID is the expected ID + DeploymentID string + + // Healthy and Unhealthy contain the expected allocation IDs that are having + // their health set + Healthy, Unhealthy []string + + // DeploymentUpdate holds the expected values of status and description. We + // don't check for exact match but string contains + DeploymentUpdate *structs.DeploymentStatusUpdate + + // JobVersion marks whether we expect a roll back job at the given version + JobVersion *uint64 + + // Eval marks whether we expect an evaluation. + Eval bool +} + +// matchDeploymentAllocHealthRequest is used to match an update request +func matchDeploymentAllocHealthRequest(c *matchDeploymentAllocHealthRequestConfig) func(args *structs.ApplyDeploymentAllocHealthRequest) bool { + return func(args *structs.ApplyDeploymentAllocHealthRequest) bool { + if args.DeploymentID != c.DeploymentID { + return false + } + + if len(c.Healthy) != len(args.HealthyAllocationIDs) { + return false + } + if len(c.Unhealthy) != len(args.UnhealthyAllocationIDs) { + return false + } + + hmap, umap := make(map[string]struct{}, len(c.Healthy)), make(map[string]struct{}, len(c.Unhealthy)) + for _, h := range c.Healthy { + hmap[h] = struct{}{} + } + for _, u := range c.Unhealthy { + umap[u] = struct{}{} + } + + for _, h := range args.HealthyAllocationIDs { + if _, ok := hmap[h]; !ok { + return false + } + } + for _, u := range args.UnhealthyAllocationIDs { + if _, ok := umap[u]; !ok { + return false + } + } + + if c.DeploymentUpdate != nil { + if args.DeploymentUpdate == nil { + return false + } + + if !strings.Contains(args.DeploymentUpdate.Status, c.DeploymentUpdate.Status) { + return false + } + if !strings.Contains(args.DeploymentUpdate.StatusDescription, c.DeploymentUpdate.StatusDescription) { + return false + } + } else if args.DeploymentUpdate != nil { + return false + } + + if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { + return false + } + + if (c.JobVersion != nil && (args.Job == nil || args.Job.Version != *c.JobVersion)) || c.JobVersion == nil && args.Job != nil { + return false + } + + return true + } +} + +func (m *mockBackend) Evaluations(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) evaluationsFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobEvaluationsResponse) + ws := memdb.NewWatchSet() + evals, _ := m.state.EvalsByJob(ws, args.JobID) + reply.Evaluations = evals + reply.Index = m.nextIndex() +} + +func (m *mockBackend) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) allocationsFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.DeploymentSpecificRequest), in.Get(1).(*structs.AllocListResponse) + ws := memdb.NewWatchSet() + allocs, _ := m.state.AllocsByDeployment(ws, args.DeploymentID) + + var stubs []*structs.AllocListStub + for _, a := range allocs { + stubs = append(stubs, a.Stub()) + } + + reply.Allocations = stubs + reply.Index = m.nextIndex() +} + +func (m *mockBackend) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) listFromState(in mocker.Arguments) { + reply := in.Get(1).(*structs.DeploymentListResponse) + ws := memdb.NewWatchSet() + iter, _ := m.state.Deployments(ws) + + var deploys []*structs.Deployment + for { + raw := iter.Next() + if raw == nil { + break + } + + deploys = append(deploys, raw.(*structs.Deployment)) + } + + reply.Deployments = deploys + reply.Index = m.nextIndex() +} + +func (m *mockBackend) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) getJobVersionsFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobVersionsResponse) + ws := memdb.NewWatchSet() + versions, _ := m.state.JobVersionsByID(ws, args.JobID) + reply.Versions = versions + reply.Index = m.nextIndex() +} + +func (m *mockBackend) GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) getJobFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.SingleJobResponse) + ws := memdb.NewWatchSet() + job, _ := m.state.JobByID(ws, args.JobID) + reply.Job = job + reply.Index = m.nextIndex() +} + +// matchDeploymentSpecificRequest is used to match that a deployment specific +// request is for the passed deployment id +func matchDeploymentSpecificRequest(dID string) func(args *structs.DeploymentSpecificRequest) bool { + return func(args *structs.DeploymentSpecificRequest) bool { + return args.DeploymentID == dID + } +} + +// matchJobSpecificRequest is used to match that a job specific +// request is for the passed job id +func matchJobSpecificRequest(jID string) func(args *structs.JobSpecificRequest) bool { + return func(args *structs.JobSpecificRequest) bool { + return args.JobID == jID + } +} + +// Tests that the watcher properly watches for deployments and reconciles them +func TestWatcher_WatchDeployments(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Return no allocations or evals + m.On("Allocations", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.AllocListResponse) + reply.Index = m.nextIndex() + }) + m.On("Evaluations", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.JobEvaluationsResponse) + reply.Index = m.nextIndex() + }) + + // Create three jobs + j1, j2, j3 := mock.Job(), mock.Job(), mock.Job() + jobs := map[string]*structs.Job{ + j1.ID: j1, + j2.ID: j2, + j3.ID: j3, + } + + // Create three deployments all running + d1, d2, d3 := mock.Deployment(), mock.Deployment(), mock.Deployment() + d1.JobID = j1.ID + d2.JobID = j2.ID + d3.JobID = j3.ID + + m.On("GetJob", mocker.Anything, mocker.Anything). + Return(nil).Run(func(args mocker.Arguments) { + in := args.Get(0).(*structs.JobSpecificRequest) + reply := args.Get(1).(*structs.SingleJobResponse) + reply.Job = jobs[in.JobID] + reply.Index = reply.Job.ModifyIndex + }) + + // Set up the calls for retrieving deployments + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.DeploymentListResponse) + reply.Deployments = []*structs.Deployment{d1} + reply.Index = m.nextIndex() + }).Once() + + // Next list 3 + block1 := make(chan time.Time) + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.DeploymentListResponse) + reply.Deployments = []*structs.Deployment{d1, d2, d3} + reply.Index = m.nextIndex() + }).Once().WaitUntil(block1) + + //// Next list 3 but have one be terminal + block2 := make(chan time.Time) + d3terminal := d3.Copy() + d3terminal.Status = structs.DeploymentStatusFailed + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.DeploymentListResponse) + reply.Deployments = []*structs.Deployment{d1, d2, d3terminal} + reply.Index = m.nextIndex() + }).WaitUntil(block2) + + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.DeploymentListResponse) + reply.Deployments = []*structs.Deployment{d1, d2, d3terminal} + reply.Index = m.nextIndex() + }) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "1 deployment returned") }) + + close(block1) + testutil.WaitForResult(func() (bool, error) { return 3 == len(w.watchers), nil }, + func(err error) { assert.Equal(3, len(w.watchers), "3 deployment returned") }) + + close(block2) + testutil.WaitForResult(func() (bool, error) { return 2 == len(w.watchers), nil }, + func(err error) { assert.Equal(3, len(w.watchers), "3 deployment returned - 1 terminal") }) +} + +// Tests that calls against an unknown deployment fail +func TestWatcher_UnknownDeployment(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + w.SetEnabled(true) + + // Set up the calls for retrieving deployments + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.DeploymentListResponse) + reply.Index = m.nextIndex() + }) + + // Request setting the health against an unknown deployment + dID := structs.GenerateUUID() + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: dID, + HealthyAllocationIDs: []string{structs.GenerateUUID()}, + } + var resp structs.DeploymentUpdateResponse + err := w.SetAllocHealth(req, &resp) + if assert.NotNil(err, "should have error for unknown deployment") { + assert.Contains(err.Error(), "not being watched") + } + + // Request promoting against an unknown deployment + req2 := &structs.DeploymentPromoteRequest{ + DeploymentID: dID, + All: true, + } + err = w.PromoteDeployment(req2, &resp) + if assert.NotNil(err, "should have error for unknown deployment") { + assert.Contains(err.Error(), "not being watched") + } + + // Request pausing against an unknown deployment + req3 := &structs.DeploymentPauseRequest{ + DeploymentID: dID, + Pause: true, + } + err = w.PauseDeployment(req3, &resp) + if assert.NotNil(err, "should have error for unknown deployment") { + assert.Contains(err.Error(), "not being watched") + } +} + +// Test setting an unknown allocation's health +func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentAllocHealth + a := mock.Alloc() + matchConfig := &matchDeploymentAllocHealthRequestConfig{ + DeploymentID: d.ID, + Healthy: []string{a.ID}, + Eval: true, + } + matcher := matchDeploymentAllocHealthRequest(matchConfig) + m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + + // Call SetAllocHealth + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a.ID}, + } + var resp structs.DeploymentUpdateResponse + err := w.SetAllocHealth(req, &resp) + if assert.NotNil(err, "Set health of unknown allocation") { + assert.Contains(err.Error(), "unknown") + } + assert.Equal(1, len(w.watchers), "Deployment should still be active") +} + +// Test setting allocation health +func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, alloc, and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.DeploymentID = d.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentAllocHealth + matchConfig := &matchDeploymentAllocHealthRequestConfig{ + DeploymentID: d.ID, + Healthy: []string{a.ID}, + Eval: true, + } + matcher := matchDeploymentAllocHealthRequest(matchConfig) + m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + + // Call SetAllocHealth + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a.ID}, + } + var resp structs.DeploymentUpdateResponse + err := w.SetAllocHealth(req, &resp) + assert.Nil(err, "SetAllocHealth") + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)) +} + +// Test setting allocation unhealthy +func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, alloc, and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.DeploymentID = d.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentAllocHealth + matchConfig := &matchDeploymentAllocHealthRequestConfig{ + DeploymentID: d.ID, + Unhealthy: []string{a.ID}, + Eval: true, + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusFailed, + StatusDescription: structs.DeploymentStatusDescriptionFailedAllocations, + }, + } + matcher := matchDeploymentAllocHealthRequest(matchConfig) + m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + + // Call SetAllocHealth + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + UnhealthyAllocationIDs: []string{a.ID}, + } + var resp structs.DeploymentUpdateResponse + err := w.SetAllocHealth(req, &resp) + assert.Nil(err, "SetAllocHealth") + + testutil.WaitForResult(func() (bool, error) { return 0 == len(w.watchers), nil }, + func(err error) { assert.Equal(0, len(w.watchers), "Should have no deployment") }) + m.AssertNumberOfCalls(t, "UpsertDeploymentAllocHealth", 1) +} + +// Test setting allocation unhealthy and that there should be a rollback +func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, alloc, and a deployment + j := mock.Job() + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.AutoRevert = true + j.Stable = true + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.DeploymentID = d.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") + + // Upsert the job again to get a new version + j2 := j.Copy() + j2.Stable = false + assert.Nil(m.state.UpsertJob(m.nextIndex(), j2), "UpsertJob2") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentAllocHealth + matchConfig := &matchDeploymentAllocHealthRequestConfig{ + DeploymentID: d.ID, + Unhealthy: []string{a.ID}, + Eval: true, + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusFailed, + StatusDescription: structs.DeploymentStatusDescriptionFailedAllocations, + }, + JobVersion: helper.Uint64ToPtr(0), + } + matcher := matchDeploymentAllocHealthRequest(matchConfig) + m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + + // Call SetAllocHealth + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + UnhealthyAllocationIDs: []string{a.ID}, + } + var resp structs.DeploymentUpdateResponse + err := w.SetAllocHealth(req, &resp) + assert.Nil(err, "SetAllocHealth") + + testutil.WaitForResult(func() (bool, error) { return 0 == len(w.watchers), nil }, + func(err error) { assert.Equal(0, len(w.watchers), "Should have no deployment") }) + m.AssertNumberOfCalls(t, "UpsertDeploymentAllocHealth", 1) +} + +// Test promoting a deployment +func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, canary alloc, and a deployment + j := mock.Job() + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.Canary = 2 + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.Canary = true + a.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + a.DeploymentID = d.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentPromotion + matchConfig := &matchDeploymentPromoteRequestConfig{ + Promotion: &structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + Eval: true, + } + matcher := matchDeploymentPromoteRequest(matchConfig) + m.On("UpsertDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) + + // Call SetAllocHealth + req := &structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + } + var resp structs.DeploymentUpdateResponse + err := w.PromoteDeployment(req, &resp) + assert.Nil(err, "PromoteDeployment") + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentPromotion", mocker.MatchedBy(matcher)) +} + +// Test promoting a deployment with unhealthy canaries +func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, canary alloc, and a deployment + j := mock.Job() + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.Canary = 2 + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.Canary = true + a.DeploymentID = d.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentPromotion + matchConfig := &matchDeploymentPromoteRequestConfig{ + Promotion: &structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + Eval: true, + } + matcher := matchDeploymentPromoteRequest(matchConfig) + m.On("UpsertDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) + + // Call SetAllocHealth + req := &structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + } + var resp structs.DeploymentUpdateResponse + err := w.PromoteDeployment(req, &resp) + if assert.NotNil(err, "PromoteDeployment") { + assert.Contains(err.Error(), "is not healthy", "Should error because canary isn't marked healthy") + } + + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentPromotion", mocker.MatchedBy(matcher)) +} + +// Test pausing a deployment that is running +func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentStatusUpdate + matchConfig := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusPaused, + StatusDescription: structs.DeploymentStatusDescriptionPaused, + } + matcher := matchDeploymentStatusUpdateRequest(matchConfig) + m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + + // Call PauseDeployment + req := &structs.DeploymentPauseRequest{ + DeploymentID: d.ID, + Pause: true, + } + var resp structs.DeploymentUpdateResponse + err := w.PauseDeployment(req, &resp) + assert.Nil(err, "PauseDeployment") + + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) +} + +// Test pausing a deployment that is paused +func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + d.Status = structs.DeploymentStatusPaused + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentStatusUpdate + matchConfig := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusPaused, + StatusDescription: structs.DeploymentStatusDescriptionPaused, + } + matcher := matchDeploymentStatusUpdateRequest(matchConfig) + m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + + // Call PauseDeployment + req := &structs.DeploymentPauseRequest{ + DeploymentID: d.ID, + Pause: true, + } + var resp structs.DeploymentUpdateResponse + err := w.PauseDeployment(req, &resp) + assert.Nil(err, "PauseDeployment") + + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) +} + +// Test unpausing a deployment that is paused +func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + d.Status = structs.DeploymentStatusPaused + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentStatusUpdate + matchConfig := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusRunning, + StatusDescription: structs.DeploymentStatusDescriptionRunning, + Eval: true, + } + matcher := matchDeploymentStatusUpdateRequest(matchConfig) + m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + + // Call PauseDeployment + req := &structs.DeploymentPauseRequest{ + DeploymentID: d.ID, + Pause: false, + } + var resp structs.DeploymentUpdateResponse + err := w.PauseDeployment(req, &resp) + assert.Nil(err, "PauseDeployment") + + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) +} + +// Test unpausing a deployment that is running +func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentStatusUpdate + matchConfig := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusRunning, + StatusDescription: structs.DeploymentStatusDescriptionRunning, + Eval: true, + } + matcher := matchDeploymentStatusUpdateRequest(matchConfig) + m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + + // Call PauseDeployment + req := &structs.DeploymentPauseRequest{ + DeploymentID: d.ID, + Pause: false, + } + var resp structs.DeploymentUpdateResponse + err := w.PauseDeployment(req, &resp) + assert.Nil(err, "PauseDeployment") + + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) +} diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index fda10de4767c..d7c5828bd602 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2019,7 +2019,6 @@ func (s *StateStore) UpsertDeploymentPromotion(index uint64, req *structs.ApplyD return nil } -// TODO test // UpsertDeploymentAllocHealth is used to update the health of allocations as // part of the deployment and potentially make a evaluation func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { diff --git a/vendor/vendor.json b/vendor/vendor.json index 974466edf35a..01f87fc24cd2 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -255,6 +255,12 @@ "revision": "7d649b46cdc2cd2ed102d350688a75a4fd7778c6", "revisionTime": "2016-11-21T13:51:53Z" }, + { + "checksumSHA1": "/5cvgU+J4l7EhMXTK76KaCAfOuU=", + "path": "github.com/davecgh/go-spew/spew", + "revision": "346938d642f2ec3594ed81d874461961cd0faa76", + "revisionTime": "2016-10-29T20:57:26Z" + }, { "checksumSHA1": "Gj+xR1VgFKKmFXYOJMnAczC3Znk=", "path": "github.com/docker/distribution/digestset", @@ -360,14 +366,14 @@ { "checksumSHA1": "iP5slJJPRZUm0rfdII8OiATAACA=", "path": "github.com/docker/docker/pkg/idtools", - "revision": "02caa73df411debed164f520a6a1304778f8b88c", - "revisionTime": "2016-05-28T10:48:36Z" + "revision": "52debcd58ac91bf68503ce60561536911b74ff05", + "revisionTime": "2016-05-20T15:17:10Z" }, { "checksumSHA1": "iP5slJJPRZUm0rfdII8OiATAACA=", "path": "github.com/docker/docker/pkg/idtools", - "revision": "52debcd58ac91bf68503ce60561536911b74ff05", - "revisionTime": "2016-05-20T15:17:10Z" + "revision": "02caa73df411debed164f520a6a1304778f8b88c", + "revisionTime": "2016-05-28T10:48:36Z" }, { "checksumSHA1": "tdhmIGUaoOMEDymMC23qTS7bt0g=", @@ -402,14 +408,14 @@ { "checksumSHA1": "rArZ5mYIe9I1L5PRQOJu8BwafFw=", "path": "github.com/docker/docker/pkg/pools", - "revision": "52debcd58ac91bf68503ce60561536911b74ff05", - "revisionTime": "2016-05-20T15:17:10Z" + "revision": "da39e9a4f920a15683dd0f23923c302d4db6eed5", + "revisionTime": "2016-05-28T08:11:04Z" }, { "checksumSHA1": "rArZ5mYIe9I1L5PRQOJu8BwafFw=", "path": "github.com/docker/docker/pkg/pools", - "revision": "da39e9a4f920a15683dd0f23923c302d4db6eed5", - "revisionTime": "2016-05-28T08:11:04Z" + "revision": "52debcd58ac91bf68503ce60561536911b74ff05", + "revisionTime": "2016-05-20T15:17:10Z" }, { "checksumSHA1": "txf3EORYff4hO6PEvwBm2lyh1MU=", @@ -1058,6 +1064,12 @@ "revision": "248dadf4e9068a0b3e79f02ed0a610d935de5302", "revisionTime": "2016-10-29T09:36:37Z" }, + { + "checksumSHA1": "LuFv4/jlrmFNnDb/5SCSEPAM9vU=", + "path": "github.com/pmezard/go-difflib/difflib", + "revision": "792786c7400a136282c1664665ae0a8db921c6c2", + "revisionTime": "2016-01-10T10:55:54Z" + }, { "checksumSHA1": "1SC2ACq72a+yfN6CYp5s5woKsR4=", "comment": "v2.0.1-8-g983d3a5", @@ -1131,6 +1143,18 @@ "revision": "bb4de0191aa41b5507caa14b0650cdbddcd9280b", "revisionTime": "2016-09-30T03:27:40Z" }, + { + "checksumSHA1": "K0crHygPTP42i1nLKWphSlvOQJw=", + "path": "github.com/stretchr/objx", + "revision": "1a9d0bb9f541897e62256577b352fdbc1fb4fd94", + "revisionTime": "2015-09-28T12:21:52Z" + }, + { + "checksumSHA1": "o+jsS/rxceTym4M3reSPfrPxaio=", + "path": "github.com/stretchr/testify/mock", + "revision": "f6abca593680b2315d2075e0f5e2a9751e3f431a", + "revisionTime": "2017-06-01T20:57:54Z" + }, { "checksumSHA1": "CoxdaTYdPZNJXr8mJfLxye428N0=", "path": "github.com/ugorji/go/codec", From bd02a845c851465a0f28988028f4ba1cb4f89163 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 12:58:05 -0700 Subject: [PATCH 028/105] more tests --- nomad/deploymentwatcher/batcher.go | 9 +- nomad/deploymentwatcher/deployment_watcher.go | 19 +- .../deploymentwatcher/deployments_watcher.go | 2 +- .../deployments_watcher_test.go | 453 ++++++------------ nomad/deploymentwatcher/testutil_test.go | 356 ++++++++++++++ nomad/structs/structs.go | 6 + 6 files changed, 517 insertions(+), 328 deletions(-) create mode 100644 nomad/deploymentwatcher/testutil_test.go diff --git a/nomad/deploymentwatcher/batcher.go b/nomad/deploymentwatcher/batcher.go index 29df3af51c7e..a47ea37fa2e9 100644 --- a/nomad/deploymentwatcher/batcher.go +++ b/nomad/deploymentwatcher/batcher.go @@ -8,6 +8,12 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +const ( + // evalBatchDuration is the duration in which evaluations are batched before + // commiting to Raft. + evalBatchDuration = 200 * time.Millisecond +) + // EvalBatcher is used to batch the creation of evaluations type EvalBatcher struct { // raft is used to actually commit the evaluations @@ -32,6 +38,7 @@ func NewEvalBatcher(raft DeploymentRaftEndpoints, ctx context.Context) *EvalBatc b := &EvalBatcher{ raft: raft, ctx: ctx, + inCh: make(chan *structs.Evaluation, 10), } go b.batcher() @@ -54,7 +61,7 @@ func (b *EvalBatcher) CreateEval(e *structs.Evaluation) *EvalFuture { // batcher is the long lived batcher goroutine func (b *EvalBatcher) batcher() { - ticker := time.NewTicker(200 * time.Millisecond) + ticker := time.NewTicker(evalBatchDuration) evals := make(map[string]*structs.Evaluation) for { select { diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index da01a0c48234..2c578f595df4 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -241,7 +241,7 @@ func (w *deploymentWatcher) watch() { // Block getting all allocations that are part of the deployment using // the last evaluation index. This will have us block waiting for // something to change past what the scheduler has evaluated. - allocs, err := w.getAllocs(latestEval) + allocResp, err := w.getAllocs(latestEval) if err != nil { if err == context.Canceled { return @@ -263,7 +263,7 @@ func (w *deploymentWatcher) watch() { // Create an evaluation trigger if there is any allocation whose // deployment status has been updated past the latest eval index. createEval, failDeployment, rollback := false, false, false - for _, alloc := range allocs { + for _, alloc := range allocResp.Allocations { if alloc.DeploymentStatus == nil || alloc.DeploymentStatus.ModifyIndex <= latestEval { continue } @@ -303,7 +303,9 @@ func (w *deploymentWatcher) watch() { // Description should include that the job is being rolled back to // version N - desc = fmt.Sprintf("%s - rolling back to job version %d", desc, j.Version) + if j != nil { + desc = structs.DeploymentStatusDescriptionRollback(desc, j.Version) + } } // Update the status of the deployment to failed and create an @@ -316,7 +318,7 @@ func (w *deploymentWatcher) watch() { } } else if createEval { // Create an eval to push the deployment along - w.createEvalBatched() + w.createEvalBatched(allocResp.Index) } } } @@ -348,7 +350,7 @@ func (w *deploymentWatcher) createEval() (evalID string, evalCreateIndex uint64, } // createEvalBatched creates an eval but batches calls together -func (w *deploymentWatcher) createEvalBatched() { +func (w *deploymentWatcher) createEvalBatched(forIndex uint64) { w.l.Lock() defer w.l.Unlock() @@ -356,6 +358,9 @@ func (w *deploymentWatcher) createEvalBatched() { return } + w.logger.Printf("[TRACE] nomad.deployment_watcher: creating eval for index %d %q", forIndex, w.d.ID) + w.outstandingBatch = true + go func() { // Sleep til the batching period is over time.Sleep(evalBatchPeriod) @@ -394,7 +399,7 @@ func (w *deploymentWatcher) getDeploymentStatusUpdate(status, desc string) *stru // getAllocs retrieves the allocations that are part of the deployment blocking // at the given index. -func (w *deploymentWatcher) getAllocs(index uint64) ([]*structs.AllocListStub, error) { +func (w *deploymentWatcher) getAllocs(index uint64) (*structs.AllocListResponse, error) { // Build the request args := &structs.DeploymentSpecificRequest{ DeploymentID: w.d.ID, @@ -414,7 +419,7 @@ func (w *deploymentWatcher) getAllocs(index uint64) ([]*structs.AllocListStub, e } } - return resp.Allocations, nil + return &resp, nil } // latestEvalIndex returns the index of the last evaluation created for diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index fb1880a52e68..a6c9528563ac 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -56,7 +56,7 @@ type DeploymentStateWatchers interface { const ( // limitStateQueriesPerSecond is the number of state queries allowed per // second - limitStateQueriesPerSecond = 10.0 + limitStateQueriesPerSecond = 15.0 ) // Watcher is used to watch deployments and their allocations created diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 2988c624ddf7..527f3778eb00 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -1,337 +1,19 @@ package deploymentwatcher import ( - "log" - "os" - "reflect" - "strings" - "sync" + "fmt" "testing" "time" memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" - "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" "github.com/stretchr/testify/assert" mocker "github.com/stretchr/testify/mock" ) -// TODO -// Test evaluations are batched between watchers -// Test allocation watcher -// Test that evaluation due to allocation changes are batched - -func testLogger() *log.Logger { - return log.New(os.Stderr, "", log.LstdFlags) -} - -type mockBackend struct { - mocker.Mock - index uint64 - state *state.StateStore - l sync.Mutex -} - -func newMockBackend(t *testing.T) *mockBackend { - state, err := state.NewStateStore(os.Stderr) - if err != nil { - t.Fatalf("err: %v", err) - } - if state == nil { - t.Fatalf("missing state") - } - return &mockBackend{ - index: 10000, - state: state, - } -} - -func (m *mockBackend) nextIndex() uint64 { - m.l.Lock() - defer m.l.Unlock() - i := m.index - m.index++ - return i -} - -func (m *mockBackend) UpsertEvals(evals []*structs.Evaluation) (uint64, error) { - m.Called(evals) - i := m.nextIndex() - return i, m.state.UpsertEvals(i, evals) -} - -func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) { - m.Called(job) - i := m.nextIndex() - return i, m.state.UpsertJob(i, job) -} - -func (m *mockBackend) UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { - m.Called(u) - i := m.nextIndex() - return i, m.state.UpsertDeploymentStatusUpdate(i, u) -} - -// matchDeploymentStatusUpdateConfig is used to configure the matching -// function -type matchDeploymentStatusUpdateConfig struct { - // DeploymentID is the expected ID - DeploymentID string - - // Status is the desired status - Status string - - // StatusDescription is the desired status description - StatusDescription string - - // JobVersion marks whether we expect a roll back job at the given version - JobVersion *uint64 - - // Eval marks whether we expect an evaluation. - Eval bool -} - -// matchDeploymentStatusUpdateRequest is used to match an update request -func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) func(args *structs.DeploymentStatusUpdateRequest) bool { - return func(args *structs.DeploymentStatusUpdateRequest) bool { - if args.DeploymentUpdate.DeploymentID != c.DeploymentID { - return false - } - - if args.DeploymentUpdate.Status != c.Status && args.DeploymentUpdate.StatusDescription != c.StatusDescription { - return false - } - - if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { - return false - } - - if (c.JobVersion != nil && (args.Job == nil || args.Job.Version != *c.JobVersion)) || c.JobVersion == nil && args.Job != nil { - return false - } - - return true - } -} - -func (m *mockBackend) UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { - m.Called(req) - i := m.nextIndex() - return i, m.state.UpsertDeploymentPromotion(i, req) -} - -// matchDeploymentPromoteRequestConfig is used to configure the matching -// function -type matchDeploymentPromoteRequestConfig struct { - // Promotion holds the expected promote request - Promotion *structs.DeploymentPromoteRequest - - // Eval marks whether we expect an evaluation. - Eval bool -} - -// matchDeploymentPromoteRequest is used to match a promote request -func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func(args *structs.ApplyDeploymentPromoteRequest) bool { - return func(args *structs.ApplyDeploymentPromoteRequest) bool { - if !reflect.DeepEqual(*c.Promotion, args.DeploymentPromoteRequest) { - return false - } - - if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { - return false - } - - return true - } -} -func (m *mockBackend) UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { - m.Called(req) - i := m.nextIndex() - return i, m.state.UpsertDeploymentAllocHealth(i, req) -} - -// matchDeploymentAllocHealthRequestConfig is used to configure the matching -// function -type matchDeploymentAllocHealthRequestConfig struct { - // DeploymentID is the expected ID - DeploymentID string - - // Healthy and Unhealthy contain the expected allocation IDs that are having - // their health set - Healthy, Unhealthy []string - - // DeploymentUpdate holds the expected values of status and description. We - // don't check for exact match but string contains - DeploymentUpdate *structs.DeploymentStatusUpdate - - // JobVersion marks whether we expect a roll back job at the given version - JobVersion *uint64 - - // Eval marks whether we expect an evaluation. - Eval bool -} - -// matchDeploymentAllocHealthRequest is used to match an update request -func matchDeploymentAllocHealthRequest(c *matchDeploymentAllocHealthRequestConfig) func(args *structs.ApplyDeploymentAllocHealthRequest) bool { - return func(args *structs.ApplyDeploymentAllocHealthRequest) bool { - if args.DeploymentID != c.DeploymentID { - return false - } - - if len(c.Healthy) != len(args.HealthyAllocationIDs) { - return false - } - if len(c.Unhealthy) != len(args.UnhealthyAllocationIDs) { - return false - } - - hmap, umap := make(map[string]struct{}, len(c.Healthy)), make(map[string]struct{}, len(c.Unhealthy)) - for _, h := range c.Healthy { - hmap[h] = struct{}{} - } - for _, u := range c.Unhealthy { - umap[u] = struct{}{} - } - - for _, h := range args.HealthyAllocationIDs { - if _, ok := hmap[h]; !ok { - return false - } - } - for _, u := range args.UnhealthyAllocationIDs { - if _, ok := umap[u]; !ok { - return false - } - } - - if c.DeploymentUpdate != nil { - if args.DeploymentUpdate == nil { - return false - } - - if !strings.Contains(args.DeploymentUpdate.Status, c.DeploymentUpdate.Status) { - return false - } - if !strings.Contains(args.DeploymentUpdate.StatusDescription, c.DeploymentUpdate.StatusDescription) { - return false - } - } else if args.DeploymentUpdate != nil { - return false - } - - if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { - return false - } - - if (c.JobVersion != nil && (args.Job == nil || args.Job.Version != *c.JobVersion)) || c.JobVersion == nil && args.Job != nil { - return false - } - - return true - } -} - -func (m *mockBackend) Evaluations(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error { - rargs := m.Called(args, reply) - return rargs.Error(0) -} - -func (m *mockBackend) evaluationsFromState(in mocker.Arguments) { - args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobEvaluationsResponse) - ws := memdb.NewWatchSet() - evals, _ := m.state.EvalsByJob(ws, args.JobID) - reply.Evaluations = evals - reply.Index = m.nextIndex() -} - -func (m *mockBackend) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { - rargs := m.Called(args, reply) - return rargs.Error(0) -} - -func (m *mockBackend) allocationsFromState(in mocker.Arguments) { - args, reply := in.Get(0).(*structs.DeploymentSpecificRequest), in.Get(1).(*structs.AllocListResponse) - ws := memdb.NewWatchSet() - allocs, _ := m.state.AllocsByDeployment(ws, args.DeploymentID) - - var stubs []*structs.AllocListStub - for _, a := range allocs { - stubs = append(stubs, a.Stub()) - } - - reply.Allocations = stubs - reply.Index = m.nextIndex() -} - -func (m *mockBackend) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { - rargs := m.Called(args, reply) - return rargs.Error(0) -} - -func (m *mockBackend) listFromState(in mocker.Arguments) { - reply := in.Get(1).(*structs.DeploymentListResponse) - ws := memdb.NewWatchSet() - iter, _ := m.state.Deployments(ws) - - var deploys []*structs.Deployment - for { - raw := iter.Next() - if raw == nil { - break - } - - deploys = append(deploys, raw.(*structs.Deployment)) - } - - reply.Deployments = deploys - reply.Index = m.nextIndex() -} - -func (m *mockBackend) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { - rargs := m.Called(args, reply) - return rargs.Error(0) -} - -func (m *mockBackend) getJobVersionsFromState(in mocker.Arguments) { - args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobVersionsResponse) - ws := memdb.NewWatchSet() - versions, _ := m.state.JobVersionsByID(ws, args.JobID) - reply.Versions = versions - reply.Index = m.nextIndex() -} - -func (m *mockBackend) GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error { - rargs := m.Called(args, reply) - return rargs.Error(0) -} - -func (m *mockBackend) getJobFromState(in mocker.Arguments) { - args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.SingleJobResponse) - ws := memdb.NewWatchSet() - job, _ := m.state.JobByID(ws, args.JobID) - reply.Job = job - reply.Index = m.nextIndex() -} - -// matchDeploymentSpecificRequest is used to match that a deployment specific -// request is for the passed deployment id -func matchDeploymentSpecificRequest(dID string) func(args *structs.DeploymentSpecificRequest) bool { - return func(args *structs.DeploymentSpecificRequest) bool { - return args.DeploymentID == dID - } -} - -// matchJobSpecificRequest is used to match that a job specific -// request is for the passed job id -func matchJobSpecificRequest(jID string) func(args *structs.JobSpecificRequest) bool { - return func(args *structs.JobSpecificRequest) bool { - return args.JobID == jID - } -} - // Tests that the watcher properly watches for deployments and reconciles them func TestWatcher_WatchDeployments(t *testing.T) { assert := assert.New(t) @@ -998,3 +680,136 @@ func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { assert.Equal(1, len(w.watchers), "Deployment should still be active") m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) } + +// Tests that the watcher properly watches for allocation changes and takes the +// proper actions +func TestDeploymentWatcher_Watch(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m) + + // Create a job, alloc, and a deployment + j := mock.Job() + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.AutoRevert = true + j.Stable = true + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.DeploymentID = d.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") + + // Upsert the job again to get a new version + j2 := j.Copy() + j2.Stable = false + assert.Nil(m.state.UpsertJob(m.nextIndex(), j2), "UpsertJob2") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we will get a createEvaluation call only once. This will + // verify that the watcher is batching allocation changes + m1 := matchUpsertEvals([]string{d.ID}) + m.On("UpsertEvals", mocker.MatchedBy(m1)).Return(nil).Once() + + // Update the allocs health to healthy which should create an evaluation + for i := 0; i < 5; i++ { + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a.ID}, + }, + } + i := m.nextIndex() + assert.Nil(m.state.UpsertDeploymentAllocHealth(i, req), "UpsertDeploymentAllocHealth") + } + + // Wait for there to be one eval + testutil.WaitForResult(func() (bool, error) { + ws := memdb.NewWatchSet() + evals, err := m.state.EvalsByJob(ws, j.ID) + if err != nil { + return false, err + } + + if l := len(evals); l != 1 { + return false, fmt.Errorf("Got %d evals; want 1", l) + } + + return true, nil + }, func(err error) { + t.Fatal(err) + }) + + // Assert that we get a call to UpsertDeploymentStatusUpdate + c := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusFailed, + StatusDescription: structs.DeploymentStatusDescriptionRollback(structs.DeploymentStatusDescriptionFailedAllocations, 0), + JobVersion: helper.Uint64ToPtr(0), + Eval: true, + } + m2 := matchDeploymentStatusUpdateRequest(c) + m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(m2)).Return(nil) + + // Update the allocs health to unhealthy which should create a job rollback, + // status update and eval + req2 := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + UnhealthyAllocationIDs: []string{a.ID}, + }, + } + i := m.nextIndex() + assert.Nil(m.state.UpsertDeploymentAllocHealth(i, req2), "UpsertDeploymentAllocHealth") + + // Wait for there to be one eval + testutil.WaitForResult(func() (bool, error) { + ws := memdb.NewWatchSet() + evals, err := m.state.EvalsByJob(ws, j.ID) + if err != nil { + return false, err + } + + if l := len(evals); l != 2 { + return false, fmt.Errorf("Got %d evals; want 1", l) + } + + return true, nil + }, func(err error) { + t.Fatal(err) + }) + + m.AssertCalled(t, "UpsertEvals", mocker.MatchedBy(m1)) + + // After we upsert the job version will go to 2. So use this to assert the + // original call happened. + c2 := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusFailed, + StatusDescription: structs.DeploymentStatusDescriptionRollback(structs.DeploymentStatusDescriptionFailedAllocations, 0), + JobVersion: helper.Uint64ToPtr(2), + Eval: true, + } + m3 := matchDeploymentStatusUpdateRequest(c2) + m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(m3)) + testutil.WaitForResult(func() (bool, error) { return 0 == len(w.watchers), nil }, + func(err error) { assert.Equal(0, len(w.watchers), "Should have no deployment") }) +} + +// Test evaluations are batched between watchers diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go new file mode 100644 index 000000000000..71b49b9cc160 --- /dev/null +++ b/nomad/deploymentwatcher/testutil_test.go @@ -0,0 +1,356 @@ +package deploymentwatcher + +import ( + "log" + "os" + "reflect" + "strings" + "sync" + "testing" + + memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/nomad/state" + "github.com/hashicorp/nomad/nomad/structs" + mocker "github.com/stretchr/testify/mock" +) + +func testLogger() *log.Logger { + return log.New(os.Stderr, "", log.LstdFlags) +} + +type mockBackend struct { + mocker.Mock + index uint64 + state *state.StateStore + l sync.Mutex +} + +func newMockBackend(t *testing.T) *mockBackend { + state, err := state.NewStateStore(os.Stderr) + if err != nil { + t.Fatalf("err: %v", err) + } + if state == nil { + t.Fatalf("missing state") + } + return &mockBackend{ + index: 10000, + state: state, + } +} + +func (m *mockBackend) nextIndex() uint64 { + m.l.Lock() + defer m.l.Unlock() + i := m.index + m.index++ + return i +} + +func (m *mockBackend) UpsertEvals(evals []*structs.Evaluation) (uint64, error) { + m.Called(evals) + i := m.nextIndex() + return i, m.state.UpsertEvals(i, evals) +} + +// matchUpsertEvals is used to match an upsert request +func matchUpsertEvals(deploymentIDs []string) func(evals []*structs.Evaluation) bool { + return func(evals []*structs.Evaluation) bool { + if len(evals) != len(deploymentIDs) { + return false + } + + dmap := make(map[string]struct{}, len(deploymentIDs)) + for _, d := range deploymentIDs { + dmap[d] = struct{}{} + } + + for _, e := range evals { + if _, ok := dmap[e.DeploymentID]; !ok { + return false + } + + delete(dmap, e.DeploymentID) + } + + return true + } +} + +func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) { + m.Called(job) + i := m.nextIndex() + return i, m.state.UpsertJob(i, job) +} + +func (m *mockBackend) UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { + m.Called(u) + i := m.nextIndex() + return i, m.state.UpsertDeploymentStatusUpdate(i, u) +} + +// matchDeploymentStatusUpdateConfig is used to configure the matching +// function +type matchDeploymentStatusUpdateConfig struct { + // DeploymentID is the expected ID + DeploymentID string + + // Status is the desired status + Status string + + // StatusDescription is the desired status description + StatusDescription string + + // JobVersion marks whether we expect a roll back job at the given version + JobVersion *uint64 + + // Eval marks whether we expect an evaluation. + Eval bool +} + +// matchDeploymentStatusUpdateRequest is used to match an update request +func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) func(args *structs.DeploymentStatusUpdateRequest) bool { + return func(args *structs.DeploymentStatusUpdateRequest) bool { + if args.DeploymentUpdate.DeploymentID != c.DeploymentID { + testLogger().Printf("deployment ids dont match") + return false + } + + if args.DeploymentUpdate.Status != c.Status && args.DeploymentUpdate.StatusDescription != c.StatusDescription { + testLogger().Printf("status's dont match") + return false + } + + if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { + testLogger().Printf("evals dont match") + return false + } + + if c.JobVersion != nil { + if args.Job == nil { + return false + } else if args.Job.Version != *c.JobVersion { + return false + } + } else if c.JobVersion == nil && args.Job != nil { + return false + } + + return true + } +} + +func (m *mockBackend) UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { + m.Called(req) + i := m.nextIndex() + return i, m.state.UpsertDeploymentPromotion(i, req) +} + +// matchDeploymentPromoteRequestConfig is used to configure the matching +// function +type matchDeploymentPromoteRequestConfig struct { + // Promotion holds the expected promote request + Promotion *structs.DeploymentPromoteRequest + + // Eval marks whether we expect an evaluation. + Eval bool +} + +// matchDeploymentPromoteRequest is used to match a promote request +func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func(args *structs.ApplyDeploymentPromoteRequest) bool { + return func(args *structs.ApplyDeploymentPromoteRequest) bool { + if !reflect.DeepEqual(*c.Promotion, args.DeploymentPromoteRequest) { + return false + } + + if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { + return false + } + + return true + } +} +func (m *mockBackend) UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { + m.Called(req) + i := m.nextIndex() + return i, m.state.UpsertDeploymentAllocHealth(i, req) +} + +// matchDeploymentAllocHealthRequestConfig is used to configure the matching +// function +type matchDeploymentAllocHealthRequestConfig struct { + // DeploymentID is the expected ID + DeploymentID string + + // Healthy and Unhealthy contain the expected allocation IDs that are having + // their health set + Healthy, Unhealthy []string + + // DeploymentUpdate holds the expected values of status and description. We + // don't check for exact match but string contains + DeploymentUpdate *structs.DeploymentStatusUpdate + + // JobVersion marks whether we expect a roll back job at the given version + JobVersion *uint64 + + // Eval marks whether we expect an evaluation. + Eval bool +} + +// matchDeploymentAllocHealthRequest is used to match an update request +func matchDeploymentAllocHealthRequest(c *matchDeploymentAllocHealthRequestConfig) func(args *structs.ApplyDeploymentAllocHealthRequest) bool { + return func(args *structs.ApplyDeploymentAllocHealthRequest) bool { + if args.DeploymentID != c.DeploymentID { + return false + } + + if len(c.Healthy) != len(args.HealthyAllocationIDs) { + return false + } + if len(c.Unhealthy) != len(args.UnhealthyAllocationIDs) { + return false + } + + hmap, umap := make(map[string]struct{}, len(c.Healthy)), make(map[string]struct{}, len(c.Unhealthy)) + for _, h := range c.Healthy { + hmap[h] = struct{}{} + } + for _, u := range c.Unhealthy { + umap[u] = struct{}{} + } + + for _, h := range args.HealthyAllocationIDs { + if _, ok := hmap[h]; !ok { + return false + } + } + for _, u := range args.UnhealthyAllocationIDs { + if _, ok := umap[u]; !ok { + return false + } + } + + if c.DeploymentUpdate != nil { + if args.DeploymentUpdate == nil { + return false + } + + if !strings.Contains(args.DeploymentUpdate.Status, c.DeploymentUpdate.Status) { + return false + } + if !strings.Contains(args.DeploymentUpdate.StatusDescription, c.DeploymentUpdate.StatusDescription) { + return false + } + } else if args.DeploymentUpdate != nil { + return false + } + + if c.Eval && args.Eval == nil || !c.Eval && args.Eval != nil { + return false + } + + if (c.JobVersion != nil && (args.Job == nil || args.Job.Version != *c.JobVersion)) || c.JobVersion == nil && args.Job != nil { + return false + } + + return true + } +} + +func (m *mockBackend) Evaluations(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) evaluationsFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobEvaluationsResponse) + ws := memdb.NewWatchSet() + evals, _ := m.state.EvalsByJob(ws, args.JobID) + reply.Evaluations = evals + reply.Index, _ = m.state.Index("evals") +} + +func (m *mockBackend) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) allocationsFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.DeploymentSpecificRequest), in.Get(1).(*structs.AllocListResponse) + ws := memdb.NewWatchSet() + allocs, _ := m.state.AllocsByDeployment(ws, args.DeploymentID) + + var stubs []*structs.AllocListStub + for _, a := range allocs { + stubs = append(stubs, a.Stub()) + } + + reply.Allocations = stubs + reply.Index, _ = m.state.Index("allocs") +} + +func (m *mockBackend) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) listFromState(in mocker.Arguments) { + reply := in.Get(1).(*structs.DeploymentListResponse) + ws := memdb.NewWatchSet() + iter, _ := m.state.Deployments(ws) + + var deploys []*structs.Deployment + for { + raw := iter.Next() + if raw == nil { + break + } + + deploys = append(deploys, raw.(*structs.Deployment)) + } + + reply.Deployments = deploys + reply.Index, _ = m.state.Index("deployment") +} + +func (m *mockBackend) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) getJobVersionsFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobVersionsResponse) + ws := memdb.NewWatchSet() + versions, _ := m.state.JobVersionsByID(ws, args.JobID) + reply.Versions = versions + reply.Index, _ = m.state.Index("jobs") +} + +func (m *mockBackend) GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + +func (m *mockBackend) getJobFromState(in mocker.Arguments) { + args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.SingleJobResponse) + ws := memdb.NewWatchSet() + job, _ := m.state.JobByID(ws, args.JobID) + reply.Job = job + reply.Index, _ = m.state.Index("jobs") +} + +// matchDeploymentSpecificRequest is used to match that a deployment specific +// request is for the passed deployment id +func matchDeploymentSpecificRequest(dID string) func(args *structs.DeploymentSpecificRequest) bool { + return func(args *structs.DeploymentSpecificRequest) bool { + return args.DeploymentID == dID + } +} + +// matchJobSpecificRequest is used to match that a job specific +// request is for the passed job id +func matchJobSpecificRequest(jID string) func(args *structs.JobSpecificRequest) bool { + return func(args *structs.JobSpecificRequest) bool { + return args.JobID == jID + } +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 7b26b838eb82..468341441ad7 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3857,6 +3857,12 @@ const ( DeploymentStatusDescriptionFailedAllocations = "Failed due to unhealthy allocations" ) +// DeploymentStatusDescriptionRollback is used to get the status description of +// a deployment when rolling back to an older job. +func DeploymentStatusDescriptionRollback(baseDescription string, jobVersion uint64) string { + return fmt.Sprintf("%s - rolling back to job version %d", baseDescription, jobVersion) +} + // Deployment is the object that represents a job deployment which is used to // transistion a job between versions. type Deployment struct { From 5d5e2879d29116e6f01be46c632760510bb485dd Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 13:19:41 -0700 Subject: [PATCH 029/105] Fix unnecessary evals --- nomad/deploymentwatcher/deployment_watcher.go | 43 +++++++++++++++---- .../deploymentwatcher/deployments_watcher.go | 2 - 2 files changed, 34 insertions(+), 11 deletions(-) diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 2c578f595df4..0d76d6ca097b 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -67,6 +67,9 @@ type deploymentWatcher struct { // the evaluation. Access should be done through the lock outstandingBatch bool + // latestEval is the latest eval for the job + latestEval uint64 + logger *log.Logger ctx context.Context exitFn context.CancelFunc @@ -175,6 +178,7 @@ func (w *deploymentWatcher) SetAllocHealth( resp.EvalID = areq.Eval.ID resp.EvalCreateIndex = index resp.DeploymentModifyIndex = index + w.setLatestEval(index) return nil } @@ -197,6 +201,7 @@ func (w *deploymentWatcher) PromoteDeployment( resp.EvalID = areq.Eval.ID resp.EvalCreateIndex = index resp.DeploymentModifyIndex = index + w.setLatestEval(index) return nil } @@ -225,6 +230,7 @@ func (w *deploymentWatcher) PauseDeployment( resp.EvalID = evalID resp.EvalCreateIndex = i resp.DeploymentModifyIndex = i + w.setLatestEval(i) return nil } @@ -236,12 +242,11 @@ func (w *deploymentWatcher) StopWatch() { // watch is the long running watcher that takes actions upon allocation changes func (w *deploymentWatcher) watch() { - latestEval := uint64(0) for { // Block getting all allocations that are part of the deployment using // the last evaluation index. This will have us block waiting for // something to change past what the scheduler has evaluated. - allocResp, err := w.getAllocs(latestEval) + allocResp, err := w.getAllocs(w.getLatestEval()) if err != nil { if err == context.Canceled { return @@ -251,7 +256,7 @@ func (w *deploymentWatcher) watch() { } // Get the latest evaluation snapshot index - latestEval, err = w.latestEvalIndex() + latestEval, err := w.latestEvalIndex() if err != nil { if err == context.Canceled { return @@ -314,7 +319,7 @@ func (w *deploymentWatcher) watch() { if index, err := w.upsertDeploymentStatusUpdate(u, e, j); err != nil { w.logger.Printf("[ERR] nomad.deployment_watcher: failed to update deployment %q status: %v", w.d.ID, err) } else { - latestEval = index + w.setLatestEval(index) } } else if createEval { // Create an eval to push the deployment along @@ -346,6 +351,9 @@ func (w *deploymentWatcher) latestStableJob() (*structs.Job, error) { func (w *deploymentWatcher) createEval() (evalID string, evalCreateIndex uint64, err error) { e := w.getEval() evalCreateIndex, err = w.createEvaluation(e) + if err != nil { + w.setLatestEval(evalCreateIndex) + } return e.ID, evalCreateIndex, err } @@ -354,24 +362,24 @@ func (w *deploymentWatcher) createEvalBatched(forIndex uint64) { w.l.Lock() defer w.l.Unlock() - if w.outstandingBatch { + if w.outstandingBatch || forIndex < w.latestEval { return } - w.logger.Printf("[TRACE] nomad.deployment_watcher: creating eval for index %d %q", forIndex, w.d.ID) w.outstandingBatch = true go func() { // Sleep til the batching period is over time.Sleep(evalBatchPeriod) + if _, _, err := w.createEval(); err != nil { + w.logger.Printf("[ERR] nomad.deployment_watcher: failed to create evaluation for deployment %q: %v", w.d.ID, err) + } + w.l.Lock() w.outstandingBatch = false w.l.Unlock() - if _, _, err := w.createEval(); err != nil { - w.logger.Printf("[ERR] nomad.deployment_watcher: failed to create evaluation for deployment %q: %v", w.d.ID, err) - } }() } @@ -440,14 +448,31 @@ func (w *deploymentWatcher) latestEvalIndex() (uint64, error) { } if len(resp.Evaluations) == 0 { + w.setLatestEval(resp.Index) return resp.Index, nil } // Prefer using the snapshot index. Otherwise use the create index e := resp.Evaluations[0] if e.SnapshotIndex != 0 { + w.setLatestEval(e.SnapshotIndex) return e.SnapshotIndex, nil } + w.setLatestEval(e.CreateIndex) return e.CreateIndex, nil } + +func (w *deploymentWatcher) setLatestEval(index uint64) { + w.l.Lock() + defer w.l.Unlock() + if index > w.latestEval { + w.latestEval = index + } +} + +func (w *deploymentWatcher) getLatestEval() uint64 { + w.l.Lock() + defer w.l.Unlock() + return w.latestEval +} diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index a6c9528563ac..e9571b4c3c11 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -227,7 +227,6 @@ func (w *Watcher) add(d *structs.Deployment) error { } w.watchers[d.ID] = newDeploymentWatcher(w.ctx, w.queryLimiter, w.logger, w.stateWatchers, d, resp.Job, w) - w.logger.Printf("[TRACE] nomad.deployments_watcher: tracking deployment %q", d.ID) return nil } @@ -245,7 +244,6 @@ func (w *Watcher) remove(d *structs.Deployment) { if watcher, ok := w.watchers[d.ID]; ok { watcher.StopWatch() delete(w.watchers, d.ID) - w.logger.Printf("[TRACE] nomad.deployments_watcher: untracking deployment %q", d.ID) } } From 3615ffdfacd9ff229121d1faa2f6b9da1d0b25e5 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 14:25:20 -0700 Subject: [PATCH 030/105] batch test --- nomad/deploymentwatcher/batcher.go | 41 ++++-- .../deploymentwatcher/deployments_watcher.go | 45 +++--- .../deployments_watcher_test.go | 137 +++++++++++++++--- nomad/deploymentwatcher/testutil_test.go | 2 +- 4 files changed, 175 insertions(+), 50 deletions(-) diff --git a/nomad/deploymentwatcher/batcher.go b/nomad/deploymentwatcher/batcher.go index a47ea37fa2e9..c6b6d9056b71 100644 --- a/nomad/deploymentwatcher/batcher.go +++ b/nomad/deploymentwatcher/batcher.go @@ -8,14 +8,11 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -const ( - // evalBatchDuration is the duration in which evaluations are batched before - // commiting to Raft. - evalBatchDuration = 200 * time.Millisecond -) - // EvalBatcher is used to batch the creation of evaluations type EvalBatcher struct { + // batch is the batching duration + batch time.Duration + // raft is used to actually commit the evaluations raft DeploymentRaftEndpoints @@ -34,11 +31,12 @@ type EvalBatcher struct { // NewEvalBatcher returns an EvalBatcher that uses the passed raft endpoints to // create the evaluations and exits the batcher when the passed exit channel is // closed. -func NewEvalBatcher(raft DeploymentRaftEndpoints, ctx context.Context) *EvalBatcher { +func NewEvalBatcher(batchDuration time.Duration, raft DeploymentRaftEndpoints, ctx context.Context) *EvalBatcher { b := &EvalBatcher{ - raft: raft, - ctx: ctx, - inCh: make(chan *structs.Evaluation, 10), + batch: batchDuration, + raft: raft, + ctx: ctx, + inCh: make(chan *structs.Evaluation, 10), } go b.batcher() @@ -49,11 +47,10 @@ func NewEvalBatcher(raft DeploymentRaftEndpoints, ctx context.Context) *EvalBatc // tracks the evaluations creation. func (b *EvalBatcher) CreateEval(e *structs.Evaluation) *EvalFuture { b.l.Lock() - defer b.l.Unlock() - if b.f == nil { b.f = NewEvalFuture() } + b.l.Unlock() b.inCh <- e return b.f @@ -61,17 +58,26 @@ func (b *EvalBatcher) CreateEval(e *structs.Evaluation) *EvalFuture { // batcher is the long lived batcher goroutine func (b *EvalBatcher) batcher() { - ticker := time.NewTicker(evalBatchDuration) + timer := time.NewTimer(b.batch) evals := make(map[string]*structs.Evaluation) for { select { case <-b.ctx.Done(): - ticker.Stop() + timer.Stop() return case e := <-b.inCh: + if len(evals) == 0 { + if !timer.Stop() { + <-timer.C + } + timer.Reset(b.batch) + } + evals[e.DeploymentID] = e - case <-ticker.C: + case <-timer.C: if len(evals) == 0 { + // Reset the timer + timer.Reset(b.batch) continue } @@ -83,6 +89,8 @@ func (b *EvalBatcher) batcher() { // Shouldn't be possible but protect ourselves if f == nil { + // Reset the timer + timer.Reset(b.batch) continue } @@ -97,6 +105,9 @@ func (b *EvalBatcher) batcher() { // Reset the evals list evals = make(map[string]*structs.Evaluation) + + // Reset the timer + timer.Reset(b.batch) } } } diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index e9571b4c3c11..dce4e4734c27 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -5,6 +5,7 @@ import ( "fmt" "log" "sync" + "time" "golang.org/x/time/rate" @@ -54,9 +55,13 @@ type DeploymentStateWatchers interface { } const ( - // limitStateQueriesPerSecond is the number of state queries allowed per + // LimitStateQueriesPerSecond is the number of state queries allowed per // second - limitStateQueriesPerSecond = 15.0 + LimitStateQueriesPerSecond = 15.0 + + // EvalBatchDuration is the duration in which evaluations are batched before + // commiting to Raft. + EvalBatchDuration = 250 * time.Millisecond ) // Watcher is used to watch deployments and their allocations created @@ -69,6 +74,10 @@ type Watcher struct { // queryLimiter is used to limit the rate of blocking queries queryLimiter *rate.Limiter + // evalBatchDuration is the duration to batch eval creation across all + // deployment watchers + evalBatchDuration time.Duration + // raft contains the set of Raft endpoints that can be used by the // deployments watcher raft DeploymentRaftEndpoints @@ -92,17 +101,23 @@ type Watcher struct { // NewDeploymentsWatcher returns a deployments watcher that is used to watch // deployments and trigger the scheduler as needed. -func NewDeploymentsWatcher(logger *log.Logger, w DeploymentStateWatchers, raft DeploymentRaftEndpoints) *Watcher { +func NewDeploymentsWatcher( + logger *log.Logger, + w DeploymentStateWatchers, + raft DeploymentRaftEndpoints, + stateQueriesPerSecond float64, + evalBatchDuration time.Duration) *Watcher { ctx, exitFn := context.WithCancel(context.Background()) return &Watcher{ - queryLimiter: rate.NewLimiter(limitStateQueriesPerSecond, 100), - stateWatchers: w, - raft: raft, - watchers: make(map[string]*deploymentWatcher, 32), - evalBatcher: NewEvalBatcher(raft, ctx), - logger: logger, - ctx: ctx, - exitFn: exitFn, + queryLimiter: rate.NewLimiter(rate.Limit(stateQueriesPerSecond), 100), + evalBatchDuration: evalBatchDuration, + stateWatchers: w, + raft: raft, + watchers: make(map[string]*deploymentWatcher, 32), + evalBatcher: NewEvalBatcher(evalBatchDuration, raft, ctx), + logger: logger, + ctx: ctx, + exitFn: exitFn, } } @@ -136,7 +151,7 @@ func (w *Watcher) Flush() { w.watchers = make(map[string]*deploymentWatcher, 32) w.ctx, w.exitFn = context.WithCancel(context.Background()) - w.evalBatcher = NewEvalBatcher(w.raft, w.ctx) + w.evalBatcher = NewEvalBatcher(w.evalBatchDuration, w.raft, w.ctx) } // watchDeployments is the long lived go-routine that watches for deployments to @@ -309,11 +324,7 @@ func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest, resp *str // createEvaluation commits the given evaluation to Raft but batches the commit // with other calls. func (w *Watcher) createEvaluation(eval *structs.Evaluation) (uint64, error) { - w.l.Lock() - f := w.evalBatcher.CreateEval(eval) - w.l.Unlock() - - return f.Results() + return w.evalBatcher.CreateEval(eval).Results() } // upsertJob commits the given job to Raft diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 527f3778eb00..2e1056c63361 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -18,7 +18,7 @@ import ( func TestWatcher_WatchDeployments(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Return no allocations or evals m.On("Allocations", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { @@ -100,7 +100,7 @@ func TestWatcher_WatchDeployments(t *testing.T) { func TestWatcher_UnknownDeployment(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) w.SetEnabled(true) // Set up the calls for retrieving deployments @@ -146,7 +146,7 @@ func TestWatcher_UnknownDeployment(t *testing.T) { func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job, and a deployment j := mock.Job() @@ -195,7 +195,7 @@ func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job, alloc, and a deployment j := mock.Job() @@ -245,7 +245,7 @@ func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job, alloc, and a deployment j := mock.Job() @@ -302,7 +302,7 @@ func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job, alloc, and a deployment j := mock.Job() @@ -371,7 +371,7 @@ func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job, canary alloc, and a deployment j := mock.Job() @@ -430,7 +430,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job, canary alloc, and a deployment j := mock.Job() @@ -489,7 +489,7 @@ func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job and a deployment j := mock.Job() @@ -537,7 +537,7 @@ func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job and a deployment j := mock.Job() @@ -586,7 +586,7 @@ func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job and a deployment j := mock.Job() @@ -636,7 +636,7 @@ func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) // Create a job and a deployment j := mock.Job() @@ -686,7 +686,7 @@ func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { func TestDeploymentWatcher_Watch(t *testing.T) { assert := assert.New(t) m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m) + w := NewDeploymentsWatcher(testLogger(), m, m, 1000.0, 1*time.Millisecond) // Create a job, alloc, and a deployment j := mock.Job() @@ -735,8 +735,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { HealthyAllocationIDs: []string{a.ID}, }, } - i := m.nextIndex() - assert.Nil(m.state.UpsertDeploymentAllocHealth(i, req), "UpsertDeploymentAllocHealth") + assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") } // Wait for there to be one eval @@ -775,8 +774,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { UnhealthyAllocationIDs: []string{a.ID}, }, } - i := m.nextIndex() - assert.Nil(m.state.UpsertDeploymentAllocHealth(i, req2), "UpsertDeploymentAllocHealth") + assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval testutil.WaitForResult(func() (bool, error) { @@ -813,3 +811,108 @@ func TestDeploymentWatcher_Watch(t *testing.T) { } // Test evaluations are batched between watchers +func TestWatcher_BatchEvals(t *testing.T) { + assert := assert.New(t) + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), m, m, 1000.0, EvalBatchDuration) + + // Create a job, alloc, for two deployments + j1 := mock.Job() + d1 := mock.Deployment() + d1.JobID = j1.ID + a1 := mock.Alloc() + a1.DeploymentID = d1.ID + + j2 := mock.Job() + d2 := mock.Deployment() + d2.JobID = j2.ID + a2 := mock.Alloc() + a2.DeploymentID = d2.ID + + assert.Nil(m.state.UpsertJob(m.nextIndex(), j1), "UpsertJob") + assert.Nil(m.state.UpsertJob(m.nextIndex(), j2), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d1, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d2, false), "UpsertDeployment") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a1}), "UpsertAllocs") + assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a2}), "UpsertAllocs") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d1.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d2.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j1.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j2.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j1.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j2.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j1.ID)), + mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) + m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j2.ID)), + mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 2 == len(w.watchers), nil }, + func(err error) { assert.Equal(2, len(w.watchers), "Should have 2 deployment") }) + + // Assert that we will get a createEvaluation call only once and it contains + // both deployments. This will verify that the watcher is batching + // allocation changes + m1 := matchUpsertEvals([]string{d1.ID, d2.ID}) + m.On("UpsertEvals", mocker.MatchedBy(m1)).Return(nil).Once() + + // Update the allocs health to healthy which should create an evaluation + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d1.ID, + HealthyAllocationIDs: []string{a1.ID}, + }, + } + assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") + + req2 := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d2.ID, + HealthyAllocationIDs: []string{a2.ID}, + }, + } + assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + + // Wait for there to be one eval for each job + testutil.WaitForResult(func() (bool, error) { + ws := memdb.NewWatchSet() + evals1, err := m.state.EvalsByJob(ws, j1.ID) + if err != nil { + return false, err + } + + evals2, err := m.state.EvalsByJob(ws, j2.ID) + if err != nil { + return false, err + } + + if l := len(evals1); l != 1 { + return false, fmt.Errorf("Got %d evals; want 1", l) + } + + if l := len(evals2); l != 1 { + return false, fmt.Errorf("Got %d evals; want 1", l) + } + + return true, nil + }, func(err error) { + t.Fatal(err) + }) + + m.AssertCalled(t, "UpsertEvals", mocker.MatchedBy(m1)) + testutil.WaitForResult(func() (bool, error) { return 2 == len(w.watchers), nil }, + func(err error) { assert.Equal(2, len(w.watchers), "Should have 2 deployment") }) +} diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index 71b49b9cc160..0c28cba9f5e3 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -15,7 +15,7 @@ import ( ) func testLogger() *log.Logger { - return log.New(os.Stderr, "", log.LstdFlags) + return log.New(os.Stderr, "", log.LstdFlags|log.Lmicroseconds) } type mockBackend struct { From aeeb0a656de921fac562911a196b4accacfcc081 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 14:32:11 -0700 Subject: [PATCH 031/105] comments --- nomad/deploymentwatcher/deployment_watcher.go | 39 +++++++++---------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 0d76d6ca097b..3112982a4dec 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -67,7 +67,9 @@ type deploymentWatcher struct { // the evaluation. Access should be done through the lock outstandingBatch bool - // latestEval is the latest eval for the job + // latestEval is the latest eval for the job. It is updated by the watch + // loop and any time an evaluation is created. The field should be accessed + // by holding the lock or using the setter and getter methods. latestEval uint64 logger *log.Logger @@ -78,14 +80,9 @@ type deploymentWatcher struct { // newDeploymentWatcher returns a deployment watcher that is used to watch // deployments and trigger the scheduler as needed. -func newDeploymentWatcher( - parent context.Context, - queryLimiter *rate.Limiter, - logger *log.Logger, - watchers DeploymentStateWatchers, - d *structs.Deployment, - j *structs.Job, - triggers deploymentTriggers) *deploymentWatcher { +func newDeploymentWatcher(parent context.Context, queryLimiter *rate.Limiter, + logger *log.Logger, watchers DeploymentStateWatchers, d *structs.Deployment, + j *structs.Job, triggers deploymentTriggers) *deploymentWatcher { ctx, exitFn := context.WithCancel(parent) w := &deploymentWatcher{ @@ -100,6 +97,7 @@ func newDeploymentWatcher( exitFn: exitFn, } + // Determine what task groups will trigger an autorevert for _, tg := range j.TaskGroups { autorevert := false if tg.Update != nil && tg.Update.AutoRevert { @@ -108,7 +106,9 @@ func newDeploymentWatcher( w.autorevert[tg.Name] = autorevert } + // Start the long lived watcher that scans for allocation updates go w.watch() + return w } @@ -255,7 +255,7 @@ func (w *deploymentWatcher) watch() { w.logger.Printf("[ERR] nomad.deployment_watcher: failed to retrieve allocations for deployment %q: %v", w.d.ID, err) } - // Get the latest evaluation snapshot index + // Get the latest evaluation index latestEval, err := w.latestEvalIndex() if err != nil { if err == context.Canceled { @@ -347,16 +347,6 @@ func (w *deploymentWatcher) latestStableJob() (*structs.Job, error) { return stable, nil } -// createEval creates an evaluation for the job and commits it to Raft. -func (w *deploymentWatcher) createEval() (evalID string, evalCreateIndex uint64, err error) { - e := w.getEval() - evalCreateIndex, err = w.createEvaluation(e) - if err != nil { - w.setLatestEval(evalCreateIndex) - } - return e.ID, evalCreateIndex, err -} - // createEvalBatched creates an eval but batches calls together func (w *deploymentWatcher) createEvalBatched(forIndex uint64) { w.l.Lock() @@ -372,8 +362,12 @@ func (w *deploymentWatcher) createEvalBatched(forIndex uint64) { // Sleep til the batching period is over time.Sleep(evalBatchPeriod) - if _, _, err := w.createEval(); err != nil { + // Create the eval + evalCreateIndex, err := w.createEvaluation(w.getEval()) + if err != nil { w.logger.Printf("[ERR] nomad.deployment_watcher: failed to create evaluation for deployment %q: %v", w.d.ID, err) + } else { + w.setLatestEval(evalCreateIndex) } w.l.Lock() @@ -463,6 +457,8 @@ func (w *deploymentWatcher) latestEvalIndex() (uint64, error) { return e.CreateIndex, nil } +// setLatestEval sets the given index as the latest eval unless the currently +// stored index is higher. func (w *deploymentWatcher) setLatestEval(index uint64) { w.l.Lock() defer w.l.Unlock() @@ -471,6 +467,7 @@ func (w *deploymentWatcher) setLatestEval(index uint64) { } } +// getLatestEval returns the latest eval index. func (w *deploymentWatcher) getLatestEval() uint64 { w.l.Lock() defer w.l.Unlock() From c86af4d9e9eab343d0435be7315b23843633b7e1 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 15:35:52 -0700 Subject: [PATCH 032/105] Add watcher to server --- nomad/deployment_watcher_shims.go | 84 +++++++++++++++++++ .../deploymentwatcher/deployments_watcher.go | 51 +++++++---- nomad/leader.go | 10 +++ nomad/rpc.go | 3 + nomad/server.go | 78 ++++++++++++----- 5 files changed, 188 insertions(+), 38 deletions(-) create mode 100644 nomad/deployment_watcher_shims.go diff --git a/nomad/deployment_watcher_shims.go b/nomad/deployment_watcher_shims.go new file mode 100644 index 000000000000..ad2dbdcdab00 --- /dev/null +++ b/nomad/deployment_watcher_shims.go @@ -0,0 +1,84 @@ +package nomad + +import "github.com/hashicorp/nomad/nomad/structs" + +// deploymentWatcherStateShim is the shim that provides the state watching +// methods. These should be set by the server and passed to the deployment +// watcher. +type deploymentWatcherStateShim struct { + // evaluations returns the set of evaluations for the given job + evaluations func(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error + + // allocations returns the set of allocations that are part of the + // deployment. + allocations func(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error + + // list is used to list all the deployments in the system + list func(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error + + // getJobVersions is used to lookup the versions of a job. This is used when + // rolling back to find the latest stable job + getJobVersions func(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error + + // getJob is used to lookup a particular job. + getJob func(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error +} + +func (d *deploymentWatcherStateShim) Evaluations(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error { + return d.evaluations(args, reply) +} + +func (d *deploymentWatcherStateShim) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { + return d.allocations(args, reply) +} + +func (d *deploymentWatcherStateShim) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { + return d.list(args, reply) +} + +func (d *deploymentWatcherStateShim) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { + return d.getJobVersions(args, reply) +} + +func (d *deploymentWatcherStateShim) GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error { + return d.getJob(args, reply) +} + +// deploymentWatcherRaftShim is the shim that provides the state watching +// methods. These should be set by the server and passed to the deployment +// watcher. +type deploymentWatcherRaftShim struct { + // apply is used to apply a message to Raft + apply raftApplyFn +} + +func (d *deploymentWatcherRaftShim) UpsertEvals(evals []*structs.Evaluation) (uint64, error) { + update := &structs.EvalUpdateRequest{ + Evals: evals, + } + _, index, err := d.apply(structs.EvalUpdateRequestType, update) + return index, err +} + +func (d *deploymentWatcherRaftShim) UpsertJob(job *structs.Job) (uint64, error) { + update := &structs.JobRegisterRequest{ + Job: job, + } + _, index, err := d.apply(structs.JobRegisterRequestType, update) + return index, err +} + +func (d *deploymentWatcherRaftShim) UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { + _, index, err := d.apply(structs.DeploymentStatusUpdateRequestType, u) + return index, err +} + +func (d *deploymentWatcherRaftShim) UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { + _, index, err := d.apply(structs.DeploymentPromoteRequestType, req) + return index, err +} + +func (d *deploymentWatcherRaftShim) UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { + _, index, err := d.apply(structs.DeploymentAllocHealthRequestType, req) + return index, err +} diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index dce4e4734c27..82425e387540 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -101,39 +101,52 @@ type Watcher struct { // NewDeploymentsWatcher returns a deployments watcher that is used to watch // deployments and trigger the scheduler as needed. -func NewDeploymentsWatcher( - logger *log.Logger, - w DeploymentStateWatchers, - raft DeploymentRaftEndpoints, - stateQueriesPerSecond float64, +func NewDeploymentsWatcher(logger *log.Logger, stateQueriesPerSecond float64, evalBatchDuration time.Duration) *Watcher { - ctx, exitFn := context.WithCancel(context.Background()) + return &Watcher{ queryLimiter: rate.NewLimiter(rate.Limit(stateQueriesPerSecond), 100), evalBatchDuration: evalBatchDuration, - stateWatchers: w, - raft: raft, - watchers: make(map[string]*deploymentWatcher, 32), - evalBatcher: NewEvalBatcher(evalBatchDuration, raft, ctx), logger: logger, - ctx: ctx, - exitFn: exitFn, } } +// SetStateWatchers sets the interface for accessing state watchers +func (w *Watcher) SetStateWatchers(watchers DeploymentStateWatchers) { + w.l.Lock() + defer w.l.Unlock() + w.stateWatchers = watchers +} + +// SetRaftEndpoints sets the interface for writing to Raft +func (w *Watcher) SetRaftEndpoints(raft DeploymentRaftEndpoints) { + w.l.Lock() + defer w.l.Unlock() + w.raft = raft +} + // SetEnabled is used to control if the watcher is enabled. The watcher // should only be enabled on the active leader. -func (w *Watcher) SetEnabled(enabled bool) { +func (w *Watcher) SetEnabled(enabled bool) error { w.l.Lock() + // Ensure our state is correct + if w.stateWatchers == nil || w.raft == nil { + return fmt.Errorf("State watchers and Raft endpoints must be set before starting") + } + wasEnabled := w.enabled w.enabled = enabled w.l.Unlock() - if !enabled { - w.Flush() - } else if !wasEnabled { - // Start the watcher if we are transistioning to an enabled state + + // Flush the state to create the necessary objects + w.Flush() + + // If we are starting now, launch the watch daemon + if enabled && !wasEnabled { go w.watchDeployments() } + + return nil } // Flush is used to clear the state of the watcher @@ -147,7 +160,9 @@ func (w *Watcher) Flush() { } // Kill everything associated with the watcher - w.exitFn() + if w.exitFn != nil { + w.exitFn() + } w.watchers = make(map[string]*deploymentWatcher, 32) w.ctx, w.exitFn = context.WithCancel(context.Background()) diff --git a/nomad/leader.go b/nomad/leader.go index 275f84de9ac3..7b80dded3f35 100644 --- a/nomad/leader.go +++ b/nomad/leader.go @@ -131,6 +131,11 @@ func (s *Server) establishLeadership(stopCh chan struct{}) error { // Enable the blocked eval tracker, since we are now the leader s.blockedEvals.SetEnabled(true) + // Enable the deployment watcher, since we are now the leader + if err := s.deploymentWatcher.SetEnabled(true); err != nil { + return err + } + // Restore the eval broker state if err := s.restoreEvals(); err != nil { return err @@ -483,6 +488,11 @@ func (s *Server) revokeLeadership() error { // Disable the Vault client as it is only useful as a leader. s.vault.SetActive(false) + // Disable the deployment watcher as it is only useful as a leader. + if err := s.deploymentWatcher.SetEnabled(false); err != nil { + return err + } + // Clear the heartbeat timers on either shutdown or step down, // since we are no longer responsible for TTL expirations. if err := s.clearAllHeartbeatTimers(); err != nil { diff --git a/nomad/rpc.go b/nomad/rpc.go index 9ba2156d19c6..b0eeff5150b2 100644 --- a/nomad/rpc.go +++ b/nomad/rpc.go @@ -297,6 +297,9 @@ func (s *Server) raftApplyFuture(t structs.MessageType, msg interface{}) (raft.A return future, nil } +// raftApplyFn is the function signature for applying a msg to Raft +type raftApplyFn func(t structs.MessageType, msg interface{}) (interface{}, uint64, error) + // raftApply is used to encode a message, run it through raft, and return // the FSM response along with any errors func (s *Server) raftApply(t structs.MessageType, msg interface{}) (interface{}, uint64, error) { diff --git a/nomad/server.go b/nomad/server.go index 70844f07fef6..ed07a94d5d40 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -22,6 +22,7 @@ import ( "github.com/hashicorp/go-multierror" "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/tlsutil" + "github.com/hashicorp/nomad/nomad/deploymentwatcher" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/raft" @@ -124,21 +125,25 @@ type Server struct { // eventCh is used to receive events from the serf cluster eventCh chan serf.Event + // BlockedEvals is used to manage evaluations that are blocked on node + // capacity changes. + blockedEvals *BlockedEvals + + // deploymentWatcher is used to watch deployments and their allocations and + // make the required calls to continue to transistion the deployment. + deploymentWatcher *deploymentwatcher.Watcher + // evalBroker is used to manage the in-progress evaluations // that are waiting to be brokered to a sub-scheduler evalBroker *EvalBroker - // BlockedEvals is used to manage evaluations that are blocked on node - // capacity changes. - blockedEvals *BlockedEvals + // periodicDispatcher is used to track and create evaluations for periodic jobs. + periodicDispatcher *PeriodicDispatch // planQueue is used to manage the submitted allocation // plans that are waiting to be assessed by the leader planQueue *PlanQueue - // periodicDispatcher is used to track and create evaluations for periodic jobs. - periodicDispatcher *PeriodicDispatch - // heartbeatTimers track the expiration time of each heartbeat that has // a TTL. On expiration, the node status is updated to be 'down'. heartbeatTimers map[string]*time.Timer @@ -219,22 +224,28 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI, logger *log.Logg incomingTLS = itls } + // Create the deployment watcher + watcher := deploymentwatcher.NewDeploymentsWatcher(logger, + deploymentwatcher.LimitStateQueriesPerSecond, + deploymentwatcher.EvalBatchDuration) + // Create the server s := &Server{ - config: config, - consulCatalog: consulCatalog, - connPool: NewPool(config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap), - logger: logger, - rpcServer: rpc.NewServer(), - peers: make(map[string][]*serverParts), - localPeers: make(map[raft.ServerAddress]*serverParts), - reconcileCh: make(chan serf.Member, 32), - eventCh: make(chan serf.Event, 256), - evalBroker: evalBroker, - blockedEvals: blockedEvals, - planQueue: planQueue, - rpcTLS: incomingTLS, - shutdownCh: make(chan struct{}), + config: config, + consulCatalog: consulCatalog, + connPool: NewPool(config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap), + logger: logger, + rpcServer: rpc.NewServer(), + peers: make(map[string][]*serverParts), + localPeers: make(map[raft.ServerAddress]*serverParts), + reconcileCh: make(chan serf.Member, 32), + eventCh: make(chan serf.Event, 256), + evalBroker: evalBroker, + blockedEvals: blockedEvals, + deploymentWatcher: watcher, + planQueue: planQueue, + rpcTLS: incomingTLS, + shutdownCh: make(chan struct{}), } // Create the periodic dispatcher for launching periodic jobs. @@ -281,6 +292,11 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI, logger *log.Logg return nil, fmt.Errorf("failed to create server Consul syncer: %v", err) } + // Setup the deployment watcher. + if err := s.setupDeploymentWatcher(); err != nil { + return nil, fmt.Errorf("failed to create deployment watcher: %v", err) + } + // Monitor leadership changes go s.monitorLeadership() @@ -663,6 +679,28 @@ func (s *Server) setupConsulSyncer() error { return nil } +// setupDeploymentWatcher creates a deployment watcher that consumes the RPC +// endpoints for state information and makes transistions via Raft through a +// shim that provides the appropriate methods. +func (s *Server) setupDeploymentWatcher() error { + + // Create the shims + stateShim := &deploymentWatcherStateShim{ + evaluations: s.endpoints.Job.Evaluations, + allocations: s.endpoints.Deployment.Allocations, + list: s.endpoints.Deployment.List, + getJobVersions: s.endpoints.Job.GetJobVersions, + getJob: s.endpoints.Job.GetJob, + } + raftShim := &deploymentWatcherRaftShim{ + apply: s.raftApply, + } + + s.deploymentWatcher.SetStateWatchers(stateShim) + s.deploymentWatcher.SetRaftEndpoints(raftShim) + return nil +} + // setupVaultClient is used to set up the Vault API client. func (s *Server) setupVaultClient() error { v, err := NewVaultClient(s.config.VaultConfig, s.logger, s.purgeVaultAccessors) From 6aae18ec1febc4e032ec54831855bcadab7d1c80 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 16:14:50 -0700 Subject: [PATCH 033/105] fix integration slightly --- nomad/deployment_watcher_shims.go | 24 ++++++++++++++++++++++++ nomad/server.go | 1 + nomad/state/schema.go | 2 +- 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/nomad/deployment_watcher_shims.go b/nomad/deployment_watcher_shims.go index ad2dbdcdab00..fe2f22cf5f3d 100644 --- a/nomad/deployment_watcher_shims.go +++ b/nomad/deployment_watcher_shims.go @@ -6,6 +6,10 @@ import "github.com/hashicorp/nomad/nomad/structs" // methods. These should be set by the server and passed to the deployment // watcher. type deploymentWatcherStateShim struct { + // region is the region the server is a member of. It is used to + // auto-populate requests that do not have it set + region string + // evaluations returns the set of evaluations for the given job evaluations func(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error @@ -25,22 +29,42 @@ type deploymentWatcherStateShim struct { } func (d *deploymentWatcherStateShim) Evaluations(args *structs.JobSpecificRequest, reply *structs.JobEvaluationsResponse) error { + if args.Region == "" { + args.Region = d.region + } + return d.evaluations(args, reply) } func (d *deploymentWatcherStateShim) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { + if args.Region == "" { + args.Region = d.region + } + return d.allocations(args, reply) } func (d *deploymentWatcherStateShim) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { + if args.Region == "" { + args.Region = d.region + } + return d.list(args, reply) } func (d *deploymentWatcherStateShim) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { + if args.Region == "" { + args.Region = d.region + } + return d.getJobVersions(args, reply) } func (d *deploymentWatcherStateShim) GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error { + if args.Region == "" { + args.Region = d.region + } + return d.getJob(args, reply) } diff --git a/nomad/server.go b/nomad/server.go index ed07a94d5d40..51a287e8e65e 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -686,6 +686,7 @@ func (s *Server) setupDeploymentWatcher() error { // Create the shims stateShim := &deploymentWatcherStateShim{ + region: s.Region(), evaluations: s.endpoints.Job.Evaluations, allocations: s.endpoints.Deployment.Allocations, list: s.endpoints.Deployment.List, diff --git a/nomad/state/schema.go b/nomad/state/schema.go index 600c551c908c..6d9675c86219 100644 --- a/nomad/state/schema.go +++ b/nomad/state/schema.go @@ -383,7 +383,7 @@ func allocTableSchema() *memdb.TableSchema { // Deployment index is used to lookup allocations by deployment "deployment": &memdb.IndexSchema{ Name: "deployment", - AllowMissing: false, + AllowMissing: true, Unique: false, Indexer: &memdb.UUIDFieldIndex{ Field: "DeploymentID", From 6019915236af960dafe1059a30a50866ddd68117 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 16:29:48 -0700 Subject: [PATCH 034/105] FailDeployment --- nomad/deploymentwatcher/deployment_watcher.go | 23 ++++ .../deploymentwatcher/deployments_watcher.go | 18 +++ .../deployments_watcher_test.go | 110 +++++++++++++----- nomad/structs/structs.go | 1 + 4 files changed, 124 insertions(+), 28 deletions(-) diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 3112982a4dec..49bf46a3c12f 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -234,6 +234,29 @@ func (w *deploymentWatcher) PauseDeployment( return nil } +func (w *deploymentWatcher) FailDeployment( + req *structs.DeploymentSpecificRequest, + resp *structs.DeploymentUpdateResponse) error { + + // Determine the status we should transistion to and if we need to create an + // evaluation + status, desc := structs.DeploymentStatusFailed, structs.DeploymentStatusDescriptionFailedByUser + update, eval := w.getDeploymentStatusUpdate(status, desc), w.getEval() + + // Commit the change + i, err := w.upsertDeploymentStatusUpdate(update, eval, nil) + if err != nil { + return err + } + + // Build the response + resp.EvalID = eval.ID + resp.EvalCreateIndex = i + resp.DeploymentModifyIndex = i + w.setLatestEval(i) + return nil +} + // StopWatch stops watching the deployment. This should be called whenever a // deployment is completed or the watcher is no longer needed. func (w *deploymentWatcher) StopWatch() { diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index 82425e387540..495a8ccbcaea 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -336,6 +336,24 @@ func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest, resp *str return watcher.PauseDeployment(req, resp) } +// FailDeployment is used to fail the deployment. +func (w *Watcher) FailDeployment(req *structs.DeploymentSpecificRequest, resp *structs.DeploymentUpdateResponse) error { + w.l.Lock() + defer w.l.Unlock() + + // Not enabled so no-op + if !w.enabled { + return nil + } + + watcher, ok := w.watchers[req.DeploymentID] + if !ok { + return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + } + + return watcher.FailDeployment(req, resp) +} + // createEvaluation commits the given evaluation to Raft but batches the commit // with other calls. func (w *Watcher) createEvaluation(eval *structs.Evaluation) (uint64, error) { diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 2e1056c63361..97443fbf4316 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -14,11 +14,22 @@ import ( mocker "github.com/stretchr/testify/mock" ) +func testDeploymentWatcher(t *testing.T, qps float64, batchDur time.Duration) (*Watcher, *mockBackend) { + m := newMockBackend(t) + w := NewDeploymentsWatcher(testLogger(), qps, batchDur) + w.SetStateWatchers(m) + w.SetRaftEndpoints(m) + return w, m +} + +func defaultTestDeploymentWatcher(t *testing.T) (*Watcher, *mockBackend) { + return testDeploymentWatcher(t, LimitStateQueriesPerSecond, EvalBatchDuration) +} + // Tests that the watcher properly watches for deployments and reconciles them func TestWatcher_WatchDeployments(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Return no allocations or evals m.On("Allocations", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { @@ -99,8 +110,7 @@ func TestWatcher_WatchDeployments(t *testing.T) { // Tests that calls against an unknown deployment fail func TestWatcher_UnknownDeployment(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) w.SetEnabled(true) // Set up the calls for retrieving deployments @@ -140,13 +150,21 @@ func TestWatcher_UnknownDeployment(t *testing.T) { if assert.NotNil(err, "should have error for unknown deployment") { assert.Contains(err.Error(), "not being watched") } + + // Request failing against an unknown deployment + req4 := &structs.DeploymentSpecificRequest{ + DeploymentID: dID, + } + err = w.FailDeployment(req4, &resp) + if assert.NotNil(err, "should have error for unknown deployment") { + assert.Contains(err.Error(), "not being watched") + } } // Test setting an unknown allocation's health func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job, and a deployment j := mock.Job() @@ -194,8 +212,7 @@ func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { // Test setting allocation health func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job, alloc, and a deployment j := mock.Job() @@ -244,8 +261,7 @@ func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { // Test setting allocation unhealthy func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job, alloc, and a deployment j := mock.Job() @@ -301,8 +317,7 @@ func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { // Test setting allocation unhealthy and that there should be a rollback func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job, alloc, and a deployment j := mock.Job() @@ -370,8 +385,7 @@ func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { // Test promoting a deployment func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job, canary alloc, and a deployment j := mock.Job() @@ -429,8 +443,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { // Test promoting a deployment with unhealthy canaries func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job, canary alloc, and a deployment j := mock.Job() @@ -488,8 +501,7 @@ func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { // Test pausing a deployment that is running func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job and a deployment j := mock.Job() @@ -536,8 +548,7 @@ func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { // Test pausing a deployment that is paused func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job and a deployment j := mock.Job() @@ -585,8 +596,7 @@ func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { // Test unpausing a deployment that is paused func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job and a deployment j := mock.Job() @@ -635,8 +645,7 @@ func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { // Test unpausing a deployment that is running func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, LimitStateQueriesPerSecond, EvalBatchDuration) + w, m := defaultTestDeploymentWatcher(t) // Create a job and a deployment j := mock.Job() @@ -681,12 +690,58 @@ func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) } +// Test failing a deployment that is running +func TestWatcher_FailDeployment_Running(t *testing.T) { + assert := assert.New(t) + w, m := defaultTestDeploymentWatcher(t) + + // Create a job and a deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + + // Assert the following methods will be called + m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) + m.On("Allocations", mocker.MatchedBy(matchDeploymentSpecificRequest(d.ID)), + mocker.Anything).Return(nil).Run(m.allocationsFromState) + m.On("Evaluations", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.evaluationsFromState) + m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + mocker.Anything).Return(nil).Run(m.getJobFromState) + + w.SetEnabled(true) + testutil.WaitForResult(func() (bool, error) { return 1 == len(w.watchers), nil }, + func(err error) { assert.Equal(1, len(w.watchers), "Should have 1 deployment") }) + + // Assert that we get a call to UpsertDeploymentStatusUpdate + matchConfig := &matchDeploymentStatusUpdateConfig{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusFailed, + StatusDescription: structs.DeploymentStatusDescriptionFailedByUser, + Eval: true, + } + matcher := matchDeploymentStatusUpdateRequest(matchConfig) + m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + + // Call PauseDeployment + req := &structs.DeploymentSpecificRequest{ + DeploymentID: d.ID, + } + var resp structs.DeploymentUpdateResponse + err := w.FailDeployment(req, &resp) + assert.Nil(err, "FailDeployment") + + assert.Equal(1, len(w.watchers), "Deployment should still be active") + m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) +} + // Tests that the watcher properly watches for allocation changes and takes the // proper actions func TestDeploymentWatcher_Watch(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, 1000.0, 1*time.Millisecond) + w, m := testDeploymentWatcher(t, 1000.0, 1*time.Millisecond) // Create a job, alloc, and a deployment j := mock.Job() @@ -813,8 +868,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { // Test evaluations are batched between watchers func TestWatcher_BatchEvals(t *testing.T) { assert := assert.New(t) - m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), m, m, 1000.0, EvalBatchDuration) + w, m := testDeploymentWatcher(t, 1000.0, 1*time.Millisecond) // Create a job, alloc, for two deployments j1 := mock.Job() diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 468341441ad7..f4eb91ca010e 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3855,6 +3855,7 @@ const ( DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" DeploymentStatusDescriptionFailedAllocations = "Failed due to unhealthy allocations" + DeploymentStatusDescriptionFailedByUser = "Deployment marked as failed" ) // DeploymentStatusDescriptionRollback is used to get the status description of From d3e02c6d7c7721b99ebc47c9f16a947fff2a44ba Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 28 Jun 2017 22:00:18 -0700 Subject: [PATCH 035/105] Tests --- nomad/deployment_endpoint.go | 194 ++++++- nomad/deployment_endpoint_test.go | 509 +++++++++++++----- nomad/deployment_watcher_shims.go | 15 +- nomad/deploymentwatcher/deployment_watcher.go | 12 +- .../deploymentwatcher/deployments_watcher.go | 120 +++-- .../deployments_watcher_test.go | 10 +- nomad/deploymentwatcher/testutil_test.go | 5 + nomad/mock/mock.go | 2 +- nomad/server.go | 1 + nomad/state/state_store.go | 4 - nomad/structs/structs.go | 35 +- 11 files changed, 704 insertions(+), 203 deletions(-) diff --git a/nomad/deployment_endpoint.go b/nomad/deployment_endpoint.go index 07e985983747..e35377d75022 100644 --- a/nomad/deployment_endpoint.go +++ b/nomad/deployment_endpoint.go @@ -1,6 +1,7 @@ package nomad import ( + "fmt" "time" metrics "github.com/armon/go-metrics" @@ -15,12 +16,201 @@ type Deployment struct { } // TODO http endpoint and api +// GetDeployment is used to request information about a specific deployment +func (d *Deployment) GetDeployment(args *structs.DeploymentSpecificRequest, + reply *structs.SingleDeploymentResponse) error { + if done, err := d.srv.forward("Deployment.GetDeployment", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "get_deployment"}, time.Now()) + + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + run: func(ws memdb.WatchSet, state *state.StateStore) error { + // Verify the arguments + if args.DeploymentID == "" { + return fmt.Errorf("missing deployment ID") + } + + // Look for the deployment + out, err := state.DeploymentByID(ws, args.DeploymentID) + if err != nil { + return err + } + + // Setup the output + reply.Deployment = out + if out != nil { + reply.Index = out.ModifyIndex + } else { + // Use the last index that affected the deployments table + index, err := state.Index("deployment") + if err != nil { + return err + } + reply.Index = index + } + + // Set the query response + d.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return d.srv.blockingRPC(&opts) +} + +// TODO HTTP +// Fail is used to force fail a deployment +func (d *Deployment) Fail(args *structs.DeploymentFailRequest, reply *structs.DeploymentUpdateResponse) error { + if done, err := d.srv.forward("Deployment.Fail", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "fail"}, time.Now()) + + // Validate the arguments + if args.DeploymentID == "" { + return fmt.Errorf("missing deployment ID") + } + + // Lookup the deployment + snap, err := d.srv.fsm.State().Snapshot() + if err != nil { + return err + } + + ws := memdb.NewWatchSet() + deploy, err := snap.DeploymentByID(ws, args.DeploymentID) + if err != nil { + return err + } + if deploy == nil { + return fmt.Errorf("deployment not found") + } + + if !deploy.Active() { + return fmt.Errorf("can't fail terminal deployment") + } + + // Call into the deployment watcher + return d.srv.deploymentWatcher.FailDeployment(args, reply) +} + +// TODO HTTP +// Pause is used to pause a deployment +func (d *Deployment) Pause(args *structs.DeploymentPauseRequest, reply *structs.DeploymentUpdateResponse) error { + if done, err := d.srv.forward("Deployment.Pause", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "pause"}, time.Now()) + + // Validate the arguments + if args.DeploymentID == "" { + return fmt.Errorf("missing deployment ID") + } + + // Lookup the deployment + snap, err := d.srv.fsm.State().Snapshot() + if err != nil { + return err + } + + ws := memdb.NewWatchSet() + deploy, err := snap.DeploymentByID(ws, args.DeploymentID) + if err != nil { + return err + } + if deploy == nil { + return fmt.Errorf("deployment not found") + } + + if !deploy.Active() { + return fmt.Errorf("can't pause terminal deployment") + } + + // Call into the deployment watcher + return d.srv.deploymentWatcher.PauseDeployment(args, reply) +} + +// TODO HTTP +// Promote is used to promote canaries in a deployment +func (d *Deployment) Promote(args *structs.DeploymentPromoteRequest, reply *structs.DeploymentUpdateResponse) error { + if done, err := d.srv.forward("Deployment.Promote", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "promote"}, time.Now()) + + // Validate the arguments + if args.DeploymentID == "" { + return fmt.Errorf("missing deployment ID") + } + + // Lookup the deployment + snap, err := d.srv.fsm.State().Snapshot() + if err != nil { + return err + } + + ws := memdb.NewWatchSet() + deploy, err := snap.DeploymentByID(ws, args.DeploymentID) + if err != nil { + return err + } + if deploy == nil { + return fmt.Errorf("deployment not found") + } + + if !deploy.Active() { + return fmt.Errorf("can't promote terminal deployment") + } + + // Call into the deployment watcher + return d.srv.deploymentWatcher.PromoteDeployment(args, reply) +} + +// TODO HTTP +// Promote is used to promote canaries in a deployment +func (d *Deployment) SetAllocHealth(args *structs.DeploymentAllocHealthRequest, reply *structs.DeploymentUpdateResponse) error { + if done, err := d.srv.forward("Deployment.SetAllocHealth", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "set_alloc_health"}, time.Now()) + + // Validate the arguments + if args.DeploymentID == "" { + return fmt.Errorf("missing deployment ID") + } + + // Lookup the deployment + snap, err := d.srv.fsm.State().Snapshot() + if err != nil { + return err + } + + ws := memdb.NewWatchSet() + deploy, err := snap.DeploymentByID(ws, args.DeploymentID) + if err != nil { + return err + } + if deploy == nil { + return fmt.Errorf("deployment not found") + } + + if !deploy.Active() { + return fmt.Errorf("can't set health of allocations for a terminal deployment") + } + + // Call into the deployment watcher + return d.srv.deploymentWatcher.SetAllocHealth(args, reply) +} + +// TODO HTTP // List returns the list of deployments in the system func (d *Deployment) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { if done, err := d.srv.forward("Deployment.List", args, args, reply); done { return err } - defer metrics.MeasureSince([]string{"nomad", "deployment", "List"}, time.Now()) + defer metrics.MeasureSince([]string{"nomad", "deployment", "list"}, time.Now()) // Setup the blocking query opts := blockingOptions{ @@ -64,7 +254,7 @@ func (d *Deployment) List(args *structs.DeploymentListRequest, reply *structs.De return d.srv.blockingRPC(&opts) } -// TODO http endpoint and api +// TODO HTTP // Allocations returns the list of allocations that are a part of the deployment func (d *Deployment) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { if done, err := d.srv.forward("Deployment.Allocations", args, args, reply); done { diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index f249940cc37a..bd3ddecd3050 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -4,64 +4,350 @@ import ( "testing" "time" + memdb "github.com/hashicorp/go-memdb" msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" + "github.com/stretchr/testify/assert" ) -func TestDeploymentEndpoint_List(t *testing.T) { +func TestDeploymentEndpoint_GetDeployment(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() codec := rpcClient(t, s1) testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) - // Create the register request - deployment := mock.Deployment() + // Create the deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID state := s1.fsm.State() - if err := state.UpsertDeployment(1000, deployment, false); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") // Lookup the deployments - get := &structs.DeploymentListRequest{ + get := &structs.DeploymentSpecificRequest{ + DeploymentID: d.ID, QueryOptions: structs.QueryOptions{Region: "global"}, } - var resp structs.DeploymentListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", get, &resp); err != nil { - t.Fatalf("err: %v", err) + var resp structs.SingleDeploymentResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.GetDeployment", get, &resp), "RPC") + assert.EqualValues(resp.Index, 1000, "resp.Index") + assert.Equal(d, resp.Deployment, "Returned deployment not equal") +} + +func TestDeploymentEndpoint_GetDeployment_Blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + state := s1.fsm.State() + assert := assert.New(t) + + // Create the deployments + j1 := mock.Job() + j2 := mock.Job() + d1 := mock.Deployment() + d1.JobID = j1.ID + d2 := mock.Deployment() + d2.JobID = j2.ID + + assert.Nil(state.UpsertJob(98, j1), "UpsertJob") + assert.Nil(state.UpsertJob(99, j2), "UpsertJob") + + // Upsert a deployment we are not interested in first. + time.AfterFunc(100*time.Millisecond, func() { + assert.Nil(state.UpsertDeployment(100, d1, false), "UpsertDeployment") + }) + + // Upsert another deployment later which should trigger the watch. + time.AfterFunc(200*time.Millisecond, func() { + assert.Nil(state.UpsertDeployment(200, d2, false), "UpsertDeployment") + }) + + // Lookup the deployments + get := &structs.DeploymentSpecificRequest{ + DeploymentID: d2.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 150, + }, } - if resp.Index != 1000 { - t.Fatalf("Bad index: %d %d", resp.Index, 1000) + start := time.Now() + var resp structs.SingleDeploymentResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.GetDeployment", get, &resp), "RPC") + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } + assert.EqualValues(resp.Index, 200, "resp.Index") + assert.Equal(d2, resp.Deployment, "deployments equal") +} + +func TestDeploymentEndpoint_Fail(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // Create the deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + state := s1.fsm.State() + + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Mark the deployment as failed + req := &structs.DeploymentFailRequest{ + DeploymentID: d.ID, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.DeploymentUpdateResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Fail", req, &resp), "RPC") + assert.NotEqual(resp.Index, uint64(0), "bad response index") + + // Lookup the evaluation + ws := memdb.NewWatchSet() + eval, err := state.EvalByID(ws, resp.EvalID) + assert.Nil(err, "EvalByID failed") + assert.NotNil(eval, "Expect eval") + assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerRollingUpdate, "eval trigger") + assert.Equal(eval.JobID, d.JobID, "eval job id") + assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") + assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") + + // Lookup the deployment + dout, err := state.DeploymentByID(ws, d.ID) + assert.Nil(err, "DeploymentByID failed") + assert.Equal(dout.Status, structs.DeploymentStatusFailed, "wrong status") + assert.Equal(dout.StatusDescription, structs.DeploymentStatusDescriptionFailedByUser, "wrong status description") + assert.Equal(dout.ModifyIndex, resp.DeploymentModifyIndex, "wrong modify index") +} + +func TestDeploymentEndpoint_Pause(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // Create the deployment + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + state := s1.fsm.State() + + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Mark the deployment as failed + req := &structs.DeploymentPauseRequest{ + DeploymentID: d.ID, + Pause: true, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.DeploymentUpdateResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Pause", req, &resp), "RPC") + assert.NotEqual(resp.Index, uint64(0), "bad response index") + assert.Zero(resp.EvalCreateIndex, "Shouldn't create eval") + assert.Zero(resp.EvalID, "Shouldn't create eval") + + // Lookup the deployment + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + assert.Nil(err, "DeploymentByID failed") + assert.Equal(dout.Status, structs.DeploymentStatusPaused, "wrong status") + assert.Equal(dout.StatusDescription, structs.DeploymentStatusDescriptionPaused, "wrong status description") + assert.Equal(dout.ModifyIndex, resp.DeploymentModifyIndex, "wrong modify index") +} - if len(resp.Deployments) != 1 { - t.Fatalf("bad: %#v", resp.Deployments) +func TestDeploymentEndpoint_Promote(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // Create the deployment, job and canary + j := mock.Job() + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.Canary = 2 + d := mock.Deployment() + d.TaskGroups["web"].DesiredCanaries = 2 + d.JobID = j.ID + a := mock.Alloc() + a.Canary = true + a.DeploymentID = d.ID + a.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), } - if resp.Deployments[0].ID != deployment.ID { - t.Fatalf("bad: %#v", resp.Deployments[0]) + + state := s1.fsm.State() + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") + + // Promote the deployment + req := &structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.DeploymentUpdateResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Promote", req, &resp), "RPC") + assert.NotEqual(resp.Index, uint64(0), "bad response index") + + // Lookup the evaluation + ws := memdb.NewWatchSet() + eval, err := state.EvalByID(ws, resp.EvalID) + assert.Nil(err, "EvalByID failed") + assert.NotNil(eval, "Expect eval") + assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerRollingUpdate, "eval trigger") + assert.Equal(eval.JobID, d.JobID, "eval job id") + assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") + assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") + + // Lookup the deployment + dout, err := state.DeploymentByID(ws, d.ID) + assert.Nil(err, "DeploymentByID failed") + assert.Equal(dout.Status, structs.DeploymentStatusRunning, "wrong status") + assert.Equal(dout.StatusDescription, structs.DeploymentStatusDescriptionRunning, "wrong status description") + assert.Equal(dout.ModifyIndex, resp.DeploymentModifyIndex, "wrong modify index") + assert.Len(dout.TaskGroups, 1, "should have one group") + assert.Contains(dout.TaskGroups, "web", "should have web group") + assert.True(dout.TaskGroups["web"].Promoted, "web group should be promoted") + + // Lookup the allocation + aout, err := state.AllocByID(ws, a.ID) + assert.Nil(err, "AllocByID") + assert.NotNil(aout, "alloc") + assert.NotNil(aout.DeploymentStatus, "alloc deployment status") + assert.True(aout.DeploymentStatus.Promoted, "alloc deployment promoted") +} + +func TestDeploymentEndpoint_SetAllocHealth(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // Create the deployment, job and canary + j := mock.Job() + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.JobID = j.ID + a.DeploymentID = d.ID + + state := s1.fsm.State() + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") + + // Set the alloc as healthy + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a.ID}, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.DeploymentUpdateResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.SetAllocHealth", req, &resp), "RPC") + assert.NotZero(resp.Index, "bad response index") + + // Lookup the evaluation + ws := memdb.NewWatchSet() + eval, err := state.EvalByID(ws, resp.EvalID) + assert.Nil(err, "EvalByID failed") + assert.NotNil(eval, "Expect eval") + assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerRollingUpdate, "eval trigger") + assert.Equal(eval.JobID, d.JobID, "eval job id") + assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") + assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") + + // Lookup the deployment + dout, err := state.DeploymentByID(ws, d.ID) + assert.Nil(err, "DeploymentByID failed") + assert.Equal(dout.Status, structs.DeploymentStatusRunning, "wrong status") + assert.Equal(dout.StatusDescription, structs.DeploymentStatusDescriptionRunning, "wrong status description") + assert.Equal(resp.DeploymentModifyIndex, dout.ModifyIndex, "wrong modify index") + assert.Len(dout.TaskGroups, 1, "should have one group") + assert.Contains(dout.TaskGroups, "web", "should have web group") + assert.Equal(1, dout.TaskGroups["web"].HealthyAllocs, "should have one healthy") + + // Lookup the allocation + aout, err := state.AllocByID(ws, a.ID) + assert.Nil(err, "AllocByID") + assert.NotNil(aout, "alloc") + assert.NotNil(aout.DeploymentStatus, "alloc deployment status") + assert.NotNil(aout.DeploymentStatus.Healthy, "alloc deployment healthy") + assert.True(*aout.DeploymentStatus.Healthy, "alloc deployment healthy") +} + +func TestDeploymentEndpoint_List(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // Create the register request + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + state := s1.fsm.State() + + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Lookup the deployments + get := &structs.DeploymentListRequest{ + QueryOptions: structs.QueryOptions{Region: "global"}, } + var resp structs.DeploymentListResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.List", get, &resp), "RPC") + assert.EqualValues(resp.Index, 1000, "Wrong Index") + assert.Len(resp.Deployments, 1, "Deployments") + assert.Equal(resp.Deployments[0].ID, d.ID, "Deployment ID") // Lookup the deploys by prefix get = &structs.DeploymentListRequest{ - QueryOptions: structs.QueryOptions{Region: "global", Prefix: deployment.ID[:4]}, + QueryOptions: structs.QueryOptions{Region: "global", Prefix: d.ID[:4]}, } var resp2 structs.DeploymentListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", get, &resp2); err != nil { - t.Fatalf("err: %v", err) - } - if resp2.Index != 1000 { - t.Fatalf("Bad index: %d %d", resp2.Index, 1000) - } - - if len(resp2.Deployments) != 1 { - t.Fatalf("bad: %#v", resp2.Deployments) - } - if resp2.Deployments[0].ID != deployment.ID { - t.Fatalf("bad: %#v", resp2.Deployments[0]) - } + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.List", get, &resp2), "RPC") + assert.EqualValues(resp.Index, 1000, "Wrong Index") + assert.Len(resp2.Deployments, 1, "Deployments") + assert.Equal(resp2.Deployments[0].ID, d.ID, "Deployment ID") } func TestDeploymentEndpoint_List_Blocking(t *testing.T) { @@ -70,15 +356,18 @@ func TestDeploymentEndpoint_List_Blocking(t *testing.T) { state := s1.fsm.State() codec := rpcClient(t, s1) testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) // Create the deployment - deployment := mock.Deployment() + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + + assert.Nil(state.UpsertJob(999, j), "UpsertJob") // Upsert alloc triggers watches time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertDeployment(3, deployment, false); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertDeployment(3, d, false), "UpsertDeployment") }) req := &structs.DeploymentListRequest{ @@ -89,46 +378,31 @@ func TestDeploymentEndpoint_List_Blocking(t *testing.T) { } start := time.Now() var resp structs.DeploymentListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", req, &resp); err != nil { - t.Fatalf("err: %v", err) - } - + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.List", req, &resp), "RPC") + assert.EqualValues(resp.Index, 3, "Wrong Index") + assert.Len(resp.Deployments, 1, "Deployments") + assert.Equal(resp.Deployments[0].ID, d.ID, "Deployment ID") if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 3 { - t.Fatalf("Bad index: %d %d", resp.Index, 3) - } - if len(resp.Deployments) != 1 || resp.Deployments[0].ID != deployment.ID { - t.Fatalf("bad: %#v", resp.Deployments) - } // Deployment updates trigger watches - deployment2 := deployment.Copy() - deployment2.Status = structs.DeploymentStatusPaused + d2 := d.Copy() + d2.Status = structs.DeploymentStatusPaused time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertDeployment(5, deployment2, false); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertDeployment(5, d2, false), "UpsertDeployment") }) req.MinQueryIndex = 3 start = time.Now() var resp2 structs.DeploymentListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.List", req, &resp2); err != nil { - t.Fatalf("err: %v", err) - } - + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.List", req, &resp2), "RPC") + assert.EqualValues(5, resp2.Index, "Wrong Index") + assert.Len(resp2.Deployments, 1, "Deployments") + assert.Equal(d2.ID, resp2.Deployments[0].ID, "Deployment ID") if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } - if resp2.Index != 5 { - t.Fatalf("Bad index: %d %d", resp2.Index, 5) - } - if len(resp2.Deployments) != 1 || resp.Deployments[0].ID != deployment2.ID || - resp2.Deployments[0].Status != structs.DeploymentStatusPaused { - t.Fatalf("bad: %#v", resp2.Deployments) - } } func TestDeploymentEndpoint_Allocations(t *testing.T) { @@ -136,43 +410,32 @@ func TestDeploymentEndpoint_Allocations(t *testing.T) { defer s1.Shutdown() codec := rpcClient(t, s1) testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) // Create the register request - deployment := mock.Deployment() - alloc := mock.Alloc() - alloc.DeploymentID = deployment.ID - summary := mock.JobSummary(alloc.JobID) + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.DeploymentID = d.ID + summary := mock.JobSummary(a.JobID) state := s1.fsm.State() - if err := state.UpsertJobSummary(999, summary); err != nil { - t.Fatalf("err: %v", err) - } - if err := state.UpsertDeployment(1000, deployment, false); err != nil { - t.Fatalf("err: %v", err) - } - if err := state.UpsertAllocs(1001, []*structs.Allocation{alloc}); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertJob(998, j), "UpsertJob") + assert.Nil(state.UpsertJobSummary(999, summary), "UpsertJobSummary") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") // Lookup the allocations get := &structs.DeploymentSpecificRequest{ - DeploymentID: deployment.ID, + DeploymentID: d.ID, QueryOptions: structs.QueryOptions{Region: "global"}, } var resp structs.AllocListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", get, &resp); err != nil { - t.Fatalf("err: %v", err) - } - if resp.Index != 1001 { - t.Fatalf("Bad index: %d %d", resp.Index, 1001) - } - - if len(resp.Allocations) != 1 { - t.Fatalf("bad: %#v", resp.Allocations) - } - if resp.Allocations[0].ID != alloc.ID { - t.Fatalf("bad: %#v", resp.Allocations[0]) - } + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", get, &resp), "RPC") + assert.EqualValues(1001, resp.Index, "Wrong Index") + assert.Len(resp.Allocations, 1, "Allocations") + assert.Equal(a.ID, resp.Allocations[0].ID, "Allocation ID") } func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { @@ -181,29 +444,27 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { state := s1.fsm.State() codec := rpcClient(t, s1) testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) // Create the alloc - deployment := mock.Deployment() - alloc := mock.Alloc() - alloc.DeploymentID = deployment.ID - summary := mock.JobSummary(alloc.JobID) + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.DeploymentID = d.ID + summary := mock.JobSummary(a.JobID) - if err := state.UpsertDeployment(1, deployment, false); err != nil { - t.Fatalf("err: %v", err) - } - if err := state.UpsertJobSummary(2, summary); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertJob(1, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(2, d, false), "UpsertDeployment") + assert.Nil(state.UpsertJobSummary(3, summary), "UpsertJobSummary") // Upsert alloc triggers watches time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertAllocs(3, []*structs.Allocation{alloc}); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertAllocs(4, []*structs.Allocation{a}), "UpsertAllocs") }) req := &structs.DeploymentSpecificRequest{ - DeploymentID: deployment.ID, + DeploymentID: d.ID, QueryOptions: structs.QueryOptions{ Region: "global", MinQueryIndex: 1, @@ -211,47 +472,33 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { } start := time.Now() var resp structs.AllocListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", req, &resp); err != nil { - t.Fatalf("err: %v", err) - } - + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", req, &resp), "RPC") + assert.EqualValues(4, resp.Index, "Wrong Index") + assert.Len(resp.Allocations, 1, "Allocations") + assert.Equal(a.ID, resp.Allocations[0].ID, "Allocation ID") if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 3 { - t.Fatalf("Bad index: %d %d", resp.Index, 3) - } - if len(resp.Allocations) != 1 || resp.Allocations[0].ID != alloc.ID { - t.Fatalf("bad: %#v", resp.Allocations) - } // Client updates trigger watches - alloc2 := mock.Alloc() - alloc2.ID = alloc.ID - alloc2.DeploymentID = alloc.DeploymentID - alloc2.ClientStatus = structs.AllocClientStatusRunning + a2 := mock.Alloc() + a2.ID = a.ID + a2.DeploymentID = a.DeploymentID + a2.ClientStatus = structs.AllocClientStatusRunning time.AfterFunc(100*time.Millisecond, func() { - state.UpsertJobSummary(4, mock.JobSummary(alloc2.JobID)) - if err := state.UpdateAllocsFromClient(5, []*structs.Allocation{alloc2}); err != nil { - t.Fatalf("err: %v", err) - } + assert.Nil(state.UpsertJobSummary(5, mock.JobSummary(a2.JobID)), "UpsertJobSummary") + assert.Nil(state.UpdateAllocsFromClient(6, []*structs.Allocation{a2}), "bpdateAllocsFromClient") }) - req.MinQueryIndex = 3 + req.MinQueryIndex = 4 start = time.Now() var resp2 structs.AllocListResponse - if err := msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", req, &resp2); err != nil { - t.Fatalf("err: %v", err) - } - + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Allocations", req, &resp2), "RPC") + assert.EqualValues(6, resp2.Index, "Wrong Index") + assert.Len(resp2.Allocations, 1, "Allocations") + assert.Equal(a.ID, resp2.Allocations[0].ID, "Allocation ID") + assert.Equal(structs.AllocClientStatusRunning, resp2.Allocations[0].ClientStatus, "Client Status") if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } - if resp2.Index != 5 { - t.Fatalf("Bad index: %d %d", resp2.Index, 5) - } - if len(resp2.Allocations) != 1 || resp.Allocations[0].ID != alloc.ID || - resp2.Allocations[0].ClientStatus != structs.AllocClientStatusRunning { - t.Fatalf("bad: %#v", resp2.Allocations) - } } diff --git a/nomad/deployment_watcher_shims.go b/nomad/deployment_watcher_shims.go index fe2f22cf5f3d..cd8f555ce0de 100644 --- a/nomad/deployment_watcher_shims.go +++ b/nomad/deployment_watcher_shims.go @@ -1,6 +1,8 @@ package nomad -import "github.com/hashicorp/nomad/nomad/structs" +import ( + "github.com/hashicorp/nomad/nomad/structs" +) // deploymentWatcherStateShim is the shim that provides the state watching // methods. These should be set by the server and passed to the deployment @@ -20,6 +22,9 @@ type deploymentWatcherStateShim struct { // list is used to list all the deployments in the system list func(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error + // GetDeployment is used to lookup a particular deployment. + getDeployment func(args *structs.DeploymentSpecificRequest, reply *structs.SingleDeploymentResponse) error + // getJobVersions is used to lookup the versions of a job. This is used when // rolling back to find the latest stable job getJobVersions func(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error @@ -52,6 +57,14 @@ func (d *deploymentWatcherStateShim) List(args *structs.DeploymentListRequest, r return d.list(args, reply) } +func (d *deploymentWatcherStateShim) GetDeployment(args *structs.DeploymentSpecificRequest, reply *structs.SingleDeploymentResponse) error { + if args.Region == "" { + args.Region = d.region + } + + return d.getDeployment(args, reply) +} + func (d *deploymentWatcherStateShim) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { if args.Region == "" { args.Region = d.region diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 49bf46a3c12f..e820959e7d22 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -178,6 +178,7 @@ func (w *deploymentWatcher) SetAllocHealth( resp.EvalID = areq.Eval.ID resp.EvalCreateIndex = index resp.DeploymentModifyIndex = index + resp.Index = index w.setLatestEval(index) return nil } @@ -201,6 +202,7 @@ func (w *deploymentWatcher) PromoteDeployment( resp.EvalID = areq.Eval.ID resp.EvalCreateIndex = index resp.DeploymentModifyIndex = index + resp.Index = index w.setLatestEval(index) return nil } @@ -227,15 +229,18 @@ func (w *deploymentWatcher) PauseDeployment( } // Build the response - resp.EvalID = evalID - resp.EvalCreateIndex = i + if evalID != "" { + resp.EvalID = evalID + resp.EvalCreateIndex = i + } resp.DeploymentModifyIndex = i + resp.Index = i w.setLatestEval(i) return nil } func (w *deploymentWatcher) FailDeployment( - req *structs.DeploymentSpecificRequest, + req *structs.DeploymentFailRequest, resp *structs.DeploymentUpdateResponse) error { // Determine the status we should transistion to and if we need to create an @@ -253,6 +258,7 @@ func (w *deploymentWatcher) FailDeployment( resp.EvalID = eval.ID resp.EvalCreateIndex = i resp.DeploymentModifyIndex = i + resp.Index = i w.setLatestEval(i) return nil } diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index 495a8ccbcaea..2e7abc5557f6 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -46,6 +46,9 @@ type DeploymentStateWatchers interface { // List is used to list all the deployments in the system List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error + // GetDeployment is used to lookup a particular deployment. + GetDeployment(args *structs.DeploymentSpecificRequest, reply *structs.SingleDeploymentResponse) error + // GetJobVersions is used to lookup the versions of a job. This is used when // rolling back to find the latest stable job GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error @@ -64,6 +67,12 @@ const ( EvalBatchDuration = 250 * time.Millisecond ) +var ( + // notEnabled is the error returned when the deployment watcher is not + // enabled + notEnabled = fmt.Errorf("deployment watcher not enabled") +) + // Watcher is used to watch deployments and their allocations created // by the scheduler and trigger the scheduler when allocation health // transistions. @@ -233,15 +242,25 @@ func (w *Watcher) getDeploys(index uint64) (*structs.DeploymentListResponse, err func (w *Watcher) add(d *structs.Deployment) error { w.l.Lock() defer w.l.Unlock() + _, err := w.addLocked(d) + return err +} +// addLocked adds a deployment to the watch list and should only be called when +// locked. +func (w *Watcher) addLocked(d *structs.Deployment) (*deploymentWatcher, error) { // Not enabled so no-op if !w.enabled { - return nil + return nil, nil + } + + if !d.Active() { + return nil, fmt.Errorf("deployment %q is terminal", d.ID) } // Already watched so no-op if _, ok := w.watchers[d.ID]; ok { - return nil + return nil, nil } // Get the job the deployment is referencing @@ -250,14 +269,15 @@ func (w *Watcher) add(d *structs.Deployment) error { } var resp structs.SingleJobResponse if err := w.stateWatchers.GetJob(args, &resp); err != nil { - return err + return nil, err } if resp.Job == nil { - return fmt.Errorf("deployment %q references unknown job %q", d.ID, d.JobID) + return nil, fmt.Errorf("deployment %q references unknown job %q", d.ID, d.JobID) } - w.watchers[d.ID] = newDeploymentWatcher(w.ctx, w.queryLimiter, w.logger, w.stateWatchers, d, resp.Job, w) - return nil + watcher := newDeploymentWatcher(w.ctx, w.queryLimiter, w.logger, w.stateWatchers, d, resp.Job, w) + w.watchers[d.ID] = watcher + return watcher, nil } // remove stops watching a deployment. This can be because the deployment is @@ -277,21 +297,49 @@ func (w *Watcher) remove(d *structs.Deployment) { } } -// SetAllocHealth is used to set the health of allocations for a deployment. If -// there are any unhealthy allocations, the deployment is updated to be failed. -// Otherwise the allocations are updated and an evaluation is created. -func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest, resp *structs.DeploymentUpdateResponse) error { +// forceAdd is used to force a lookup of the given deployment object and create +// a watcher. If the deployment does not exist or is terminal an error is +// returned. +func (w *Watcher) forceAdd(dID string) (*deploymentWatcher, error) { + // Build the request + args := &structs.DeploymentSpecificRequest{DeploymentID: dID} + var resp structs.SingleDeploymentResponse + if err := w.stateWatchers.GetDeployment(args, &resp); err != nil { + return nil, err + } + + if resp.Deployment == nil { + return nil, fmt.Errorf("unknown deployment %q", dID) + } + + return w.addLocked(resp.Deployment) +} + +// getWatcher returns the deployment watcher for the given deployment ID. +func (w *Watcher) getWatcher(dID string) (*deploymentWatcher, error) { w.l.Lock() defer w.l.Unlock() // Not enabled so no-op if !w.enabled { - return nil + return nil, notEnabled } - watcher, ok := w.watchers[req.DeploymentID] - if !ok { - return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + watcher, ok := w.watchers[dID] + if ok { + return watcher, nil + } + + return w.forceAdd(dID) +} + +// SetAllocHealth is used to set the health of allocations for a deployment. If +// there are any unhealthy allocations, the deployment is updated to be failed. +// Otherwise the allocations are updated and an evaluation is created. +func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest, resp *structs.DeploymentUpdateResponse) error { + watcher, err := w.getWatcher(req.DeploymentID) + if err != nil { + return err } return watcher.SetAllocHealth(req, resp) @@ -301,17 +349,9 @@ func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest, resp // deployment is marked as failed. Otherwise the deployment is updated and an // evaluation is created. func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest, resp *structs.DeploymentUpdateResponse) error { - w.l.Lock() - defer w.l.Unlock() - - // Not enabled so no-op - if !w.enabled { - return nil - } - - watcher, ok := w.watchers[req.DeploymentID] - if !ok { - return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + watcher, err := w.getWatcher(req.DeploymentID) + if err != nil { + return err } return watcher.PromoteDeployment(req, resp) @@ -320,35 +360,19 @@ func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest, resp // PauseDeployment is used to toggle the pause state on a deployment. If the // deployment is being unpaused, an evaluation is created. func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest, resp *structs.DeploymentUpdateResponse) error { - w.l.Lock() - defer w.l.Unlock() - - // Not enabled so no-op - if !w.enabled { - return nil - } - - watcher, ok := w.watchers[req.DeploymentID] - if !ok { - return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) + watcher, err := w.getWatcher(req.DeploymentID) + if err != nil { + return err } return watcher.PauseDeployment(req, resp) } // FailDeployment is used to fail the deployment. -func (w *Watcher) FailDeployment(req *structs.DeploymentSpecificRequest, resp *structs.DeploymentUpdateResponse) error { - w.l.Lock() - defer w.l.Unlock() - - // Not enabled so no-op - if !w.enabled { - return nil - } - - watcher, ok := w.watchers[req.DeploymentID] - if !ok { - return fmt.Errorf("deployment %q not being watched for updates", req.DeploymentID) +func (w *Watcher) FailDeployment(req *structs.DeploymentFailRequest, resp *structs.DeploymentUpdateResponse) error { + watcher, err := w.getWatcher(req.DeploymentID) + if err != nil { + return err } return watcher.FailDeployment(req, resp) diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 97443fbf4316..a013811d12b8 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -118,6 +118,10 @@ func TestWatcher_UnknownDeployment(t *testing.T) { reply := args.Get(1).(*structs.DeploymentListResponse) reply.Index = m.nextIndex() }) + m.On("GetDeployment", mocker.Anything, mocker.Anything).Return(nil).Run(func(args mocker.Arguments) { + reply := args.Get(1).(*structs.SingleDeploymentResponse) + reply.Index = m.nextIndex() + }) // Request setting the health against an unknown deployment dID := structs.GenerateUUID() @@ -152,7 +156,7 @@ func TestWatcher_UnknownDeployment(t *testing.T) { } // Request failing against an unknown deployment - req4 := &structs.DeploymentSpecificRequest{ + req4 := &structs.DeploymentFailRequest{ DeploymentID: dID, } err = w.FailDeployment(req4, &resp) @@ -428,7 +432,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { matcher := matchDeploymentPromoteRequest(matchConfig) m.On("UpsertDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) - // Call SetAllocHealth + // Call PromoteDeployment req := &structs.DeploymentPromoteRequest{ DeploymentID: d.ID, All: true, @@ -726,7 +730,7 @@ func TestWatcher_FailDeployment_Running(t *testing.T) { m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) // Call PauseDeployment - req := &structs.DeploymentSpecificRequest{ + req := &structs.DeploymentFailRequest{ DeploymentID: d.ID, } var resp structs.DeploymentUpdateResponse diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index 0c28cba9f5e3..89a9fd95fbba 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -313,6 +313,11 @@ func (m *mockBackend) listFromState(in mocker.Arguments) { reply.Index, _ = m.state.Index("deployment") } +func (m *mockBackend) GetDeployment(args *structs.DeploymentSpecificRequest, reply *structs.SingleDeploymentResponse) error { + rargs := m.Called(args, reply) + return rargs.Error(0) +} + func (m *mockBackend) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { rargs := m.Called(args, reply) return rargs.Error(0) diff --git a/nomad/mock/mock.go b/nomad/mock/mock.go index 8509b83ad889..883b3e4dec02 100644 --- a/nomad/mock/mock.go +++ b/nomad/mock/mock.go @@ -321,7 +321,7 @@ func Deployment() *structs.Deployment { }, }, Status: structs.DeploymentStatusRunning, - StatusDescription: structs.DeploymentStatusRunning, + StatusDescription: structs.DeploymentStatusDescriptionRunning, ModifyIndex: 23, CreateIndex: 21, } diff --git a/nomad/server.go b/nomad/server.go index 51a287e8e65e..a7de186a820b 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -690,6 +690,7 @@ func (s *Server) setupDeploymentWatcher() error { evaluations: s.endpoints.Job.Evaluations, allocations: s.endpoints.Deployment.Allocations, list: s.endpoints.Deployment.List, + getDeployment: s.endpoints.Deployment.GetDeployment, getJobVersions: s.endpoints.Job.GetJobVersions, getJob: s.endpoints.Job.GetJob, } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index d7c5828bd602..6f9660a1a298 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2538,10 +2538,6 @@ func (s *StateStore) updateDeploymentWithAlloc(index uint64, alloc, existing *st // Do not modify in-place. Instead keep track of what must be done placed := 0 - - // TODO test when I am sure of what this method will do - // XXX Unclear whether this will be helpful because a seperate code path is - // likely need for setting health healthy := 0 unhealthy := 0 diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index f4eb91ca010e..ec40deee2cef 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -525,6 +525,8 @@ type DeploymentAllocHealthRequest struct { // Any unhealthy allocations fail the deployment UnhealthyAllocationIDs []string + + WriteRequest } // ApplyDeploymentAllocHealthRequest is used to apply an alloc health request via Raft @@ -552,6 +554,8 @@ type DeploymentPromoteRequest struct { // Groups is used to set the promotion status per task group Groups map[string]bool + + WriteRequest } // ApplyDeploymentPromoteRequest is used to apply a promotion request via Raft @@ -568,6 +572,8 @@ type DeploymentPauseRequest struct { // Pause sets the pause status Pause bool + + WriteRequest } // DeploymentSpecificRequest is used to make a request specific to a particular @@ -577,6 +583,18 @@ type DeploymentSpecificRequest struct { QueryOptions } +// DeploymentFailRequest is used to fail a particular deployment +type DeploymentFailRequest struct { + DeploymentID string + WriteRequest +} + +// SingleDeploymentResponse is used to respond with a single deployment +type SingleDeploymentResponse struct { + Deployment *Deployment + QueryMeta +} + // GenericResponse is used to respond to a request where no // specific response information is needed. type GenericResponse struct { @@ -828,6 +846,7 @@ type DeploymentUpdateResponse struct { EvalID string EvalCreateIndex uint64 DeploymentModifyIndex uint64 + WriteMeta } const ( @@ -3938,7 +3957,7 @@ func (d *Deployment) Active() bool { } func (d *Deployment) GoString() string { - base := fmt.Sprintf("Deployment ID %q for job %q has status %q:", d.ID, d.JobID, d.Status) + base := fmt.Sprintf("Deployment ID %q for job %q has status %q (%v):", d.ID, d.JobID, d.Status, d.StatusDescription) for group, state := range d.TaskGroups { base += fmt.Sprintf("\nTask Group %q has state:\n%#v", group, state) } @@ -3969,15 +3988,11 @@ type DeploymentState struct { func (d *DeploymentState) GoString() string { base := fmt.Sprintf("Desired Total: %d", d.DesiredTotal) - if d.DesiredCanaries > 0 { - base += fmt.Sprintf("\nDesired Canaries: %d", d.DesiredCanaries) - base += fmt.Sprintf("\nPromoted: %v", d.Promoted) - } - if d.PlacedAllocs > 0 { - base := fmt.Sprintf("\nPlaced: %d", d.PlacedAllocs) - base += fmt.Sprintf("\nHealthy: %d", d.HealthyAllocs) - base += fmt.Sprintf("\nUnhealthy: %d", d.UnhealthyAllocs) - } + base += fmt.Sprintf("\nDesired Canaries: %d", d.DesiredCanaries) + base += fmt.Sprintf("\nPromoted: %v", d.Promoted) + base += fmt.Sprintf("\nPlaced: %d", d.PlacedAllocs) + base += fmt.Sprintf("\nHealthy: %d", d.HealthyAllocs) + base += fmt.Sprintf("\nUnhealthy: %d", d.UnhealthyAllocs) return base } From bb0d97ab740c189d6ece6492e7ab2bd758f3d678 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 11:01:41 -0700 Subject: [PATCH 036/105] Fix tests --- .../deploymentwatcher/deployments_watcher_test.go | 15 +++++++++------ nomad/fsm_test.go | 5 +++-- nomad/job_endpoint.go | 14 ++++++++------ nomad/job_endpoint_test.go | 11 ++++++----- 4 files changed, 26 insertions(+), 19 deletions(-) diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index a013811d12b8..56de25c07893 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -123,8 +123,11 @@ func TestWatcher_UnknownDeployment(t *testing.T) { reply.Index = m.nextIndex() }) - // Request setting the health against an unknown deployment + // The expected error is that it should be an unknown deployment dID := structs.GenerateUUID() + expected := fmt.Sprintf("unknown deployment %q", dID) + + // Request setting the health against an unknown deployment req := &structs.DeploymentAllocHealthRequest{ DeploymentID: dID, HealthyAllocationIDs: []string{structs.GenerateUUID()}, @@ -132,7 +135,7 @@ func TestWatcher_UnknownDeployment(t *testing.T) { var resp structs.DeploymentUpdateResponse err := w.SetAllocHealth(req, &resp) if assert.NotNil(err, "should have error for unknown deployment") { - assert.Contains(err.Error(), "not being watched") + assert.Contains(err.Error(), expected) } // Request promoting against an unknown deployment @@ -142,7 +145,7 @@ func TestWatcher_UnknownDeployment(t *testing.T) { } err = w.PromoteDeployment(req2, &resp) if assert.NotNil(err, "should have error for unknown deployment") { - assert.Contains(err.Error(), "not being watched") + assert.Contains(err.Error(), expected) } // Request pausing against an unknown deployment @@ -152,7 +155,7 @@ func TestWatcher_UnknownDeployment(t *testing.T) { } err = w.PauseDeployment(req3, &resp) if assert.NotNil(err, "should have error for unknown deployment") { - assert.Contains(err.Error(), "not being watched") + assert.Contains(err.Error(), expected) } // Request failing against an unknown deployment @@ -161,7 +164,7 @@ func TestWatcher_UnknownDeployment(t *testing.T) { } err = w.FailDeployment(req4, &resp) if assert.NotNil(err, "should have error for unknown deployment") { - assert.Contains(err.Error(), "not being watched") + assert.Contains(err.Error(), expected) } } @@ -872,7 +875,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { // Test evaluations are batched between watchers func TestWatcher_BatchEvals(t *testing.T) { assert := assert.New(t) - w, m := testDeploymentWatcher(t, 1000.0, 1*time.Millisecond) + w, m := testDeploymentWatcher(t, 1000.0, 1*time.Second) // Create a job, alloc, for two deployments j1 := mock.Job() diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index b706f8135058..c06bb93771f6 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -15,6 +15,7 @@ import ( "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" "github.com/hashicorp/raft" + "github.com/kr/pretty" ) type MockSink struct { @@ -1864,7 +1865,7 @@ func TestFSM_ReconcileSummaries(t *testing.T) { JobID: alloc.Job.ID, Summary: map[string]structs.TaskGroupSummary{ "web": structs.TaskGroupSummary{ - Queued: 10, + Queued: 9, Starting: 1, }, }, @@ -1872,6 +1873,6 @@ func TestFSM_ReconcileSummaries(t *testing.T) { ModifyIndex: out2.ModifyIndex, } if !reflect.DeepEqual(&expected, out2) { - t.Fatalf("expected: %#v, actual: %#v", &expected, out2) + t.Fatalf("Diff % #v", pretty.Diff(&expected, out2)) } } diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index e7ead135356a..87cd2ad8cebe 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -769,14 +769,16 @@ func (j *Job) Plan(args *structs.JobPlanRequest, reply *structs.JobPlanResponse) var index uint64 var updatedIndex uint64 - // We want to reused deployments where possible, so only insert the job if - // it has changed or the job didn't exist - if oldJob != nil && oldJob.SpecChanged(args.Job) { + if oldJob != nil { index = oldJob.JobModifyIndex - updatedIndex = oldJob.JobModifyIndex + 1 - // Insert the updated Job into the snapshot - snap.UpsertJob(updatedIndex, args.Job) + // We want to reuse deployments where possible, so only insert the job if + // it has changed or the job didn't exist + if oldJob.SpecChanged(args.Job) { + // Insert the updated Job into the snapshot + updatedIndex = oldJob.JobModifyIndex + 1 + snap.UpsertJob(updatedIndex, args.Job) + } } else if oldJob == nil { // Insert the updated Job into the snapshot snap.UpsertJob(100, args.Job) diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index efb923cfd6ee..ea0c0a5f4871 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -834,7 +834,8 @@ func TestJobEndpoint_Revert(t *testing.T) { t.Fatalf("bad job modify index: %d", resp.JobModifyIndex) } - // Create revert request and don't enforce + // Create revert request and don't enforce. We are at version 2 but it is + // the same as version 0 revertReq = &structs.JobRevertRequest{ JobID: job.ID, JobVersion: 0, @@ -869,8 +870,8 @@ func TestJobEndpoint_Revert(t *testing.T) { if out.Priority != job.Priority { t.Fatalf("priority mis-match") } - if out.Version != 3 { - t.Fatalf("got version %d; want %d", out.Version, 3) + if out.Version != 2 { + t.Fatalf("got version %d; want %d", out.Version, 2) } eout, err := state.EvalByID(ws, resp.EvalID) @@ -888,8 +889,8 @@ func TestJobEndpoint_Revert(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - if len(versions) != 4 { - t.Fatalf("got %d versions; want %d", len(versions), 4) + if len(versions) != 3 { + t.Fatalf("got %d versions; want %d", len(versions), 3) } } From 28e4fe716314ee50c9f65b8ef920aa21d1c13e7a Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Sun, 2 Jul 2017 21:54:40 -0700 Subject: [PATCH 037/105] fix index --- nomad/deploymentwatcher/deployment_watcher.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index e820959e7d22..49e2acfb8f7b 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -271,11 +271,12 @@ func (w *deploymentWatcher) StopWatch() { // watch is the long running watcher that takes actions upon allocation changes func (w *deploymentWatcher) watch() { + allocIndex := uint64(0) for { // Block getting all allocations that are part of the deployment using // the last evaluation index. This will have us block waiting for // something to change past what the scheduler has evaluated. - allocResp, err := w.getAllocs(w.getLatestEval()) + allocResp, err := w.getAllocs(allocIndex) if err != nil { if err == context.Canceled { return @@ -283,6 +284,7 @@ func (w *deploymentWatcher) watch() { w.logger.Printf("[ERR] nomad.deployment_watcher: failed to retrieve allocations for deployment %q: %v", w.d.ID, err) } + allocIndex = allocResp.Index // Get the latest evaluation index latestEval, err := w.latestEvalIndex() From 1a4bd26da4e064c4c22ada66ac1b1926c5203ad5 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 3 Jul 2017 11:08:35 -0700 Subject: [PATCH 038/105] comments on watcher --- nomad/deploymentwatcher/deployment_watcher.go | 50 ++++++++++--------- nomad/state/state_store.go | 10 ++-- nomad/structs/structs.go | 17 ++++--- 3 files changed, 39 insertions(+), 38 deletions(-) diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 49e2acfb8f7b..295fb29b4579 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -13,9 +13,9 @@ import ( ) const ( - // evalBatchPeriod is the batching length before creating an evaluation to + // perJobEvalBatchPeriod is the batching length before creating an evaluation to // trigger the scheduler when allocations are marked as healthy. - evalBatchPeriod = 1 * time.Second + perJobEvalBatchPeriod = 1 * time.Second ) // deploymentTriggers are the set of functions required to trigger changes on @@ -51,7 +51,7 @@ type deploymentWatcher struct { // DeploymentStateWatchers holds the methods required to watch objects for // changes on behalf of the deployment - DeploymentStateWatchers + watchers DeploymentStateWatchers // d is the deployment being watched d *structs.Deployment @@ -86,15 +86,15 @@ func newDeploymentWatcher(parent context.Context, queryLimiter *rate.Limiter, ctx, exitFn := context.WithCancel(parent) w := &deploymentWatcher{ - queryLimiter: queryLimiter, - d: d, - j: j, - autorevert: make(map[string]bool, len(j.TaskGroups)), - DeploymentStateWatchers: watchers, - deploymentTriggers: triggers, - logger: logger, - ctx: ctx, - exitFn: exitFn, + queryLimiter: queryLimiter, + d: d, + j: j, + autorevert: make(map[string]bool, len(j.TaskGroups)), + watchers: watchers, + deploymentTriggers: triggers, + logger: logger, + ctx: ctx, + exitFn: exitFn, } // Determine what task groups will trigger an autorevert @@ -132,10 +132,11 @@ func (w *deploymentWatcher) SetAllocHealth( // Get the allocations for the deployment args := &structs.DeploymentSpecificRequest{DeploymentID: req.DeploymentID} var resp structs.AllocListResponse - if err := w.Allocations(args, &resp); err != nil { + if err := w.watchers.Allocations(args, &resp); err != nil { return err } + // Determine if we should autorevert to an older job desc := structs.DeploymentStatusDescriptionFailedAllocations for _, alloc := range resp.Allocations { // Check that the alloc has been marked unhealthy @@ -246,7 +247,8 @@ func (w *deploymentWatcher) FailDeployment( // Determine the status we should transistion to and if we need to create an // evaluation status, desc := structs.DeploymentStatusFailed, structs.DeploymentStatusDescriptionFailedByUser - update, eval := w.getDeploymentStatusUpdate(status, desc), w.getEval() + update := w.getDeploymentStatusUpdate(status, desc) + eval := w.getEval() // Commit the change i, err := w.upsertDeploymentStatusUpdate(update, eval, nil) @@ -283,6 +285,7 @@ func (w *deploymentWatcher) watch() { } w.logger.Printf("[ERR] nomad.deployment_watcher: failed to retrieve allocations for deployment %q: %v", w.d.ID, err) + return } allocIndex = allocResp.Index @@ -294,6 +297,7 @@ func (w *deploymentWatcher) watch() { } w.logger.Printf("[ERR] nomad.deployment_watcher: failed to determine last evaluation index for job %q: %v", w.d.JobID, err) + return } // Create an evaluation trigger if there is any allocation whose @@ -346,7 +350,8 @@ func (w *deploymentWatcher) watch() { // Update the status of the deployment to failed and create an // evaluation. - e, u := w.getEval(), w.getDeploymentStatusUpdate(structs.DeploymentStatusFailed, desc) + e := w.getEval() + u := w.getDeploymentStatusUpdate(structs.DeploymentStatusFailed, desc) if index, err := w.upsertDeploymentStatusUpdate(u, e, j); err != nil { w.logger.Printf("[ERR] nomad.deployment_watcher: failed to update deployment %q status: %v", w.d.ID, err) } else { @@ -363,7 +368,7 @@ func (w *deploymentWatcher) watch() { func (w *deploymentWatcher) latestStableJob() (*structs.Job, error) { args := &structs.JobSpecificRequest{JobID: w.d.JobID} var resp structs.JobVersionsResponse - if err := w.GetJobVersions(args, &resp); err != nil { + if err := w.watchers.GetJobVersions(args, &resp); err != nil { return nil, err } @@ -389,10 +394,7 @@ func (w *deploymentWatcher) createEvalBatched(forIndex uint64) { w.outstandingBatch = true - go func() { - // Sleep til the batching period is over - time.Sleep(evalBatchPeriod) - + time.AfterFunc(perJobEvalBatchPeriod, func() { // Create the eval evalCreateIndex, err := w.createEvaluation(w.getEval()) if err != nil { @@ -405,7 +407,7 @@ func (w *deploymentWatcher) createEvalBatched(forIndex uint64) { w.outstandingBatch = false w.l.Unlock() - }() + }) } // getEval returns an evaluation suitable for the deployment @@ -414,7 +416,7 @@ func (w *deploymentWatcher) getEval() *structs.Evaluation { ID: structs.GenerateUUID(), Priority: w.j.Priority, Type: w.j.Type, - TriggeredBy: structs.EvalTriggerRollingUpdate, + TriggeredBy: structs.EvalTriggerDeploymentWatcher, JobID: w.j.ID, DeploymentID: w.d.ID, Status: structs.EvalStatusPending, @@ -447,7 +449,7 @@ func (w *deploymentWatcher) getAllocs(index uint64) (*structs.AllocListResponse, return nil, err } - if err := w.Allocations(args, &resp); err != nil { + if err := w.watchers.Allocations(args, &resp); err != nil { return nil, err } } @@ -467,7 +469,7 @@ func (w *deploymentWatcher) latestEvalIndex() (uint64, error) { JobID: w.d.JobID, } var resp structs.JobEvaluationsResponse - err := w.Evaluations(args, &resp) + err := w.watchers.Evaluations(args, &resp) if err != nil { return 0, err } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 6f9660a1a298..3e27c0ee074e 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -92,8 +92,6 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR txn := s.db.Txn(true) defer txn.Abort() - //s.logger.Printf("ALEX: INSERTING %# v", pretty.Formatter(results)) - // Upsert the newly created deployment if results.CreatedDeployment != nil { if err := s.upsertDeploymentImpl(index, results.CreatedDeployment, true, txn); err != nil { @@ -1865,7 +1863,7 @@ func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.Dep txn := s.db.Txn(true) defer txn.Abort() - if err := s.upsertDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { + if err := s.updateDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { return err } @@ -1887,8 +1885,8 @@ func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.Dep return nil } -// upsertDeploymentStatusUpdateImpl is used to upsert deployment status updates -func (s *StateStore) upsertDeploymentStatusUpdateImpl(index uint64, u *structs.DeploymentStatusUpdate, txn *memdb.Txn) error { +// updateDeploymentStatusUpdateImpl is used to upsert deployment status updates +func (s *StateStore) updateDeploymentStatusUpdateImpl(index uint64, u *structs.DeploymentStatusUpdate, txn *memdb.Txn) error { // Retrieve deployment ws := memdb.NewWatchSet() deployment, err := s.deploymentByIDImpl(ws, u.DeploymentID, txn) @@ -2090,7 +2088,7 @@ func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.Appl // Update the deployment status as needed. if req.DeploymentUpdate != nil { - if err := s.upsertDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { + if err := s.updateDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { return err } } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index ec40deee2cef..c1ef63a91a40 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4449,14 +4449,15 @@ const ( ) const ( - EvalTriggerJobRegister = "job-register" - EvalTriggerJobDeregister = "job-deregister" - EvalTriggerPeriodicJob = "periodic-job" - EvalTriggerNodeUpdate = "node-update" - EvalTriggerScheduled = "scheduled" - EvalTriggerRollingUpdate = "rolling-update" - EvalTriggerFailedFollowUp = "failed-follow-up" - EvalTriggerMaxPlans = "max-plan-attempts" + EvalTriggerJobRegister = "job-register" + EvalTriggerJobDeregister = "job-deregister" + EvalTriggerPeriodicJob = "periodic-job" + EvalTriggerNodeUpdate = "node-update" + EvalTriggerScheduled = "scheduled" + EvalTriggerRollingUpdate = "rolling-update" + EvalTriggerDeploymentWatcher = "deployment-watcher" + EvalTriggerFailedFollowUp = "failed-follow-up" + EvalTriggerMaxPlans = "max-plan-attempts" ) const ( From e6a1266696b3c098ca2f6e7a3e9267c5e9693901 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 3 Jul 2017 11:26:45 -0700 Subject: [PATCH 039/105] Remove setters --- .../deploymentwatcher/deployments_watcher.go | 24 ++--------- .../deployments_watcher_test.go | 4 +- nomad/server.go | 42 +++++++++---------- 3 files changed, 25 insertions(+), 45 deletions(-) diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index 2e7abc5557f6..29382a22c2fb 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -110,39 +110,23 @@ type Watcher struct { // NewDeploymentsWatcher returns a deployments watcher that is used to watch // deployments and trigger the scheduler as needed. -func NewDeploymentsWatcher(logger *log.Logger, stateQueriesPerSecond float64, +func NewDeploymentsWatcher(logger *log.Logger, watchers DeploymentStateWatchers, + raft DeploymentRaftEndpoints, stateQueriesPerSecond float64, evalBatchDuration time.Duration) *Watcher { return &Watcher{ + stateWatchers: watchers, + raft: raft, queryLimiter: rate.NewLimiter(rate.Limit(stateQueriesPerSecond), 100), evalBatchDuration: evalBatchDuration, logger: logger, } } -// SetStateWatchers sets the interface for accessing state watchers -func (w *Watcher) SetStateWatchers(watchers DeploymentStateWatchers) { - w.l.Lock() - defer w.l.Unlock() - w.stateWatchers = watchers -} - -// SetRaftEndpoints sets the interface for writing to Raft -func (w *Watcher) SetRaftEndpoints(raft DeploymentRaftEndpoints) { - w.l.Lock() - defer w.l.Unlock() - w.raft = raft -} - // SetEnabled is used to control if the watcher is enabled. The watcher // should only be enabled on the active leader. func (w *Watcher) SetEnabled(enabled bool) error { w.l.Lock() - // Ensure our state is correct - if w.stateWatchers == nil || w.raft == nil { - return fmt.Errorf("State watchers and Raft endpoints must be set before starting") - } - wasEnabled := w.enabled w.enabled = enabled w.l.Unlock() diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 56de25c07893..553ab762821c 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -16,9 +16,7 @@ import ( func testDeploymentWatcher(t *testing.T, qps float64, batchDur time.Duration) (*Watcher, *mockBackend) { m := newMockBackend(t) - w := NewDeploymentsWatcher(testLogger(), qps, batchDur) - w.SetStateWatchers(m) - w.SetRaftEndpoints(m) + w := NewDeploymentsWatcher(testLogger(), m, m, qps, batchDur) return w, m } diff --git a/nomad/server.go b/nomad/server.go index a7de186a820b..095c88a69807 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -224,28 +224,22 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI, logger *log.Logg incomingTLS = itls } - // Create the deployment watcher - watcher := deploymentwatcher.NewDeploymentsWatcher(logger, - deploymentwatcher.LimitStateQueriesPerSecond, - deploymentwatcher.EvalBatchDuration) - // Create the server s := &Server{ - config: config, - consulCatalog: consulCatalog, - connPool: NewPool(config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap), - logger: logger, - rpcServer: rpc.NewServer(), - peers: make(map[string][]*serverParts), - localPeers: make(map[raft.ServerAddress]*serverParts), - reconcileCh: make(chan serf.Member, 32), - eventCh: make(chan serf.Event, 256), - evalBroker: evalBroker, - blockedEvals: blockedEvals, - deploymentWatcher: watcher, - planQueue: planQueue, - rpcTLS: incomingTLS, - shutdownCh: make(chan struct{}), + config: config, + consulCatalog: consulCatalog, + connPool: NewPool(config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap), + logger: logger, + rpcServer: rpc.NewServer(), + peers: make(map[string][]*serverParts), + localPeers: make(map[raft.ServerAddress]*serverParts), + reconcileCh: make(chan serf.Member, 32), + eventCh: make(chan serf.Event, 256), + evalBroker: evalBroker, + blockedEvals: blockedEvals, + planQueue: planQueue, + rpcTLS: incomingTLS, + shutdownCh: make(chan struct{}), } // Create the periodic dispatcher for launching periodic jobs. @@ -698,8 +692,12 @@ func (s *Server) setupDeploymentWatcher() error { apply: s.raftApply, } - s.deploymentWatcher.SetStateWatchers(stateShim) - s.deploymentWatcher.SetRaftEndpoints(raftShim) + // Create the deployment watcher + s.deploymentWatcher = deploymentwatcher.NewDeploymentsWatcher( + s.logger, stateShim, raftShim, + deploymentwatcher.LimitStateQueriesPerSecond, + deploymentwatcher.EvalBatchDuration) + return nil } From 174e4f778068fbbda81f091f1e0551ca3ab1a6f2 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 3 Jul 2017 11:51:58 -0700 Subject: [PATCH 040/105] simplify the batcher's timers --- nomad/deploymentwatcher/batcher.go | 33 +++++++++--------------------- 1 file changed, 10 insertions(+), 23 deletions(-) diff --git a/nomad/deploymentwatcher/batcher.go b/nomad/deploymentwatcher/batcher.go index c6b6d9056b71..4bd529598c52 100644 --- a/nomad/deploymentwatcher/batcher.go +++ b/nomad/deploymentwatcher/batcher.go @@ -58,40 +58,28 @@ func (b *EvalBatcher) CreateEval(e *structs.Evaluation) *EvalFuture { // batcher is the long lived batcher goroutine func (b *EvalBatcher) batcher() { - timer := time.NewTimer(b.batch) + var timerCh <-chan time.Time evals := make(map[string]*structs.Evaluation) for { select { case <-b.ctx.Done(): - timer.Stop() return case e := <-b.inCh: - if len(evals) == 0 { - if !timer.Stop() { - <-timer.C - } - timer.Reset(b.batch) + if timerCh == nil { + timerCh = time.After(b.batch) } evals[e.DeploymentID] = e - case <-timer.C: - if len(evals) == 0 { - // Reset the timer - timer.Reset(b.batch) - continue - } - + case <-timerCh: // Capture the future b.l.Lock() f := b.f b.f = nil b.l.Unlock() - // Shouldn't be possible but protect ourselves + // Shouldn't be possible if f == nil { - // Reset the timer - timer.Reset(b.batch) - continue + panic("no future") } // Capture the evals @@ -100,14 +88,13 @@ func (b *EvalBatcher) batcher() { all = append(all, e) } - // Upsert the evals - f.Set(b.raft.UpsertEvals(all)) + // Upsert the evals in a go routine + go f.Set(b.raft.UpsertEvals(all)) - // Reset the evals list + // Reset the evals list and timer evals = make(map[string]*structs.Evaluation) + timerCh = nil - // Reset the timer - timer.Reset(b.batch) } } } From 0ba6722aed7962757679ae3c8794d93d3990a1e6 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 3 Jul 2017 12:05:01 -0700 Subject: [PATCH 041/105] more comment fixes --- nomad/deployment_endpoint_test.go | 6 +- nomad/deployment_watcher_shims.go | 6 +- .../deploymentwatcher/deployments_watcher.go | 68 +++++++++---------- .../deployments_watcher_test.go | 10 +-- nomad/deploymentwatcher/testutil_test.go | 12 ++-- nomad/fsm.go | 6 +- nomad/server.go | 2 +- nomad/state/state_store.go | 20 +++--- nomad/state/state_store_test.go | 26 +++---- 9 files changed, 78 insertions(+), 78 deletions(-) diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index bd3ddecd3050..7da8dbf13e1e 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -122,7 +122,7 @@ func TestDeploymentEndpoint_Fail(t *testing.T) { assert.Nil(err, "EvalByID failed") assert.NotNil(eval, "Expect eval") assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") - assert.Equal(eval.TriggeredBy, structs.EvalTriggerRollingUpdate, "eval trigger") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerDeploymentWatcher, "eval trigger") assert.Equal(eval.JobID, d.JobID, "eval job id") assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") @@ -223,7 +223,7 @@ func TestDeploymentEndpoint_Promote(t *testing.T) { assert.Nil(err, "EvalByID failed") assert.NotNil(eval, "Expect eval") assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") - assert.Equal(eval.TriggeredBy, structs.EvalTriggerRollingUpdate, "eval trigger") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerDeploymentWatcher, "eval trigger") assert.Equal(eval.JobID, d.JobID, "eval job id") assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") @@ -288,7 +288,7 @@ func TestDeploymentEndpoint_SetAllocHealth(t *testing.T) { assert.Nil(err, "EvalByID failed") assert.NotNil(eval, "Expect eval") assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") - assert.Equal(eval.TriggeredBy, structs.EvalTriggerRollingUpdate, "eval trigger") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerDeploymentWatcher, "eval trigger") assert.Equal(eval.JobID, d.JobID, "eval job id") assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") diff --git a/nomad/deployment_watcher_shims.go b/nomad/deployment_watcher_shims.go index cd8f555ce0de..25de8fb2a1fd 100644 --- a/nomad/deployment_watcher_shims.go +++ b/nomad/deployment_watcher_shims.go @@ -105,17 +105,17 @@ func (d *deploymentWatcherRaftShim) UpsertJob(job *structs.Job) (uint64, error) return index, err } -func (d *deploymentWatcherRaftShim) UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { +func (d *deploymentWatcherRaftShim) UpdateDeploymentStatus(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { _, index, err := d.apply(structs.DeploymentStatusUpdateRequestType, u) return index, err } -func (d *deploymentWatcherRaftShim) UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { +func (d *deploymentWatcherRaftShim) UpdateDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { _, index, err := d.apply(structs.DeploymentPromoteRequestType, req) return index, err } -func (d *deploymentWatcherRaftShim) UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { +func (d *deploymentWatcherRaftShim) UpdateDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { _, index, err := d.apply(structs.DeploymentAllocHealthRequestType, req) return index, err } diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index 29382a22c2fb..8cd5fa428294 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -12,6 +12,22 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +const ( + // LimitStateQueriesPerSecond is the number of state queries allowed per + // second + LimitStateQueriesPerSecond = 100.0 + + // CrossDeploymentEvalBatchDuration is the duration in which evaluations are + // batched across all deployment watchers before commiting to Raft. + CrossDeploymentEvalBatchDuration = 250 * time.Millisecond +) + +var ( + // notEnabled is the error returned when the deployment watcher is not + // enabled + notEnabled = fmt.Errorf("deployment watcher not enabled") +) + // DeploymentRaftEndpoints exposes the deployment watcher to a set of functions // to apply data transforms via Raft. type DeploymentRaftEndpoints interface { @@ -21,16 +37,16 @@ type DeploymentRaftEndpoints interface { // UpsertJob is used to upsert a job UpsertJob(job *structs.Job) (uint64, error) - // UpsertDeploymentStatusUpdate is used to upsert a deployment status update + // UpdateDeploymentStatus is used to make a deployment status update // and potentially create an evaluation. - UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) + UpdateDeploymentStatus(u *structs.DeploymentStatusUpdateRequest) (uint64, error) - // UpsertDeploymentPromotion is used to promote canaries in a deployment - UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) + // UpdateDeploymentPromotion is used to promote canaries in a deployment + UpdateDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) - // UpsertDeploymentAllocHealth is used to set the health of allocations in a + // UpdateDeploymentAllocHealth is used to set the health of allocations in a // deployment - UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) + UpdateDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) } // DeploymentStateWatchers are the set of functions required to watch objects on @@ -57,22 +73,6 @@ type DeploymentStateWatchers interface { GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error } -const ( - // LimitStateQueriesPerSecond is the number of state queries allowed per - // second - LimitStateQueriesPerSecond = 15.0 - - // EvalBatchDuration is the duration in which evaluations are batched before - // commiting to Raft. - EvalBatchDuration = 250 * time.Millisecond -) - -var ( - // notEnabled is the error returned when the deployment watcher is not - // enabled - notEnabled = fmt.Errorf("deployment watcher not enabled") -) - // Watcher is used to watch deployments and their allocations created // by the scheduler and trigger the scheduler when allocation health // transistions. @@ -132,7 +132,7 @@ func (w *Watcher) SetEnabled(enabled bool) error { w.l.Unlock() // Flush the state to create the necessary objects - w.Flush() + w.flush() // If we are starting now, launch the watch daemon if enabled && !wasEnabled { @@ -142,8 +142,8 @@ func (w *Watcher) SetEnabled(enabled bool) error { return nil } -// Flush is used to clear the state of the watcher -func (w *Watcher) Flush() { +// flush is used to clear the state of the watcher +func (w *Watcher) flush() { w.l.Lock() defer w.l.Unlock() @@ -299,8 +299,8 @@ func (w *Watcher) forceAdd(dID string) (*deploymentWatcher, error) { return w.addLocked(resp.Deployment) } -// getWatcher returns the deployment watcher for the given deployment ID. -func (w *Watcher) getWatcher(dID string) (*deploymentWatcher, error) { +// getOrCreateWatcher returns the deployment watcher for the given deployment ID. +func (w *Watcher) getOrCreateWatcher(dID string) (*deploymentWatcher, error) { w.l.Lock() defer w.l.Unlock() @@ -321,7 +321,7 @@ func (w *Watcher) getWatcher(dID string) (*deploymentWatcher, error) { // there are any unhealthy allocations, the deployment is updated to be failed. // Otherwise the allocations are updated and an evaluation is created. func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest, resp *structs.DeploymentUpdateResponse) error { - watcher, err := w.getWatcher(req.DeploymentID) + watcher, err := w.getOrCreateWatcher(req.DeploymentID) if err != nil { return err } @@ -333,7 +333,7 @@ func (w *Watcher) SetAllocHealth(req *structs.DeploymentAllocHealthRequest, resp // deployment is marked as failed. Otherwise the deployment is updated and an // evaluation is created. func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest, resp *structs.DeploymentUpdateResponse) error { - watcher, err := w.getWatcher(req.DeploymentID) + watcher, err := w.getOrCreateWatcher(req.DeploymentID) if err != nil { return err } @@ -344,7 +344,7 @@ func (w *Watcher) PromoteDeployment(req *structs.DeploymentPromoteRequest, resp // PauseDeployment is used to toggle the pause state on a deployment. If the // deployment is being unpaused, an evaluation is created. func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest, resp *structs.DeploymentUpdateResponse) error { - watcher, err := w.getWatcher(req.DeploymentID) + watcher, err := w.getOrCreateWatcher(req.DeploymentID) if err != nil { return err } @@ -354,7 +354,7 @@ func (w *Watcher) PauseDeployment(req *structs.DeploymentPauseRequest, resp *str // FailDeployment is used to fail the deployment. func (w *Watcher) FailDeployment(req *structs.DeploymentFailRequest, resp *structs.DeploymentUpdateResponse) error { - watcher, err := w.getWatcher(req.DeploymentID) + watcher, err := w.getOrCreateWatcher(req.DeploymentID) if err != nil { return err } @@ -379,7 +379,7 @@ func (w *Watcher) upsertDeploymentStatusUpdate( u *structs.DeploymentStatusUpdate, e *structs.Evaluation, j *structs.Job) (uint64, error) { - return w.raft.UpsertDeploymentStatusUpdate(&structs.DeploymentStatusUpdateRequest{ + return w.raft.UpdateDeploymentStatus(&structs.DeploymentStatusUpdateRequest{ DeploymentUpdate: u, Eval: e, Job: j, @@ -388,11 +388,11 @@ func (w *Watcher) upsertDeploymentStatusUpdate( // upsertDeploymentPromotion commits the given deployment promotion to Raft func (w *Watcher) upsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { - return w.raft.UpsertDeploymentPromotion(req) + return w.raft.UpdateDeploymentPromotion(req) } // upsertDeploymentAllocHealth commits the given allocation health changes to // Raft func (w *Watcher) upsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { - return w.raft.UpsertDeploymentAllocHealth(req) + return w.raft.UpdateDeploymentAllocHealth(req) } diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 553ab762821c..14e24a5386aa 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -21,7 +21,7 @@ func testDeploymentWatcher(t *testing.T, qps float64, batchDur time.Duration) (* } func defaultTestDeploymentWatcher(t *testing.T) (*Watcher, *mockBackend) { - return testDeploymentWatcher(t, LimitStateQueriesPerSecond, EvalBatchDuration) + return testDeploymentWatcher(t, LimitStateQueriesPerSecond, CrossDeploymentEvalBatchDuration) } // Tests that the watcher properly watches for deployments and reconciles them @@ -795,7 +795,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { HealthyAllocationIDs: []string{a.ID}, }, } - assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") + assert.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") } // Wait for there to be one eval @@ -834,7 +834,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { UnhealthyAllocationIDs: []string{a.ID}, }, } - assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + assert.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval testutil.WaitForResult(func() (bool, error) { @@ -935,7 +935,7 @@ func TestWatcher_BatchEvals(t *testing.T) { HealthyAllocationIDs: []string{a1.ID}, }, } - assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") + assert.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") req2 := &structs.ApplyDeploymentAllocHealthRequest{ DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ @@ -943,7 +943,7 @@ func TestWatcher_BatchEvals(t *testing.T) { HealthyAllocationIDs: []string{a2.ID}, }, } - assert.Nil(m.state.UpsertDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + assert.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval for each job testutil.WaitForResult(func() (bool, error) { diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index 89a9fd95fbba..631b7de4eb8b 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -83,10 +83,10 @@ func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) { return i, m.state.UpsertJob(i, job) } -func (m *mockBackend) UpsertDeploymentStatusUpdate(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { +func (m *mockBackend) UpdateDeploymentStatus(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { m.Called(u) i := m.nextIndex() - return i, m.state.UpsertDeploymentStatusUpdate(i, u) + return i, m.state.UpdateDeploymentStatus(i, u) } // matchDeploymentStatusUpdateConfig is used to configure the matching @@ -140,10 +140,10 @@ func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) fu } } -func (m *mockBackend) UpsertDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { +func (m *mockBackend) UpdateDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { m.Called(req) i := m.nextIndex() - return i, m.state.UpsertDeploymentPromotion(i, req) + return i, m.state.UpdateDeploymentPromotion(i, req) } // matchDeploymentPromoteRequestConfig is used to configure the matching @@ -170,10 +170,10 @@ func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func( return true } } -func (m *mockBackend) UpsertDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { +func (m *mockBackend) UpdateDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { m.Called(req) i := m.nextIndex() - return i, m.state.UpsertDeploymentAllocHealth(i, req) + return i, m.state.UpdateDeploymentAllocHealth(i, req) } // matchDeploymentAllocHealthRequestConfig is used to configure the matching diff --git a/nomad/fsm.go b/nomad/fsm.go index 56151d024585..066caa08abd5 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -580,7 +580,7 @@ func (n *nomadFSM) applyDeploymentStatusUpdate(buf []byte, index uint64) interfa panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpsertDeploymentStatusUpdate(index, &req); err != nil { + if err := n.state.UpdateDeploymentStatus(index, &req); err != nil { n.logger.Printf("[ERR] nomad.fsm: UpsertDeploymentStatusUpdate failed: %v", err) return err } @@ -600,7 +600,7 @@ func (n *nomadFSM) applyDeploymentPromotion(buf []byte, index uint64) interface{ panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpsertDeploymentPromotion(index, &req); err != nil { + if err := n.state.UpdateDeploymentPromotion(index, &req); err != nil { n.logger.Printf("[ERR] nomad.fsm: UpsertDeploymentPromotion failed: %v", err) return err } @@ -621,7 +621,7 @@ func (n *nomadFSM) applyDeploymentAllocHealth(buf []byte, index uint64) interfac panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpsertDeploymentAllocHealth(index, &req); err != nil { + if err := n.state.UpdateDeploymentAllocHealth(index, &req); err != nil { n.logger.Printf("[ERR] nomad.fsm: UpsertDeploymentAllocHealth failed: %v", err) return err } diff --git a/nomad/server.go b/nomad/server.go index 095c88a69807..19d8a6d054b9 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -696,7 +696,7 @@ func (s *Server) setupDeploymentWatcher() error { s.deploymentWatcher = deploymentwatcher.NewDeploymentsWatcher( s.logger, stateShim, raftShim, deploymentwatcher.LimitStateQueriesPerSecond, - deploymentwatcher.EvalBatchDuration) + deploymentwatcher.CrossDeploymentEvalBatchDuration) return nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 3e27c0ee074e..db85cf52510b 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -1857,13 +1857,13 @@ func (s *StateStore) VaultAccessorsByNode(ws memdb.WatchSet, nodeID string) ([]* return out, nil } -// UpsertDeploymentStatusUpdate is used to upsert deployment status updates and +// UpdateDeploymentStatus is used to make deployment status updates and // potentially make a evaluation -func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.DeploymentStatusUpdateRequest) error { +func (s *StateStore) UpdateDeploymentStatus(index uint64, req *structs.DeploymentStatusUpdateRequest) error { txn := s.db.Txn(true) defer txn.Abort() - if err := s.updateDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { + if err := s.updateDeploymentStatusImpl(index, req.DeploymentUpdate, txn); err != nil { return err } @@ -1885,8 +1885,8 @@ func (s *StateStore) UpsertDeploymentStatusUpdate(index uint64, req *structs.Dep return nil } -// updateDeploymentStatusUpdateImpl is used to upsert deployment status updates -func (s *StateStore) updateDeploymentStatusUpdateImpl(index uint64, u *structs.DeploymentStatusUpdate, txn *memdb.Txn) error { +// updateDeploymentStatusImpl is used to make deployment status updates +func (s *StateStore) updateDeploymentStatusImpl(index uint64, u *structs.DeploymentStatusUpdate, txn *memdb.Txn) error { // Retrieve deployment ws := memdb.NewWatchSet() deployment, err := s.deploymentByIDImpl(ws, u.DeploymentID, txn) @@ -1917,9 +1917,9 @@ func (s *StateStore) updateDeploymentStatusUpdateImpl(index uint64, u *structs.D return nil } -// UpsertDeploymentPromotion is used to promote canaries in a deployment and +// UpdateDeploymentPromotion is used to promote canaries in a deployment and // potentially make a evaluation -func (s *StateStore) UpsertDeploymentPromotion(index uint64, req *structs.ApplyDeploymentPromoteRequest) error { +func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyDeploymentPromoteRequest) error { txn := s.db.Txn(true) defer txn.Abort() @@ -2017,9 +2017,9 @@ func (s *StateStore) UpsertDeploymentPromotion(index uint64, req *structs.ApplyD return nil } -// UpsertDeploymentAllocHealth is used to update the health of allocations as +// UpdateDeploymentAllocHealth is used to update the health of allocations as // part of the deployment and potentially make a evaluation -func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { +func (s *StateStore) UpdateDeploymentAllocHealth(index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { txn := s.db.Txn(true) defer txn.Abort() @@ -2088,7 +2088,7 @@ func (s *StateStore) UpsertDeploymentAllocHealth(index uint64, req *structs.Appl // Update the deployment status as needed. if req.DeploymentUpdate != nil { - if err := s.updateDeploymentStatusUpdateImpl(index, req.DeploymentUpdate, txn); err != nil { + if err := s.updateDeploymentStatusImpl(index, req.DeploymentUpdate, txn); err != nil { return err } } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index c2134c298020..908643c55282 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -4854,7 +4854,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonExistant(t *testing.T) { Status: structs.DeploymentStatusRunning, }, } - err := state.UpsertDeploymentStatusUpdate(2, req) + err := state.UpdateDeploymentStatus(2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error updating the status because the deployment doesn't exist") } @@ -4879,7 +4879,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Terminal(t *testing.T) { Status: structs.DeploymentStatusRunning, }, } - err := state.UpsertDeploymentStatusUpdate(2, req) + err := state.UpdateDeploymentStatus(2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error updating the status because the deployment is terminal") } @@ -4911,7 +4911,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) { Job: j, Eval: e, } - err := state.UpsertDeploymentStatusUpdate(2, req) + err := state.UpdateDeploymentStatus(2, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -4956,7 +4956,7 @@ func TestStateStore_UpsertDeploymentPromotion_NonExistant(t *testing.T) { All: true, }, } - err := state.UpsertDeploymentPromotion(2, req) + err := state.UpdateDeploymentPromotion(2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error promoting because the deployment doesn't exist") } @@ -4981,7 +4981,7 @@ func TestStateStore_UpsertDeploymentPromotion_Terminal(t *testing.T) { All: true, }, } - err := state.UpsertDeploymentPromotion(2, req) + err := state.UpdateDeploymentPromotion(2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error updating the status because the deployment is terminal: %v", err) } @@ -5025,7 +5025,7 @@ func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) { All: true, }, } - err := state.UpsertDeploymentPromotion(4, req) + err := state.UpdateDeploymentPromotion(4, req) if err == nil { t.Fatalf("bad: %v", err) } @@ -5100,7 +5100,7 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { }, Eval: e, } - err := state.UpsertDeploymentPromotion(4, req) + err := state.UpdateDeploymentPromotion(4, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -5214,7 +5214,7 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { }, Eval: e, } - err := state.UpsertDeploymentPromotion(4, req) + err := state.UpdateDeploymentPromotion(4, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -5274,7 +5274,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_NonExistant(t *testing.T) { HealthyAllocationIDs: []string{structs.GenerateUUID()}, }, } - err := state.UpsertDeploymentAllocHealth(2, req) + err := state.UpdateDeploymentAllocHealth(2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error because the deployment doesn't exist: %v", err) } @@ -5299,7 +5299,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Terminal(t *testing.T) { HealthyAllocationIDs: []string{structs.GenerateUUID()}, }, } - err := state.UpsertDeploymentAllocHealth(2, req) + err := state.UpdateDeploymentAllocHealth(2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error because the deployment is terminal: %v", err) } @@ -5322,7 +5322,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_NonExistant(t *testing. HealthyAllocationIDs: []string{structs.GenerateUUID()}, }, } - err := state.UpsertDeploymentAllocHealth(2, req) + err := state.UpdateDeploymentAllocHealth(2, req) if err == nil || !strings.Contains(err.Error(), "unknown alloc") { t.Fatalf("expected error because the alloc doesn't exist: %v", err) } @@ -5357,7 +5357,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_MismatchDeployment(t *t HealthyAllocationIDs: []string{a.ID}, }, } - err := state.UpsertDeploymentAllocHealth(4, req) + err := state.UpdateDeploymentAllocHealth(4, req) if err == nil || !strings.Contains(err.Error(), "not part of deployment") { t.Fatalf("expected error because the alloc isn't part of the deployment: %v", err) } @@ -5407,7 +5407,7 @@ func TestStateStore_UpsertDeploymentAllocHealth(t *testing.T) { Eval: e, DeploymentUpdate: u, } - err := state.UpsertDeploymentAllocHealth(3, req) + err := state.UpdateDeploymentAllocHealth(3, req) if err != nil { t.Fatalf("bad: %v", err) } From 31daf937798089c05957f949d9fb92045940c7ba Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 14:50:47 -0700 Subject: [PATCH 042/105] HTTP Endpoints --- command/agent/deployment_endpoint.go | 194 +++++++++++++++ command/agent/deployment_endpoint_test.go | 284 ++++++++++++++++++++++ command/agent/http.go | 3 + nomad/deployment_endpoint.go | 14 +- 4 files changed, 487 insertions(+), 8 deletions(-) create mode 100644 command/agent/deployment_endpoint.go create mode 100644 command/agent/deployment_endpoint_test.go diff --git a/command/agent/deployment_endpoint.go b/command/agent/deployment_endpoint.go new file mode 100644 index 000000000000..193ebd3d4356 --- /dev/null +++ b/command/agent/deployment_endpoint.go @@ -0,0 +1,194 @@ +package agent + +import ( + "net/http" + "strings" + + "github.com/hashicorp/nomad/nomad/structs" +) + +func (s *HTTPServer) DeploymentsRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + if req.Method != "GET" { + return nil, CodedError(405, ErrInvalidMethod) + } + + args := structs.DeploymentListRequest{} + if s.parse(resp, req, &args.Region, &args.QueryOptions) { + return nil, nil + } + + var out structs.DeploymentListResponse + if err := s.agent.RPC("Deployment.List", &args, &out); err != nil { + return nil, err + } + + setMeta(resp, &out.QueryMeta) + if out.Deployments == nil { + out.Deployments = make([]*structs.Deployment, 0) + } + return out.Deployments, nil +} + +func (s *HTTPServer) DeploymentSpecificRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + path := strings.TrimPrefix(req.URL.Path, "/v1/deployment/") + switch { + case strings.HasPrefix(path, "allocations/"): + deploymentID := strings.TrimPrefix(path, "allocations/") + return s.deploymentAllocations(resp, req, deploymentID) + case strings.HasPrefix(path, "fail/"): + deploymentID := strings.TrimPrefix(path, "fail/") + return s.deploymentFail(resp, req, deploymentID) + case strings.HasPrefix(path, "pause/"): + deploymentID := strings.TrimPrefix(path, "pause/") + return s.deploymentPause(resp, req, deploymentID) + case strings.HasPrefix(path, "promote/"): + deploymentID := strings.TrimPrefix(path, "promote/") + return s.deploymentPromote(resp, req, deploymentID) + case strings.HasPrefix(path, "allocation-health/"): + deploymentID := strings.TrimPrefix(path, "allocation-health/") + return s.deploymentSetAllocHealth(resp, req, deploymentID) + default: + return s.deploymentQuery(resp, req, path) + } +} + +// TODO test and api +func (s *HTTPServer) deploymentFail(resp http.ResponseWriter, req *http.Request, deploymentID string) (interface{}, error) { + if req.Method != "PUT" && req.Method != "POST" { + return nil, CodedError(405, ErrInvalidMethod) + } + args := structs.DeploymentFailRequest{ + DeploymentID: deploymentID, + } + s.parseRegion(req, &args.Region) + + var out structs.DeploymentUpdateResponse + if err := s.agent.RPC("Deployment.Fail", &args, &out); err != nil { + return nil, err + } + setIndex(resp, out.Index) + return out, nil +} + +func (s *HTTPServer) deploymentPause(resp http.ResponseWriter, req *http.Request, deploymentID string) (interface{}, error) { + if req.Method != "PUT" && req.Method != "POST" { + return nil, CodedError(405, ErrInvalidMethod) + } + + var pauseRequest structs.DeploymentPauseRequest + if err := decodeBody(req, &pauseRequest); err != nil { + return nil, CodedError(400, err.Error()) + } + if pauseRequest.DeploymentID == "" { + return nil, CodedError(400, "DeploymentID must be specified") + } + if pauseRequest.DeploymentID != deploymentID { + return nil, CodedError(400, "Deployment ID does not match") + } + s.parseRegion(req, &pauseRequest.Region) + + var out structs.DeploymentUpdateResponse + if err := s.agent.RPC("Deployment.Pause", &pauseRequest, &out); err != nil { + return nil, err + } + setIndex(resp, out.Index) + return out, nil +} + +func (s *HTTPServer) deploymentPromote(resp http.ResponseWriter, req *http.Request, deploymentID string) (interface{}, error) { + if req.Method != "PUT" && req.Method != "POST" { + return nil, CodedError(405, ErrInvalidMethod) + } + + var promoteRequest structs.DeploymentPromoteRequest + if err := decodeBody(req, &promoteRequest); err != nil { + return nil, CodedError(400, err.Error()) + } + if promoteRequest.DeploymentID == "" { + return nil, CodedError(400, "DeploymentID must be specified") + } + if promoteRequest.DeploymentID != deploymentID { + return nil, CodedError(400, "Deployment ID does not match") + } + s.parseRegion(req, &promoteRequest.Region) + + var out structs.DeploymentUpdateResponse + if err := s.agent.RPC("Deployment.Promote", &promoteRequest, &out); err != nil { + return nil, err + } + setIndex(resp, out.Index) + return out, nil +} + +func (s *HTTPServer) deploymentSetAllocHealth(resp http.ResponseWriter, req *http.Request, deploymentID string) (interface{}, error) { + if req.Method != "PUT" && req.Method != "POST" { + return nil, CodedError(405, ErrInvalidMethod) + } + + var healthRequest structs.DeploymentAllocHealthRequest + if err := decodeBody(req, &healthRequest); err != nil { + return nil, CodedError(400, err.Error()) + } + if healthRequest.DeploymentID == "" { + return nil, CodedError(400, "DeploymentID must be specified") + } + if healthRequest.DeploymentID != deploymentID { + return nil, CodedError(400, "Deployment ID does not match") + } + s.parseRegion(req, &healthRequest.Region) + + var out structs.DeploymentUpdateResponse + if err := s.agent.RPC("Deployment.SetAllocHealth", &healthRequest, &out); err != nil { + return nil, err + } + setIndex(resp, out.Index) + return out, nil +} + +func (s *HTTPServer) deploymentAllocations(resp http.ResponseWriter, req *http.Request, deploymentID string) (interface{}, error) { + if req.Method != "GET" { + return nil, CodedError(405, ErrInvalidMethod) + } + + args := structs.DeploymentSpecificRequest{ + DeploymentID: deploymentID, + } + if s.parse(resp, req, &args.Region, &args.QueryOptions) { + return nil, nil + } + + var out structs.AllocListResponse + if err := s.agent.RPC("Deployment.Allocations", &args, &out); err != nil { + return nil, err + } + + setMeta(resp, &out.QueryMeta) + if out.Allocations == nil { + out.Allocations = make([]*structs.AllocListStub, 0) + } + return out.Allocations, nil +} + +func (s *HTTPServer) deploymentQuery(resp http.ResponseWriter, req *http.Request, deploymentID string) (interface{}, error) { + if req.Method != "GET" { + return nil, CodedError(405, ErrInvalidMethod) + } + + args := structs.DeploymentSpecificRequest{ + DeploymentID: deploymentID, + } + if s.parse(resp, req, &args.Region, &args.QueryOptions) { + return nil, nil + } + + var out structs.SingleDeploymentResponse + if err := s.agent.RPC("Deployment.GetDeployment", &args, &out); err != nil { + return nil, err + } + + setMeta(resp, &out.QueryMeta) + if out.Deployment == nil { + return nil, CodedError(404, "deployment not found") + } + return out.Deployment, nil +} diff --git a/command/agent/deployment_endpoint_test.go b/command/agent/deployment_endpoint_test.go new file mode 100644 index 000000000000..8a4ded4a70e9 --- /dev/null +++ b/command/agent/deployment_endpoint_test.go @@ -0,0 +1,284 @@ +package agent + +import ( + "net/http" + "net/http/httptest" + "testing" + + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/assert" +) + +func TestHTTP_DeploymentList(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + d1 := mock.Deployment() + d2 := mock.Deployment() + assert.Nil(state.UpsertDeployment(999, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d2, false), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("GET", "/v1/deployments", nil) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentsRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check for the index + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + assert.Equal("true", respW.HeaderMap.Get("X-Nomad-KnownLeader"), "missing known leader") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-LastContact"), "missing last contact") + + // Check the deployments + deploys := obj.([]*structs.Deployment) + assert.Len(deploys, 2, "Deployments") + }) +} + +func TestHTTP_DeploymentPrefixList(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + d1 := mock.Deployment() + d1.ID = "aaabbbbb-e8f7-fd38-c855-ab94ceb89706" + d2 := mock.Deployment() + d2.ID = "aaabbbbb-e8f7-fd38-c855-ab94ceb89706" + assert.Nil(state.UpsertDeployment(999, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d2, false), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("GET", "/v1/deployments?prefix=aaab", nil) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentsRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check for the index + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + assert.Equal("true", respW.HeaderMap.Get("X-Nomad-KnownLeader"), "missing known leader") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-LastContact"), "missing last contact") + + // Check the deployments + deploys := obj.([]*structs.Deployment) + assert.Len(deploys, 1, "Deployments") + assert.Equal(d1.ID, deploys[0].ID, "Wrong Deployment") + }) +} + +func TestHTTP_DeploymentAllocations(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + a1 := mock.Alloc() + a1.JobID = j.ID + a1.DeploymentID = d.ID + a2 := mock.Alloc() + a2.JobID = j.ID + a2.DeploymentID = d.ID + assert.Nil(state.UpsertJob(998, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(999, d, false), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1000, []*structs.Allocation{a1, a2}), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("GET", "/v1/deployment/allocations/"+d.ID, nil) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentSpecificRequest(respW, req) + assert.Nil(err, "DeploymentSpecificRequest") + + // Check for the index + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + assert.Equal("true", respW.HeaderMap.Get("X-Nomad-KnownLeader"), "missing known leader") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-LastContact"), "missing last contact") + + // Check the ouptput + allocs := obj.([]*structs.AllocListStub) + assert.Len(allocs, 2, "Deployment Allocs") + }) +} + +func TestHTTP_DeploymentQuery(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + d := mock.Deployment() + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("GET", "/v1/deployment/"+d.ID, nil) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentSpecificRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check for the index + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + assert.Equal("true", respW.HeaderMap.Get("X-Nomad-KnownLeader"), "missing known leader") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-LastContact"), "missing last contact") + + // Check the job + out := obj.(*structs.Deployment) + assert.Equal(d.ID, out.ID, "ID mismatch") + }) +} + +func TestHTTP_DeploymentPause(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Create the pause request + args := structs.DeploymentPauseRequest{ + DeploymentID: d.ID, + Pause: false, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + buf := encodeReq(args) + + // Make the HTTP request + req, err := http.NewRequest("PUT", "/v1/deployment/pause/"+d.ID, buf) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentSpecificRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check the response + resp := obj.(structs.DeploymentUpdateResponse) + assert.NotZero(resp.EvalID, "Expect Eval") + assert.NotZero(resp.EvalCreateIndex, "Expect Eval") + assert.NotZero(resp.DeploymentModifyIndex, "Expect Deployment to be Modified") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + }) +} + +func TestHTTP_DeploymentPromote(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(state.UpsertJob(999, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Create the pause request + args := structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + buf := encodeReq(args) + + // Make the HTTP request + req, err := http.NewRequest("PUT", "/v1/deployment/pause/"+d.ID, buf) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentSpecificRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check the response + resp := obj.(structs.DeploymentUpdateResponse) + assert.NotZero(resp.EvalID, "Expect Eval") + assert.NotZero(resp.EvalCreateIndex, "Expect Eval") + assert.NotZero(resp.DeploymentModifyIndex, "Expect Deployment to be Modified") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + }) +} + +func TestHTTP_DeploymentAllocHealth(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + a := mock.Alloc() + a.JobID = j.ID + a.DeploymentID = d.ID + assert.Nil(state.UpsertJob(998, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(999, d, false), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1000, []*structs.Allocation{a}), "UpsertAllocs") + + // Create the pause request + args := structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{a.ID}, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + buf := encodeReq(args) + + // Make the HTTP request + req, err := http.NewRequest("PUT", "/v1/deployment/allocation-health/"+d.ID, buf) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentSpecificRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check the response + resp := obj.(structs.DeploymentUpdateResponse) + assert.NotZero(resp.EvalID, "Expect Eval") + assert.NotZero(resp.EvalCreateIndex, "Expect Eval") + assert.NotZero(resp.DeploymentModifyIndex, "Expect Deployment to be Modified") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + }) +} + +func TestHTTP_DeploymentFail(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Directly manipulate the state + state := s.Agent.server.State() + j := mock.Job() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(state.UpsertJob(998, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(999, d, false), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("PUT", "/v1/deployment/fail/"+d.ID, nil) + assert.Nil(err, "HTTP Request") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.DeploymentSpecificRequest(respW, req) + assert.Nil(err, "Deployment Request") + + // Check the response + resp := obj.(structs.DeploymentUpdateResponse) + assert.NotZero(resp.EvalID, "Expect Eval") + assert.NotZero(resp.EvalCreateIndex, "Expect Eval") + assert.NotZero(resp.DeploymentModifyIndex, "Expect Deployment to be Modified") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + }) +} diff --git a/command/agent/http.go b/command/agent/http.go index d5c7e178985a..d79250ff8367 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -145,6 +145,9 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/evaluations", s.wrap(s.EvalsRequest)) s.mux.HandleFunc("/v1/evaluation/", s.wrap(s.EvalSpecificRequest)) + s.mux.HandleFunc("/v1/deployments", s.wrap(s.DeploymentsRequest)) + s.mux.HandleFunc("/v1/deployment/", s.wrap(s.DeploymentSpecificRequest)) + s.mux.HandleFunc("/v1/client/fs/", s.wrap(s.FsRequest)) s.mux.HandleFunc("/v1/client/stats", s.wrap(s.ClientStatsRequest)) s.mux.HandleFunc("/v1/client/allocation/", s.wrap(s.ClientAllocRequest)) diff --git a/nomad/deployment_endpoint.go b/nomad/deployment_endpoint.go index e35377d75022..be1bfec476e5 100644 --- a/nomad/deployment_endpoint.go +++ b/nomad/deployment_endpoint.go @@ -15,7 +15,6 @@ type Deployment struct { srv *Server } -// TODO http endpoint and api // GetDeployment is used to request information about a specific deployment func (d *Deployment) GetDeployment(args *structs.DeploymentSpecificRequest, reply *structs.SingleDeploymentResponse) error { @@ -60,7 +59,6 @@ func (d *Deployment) GetDeployment(args *structs.DeploymentSpecificRequest, return d.srv.blockingRPC(&opts) } -// TODO HTTP // Fail is used to force fail a deployment func (d *Deployment) Fail(args *structs.DeploymentFailRequest, reply *structs.DeploymentUpdateResponse) error { if done, err := d.srv.forward("Deployment.Fail", args, args, reply); done { @@ -96,7 +94,6 @@ func (d *Deployment) Fail(args *structs.DeploymentFailRequest, reply *structs.De return d.srv.deploymentWatcher.FailDeployment(args, reply) } -// TODO HTTP // Pause is used to pause a deployment func (d *Deployment) Pause(args *structs.DeploymentPauseRequest, reply *structs.DeploymentUpdateResponse) error { if done, err := d.srv.forward("Deployment.Pause", args, args, reply); done { @@ -132,7 +129,6 @@ func (d *Deployment) Pause(args *structs.DeploymentPauseRequest, reply *structs. return d.srv.deploymentWatcher.PauseDeployment(args, reply) } -// TODO HTTP // Promote is used to promote canaries in a deployment func (d *Deployment) Promote(args *structs.DeploymentPromoteRequest, reply *structs.DeploymentUpdateResponse) error { if done, err := d.srv.forward("Deployment.Promote", args, args, reply); done { @@ -168,8 +164,8 @@ func (d *Deployment) Promote(args *structs.DeploymentPromoteRequest, reply *stru return d.srv.deploymentWatcher.PromoteDeployment(args, reply) } -// TODO HTTP -// Promote is used to promote canaries in a deployment +// SetAllocHealth is used to set the health of allocations that are part of the +// deployment. func (d *Deployment) SetAllocHealth(args *structs.DeploymentAllocHealthRequest, reply *structs.DeploymentUpdateResponse) error { if done, err := d.srv.forward("Deployment.SetAllocHealth", args, args, reply); done { return err @@ -181,6 +177,10 @@ func (d *Deployment) SetAllocHealth(args *structs.DeploymentAllocHealthRequest, return fmt.Errorf("missing deployment ID") } + if len(args.HealthyAllocationIDs)+len(args.UnhealthyAllocationIDs) == 0 { + return fmt.Errorf("must specify at least one healthy/unhealthy allocation ID") + } + // Lookup the deployment snap, err := d.srv.fsm.State().Snapshot() if err != nil { @@ -204,7 +204,6 @@ func (d *Deployment) SetAllocHealth(args *structs.DeploymentAllocHealthRequest, return d.srv.deploymentWatcher.SetAllocHealth(args, reply) } -// TODO HTTP // List returns the list of deployments in the system func (d *Deployment) List(args *structs.DeploymentListRequest, reply *structs.DeploymentListResponse) error { if done, err := d.srv.forward("Deployment.List", args, args, reply); done { @@ -254,7 +253,6 @@ func (d *Deployment) List(args *structs.DeploymentListRequest, reply *structs.De return d.srv.blockingRPC(&opts) } -// TODO HTTP // Allocations returns the list of allocations that are a part of the deployment func (d *Deployment) Allocations(args *structs.DeploymentSpecificRequest, reply *structs.AllocListResponse) error { if done, err := d.srv.forward("Deployment.Allocations", args, args, reply); done { From aa5bf9b49eda38cf61cab6066b94d0d27ec0e580 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 15:15:21 -0700 Subject: [PATCH 043/105] deployment api --- api/deployments.go | 230 ++++++++++++++++++++++++++++++++ nomad/state/state_store.go | 13 +- nomad/state/state_store_test.go | 4 +- nomad/structs/structs.go | 2 +- 4 files changed, 241 insertions(+), 8 deletions(-) create mode 100644 api/deployments.go diff --git a/api/deployments.go b/api/deployments.go new file mode 100644 index 000000000000..f8b6b879d010 --- /dev/null +++ b/api/deployments.go @@ -0,0 +1,230 @@ +package api + +import ( + "sort" +) + +// Deployments is used to query the deployments endpoints. +type Deployments struct { + client *Client +} + +// Deployments returns a new handle on the deployments. +func (c *Client) Deployments() *Deployments { + return &Deployments{client: c} +} + +// List is used to dump all of the evaluations. +func (d *Deployments) List(q *QueryOptions) ([]*Deployment, *QueryMeta, error) { + var resp []*Deployment + qm, err := d.client.query("/v1/deployments", &resp, q) + if err != nil { + return nil, nil, err + } + sort.Sort(DeploymentIndexSort(resp)) + return resp, qm, nil +} + +func (d *Deployments) PrefixList(prefix string) ([]*Deployment, *QueryMeta, error) { + return d.List(&QueryOptions{Prefix: prefix}) +} + +// Info is used to query a single evaluation by its ID. +func (d *Deployments) Info(deploymentID string, q *QueryOptions) (*Deployment, *QueryMeta, error) { + var resp Deployment + qm, err := d.client.query("/v1/deployment/"+deploymentID, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, qm, nil +} + +// Allocations is used to retrieve a set of allocations that are part of the +// deployment +func (d *Deployments) Allocations(deploymentID string, q *QueryOptions) ([]*AllocationListStub, *QueryMeta, error) { + var resp []*AllocationListStub + qm, err := d.client.query("/v1/deployment/allocations/"+deploymentID, &resp, q) + if err != nil { + return nil, nil, err + } + sort.Sort(AllocIndexSort(resp)) + return resp, qm, nil +} + +// Fail is used to fail the given deployment. +func (d *Deployments) Fail(deploymentID string, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { + var resp DeploymentUpdateResponse + req := &DeploymentFailRequest{ + DeploymentID: deploymentID, + } + wm, err := d.client.write("/v1/deployment/fail/"+deploymentID, req, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, wm, nil +} + +// Pause is used to pause or unpause the given deployment. +func (d *Deployments) Pause(deploymentID string, pause bool, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { + var resp DeploymentUpdateResponse + req := &DeploymentPauseRequest{ + DeploymentID: deploymentID, + Pause: pause, + } + wm, err := d.client.write("/v1/deployment/pause/"+deploymentID, req, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, wm, nil +} + +// PromoteAll is used to promote all canaries in the given deployment +func (d *Deployments) PromoteAll(deploymentID string, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { + var resp DeploymentUpdateResponse + req := &DeploymentPromoteRequest{ + DeploymentID: deploymentID, + All: true, + } + wm, err := d.client.write("/v1/deployment/promote/"+deploymentID, req, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, wm, nil +} + +// PromoteGroups is used to promote canaries in the passed groups in the given deployment +func (d *Deployments) PromoteGroups(deploymentID string, groups []string, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { + var resp DeploymentUpdateResponse + req := &DeploymentPromoteRequest{ + DeploymentID: deploymentID, + Groups: groups, + } + wm, err := d.client.write("/v1/deployment/promote/"+deploymentID, req, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, wm, nil +} + +// SetAllocHealth is used to set allocation health for allocs that are part of +// the given deployment +func (d *Deployments) SetAllocHealth(deploymentID string, healthy, unhealthy []string, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { + var resp DeploymentUpdateResponse + req := &DeploymentAllocHealthRequest{ + DeploymentID: deploymentID, + HealthyAllocationIDs: healthy, + UnhealthyAllocationIDs: unhealthy, + } + wm, err := d.client.write("/v1/deployment/allocation-health/"+deploymentID, req, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, wm, nil +} + +// Deployment is used to serialize an deployment. +type Deployment struct { + ID string + JobID string + JobVersion uint64 + JobModifyIndex uint64 + JobCreateIndex uint64 + TaskGroups map[string]*DeploymentState + Status string + StatusDescription string + CreateIndex uint64 + ModifyIndex uint64 +} + +// DeploymentState tracks the state of a deployment for a given task group. +type DeploymentState struct { + Promoted bool + DesiredCanaries int + DesiredTotal int + PlacedAllocs int + HealthyAllocs int + UnhealthyAllocs int +} + +// DeploymentIndexSort is a wrapper to sort deployments by CreateIndex. We +// reverse the test so that we get the highest index first. +type DeploymentIndexSort []*Deployment + +func (d DeploymentIndexSort) Len() int { + return len(d) +} + +func (d DeploymentIndexSort) Less(i, j int) bool { + return d[i].CreateIndex > d[j].CreateIndex +} + +func (d DeploymentIndexSort) Swap(i, j int) { + d[i], d[j] = d[j], d[i] +} + +// DeploymentUpdateResponse is used to respond to a deployment change. The +// response will include the modify index of the deployment as well as details +// of any triggered evaluation. +type DeploymentUpdateResponse struct { + EvalID string + EvalCreateIndex uint64 + DeploymentModifyIndex uint64 + WriteMeta +} + +// DeploymentAllocHealthRequest is used to set the health of a set of +// allocations as part of a deployment. +type DeploymentAllocHealthRequest struct { + DeploymentID string + + // Marks these allocations as healthy, allow further allocations + // to be rolled. + HealthyAllocationIDs []string + + // Any unhealthy allocations fail the deployment + UnhealthyAllocationIDs []string + + WriteRequest +} + +// DeploymentPromoteRequest is used to promote task groups in a deployment +type DeploymentPromoteRequest struct { + DeploymentID string + + // All is to promote all task groups + All bool + + // Groups is used to set the promotion status per task group + Groups []string + + WriteRequest +} + +// DeploymentPauseRequest is used to pause a deployment +type DeploymentPauseRequest struct { + DeploymentID string + + // Pause sets the pause status + Pause bool + + WriteRequest +} + +// DeploymentSpecificRequest is used to make a request specific to a particular +// deployment +type DeploymentSpecificRequest struct { + DeploymentID string + QueryOptions +} + +// DeploymentFailRequest is used to fail a particular deployment +type DeploymentFailRequest struct { + DeploymentID string + WriteRequest +} + +// SingleDeploymentResponse is used to respond with a single deployment +type SingleDeploymentResponse struct { + Deployment *Deployment + QueryMeta +} diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index db85cf52510b..6ebdf89d1cf0 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -1940,6 +1940,11 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD return err } + groupIndex := make(map[string]struct{}, len(req.Groups)) + for _, g := range req.Groups { + groupIndex[g] = struct{}{} + } + var unhealthyErr multierror.Error var canaries []*structs.Allocation for { @@ -1954,8 +1959,8 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD } // Check that the canary is part of a group being promoted - inGroup, ok := req.Groups[alloc.TaskGroup] - if !req.All && (!ok || !inGroup) { + _, ok := groupIndex[alloc.TaskGroup] + if !req.All && !ok { continue } @@ -1988,8 +1993,8 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD copy := deployment.Copy() copy.ModifyIndex = index for tg, status := range copy.TaskGroups { - inGroup, ok := req.Groups[tg] - if !req.All && (!ok || !inGroup) { + _, ok := groupIndex[tg] + if !req.All && !ok { continue } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 908643c55282..f9b256f187f5 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -5208,9 +5208,7 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { req := &structs.ApplyDeploymentPromoteRequest{ DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ DeploymentID: d.ID, - Groups: map[string]bool{ - "web": true, - }, + Groups: []string{"web"}, }, Eval: e, } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index c1ef63a91a40..d936b7488e7b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -553,7 +553,7 @@ type DeploymentPromoteRequest struct { All bool // Groups is used to set the promotion status per task group - Groups map[string]bool + Groups []string WriteRequest } From 89d86bc4bdf8fca563719b88e3e93bc27febda5e Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 16:15:13 -0700 Subject: [PATCH 044/105] Deployments HTTP docs --- api/deployments.go | 1 + api/jobs.go | 1 + website/source/api/deployments.html.md | 480 ++++++++++++++++++ website/source/api/index.html.md | 4 +- website/source/api/jobs.html.md | 2 + .../docs/job-specification/group.html.md | 1 + website/source/layouts/api.erb | 4 + 7 files changed, 492 insertions(+), 1 deletion(-) create mode 100644 website/source/api/deployments.html.md diff --git a/api/deployments.go b/api/deployments.go index f8b6b879d010..feeedad8ad54 100644 --- a/api/deployments.go +++ b/api/deployments.go @@ -106,6 +106,7 @@ func (d *Deployments) PromoteGroups(deploymentID string, groups []string, q *Wri return &resp, wm, nil } +// XXX // SetAllocHealth is used to set allocation health for allocs that are part of // the given deployment func (d *Deployments) SetAllocHealth(deploymentID string, healthy, unhealthy []string, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { diff --git a/api/jobs.go b/api/jobs.go index 3766d6b52341..889791485574 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -224,6 +224,7 @@ func (j *Jobs) Dispatch(jobID string, meta map[string]string, return &resp, wm, nil } +// XXX // Revert is used to revert the given job to the passed version. If // enforceVersion is set, the job is only reverted if the current version is at // the passed version. diff --git a/website/source/api/deployments.html.md b/website/source/api/deployments.html.md new file mode 100644 index 000000000000..8154efb604d9 --- /dev/null +++ b/website/source/api/deployments.html.md @@ -0,0 +1,480 @@ +--- +layout: api +page_title: Deployments - HTTP API +sidebar_current: api-deployments +description: |- + The /deployment are used to query for and interact with deployments. +--- + +# Deployments HTTP API + +The `/deployment` endpoints are used to query for and interact with deployments. + +## List Deployments + +This endpoint lists all deployments. + +| Method | Path | Produces | +| ------ | ------------------------ | -------------------------- | +| `GET` | `/v1/deployments` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `YES` | `none` | + +### Parameters + +- `prefix` `(string: "")`- Specifies a string to filter deployments on based on + an index prefix. This is specified as a querystring parameter. + +### Sample Request + +```text +$ curl \ + https://nomad.rocks/v1/deployments +``` + +```text +$ curl \ + https://nomad.rocks/v1/deployments?prefix=25ba81c +``` + +### Sample Response + +```json +[ + { + "ID": "70638f62-5c19-193e-30d6-f9d6e689ab8e", + "JobID": "example", + "JobVersion": 1, + "JobModifyIndex": 17, + "JobCreateIndex": 7, + "TaskGroups": { + "cache": { + "Promoted": false, + "DesiredCanaries": 1, + "DesiredTotal": 3, + "PlacedAllocs": 1, + "HealthyAllocs": 0, + "UnhealthyAllocs": 0 + } + }, + "Status": "running", + "StatusDescription": "", + "CreateIndex": 19, + "ModifyIndex": 19 + } +] +``` + +## Read Deployment + +This endpoint reads information about a specific deployment by ID. + +| Method | Path | Produces | +| ------ | -------------------------------- | -------------------------- | +| `GET` | `/v1/deployment/:deployment_id` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `YES` | `none` | + +### Parameters + +- `:deployment_id` `(string: )`- Specifies the UUID of the deployment. + This must be the full UUID, not the short 8-character one. This is specified + as part of the path. + +### Sample Request + +```text +$ curl \ + https://nomad.rocks/v1/deployment/70638f62-5c19-193e-30d6-f9d6e689ab8e +``` + +### Sample Response + +```json +{ + "ID": "70638f62-5c19-193e-30d6-f9d6e689ab8e", + "JobID": "example", + "JobVersion": 1, + "JobModifyIndex": 17, + "JobCreateIndex": 7, + "TaskGroups": { + "cache": { + "Promoted": false, + "DesiredCanaries": 1, + "DesiredTotal": 3, + "PlacedAllocs": 1, + "HealthyAllocs": 0, + "UnhealthyAllocs": 0 + } + }, + "Status": "running", + "StatusDescription": "", + "CreateIndex": 19, + "ModifyIndex": 19 +} +``` + +## List Allocations for Deployment + +This endpoint lists the allocations created or modified for the given +deployment. + +| Method | Path | Produces | +| ------ | ------------------------------------------- | -------------------------- | +| `GET` | `/v1/deployment/allocations/:deployment_id` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `YES` | `none` | + +### Parameters + +- `:deployment_id` `(string: )`- Specifies the UUID of the deployment. + This must be the full UUID, not the short 8-character one. This is specified + as part of the path. + +### Sample Request + +```text +$ curl \ + https://nomad.rocks/v1/deployment/allocations/5456bd7a-9fc0-c0dd-6131-cbee77f57577 +``` + +### Sample Response + +```json +[ + { + "ID": "287b65cc-6c25-cea9-0332-e4a75ca2af98", + "EvalID": "9751cb74-1a0d-190e-d026-ad2bc666ad2c", + "Name": "example.cache[0]", + "NodeID": "cb1f6030-a220-4f92-57dc-7baaabdc3823", + "JobID": "example", + "TaskGroup": "cache", + "DesiredStatus": "run", + "DesiredDescription": "", + "ClientStatus": "running", + "ClientDescription": "", + "TaskStates": { + "redis": { + "State": "running", + "Failed": false, + "StartedAt": "2017-06-29T22:29:41.52000268Z", + "FinishedAt": "0001-01-01T00:00:00Z", + "Events": [ + { + "Type": "Received", + "Time": 1498775380693307400, + "FailsTask": false, + "RestartReason": "", + "SetupError": "", + "DriverError": "", + "ExitCode": 0, + "Signal": 0, + "Message": "", + "KillTimeout": 0, + "KillError": "", + "KillReason": "", + "StartDelay": 0, + "DownloadError": "", + "ValidationError": "", + "DiskLimit": 0, + "FailedSibling": "", + "VaultError": "", + "TaskSignalReason": "", + "TaskSignal": "", + "DriverMessage": "" + }, + { + "Type": "Task Setup", + "Time": 1498775380693659000, + "FailsTask": false, + "RestartReason": "", + "SetupError": "", + "DriverError": "", + "ExitCode": 0, + "Signal": 0, + "Message": "Building Task Directory", + "KillTimeout": 0, + "KillError": "", + "KillReason": "", + "StartDelay": 0, + "DownloadError": "", + "ValidationError": "", + "DiskLimit": 0, + "FailedSibling": "", + "VaultError": "", + "TaskSignalReason": "", + "TaskSignal": "", + "DriverMessage": "" + }, + { + "Type": "Started", + "Time": 1498775381508493800, + "FailsTask": false, + "RestartReason": "", + "SetupError": "", + "DriverError": "", + "ExitCode": 0, + "Signal": 0, + "Message": "", + "KillTimeout": 0, + "KillError": "", + "KillReason": "", + "StartDelay": 0, + "DownloadError": "", + "ValidationError": "", + "DiskLimit": 0, + "FailedSibling": "", + "VaultError": "", + "TaskSignalReason": "", + "TaskSignal": "", + "DriverMessage": "" + } + ] + } + }, + "DeploymentStatus": null, + "CreateIndex": 19, + "ModifyIndex": 22, + "CreateTime": 1498775380678486300 + } +] +``` + +## Fail Deployment + +This endpoint is used to mark a deployment as failed. This should be done to +force the scheduler to stop creating allocations as part of the deployment or to +cause a rollback to a previous job version. + +| Method | Path | Produces | +| ------- | ------------------------------------ | -------------------------- | +| `POST` | `/v1/deployment/fail/:deployment_id` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `none` | + +### Parameters + +- `:deployment_id` `(string: )`- Specifies the UUID of the deployment. + This must be the full UUID, not the short 8-character one. This is specified + as part of the path. + +### Sample Request + +```text +$ curl \ + --request POST \ + https://nomad.rocks/v1/deployment/fail/5456bd7a-9fc0-c0dd-6131-cbee77f57577 +``` + +### Sample Response + +```json +{ + "EvalID": "0d834913-58a0-81ac-6e33-e452d83a0c66", + "EvalCreateIndex": 20, + "DeploymentModifyIndex": 20, + "Index": 20 +} +``` + +## Pause Deployment + +This endpoint is used to pause or unpause a deployment. This is done to pause +a rolling upgrade or resume it. + +| Method | Path | Produces | +| ------- | ------------------------------------- | -------------------------- | +| `POST` | `/v1/deployment/pause/:deployment_id` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `none` | + +### Parameters + +- `:deployment_id` `(string: )`- Specifies the UUID of the deployment. + This must be the full UUID, not the short 8-character one. This is specified + as part of the path. + +- `Pause` `(bool: false)` - Specifies whether to pause or resume the deployment. + +### Sample Payload + +```javascript +{ + "Pause": true +} +``` + +### Sample Request + +```text +$ curl \ + --request POST \ + https://nomad.rocks/v1/deployment/pause/5456bd7a-9fc0-c0dd-6131-cbee77f57577 +``` + +### Sample Response + +```json +{ + "EvalID": "0d834913-58a0-81ac-6e33-e452d83a0c66", + "EvalCreateIndex": 20, + "DeploymentModifyIndex": 20, + "Index": 20 +} +``` + +## Promote Deployment + +This endpoint is used to promote task groups that have canaries for a +deployment. This should be done when the placed canaries are healthy and the +rolling upgrade of the remaining allocations should begin. + +| Method | Path | Produces | +| ------- | ------------------------------------- | -------------------------- | +| `POST` | `/v1/deployment/promote/:deployment_id` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `none` | + +### Parameters + +- `:deployment_id` `(string: )`- Specifies the UUID of the deployment. + This must be the full UUID, not the short 8-character one. This is specified + as part of the path. + +- `All` `(bool: false)` - Specifies whether all task groups should be promoted. + +- `Groups` `(array: nil)` - Specifies a particular set of task groups + that should be promoted. + +### Sample Payload + +```javascript +{ + "All": true +} +``` + +```javascript +{ + "Groups": ["web", "api-server"] +} +``` + +### Sample Request + +```text +$ curl \ + --request POST \ + https://nomad.rocks/v1/deployment/promote/5456bd7a-9fc0-c0dd-6131-cbee77f57577 +``` + +### Sample Response + +```json +{ + "EvalID": "0d834913-58a0-81ac-6e33-e452d83a0c66", + "EvalCreateIndex": 20, + "DeploymentModifyIndex": 20, + "Index": 20 +} +``` + +## Set Allocation Health in Deployment + +This endpoint is used to set the health of an allocation that is in the +deployment manually. In some use cases, automatic detection of allocation health +may not be desired. As such those task groups can be marked with an upgrade +policy that uses `health_check = "manual"`. Those allocations must have their +health marked manually using this endpoint. Marking an allocation as healthy +will allow the rolling upgrade to proceed. Marking it as failed will cause the +deployment to fail. + +| Method | Path | Produces | +| ------- | ------------------------------------------------- | -------------------------- | +| `POST` | `/v1/deployment/allocation-health/:deployment_id` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `none` | + +### Parameters + +- `:deployment_id` `(string: )`- Specifies the UUID of the deployment. + This must be the full UUID, not the short 8-character one. This is specified + as part of the path. + +- `HealthyAllocationIDs` `(array: nil)` - Specifies the set of + allocation that should be marked as healthy. + +- `UnhealthyAllocationIDs` `(array: nil)` - Specifies the set of + allocation that should be marked as unhealthy. + +### Sample Payload + +```javascript +{ + "HealthyAllocationIDs": [ + "eb13bc8a-7300-56f3-14c0-d4ad115ec3f5", + "6584dad8-7ae3-360f-3069-0b4309711cc1" + ] +} +``` + +### Sample Request + +```text +$ curl \ + --request POST \ + https://nomad.rocks/v1/deployment/allocation-health/5456bd7a-9fc0-c0dd-6131-cbee77f57577 +``` + +### Sample Response + +```json +{ + "EvalID": "0d834913-58a0-81ac-6e33-e452d83a0c66", + "EvalCreateIndex": 20, + "DeploymentModifyIndex": 20, + "Index": 20 +} +``` diff --git a/website/source/api/index.html.md b/website/source/api/index.html.md index c2e3fabd1c3a..8b8fa4f08949 100644 --- a/website/source/api/index.html.md +++ b/website/source/api/index.html.md @@ -52,6 +52,7 @@ There are four primary nouns in Nomad: - jobs - nodes - allocations +- deployments - evaluations [![Nomad Data Model](/assets/images/nomad-data-model.png)](/assets/images/nomad-data-model.png) @@ -62,7 +63,8 @@ require resources. Nodes are the servers in the clusters that tasks can be scheduled on. The mapping of tasks in a job to nodes is done using allocations. An allocation is used to declare that a set of tasks in a job should be run on a particular node. Scheduling is the process of determining the appropriate -allocations and is done as part of an evaluation. +allocations and is done as part of an evaluation. Deployments are objects to +track a rolling update of allocations between two versions of a job. The API is modeled closely on the underlying data model. Use the links to the left for documentation about specific endpoints. There are also "Agent" APIs diff --git a/website/source/api/jobs.html.md b/website/source/api/jobs.html.md index 7d4af5be5307..317d2f0825f6 100644 --- a/website/source/api/jobs.html.md +++ b/website/source/api/jobs.html.md @@ -887,6 +887,8 @@ The table below shows this endpoint's support for ```text $ curl \ + --request POST \ + --payload @payload.json \ https://nomad.rocks/v1/job/my-job/summary ``` diff --git a/website/source/docs/job-specification/group.html.md b/website/source/docs/job-specification/group.html.md index d5b075e94864..c202a8c29363 100644 --- a/website/source/docs/job-specification/group.html.md +++ b/website/source/docs/job-specification/group.html.md @@ -113,3 +113,4 @@ group "example" { [ephemeraldisk]: /docs/job-specification/ephemeral_disk.html "Nomad ephemeral_disk Job Specification" [meta]: /docs/job-specification/meta.html "Nomad meta Job Specification" [restart]: /docs/job-specification/restart.html "Nomad restart Job Specification" +[vault]: /docs/job-specification/vault.html "Nomad vault Job Specification" diff --git a/website/source/layouts/api.erb b/website/source/layouts/api.erb index 0cf29528ffa1..cf0f9fbdd302 100644 --- a/website/source/layouts/api.erb +++ b/website/source/layouts/api.erb @@ -27,6 +27,10 @@ Client + > + Deployments + + > Evaluations From d0a83321403a33cb9953c578272d620d2bbf6e64 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 17:23:34 -0700 Subject: [PATCH 045/105] job deployment endpoint + api --- api/jobs.go | 25 ++++- command/agent/job_endpoint.go | 51 ++++++++++ command/agent/job_endpoint_test.go | 77 +++++++++++++++ nomad/job_endpoint.go | 80 ++++++++++++++- nomad/job_endpoint_test.go | 150 +++++++++++++++++++++++++++++ 5 files changed, 380 insertions(+), 3 deletions(-) diff --git a/api/jobs.go b/api/jobs.go index 889791485574..9981577d4a13 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -138,8 +138,31 @@ func (j *Jobs) Allocations(jobID string, allAllocs bool, q *QueryOptions) ([]*Al return resp, qm, nil } -// Evaluations is used to query the evaluations associated with +// Deployments is used to query the deployments associated with the given job +// ID. +func (j *Jobs) Deployments(jobID string, q *QueryOptions) ([]*Deployment, *QueryMeta, error) { + var resp []*Deployment + qm, err := j.client.query("/v1/job/"+jobID+"/deployments", &resp, q) + if err != nil { + return nil, nil, err + } + sort.Sort(DeploymentIndexSort(resp)) + return resp, qm, nil +} + +// LatestDeployment is used to query for the latest deployment associated with // the given job ID. +func (j *Jobs) LatestDeployment(jobID string, q *QueryOptions) (*Deployment, *QueryMeta, error) { + var resp *Deployment + qm, err := j.client.query("/v1/job/"+jobID+"/deployment", &resp, q) + if err != nil { + return nil, nil, err + } + return resp, qm, nil +} + +// Evaluations is used to query the evaluations associated with the given job +// ID. func (j *Jobs) Evaluations(jobID string, q *QueryOptions) ([]*Evaluation, *QueryMeta, error) { var resp []*Evaluation qm, err := j.client.query("/v1/job/"+jobID+"/evaluations", &resp, q) diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index b5bed9142373..4583070b5dcc 100644 --- a/command/agent/job_endpoint.go +++ b/command/agent/job_endpoint.go @@ -70,6 +70,12 @@ func (s *HTTPServer) JobSpecificRequest(resp http.ResponseWriter, req *http.Requ case strings.HasSuffix(path, "/revert"): jobName := strings.TrimSuffix(path, "/revert") return s.jobRevert(resp, req, jobName) + case strings.HasSuffix(path, "/deployments"): + jobName := strings.TrimSuffix(path, "/deployments") + return s.jobDeployments(resp, req, jobName) + case strings.HasSuffix(path, "/deployment"): + jobName := strings.TrimSuffix(path, "/deployment") + return s.jobLatestDeployment(resp, req, jobName) default: return s.jobCRUD(resp, req, path) } @@ -231,6 +237,51 @@ func (s *HTTPServer) jobEvaluations(resp http.ResponseWriter, req *http.Request, return out.Evaluations, nil } +func (s *HTTPServer) jobDeployments(resp http.ResponseWriter, req *http.Request, + jobName string) (interface{}, error) { + if req.Method != "GET" { + return nil, CodedError(405, ErrInvalidMethod) + } + args := structs.JobSpecificRequest{ + JobID: jobName, + } + if s.parse(resp, req, &args.Region, &args.QueryOptions) { + return nil, nil + } + + var out structs.DeploymentListResponse + if err := s.agent.RPC("Job.Deployments", &args, &out); err != nil { + return nil, err + } + + setMeta(resp, &out.QueryMeta) + if out.Deployments == nil { + out.Deployments = make([]*structs.Deployment, 0) + } + return out.Deployments, nil +} + +func (s *HTTPServer) jobLatestDeployment(resp http.ResponseWriter, req *http.Request, + jobName string) (interface{}, error) { + if req.Method != "GET" { + return nil, CodedError(405, ErrInvalidMethod) + } + args := structs.JobSpecificRequest{ + JobID: jobName, + } + if s.parse(resp, req, &args.Region, &args.QueryOptions) { + return nil, nil + } + + var out structs.SingleDeploymentResponse + if err := s.agent.RPC("Job.LatestDeployment", &args, &out); err != nil { + return nil, err + } + + setMeta(resp, &out.QueryMeta) + return out.Deployment, nil +} + func (s *HTTPServer) jobCRUD(resp http.ResponseWriter, req *http.Request, jobName string) (interface{}, error) { switch req.Method { diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index 3d189773a9ce..fccc33a2e548 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/kr/pretty" + "github.com/stretchr/testify/assert" ) func TestHTTP_JobsList(t *testing.T) { @@ -602,6 +603,82 @@ func TestHTTP_JobAllocations(t *testing.T) { }) } +func TestHTTP_JobDeployments(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Create the job + j := mock.Job() + args := structs.JobRegisterRequest{ + Job: j, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + var resp structs.JobRegisterResponse + assert.Nil(s.Agent.RPC("Job.Register", &args, &resp), "JobRegister") + + // Directly manipulate the state + state := s.Agent.server.State() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("GET", "/v1/job/"+j.ID+"/deployments", nil) + assert.Nil(err, "HTTP") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.JobSpecificRequest(respW, req) + assert.Nil(err, "JobSpecificRequest") + + // Check the response + deploys := obj.([]*structs.Deployment) + assert.Len(deploys, 1, "deployments") + assert.Equal(d.ID, deploys[0].ID, "deployment id") + + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + assert.Equal("true", respW.HeaderMap.Get("X-Nomad-KnownLeader"), "missing known leader") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-LastContact"), "missing last contact") + }) +} + +func TestHTTP_JobDeployment(t *testing.T) { + assert := assert.New(t) + httpTest(t, nil, func(s *TestServer) { + // Create the job + j := mock.Job() + args := structs.JobRegisterRequest{ + Job: j, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + var resp structs.JobRegisterResponse + assert.Nil(s.Agent.RPC("Job.Register", &args, &resp), "JobRegister") + + // Directly manipulate the state + state := s.Agent.server.State() + d := mock.Deployment() + d.JobID = j.ID + assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + + // Make the HTTP request + req, err := http.NewRequest("GET", "/v1/job/"+j.ID+"/deployment", nil) + assert.Nil(err, "HTTP") + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.JobSpecificRequest(respW, req) + assert.Nil(err, "JobSpecificRequest") + + // Check the response + out := obj.(*structs.Deployment) + assert.NotNil(out, "deployment") + assert.Equal(d.ID, out.ID, "deployment id") + + assert.NotZero(respW.HeaderMap.Get("X-Nomad-Index"), "missing index") + assert.Equal("true", respW.HeaderMap.Get("X-Nomad-KnownLeader"), "missing known leader") + assert.NotZero(respW.HeaderMap.Get("X-Nomad-LastContact"), "missing last contact") + }) +} + func TestHTTP_JobVersions(t *testing.T) { httpTest(t, nil, func(s *TestServer) { // Create the job diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 87cd2ad8cebe..9431b058dfbc 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -3,6 +3,7 @@ package nomad import ( "context" "fmt" + "sort" "strings" "time" @@ -143,7 +144,7 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis args.Job.VaultToken = "" // Check if the job has changed at all - if currentJob == nil || currentJob.SpecChanged(args.Job) { + if existingJob == nil || existingJob.SpecChanged(args.Job) { // Commit this update via Raft _, index, err := j.srv.raftApply(structs.JobRegisterRequestType, args) if err != nil { @@ -154,7 +155,7 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis // Populate the reply with job information reply.JobModifyIndex = index } else { - reply.JobModifyIndex = currentJob.JobModifyIndex + reply.JobModifyIndex = existingJob.JobModifyIndex } // If the job is periodic or parameterized, we don't create an eval. @@ -726,6 +727,81 @@ func (j *Job) Evaluations(args *structs.JobSpecificRequest, return j.srv.blockingRPC(&opts) } +// Deployments is used to list the deployments for a job +func (j *Job) Deployments(args *structs.JobSpecificRequest, + reply *structs.DeploymentListResponse) error { + if done, err := j.srv.forward("Job.Deployments", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "job", "deployments"}, time.Now()) + + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + run: func(ws memdb.WatchSet, state *state.StateStore) error { + // Capture the deployments + deploys, err := state.DeploymentsByJobID(ws, args.JobID) + if err != nil { + return err + } + + // Use the last index that affected the deployment table + index, err := state.Index("deployment") + if err != nil { + return err + } + reply.Index = index + reply.Deployments = deploys + + // Set the query response + j.srv.setQueryMeta(&reply.QueryMeta) + return nil + + }} + return j.srv.blockingRPC(&opts) +} + +// LatestDeployment is used to retrieve the latest deployment for a job +func (j *Job) LatestDeployment(args *structs.JobSpecificRequest, + reply *structs.SingleDeploymentResponse) error { + if done, err := j.srv.forward("Job.LatestDeployment", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "job", "latest_deployment"}, time.Now()) + + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + run: func(ws memdb.WatchSet, state *state.StateStore) error { + // Capture the deployments + deploys, err := state.DeploymentsByJobID(ws, args.JobID) + if err != nil { + return err + } + + // Use the last index that affected the deployment table + index, err := state.Index("deployment") + if err != nil { + return err + } + reply.Index = index + if len(deploys) > 0 { + sort.Slice(deploys, func(i, j int) bool { + return deploys[i].CreateIndex > deploys[j].CreateIndex + }) + reply.Deployment = deploys[0] + } + + // Set the query response + j.srv.setQueryMeta(&reply.QueryMeta) + return nil + + }} + return j.srv.blockingRPC(&opts) +} + // Plan is used to cause a dry-run evaluation of the Job and return the results // with a potential diff containing annotations. func (j *Job) Plan(args *structs.JobPlanRequest, reply *structs.JobPlanResponse) error { diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index ea0c0a5f4871..03edb0f9df2d 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" "github.com/kr/pretty" + "github.com/stretchr/testify/assert" ) func TestJobEndpoint_Register(t *testing.T) { @@ -2063,6 +2064,155 @@ func TestJobEndpoint_Evaluations_Blocking(t *testing.T) { } } +func TestJobEndpoint_Deployments(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + state := s1.fsm.State() + assert := assert.New(t) + + // Create the register request + j := mock.Job() + d1 := mock.Deployment() + d2 := mock.Deployment() + d1.JobID = j.ID + d2.JobID = j.ID + assert.Nil(state.UpsertJob(1000, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1001, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1002, d2, false), "UpsertDeployment") + + // Lookup the jobs + get := &structs.JobSpecificRequest{ + JobID: j.ID, + QueryOptions: structs.QueryOptions{Region: "global"}, + } + var resp structs.DeploymentListResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Job.Deployments", get, &resp), "RPC") + assert.EqualValues(1002, resp.Index, "response index") + assert.Len(resp.Deployments, 2, "deployments for job") +} + +func TestJobEndpoint_Deployments_Blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + state := s1.fsm.State() + assert := assert.New(t) + + // Create the register request + j := mock.Job() + d1 := mock.Deployment() + d2 := mock.Deployment() + d2.JobID = j.ID + assert.Nil(state.UpsertJob(50, j), "UpsertJob") + + // First upsert an unrelated eval + time.AfterFunc(100*time.Millisecond, func() { + assert.Nil(state.UpsertDeployment(100, d1, false), "UpsertDeployment") + }) + + // Upsert an eval for the job we are interested in later + time.AfterFunc(200*time.Millisecond, func() { + assert.Nil(state.UpsertDeployment(200, d2, false), "UpsertDeployment") + }) + + // Lookup the jobs + get := &structs.JobSpecificRequest{ + JobID: d2.JobID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 150, + }, + } + var resp structs.DeploymentListResponse + start := time.Now() + assert.Nil(msgpackrpc.CallWithCodec(codec, "Job.Deployments", get, &resp), "RPC") + assert.EqualValues(200, resp.Index, "response index") + assert.Len(resp.Deployments, 1, "deployments for job") + assert.Equal(d2.ID, resp.Deployments[0], "returned deployment") + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } +} + +func TestJobEndpoint_LatestDeployment(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + state := s1.fsm.State() + assert := assert.New(t) + + // Create the register request + j := mock.Job() + d1 := mock.Deployment() + d2 := mock.Deployment() + d1.JobID = j.ID + d2.JobID = j.ID + d2.CreateIndex = d1.CreateIndex + 100 + d2.ModifyIndex = d2.CreateIndex + 100 + assert.Nil(state.UpsertJob(1000, j), "UpsertJob") + assert.Nil(state.UpsertDeployment(1001, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1002, d2, false), "UpsertDeployment") + + // Lookup the jobs + get := &structs.JobSpecificRequest{ + JobID: j.ID, + QueryOptions: structs.QueryOptions{Region: "global"}, + } + var resp structs.SingleDeploymentResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Job.LatestDeployment", get, &resp), "RPC") + assert.EqualValues(1002, resp.Index, "response index") + assert.NotNil(resp.Deployment, "want a deployment") + assert.Equal(d2.ID, resp.Deployment.ID, "latest deployment for job") +} + +func TestJobEndpoint_LatestDeployment_Blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + state := s1.fsm.State() + assert := assert.New(t) + + // Create the register request + j := mock.Job() + d1 := mock.Deployment() + d2 := mock.Deployment() + d2.JobID = j.ID + assert.Nil(state.UpsertJob(50, j), "UpsertJob") + + // First upsert an unrelated eval + time.AfterFunc(100*time.Millisecond, func() { + assert.Nil(state.UpsertDeployment(100, d1, false), "UpsertDeployment") + }) + + // Upsert an eval for the job we are interested in later + time.AfterFunc(200*time.Millisecond, func() { + assert.Nil(state.UpsertDeployment(200, d2, false), "UpsertDeployment") + }) + + // Lookup the jobs + get := &structs.JobSpecificRequest{ + JobID: d2.JobID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 150, + }, + } + var resp structs.SingleDeploymentResponse + start := time.Now() + assert.Nil(msgpackrpc.CallWithCodec(codec, "Job.LatestDeployment", get, &resp), "RPC") + assert.EqualValues(200, resp.Index, "response index") + assert.NotNil(resp.Deployment, "deployment for job") + assert.Equal(d2.ID, resp.Deployment.ID, "returned deployment") + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } +} + func TestJobEndpoint_Plan_WithDiff(t *testing.T) { s1 := testServer(t, func(c *Config) { c.NumSchedulers = 0 // Prevent automatic dequeue From 86e49b97d9ca70603331b1aef676b36c963aab8f Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 11:29:44 -0700 Subject: [PATCH 046/105] Add config options --- command/agent/agent.go | 7 ++++ command/agent/config-test-fixtures/basic.hcl | 1 + command/agent/config.go | 8 +++++ command/agent/config_parse.go | 1 + command/agent/config_parse_test.go | 33 ++++++++++--------- nomad/config.go | 12 ++++++- .../docs/agent/configuration/server.html.md | 4 +++ 7 files changed, 49 insertions(+), 17 deletions(-) diff --git a/command/agent/agent.go b/command/agent/agent.go index 47c29326b8a3..27c0f07f3fba 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -184,6 +184,13 @@ func convertServerConfig(agentConfig *Config, logOutput io.Writer) (*nomad.Confi } conf.EvalGCThreshold = dur } + if gcThreshold := agentConfig.Server.DeploymentGCThreshold; gcThreshold != "" { + dur, err := time.ParseDuration(gcThreshold) + if err != nil { + return nil, err + } + conf.DeploymentGCThreshold = dur + } if heartbeatGrace := agentConfig.Server.HeartbeatGrace; heartbeatGrace != "" { dur, err := time.ParseDuration(heartbeatGrace) diff --git a/command/agent/config-test-fixtures/basic.hcl b/command/agent/config-test-fixtures/basic.hcl index 8347dc4c659c..faf2474e2422 100644 --- a/command/agent/config-test-fixtures/basic.hcl +++ b/command/agent/config-test-fixtures/basic.hcl @@ -71,6 +71,7 @@ server { node_gc_threshold = "12h" job_gc_threshold = "12h" eval_gc_threshold = "12h" + deployment_gc_threshold = "12h" heartbeat_grace = "30s" retry_join = [ "1.1.1.1", "2.2.2.2" ] start_join = [ "1.1.1.1", "2.2.2.2" ] diff --git a/command/agent/config.go b/command/agent/config.go index 02e1c9f51c3d..b533a245b4f9 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -270,6 +270,11 @@ type ServerConfig struct { // can be used to filter by age. EvalGCThreshold string `mapstructure:"eval_gc_threshold"` + // DeploymentGCThreshold controls how "old" a deployment must be to be + // collected by GC. Age is not the only requirement for a deployment to be + // GCed but the threshold can be used to filter by age. + DeploymentGCThreshold string `mapstructure:"deployment_gc_threshold"` + // HeartbeatGrace is the grace period beyond the TTL to account for network, // processing delays and clock skew before marking a node as "down". HeartbeatGrace string `mapstructure:"heartbeat_grace"` @@ -916,6 +921,9 @@ func (a *ServerConfig) Merge(b *ServerConfig) *ServerConfig { if b.EvalGCThreshold != "" { result.EvalGCThreshold = b.EvalGCThreshold } + if b.DeploymentGCThreshold != "" { + result.DeploymentGCThreshold = b.DeploymentGCThreshold + } if b.HeartbeatGrace != "" { result.HeartbeatGrace = b.HeartbeatGrace } diff --git a/command/agent/config_parse.go b/command/agent/config_parse.go index 1d0f387bf273..f8011527b5b8 100644 --- a/command/agent/config_parse.go +++ b/command/agent/config_parse.go @@ -504,6 +504,7 @@ func parseServer(result **ServerConfig, list *ast.ObjectList) error { "node_gc_threshold", "eval_gc_threshold", "job_gc_threshold", + "deployment_gc_threshold", "heartbeat_grace", "start_join", "retry_join", diff --git a/command/agent/config_parse_test.go b/command/agent/config_parse_test.go index 543cb046b723..7d9b969cc119 100644 --- a/command/agent/config_parse_test.go +++ b/command/agent/config_parse_test.go @@ -82,22 +82,23 @@ func TestConfig_Parse(t *testing.T) { NoHostUUID: helper.BoolToPtr(false), }, Server: &ServerConfig{ - Enabled: true, - BootstrapExpect: 5, - DataDir: "/tmp/data", - ProtocolVersion: 3, - NumSchedulers: 2, - EnabledSchedulers: []string{"test"}, - NodeGCThreshold: "12h", - EvalGCThreshold: "12h", - JobGCThreshold: "12h", - HeartbeatGrace: "30s", - RetryJoin: []string{"1.1.1.1", "2.2.2.2"}, - StartJoin: []string{"1.1.1.1", "2.2.2.2"}, - RetryInterval: "15s", - RejoinAfterLeave: true, - RetryMaxAttempts: 3, - EncryptKey: "abc", + Enabled: true, + BootstrapExpect: 5, + DataDir: "/tmp/data", + ProtocolVersion: 3, + NumSchedulers: 2, + EnabledSchedulers: []string{"test"}, + NodeGCThreshold: "12h", + EvalGCThreshold: "12h", + JobGCThreshold: "12h", + DeploymentGCThreshold: "12h", + HeartbeatGrace: "30s", + RetryJoin: []string{"1.1.1.1", "2.2.2.2"}, + StartJoin: []string{"1.1.1.1", "2.2.2.2"}, + RetryInterval: "15s", + RejoinAfterLeave: true, + RetryMaxAttempts: 3, + EncryptKey: "abc", }, Telemetry: &Telemetry{ StatsiteAddr: "127.0.0.1:1234", diff --git a/nomad/config.go b/nomad/config.go index 92e16b1e71f8..29161ee57f9f 100644 --- a/nomad/config.go +++ b/nomad/config.go @@ -142,10 +142,18 @@ type Config struct { // NodeGCInterval is how often we dispatch a job to GC failed nodes. NodeGCInterval time.Duration - // NodeGCThreshold is how "old" a nodemust be to be eligible + // NodeGCThreshold is how "old" a node must be to be eligible // for GC. This gives users some time to view and debug a failed nodes. NodeGCThreshold time.Duration + // DeploymentGCInterval is how often we dispatch a job to GC terminal + // deployments. + DeploymentGCInterval time.Duration + + // DeploymentGCThreshold is how "old" a deployment must be to be eligible + // for GC. This gives users some time to view terminal deployments. + DeploymentGCThreshold time.Duration + // EvalNackTimeout controls how long we allow a sub-scheduler to // work on an evaluation before we consider it failed and Nack it. // This allows that evaluation to be handed to another sub-scheduler @@ -255,6 +263,8 @@ func DefaultConfig() *Config { JobGCThreshold: 4 * time.Hour, NodeGCInterval: 5 * time.Minute, NodeGCThreshold: 24 * time.Hour, + DeploymentGCInterval: 5 * time.Minute, + DeploymentGCThreshold: 1 * time.Hour, EvalNackTimeout: 60 * time.Second, EvalDeliveryLimit: 3, EvalNackInitialReenqueueDelay: 1 * time.Second, diff --git a/website/source/docs/agent/configuration/server.html.md b/website/source/docs/agent/configuration/server.html.md index 15c6d9f6efd2..90f9863ad50d 100644 --- a/website/source/docs/agent/configuration/server.html.md +++ b/website/source/docs/agent/configuration/server.html.md @@ -76,6 +76,10 @@ server { evaluation must be in the terminal state before it is eligible for garbage collection. This is specified using a label suffix like "30s" or "1h". +- `deployment_gc_threshold` `(string: "1h")` - Specifies the minimum time a + deployment must be in the terminal state before it is eligible for garbage + collection. This is specified using a label suffix like "30s" or "1h". + - `num_schedulers` `(int: [num-cores])` - Specifies the number of parallel scheduler threads to run. This can be as many as one per core, or `0` to disallow this server from making any scheduling decisions. This defaults to From 24635f8b95655ce6e5d40c464b83ec525e0bcda8 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 12:32:37 -0700 Subject: [PATCH 047/105] Deployment GC This PR implements the garbage collector for deployments. Deployments will by default be garbage collected after 1 hour. --- nomad/core_sched.go | 107 +++++++++++++++++++++++++++- nomad/core_sched_test.go | 114 +++++++++++++++++++++++++++++- nomad/deployment_endpoint.go | 19 +++++ nomad/deployment_endpoint_test.go | 27 +++++++ nomad/fsm.go | 18 +++++ nomad/fsm_test.go | 34 +++++++++ nomad/state/state_store.go | 31 ++++---- nomad/state/state_store_test.go | 18 +++-- nomad/structs/structs.go | 12 ++++ 9 files changed, 356 insertions(+), 24 deletions(-) diff --git a/nomad/core_sched.go b/nomad/core_sched.go index 83f8bd10e1ed..8c2601a9444a 100644 --- a/nomad/core_sched.go +++ b/nomad/core_sched.go @@ -44,6 +44,8 @@ func (c *CoreScheduler) Process(eval *structs.Evaluation) error { return c.nodeGC(eval) case structs.CoreJobJobGC: return c.jobGC(eval) + case structs.CoreJobDeploymentGC: + return c.deploymentGC(eval) case structs.CoreJobForceGC: return c.forceGC(eval) default: @@ -59,6 +61,9 @@ func (c *CoreScheduler) forceGC(eval *structs.Evaluation) error { if err := c.evalGC(eval); err != nil { return err } + if err := c.deploymentGC(eval); err != nil { + return err + } // Node GC must occur after the others to ensure the allocations are // cleared. @@ -295,7 +300,7 @@ func (c *CoreScheduler) gcEval(eval *structs.Evaluation, thresholdIndex uint64, // allocs. func (c *CoreScheduler) evalReap(evals, allocs []string) error { // Call to the leader to issue the reap - for _, req := range c.partitionReap(evals, allocs) { + for _, req := range c.partitionEvalReap(evals, allocs) { var resp structs.GenericResponse if err := c.srv.RPC("Eval.Reap", req, &resp); err != nil { c.srv.logger.Printf("[ERR] sched.core: eval reap failed: %v", err) @@ -306,10 +311,10 @@ func (c *CoreScheduler) evalReap(evals, allocs []string) error { return nil } -// partitionReap returns a list of EvalDeleteRequest to make, ensuring a single +// partitionEvalReap returns a list of EvalDeleteRequest to make, ensuring a single // request does not contain too many allocations and evaluations. This is // necessary to ensure that the Raft transaction does not become too large. -func (c *CoreScheduler) partitionReap(evals, allocs []string) []*structs.EvalDeleteRequest { +func (c *CoreScheduler) partitionEvalReap(evals, allocs []string) []*structs.EvalDeleteRequest { var requests []*structs.EvalDeleteRequest submittedEvals, submittedAllocs := 0, 0 for submittedEvals != len(evals) || submittedAllocs != len(allocs) { @@ -437,3 +442,99 @@ OUTER: } return nil } + +// deploymentGC is used to garbage collect old deployments +func (c *CoreScheduler) deploymentGC(eval *structs.Evaluation) error { + // Iterate over the deployments + ws := memdb.NewWatchSet() + iter, err := c.snap.Deployments(ws) + if err != nil { + return err + } + + var oldThreshold uint64 + if eval.JobID == structs.CoreJobForceGC { + // The GC was forced, so set the threshold to its maximum so everything + // will GC. + oldThreshold = math.MaxUint64 + c.srv.logger.Println("[DEBUG] sched.core: forced deployment GC") + } else { + // Compute the old threshold limit for GC using the FSM + // time table. This is a rough mapping of a time to the + // Raft index it belongs to. + tt := c.srv.fsm.TimeTable() + cutoff := time.Now().UTC().Add(-1 * c.srv.config.DeploymentGCThreshold) + oldThreshold = tt.NearestIndex(cutoff) + c.srv.logger.Printf("[DEBUG] sched.core: deployment GC: scanning before index %d (%v)", + oldThreshold, c.srv.config.DeploymentGCThreshold) + } + + // Collect the deployments to GC + var gcDeployment []string + for { + raw := iter.Next() + if raw == nil { + break + } + deploy := raw.(*structs.Deployment) + + // Ignore non-terminal and new deployments + if deploy.Active() || deploy.ModifyIndex > oldThreshold { + continue + } + + // Deployment is eligible for garbage collection + gcDeployment = append(gcDeployment, deploy.ID) + } + + // Fast-path the nothing case + if len(gcDeployment) == 0 { + return nil + } + c.srv.logger.Printf("[DEBUG] sched.core: deployment GC: %d nodes eligible", len(gcDeployment)) + return c.deploymentReap(gcDeployment) +} + +// deploymentReap contacts the leader and issues a reap on the passed +// deployments. +func (c *CoreScheduler) deploymentReap(deployments []string) error { + // Call to the leader to issue the reap + for _, req := range c.partitionDeploymentReap(deployments) { + var resp structs.GenericResponse + if err := c.srv.RPC("Deployment.Reap", req, &resp); err != nil { + c.srv.logger.Printf("[ERR] sched.core: deployment reap failed: %v", err) + return err + } + } + + return nil +} + +// partitionDeploymentReap returns a list of DeploymentDeleteRequest to make, +// ensuring a single request does not contain too many deployments. This is +// necessary to ensure that the Raft transaction does not become too large. +func (c *CoreScheduler) partitionDeploymentReap(deployments []string) []*structs.DeploymentDeleteRequest { + var requests []*structs.DeploymentDeleteRequest + submittedDeployments := 0 + for submittedDeployments != len(deployments) { + req := &structs.DeploymentDeleteRequest{ + WriteRequest: structs.WriteRequest{ + Region: c.srv.config.Region, + }, + } + requests = append(requests, req) + available := maxIdsPerReap + + if remaining := len(deployments) - submittedDeployments; remaining > 0 { + if remaining <= available { + req.Deployments = deployments[submittedDeployments:] + submittedDeployments += remaining + } else { + req.Deployments = deployments[submittedDeployments : submittedDeployments+available] + submittedDeployments += available + } + } + } + + return requests +} diff --git a/nomad/core_sched_test.go b/nomad/core_sched_test.go index a10a4c16d3a0..9496c2051197 100644 --- a/nomad/core_sched_test.go +++ b/nomad/core_sched_test.go @@ -8,6 +8,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" + "github.com/stretchr/testify/assert" ) func TestCoreScheduler_EvalGC(t *testing.T) { @@ -1315,7 +1316,81 @@ func TestCoreScheduler_JobGC_Periodic(t *testing.T) { } } -func TestCoreScheduler_PartitionReap(t *testing.T) { +func TestCoreScheduler_DeploymentGC(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // COMPAT Remove in 0.6: Reset the FSM time table since we reconcile which sets index 0 + s1.fsm.timetable.table = make([]TimeTableEntry, 1, 10) + + // Insert terminal and active deployment + state := s1.fsm.State() + d1, d2 := mock.Deployment(), mock.Deployment() + d1.Status = structs.DeploymentStatusFailed + assert.Nil(state.UpsertDeployment(1000, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1001, d2, false), "UpsertDeployment") + + // Update the time tables to make this work + tt := s1.fsm.TimeTable() + tt.Witness(2000, time.Now().UTC().Add(-1*s1.config.DeploymentGCThreshold)) + + // Create a core scheduler + snap, err := state.Snapshot() + assert.Nil(err, "Snapshot") + core := NewCoreScheduler(s1, snap) + + // Attempt the GC + gc := s1.coreJobEval(structs.CoreJobDeploymentGC, 2000) + assert.Nil(core.Process(gc), "Process GC") + + // Should be gone + ws := memdb.NewWatchSet() + out, err := state.DeploymentByID(ws, d1.ID) + assert.Nil(err, "DeploymentByID") + assert.Nil(out, "Terminal Deployment") + out2, err := state.DeploymentByID(ws, d2.ID) + assert.Nil(err, "DeploymentByID") + assert.NotNil(out2, "Active Deployment") +} + +func TestCoreScheduler_DeploymentGC_Force(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // COMPAT Remove in 0.6: Reset the FSM time table since we reconcile which sets index 0 + s1.fsm.timetable.table = make([]TimeTableEntry, 1, 10) + + // Insert terminal and active deployment + state := s1.fsm.State() + d1, d2 := mock.Deployment(), mock.Deployment() + d1.Status = structs.DeploymentStatusFailed + assert.Nil(state.UpsertDeployment(1000, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1001, d2, false), "UpsertDeployment") + + // Create a core scheduler + snap, err := state.Snapshot() + assert.Nil(err, "Snapshot") + core := NewCoreScheduler(s1, snap) + + // Attempt the GC + gc := s1.coreJobEval(structs.CoreJobForceGC, 1000) + assert.Nil(core.Process(gc), "Process Force GC") + + // Should be gone + ws := memdb.NewWatchSet() + out, err := state.DeploymentByID(ws, d1.ID) + assert.Nil(err, "DeploymentByID") + assert.Nil(out, "Terminal Deployment") + out2, err := state.DeploymentByID(ws, d2.ID) + assert.Nil(err, "DeploymentByID") + assert.NotNil(out2, "Active Deployment") +} + +func TestCoreScheduler_PartitionEvalReap(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() testutil.WaitForLeader(t, s1.RPC) @@ -1335,7 +1410,7 @@ func TestCoreScheduler_PartitionReap(t *testing.T) { evals := []string{"a", "b", "c"} allocs := []string{"1", "2", "3"} - requests := core.(*CoreScheduler).partitionReap(evals, allocs) + requests := core.(*CoreScheduler).partitionEvalReap(evals, allocs) if len(requests) != 3 { t.Fatalf("Expected 3 requests got: %v", requests) } @@ -1355,3 +1430,38 @@ func TestCoreScheduler_PartitionReap(t *testing.T) { t.Fatalf("Unexpected third request: %v", third) } } + +func TestCoreScheduler_PartitionDeploymentReap(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + testutil.WaitForLeader(t, s1.RPC) + + // COMPAT Remove in 0.6: Reset the FSM time table since we reconcile which sets index 0 + s1.fsm.timetable.table = make([]TimeTableEntry, 1, 10) + + // Create a core scheduler + snap, err := s1.fsm.State().Snapshot() + if err != nil { + t.Fatalf("err: %v", err) + } + core := NewCoreScheduler(s1, snap) + + // Set the max ids per reap to something lower. + maxIdsPerReap = 2 + + deployments := []string{"a", "b", "c"} + requests := core.(*CoreScheduler).partitionDeploymentReap(deployments) + if len(requests) != 2 { + t.Fatalf("Expected 2 requests got: %v", requests) + } + + first := requests[0] + if len(first.Deployments) != 2 { + t.Fatalf("Unexpected first request: %v", first) + } + + second := requests[1] + if len(second.Deployments) != 1 { + t.Fatalf("Unexpected second request: %v", second) + } +} diff --git a/nomad/deployment_endpoint.go b/nomad/deployment_endpoint.go index be1bfec476e5..98b03d7348cf 100644 --- a/nomad/deployment_endpoint.go +++ b/nomad/deployment_endpoint.go @@ -290,3 +290,22 @@ func (d *Deployment) Allocations(args *structs.DeploymentSpecificRequest, reply }} return d.srv.blockingRPC(&opts) } + +// Reap is used to cleanup terminal deployments +func (d *Deployment) Reap(args *structs.DeploymentDeleteRequest, + reply *structs.GenericResponse) error { + if done, err := d.srv.forward("Deployment.Reap", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "deployment", "reap"}, time.Now()) + + // Update via Raft + _, index, err := d.srv.raftApply(structs.DeploymentDeleteRequestType, args) + if err != nil { + return err + } + + // Update the index + reply.Index = index + return nil +} diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index 7da8dbf13e1e..10eefcedaa89 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -502,3 +502,30 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } } + +func TestDeploymentEndpoint_Reap(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + + // Create the register request + d1 := mock.Deployment() + assert.Nil(s1.fsm.State().UpsertDeployment(1000, d1, false), "UpsertDeployment") + + // Reap the eval + get := &structs.DeploymentDeleteRequest{ + Deployments: []string{d1.ID}, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + var resp structs.GenericResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Reap", get, &resp), "RPC") + assert.NotZero(resp.Index, "bad response index") + + // Ensure deleted + ws := memdb.NewWatchSet() + outD, err := s1.fsm.State().DeploymentByID(ws, d1.ID) + assert.Nil(err, "DeploymentByID") + assert.Nil(outD, "Deleted Deployment") +} diff --git a/nomad/fsm.go b/nomad/fsm.go index 066caa08abd5..e4bc3db2db67 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -163,6 +163,8 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { return n.applyDeploymentPromotion(buf[1:], log.Index) case structs.DeploymentAllocHealthRequestType: return n.applyDeploymentAllocHealth(buf[1:], log.Index) + case structs.DeploymentDeleteRequestType: + return n.applyDeploymentDelete(buf[1:], log.Index) default: if ignoreUnknown { n.logger.Printf("[WARN] nomad.fsm: ignoring unknown message type (%d), upgrade to newer version", msgType) @@ -633,6 +635,22 @@ func (n *nomadFSM) applyDeploymentAllocHealth(buf []byte, index uint64) interfac return nil } +// applyDeploymentDelete is used to delete a set of deployments +func (n *nomadFSM) applyDeploymentDelete(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_delete"}, time.Now()) + var req structs.DeploymentDeleteRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + + if err := n.state.DeleteDeployment(index, req.Deployments); err != nil { + n.logger.Printf("[ERR] nomad.fsm: DeleteDeployment failed: %v", err) + return err + } + + return nil +} + func (n *nomadFSM) Snapshot() (raft.FSMSnapshot, error) { // Create a new snapshot snap, err := n.state.Snapshot() diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index c06bb93771f6..67b4a885d7a0 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1439,6 +1439,40 @@ func TestFSM_DeploymentAllocHealth(t *testing.T) { } } +func TestFSM_DeleteDeployment(t *testing.T) { + fsm := testFSM(t) + state := fsm.State() + + // Upsert a deployments + d := mock.Deployment() + if err := state.UpsertDeployment(1, d, false); err != nil { + t.Fatalf("bad: %v", err) + } + + req := structs.DeploymentDeleteRequest{ + Deployments: []string{d.ID}, + } + buf, err := structs.Encode(structs.DeploymentDeleteRequestType, req) + if err != nil { + t.Fatalf("err: %v", err) + } + + resp := fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Verify we are NOT registered + ws := memdb.NewWatchSet() + deployment, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + if deployment != nil { + t.Fatalf("deployment found!") + } +} + func testSnapshotRestore(t *testing.T, fsm *nomadFSM) *nomadFSM { // Snapshot snap, err := fsm.Snapshot() diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 6ebdf89d1cf0..fbb2aed7212a 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -396,24 +396,31 @@ func (s *StateStore) LatestDeploymentByJobID(ws memdb.WatchSet, jobID string) (* return out, nil } -// DeleteDeployment is used to delete a deployment by ID -func (s *StateStore) DeleteDeployment(index uint64, deploymentID string) error { +// DeleteDeployment is used to delete a set of deployments by ID +func (s *StateStore) DeleteDeployment(index uint64, deploymentIDs []string) error { txn := s.db.Txn(true) defer txn.Abort() - // Lookup the deployment - existing, err := txn.First("deployment", "id", deploymentID) - if err != nil { - return fmt.Errorf("deployment lookup failed: %v", err) - } - if existing == nil { - return fmt.Errorf("deployment not found") + if len(deploymentIDs) == 0 { + return nil } - // Delete the deployment - if err := txn.Delete("deployment", existing); err != nil { - return fmt.Errorf("deployment delete failed: %v", err) + for _, deploymentID := range deploymentIDs { + // Lookup the deployment + existing, err := txn.First("deployment", "id", deploymentID) + if err != nil { + return fmt.Errorf("deployment lookup failed: %v", err) + } + if existing == nil { + return fmt.Errorf("deployment not found") + } + + // Delete the deployment + if err := txn.Delete("deployment", existing); err != nil { + return fmt.Errorf("deployment delete failed: %v", err) + } } + if err := txn.Insert("index", &IndexEntry{"deployment", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index f9b256f187f5..81de030de946 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -422,20 +422,24 @@ func TestStateStore_UpsertDeployment_Cancel(t *testing.T) { func TestStateStore_DeleteDeployment(t *testing.T) { state := testStateStore(t) - deployment := mock.Deployment() + d1 := mock.Deployment() + d2 := mock.Deployment() - err := state.UpsertDeployment(1000, deployment, false) + err := state.UpsertDeployment(1000, d1, false) if err != nil { t.Fatalf("err: %v", err) } + if err := state.UpsertDeployment(1001, d2, false); err != nil { + t.Fatalf("err: %v", err) + } // Create a watchset so we can test that delete fires the watch ws := memdb.NewWatchSet() - if _, err := state.DeploymentByID(ws, deployment.ID); err != nil { + if _, err := state.DeploymentByID(ws, d1.ID); err != nil { t.Fatalf("bad: %v", err) } - err = state.DeleteDeployment(1001, deployment.ID) + err = state.DeleteDeployment(1002, []string{d1.ID, d2.ID}) if err != nil { t.Fatalf("err: %v", err) } @@ -445,20 +449,20 @@ func TestStateStore_DeleteDeployment(t *testing.T) { } ws = memdb.NewWatchSet() - out, err := state.DeploymentByID(ws, deployment.ID) + out, err := state.DeploymentByID(ws, d1.ID) if err != nil { t.Fatalf("err: %v", err) } if out != nil { - t.Fatalf("bad: %#v %#v", deployment, out) + t.Fatalf("bad: %#v %#v", d1, out) } index, err := state.Index("deployment") if err != nil { t.Fatalf("err: %v", err) } - if index != 1001 { + if index != 1002 { t.Fatalf("bad: %d", index) } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index d936b7488e7b..caae8f782a8c 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -57,6 +57,7 @@ const ( DeploymentStatusUpdateRequestType DeploymentPromoteRequestType DeploymentAllocHealthRequestType + DeploymentDeleteRequestType ) const ( @@ -497,6 +498,12 @@ type DeploymentListRequest struct { QueryOptions } +// DeploymentDeleteRequest is used for deleting deployments. +type DeploymentDeleteRequest struct { + Deployments []string + WriteRequest +} + // DeploymentStatusUpdateRequest is used to update the status of a deployment as // well as optionally creating an evaluation atomically. type DeploymentStatusUpdateRequest struct { @@ -4478,6 +4485,11 @@ const ( // the system. CoreJobJobGC = "job-gc" + // CoreJobDeploymentGC is used for the garbage collection of eligible + // deployments. We periodically scan garbage collectible deployments and + // check if they are terminal. If so, we delete these out of the system. + CoreJobDeploymentGC = "deployment-gc" + // CoreJobForceGC is used to force garbage collection of all GCable objects. CoreJobForceGC = "force-gc" ) From 9220836cb4e58f2da385817454e6ce980838e349 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 17:16:20 -0700 Subject: [PATCH 048/105] JobVersions returns struct with optional diff --- api/jobs.go | 21 ++-- command/agent/job_endpoint.go | 16 +++- command/agent/job_endpoint_test.go | 9 +- command/job_history.go | 81 ++++++++++++++++ commands.go | 5 + main.go | 2 +- nomad/deployment_watcher_shims.go | 4 +- nomad/deploymentwatcher/deployment_watcher.go | 2 +- .../deploymentwatcher/deployments_watcher.go | 2 +- nomad/job_endpoint.go | 13 ++- nomad/job_endpoint_test.go | 95 ++++++++++++++++++- nomad/structs/structs.go | 8 ++ 12 files changed, 238 insertions(+), 20 deletions(-) create mode 100644 command/job_history.go diff --git a/api/jobs.go b/api/jobs.go index 9981577d4a13..3daa438f8082 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -107,15 +107,15 @@ func (j *Jobs) Info(jobID string, q *QueryOptions) (*Job, *QueryMeta, error) { return &resp, qm, nil } -// Versions is used to retrieve all versions of a particular -// job given its unique ID. -func (j *Jobs) Versions(jobID string, q *QueryOptions) ([]*Job, *QueryMeta, error) { - var resp []*Job - qm, err := j.client.query("/v1/job/"+jobID+"/versions", &resp, q) +// Versions is used to retrieve all versions of a particular job given its +// unique ID. +func (j *Jobs) Versions(jobID string, diffs bool, q *QueryOptions) ([]*Job, []*JobDiff, *QueryMeta, error) { + var resp JobVersionsResponse + qm, err := j.client.query(fmt.Sprintf("/v1/job/%s/versions?diffs=%v", jobID, diffs), &resp, q) if err != nil { - return nil, nil, err + return nil, nil, nil, err } - return resp, qm, nil + return resp.Versions, resp.Diffs, qm, nil } // Allocations is used to return the allocs for a given job ID. @@ -831,3 +831,10 @@ type JobDispatchResponse struct { JobCreateIndex uint64 WriteMeta } + +// JobVersionsResponse is used for a job get versions request +type JobVersionsResponse struct { + Versions []*Job + Diffs []*JobDiff + QueryMeta +} diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index 4583070b5dcc..df2c630e1388 100644 --- a/command/agent/job_endpoint.go +++ b/command/agent/job_endpoint.go @@ -394,8 +394,20 @@ func (s *HTTPServer) jobDelete(resp http.ResponseWriter, req *http.Request, func (s *HTTPServer) jobVersions(resp http.ResponseWriter, req *http.Request, jobName string) (interface{}, error) { - args := structs.JobSpecificRequest{ + + diffsStr := req.URL.Query().Get("diffs") + var diffsBool bool + if diffsStr != "" { + var err error + diffsBool, err = strconv.ParseBool(diffsStr) + if err != nil { + return nil, fmt.Errorf("Failed to parse value of %q (%v) as a bool: %v", "diffs", diffsStr, err) + } + } + + args := structs.JobVersionsRequest{ JobID: jobName, + Diffs: diffsBool, } if s.parse(resp, req, &args.Region, &args.QueryOptions) { return nil, nil @@ -411,7 +423,7 @@ func (s *HTTPServer) jobVersions(resp http.ResponseWriter, req *http.Request, return nil, CodedError(404, "job versions not found") } - return out.Versions, nil + return out, nil } func (s *HTTPServer) jobRevert(resp http.ResponseWriter, req *http.Request, diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index fccc33a2e548..95915c06a09f 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -706,7 +706,7 @@ func TestHTTP_JobVersions(t *testing.T) { } // Make the HTTP request - req, err := http.NewRequest("GET", "/v1/job/"+job.ID+"/versions", nil) + req, err := http.NewRequest("GET", "/v1/job/"+job.ID+"/versions?diffs=true", nil) if err != nil { t.Fatalf("err: %v", err) } @@ -719,7 +719,8 @@ func TestHTTP_JobVersions(t *testing.T) { } // Check the response - versions := obj.([]*structs.Job) + vResp := obj.(structs.JobVersionsResponse) + versions := vResp.Versions if len(versions) != 2 { t.Fatalf("got %d versions; want 2", len(versions)) } @@ -732,6 +733,10 @@ func TestHTTP_JobVersions(t *testing.T) { t.Fatalf("bad %v", v) } + if len(vResp.Diffs) != 1 { + t.Fatalf("bad %v", vResp) + } + // Check for the index if respW.HeaderMap.Get("X-Nomad-Index") == "" { t.Fatalf("missing index") diff --git a/command/job_history.go b/command/job_history.go new file mode 100644 index 000000000000..fbaf2e51e448 --- /dev/null +++ b/command/job_history.go @@ -0,0 +1,81 @@ +package command + +import ( + "fmt" + "strings" +) + +type JobHistoryCommand struct { + Meta +} + +func (c *JobHistoryCommand) Help() string { + helpText := ` +Usage: nomad job history [options] + +History is used to display the known versions of a particular job. The command +can display the diff between job versions and can be useful for understanding +the changes that occured to the job as well as deciding job versions to revert +to. + +General Options: + + ` + generalOptionsUsage() + ` + +History Options: + + -p + Display the difference between each job and its predecessor. + + -full + Display the full job definition for each version. + + -version + Display only the history for the given job version. +` + return strings.TrimSpace(helpText) +} + +func (c *JobHistoryCommand) Synopsis() string { + return "Display all tracked versions of a job" +} + +func (c *JobHistoryCommand) Run(args []string) int { + var diff, full bool + var version uint64 + + flags := c.Meta.FlagSet("job history", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&diff, "p", false, "") + flags.BoolVar(&full, "full", false, "") + flags.Uint64Var(&version, "version", 0, "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got exactly one node + args = flags.Args() + if l := len(args); l < 1 || l > 2 { + c.Ui.Error(c.Help()) + return 1 + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + jobID := args[0] + versions, _, err := client.Jobs().Versions(jobID, nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving job versions: %s", err)) + return 1 + } + + c.Ui.Output(jobID) + c.Ui.Output(fmt.Sprintf("%d", len(versions))) + return 0 +} diff --git a/commands.go b/commands.go index 9bc568deab10..aabdc90c43a8 100644 --- a/commands.go +++ b/commands.go @@ -99,6 +99,11 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "job history": func() (cli.Command, error) { + return &command.JobHistoryCommand{ + Meta: meta, + }, nil + }, "logs": func() (cli.Command, error) { return &command.LogsCommand{ Meta: meta, diff --git a/main.go b/main.go index cc42fd279879..7bb9afe471b9 100644 --- a/main.go +++ b/main.go @@ -40,7 +40,7 @@ func RunCustom(args []string, commands map[string]cli.CommandFactory) int { case "executor": case "syslog": case "operator raft", "operator raft list-peers", "operator raft remove-peer": - case "job dispatch": + case "job dispatch", "job history": case "fs ls", "fs cat", "fs stat": case "check": default: diff --git a/nomad/deployment_watcher_shims.go b/nomad/deployment_watcher_shims.go index 25de8fb2a1fd..f21ed6af066e 100644 --- a/nomad/deployment_watcher_shims.go +++ b/nomad/deployment_watcher_shims.go @@ -27,7 +27,7 @@ type deploymentWatcherStateShim struct { // getJobVersions is used to lookup the versions of a job. This is used when // rolling back to find the latest stable job - getJobVersions func(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error + getJobVersions func(args *structs.JobVersionsRequest, reply *structs.JobVersionsResponse) error // getJob is used to lookup a particular job. getJob func(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error @@ -65,7 +65,7 @@ func (d *deploymentWatcherStateShim) GetDeployment(args *structs.DeploymentSpeci return d.getDeployment(args, reply) } -func (d *deploymentWatcherStateShim) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { +func (d *deploymentWatcherStateShim) GetJobVersions(args *structs.JobVersionsRequest, reply *structs.JobVersionsResponse) error { if args.Region == "" { args.Region = d.region } diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 295fb29b4579..3151c910b5e7 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -366,7 +366,7 @@ func (w *deploymentWatcher) watch() { // latestStableJob returns the latest stable job. It may be nil if none exist func (w *deploymentWatcher) latestStableJob() (*structs.Job, error) { - args := &structs.JobSpecificRequest{JobID: w.d.JobID} + args := &structs.JobVersionsRequest{JobID: w.d.JobID} var resp structs.JobVersionsResponse if err := w.watchers.GetJobVersions(args, &resp); err != nil { return nil, err diff --git a/nomad/deploymentwatcher/deployments_watcher.go b/nomad/deploymentwatcher/deployments_watcher.go index 8cd5fa428294..3eb6a281dc3c 100644 --- a/nomad/deploymentwatcher/deployments_watcher.go +++ b/nomad/deploymentwatcher/deployments_watcher.go @@ -67,7 +67,7 @@ type DeploymentStateWatchers interface { // GetJobVersions is used to lookup the versions of a job. This is used when // rolling back to find the latest stable job - GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error + GetJobVersions(args *structs.JobVersionsRequest, reply *structs.JobVersionsResponse) error // GetJob is used to lookup a particular job. GetJob(args *structs.JobSpecificRequest, reply *structs.SingleJobResponse) error diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 9431b058dfbc..5ae2cd6ef8c7 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -558,7 +558,7 @@ func (j *Job) GetJob(args *structs.JobSpecificRequest, } // GetJobVersions is used to retrieve all tracked versions of a job. -func (j *Job) GetJobVersions(args *structs.JobSpecificRequest, +func (j *Job) GetJobVersions(args *structs.JobVersionsRequest, reply *structs.JobVersionsResponse) error { if done, err := j.srv.forward("Job.GetJobVersions", args, args, reply); done { return err @@ -580,6 +580,17 @@ func (j *Job) GetJobVersions(args *structs.JobSpecificRequest, reply.Versions = out if len(out) != 0 { reply.Index = out[0].ModifyIndex + + // Compute the diffs + for i := 0; i < len(out)-1; i++ { + old, new := out[i+1], out[i] + d, err := old.Diff(new, true) + if err != nil { + return fmt.Errorf("failed to create job diff: %v", err) + } + reply.Diffs = append(reply.Diffs, d) + } + } else { // Use the last index that affected the nodes table index, err := state.Index("job_version") diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 03edb0f9df2d..b6a31f5ca087 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -1503,7 +1503,7 @@ func TestJobEndpoint_GetJobVersions(t *testing.T) { } // Lookup the job - get := &structs.JobSpecificRequest{ + get := &structs.JobVersionsRequest{ JobID: job.ID, QueryOptions: structs.QueryOptions{Region: "global"}, } @@ -1541,6 +1541,95 @@ func TestJobEndpoint_GetJobVersions(t *testing.T) { } } +func TestJobEndpoint_GetJobVersions_Diff(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the register request + job := mock.Job() + job.Priority = 88 + reg := &structs.JobRegisterRequest{ + Job: job, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.JobRegisterResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.Register", reg, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + // Register the job again to create another version + job.Priority = 90 + if err := msgpackrpc.CallWithCodec(codec, "Job.Register", reg, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + // Register the job again to create another version + job.Priority = 100 + if err := msgpackrpc.CallWithCodec(codec, "Job.Register", reg, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + // Lookup the job + get := &structs.JobVersionsRequest{ + JobID: job.ID, + Diffs: true, + QueryOptions: structs.QueryOptions{Region: "global"}, + } + var versionsResp structs.JobVersionsResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.GetJobVersions", get, &versionsResp); err != nil { + t.Fatalf("err: %v", err) + } + if versionsResp.Index != resp.JobModifyIndex { + t.Fatalf("Bad index: %d %d", versionsResp.Index, resp.Index) + } + + // Make sure there are two job versions + versions := versionsResp.Versions + if l := len(versions); l != 3 { + t.Fatalf("Got %d versions; want 3", l) + } + + if v := versions[0]; v.Priority != 100 || v.ID != job.ID || v.Version != 2 { + t.Fatalf("bad: %+v", v) + } + if v := versions[1]; v.Priority != 90 || v.ID != job.ID || v.Version != 1 { + t.Fatalf("bad: %+v", v) + } + if v := versions[2]; v.Priority != 88 || v.ID != job.ID || v.Version != 0 { + t.Fatalf("bad: %+v", v) + } + + // Ensure we got diffs + diffs := versionsResp.Diffs + if l := len(diffs); l != 2 { + t.Fatalf("Got %d diffs; want 2", l) + } + d1 := diffs[0] + if len(d1.Fields) != 1 { + t.Fatalf("Got too many diffs: %#v", d1) + } + if d1.Fields[0].Name != "Priority" { + t.Fatalf("Got wrong field: %#v", d1) + } + if d1.Fields[0].Old != "90" && d1.Fields[0].New != "100" { + t.Fatalf("Got wrong field values: %#v", d1) + } + d2 := diffs[1] + if len(d2.Fields) != 1 { + t.Fatalf("Got too many diffs: %#v", d2) + } + if d2.Fields[0].Name != "Priority" { + t.Fatalf("Got wrong field: %#v", d2) + } + if d2.Fields[0].Old != "88" && d1.Fields[0].New != "90" { + t.Fatalf("Got wrong field values: %#v", d2) + } +} + func TestJobEndpoint_GetJobVersions_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() @@ -1569,7 +1658,7 @@ func TestJobEndpoint_GetJobVersions_Blocking(t *testing.T) { } }) - req := &structs.JobSpecificRequest{ + req := &structs.JobVersionsRequest{ JobID: job2.ID, QueryOptions: structs.QueryOptions{ Region: "global", @@ -1599,7 +1688,7 @@ func TestJobEndpoint_GetJobVersions_Blocking(t *testing.T) { } }) - req2 := &structs.JobSpecificRequest{ + req2 := &structs.JobVersionsRequest{ JobID: job3.ID, QueryOptions: structs.QueryOptions{ Region: "global", diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index caae8f782a8c..6c7f80c658e2 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -734,9 +734,17 @@ type JobListResponse struct { QueryMeta } +// JobVersionsRequest is used to get a jobs versions +type JobVersionsRequest struct { + JobID string + Diffs bool + QueryOptions +} + // JobVersionsResponse is used for a job get versions request type JobVersionsResponse struct { Versions []*Job + Diffs []*JobDiff QueryMeta } From 1c425defd8c28009f44b7b286850eeefb3b0fb64 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 18:42:37 -0700 Subject: [PATCH 049/105] job history --- command/job_history.go | 150 +++++++++++++++++- command/job_history_test.go | 34 ++++ nomad/job_endpoint.go | 15 +- .../ryanuber/columnize/columnize.go | 10 +- 4 files changed, 195 insertions(+), 14 deletions(-) create mode 100644 command/job_history_test.go diff --git a/command/job_history.go b/command/job_history.go index fbaf2e51e448..3dedecaf9cfe 100644 --- a/command/job_history.go +++ b/command/job_history.go @@ -2,11 +2,16 @@ package command import ( "fmt" + "strconv" "strings" + + "github.com/dadgar/columnize" + "github.com/hashicorp/nomad/api" ) type JobHistoryCommand struct { Meta + formatter DataFormatter } func (c *JobHistoryCommand) Help() string { @@ -30,7 +35,7 @@ History Options: -full Display the full job definition for each version. - -version + -job-version Display only the history for the given job version. ` return strings.TrimSpace(helpText) @@ -42,13 +47,13 @@ func (c *JobHistoryCommand) Synopsis() string { func (c *JobHistoryCommand) Run(args []string) int { var diff, full bool - var version uint64 + var versionStr string flags := c.Meta.FlagSet("job history", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } flags.BoolVar(&diff, "p", false, "") flags.BoolVar(&full, "full", false, "") - flags.Uint64Var(&version, "version", 0, "") + flags.StringVar(&versionStr, "job-version", "", "") if err := flags.Parse(args); err != nil { return 1 @@ -69,13 +74,146 @@ func (c *JobHistoryCommand) Run(args []string) int { } jobID := args[0] - versions, _, err := client.Jobs().Versions(jobID, nil) + + // Check if the job exists + jobs, _, err := client.Jobs().PrefixList(jobID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error listing jobs: %s", err)) + return 1 + } + if len(jobs) == 0 { + c.Ui.Error(fmt.Sprintf("No job(s) with prefix or id %q found", jobID)) + return 1 + } + if len(jobs) > 1 && strings.TrimSpace(jobID) != jobs[0].ID { + out := make([]string, len(jobs)+1) + out[0] = "ID|Type|Priority|Status" + for i, job := range jobs { + out[i+1] = fmt.Sprintf("%s|%s|%d|%s", + job.ID, + job.Type, + job.Priority, + job.Status) + } + c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", formatList(out))) + return 0 + } + + // Prefix lookup matched a single job + versions, diffs, _, err := client.Jobs().Versions(jobs[0].ID, diff, nil) if err != nil { c.Ui.Error(fmt.Sprintf("Error retrieving job versions: %s", err)) return 1 } - c.Ui.Output(jobID) - c.Ui.Output(fmt.Sprintf("%d", len(versions))) + f, err := DataFormat("json", "") + if err != nil { + c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + return 1 + } + c.formatter = f + + if versionStr != "" { + version, _, err := parseVersion(versionStr) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error parsing version value %q: %v", versionStr, err)) + return 1 + } + + var job *api.Job + var diff *api.JobDiff + var nextVersion uint64 + for i, v := range versions { + if *v.Version != version { + continue + } + + job = v + if i+1 <= len(diffs) { + diff = diffs[i] + nextVersion = *versions[i+1].Version + } + } + + if err := c.formatJobVersion(job, diff, nextVersion, full); err != nil { + c.Ui.Error(err.Error()) + return 1 + } + + } else { + if err := c.formatJobVersions(versions, diffs, full); err != nil { + c.Ui.Error(err.Error()) + return 1 + } + } + return 0 } + +// parseVersion parses the version flag and returns the index, whether it +// was set and potentially an error during parsing. +func parseVersion(input string) (uint64, bool, error) { + if input == "" { + return 0, false, nil + } + + u, err := strconv.ParseUint(input, 10, 64) + return u, true, err +} + +func (c *JobHistoryCommand) formatJobVersions(versions []*api.Job, diffs []*api.JobDiff, full bool) error { + vLen := len(versions) + dLen := len(diffs) + if dLen != 0 && vLen != dLen+1 { + return fmt.Errorf("Number of job versions %d doesn't match number of diffs %d", vLen, dLen) + } + + for i, version := range versions { + var diff *api.JobDiff + var nextVersion uint64 + if i+1 <= dLen { + diff = diffs[i] + nextVersion = *versions[i+1].Version + } + + if err := c.formatJobVersion(version, diff, nextVersion, full); err != nil { + return err + } + + // Insert a blank + if i != vLen-1 { + c.Ui.Output("") + } + } + + return nil +} + +func (c *JobHistoryCommand) formatJobVersion(job *api.Job, diff *api.JobDiff, nextVersion uint64, full bool) error { + basic := []string{ + fmt.Sprintf("Version|%d", *job.Version), + fmt.Sprintf("Stable|%v", *job.Stable), + } + + if diff != nil { + //diffStr := fmt.Sprintf("Difference between version %d and %d:", *job.Version, nextVersion) + basic = append(basic, fmt.Sprintf("Diff|\n%s", strings.TrimSpace(formatJobDiff(diff, false)))) + } + + if full { + out, err := c.formatter.TransformData(job) + if err != nil { + return fmt.Errorf("Error formatting the data: %s", err) + } + + basic = append(basic, fmt.Sprintf("Full|JSON Job:\n%s", out)) + } + + columnConf := columnize.DefaultConfig() + columnConf.Glue = " = " + columnConf.NoTrim = true + output := columnize.Format(basic, columnConf) + + c.Ui.Output(c.Colorize().Color(output)) + return nil +} diff --git a/command/job_history_test.go b/command/job_history_test.go new file mode 100644 index 000000000000..bf8dd5dfb63d --- /dev/null +++ b/command/job_history_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestJobHistoryCommand_Implements(t *testing.T) { + var _ cli.Command = &JobDispatchCommand{} +} + +func TestJobHistoryCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &JobHistoryCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "foo"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error listing jobs") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 5ae2cd6ef8c7..7317db878354 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -582,15 +582,16 @@ func (j *Job) GetJobVersions(args *structs.JobVersionsRequest, reply.Index = out[0].ModifyIndex // Compute the diffs - for i := 0; i < len(out)-1; i++ { - old, new := out[i+1], out[i] - d, err := old.Diff(new, true) - if err != nil { - return fmt.Errorf("failed to create job diff: %v", err) + if args.Diffs { + for i := 0; i < len(out)-1; i++ { + old, new := out[i+1], out[i] + d, err := old.Diff(new, true) + if err != nil { + return fmt.Errorf("failed to create job diff: %v", err) + } + reply.Diffs = append(reply.Diffs, d) } - reply.Diffs = append(reply.Diffs, d) } - } else { // Use the last index that affected the nodes table index, err := state.Index("job_version") diff --git a/vendor/github.com/ryanuber/columnize/columnize.go b/vendor/github.com/ryanuber/columnize/columnize.go index 9f0fe72944c2..cbae439c2bc6 100644 --- a/vendor/github.com/ryanuber/columnize/columnize.go +++ b/vendor/github.com/ryanuber/columnize/columnize.go @@ -18,6 +18,8 @@ type Config struct { // A replacement string to replace empty fields Empty string + + NoTrim bool } // Returns a Config with default values. @@ -35,7 +37,10 @@ func getElementsFromLine(config *Config, line string) []interface{} { seperated := strings.Split(line, config.Delim) elements := make([]interface{}, len(seperated)) for i, field := range seperated { - value := strings.TrimSpace(field) + value := field + if !config.NoTrim { + value = strings.TrimSpace(field) + } if value == "" && config.Empty != "" { value = config.Empty } @@ -107,6 +112,9 @@ func MergeConfig(a, b *Config) *Config { if b.Empty != "" { result.Empty = b.Empty } + if b.NoTrim { + result.NoTrim = true + } return &result } From 3935656d141e88150b5504d1154c128a83d84e3b Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 19:08:25 -0700 Subject: [PATCH 050/105] Show submit time --- api/jobs.go | 5 +++++ command/inspect.go | 11 +---------- command/job_history.go | 13 +++---------- command/status.go | 8 +++++--- command/stop.go | 11 +---------- helper/funcs.go | 5 +++++ nomad/deployment_watcher_shims.go | 1 + nomad/job_endpoint.go | 4 ++++ nomad/periodic.go | 1 + nomad/structs/structs.go | 11 +++++++++++ 10 files changed, 37 insertions(+), 33 deletions(-) diff --git a/api/jobs.go b/api/jobs.go index 3daa438f8082..b79823159271 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -458,6 +458,7 @@ type Job struct { StatusDescription *string Stable *bool Version *uint64 + SubmitTime *int64 CreateIndex *uint64 ModifyIndex *uint64 JobModifyIndex *uint64 @@ -513,6 +514,9 @@ func (j *Job) Canonicalize() { if j.Version == nil { j.Version = helper.Uint64ToPtr(0) } + if j.SubmitTime == nil { + j.SubmitTime = helper.Int64ToPtr(0) + } if j.CreateIndex == nil { j.CreateIndex = helper.Uint64ToPtr(0) } @@ -588,6 +592,7 @@ type JobListStub struct { CreateIndex uint64 ModifyIndex uint64 JobModifyIndex uint64 + SubmitTime int64 } // JobIDSort is used to sort jobs by their job ID's. diff --git a/command/inspect.go b/command/inspect.go index 18fee7916509..d1e3c20cacc9 100644 --- a/command/inspect.go +++ b/command/inspect.go @@ -112,16 +112,7 @@ func (c *InspectCommand) Run(args []string) int { return 1 } if len(jobs) > 1 && strings.TrimSpace(jobID) != jobs[0].ID { - out := make([]string, len(jobs)+1) - out[0] = "ID|Type|Priority|Status" - for i, job := range jobs { - out[i+1] = fmt.Sprintf("%s|%s|%d|%s", - job.ID, - job.Type, - job.Priority, - job.Status) - } - c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", formatList(out))) + c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", createStatusListOutput(jobs))) return 0 } diff --git a/command/job_history.go b/command/job_history.go index 3dedecaf9cfe..4c9fb282ba9c 100644 --- a/command/job_history.go +++ b/command/job_history.go @@ -4,6 +4,7 @@ import ( "fmt" "strconv" "strings" + "time" "github.com/dadgar/columnize" "github.com/hashicorp/nomad/api" @@ -86,16 +87,7 @@ func (c *JobHistoryCommand) Run(args []string) int { return 1 } if len(jobs) > 1 && strings.TrimSpace(jobID) != jobs[0].ID { - out := make([]string, len(jobs)+1) - out[0] = "ID|Type|Priority|Status" - for i, job := range jobs { - out[i+1] = fmt.Sprintf("%s|%s|%d|%s", - job.ID, - job.Type, - job.Priority, - job.Status) - } - c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", formatList(out))) + c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", createStatusListOutput(jobs))) return 0 } @@ -193,6 +185,7 @@ func (c *JobHistoryCommand) formatJobVersion(job *api.Job, diff *api.JobDiff, ne basic := []string{ fmt.Sprintf("Version|%d", *job.Version), fmt.Sprintf("Stable|%v", *job.Stable), + fmt.Sprintf("Submit Date|%v", formatTime(time.Unix(0, *job.SubmitTime))), } if diff != nil { diff --git a/command/status.go b/command/status.go index 9a09cff0d161..9dffc6ff852a 100644 --- a/command/status.go +++ b/command/status.go @@ -138,6 +138,7 @@ func (c *StatusCommand) Run(args []string) int { basic := []string{ fmt.Sprintf("ID|%s", *job.ID), fmt.Sprintf("Name|%s", *job.Name), + fmt.Sprintf("Submit Date|%s", formatTime(time.Unix(0, *job.SubmitTime))), fmt.Sprintf("Type|%s", *job.Type), fmt.Sprintf("Priority|%d", *job.Priority), fmt.Sprintf("Datacenters|%s", strings.Join(job.Datacenters, ",")), @@ -446,13 +447,14 @@ func (c *StatusCommand) outputFailedPlacements(failedEval *api.Evaluation) { // list general information about a list of jobs func createStatusListOutput(jobs []*api.JobListStub) string { out := make([]string, len(jobs)+1) - out[0] = "ID|Type|Priority|Status" + out[0] = "ID|Type|Priority|Status|Submit Date" for i, job := range jobs { - out[i+1] = fmt.Sprintf("%s|%s|%d|%s", + out[i+1] = fmt.Sprintf("%s|%s|%d|%s|%s", job.ID, getTypeString(job), job.Priority, - getStatusString(job.Status, job.Stop)) + getStatusString(job.Status, job.Stop), + formatTime(time.Unix(0, job.SubmitTime))) } return formatList(out) } diff --git a/command/stop.go b/command/stop.go index 7fa25c99a8c0..d17990d177fa 100644 --- a/command/stop.go +++ b/command/stop.go @@ -94,16 +94,7 @@ func (c *StopCommand) Run(args []string) int { return 1 } if len(jobs) > 1 && strings.TrimSpace(jobID) != jobs[0].ID { - out := make([]string, len(jobs)+1) - out[0] = "ID|Type|Priority|Status" - for i, job := range jobs { - out[i+1] = fmt.Sprintf("%s|%s|%d|%s", - job.ID, - job.Type, - job.Priority, - job.Status) - } - c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", formatList(out))) + c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", createStatusListOutput(jobs))) return 0 } // Prefix lookup matched a single job diff --git a/helper/funcs.go b/helper/funcs.go index f461669a9846..cd1e9c6d5217 100644 --- a/helper/funcs.go +++ b/helper/funcs.go @@ -49,6 +49,11 @@ func IntToPtr(i int) *int { return &i } +// Int64ToPtr returns the pointer to an int +func Int64ToPtr(i int64) *int64 { + return &i +} + // UintToPtr returns the pointer to an uint func Uint64ToPtr(u uint64) *uint64 { return &u diff --git a/nomad/deployment_watcher_shims.go b/nomad/deployment_watcher_shims.go index f21ed6af066e..b52f64ce5c8d 100644 --- a/nomad/deployment_watcher_shims.go +++ b/nomad/deployment_watcher_shims.go @@ -98,6 +98,7 @@ func (d *deploymentWatcherRaftShim) UpsertEvals(evals []*structs.Evaluation) (ui } func (d *deploymentWatcherRaftShim) UpsertJob(job *structs.Job) (uint64, error) { + job.SetSubmitTime() update := &structs.JobRegisterRequest{ Job: job, } diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 7317db878354..3d1f608c6e8d 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -145,6 +145,9 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis // Check if the job has changed at all if existingJob == nil || existingJob.SpecChanged(args.Job) { + // Set the submit time + args.Job.SetSubmitTime() + // Commit this update via Raft _, index, err := j.srv.raftApply(structs.JobRegisterRequestType, args) if err != nil { @@ -1070,6 +1073,7 @@ func (j *Job) Dispatch(args *structs.JobDispatchRequest, reply *structs.JobDispa dispatchJob.ID = structs.DispatchedID(parameterizedJob.ID, time.Now()) dispatchJob.ParentID = parameterizedJob.ID dispatchJob.Name = dispatchJob.ID + dispatchJob.SetSubmitTime() // Merge in the meta data for k, v := range args.Meta { diff --git a/nomad/periodic.go b/nomad/periodic.go index 7d4e324580b7..d01f26acecac 100644 --- a/nomad/periodic.go +++ b/nomad/periodic.go @@ -46,6 +46,7 @@ type JobEvalDispatcher interface { // evaluation and the job to the raft log. It returns the eval. func (s *Server) DispatchJob(job *structs.Job) (*structs.Evaluation, error) { // Commit this update via Raft + job.SetSubmitTime() req := structs.JobRegisterRequest{Job: job} _, index, err := s.raftApply(structs.JobRegisterRequestType, req) if err != nil { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 6c7f80c658e2..5d342df88ca8 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1411,6 +1411,10 @@ type Job struct { // on each job register. Version uint64 + // SubmitTime is the time at which the job was submitted as a UnixNano in + // UTC + SubmitTime int64 + // Raft Indexes CreateIndex uint64 ModifyIndex uint64 @@ -1679,6 +1683,7 @@ func (j *Job) Stub(summary *JobSummary) *JobListStub { CreateIndex: j.CreateIndex, ModifyIndex: j.ModifyIndex, JobModifyIndex: j.JobModifyIndex, + SubmitTime: j.SubmitTime, JobSummary: summary, } } @@ -1783,11 +1788,16 @@ func (j *Job) SpecChanged(new *Job) bool { c.CreateIndex = j.CreateIndex c.ModifyIndex = j.ModifyIndex c.JobModifyIndex = j.JobModifyIndex + c.SubmitTime = j.SubmitTime // Deep equals the jobs return !reflect.DeepEqual(j, c) } +func (j *Job) SetSubmitTime() { + j.SubmitTime = time.Now().UTC().UnixNano() +} + // JobListStub is used to return a subset of job information // for the job list type JobListStub struct { @@ -1805,6 +1815,7 @@ type JobListStub struct { CreateIndex uint64 ModifyIndex uint64 JobModifyIndex uint64 + SubmitTime int64 } // JobSummary summarizes the state of the allocations of a job From fbd2b739048bd61c131685ed476e915947de21d9 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 20:28:52 -0700 Subject: [PATCH 051/105] job revert --- api/jobs.go | 1 - command/job_revert.go | 120 +++++++++++++++++++++++++++++++++++++ command/job_revert_test.go | 34 +++++++++++ commands.go | 5 ++ main.go | 2 +- 5 files changed, 160 insertions(+), 2 deletions(-) create mode 100644 command/job_revert.go create mode 100644 command/job_revert_test.go diff --git a/api/jobs.go b/api/jobs.go index b79823159271..7687e9b7d14c 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -247,7 +247,6 @@ func (j *Jobs) Dispatch(jobID string, meta map[string]string, return &resp, wm, nil } -// XXX // Revert is used to revert the given job to the passed version. If // enforceVersion is set, the job is only reverted if the current version is at // the passed version. diff --git a/command/job_revert.go b/command/job_revert.go new file mode 100644 index 000000000000..819638fea79c --- /dev/null +++ b/command/job_revert.go @@ -0,0 +1,120 @@ +package command + +import ( + "fmt" + "strings" +) + +type JobRevertCommand struct { + Meta + formatter DataFormatter +} + +func (c *JobRevertCommand) Help() string { + helpText := ` +Usage: nomad job revert [options] + +Revert is used to revert a job to a prior version of the job. The available +versions to revert to can be found using "nomad job history" command. + +General Options: + + ` + generalOptionsUsage() + ` + +History Options: + + -job-version + Revert to the given job version. + + -detach + Return immediately instead of entering monitor mode. After job dispatch, + the evaluation ID will be printed to the screen, which can be used to + examine the evaluation using the eval-status command. + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *JobRevertCommand) Synopsis() string { + return "Revert to a prior version of the job" +} + +func (c *JobRevertCommand) Run(args []string) int { + var detach, verbose bool + var versionStr string + + flags := c.Meta.FlagSet("job history", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.StringVar(&versionStr, "job-version", "", "") + flags.BoolVar(&detach, "detach", false, "") + flags.BoolVar(&verbose, "verbose", false, "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Check that we got exactly one node + args = flags.Args() + if l := len(args); l < 1 || l > 2 { + c.Ui.Error(c.Help()) + return 1 + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + jobID := args[0] + + // Check if the job exists + jobs, _, err := client.Jobs().PrefixList(jobID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error listing jobs: %s", err)) + return 1 + } + if len(jobs) == 0 { + c.Ui.Error(fmt.Sprintf("No job(s) with prefix or id %q found", jobID)) + return 1 + } + if len(jobs) > 1 && strings.TrimSpace(jobID) != jobs[0].ID { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", createStatusListOutput(jobs))) + return 0 + } + + revertVersion, ok, err := parseVersion(versionStr) + if !ok { + c.Ui.Error("The job version to revert to must be specified using the -job-version flag") + return 1 + } + if err != nil { + c.Ui.Error(fmt.Sprintf("Failed to parse job-version flag: %v", err)) + return 1 + } + + // Prefix lookup matched a single job + resp, _, err := client.Jobs().Revert(jobs[0].ID, revertVersion, nil, nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving job versions: %s", err)) + return 1 + } + + // Nothing to do + evalCreated := resp.EvalID != "" + if detach || !evalCreated { + return 0 + } + + mon := newMonitor(c.Ui, client, length) + return mon.monitor(resp.EvalID, false) +} diff --git a/command/job_revert_test.go b/command/job_revert_test.go new file mode 100644 index 000000000000..fbc665bb9f45 --- /dev/null +++ b/command/job_revert_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestJobRevertCommand_Implements(t *testing.T) { + var _ cli.Command = &JobDispatchCommand{} +} + +func TestJobRevertCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &JobRevertCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "foo"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error listing jobs") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/commands.go b/commands.go index aabdc90c43a8..b3164ea55f23 100644 --- a/commands.go +++ b/commands.go @@ -104,6 +104,11 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "job revert": func() (cli.Command, error) { + return &command.JobRevertCommand{ + Meta: meta, + }, nil + }, "logs": func() (cli.Command, error) { return &command.LogsCommand{ Meta: meta, diff --git a/main.go b/main.go index 7bb9afe471b9..c036d386669a 100644 --- a/main.go +++ b/main.go @@ -40,7 +40,7 @@ func RunCustom(args []string, commands map[string]cli.CommandFactory) int { case "executor": case "syslog": case "operator raft", "operator raft list-peers", "operator raft remove-peer": - case "job dispatch", "job history": + case "job dispatch", "job history", "job revert": case "fs ls", "fs cat", "fs stat": case "check": default: From 3d77a58f28db0a7da83705bc2ac390d521c6aa0b Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 29 Jun 2017 20:42:53 -0700 Subject: [PATCH 052/105] small fixes --- command/job_revert.go | 34 ++++++------- vendor/github.com/mitchellh/cli/cli.go | 12 +++-- vendor/github.com/mitchellh/cli/ui_mock.go | 55 +++++++++++++++++++--- vendor/vendor.json | 6 +-- 4 files changed, 75 insertions(+), 32 deletions(-) diff --git a/command/job_revert.go b/command/job_revert.go index 819638fea79c..9226f36e243e 100644 --- a/command/job_revert.go +++ b/command/job_revert.go @@ -12,7 +12,7 @@ type JobRevertCommand struct { func (c *JobRevertCommand) Help() string { helpText := ` -Usage: nomad job revert [options] +Usage: nomad job revert [options] Revert is used to revert a job to a prior version of the job. The available versions to revert to can be found using "nomad job history" command. @@ -21,10 +21,7 @@ General Options: ` + generalOptionsUsage() + ` -History Options: - - -job-version - Revert to the given job version. +Revert Options: -detach Return immediately instead of entering monitor mode. After job dispatch, @@ -43,11 +40,9 @@ func (c *JobRevertCommand) Synopsis() string { func (c *JobRevertCommand) Run(args []string) int { var detach, verbose bool - var versionStr string - flags := c.Meta.FlagSet("job history", FlagSetClient) + flags := c.Meta.FlagSet("job revert", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } - flags.StringVar(&versionStr, "job-version", "", "") flags.BoolVar(&detach, "detach", false, "") flags.BoolVar(&verbose, "verbose", false, "") @@ -61,9 +56,9 @@ func (c *JobRevertCommand) Run(args []string) int { length = fullId } - // Check that we got exactly one node + // Check that we got two args args = flags.Args() - if l := len(args); l < 1 || l > 2 { + if l := len(args); l != 2 { c.Ui.Error(c.Help()) return 1 } @@ -76,6 +71,15 @@ func (c *JobRevertCommand) Run(args []string) int { } jobID := args[0] + revertVersion, ok, err := parseVersion(args[1]) + if !ok { + c.Ui.Error("The job version to revert to must be specified using the -job-version flag") + return 1 + } + if err != nil { + c.Ui.Error(fmt.Sprintf("Failed to parse job-version flag: %v", err)) + return 1 + } // Check if the job exists jobs, _, err := client.Jobs().PrefixList(jobID) @@ -92,16 +96,6 @@ func (c *JobRevertCommand) Run(args []string) int { return 0 } - revertVersion, ok, err := parseVersion(versionStr) - if !ok { - c.Ui.Error("The job version to revert to must be specified using the -job-version flag") - return 1 - } - if err != nil { - c.Ui.Error(fmt.Sprintf("Failed to parse job-version flag: %v", err)) - return 1 - } - // Prefix lookup matched a single job resp, _, err := client.Jobs().Revert(jobs[0].ID, revertVersion, nil, nil) if err != nil { diff --git a/vendor/github.com/mitchellh/cli/cli.go b/vendor/github.com/mitchellh/cli/cli.go index 350575e64682..4a69d176db7e 100644 --- a/vendor/github.com/mitchellh/cli/cli.go +++ b/vendor/github.com/mitchellh/cli/cli.go @@ -120,7 +120,13 @@ func (c *CLI) Run() (int, error) { // Just show the version and exit if instructed. if c.IsVersion() && c.Version != "" { c.HelpWriter.Write([]byte(c.Version + "\n")) - return 1, nil + return 0, nil + } + + // Just print the help when only '-h' or '--help' is passed. + if c.IsHelp() && c.Subcommand() == "" { + c.HelpWriter.Write([]byte(c.HelpFunc(c.Commands) + "\n")) + return 0, nil } // Attempt to get the factory function for creating the command @@ -133,13 +139,13 @@ func (c *CLI) Run() (int, error) { command, err := raw.(CommandFactory)() if err != nil { - return 0, err + return 1, err } // If we've been instructed to just print the help, then print it if c.IsHelp() { c.commandHelp(command) - return 1, nil + return 0, nil } // If there is an invalid flag, then error diff --git a/vendor/github.com/mitchellh/cli/ui_mock.go b/vendor/github.com/mitchellh/cli/ui_mock.go index c46772855e31..bdae2a664601 100644 --- a/vendor/github.com/mitchellh/cli/ui_mock.go +++ b/vendor/github.com/mitchellh/cli/ui_mock.go @@ -7,12 +7,25 @@ import ( "sync" ) -// MockUi is a mock UI that is used for tests and is exported publicly for -// use in external tests if needed as well. +// NewMockUi returns a fully initialized MockUi instance +// which is safe for concurrent use. +func NewMockUi() *MockUi { + m := new(MockUi) + m.once.Do(m.init) + return m +} + +// MockUi is a mock UI that is used for tests and is exported publicly +// for use in external tests if needed as well. Do not instantite this +// directly since the buffers will be initialized on the first write. If +// there is no write then you will get a nil panic. Please use the +// NewMockUi() constructor function instead. You can fix your code with +// +// sed -i -e 's/new(cli.MockUi)/cli.NewMockUi()/g' *_test.go type MockUi struct { InputReader io.Reader - ErrorWriter *bytes.Buffer - OutputWriter *bytes.Buffer + ErrorWriter *syncBuffer + OutputWriter *syncBuffer once sync.Once } @@ -59,6 +72,36 @@ func (u *MockUi) Warn(message string) { } func (u *MockUi) init() { - u.ErrorWriter = new(bytes.Buffer) - u.OutputWriter = new(bytes.Buffer) + u.ErrorWriter = new(syncBuffer) + u.OutputWriter = new(syncBuffer) +} + +type syncBuffer struct { + sync.RWMutex + b bytes.Buffer +} + +func (b *syncBuffer) Write(data []byte) (int, error) { + b.Lock() + defer b.Unlock() + return b.b.Write(data) +} + +func (b *syncBuffer) Read(data []byte) (int, error) { + b.RLock() + defer b.RUnlock() + return b.b.Read(data) +} + +func (b *syncBuffer) Reset() { + b.Lock() + b.b.Reset() + b.Unlock() +} + +func (b *syncBuffer) String() string { + b.RLock() + data := b.b.Bytes() + b.RUnlock() + return string(data) } diff --git a/vendor/vendor.json b/vendor/vendor.json index 01f87fc24cd2..6920b7b8b206 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -960,10 +960,10 @@ "revision": "7e024ce8ce18b21b475ac6baf8fa3c42536bf2fa" }, { - "checksumSHA1": "PfnjkP75J4WIemUueJcrH2VbRHY=", + "checksumSHA1": "bUuI7AVR3IZPLlBaEKmw/ke7wqA=", "path": "github.com/mitchellh/cli", - "revision": "8d6d9ab3c912dcb005ece87c40a41b9e73e1999a", - "revisionTime": "2017-03-03T02:36:54Z" + "revision": "b481eac70eea3ad671b7c360a013f89bb759b252", + "revisionTime": "2017-05-23T17:27:49Z" }, { "checksumSHA1": "ttEN1Aupb7xpPMkQLqb3tzLFdXs=", From dc3d50011944eb7a635347744de91f829983d341 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 10:59:19 -0700 Subject: [PATCH 053/105] deployment list --- command/deployment.go | 19 +++++++ command/deployment_list.go | 97 +++++++++++++++++++++++++++++++++ command/deployment_list_test.go | 34 ++++++++++++ commands.go | 10 ++++ main.go | 9 +-- nomad/structs/structs.go | 15 ++--- 6 files changed, 173 insertions(+), 11 deletions(-) create mode 100644 command/deployment.go create mode 100644 command/deployment_list.go create mode 100644 command/deployment_list_test.go diff --git a/command/deployment.go b/command/deployment.go new file mode 100644 index 000000000000..8c4328a53806 --- /dev/null +++ b/command/deployment.go @@ -0,0 +1,19 @@ +package command + +import "github.com/mitchellh/cli" + +type DeploymentCommand struct { + Meta +} + +func (f *DeploymentCommand) Help() string { + return "This command is accessed by using one of the subcommands below." +} + +func (f *DeploymentCommand) Synopsis() string { + return "Interact with deployments" +} + +func (f *DeploymentCommand) Run(args []string) int { + return cli.RunResultHelp +} diff --git a/command/deployment_list.go b/command/deployment_list.go new file mode 100644 index 000000000000..16215e2bd832 --- /dev/null +++ b/command/deployment_list.go @@ -0,0 +1,97 @@ +package command + +import ( + "fmt" + "strings" + + "github.com/hashicorp/nomad/api" +) + +type DeploymentListCommand struct { + Meta +} + +func (c *DeploymentListCommand) Help() string { + helpText := ` +Usage: nomad deployment list [options] + +List is used to list the set of deployments tracked by Nomad. + +General Options: + + ` + generalOptionsUsage() + ` + +List Options: + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *DeploymentListCommand) Synopsis() string { + return "List all deployments" +} + +func (c *DeploymentListCommand) Run(args []string) int { + var diff, full, verbose bool + var versionStr string + + flags := c.Meta.FlagSet("job history", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&diff, "p", false, "") + flags.BoolVar(&full, "full", false, "") + flags.BoolVar(&verbose, "verbose", false, "") + flags.StringVar(&versionStr, "job-version", "", "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got no arguments + args = flags.Args() + if l := len(args); l != 0 { + c.Ui.Error(c.Help()) + return 1 + } + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + deploys, _, err := client.Deployments().List(nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployments: %s", err)) + return 1 + } + + c.Ui.Output(formatDeployments(deploys, length)) + return 0 +} + +func formatDeployments(deploys []*api.Deployment, uuidLength int) string { + if len(deploys) == 0 { + return "No deployments found" + } + + rows := make([]string, len(deploys)+1) + rows[0] = "ID|Job ID|Job Version|Status|Description" + for i, d := range deploys { + rows[i+1] = fmt.Sprintf("%s|%s|%d|%s|%s", + limit(d.ID, uuidLength), + d.JobID, + d.JobVersion, + d.Status, + d.StatusDescription) + } + return formatList(rows) +} diff --git a/command/deployment_list_test.go b/command/deployment_list_test.go new file mode 100644 index 000000000000..eaeb00e39201 --- /dev/null +++ b/command/deployment_list_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestDeploymentListCommand_Implements(t *testing.T) { + var _ cli.Command = &DeploymentListCommand{} +} + +func TestDeploymentListCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &DeploymentListCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployments") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/commands.go b/commands.go index b3164ea55f23..7b8fa0d9b328 100644 --- a/commands.go +++ b/commands.go @@ -54,6 +54,16 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "deployment": func() (cli.Command, error) { + return &command.DeploymentCommand{ + Meta: meta, + }, nil + }, + "deployment list": func() (cli.Command, error) { + return &command.DeploymentListCommand{ + Meta: meta, + }, nil + }, "eval-status": func() (cli.Command, error) { return &command.EvalStatusCommand{ Meta: meta, diff --git a/main.go b/main.go index c036d386669a..cd20f7d5ec9f 100644 --- a/main.go +++ b/main.go @@ -37,12 +37,13 @@ func RunCustom(args []string, commands map[string]cli.CommandFactory) int { commandsInclude := make([]string, 0, len(commands)) for k, _ := range commands { switch k { + case "check": + case "deployment list", "deployment status", "deployment pause", "deployment resume", "deployment fail": case "executor": - case "syslog": - case "operator raft", "operator raft list-peers", "operator raft remove-peer": - case "job dispatch", "job history", "job revert": case "fs ls", "fs cat", "fs stat": - case "check": + case "job dispatch", "job history", "job revert": + case "operator raft", "operator raft list-peers", "operator raft remove-peer": + case "syslog": default: commandsInclude = append(commandsInclude, k) } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 5d342df88ca8..6066f56d3ff6 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3947,13 +3947,14 @@ type Deployment struct { // NewDeployment creates a new deployment given the job. func NewDeployment(job *Job) *Deployment { return &Deployment{ - ID: GenerateUUID(), - JobID: job.ID, - JobVersion: job.Version, - JobModifyIndex: job.ModifyIndex, - JobCreateIndex: job.CreateIndex, - Status: DeploymentStatusRunning, - TaskGroups: make(map[string]*DeploymentState, len(job.TaskGroups)), + ID: GenerateUUID(), + JobID: job.ID, + JobVersion: job.Version, + JobModifyIndex: job.ModifyIndex, + JobCreateIndex: job.CreateIndex, + Status: DeploymentStatusRunning, + StatusDescription: DeploymentStatusDescriptionRunning, + TaskGroups: make(map[string]*DeploymentState, len(job.TaskGroups)), } } From bab25f683414c872d7ed5d3ffbd9783d8dde9e04 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 12:35:59 -0700 Subject: [PATCH 054/105] deployment status --- api/deployments.go | 1 + command/deployment_list.go | 8 +- command/deployment_status.go | 215 ++++++++++++++++++ command/deployment_status_test.go | 34 +++ commands.go | 5 + nomad/deploymentwatcher/deployment_watcher.go | 20 +- nomad/state/state_store.go | 2 +- nomad/structs/structs.go | 5 + scheduler/reconcile.go | 8 +- 9 files changed, 274 insertions(+), 24 deletions(-) create mode 100644 command/deployment_status.go create mode 100644 command/deployment_status_test.go diff --git a/api/deployments.go b/api/deployments.go index feeedad8ad54..36e86e6df1f5 100644 --- a/api/deployments.go +++ b/api/deployments.go @@ -139,6 +139,7 @@ type Deployment struct { // DeploymentState tracks the state of a deployment for a given task group. type DeploymentState struct { + AutoRevert bool Promoted bool DesiredCanaries int DesiredTotal int diff --git a/command/deployment_list.go b/command/deployment_list.go index 16215e2bd832..600a4e0dfabe 100644 --- a/command/deployment_list.go +++ b/command/deployment_list.go @@ -34,15 +34,11 @@ func (c *DeploymentListCommand) Synopsis() string { } func (c *DeploymentListCommand) Run(args []string) int { - var diff, full, verbose bool - var versionStr string + var verbose bool - flags := c.Meta.FlagSet("job history", FlagSetClient) + flags := c.Meta.FlagSet("deployment list", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } - flags.BoolVar(&diff, "p", false, "") - flags.BoolVar(&full, "full", false, "") flags.BoolVar(&verbose, "verbose", false, "") - flags.StringVar(&versionStr, "job-version", "", "") if err := flags.Parse(args); err != nil { return 1 diff --git a/command/deployment_status.go b/command/deployment_status.go new file mode 100644 index 000000000000..5a6b89a80a03 --- /dev/null +++ b/command/deployment_status.go @@ -0,0 +1,215 @@ +package command + +import ( + "fmt" + "strings" + + "github.com/hashicorp/nomad/api" +) + +type DeploymentStatusCommand struct { + Meta +} + +func (c *DeploymentStatusCommand) Help() string { + helpText := ` +Usage: nomad deployment status [options] + +Status is used to display the status of a deployment. The status will display +the number of desired changes as well as the currently applied changes. + +General Options: + + ` + generalOptionsUsage() + ` + +Status Options: + + -verbose + Display full information. + + -json + Output the allocation in its JSON format. + + -t + Format and display allocation using a Go template. +` + return strings.TrimSpace(helpText) +} + +func (c *DeploymentStatusCommand) Synopsis() string { + return "Display the status of a deployment" +} + +func (c *DeploymentStatusCommand) Run(args []string) int { + var json, verbose bool + var tmpl string + + flags := c.Meta.FlagSet("deployment status", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&verbose, "verbose", false, "") + flags.BoolVar(&json, "json", false, "") + flags.StringVar(&tmpl, "t", "", "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got no arguments + args = flags.Args() + if l := len(args); l != 1 { + c.Ui.Error(c.Help()) + return 1 + } + + dID := args[0] + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + // Do a prefix lookup + deploy, possible, err := getDeployment(client.Deployments(), dID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployment: %s", err)) + return 1 + } + + if len(possible) != 0 { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple deployments\n\n%s", formatDeployments(possible, length))) + return 0 + } + + var format string + if json && len(tmpl) > 0 { + c.Ui.Error("Both -json and -t are not allowed") + return 1 + } else if json { + format = "json" + } else if len(tmpl) > 0 { + format = "template" + } + if len(format) > 0 { + f, err := DataFormat(format, tmpl) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + return 1 + } + + out, err := f.TransformData(deploy) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) + return 1 + } + c.Ui.Output(out) + return 0 + } + + c.Ui.Output(c.Colorize().Color(formatDeployment(deploy, length))) + return 0 +} + +func getDeployment(client *api.Deployments, dID string) (match *api.Deployment, possible []*api.Deployment, err error) { + // First attempt an immediate lookup if we have a proper length + if len(dID) == 36 { + d, _, err := client.Info(dID, nil) + if err != nil { + return nil, nil, err + } + + return d, nil, nil + } + + dID = strings.Replace(dID, "-", "", -1) + if len(dID) == 1 { + return nil, nil, fmt.Errorf("Identifier must contain at least two characters.") + } + if len(dID)%2 == 1 { + // Identifiers must be of even length, so we strip off the last byte + // to provide a consistent user experience. + dID = dID[:len(dID)-1] + } + + // Have to do a prefix lookup + deploys, _, err := client.PrefixList(dID) + if err != nil { + return nil, nil, err + } + + l := len(deploys) + switch { + case l == 0: + return nil, nil, fmt.Errorf("Deployment ID %q matched no deployments", dID) + case l == 1: + return deploys[0], nil, nil + default: + return nil, deploys, nil + } +} + +func formatDeployment(d *api.Deployment, uuidLength int) string { + // Format the high-level elements + high := []string{ + fmt.Sprintf("ID|%s", limit(d.ID, uuidLength)), + fmt.Sprintf("Job ID|%s", limit(d.JobID, uuidLength)), + fmt.Sprintf("Job Version|%d", d.JobVersion), + fmt.Sprintf("Status|%s", d.Status), + fmt.Sprintf("Description|%s", d.StatusDescription), + } + + base := formatKV(high) + if len(d.TaskGroups) == 0 { + return base + } + base += "\n\n[bold]Deployed[reset]\n" + + // Detect if we need to add these columns + canaries, autorevert := false, false + for _, state := range d.TaskGroups { + if state.AutoRevert { + autorevert = true + } + if state.DesiredCanaries > 0 { + canaries = true + } + } + + // Build the row string + rowString := "Task Group|" + if autorevert { + rowString += "Auto Revert|" + } + rowString += "Desired|" + if canaries { + rowString += "Canaries|" + } + rowString += "Placed|Healthy|Unhealthy" + + rows := make([]string, len(d.TaskGroups)+1) + rows[0] = rowString + i := 1 + for tg, state := range d.TaskGroups { + row := fmt.Sprintf("%s|", tg) + if autorevert { + row += fmt.Sprintf("%v|", state.AutoRevert) + } + row += fmt.Sprintf("%d|", state.DesiredTotal) + if canaries { + row += fmt.Sprintf("%d|", state.DesiredCanaries) + } + row += fmt.Sprintf("%d|%d|%d", state.PlacedAllocs, state.HealthyAllocs, state.UnhealthyAllocs) + rows[i] = row + i++ + } + + base += formatList(rows) + return base +} diff --git a/command/deployment_status_test.go b/command/deployment_status_test.go new file mode 100644 index 000000000000..e257b1600b2d --- /dev/null +++ b/command/deployment_status_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestDeploymentStatusCommand_Implements(t *testing.T) { + var _ cli.Command = &DeploymentStatusCommand{} +} + +func TestDeploymentStatusCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &DeploymentStatusCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "12"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployment") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/commands.go b/commands.go index 7b8fa0d9b328..5fafe16dc130 100644 --- a/commands.go +++ b/commands.go @@ -64,6 +64,11 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "deployment status": func() (cli.Command, error) { + return &command.DeploymentStatusCommand{ + Meta: meta, + }, nil + }, "eval-status": func() (cli.Command, error) { return &command.EvalStatusCommand{ Meta: meta, diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 3151c910b5e7..1e751d3ca917 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -59,10 +59,6 @@ type deploymentWatcher struct { // j is the job the deployment is for j *structs.Job - // autorevert is used to lookup if an task group should autorevert on - // unhealthy allocations - autorevert map[string]bool - // outstandingBatch marks whether an outstanding function exists to create // the evaluation. Access should be done through the lock outstandingBatch bool @@ -89,7 +85,6 @@ func newDeploymentWatcher(parent context.Context, queryLimiter *rate.Limiter, queryLimiter: queryLimiter, d: d, j: j, - autorevert: make(map[string]bool, len(j.TaskGroups)), watchers: watchers, deploymentTriggers: triggers, logger: logger, @@ -97,15 +92,6 @@ func newDeploymentWatcher(parent context.Context, queryLimiter *rate.Limiter, exitFn: exitFn, } - // Determine what task groups will trigger an autorevert - for _, tg := range j.TaskGroups { - autorevert := false - if tg.Update != nil && tg.Update.AutoRevert { - autorevert = true - } - w.autorevert[tg.Name] = autorevert - } - // Start the long lived watcher that scans for allocation updates go w.watch() @@ -145,7 +131,8 @@ func (w *deploymentWatcher) SetAllocHealth( } // Check if the group has autorevert set - if !w.autorevert[alloc.TaskGroup] { + group, ok := w.d.TaskGroups[alloc.TaskGroup] + if !ok || !group.AutoRevert { continue } @@ -313,7 +300,8 @@ func (w *deploymentWatcher) watch() { if alloc.DeploymentStatus.IsUnhealthy() { // Check if the group has autorevert set - if w.autorevert[alloc.TaskGroup] { + group, ok := w.d.TaskGroups[alloc.TaskGroup] + if ok && group.AutoRevert { rollback = true } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index fbb2aed7212a..fb6aeb5e8296 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -331,7 +331,7 @@ func (s *StateStore) DeploymentByID(ws memdb.WatchSet, deploymentID string) (*st func (s *StateStore) deploymentByIDImpl(ws memdb.WatchSet, deploymentID string, txn *memdb.Txn) (*structs.Deployment, error) { watchCh, existing, err := txn.FirstWatch("deployment", "id", deploymentID) if err != nil { - return nil, fmt.Errorf("node lookup failed: %v", err) + return nil, fmt.Errorf("deployment lookup failed: %v", err) } ws.Add(watchCh) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 6066f56d3ff6..eb1784ad455f 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3993,6 +3993,10 @@ func (d *Deployment) GoString() string { // DeploymentState tracks the state of a deployment for a given task group. type DeploymentState struct { + // AutoRevert marks whether the task group has indicated the job should be + // reverted on failure + AutoRevert bool + // Promoted marks whether the canaries have been promoted Promoted bool @@ -4020,6 +4024,7 @@ func (d *DeploymentState) GoString() string { base += fmt.Sprintf("\nPlaced: %d", d.PlacedAllocs) base += fmt.Sprintf("\nHealthy: %d", d.HealthyAllocs) base += fmt.Sprintf("\nUnhealthy: %d", d.UnhealthyAllocs) + base += fmt.Sprintf("\nAutoRevert: %v", d.AutoRevert) return base } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index e6765719b74b..d36904f9c4e1 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -234,7 +234,13 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { dstate, existingDeployment = a.deployment.TaskGroups[group] } if !existingDeployment { - dstate = &structs.DeploymentState{} + autorevert := false + if tg.Update != nil && tg.Update.AutoRevert { + autorevert = true + } + dstate = &structs.DeploymentState{ + AutoRevert: autorevert, + } } // Handle stopping unneeded canaries and tracking placed canaries From d18a3df789ae2ffb862fe3fc15f542478ae0b0af Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 14:27:13 -0700 Subject: [PATCH 055/105] fail,pause,resume commands --- command/deployment_fail.go | 105 ++++++++++++++++++++++++++++++ command/deployment_fail_test.go | 34 ++++++++++ command/deployment_pause.go | 87 +++++++++++++++++++++++++ command/deployment_pause_test.go | 34 ++++++++++ command/deployment_resume.go | 103 +++++++++++++++++++++++++++++ command/deployment_resume_test.go | 34 ++++++++++ command/job_revert.go | 2 +- commands.go | 15 +++++ 8 files changed, 413 insertions(+), 1 deletion(-) create mode 100644 command/deployment_fail.go create mode 100644 command/deployment_fail_test.go create mode 100644 command/deployment_pause.go create mode 100644 command/deployment_pause_test.go create mode 100644 command/deployment_resume.go create mode 100644 command/deployment_resume_test.go diff --git a/command/deployment_fail.go b/command/deployment_fail.go new file mode 100644 index 000000000000..fdd3c59ba616 --- /dev/null +++ b/command/deployment_fail.go @@ -0,0 +1,105 @@ +package command + +import ( + "fmt" + "strings" +) + +type DeploymentFailCommand struct { + Meta +} + +func (c *DeploymentFailCommand) Help() string { + helpText := ` +Usage: nomad deployment fail [options] + +Fail is used to mark a deployment as failed. Failing a deployment will +stop the placement of new allocations as part of rolling deployment and +if the job is configured to auto revert, the job will attempt to roll back to a +stable version. + +General Options: + + ` + generalOptionsUsage() + ` + +Fail Options: + + -detach + Return immediately instead of entering monitor mode. After deployment + resume, the evaluation ID will be printed to the screen, which can be used + to examine the evaluation using the eval-status command. + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *DeploymentFailCommand) Synopsis() string { + return "Manually fail a deployment" +} + +func (c *DeploymentFailCommand) Run(args []string) int { + var detach, verbose bool + + flags := c.Meta.FlagSet("deployment resume", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&detach, "detach", false, "") + flags.BoolVar(&verbose, "verbose", false, "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got no arguments + args = flags.Args() + if l := len(args); l != 1 { + c.Ui.Error(c.Help()) + return 1 + } + + dID := args[0] + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + // Do a prefix lookup + deploy, possible, err := getDeployment(client.Deployments(), dID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployment: %s", err)) + return 1 + } + + if len(possible) != 0 { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple deployments\n\n%s", formatDeployments(possible, length))) + return 0 + } + + u, _, err := client.Deployments().Fail(deploy.ID, nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error failing deployment: %s", err)) + return 1 + } + + c.Ui.Output(fmt.Sprintf("Deployment %q failed", deploy.ID)) + evalCreated := u.EvalID != "" + + // Nothing to do + if detach || !evalCreated { + return 0 + } + + c.Ui.Output("") + mon := newMonitor(c.Ui, client, length) + return mon.monitor(u.EvalID, false) +} diff --git a/command/deployment_fail_test.go b/command/deployment_fail_test.go new file mode 100644 index 000000000000..7f734b000cfb --- /dev/null +++ b/command/deployment_fail_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestDeploymentFailCommand_Implements(t *testing.T) { + var _ cli.Command = &DeploymentFailCommand{} +} + +func TestDeploymentFailCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &DeploymentFailCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "12"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployment") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/command/deployment_pause.go b/command/deployment_pause.go new file mode 100644 index 000000000000..838c9e2e64db --- /dev/null +++ b/command/deployment_pause.go @@ -0,0 +1,87 @@ +package command + +import ( + "fmt" + "strings" +) + +type DeploymentPauseCommand struct { + Meta +} + +func (c *DeploymentPauseCommand) Help() string { + helpText := ` +Usage: nomad deployment pause [options] + +Pause is used to pause a deployment. Pausing a deployment will pause the +placement of new allocations as part of rolling deployment. + +General Options: + + ` + generalOptionsUsage() + ` + +Pause Options: + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *DeploymentPauseCommand) Synopsis() string { + return "Pause a deployment" +} + +func (c *DeploymentPauseCommand) Run(args []string) int { + var verbose bool + + flags := c.Meta.FlagSet("deployment pause", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&verbose, "verbose", false, "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got no arguments + args = flags.Args() + if l := len(args); l != 1 { + c.Ui.Error(c.Help()) + return 1 + } + + dID := args[0] + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + // Do a prefix lookup + deploy, possible, err := getDeployment(client.Deployments(), dID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployment: %s", err)) + return 1 + } + + if len(possible) != 0 { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple deployments\n\n%s", formatDeployments(possible, length))) + return 0 + } + + if _, _, err := client.Deployments().Pause(deploy.ID, true, nil); err != nil { + c.Ui.Error(fmt.Sprintf("Error pausing deployment: %s", err)) + return 1 + } + + c.Ui.Output(fmt.Sprintf("Deployment %q paused", deploy.ID)) + return 0 +} diff --git a/command/deployment_pause_test.go b/command/deployment_pause_test.go new file mode 100644 index 000000000000..279a4a9d2b1b --- /dev/null +++ b/command/deployment_pause_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestDeploymentPauseCommand_Implements(t *testing.T) { + var _ cli.Command = &DeploymentPauseCommand{} +} + +func TestDeploymentPauseCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &DeploymentPauseCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "12"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployment") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/command/deployment_resume.go b/command/deployment_resume.go new file mode 100644 index 000000000000..ff999aa77c3d --- /dev/null +++ b/command/deployment_resume.go @@ -0,0 +1,103 @@ +package command + +import ( + "fmt" + "strings" +) + +type DeploymentResumeCommand struct { + Meta +} + +func (c *DeploymentResumeCommand) Help() string { + helpText := ` +Usage: nomad deployment resume [options] + +Resume is used to unpause a paused deployment. Resuming a deployment will +resume the placement of new allocations as part of rolling deployment. + +General Options: + + ` + generalOptionsUsage() + ` + +Resume Options: + + -detach + Return immediately instead of entering monitor mode. After deployment + resume, the evaluation ID will be printed to the screen, which can be used + to examine the evaluation using the eval-status command. + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *DeploymentResumeCommand) Synopsis() string { + return "Resume a paused deployment" +} + +func (c *DeploymentResumeCommand) Run(args []string) int { + var detach, verbose bool + + flags := c.Meta.FlagSet("deployment resume", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&detach, "detach", false, "") + flags.BoolVar(&verbose, "verbose", false, "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got no arguments + args = flags.Args() + if l := len(args); l != 1 { + c.Ui.Error(c.Help()) + return 1 + } + + dID := args[0] + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + // Do a prefix lookup + deploy, possible, err := getDeployment(client.Deployments(), dID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployment: %s", err)) + return 1 + } + + if len(possible) != 0 { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple deployments\n\n%s", formatDeployments(possible, length))) + return 0 + } + + u, _, err := client.Deployments().Pause(deploy.ID, false, nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error resuming deployment: %s", err)) + return 1 + } + + c.Ui.Output(fmt.Sprintf("Deployment %q resumed", deploy.ID)) + evalCreated := u.EvalID != "" + + // Nothing to do + if detach || !evalCreated { + return 0 + } + + c.Ui.Output("") + mon := newMonitor(c.Ui, client, length) + return mon.monitor(u.EvalID, false) +} diff --git a/command/deployment_resume_test.go b/command/deployment_resume_test.go new file mode 100644 index 000000000000..b6833bab7231 --- /dev/null +++ b/command/deployment_resume_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestDeploymentResumeCommand_Implements(t *testing.T) { + var _ cli.Command = &DeploymentResumeCommand{} +} + +func TestDeploymentResumeCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &DeploymentResumeCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "12"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployment") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/command/job_revert.go b/command/job_revert.go index 9226f36e243e..141e33b9b9a4 100644 --- a/command/job_revert.go +++ b/command/job_revert.go @@ -24,7 +24,7 @@ General Options: Revert Options: -detach - Return immediately instead of entering monitor mode. After job dispatch, + Return immediately instead of entering monitor mode. After job revert, the evaluation ID will be printed to the screen, which can be used to examine the evaluation using the eval-status command. diff --git a/commands.go b/commands.go index 5fafe16dc130..989a263f7097 100644 --- a/commands.go +++ b/commands.go @@ -59,11 +59,26 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "deployment fail": func() (cli.Command, error) { + return &command.DeploymentFailCommand{ + Meta: meta, + }, nil + }, "deployment list": func() (cli.Command, error) { return &command.DeploymentListCommand{ Meta: meta, }, nil }, + "deployment pause": func() (cli.Command, error) { + return &command.DeploymentPauseCommand{ + Meta: meta, + }, nil + }, + "deployment resume": func() (cli.Command, error) { + return &command.DeploymentResumeCommand{ + Meta: meta, + }, nil + }, "deployment status": func() (cli.Command, error) { return &command.DeploymentStatusCommand{ Meta: meta, From 004a7661711c481ce9680590b0023b7a6b394355 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 16:22:06 -0700 Subject: [PATCH 056/105] deployment promote --- api/deployments.go | 1 - command/deployment_promote.go | 125 +++++++++++++++++++++++++++++ command/deployment_promote_test.go | 34 ++++++++ commands.go | 5 ++ main.go | 3 +- 5 files changed, 166 insertions(+), 2 deletions(-) create mode 100644 command/deployment_promote.go create mode 100644 command/deployment_promote_test.go diff --git a/api/deployments.go b/api/deployments.go index 36e86e6df1f5..5545c88d58d8 100644 --- a/api/deployments.go +++ b/api/deployments.go @@ -106,7 +106,6 @@ func (d *Deployments) PromoteGroups(deploymentID string, groups []string, q *Wri return &resp, wm, nil } -// XXX // SetAllocHealth is used to set allocation health for allocs that are part of // the given deployment func (d *Deployments) SetAllocHealth(deploymentID string, healthy, unhealthy []string, q *WriteOptions) (*DeploymentUpdateResponse, *WriteMeta, error) { diff --git a/command/deployment_promote.go b/command/deployment_promote.go new file mode 100644 index 000000000000..c0743dc05c05 --- /dev/null +++ b/command/deployment_promote.go @@ -0,0 +1,125 @@ +package command + +import ( + "fmt" + "strings" + + "github.com/hashicorp/nomad/api" + flaghelper "github.com/hashicorp/nomad/helper/flag-helpers" +) + +type DeploymentPromoteCommand struct { + Meta +} + +func (c *DeploymentPromoteCommand) Help() string { + helpText := ` +Usage: nomad deployment promote [options] + +Promote is used to promote task groups in a deployment. Promotion should occur +when the deployment has placed canaries for a task group and those canaries have +been deemed healthy. When a task group is promoted, the rolling upgrade of the +remaining allocations is unblocked. If the canaries are found to be unhealthy, +the deployment may either be failed using the "nomad deployment fail" command, +the job can be failed forward by submitting a new version or failed backwards by +reverting to an older version using the "nomad job revert" command. + +General Options: + + ` + generalOptionsUsage() + ` + +Promote Options: + + -all + All promotes all task groups in the deployment. + + -group + Group may be specified many times and is used to promote that particular + group. + + -detach + Return immediately instead of entering monitor mode. After deployment + resume, the evaluation ID will be printed to the screen, which can be used + to examine the evaluation using the eval-status command. + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *DeploymentPromoteCommand) Synopsis() string { + return "Manually fail a deployment" +} + +func (c *DeploymentPromoteCommand) Run(args []string) int { + var all, detach, verbose bool + var groups []string + + flags := c.Meta.FlagSet("deployment resume", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&detach, "all", false, "") + flags.BoolVar(&detach, "detach", false, "") + flags.BoolVar(&verbose, "verbose", false, "") + flags.Var((*flaghelper.StringFlag)(&groups), "group", "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got no arguments + args = flags.Args() + if l := len(args); l != 1 { + c.Ui.Error(c.Help()) + return 1 + } + + dID := args[0] + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + // Do a prefix lookup + deploy, possible, err := getDeployment(client.Deployments(), dID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployment: %s", err)) + return 1 + } + + if len(possible) != 0 { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple deployments\n\n%s", formatDeployments(possible, length))) + return 0 + } + + var u *api.DeploymentUpdateResponse + if all { + u, _, err = client.Deployments().PromoteAll(deploy.ID, nil) + } else { + u, _, err = client.Deployments().PromoteGroups(deploy.ID, groups, nil) + } + + if err != nil { + c.Ui.Error(fmt.Sprintf("Error failing deployment: %s", err)) + return 1 + } + + // Nothing to do + evalCreated := u.EvalID != "" + if detach || !evalCreated { + return 0 + } + + c.Ui.Output("") + mon := newMonitor(c.Ui, client, length) + return mon.monitor(u.EvalID, false) +} diff --git a/command/deployment_promote_test.go b/command/deployment_promote_test.go new file mode 100644 index 000000000000..b3ce0c85a7ab --- /dev/null +++ b/command/deployment_promote_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestDeploymentPromoteCommand_Implements(t *testing.T) { + var _ cli.Command = &DeploymentPromoteCommand{} +} + +func TestDeploymentPromoteCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &DeploymentPromoteCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "12"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployment") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/commands.go b/commands.go index 989a263f7097..13705a882b34 100644 --- a/commands.go +++ b/commands.go @@ -74,6 +74,11 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "deployment promote": func() (cli.Command, error) { + return &command.DeploymentPromoteCommand{ + Meta: meta, + }, nil + }, "deployment resume": func() (cli.Command, error) { return &command.DeploymentResumeCommand{ Meta: meta, diff --git a/main.go b/main.go index cd20f7d5ec9f..b5e216c60372 100644 --- a/main.go +++ b/main.go @@ -38,7 +38,8 @@ func RunCustom(args []string, commands map[string]cli.CommandFactory) int { for k, _ := range commands { switch k { case "check": - case "deployment list", "deployment status", "deployment pause", "deployment resume", "deployment fail": + case "deployment list", "deployment status", "deployment pause", + "deployment resume", "deployment fail", "deployment promote": case "executor": case "fs ls", "fs cat", "fs stat": case "job dispatch", "job history", "job revert": From 4ce7b62105769d9738e4b5e9f082810ca08431df Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 17:37:12 -0700 Subject: [PATCH 057/105] job deployments --- command/job_deployments.go | 107 ++++++++++++++++++++++++++++++++ command/job_deployments_test.go | 34 ++++++++++ commands.go | 5 ++ main.go | 2 +- 4 files changed, 147 insertions(+), 1 deletion(-) create mode 100644 command/job_deployments.go create mode 100644 command/job_deployments_test.go diff --git a/command/job_deployments.go b/command/job_deployments.go new file mode 100644 index 000000000000..8289674cf61b --- /dev/null +++ b/command/job_deployments.go @@ -0,0 +1,107 @@ +package command + +import ( + "fmt" + "strings" +) + +type JobDeploymentsCommand struct { + Meta + formatter DataFormatter +} + +func (c *JobDeploymentsCommand) Help() string { + helpText := ` +Usage: nomad job deployments [options] + +Deployments is used to display the deployments for a particular job. + +General Options: + + ` + generalOptionsUsage() + ` + +Deployments Options: + + -latest + Display the latest deployment only. + + -verbose + Display full information. +` + return strings.TrimSpace(helpText) +} + +func (c *JobDeploymentsCommand) Synopsis() string { + return "List deployments for a job" +} + +func (c *JobDeploymentsCommand) Run(args []string) int { + var latest, verbose bool + + flags := c.Meta.FlagSet("job deployments", FlagSetClient) + flags.Usage = func() { c.Ui.Output(c.Help()) } + flags.BoolVar(&latest, "latest", false, "") + flags.BoolVar(&verbose, "verbose", false, "") + + if err := flags.Parse(args); err != nil { + return 1 + } + + // Check that we got exactly one node + args = flags.Args() + if l := len(args); l != 1 { + c.Ui.Error(c.Help()) + return 1 + } + + // Get the HTTP client + client, err := c.Meta.Client() + if err != nil { + c.Ui.Error(fmt.Sprintf("Error initializing client: %s", err)) + return 1 + } + + jobID := args[0] + + // Check if the job exists + jobs, _, err := client.Jobs().PrefixList(jobID) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error listing jobs: %s", err)) + return 1 + } + if len(jobs) == 0 { + c.Ui.Error(fmt.Sprintf("No job(s) with prefix or id %q found", jobID)) + return 1 + } + if len(jobs) > 1 && strings.TrimSpace(jobID) != jobs[0].ID { + c.Ui.Output(fmt.Sprintf("Prefix matched multiple jobs\n\n%s", createStatusListOutput(jobs))) + return 0 + } + jobID = jobs[0].ID + + // Truncate the id unless full length is requested + length := shortId + if verbose { + length = fullId + } + + if latest { + deploy, _, err := client.Jobs().LatestDeployment(jobID, nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployments: %s", err)) + return 1 + } + + c.Ui.Output(c.Colorize().Color(formatDeployment(deploy, length))) + return 0 + } + + deploys, _, err := client.Jobs().Deployments(jobID, nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error retrieving deployments: %s", err)) + return 1 + } + + c.Ui.Output(formatDeployments(deploys, length)) + return 0 +} diff --git a/command/job_deployments_test.go b/command/job_deployments_test.go new file mode 100644 index 000000000000..f46f0826ebeb --- /dev/null +++ b/command/job_deployments_test.go @@ -0,0 +1,34 @@ +package command + +import ( + "strings" + "testing" + + "github.com/mitchellh/cli" +) + +func TestJobDeploymentsCommand_Implements(t *testing.T) { + var _ cli.Command = &JobDeploymentsCommand{} +} + +func TestJobDeploymentsCommand_Fails(t *testing.T) { + ui := new(cli.MockUi) + cmd := &JobDeploymentsCommand{Meta: Meta{Ui: ui}} + + // Fails on misuse + if code := cmd.Run([]string{"some", "bad", "args"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, cmd.Help()) { + t.Fatalf("expected help output, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "foo"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error listing jobs") { + t.Fatalf("expected failed query error, got: %s", out) + } + ui.ErrorWriter.Reset() +} diff --git a/commands.go b/commands.go index 13705a882b34..b24cf1bd48d4 100644 --- a/commands.go +++ b/commands.go @@ -129,6 +129,11 @@ func Commands(metaPtr *command.Meta) map[string]cli.CommandFactory { Meta: meta, }, nil }, + "job deployments": func() (cli.Command, error) { + return &command.JobDeploymentsCommand{ + Meta: meta, + }, nil + }, "job dispatch": func() (cli.Command, error) { return &command.JobDispatchCommand{ Meta: meta, diff --git a/main.go b/main.go index b5e216c60372..62d20dd99804 100644 --- a/main.go +++ b/main.go @@ -42,7 +42,7 @@ func RunCustom(args []string, commands map[string]cli.CommandFactory) int { "deployment resume", "deployment fail", "deployment promote": case "executor": case "fs ls", "fs cat", "fs stat": - case "job dispatch", "job history", "job revert": + case "job deployments", "job dispatch", "job history", "job revert": case "operator raft", "operator raft list-peers", "operator raft remove-peer": case "syslog": default: From 468c886282fa511d139da1fa78bf448885784537 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 30 Jun 2017 18:10:19 -0700 Subject: [PATCH 058/105] Formatting abilities --- command/alloc_status.go | 62 ++++++++------------------------ command/data_format.go | 25 +++++++++++++ command/deployment_list.go | 22 +++++++++++- command/deployment_status.go | 24 +++---------- command/eval_status.go | 59 ++++++++----------------------- command/inspect.go | 68 ++++++++++-------------------------- command/job_deployments.go | 12 +++++-- command/job_history.go | 39 +++++++++++++++++++-- command/job_revert.go | 1 - command/node_status.go | 52 +++++++-------------------- 10 files changed, 157 insertions(+), 207 deletions(-) diff --git a/command/alloc_status.go b/command/alloc_status.go index 01f9e790936e..c9d420c37073 100644 --- a/command/alloc_status.go +++ b/command/alloc_status.go @@ -85,41 +85,21 @@ func (c *AllocStatusCommand) Run(args []string) int { } // If args not specified but output format is specified, format and output the allocations data list - if len(args) == 0 { - var format string - if json && len(tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") + if len(args) == 0 && json || len(tmpl) > 0 { + allocs, _, err := client.Allocations().List(nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error querying allocations: %v", err)) return 1 - } else if json { - format = "json" - } else if len(tmpl) > 0 { - format = "template" } - if len(format) > 0 { - allocs, _, err := client.Allocations().List(nil) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error querying allocations: %v", err)) - return 1 - } - // Return nothing if no allocations found - if len(allocs) == 0 { - return 0 - } - - f, err := DataFormat(format, tmpl) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) - return 1 - } - out, err := f.TransformData(allocs) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } - c.Ui.Output(out) - return 0 + out, err := Format(json, tmpl, allocs) + if err != nil { + c.Ui.Error(err.Error()) + return 1 } + + c.Ui.Output(out) + return 0 } if len(args) != 1 { @@ -179,27 +159,13 @@ func (c *AllocStatusCommand) Run(args []string) int { } // If output format is specified, format and output the data - var format string - if json && len(tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") - return 1 - } else if json { - format = "json" - } else if len(tmpl) > 0 { - format = "template" - } - if len(format) > 0 { - f, err := DataFormat(format, tmpl) + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, alloc) if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + c.Ui.Error(err.Error()) return 1 } - out, err := f.TransformData(alloc) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } c.Ui.Output(out) return 0 } diff --git a/command/data_format.go b/command/data_format.go index e37097846f61..d1c218e610be 100644 --- a/command/data_format.go +++ b/command/data_format.go @@ -73,3 +73,28 @@ func (p *TemplateFormat) TransformData(data interface{}) (string, error) { } return fmt.Sprint(out), nil } + +func Format(json bool, template string, data interface{}) (string, error) { + var format string + if json && len(template) > 0 { + return "", fmt.Errorf("Both json and template formatting are not allowed") + } else if json { + format = "json" + } else if len(template) > 0 { + format = "template" + } else { + return "", fmt.Errorf("no formatting option given") + } + + f, err := DataFormat(format, template) + if err != nil { + return "", err + } + + out, err := f.TransformData(data) + if err != nil { + return "", fmt.Errorf("Error formatting the data: %s", err) + } + + return out, nil +} diff --git a/command/deployment_list.go b/command/deployment_list.go index 600a4e0dfabe..924ab5fd2e3e 100644 --- a/command/deployment_list.go +++ b/command/deployment_list.go @@ -23,6 +23,12 @@ General Options: List Options: + -json + Output the deployments in a JSON format. + + -t + Format and display the deployments using a Go template. + -verbose Display full information. ` @@ -34,11 +40,14 @@ func (c *DeploymentListCommand) Synopsis() string { } func (c *DeploymentListCommand) Run(args []string) int { - var verbose bool + var json, verbose bool + var tmpl string flags := c.Meta.FlagSet("deployment list", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } flags.BoolVar(&verbose, "verbose", false, "") + flags.BoolVar(&json, "json", false, "") + flags.StringVar(&tmpl, "t", "", "") if err := flags.Parse(args); err != nil { return 1 @@ -70,6 +79,17 @@ func (c *DeploymentListCommand) Run(args []string) int { return 1 } + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, deploys) + if err != nil { + c.Ui.Error(err.Error()) + return 1 + } + + c.Ui.Output(out) + return 0 + } + c.Ui.Output(formatDeployments(deploys, length)) return 0 } diff --git a/command/deployment_status.go b/command/deployment_status.go index 5a6b89a80a03..e75508a38c87 100644 --- a/command/deployment_status.go +++ b/command/deployment_status.go @@ -28,10 +28,10 @@ Status Options: Display full information. -json - Output the allocation in its JSON format. + Output the deployment in its JSON format. -t - Format and display allocation using a Go template. + Format and display deployment using a Go template. ` return strings.TrimSpace(helpText) } @@ -88,27 +88,13 @@ func (c *DeploymentStatusCommand) Run(args []string) int { return 0 } - var format string - if json && len(tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") - return 1 - } else if json { - format = "json" - } else if len(tmpl) > 0 { - format = "template" - } - if len(format) > 0 { - f, err := DataFormat(format, tmpl) + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, deploy) if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + c.Ui.Error(err.Error()) return 1 } - out, err := f.TransformData(deploy) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } c.Ui.Output(out) return 0 } diff --git a/command/eval_status.go b/command/eval_status.go index 67d854cd227e..f171070886ec 100644 --- a/command/eval_status.go +++ b/command/eval_status.go @@ -72,41 +72,21 @@ func (c *EvalStatusCommand) Run(args []string) int { } // If args not specified but output format is specified, format and output the evaluations data list - if len(args) == 0 { - var format string - if json && len(tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") + if len(args) == 0 && json || len(tmpl) > 0 { + evals, _, err := client.Evaluations().List(nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error querying evaluations: %v", err)) return 1 - } else if json { - format = "json" - } else if len(tmpl) > 0 { - format = "template" } - if len(format) > 0 { - evals, _, err := client.Evaluations().List(nil) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error querying evaluations: %v", err)) - return 1 - } - // Return nothing if no evaluations found - if len(evals) == 0 { - return 0 - } - - f, err := DataFormat(format, tmpl) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) - return 1 - } - out, err := f.TransformData(evals) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } - c.Ui.Output(out) - return 0 + out, err := Format(json, tmpl, evals) + if err != nil { + c.Ui.Error(err.Error()) + return 1 } + + c.Ui.Output(out) + return 0 } if len(args) != 1 { @@ -175,24 +155,13 @@ func (c *EvalStatusCommand) Run(args []string) int { } // If output format is specified, format and output the data - var format string - if json { - format = "json" - } else if len(tmpl) > 0 { - format = "template" - } - if len(format) > 0 { - f, err := DataFormat(format, tmpl) + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, eval) if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + c.Ui.Error(err.Error()) return 1 } - out, err := f.TransformData(eval) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } c.Ui.Output(out) return 0 } diff --git a/command/inspect.go b/command/inspect.go index d1e3c20cacc9..ac35d6a8c33d 100644 --- a/command/inspect.go +++ b/command/inspect.go @@ -24,10 +24,10 @@ General Options: Inspect Options: -json - Output the evaluation in its JSON format. + Output the job in its JSON format. -t - Format and display evaluation using a Go template. + Format and display job using a Go template. ` return strings.TrimSpace(helpText) } @@ -37,12 +37,12 @@ func (c *InspectCommand) Synopsis() string { } func (c *InspectCommand) Run(args []string) int { - var ojson bool + var json bool var tmpl string flags := c.Meta.FlagSet("inspect", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } - flags.BoolVar(&ojson, "json", false, "") + flags.BoolVar(&json, "json", false, "") flags.StringVar(&tmpl, "t", "", "") if err := flags.Parse(args); err != nil { @@ -58,40 +58,21 @@ func (c *InspectCommand) Run(args []string) int { } // If args not specified but output format is specified, format and output the jobs data list - if len(args) == 0 { - var format string - if ojson && len(tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") + if len(args) == 0 && json || len(tmpl) > 0 { + jobs, _, err := client.Jobs().List(nil) + if err != nil { + c.Ui.Error(fmt.Sprintf("Error querying jobs: %v", err)) return 1 - } else if ojson { - format = "json" - } else if len(tmpl) > 0 { - format = "template" } - if len(format) > 0 { - jobs, _, err := client.Jobs().List(nil) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error querying jobs: %v", err)) - return 1 - } - f, err := DataFormat(format, tmpl) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) - return 1 - } - // Return nothing if no jobs found - if len(jobs) == 0 { - return 0 - } - - out, err := f.TransformData(jobs) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } - c.Ui.Output(out) - return 0 + + out, err := Format(json, tmpl, jobs) + if err != nil { + c.Ui.Error(err.Error()) + return 1 } + + c.Ui.Output(out) + return 0 } // Check that we got exactly one job @@ -124,24 +105,13 @@ func (c *InspectCommand) Run(args []string) int { } // If output format is specified, format and output the data - var format string - if ojson { - format = "json" - } else if len(tmpl) > 0 { - format = "template" - } - if len(format) > 0 { - f, err := DataFormat(format, tmpl) + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, job) if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + c.Ui.Error(err.Error()) return 1 } - out, err := f.TransformData(job) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } c.Ui.Output(out) return 0 } diff --git a/command/job_deployments.go b/command/job_deployments.go index 8289674cf61b..3efbdad0ba5d 100644 --- a/command/job_deployments.go +++ b/command/job_deployments.go @@ -7,7 +7,6 @@ import ( type JobDeploymentsCommand struct { Meta - formatter DataFormatter } func (c *JobDeploymentsCommand) Help() string { @@ -22,6 +21,12 @@ General Options: Deployments Options: + -json + Output the deployments in a JSON format. + + -t + Format and display deployments using a Go template. + -latest Display the latest deployment only. @@ -36,12 +41,15 @@ func (c *JobDeploymentsCommand) Synopsis() string { } func (c *JobDeploymentsCommand) Run(args []string) int { - var latest, verbose bool + var json, latest, verbose bool + var tmpl string flags := c.Meta.FlagSet("job deployments", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } flags.BoolVar(&latest, "latest", false, "") flags.BoolVar(&verbose, "verbose", false, "") + flags.BoolVar(&json, "json", false, "") + flags.StringVar(&tmpl, "t", "", "") if err := flags.Parse(args); err != nil { return 1 diff --git a/command/job_history.go b/command/job_history.go index 4c9fb282ba9c..23dbceac8dfc 100644 --- a/command/job_history.go +++ b/command/job_history.go @@ -38,6 +38,12 @@ History Options: -job-version Display only the history for the given job version. + + -json + Output the job versions in a JSON format. + + -t + Format and display the job versions using a Go template. ` return strings.TrimSpace(helpText) } @@ -47,14 +53,16 @@ func (c *JobHistoryCommand) Synopsis() string { } func (c *JobHistoryCommand) Run(args []string) int { - var diff, full bool - var versionStr string + var json, diff, full bool + var tmpl, versionStr string flags := c.Meta.FlagSet("job history", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } flags.BoolVar(&diff, "p", false, "") flags.BoolVar(&full, "full", false, "") + flags.BoolVar(&json, "json", false, "") flags.StringVar(&versionStr, "job-version", "", "") + flags.StringVar(&tmpl, "t", "", "") if err := flags.Parse(args); err != nil { return 1 @@ -67,6 +75,11 @@ func (c *JobHistoryCommand) Run(args []string) int { return 1 } + if (json || len(tmpl) != 0) && (diff || full) { + c.Ui.Error("-json and -t are exclusive with -p and -full") + return 1 + } + // Get the HTTP client client, err := c.Meta.Client() if err != nil { @@ -127,12 +140,34 @@ func (c *JobHistoryCommand) Run(args []string) int { } } + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, job) + if err != nil { + c.Ui.Error(err.Error()) + return 1 + } + + c.Ui.Output(out) + return 0 + } + if err := c.formatJobVersion(job, diff, nextVersion, full); err != nil { c.Ui.Error(err.Error()) return 1 } } else { + if json || len(tmpl) > 0 { + out, err := Format(json, tmpl, versions) + if err != nil { + c.Ui.Error(err.Error()) + return 1 + } + + c.Ui.Output(out) + return 0 + } + if err := c.formatJobVersions(versions, diffs, full); err != nil { c.Ui.Error(err.Error()) return 1 diff --git a/command/job_revert.go b/command/job_revert.go index 141e33b9b9a4..a104b31899af 100644 --- a/command/job_revert.go +++ b/command/job_revert.go @@ -7,7 +7,6 @@ import ( type JobRevertCommand struct { Meta - formatter DataFormatter } func (c *JobRevertCommand) Help() string { diff --git a/command/node_status.go b/command/node_status.go index 22def1899dbe..d3cba2bbda04 100644 --- a/command/node_status.go +++ b/command/node_status.go @@ -121,16 +121,6 @@ func (c *NodeStatusCommand) Run(args []string) int { // Use list mode if no node name was provided if len(args) == 0 && !c.self { - // If output format is specified, format and output the node data list - var format string - if c.json && len(c.tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") - return 1 - } else if c.json { - format = "json" - } else if len(c.tmpl) > 0 { - format = "template" - } // Query the node info nodes, _, err := client.Nodes().List(nil) @@ -139,27 +129,23 @@ func (c *NodeStatusCommand) Run(args []string) int { return 1 } - // Return nothing if no nodes found - if len(nodes) == 0 { - return 0 - } - - if len(format) > 0 { - f, err := DataFormat(format, c.tmpl) + // If output format is specified, format and output the node data list + if c.json || len(c.tmpl) > 0 { + out, err := Format(c.json, c.tmpl, nodes) if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + c.Ui.Error(err.Error()) return 1 } - out, err := f.TransformData(nodes) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } c.Ui.Output(out) return 0 } + // Return nothing if no nodes found + if len(nodes) == 0 { + return 0 + } + // Format the nodes list out := make([]string, len(nodes)+1) if c.list_allocs { @@ -256,27 +242,13 @@ func (c *NodeStatusCommand) Run(args []string) int { } // If output format is specified, format and output the data - var format string - if c.json && len(c.tmpl) > 0 { - c.Ui.Error("Both -json and -t are not allowed") - return 1 - } else if c.json { - format = "json" - } else if len(c.tmpl) > 0 { - format = "template" - } - if len(format) > 0 { - f, err := DataFormat(format, c.tmpl) + if c.json || len(c.tmpl) > 0 { + out, err := Format(c.json, c.tmpl, node) if err != nil { - c.Ui.Error(fmt.Sprintf("Error getting formatter: %s", err)) + c.Ui.Error(err.Error()) return 1 } - out, err := f.TransformData(node) - if err != nil { - c.Ui.Error(fmt.Sprintf("Error formatting the data: %s", err)) - return 1 - } c.Ui.Output(out) return 0 } From 74563518b69b203e65d9804308fd749e993bb345 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 4 Jul 2017 13:08:20 -0700 Subject: [PATCH 059/105] Small fixes --- api/jobs_test.go | 4 ++-- nomad/deploymentwatcher/testutil_test.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/api/jobs_test.go b/api/jobs_test.go index 78f809bca715..d7af4023ba67 100644 --- a/api/jobs_test.go +++ b/api/jobs_test.go @@ -737,7 +737,7 @@ func TestJobs_Versions(t *testing.T) { jobs := c.Jobs() // Trying to retrieve a job by ID before it exists returns an error - _, _, err := jobs.Versions("job1", nil) + _, _, _, err := jobs.Versions("job1", false, nil) if err == nil || !strings.Contains(err.Error(), "not found") { t.Fatalf("expected not found error, got: %#v", err) } @@ -751,7 +751,7 @@ func TestJobs_Versions(t *testing.T) { assertWriteMeta(t, wm) // Query the job again and ensure it exists - result, qm, err := jobs.Versions("job1", nil) + result, _, qm, err := jobs.Versions("job1", false, nil) if err != nil { t.Fatalf("err: %s", err) } diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index 631b7de4eb8b..c3492d06e17b 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -318,7 +318,7 @@ func (m *mockBackend) GetDeployment(args *structs.DeploymentSpecificRequest, rep return rargs.Error(0) } -func (m *mockBackend) GetJobVersions(args *structs.JobSpecificRequest, reply *structs.JobVersionsResponse) error { +func (m *mockBackend) GetJobVersions(args *structs.JobVersionsRequest, reply *structs.JobVersionsResponse) error { rargs := m.Called(args, reply) return rargs.Error(0) } From 7e507719f84c3482123eab20fe38e42540d515ac Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Sun, 2 Jul 2017 14:01:33 -0700 Subject: [PATCH 060/105] Add deployment id to alloc --- api/allocations.go | 1 + 1 file changed, 1 insertion(+) diff --git a/api/allocations.go b/api/allocations.go index 9dbc9bf6e498..e340b8d8f50f 100644 --- a/api/allocations.go +++ b/api/allocations.go @@ -106,6 +106,7 @@ type Allocation struct { ClientStatus string ClientDescription string TaskStates map[string]*TaskState + DeploymentID string DeploymentStatus *AllocDeploymentStatus PreviousAllocation string CreateIndex uint64 From da82a6e81486933c2f842aaf17209fda543dfd49 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Sun, 2 Jul 2017 21:49:56 -0700 Subject: [PATCH 061/105] initial watcher --- api/tasks.go | 1 + client/alloc_runner.go | 46 +++++--- client/alloc_runner_health_watcher.go | 145 ++++++++++++++++++++++++++ client/client.go | 1 + client/structs/funcs.go | 81 ++++++++++++++ nomad/state/state_store.go | 6 ++ nomad/structs/structs.go | 8 +- 7 files changed, 270 insertions(+), 18 deletions(-) create mode 100644 client/alloc_runner_health_watcher.go create mode 100644 client/structs/funcs.go diff --git a/api/tasks.go b/api/tasks.go index bedbaa14972a..1659dd43bf28 100644 --- a/api/tasks.go +++ b/api/tasks.go @@ -475,6 +475,7 @@ func (t *Task) SetLogConfig(l *LogConfig) *Task { type TaskState struct { State string Failed bool + Restarts uint64 StartedAt time.Time FinishedAt time.Time Events []*TaskEvent diff --git a/client/alloc_runner.go b/client/alloc_runner.go index 1c75aeb8af19..fa73bcd02a7f 100644 --- a/client/alloc_runner.go +++ b/client/alloc_runner.go @@ -13,6 +13,7 @@ import ( "github.com/hashicorp/nomad/client/allocdir" "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/vaultclient" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" cstructs "github.com/hashicorp/nomad/client/structs" @@ -52,6 +53,8 @@ type AllocRunner struct { alloc *structs.Allocation allocClientStatus string // Explicit status of allocation. Set when there are failures allocClientDescription string + allocHealth *bool // Whether the allocation is healthy + allocBroadcast *cstructs.AllocBroadcaster allocLock sync.Mutex dirtyCh chan struct{} @@ -143,20 +146,21 @@ func NewAllocRunner(logger *log.Logger, config *config.Config, stateDB *bolt.DB, consulClient ConsulServiceAPI) *AllocRunner { ar := &AllocRunner{ - config: config, - stateDB: stateDB, - updater: updater, - logger: logger, - alloc: alloc, - dirtyCh: make(chan struct{}, 1), - tasks: make(map[string]*TaskRunner), - taskStates: copyTaskStates(alloc.TaskStates), - restored: make(map[string]struct{}), - updateCh: make(chan *structs.Allocation, 64), - destroyCh: make(chan struct{}), - waitCh: make(chan struct{}), - vaultClient: vaultClient, - consulClient: consulClient, + config: config, + stateDB: stateDB, + updater: updater, + logger: logger, + alloc: alloc, + allocBroadcast: cstructs.NewAllocBroadcaster(0), + dirtyCh: make(chan struct{}, 1), + tasks: make(map[string]*TaskRunner), + taskStates: copyTaskStates(alloc.TaskStates), + restored: make(map[string]struct{}), + updateCh: make(chan *structs.Allocation, 64), + destroyCh: make(chan struct{}), + waitCh: make(chan struct{}), + vaultClient: vaultClient, + consulClient: consulClient, } return ar } @@ -475,6 +479,14 @@ func (r *AllocRunner) Alloc() *structs.Allocation { r.allocLock.Unlock() return alloc } + + // The health has been set + if r.allocHealth != nil { + if alloc.DeploymentStatus == nil { + alloc.DeploymentStatus = &structs.AllocDeploymentStatus{} + } + alloc.DeploymentStatus.Healthy = helper.BoolToPtr(*r.allocHealth) + } r.allocLock.Unlock() // Scan the task states to determine the status of the alloc @@ -536,6 +548,7 @@ func (r *AllocRunner) syncStatus() error { // Get a copy of our alloc, update status server side and sync to disk alloc := r.Alloc() r.updater(alloc) + r.allocBroadcast.Send(alloc) return r.saveAllocRunnerState() } @@ -567,6 +580,9 @@ func (r *AllocRunner) setTaskState(taskName, state string, event *structs.TaskEv if event.FailsTask { taskState.Failed = true } + if event.Type == structs.TaskRestarting { + taskState.Restarts++ + } r.appendTaskEvent(taskState, event) } @@ -650,6 +666,7 @@ func (r *AllocRunner) appendTaskEvent(state *structs.TaskState, event *structs.T func (r *AllocRunner) Run() { defer close(r.waitCh) go r.dirtySyncState() + go r.watchHealth() // Find the task group to run in the allocation alloc := r.Alloc() @@ -913,6 +930,7 @@ func (r *AllocRunner) Destroy() { } r.destroy = true close(r.destroyCh) + r.allocBroadcast.Close() } // WaitCh returns a channel to wait for termination diff --git a/client/alloc_runner_health_watcher.go b/client/alloc_runner_health_watcher.go new file mode 100644 index 000000000000..39ede4386447 --- /dev/null +++ b/client/alloc_runner_health_watcher.go @@ -0,0 +1,145 @@ +package client + +import ( + "time" + + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/nomad/structs" +) + +// watchHealth is responsible for watching an allocation's task status and +// potentially consul health check status to determine if the allocation is +// healthy or unhealthy. +func (r *AllocRunner) watchHealth() { + // Get our alloc and the task group + alloc := r.Alloc() + tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) + if tg == nil { + r.logger.Printf("[ERR] client.alloc_watcher: failed to lookup allocation's task group. Exiting watcher") + return + } + + u := tg.Update + + // Checks marks whether we should be watching for Consul health checks + checks := false + r.logger.Printf("XXX %v", checks) + + switch { + case u == nil: + r.logger.Printf("[TRACE] client.alloc_watcher: no update block for alloc %q. exiting", alloc.ID) + return + case u.HealthCheck == structs.UpdateStrategyHealthCheck_Manual: + r.logger.Printf("[TRACE] client.alloc_watcher: update block has manual checks for alloc %q. exiting", alloc.ID) + return + case u.HealthCheck == structs.UpdateStrategyHealthCheck_Checks: + checks = true + } + + // Get a listener so we know when an allocation is updated. + l := r.allocBroadcast.Listen() + + // Create a deadline timer for the health + deadline := time.NewTimer(u.HealthyDeadline) + + // Create a healthy timer + latestHealthyTime := time.Unix(0, 0) + healthyTimer := time.NewTimer(0) + if !healthyTimer.Stop() { + <-healthyTimer.C + } + + // Cleanup function + defer func() { + if !deadline.Stop() { + <-deadline.C + } + if !healthyTimer.Stop() { + <-healthyTimer.C + } + l.Close() + }() + + setHealth := func(h bool) { + r.allocLock.Lock() + r.allocHealth = helper.BoolToPtr(h) + r.allocLock.Unlock() + r.syncStatus() + } + + first := true +OUTER: + for { + if !first { + select { + case <-r.destroyCh: + return + case newAlloc, ok := <-l.Ch: + if !ok { + return + } + + alloc = newAlloc + if alloc.DeploymentID == "" { + continue OUTER + } + + r.logger.Printf("[TRACE] client.alloc_watcher: new alloc version for %q", alloc.ID) + case <-deadline.C: + // We have exceeded our deadline without being healthy. + setHealth(false) + return + case <-healthyTimer.C: + r.logger.Printf("[TRACE] client.alloc_watcher: alloc %q is healthy", alloc.ID) + setHealth(true) + } + } + + first = false + + // If the alloc is being stopped by the server just exit + switch alloc.DesiredStatus { + case structs.AllocDesiredStatusStop, structs.AllocDesiredStatusEvict: + r.logger.Printf("[TRACE] client.alloc_watcher: desired status terminal for alloc %q", alloc.ID) + return + } + + // If the task is dead or has restarted, fail + for _, tstate := range alloc.TaskStates { + if tstate.Failed || !tstate.FinishedAt.IsZero() || tstate.Restarts != 0 { + r.logger.Printf("[TRACE] client.alloc_watcher: setting health to false for alloc %q", alloc.ID) + setHealth(false) + return + } + } + + // Determine if the allocation is healthy + for task, tstate := range alloc.TaskStates { + if tstate.State != structs.TaskStateRunning { + r.logger.Printf("[TRACE] client.alloc_watcher: continuing since task %q hasn't started for alloc %q", task, alloc.ID) + continue OUTER + } + + if tstate.StartedAt.After(latestHealthyTime) { + latestHealthyTime = tstate.StartedAt + } + } + + // If we are already healthy we don't set the timer + healthyThreshold := latestHealthyTime.Add(u.MinHealthyTime) + if time.Now().After(healthyThreshold) { + continue OUTER + } + + // Start the time til we are healthy + if !healthyTimer.Stop() { + select { + case <-healthyTimer.C: + default: + } + } + d := time.Until(healthyThreshold) + healthyTimer.Reset(d) + r.logger.Printf("[TRACE] client.alloc_watcher: setting healthy timer to %v for alloc %q", d, alloc.ID) + } +} diff --git a/client/client.go b/client/client.go index 486dea9ff6fb..1e81fe79d1f8 100644 --- a/client/client.go +++ b/client/client.go @@ -1295,6 +1295,7 @@ func (c *Client) updateAllocStatus(alloc *structs.Allocation) { stripped.TaskStates = alloc.TaskStates stripped.ClientStatus = alloc.ClientStatus stripped.ClientDescription = alloc.ClientDescription + stripped.DeploymentStatus = alloc.DeploymentStatus select { case c.allocUpdates <- stripped: diff --git a/client/structs/funcs.go b/client/structs/funcs.go new file mode 100644 index 000000000000..a82a5a5199f9 --- /dev/null +++ b/client/structs/funcs.go @@ -0,0 +1,81 @@ +package structs + +import ( + "sync" + + "github.com/hashicorp/nomad/nomad/structs" +) + +// AllocBroadcaster implements an allocation broadcast channel. +// The zero value is a usable unbuffered channel. +type AllocBroadcaster struct { + m sync.Mutex + listeners map[int]chan<- *structs.Allocation // lazy init + nextId int + capacity int + closed bool +} + +// NewAllocBroadcaster returns a new AllocBroadcaster with the given capacity (0 means unbuffered). +func NewAllocBroadcaster(n int) *AllocBroadcaster { + return &AllocBroadcaster{capacity: n} +} + +// AllocListener implements a listening endpoint for an allocation broadcast channel. +type AllocListener struct { + // Ch receives the broadcast messages. + Ch <-chan *structs.Allocation + b *AllocBroadcaster + id int +} + +// Send broadcasts a message to the channel. +// Sending on a closed channel causes a runtime panic. +func (b *AllocBroadcaster) Send(v *structs.Allocation) { + b.m.Lock() + defer b.m.Unlock() + if b.closed { + return + } + for _, l := range b.listeners { + select { + case l <- v: + default: + } + } +} + +// Close closes the channel, disabling the sending of further messages. +func (b *AllocBroadcaster) Close() { + b.m.Lock() + defer b.m.Unlock() + b.closed = true + for _, l := range b.listeners { + close(l) + } +} + +// Listen returns a Listener for the broadcast channel. +func (b *AllocBroadcaster) Listen() *AllocListener { + b.m.Lock() + defer b.m.Unlock() + if b.listeners == nil { + b.listeners = make(map[int]chan<- *structs.Allocation) + } + for b.listeners[b.nextId] != nil { + b.nextId++ + } + ch := make(chan *structs.Allocation, b.capacity) + if b.closed { + close(ch) + } + b.listeners[b.nextId] = ch + return &AllocListener{ch, b, b.nextId} +} + +// Close closes the Listener, disabling the receival of further messages. +func (l *AllocListener) Close() { + l.b.m.Lock() + defer l.b.m.Unlock() + delete(l.b.listeners, l.id) +} diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index fb6aeb5e8296..ac1a79ea2f44 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -1434,10 +1434,16 @@ func (s *StateStore) nestedUpdateAllocFromClient(txn *memdb.Txn, index uint64, a copyAlloc.ClientStatus = alloc.ClientStatus copyAlloc.ClientDescription = alloc.ClientDescription copyAlloc.TaskStates = alloc.TaskStates + copyAlloc.DeploymentStatus = alloc.DeploymentStatus // Update the modify index copyAlloc.ModifyIndex = index + // TODO TEST + if err := s.updateDeploymentWithAlloc(index, copyAlloc, exist, txn); err != nil { + return fmt.Errorf("error updating deployment: %v", err) + } + if err := s.updateSummaryWithAlloc(index, copyAlloc, exist, txn); err != nil { return fmt.Errorf("error updating job summary: %v", err) } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index eb1784ad455f..5a9f7bcbf097 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3262,6 +3262,9 @@ type TaskState struct { // Failed marks a task as having failed Failed bool + // Restarts is the number of times the task has restarted + Restarts uint64 + // StartedAt is the time the task is started. It is updated each time the // task starts StartedAt time.Time @@ -3279,10 +3282,7 @@ func (ts *TaskState) Copy() *TaskState { return nil } copy := new(TaskState) - copy.State = ts.State - copy.Failed = ts.Failed - copy.StartedAt = ts.StartedAt - copy.FinishedAt = ts.FinishedAt + *copy = *ts if ts.Events != nil { copy.Events = make([]*TaskEvent, len(ts.Events)) From 8e58ddcceb9c35de8908f266acdc007f40573d3a Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Sun, 2 Jul 2017 22:13:28 -0700 Subject: [PATCH 062/105] Update index --- client/alloc_runner_health_watcher.go | 1 + nomad/state/state_store.go | 8 +++++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/client/alloc_runner_health_watcher.go b/client/alloc_runner_health_watcher.go index 39ede4386447..8870c184fe85 100644 --- a/client/alloc_runner_health_watcher.go +++ b/client/alloc_runner_health_watcher.go @@ -88,6 +88,7 @@ OUTER: case <-deadline.C: // We have exceeded our deadline without being healthy. setHealth(false) + // XXX Think about in-place return case <-healthyTimer.C: r.logger.Printf("[TRACE] client.alloc_watcher: alloc %q is healthy", alloc.ID) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index ac1a79ea2f44..ed3207ccabad 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2527,7 +2527,8 @@ func (s *StateStore) updateSummaryWithJob(index uint64, job *structs.Job, } // updateDeploymentWithAlloc is used to update the deployment state associated -// with the given allocation +// with the given allocation. The passed alloc may be updated if the deployment +// status has changed to capture the modify index at which it has changed. func (s *StateStore) updateDeploymentWithAlloc(index uint64, alloc, existing *structs.Allocation, txn *memdb.Txn) error { // Nothing to do if the allocation is not associated with a deployment if alloc.DeploymentID == "" { @@ -2582,6 +2583,11 @@ func (s *StateStore) updateDeploymentWithAlloc(index uint64, alloc, existing *st return nil } + // Update the allocation's deployment status modify index + if alloc.DeploymentStatus != nil && healthy+unhealthy != 0 { + alloc.DeploymentStatus.ModifyIndex = index + } + // Create a copy of the deployment object deploymentCopy := deployment.Copy() deploymentCopy.ModifyIndex = index From d165f65013fecc7a41bf0ba387008b1e5e54a1b0 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Mon, 3 Jul 2017 15:03:42 -0700 Subject: [PATCH 063/105] watcher per alloc --- client/alloc_runner.go | 38 +++++++++++++++------------ client/alloc_runner_health_watcher.go | 20 +++++++------- 2 files changed, 31 insertions(+), 27 deletions(-) diff --git a/client/alloc_runner.go b/client/alloc_runner.go index fa73bcd02a7f..824577ddf23f 100644 --- a/client/alloc_runner.go +++ b/client/alloc_runner.go @@ -1,6 +1,7 @@ package client import ( + "context" "fmt" "log" "os" @@ -76,10 +77,9 @@ type AllocRunner struct { otherAllocDir *allocdir.AllocDir - destroy bool - destroyCh chan struct{} - destroyLock sync.Mutex - waitCh chan struct{} + ctx context.Context + exitFn context.CancelFunc + waitCh chan struct{} // State related fields // stateDB is used to store the alloc runners state @@ -157,11 +157,13 @@ func NewAllocRunner(logger *log.Logger, config *config.Config, stateDB *bolt.DB, taskStates: copyTaskStates(alloc.TaskStates), restored: make(map[string]struct{}), updateCh: make(chan *structs.Allocation, 64), - destroyCh: make(chan struct{}), waitCh: make(chan struct{}), vaultClient: vaultClient, consulClient: consulClient, } + + // TODO Should be passed a context + ar.ctx, ar.exitFn = context.WithCancel(context.TODO()) return ar } @@ -537,7 +539,7 @@ func (r *AllocRunner) dirtySyncState() { select { case <-r.dirtyCh: r.syncStatus() - case <-r.destroyCh: + case <-r.ctx.Done(): return } } @@ -666,7 +668,10 @@ func (r *AllocRunner) appendTaskEvent(state *structs.TaskState, event *structs.T func (r *AllocRunner) Run() { defer close(r.waitCh) go r.dirtySyncState() - go r.watchHealth() + + // Start the watcher + wCtx, watcherCancel := context.WithCancel(r.ctx) + go r.watchHealth(wCtx) // Find the task group to run in the allocation alloc := r.Alloc() @@ -743,6 +748,7 @@ OUTER: r.allocLock.Lock() r.alloc = update r.allocLock.Unlock() + r.logger.Printf("[ALEX---------------] client: alloc update") // Check if we're in a terminal status if update.TerminalStatus() { @@ -750,6 +756,11 @@ OUTER: break OUTER } + // Create a new watcher + watcherCancel() + wCtx, watcherCancel = context.WithCancel(r.ctx) + go r.watchHealth(wCtx) + // Update the task groups runners := r.getTaskRunners() for _, tr := range runners { @@ -759,7 +770,7 @@ OUTER: if err := r.syncStatus(); err != nil { r.logger.Printf("[WARN] client: failed to sync status upon receiving alloc update: %v", err) } - case <-r.destroyCh: + case <-r.ctx.Done(): taskDestroyEvent = structs.NewTaskEvent(structs.TaskKilled) break OUTER } @@ -815,7 +826,7 @@ func (r *AllocRunner) handleDestroy() { for { select { - case <-r.destroyCh: + case <-r.ctx.Done(): if err := r.DestroyContext(); err != nil { r.logger.Printf("[ERR] client: failed to destroy context for alloc '%s': %v", r.alloc.ID, err) @@ -922,14 +933,7 @@ func (r *AllocRunner) shouldUpdate(serverIndex uint64) bool { // Destroy is used to indicate that the allocation context should be destroyed func (r *AllocRunner) Destroy() { - r.destroyLock.Lock() - defer r.destroyLock.Unlock() - - if r.destroy { - return - } - r.destroy = true - close(r.destroyCh) + r.exitFn() r.allocBroadcast.Close() } diff --git a/client/alloc_runner_health_watcher.go b/client/alloc_runner_health_watcher.go index 8870c184fe85..4e33316314c6 100644 --- a/client/alloc_runner_health_watcher.go +++ b/client/alloc_runner_health_watcher.go @@ -1,6 +1,7 @@ package client import ( + "context" "time" "github.com/hashicorp/nomad/helper" @@ -10,15 +11,21 @@ import ( // watchHealth is responsible for watching an allocation's task status and // potentially consul health check status to determine if the allocation is // healthy or unhealthy. -func (r *AllocRunner) watchHealth() { +func (r *AllocRunner) watchHealth(ctx context.Context) { // Get our alloc and the task group alloc := r.Alloc() + + // See if we should watch the allocs health + if alloc.DeploymentID == "" { + r.logger.Printf("[TRACE] client.alloc_watcher: exiting because alloc isn't part of a deployment") + return + } + tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) if tg == nil { r.logger.Printf("[ERR] client.alloc_watcher: failed to lookup allocation's task group. Exiting watcher") return } - u := tg.Update // Checks marks whether we should be watching for Consul health checks @@ -72,7 +79,7 @@ OUTER: for { if !first { select { - case <-r.destroyCh: + case <-ctx.Done(): return case newAlloc, ok := <-l.Ch: if !ok { @@ -80,22 +87,15 @@ OUTER: } alloc = newAlloc - if alloc.DeploymentID == "" { - continue OUTER - } - r.logger.Printf("[TRACE] client.alloc_watcher: new alloc version for %q", alloc.ID) case <-deadline.C: // We have exceeded our deadline without being healthy. setHealth(false) - // XXX Think about in-place - return case <-healthyTimer.C: r.logger.Printf("[TRACE] client.alloc_watcher: alloc %q is healthy", alloc.ID) setHealth(true) } } - first = false // If the alloc is being stopped by the server just exit From f72bbaa370915b20a760497847c88781c2f7e3c1 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 4 Jul 2017 12:24:27 -0700 Subject: [PATCH 064/105] Client watches for allocation health using task state and Consul checks This PR adds watching of allocation health at the client. The client can watch for health based on the tasks running on time and also based on the consul checks passing. --- client/alloc_runner.go | 17 +- client/alloc_runner_health_watcher.go | 87 +++++++-- client/alloc_runner_test.go | 258 +++++++++++++++++++++++++- client/consul.go | 2 + client/consul_test.go | 19 +- client/driver/mock_driver.go | 7 + client/task_runner_test.go | 2 +- command/agent/consul/client.go | 35 ++++ command/agent/consul/int_test.go | 10 + scheduler/generic_sched.go | 3 +- scheduler/system_sched.go | 3 +- 11 files changed, 415 insertions(+), 28 deletions(-) diff --git a/client/alloc_runner.go b/client/alloc_runner.go index 824577ddf23f..efd122627ba4 100644 --- a/client/alloc_runner.go +++ b/client/alloc_runner.go @@ -746,9 +746,19 @@ OUTER: case update := <-r.updateCh: // Store the updated allocation. r.allocLock.Lock() + + // If the deployment ids have changed clear the health + if r.alloc.DeploymentID != update.DeploymentID { + r.allocHealth = nil + } + r.alloc = update r.allocLock.Unlock() - r.logger.Printf("[ALEX---------------] client: alloc update") + + // Create a new watcher + watcherCancel() + wCtx, watcherCancel = context.WithCancel(r.ctx) + go r.watchHealth(wCtx) // Check if we're in a terminal status if update.TerminalStatus() { @@ -756,11 +766,6 @@ OUTER: break OUTER } - // Create a new watcher - watcherCancel() - wCtx, watcherCancel = context.WithCancel(r.ctx) - go r.watchHealth(wCtx) - // Update the task groups runners := r.getTaskRunners() for _, tr := range runners { diff --git a/client/alloc_runner_health_watcher.go b/client/alloc_runner_health_watcher.go index 4e33316314c6..332834c931a9 100644 --- a/client/alloc_runner_health_watcher.go +++ b/client/alloc_runner_health_watcher.go @@ -4,18 +4,23 @@ import ( "context" "time" + "github.com/hashicorp/consul/api" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" ) +const ( + // consulCheckLookupInterval is the interval at which we check if the + // Consul checks are healthy or unhealthy. + consulCheckLookupInterval = 500 * time.Millisecond +) + // watchHealth is responsible for watching an allocation's task status and // potentially consul health check status to determine if the allocation is // healthy or unhealthy. func (r *AllocRunner) watchHealth(ctx context.Context) { - // Get our alloc and the task group - alloc := r.Alloc() - // See if we should watch the allocs health + alloc := r.Alloc() if alloc.DeploymentID == "" { r.logger.Printf("[TRACE] client.alloc_watcher: exiting because alloc isn't part of a deployment") return @@ -26,12 +31,13 @@ func (r *AllocRunner) watchHealth(ctx context.Context) { r.logger.Printf("[ERR] client.alloc_watcher: failed to lookup allocation's task group. Exiting watcher") return } - u := tg.Update // Checks marks whether we should be watching for Consul health checks - checks := false - r.logger.Printf("XXX %v", checks) + desiredChecks := 0 + var checkTicker *time.Ticker + var checkCh <-chan time.Time + u := tg.Update switch { case u == nil: r.logger.Printf("[TRACE] client.alloc_watcher: no update block for alloc %q. exiting", alloc.ID) @@ -40,7 +46,14 @@ func (r *AllocRunner) watchHealth(ctx context.Context) { r.logger.Printf("[TRACE] client.alloc_watcher: update block has manual checks for alloc %q. exiting", alloc.ID) return case u.HealthCheck == structs.UpdateStrategyHealthCheck_Checks: - checks = true + for _, task := range tg.Tasks { + for _, s := range task.Services { + desiredChecks += len(s.Checks) + } + } + + checkTicker = time.NewTicker(consulCheckLookupInterval) + checkCh = checkTicker.C } // Get a listener so we know when an allocation is updated. @@ -50,7 +63,8 @@ func (r *AllocRunner) watchHealth(ctx context.Context) { deadline := time.NewTimer(u.HealthyDeadline) // Create a healthy timer - latestHealthyTime := time.Unix(0, 0) + latestTaskHealthy := time.Unix(0, 0) + latestChecksHealthy := time.Unix(0, 0) healthyTimer := time.NewTimer(0) if !healthyTimer.Stop() { <-healthyTimer.C @@ -64,6 +78,9 @@ func (r *AllocRunner) watchHealth(ctx context.Context) { if !healthyTimer.Stop() { <-healthyTimer.C } + if checkTicker != nil { + checkTicker.Stop() + } l.Close() }() @@ -74,6 +91,7 @@ func (r *AllocRunner) watchHealth(ctx context.Context) { r.syncStatus() } + var checks []*api.AgentCheck first := true OUTER: for { @@ -88,12 +106,22 @@ OUTER: alloc = newAlloc r.logger.Printf("[TRACE] client.alloc_watcher: new alloc version for %q", alloc.ID) + case <-checkCh: + newChecks, err := r.consulClient.Checks(alloc) + if err != nil { + r.logger.Printf("[TRACE] client.alloc_watcher: failed to lookup consul checks for allocation %q: %v", alloc.ID, err) + } + + checks = newChecks case <-deadline.C: // We have exceeded our deadline without being healthy. + r.logger.Printf("[TRACE] client.alloc_watcher: alloc %q hit healthy deadline", alloc.ID) setHealth(false) + return case <-healthyTimer.C: r.logger.Printf("[TRACE] client.alloc_watcher: alloc %q is healthy", alloc.ID) setHealth(true) + return } } first = false @@ -105,6 +133,11 @@ OUTER: return } + if len(alloc.TaskStates) != len(tg.Tasks) { + r.logger.Printf("[TRACE] client.alloc_watcher: all task runners haven't started") + continue OUTER + } + // If the task is dead or has restarted, fail for _, tstate := range alloc.TaskStates { if tstate.Failed || !tstate.FinishedAt.IsZero() || tstate.Restarts != 0 { @@ -114,6 +147,24 @@ OUTER: } } + // If we should have checks and they aren't all healthy continue + if len(checks) != desiredChecks { + r.logger.Printf("[TRACE] client.alloc_watcher: continuing since all checks (want %d; got %d) haven't been registered for alloc %q", desiredChecks, len(checks), alloc.ID) + continue OUTER + } + + // Check if all the checks are passing + for _, check := range checks { + if check.Status != api.HealthPassing { + r.logger.Printf("[TRACE] client.alloc_watcher: continuing since check %q isn't passing for alloc %q", check.CheckID, alloc.ID) + latestChecksHealthy = time.Time{} + continue OUTER + } + } + if latestChecksHealthy.IsZero() { + latestChecksHealthy = time.Now() + } + // Determine if the allocation is healthy for task, tstate := range alloc.TaskStates { if tstate.State != structs.TaskStateRunning { @@ -121,25 +172,31 @@ OUTER: continue OUTER } - if tstate.StartedAt.After(latestHealthyTime) { - latestHealthyTime = tstate.StartedAt + if tstate.StartedAt.After(latestTaskHealthy) { + latestTaskHealthy = tstate.StartedAt } } - // If we are already healthy we don't set the timer - healthyThreshold := latestHealthyTime.Add(u.MinHealthyTime) - if time.Now().After(healthyThreshold) { + // Don't need to set the timer if we are healthy and have marked + // ourselves healthy. + if alloc.DeploymentStatus != nil && alloc.DeploymentStatus.Healthy != nil && *alloc.DeploymentStatus.Healthy { continue OUTER } - // Start the time til we are healthy + // Determine when we can mark ourselves as healthy. + totalHealthy := latestTaskHealthy + if totalHealthy.Before(latestChecksHealthy) { + totalHealthy = latestChecksHealthy + } + d := time.Until(totalHealthy.Add(u.MinHealthyTime)) + if !healthyTimer.Stop() { select { case <-healthyTimer.C: default: } } - d := time.Until(healthyThreshold) + healthyTimer.Reset(d) r.logger.Printf("[TRACE] client.alloc_watcher: setting healthy timer to %v for alloc %q", d, alloc.ID) } diff --git a/client/alloc_runner_test.go b/client/alloc_runner_test.go index d01eaa2b4415..cfdbd1c8fbb4 100644 --- a/client/alloc_runner_test.go +++ b/client/alloc_runner_test.go @@ -11,6 +11,7 @@ import ( "time" "github.com/boltdb/bolt" + "github.com/hashicorp/consul/api" "github.com/hashicorp/go-multierror" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" @@ -73,6 +74,260 @@ func TestAllocRunner_SimpleRun(t *testing.T) { }) } +// Test that the watcher will mark the allocation as unhealthy. +func TestAllocRunner_DeploymentHealth_Unhealthy_BadStart(t *testing.T) { + ctestutil.ExecCompatible(t) + + // Ensure the task fails and restarts + upd, ar := testAllocRunner(false) + + // Make the task fail + task := ar.alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config["start_error"] = "test error" + + // Make the alloc be part of a deployment + ar.alloc.DeploymentID = structs.GenerateUUID() + ar.alloc.Job.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + ar.alloc.Job.TaskGroups[0].Update.HealthCheck = structs.UpdateStrategyHealthCheck_TaskStates + ar.alloc.Job.TaskGroups[0].Update.MaxParallel = 1 + + go ar.Run() + defer ar.Destroy() + + testutil.WaitForResult(func() (bool, error) { + if upd.Count == 0 { + return false, fmt.Errorf("No updates") + } + last := upd.Allocs[upd.Count-1] + if last.DeploymentStatus == nil || last.DeploymentStatus.Healthy == nil { + return false, fmt.Errorf("want deployment status unhealthy; got unset") + } else if *last.DeploymentStatus.Healthy { + return false, fmt.Errorf("want deployment status unhealthy; got healthy") + } + return true, nil + }, func(err error) { + t.Fatalf("err: %v", err) + }) +} + +// Test that the watcher will mark the allocation as unhealthy if it hits its +// deadline. +func TestAllocRunner_DeploymentHealth_Unhealthy_Deadline(t *testing.T) { + ctestutil.ExecCompatible(t) + + // Ensure the task fails and restarts + upd, ar := testAllocRunner(false) + + // Make the task block + task := ar.alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config["start_block_for"] = "2s" + task.Config["run_for"] = "10s" + + // Make the alloc be part of a deployment + ar.alloc.DeploymentID = structs.GenerateUUID() + ar.alloc.Job.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + ar.alloc.Job.TaskGroups[0].Update.HealthCheck = structs.UpdateStrategyHealthCheck_TaskStates + ar.alloc.Job.TaskGroups[0].Update.MaxParallel = 1 + ar.alloc.Job.TaskGroups[0].Update.HealthyDeadline = 100 * time.Millisecond + + go ar.Run() + defer ar.Destroy() + + testutil.WaitForResult(func() (bool, error) { + if upd.Count == 0 { + return false, fmt.Errorf("No updates") + } + last := upd.Allocs[upd.Count-1] + if last.DeploymentStatus == nil || last.DeploymentStatus.Healthy == nil { + return false, fmt.Errorf("want deployment status unhealthy; got unset") + } else if *last.DeploymentStatus.Healthy { + return false, fmt.Errorf("want deployment status unhealthy; got healthy") + } + return true, nil + }, func(err error) { + t.Fatalf("err: %v", err) + }) +} + +// Test that the watcher will mark the allocation as healthy. +func TestAllocRunner_DeploymentHealth_Healthy_NoChecks(t *testing.T) { + ctestutil.ExecCompatible(t) + + // Ensure the task fails and restarts + upd, ar := testAllocRunner(false) + + // Make the task run healthy + task := ar.alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config["run_for"] = "10s" + + // Create a task that takes longer to become healthy + ar.alloc.Job.TaskGroups[0].Tasks = append(ar.alloc.Job.TaskGroups[0].Tasks, task.Copy()) + task2 := ar.alloc.Job.TaskGroups[0].Tasks[1] + task2.Name = "task 2" + task2.Config["start_block_for"] = "500ms" + + // Make the alloc be part of a deployment + ar.alloc.DeploymentID = structs.GenerateUUID() + ar.alloc.Job.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + ar.alloc.Job.TaskGroups[0].Update.HealthCheck = structs.UpdateStrategyHealthCheck_TaskStates + ar.alloc.Job.TaskGroups[0].Update.MaxParallel = 1 + ar.alloc.Job.TaskGroups[0].Update.MinHealthyTime = 100 * time.Millisecond + + start := time.Now() + go ar.Run() + defer ar.Destroy() + + testutil.WaitForResult(func() (bool, error) { + if upd.Count == 0 { + return false, fmt.Errorf("No updates") + } + last := upd.Allocs[upd.Count-1] + if last.DeploymentStatus == nil || last.DeploymentStatus.Healthy == nil { + return false, fmt.Errorf("want deployment status unhealthy; got unset") + } else if !*last.DeploymentStatus.Healthy { + return false, fmt.Errorf("want deployment status healthy; got unhealthy") + } + return true, nil + }, func(err error) { + t.Fatalf("err: %v", err) + }) + if d := time.Now().Sub(start); d < 500*time.Millisecond { + t.Fatalf("didn't wait for second task group. Only took %v", d) + } +} + +// Test that the watcher will mark the allocation as healthy with checks +func TestAllocRunner_DeploymentHealth_Healthy_Checks(t *testing.T) { + ctestutil.ExecCompatible(t) + + // Ensure the task fails and restarts + upd, ar := testAllocRunner(false) + + // Make the task fail + task := ar.alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config["run_for"] = "10s" + + // Create a task that has no checks + ar.alloc.Job.TaskGroups[0].Tasks = append(ar.alloc.Job.TaskGroups[0].Tasks, task.Copy()) + task2 := ar.alloc.Job.TaskGroups[0].Tasks[1] + task2.Name = "task 2" + task2.Services = nil + + // Make the alloc be part of a deployment + ar.alloc.DeploymentID = structs.GenerateUUID() + ar.alloc.Job.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + ar.alloc.Job.TaskGroups[0].Update.HealthCheck = structs.UpdateStrategyHealthCheck_Checks + ar.alloc.Job.TaskGroups[0].Update.MaxParallel = 1 + ar.alloc.Job.TaskGroups[0].Update.MinHealthyTime = 100 * time.Millisecond + + checkHealthy := &api.AgentCheck{ + CheckID: structs.GenerateUUID(), + Status: api.HealthPassing, + } + checkUnhealthy := &api.AgentCheck{ + CheckID: checkHealthy.CheckID, + Status: api.HealthWarning, + } + + // Only return the check as healthy after a duration + trigger := time.After(500 * time.Millisecond) + ar.consulClient.(*mockConsulServiceClient).checksFn = func(a *structs.Allocation) ([]*api.AgentCheck, error) { + select { + case <-trigger: + return []*api.AgentCheck{checkHealthy}, nil + default: + return []*api.AgentCheck{checkUnhealthy}, nil + } + } + + start := time.Now() + go ar.Run() + defer ar.Destroy() + + testutil.WaitForResult(func() (bool, error) { + if upd.Count == 0 { + return false, fmt.Errorf("No updates") + } + last := upd.Allocs[upd.Count-1] + if last.DeploymentStatus == nil || last.DeploymentStatus.Healthy == nil { + return false, fmt.Errorf("want deployment status unhealthy; got unset") + } else if !*last.DeploymentStatus.Healthy { + return false, fmt.Errorf("want deployment status healthy; got unhealthy") + } + return true, nil + }, func(err error) { + t.Fatalf("err: %v", err) + }) + + if d := time.Now().Sub(start); d < 500*time.Millisecond { + t.Fatalf("didn't wait for second task group. Only took %v", d) + } +} + +// Test that the watcher will mark the allocation as healthy. +func TestAllocRunner_DeploymentHealth_Healthy_UpdatedDeployment(t *testing.T) { + ctestutil.ExecCompatible(t) + + // Ensure the task fails and restarts + upd, ar := testAllocRunner(false) + + // Make the task run healthy + task := ar.alloc.Job.TaskGroups[0].Tasks[0] + task.Driver = "mock_driver" + task.Config["run_for"] = "30s" + + // Make the alloc be part of a deployment + ar.alloc.DeploymentID = structs.GenerateUUID() + ar.alloc.Job.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + ar.alloc.Job.TaskGroups[0].Update.HealthCheck = structs.UpdateStrategyHealthCheck_TaskStates + ar.alloc.Job.TaskGroups[0].Update.MaxParallel = 1 + ar.alloc.Job.TaskGroups[0].Update.MinHealthyTime = 100 * time.Millisecond + + go ar.Run() + defer ar.Destroy() + + testutil.WaitForResult(func() (bool, error) { + if upd.Count == 0 { + return false, fmt.Errorf("No updates") + } + last := upd.Allocs[upd.Count-1] + if last.DeploymentStatus == nil || last.DeploymentStatus.Healthy == nil { + return false, fmt.Errorf("want deployment status unhealthy; got unset") + } else if !*last.DeploymentStatus.Healthy { + return false, fmt.Errorf("want deployment status healthy; got unhealthy") + } + return true, nil + }, func(err error) { + t.Fatalf("err: %v", err) + }) + + // Mimick an update to a new deployment id + oldCount := upd.Count + last := upd.Allocs[oldCount-1].Copy() + last.DeploymentStatus = nil + last.DeploymentID = structs.GenerateUUID() + ar.Update(last) + + testutil.WaitForResult(func() (bool, error) { + if upd.Count <= oldCount { + return false, fmt.Errorf("No new updates") + } + last := upd.Allocs[upd.Count-1] + if last.DeploymentStatus == nil || last.DeploymentStatus.Healthy == nil { + return false, fmt.Errorf("want deployment status unhealthy; got unset") + } else if !*last.DeploymentStatus.Healthy { + return false, fmt.Errorf("want deployment status healthy; got unhealthy") + } + return true, nil + }, func(err error) { + t.Fatalf("err: %v", err) + }) +} + // TestAllocRuner_RetryArtifact ensures that if one task in a task group is // retrying fetching an artifact, other tasks in the group should be able // to proceed. @@ -427,9 +682,6 @@ func TestAllocRunner_SaveRestoreState_TerminalAlloc(t *testing.T) { ar.allocLock.Lock() defer ar.allocLock.Unlock() - // Ensure both alloc runners don't destroy - ar.destroy = true - // Create a new alloc runner ar2 := NewAllocRunner(ar.logger, ar.config, ar.stateDB, upd.Update, &structs.Allocation{ID: ar.alloc.ID}, ar.vaultClient, ar.consulClient) diff --git a/client/consul.go b/client/consul.go index 5635bc362260..d470e3dea285 100644 --- a/client/consul.go +++ b/client/consul.go @@ -1,6 +1,7 @@ package client import ( + "github.com/hashicorp/consul/api" "github.com/hashicorp/nomad/client/driver" cstructs "github.com/hashicorp/nomad/client/structs" "github.com/hashicorp/nomad/nomad/structs" @@ -12,4 +13,5 @@ type ConsulServiceAPI interface { RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) error RemoveTask(allocID string, task *structs.Task) UpdateTask(allocID string, existing, newTask *structs.Task, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) error + Checks(alloc *structs.Allocation) ([]*api.AgentCheck, error) } diff --git a/client/consul_test.go b/client/consul_test.go index b8f282ebd801..897303f04ce3 100644 --- a/client/consul_test.go +++ b/client/consul_test.go @@ -8,6 +8,7 @@ import ( "sync" "testing" + "github.com/hashicorp/consul/api" "github.com/hashicorp/nomad/client/driver" cstructs "github.com/hashicorp/nomad/client/structs" "github.com/hashicorp/nomad/nomad/structs" @@ -23,7 +24,7 @@ type mockConsulOp struct { } func newMockConsulOp(op, allocID string, task *structs.Task, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) mockConsulOp { - if op != "add" && op != "remove" && op != "update" { + if op != "add" && op != "remove" && op != "update" && op != "checks" { panic(fmt.Errorf("invalid consul op: %s", op)) } return mockConsulOp{ @@ -42,6 +43,9 @@ type mockConsulServiceClient struct { mu sync.Mutex logger *log.Logger + + // checksFn allows injecting return values for the Checks function. + checksFn func(*structs.Allocation) ([]*api.AgentCheck, error) } func newMockConsulServiceClient() *mockConsulServiceClient { @@ -77,3 +81,16 @@ func (m *mockConsulServiceClient) RemoveTask(allocID string, task *structs.Task) m.logger.Printf("[TEST] mock_consul: RemoveTask(%q, %q)", allocID, task.Name) m.ops = append(m.ops, newMockConsulOp("remove", allocID, task, nil, nil)) } + +func (m *mockConsulServiceClient) Checks(alloc *structs.Allocation) ([]*api.AgentCheck, error) { + m.mu.Lock() + defer m.mu.Unlock() + m.logger.Printf("[TEST] mock_consul: Checks(%q)", alloc.ID) + m.ops = append(m.ops, newMockConsulOp("checks", alloc.ID, nil, nil, nil)) + + if m.checksFn != nil { + return m.checksFn(alloc) + } + + return nil, nil +} diff --git a/client/driver/mock_driver.go b/client/driver/mock_driver.go index 25493f0ddcfe..c1c64d5fac47 100644 --- a/client/driver/mock_driver.go +++ b/client/driver/mock_driver.go @@ -36,6 +36,9 @@ type MockDriverConfig struct { // StartErrRecoverable marks the error returned is recoverable StartErrRecoverable bool `mapstructure:"start_error_recoverable"` + // StartBlockFor specifies a duration in which to block before returning + StartBlockFor time.Duration `mapstructure:"start_block_for"` + // KillAfter is the duration after which the mock driver indicates the task // has exited after getting the initial SIGINT signal KillAfter time.Duration `mapstructure:"kill_after"` @@ -103,6 +106,10 @@ func (m *MockDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse return nil, err } + if driverConfig.StartBlockFor != 0 { + time.Sleep(driverConfig.StartBlockFor) + } + if driverConfig.StartErr != "" { return nil, structs.NewRecoverableError(errors.New(driverConfig.StartErr), driverConfig.StartErrRecoverable) } diff --git a/client/task_runner_test.go b/client/task_runner_test.go index 50ce28f6725e..ddd36f054090 100644 --- a/client/task_runner_test.go +++ b/client/task_runner_test.go @@ -32,7 +32,7 @@ func testLogger() *log.Logger { func prefixedTestLogger(prefix string) *log.Logger { if testing.Verbose() { - return log.New(os.Stderr, prefix, log.LstdFlags) + return log.New(os.Stderr, prefix, log.LstdFlags|log.Lmicroseconds) } return log.New(ioutil.Discard, "", 0) } diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index b5c1e4c201cc..1c1d762aad0a 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -606,6 +606,41 @@ func (c *ServiceClient) RemoveTask(allocID string, task *structs.Task) { c.commit(&ops) } +// Checks returns the checks registered against the agent for the given +// allocation. +func (c *ServiceClient) Checks(a *structs.Allocation) ([]*api.AgentCheck, error) { + tg := a.Job.LookupTaskGroup(a.TaskGroup) + if tg == nil { + return nil, fmt.Errorf("failed to find task group in alloc") + } + + // Determine the checks that are relevant + relevant := make(map[string]struct{}, 4) + for _, task := range tg.Tasks { + for _, service := range task.Services { + id := makeTaskServiceID(a.ID, task.Name, service) + for _, check := range service.Checks { + relevant[createCheckID(id, check)] = struct{}{} + } + } + } + + // Query all the checks + checks, err := c.client.Checks() + if err != nil { + return nil, err + } + + allocChecks := make([]*api.AgentCheck, 0, len(relevant)) + for checkID := range relevant { + if check, ok := checks[checkID]; ok { + allocChecks = append(allocChecks, check) + } + } + + return allocChecks, nil +} + // Shutdown the Consul client. Update running task registations and deregister // agent from Consul. On first call blocks up to shutdownWait before giving up // on syncing operations. diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go index 82357c62a476..737f22875273 100644 --- a/command/agent/consul/int_test.go +++ b/command/agent/consul/int_test.go @@ -214,6 +214,16 @@ func TestConsul_Integration(t *testing.T) { } } + // Assert the service client returns all the checks for the allocation. + checksOut, err := serviceClient.Checks(alloc) + if err != nil { + t.Fatalf("unexpected error retrieving allocation checks: %v", err) + } + + if l := len(checksOut); l != 2 { + t.Fatalf("got %d checks; want %d", l, 2) + } + logger.Printf("[TEST] consul.test: killing task") // Kill the task diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index aa541b936582..686f6a450559 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -111,7 +111,8 @@ func (s *GenericScheduler) Process(eval *structs.Evaluation) error { switch eval.TriggeredBy { case structs.EvalTriggerJobRegister, structs.EvalTriggerNodeUpdate, structs.EvalTriggerJobDeregister, structs.EvalTriggerRollingUpdate, - structs.EvalTriggerPeriodicJob, structs.EvalTriggerMaxPlans: + structs.EvalTriggerPeriodicJob, structs.EvalTriggerMaxPlans, + structs.EvalTriggerDeploymentWatcher: default: desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", eval.TriggeredBy) diff --git a/scheduler/system_sched.go b/scheduler/system_sched.go index a90d27dd2eb1..248deb0840ca 100644 --- a/scheduler/system_sched.go +++ b/scheduler/system_sched.go @@ -60,7 +60,8 @@ func (s *SystemScheduler) Process(eval *structs.Evaluation) error { // Verify the evaluation trigger reason is understood switch eval.TriggeredBy { case structs.EvalTriggerJobRegister, structs.EvalTriggerNodeUpdate, - structs.EvalTriggerJobDeregister, structs.EvalTriggerRollingUpdate: + structs.EvalTriggerJobDeregister, structs.EvalTriggerRollingUpdate, + structs.EvalTriggerDeploymentWatcher: default: desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", eval.TriggeredBy) From b8ba29bf934621fb40ebfe460ab657f67184979b Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 09:57:49 -0700 Subject: [PATCH 065/105] Warn log --- client/alloc_runner_health_watcher.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/alloc_runner_health_watcher.go b/client/alloc_runner_health_watcher.go index 332834c931a9..92e5c4149357 100644 --- a/client/alloc_runner_health_watcher.go +++ b/client/alloc_runner_health_watcher.go @@ -109,7 +109,7 @@ OUTER: case <-checkCh: newChecks, err := r.consulClient.Checks(alloc) if err != nil { - r.logger.Printf("[TRACE] client.alloc_watcher: failed to lookup consul checks for allocation %q: %v", alloc.ID, err) + r.logger.Printf("[WARN] client.alloc_watcher: failed to lookup consul checks for allocation %q: %v", alloc.ID, err) } checks = newChecks From 591ef9c9f3d05b142982849fb80bc19b2fff64ab Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 4 Jul 2017 13:31:01 -0700 Subject: [PATCH 066/105] Rename CreateDeployments and remove cancelling behavior in state_store --- command/agent/deployment_endpoint_test.go | 22 +- command/agent/job_endpoint_test.go | 4 +- nomad/core_sched_test.go | 8 +- nomad/deployment_endpoint_test.go | 26 +-- .../deployments_watcher_test.go | 84 ++++---- nomad/deploymentwatcher/testutil_test.go | 10 +- nomad/fsm_test.go | 14 +- nomad/job_endpoint_test.go | 22 +- nomad/plan_apply.go | 2 +- nomad/plan_apply_test.go | 8 +- nomad/state/state_store.go | 54 +---- nomad/state/state_store_test.go | 201 ++---------------- nomad/structs/diff.go | 2 +- nomad/structs/structs.go | 22 +- scheduler/generic_sched.go | 6 +- scheduler/generic_sched_test.go | 4 +- scheduler/reconcile.go | 8 +- scheduler/reconcile_test.go | 14 +- scheduler/testing.go | 2 +- 19 files changed, 166 insertions(+), 347 deletions(-) diff --git a/command/agent/deployment_endpoint_test.go b/command/agent/deployment_endpoint_test.go index 8a4ded4a70e9..e6dd7f06b3d1 100644 --- a/command/agent/deployment_endpoint_test.go +++ b/command/agent/deployment_endpoint_test.go @@ -17,8 +17,8 @@ func TestHTTP_DeploymentList(t *testing.T) { state := s.Agent.server.State() d1 := mock.Deployment() d2 := mock.Deployment() - assert.Nil(state.UpsertDeployment(999, d1, false), "UpsertDeployment") - assert.Nil(state.UpsertDeployment(1000, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(999, d1), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d2), "UpsertDeployment") // Make the HTTP request req, err := http.NewRequest("GET", "/v1/deployments", nil) @@ -49,8 +49,8 @@ func TestHTTP_DeploymentPrefixList(t *testing.T) { d1.ID = "aaabbbbb-e8f7-fd38-c855-ab94ceb89706" d2 := mock.Deployment() d2.ID = "aaabbbbb-e8f7-fd38-c855-ab94ceb89706" - assert.Nil(state.UpsertDeployment(999, d1, false), "UpsertDeployment") - assert.Nil(state.UpsertDeployment(1000, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(999, d1), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d2), "UpsertDeployment") // Make the HTTP request req, err := http.NewRequest("GET", "/v1/deployments?prefix=aaab", nil) @@ -88,8 +88,8 @@ func TestHTTP_DeploymentAllocations(t *testing.T) { a2.JobID = j.ID a2.DeploymentID = d.ID assert.Nil(state.UpsertJob(998, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(999, d, false), "UpsertDeployment") - assert.Nil(state.UpsertAllocs(1000, []*structs.Allocation{a1, a2}), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(999, d), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1000, []*structs.Allocation{a1, a2}), "UpsertAllocs") // Make the HTTP request req, err := http.NewRequest("GET", "/v1/deployment/allocations/"+d.ID, nil) @@ -117,7 +117,7 @@ func TestHTTP_DeploymentQuery(t *testing.T) { // Directly manipulate the state state := s.Agent.server.State() d := mock.Deployment() - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Make the HTTP request req, err := http.NewRequest("GET", "/v1/deployment/"+d.ID, nil) @@ -148,7 +148,7 @@ func TestHTTP_DeploymentPause(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Create the pause request args := structs.DeploymentPauseRequest{ @@ -185,7 +185,7 @@ func TestHTTP_DeploymentPromote(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Create the pause request args := structs.DeploymentPromoteRequest{ @@ -225,7 +225,7 @@ func TestHTTP_DeploymentAllocHealth(t *testing.T) { a.JobID = j.ID a.DeploymentID = d.ID assert.Nil(state.UpsertJob(998, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(999, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(999, d), "UpsertDeployment") assert.Nil(state.UpsertAllocs(1000, []*structs.Allocation{a}), "UpsertAllocs") // Create the pause request @@ -263,7 +263,7 @@ func TestHTTP_DeploymentFail(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(state.UpsertJob(998, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(999, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(999, d), "UpsertDeployment") // Make the HTTP request req, err := http.NewRequest("PUT", "/v1/deployment/fail/"+d.ID, nil) diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index 95915c06a09f..6e71235d67f8 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -619,7 +619,7 @@ func TestHTTP_JobDeployments(t *testing.T) { state := s.Agent.server.State() d := mock.Deployment() d.JobID = j.ID - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Make the HTTP request req, err := http.NewRequest("GET", "/v1/job/"+j.ID+"/deployments", nil) @@ -657,7 +657,7 @@ func TestHTTP_JobDeployment(t *testing.T) { state := s.Agent.server.State() d := mock.Deployment() d.JobID = j.ID - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Make the HTTP request req, err := http.NewRequest("GET", "/v1/job/"+j.ID+"/deployment", nil) diff --git a/nomad/core_sched_test.go b/nomad/core_sched_test.go index 9496c2051197..76da207dd60e 100644 --- a/nomad/core_sched_test.go +++ b/nomad/core_sched_test.go @@ -1329,8 +1329,8 @@ func TestCoreScheduler_DeploymentGC(t *testing.T) { state := s1.fsm.State() d1, d2 := mock.Deployment(), mock.Deployment() d1.Status = structs.DeploymentStatusFailed - assert.Nil(state.UpsertDeployment(1000, d1, false), "UpsertDeployment") - assert.Nil(state.UpsertDeployment(1001, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d1), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1001, d2), "UpsertDeployment") // Update the time tables to make this work tt := s1.fsm.TimeTable() @@ -1368,8 +1368,8 @@ func TestCoreScheduler_DeploymentGC_Force(t *testing.T) { state := s1.fsm.State() d1, d2 := mock.Deployment(), mock.Deployment() d1.Status = structs.DeploymentStatusFailed - assert.Nil(state.UpsertDeployment(1000, d1, false), "UpsertDeployment") - assert.Nil(state.UpsertDeployment(1001, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d1), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1001, d2), "UpsertDeployment") // Create a core scheduler snap, err := state.Snapshot() diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index 10eefcedaa89..25a91f0d3794 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -27,7 +27,7 @@ func TestDeploymentEndpoint_GetDeployment(t *testing.T) { state := s1.fsm.State() assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Lookup the deployments get := &structs.DeploymentSpecificRequest{ @@ -61,12 +61,12 @@ func TestDeploymentEndpoint_GetDeployment_Blocking(t *testing.T) { // Upsert a deployment we are not interested in first. time.AfterFunc(100*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(100, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(100, d1), "UpsertDeployment") }) // Upsert another deployment later which should trigger the watch. time.AfterFunc(200*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(200, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(200, d2), "UpsertDeployment") }) // Lookup the deployments @@ -103,7 +103,7 @@ func TestDeploymentEndpoint_Fail(t *testing.T) { state := s1.fsm.State() assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Mark the deployment as failed req := &structs.DeploymentFailRequest{ @@ -151,7 +151,7 @@ func TestDeploymentEndpoint_Pause(t *testing.T) { state := s1.fsm.State() assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Mark the deployment as failed req := &structs.DeploymentPauseRequest{ @@ -202,7 +202,7 @@ func TestDeploymentEndpoint_Promote(t *testing.T) { state := s1.fsm.State() assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") // Promote the deployment @@ -267,7 +267,7 @@ func TestDeploymentEndpoint_SetAllocHealth(t *testing.T) { state := s1.fsm.State() assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") // Set the alloc as healthy @@ -326,7 +326,7 @@ func TestDeploymentEndpoint_List(t *testing.T) { state := s1.fsm.State() assert.Nil(state.UpsertJob(999, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") // Lookup the deployments get := &structs.DeploymentListRequest{ @@ -367,7 +367,7 @@ func TestDeploymentEndpoint_List_Blocking(t *testing.T) { // Upsert alloc triggers watches time.AfterFunc(100*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(3, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(3, d), "UpsertDeployment") }) req := &structs.DeploymentListRequest{ @@ -390,7 +390,7 @@ func TestDeploymentEndpoint_List_Blocking(t *testing.T) { d2 := d.Copy() d2.Status = structs.DeploymentStatusPaused time.AfterFunc(100*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(5, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(5, d2), "UpsertDeployment") }) req.MinQueryIndex = 3 @@ -423,7 +423,7 @@ func TestDeploymentEndpoint_Allocations(t *testing.T) { assert.Nil(state.UpsertJob(998, j), "UpsertJob") assert.Nil(state.UpsertJobSummary(999, summary), "UpsertJobSummary") - assert.Nil(state.UpsertDeployment(1000, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") // Lookup the allocations @@ -455,7 +455,7 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { summary := mock.JobSummary(a.JobID) assert.Nil(state.UpsertJob(1, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(2, d, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(2, d), "UpsertDeployment") assert.Nil(state.UpsertJobSummary(3, summary), "UpsertJobSummary") // Upsert alloc triggers watches @@ -512,7 +512,7 @@ func TestDeploymentEndpoint_Reap(t *testing.T) { // Create the register request d1 := mock.Deployment() - assert.Nil(s1.fsm.State().UpsertDeployment(1000, d1, false), "UpsertDeployment") + assert.Nil(s1.fsm.State().UpsertDeployment(1000, d1), "UpsertDeployment") // Reap the eval get := &structs.DeploymentDeleteRequest{ diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 14e24a5386aa..770e3af410f6 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -176,7 +176,7 @@ func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") // Assert the following methods will be called m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) @@ -199,7 +199,7 @@ func TestWatcher_SetAllocHealth_Unknown(t *testing.T) { Eval: true, } matcher := matchDeploymentAllocHealthRequest(matchConfig) - m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) // Call SetAllocHealth req := &structs.DeploymentAllocHealthRequest{ @@ -226,7 +226,7 @@ func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { a := mock.Alloc() a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") // Assert the following methods will be called @@ -249,7 +249,7 @@ func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { Eval: true, } matcher := matchDeploymentAllocHealthRequest(matchConfig) - m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) // Call SetAllocHealth req := &structs.DeploymentAllocHealthRequest{ @@ -260,7 +260,7 @@ func TestWatcher_SetAllocHealth_Healthy(t *testing.T) { err := w.SetAllocHealth(req, &resp) assert.Nil(err, "SetAllocHealth") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentAllocHealth", mocker.MatchedBy(matcher)) } // Test setting allocation unhealthy @@ -275,7 +275,7 @@ func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { a := mock.Alloc() a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") // Assert the following methods will be called @@ -303,7 +303,7 @@ func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { }, } matcher := matchDeploymentAllocHealthRequest(matchConfig) - m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) // Call SetAllocHealth req := &structs.DeploymentAllocHealthRequest{ @@ -316,7 +316,7 @@ func TestWatcher_SetAllocHealth_Unhealthy(t *testing.T) { testutil.WaitForResult(func() (bool, error) { return 0 == len(w.watchers), nil }, func(err error) { assert.Equal(0, len(w.watchers), "Should have no deployment") }) - m.AssertNumberOfCalls(t, "UpsertDeploymentAllocHealth", 1) + m.AssertNumberOfCalls(t, "UpdateDeploymentAllocHealth", 1) } // Test setting allocation unhealthy and that there should be a rollback @@ -332,10 +332,11 @@ func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { j.Stable = true d := mock.Deployment() d.JobID = j.ID + d.TaskGroups["web"].AutoRevert = true a := mock.Alloc() a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") // Upsert the job again to get a new version @@ -351,7 +352,7 @@ func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { mocker.Anything).Return(nil).Run(m.evaluationsFromState) m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), mocker.Anything).Return(nil).Run(m.getJobFromState) - m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + m.On("GetJobVersions", mocker.MatchedBy(matchJobVersionsRequest(j.ID)), mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) w.SetEnabled(true) @@ -371,7 +372,7 @@ func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { JobVersion: helper.Uint64ToPtr(0), } matcher := matchDeploymentAllocHealthRequest(matchConfig) - m.On("UpsertDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentAllocHealth", mocker.MatchedBy(matcher)).Return(nil) // Call SetAllocHealth req := &structs.DeploymentAllocHealthRequest{ @@ -384,7 +385,7 @@ func TestWatcher_SetAllocHealth_Unhealthy_Rollback(t *testing.T) { testutil.WaitForResult(func() (bool, error) { return 0 == len(w.watchers), nil }, func(err error) { assert.Equal(0, len(w.watchers), "Should have no deployment") }) - m.AssertNumberOfCalls(t, "UpsertDeploymentAllocHealth", 1) + m.AssertNumberOfCalls(t, "UpdateDeploymentAllocHealth", 1) } // Test promoting a deployment @@ -406,7 +407,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { } a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") // Assert the following methods will be called @@ -431,7 +432,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { Eval: true, } matcher := matchDeploymentPromoteRequest(matchConfig) - m.On("UpsertDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) // Call PromoteDeployment req := &structs.DeploymentPromoteRequest{ @@ -442,7 +443,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { err := w.PromoteDeployment(req, &resp) assert.Nil(err, "PromoteDeployment") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentPromotion", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentPromotion", mocker.MatchedBy(matcher)) } // Test promoting a deployment with unhealthy canaries @@ -461,7 +462,7 @@ func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { a.Canary = true a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") // Assert the following methods will be called @@ -486,7 +487,7 @@ func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { Eval: true, } matcher := matchDeploymentPromoteRequest(matchConfig) - m.On("UpsertDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentPromotion", mocker.MatchedBy(matcher)).Return(nil) // Call SetAllocHealth req := &structs.DeploymentPromoteRequest{ @@ -500,7 +501,7 @@ func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { } assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentPromotion", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentPromotion", mocker.MatchedBy(matcher)) } // Test pausing a deployment that is running @@ -513,7 +514,7 @@ func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") // Assert the following methods will be called m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) @@ -535,7 +536,7 @@ func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { StatusDescription: structs.DeploymentStatusDescriptionPaused, } matcher := matchDeploymentStatusUpdateRequest(matchConfig) - m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentStatus", mocker.MatchedBy(matcher)).Return(nil) // Call PauseDeployment req := &structs.DeploymentPauseRequest{ @@ -547,7 +548,7 @@ func TestWatcher_PauseDeployment_Pause_Running(t *testing.T) { assert.Nil(err, "PauseDeployment") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentStatus", mocker.MatchedBy(matcher)) } // Test pausing a deployment that is paused @@ -561,7 +562,7 @@ func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { d.JobID = j.ID d.Status = structs.DeploymentStatusPaused assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") // Assert the following methods will be called m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) @@ -583,7 +584,7 @@ func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { StatusDescription: structs.DeploymentStatusDescriptionPaused, } matcher := matchDeploymentStatusUpdateRequest(matchConfig) - m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentStatus", mocker.MatchedBy(matcher)).Return(nil) // Call PauseDeployment req := &structs.DeploymentPauseRequest{ @@ -595,7 +596,7 @@ func TestWatcher_PauseDeployment_Pause_Paused(t *testing.T) { assert.Nil(err, "PauseDeployment") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentStatus", mocker.MatchedBy(matcher)) } // Test unpausing a deployment that is paused @@ -609,7 +610,7 @@ func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { d.JobID = j.ID d.Status = structs.DeploymentStatusPaused assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") // Assert the following methods will be called m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) @@ -632,7 +633,7 @@ func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { Eval: true, } matcher := matchDeploymentStatusUpdateRequest(matchConfig) - m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentStatus", mocker.MatchedBy(matcher)).Return(nil) // Call PauseDeployment req := &structs.DeploymentPauseRequest{ @@ -644,7 +645,7 @@ func TestWatcher_PauseDeployment_Unpause_Paused(t *testing.T) { assert.Nil(err, "PauseDeployment") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentStatus", mocker.MatchedBy(matcher)) } // Test unpausing a deployment that is running @@ -657,7 +658,7 @@ func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") // Assert the following methods will be called m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) @@ -680,7 +681,7 @@ func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { Eval: true, } matcher := matchDeploymentStatusUpdateRequest(matchConfig) - m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentStatus", mocker.MatchedBy(matcher)).Return(nil) // Call PauseDeployment req := &structs.DeploymentPauseRequest{ @@ -692,7 +693,7 @@ func TestWatcher_PauseDeployment_Unpause_Running(t *testing.T) { assert.Nil(err, "PauseDeployment") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentStatus", mocker.MatchedBy(matcher)) } // Test failing a deployment that is running @@ -705,7 +706,7 @@ func TestWatcher_FailDeployment_Running(t *testing.T) { d := mock.Deployment() d.JobID = j.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") // Assert the following methods will be called m.On("List", mocker.Anything, mocker.Anything).Return(nil).Run(m.listFromState) @@ -728,7 +729,7 @@ func TestWatcher_FailDeployment_Running(t *testing.T) { Eval: true, } matcher := matchDeploymentStatusUpdateRequest(matchConfig) - m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)).Return(nil) + m.On("UpdateDeploymentStatus", mocker.MatchedBy(matcher)).Return(nil) // Call PauseDeployment req := &structs.DeploymentFailRequest{ @@ -739,7 +740,7 @@ func TestWatcher_FailDeployment_Running(t *testing.T) { assert.Nil(err, "FailDeployment") assert.Equal(1, len(w.watchers), "Deployment should still be active") - m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(matcher)) + m.AssertCalled(t, "UpdateDeploymentStatus", mocker.MatchedBy(matcher)) } // Tests that the watcher properly watches for allocation changes and takes the @@ -756,10 +757,11 @@ func TestDeploymentWatcher_Watch(t *testing.T) { j.Stable = true d := mock.Deployment() d.JobID = j.ID + d.TaskGroups["web"].AutoRevert = true a := mock.Alloc() a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a}), "UpsertAllocs") // Upsert the job again to get a new version @@ -775,7 +777,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { mocker.Anything).Return(nil).Run(m.evaluationsFromState) m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), mocker.Anything).Return(nil).Run(m.getJobFromState) - m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j.ID)), + m.On("GetJobVersions", mocker.MatchedBy(matchJobVersionsRequest(j.ID)), mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) w.SetEnabled(true) @@ -824,7 +826,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { Eval: true, } m2 := matchDeploymentStatusUpdateRequest(c) - m.On("UpsertDeploymentStatusUpdate", mocker.MatchedBy(m2)).Return(nil) + m.On("UpdateDeploymentStatus", mocker.MatchedBy(m2)).Return(nil) // Update the allocs health to unhealthy which should create a job rollback, // status update and eval @@ -865,7 +867,7 @@ func TestDeploymentWatcher_Watch(t *testing.T) { Eval: true, } m3 := matchDeploymentStatusUpdateRequest(c2) - m.AssertCalled(t, "UpsertDeploymentStatusUpdate", mocker.MatchedBy(m3)) + m.AssertCalled(t, "UpdateDeploymentStatus", mocker.MatchedBy(m3)) testutil.WaitForResult(func() (bool, error) { return 0 == len(w.watchers), nil }, func(err error) { assert.Equal(0, len(w.watchers), "Should have no deployment") }) } @@ -890,8 +892,8 @@ func TestWatcher_BatchEvals(t *testing.T) { assert.Nil(m.state.UpsertJob(m.nextIndex(), j1), "UpsertJob") assert.Nil(m.state.UpsertJob(m.nextIndex(), j2), "UpsertJob") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d1, false), "UpsertDeployment") - assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d2, false), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d1), "UpsertDeployment") + assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d2), "UpsertDeployment") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a1}), "UpsertAllocs") assert.Nil(m.state.UpsertAllocs(m.nextIndex(), []*structs.Allocation{a2}), "UpsertAllocs") @@ -913,9 +915,9 @@ func TestWatcher_BatchEvals(t *testing.T) { m.On("GetJob", mocker.MatchedBy(matchJobSpecificRequest(j2.ID)), mocker.Anything).Return(nil).Run(m.getJobFromState) - m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j1.ID)), + m.On("GetJobVersions", mocker.MatchedBy(matchJobVersionsRequest(j1.ID)), mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) - m.On("GetJobVersions", mocker.MatchedBy(matchJobSpecificRequest(j2.ID)), + m.On("GetJobVersions", mocker.MatchedBy(matchJobVersionsRequest(j2.ID)), mocker.Anything).Return(nil).Run(m.getJobVersionsFromState) w.SetEnabled(true) diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index c3492d06e17b..768a287178df 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -324,7 +324,7 @@ func (m *mockBackend) GetJobVersions(args *structs.JobVersionsRequest, reply *st } func (m *mockBackend) getJobVersionsFromState(in mocker.Arguments) { - args, reply := in.Get(0).(*structs.JobSpecificRequest), in.Get(1).(*structs.JobVersionsResponse) + args, reply := in.Get(0).(*structs.JobVersionsRequest), in.Get(1).(*structs.JobVersionsResponse) ws := memdb.NewWatchSet() versions, _ := m.state.JobVersionsByID(ws, args.JobID) reply.Versions = versions @@ -359,3 +359,11 @@ func matchJobSpecificRequest(jID string) func(args *structs.JobSpecificRequest) return args.JobID == jID } } + +// matchJobVersionsRequest is used to match that a job version +// request is for the passed job id +func matchJobVersionsRequest(jID string) func(args *structs.JobVersionsRequest) bool { + return func(args *structs.JobVersionsRequest) bool { + return args.JobID == jID + } +} diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 67b4a885d7a0..d949df2aa80c 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1070,7 +1070,7 @@ func TestFSM_ApplyPlanResults(t *testing.T) { Job: job, Alloc: []*structs.Allocation{alloc}, }, - CreatedDeployment: d, + Deployment: d, } buf, err := structs.Encode(structs.ApplyPlanResultsRequestType, req) if err != nil { @@ -1149,7 +1149,7 @@ func TestFSM_DeploymentStatusUpdate(t *testing.T) { // Upsert a deployment d := mock.Deployment() - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -1238,7 +1238,7 @@ func TestFSM_DeploymentPromotion(t *testing.T) { DesiredCanaries: 1, }, } - if err := state.UpsertDeployment(2, d, false); err != nil { + if err := state.UpsertDeployment(2, d); err != nil { t.Fatalf("bad: %v", err) } @@ -1340,7 +1340,7 @@ func TestFSM_DeploymentAllocHealth(t *testing.T) { // Insert a deployment d := mock.Deployment() - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -1445,7 +1445,7 @@ func TestFSM_DeleteDeployment(t *testing.T) { // Upsert a deployments d := mock.Deployment() - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -1784,8 +1784,8 @@ func TestFSM_SnapshotRestore_Deployments(t *testing.T) { state := fsm.State() d1 := mock.Deployment() d2 := mock.Deployment() - state.UpsertDeployment(1000, d1, false) - state.UpsertDeployment(1001, d2, false) + state.UpsertDeployment(1000, d1) + state.UpsertDeployment(1001, d2) // Verify the contents fsm2 := testSnapshotRestore(t, fsm) diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index b6a31f5ca087..233bf3ad58e0 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -1385,6 +1385,10 @@ func TestJobEndpoint_GetJob(t *testing.T) { } } + // Clear the submit times + j.SubmitTime = 0 + resp2.Job.SubmitTime = 0 + if !reflect.DeepEqual(j, resp2.Job) { t.Fatalf("bad: %#v %#v", job, resp2.Job) } @@ -2168,8 +2172,8 @@ func TestJobEndpoint_Deployments(t *testing.T) { d1.JobID = j.ID d2.JobID = j.ID assert.Nil(state.UpsertJob(1000, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1001, d1, false), "UpsertDeployment") - assert.Nil(state.UpsertDeployment(1002, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1001, d1), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1002, d2), "UpsertDeployment") // Lookup the jobs get := &structs.JobSpecificRequest{ @@ -2199,12 +2203,12 @@ func TestJobEndpoint_Deployments_Blocking(t *testing.T) { // First upsert an unrelated eval time.AfterFunc(100*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(100, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(100, d1), "UpsertDeployment") }) // Upsert an eval for the job we are interested in later time.AfterFunc(200*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(200, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(200, d2), "UpsertDeployment") }) // Lookup the jobs @@ -2220,7 +2224,7 @@ func TestJobEndpoint_Deployments_Blocking(t *testing.T) { assert.Nil(msgpackrpc.CallWithCodec(codec, "Job.Deployments", get, &resp), "RPC") assert.EqualValues(200, resp.Index, "response index") assert.Len(resp.Deployments, 1, "deployments for job") - assert.Equal(d2.ID, resp.Deployments[0], "returned deployment") + assert.Equal(d2.ID, resp.Deployments[0].ID, "returned deployment") if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } @@ -2243,8 +2247,8 @@ func TestJobEndpoint_LatestDeployment(t *testing.T) { d2.CreateIndex = d1.CreateIndex + 100 d2.ModifyIndex = d2.CreateIndex + 100 assert.Nil(state.UpsertJob(1000, j), "UpsertJob") - assert.Nil(state.UpsertDeployment(1001, d1, false), "UpsertDeployment") - assert.Nil(state.UpsertDeployment(1002, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1001, d1), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(1002, d2), "UpsertDeployment") // Lookup the jobs get := &structs.JobSpecificRequest{ @@ -2275,12 +2279,12 @@ func TestJobEndpoint_LatestDeployment_Blocking(t *testing.T) { // First upsert an unrelated eval time.AfterFunc(100*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(100, d1, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(100, d1), "UpsertDeployment") }) // Upsert an eval for the job we are interested in later time.AfterFunc(200*time.Millisecond, func() { - assert.Nil(state.UpsertDeployment(200, d2, false), "UpsertDeployment") + assert.Nil(state.UpsertDeployment(200, d2), "UpsertDeployment") }) // Lookup the jobs diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 96b6edca9347..cb5c5eb10388 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -132,7 +132,7 @@ func (s *Server) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap Job: plan.Job, Alloc: make([]*structs.Allocation, 0, minUpdates), }, - CreatedDeployment: plan.CreatedDeployment, + Deployment: plan.Deployment, DeploymentUpdates: plan.DeploymentUpdates, } for _, updateList := range result.NodeUpdate { diff --git a/nomad/plan_apply_test.go b/nomad/plan_apply_test.go index d6e1cd193543..b4b16bbac6f6 100644 --- a/nomad/plan_apply_test.go +++ b/nomad/plan_apply_test.go @@ -84,8 +84,8 @@ func TestPlanApply_applyPlan(t *testing.T) { // Create the plan with a deployment plan := &structs.Plan{ - Job: alloc.Job, - CreatedDeployment: mock.Deployment(), + Job: alloc.Job, + Deployment: mock.Deployment(), } // Apply the plan @@ -100,7 +100,7 @@ func TestPlanApply_applyPlan(t *testing.T) { t.Fatalf("bad: %v %v", out, err) } - if out, err := snap.DeploymentByID(ws, plan.CreatedDeployment.ID); err != nil || out == nil { + if out, err := snap.DeploymentByID(ws, plan.Deployment.ID); err != nil || out == nil { t.Fatalf("bad: %v %v", out, err) } @@ -124,7 +124,7 @@ func TestPlanApply_applyPlan(t *testing.T) { } // Lookup the deployment - dout, err := fsmState.DeploymentByID(ws, plan.CreatedDeployment.ID) + dout, err := fsmState.DeploymentByID(ws, plan.Deployment.ID) if err != nil { t.Fatalf("err: %v", err) } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index ed3207ccabad..1c4afeeda5ab 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -92,9 +92,9 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR txn := s.db.Txn(true) defer txn.Abort() - // Upsert the newly created deployment - if results.CreatedDeployment != nil { - if err := s.upsertDeploymentImpl(index, results.CreatedDeployment, true, txn); err != nil { + // Upsert the newly created or updated deployment + if results.Deployment != nil { + if err := s.upsertDeploymentImpl(index, results.Deployment, txn); err != nil { return err } } @@ -220,22 +220,17 @@ func (s *StateStore) DeleteJobSummary(index uint64, id string) error { // UpsertDeployment is used to insert a new deployment. If cancelPrior is set to // true, all prior deployments for the same job will be cancelled. -func (s *StateStore) UpsertDeployment(index uint64, deployment *structs.Deployment, cancelPrior bool) error { +func (s *StateStore) UpsertDeployment(index uint64, deployment *structs.Deployment) error { txn := s.db.Txn(true) defer txn.Abort() - if err := s.upsertDeploymentImpl(index, deployment, cancelPrior, txn); err != nil { + if err := s.upsertDeploymentImpl(index, deployment, txn); err != nil { return err } txn.Commit() return nil } -func (s *StateStore) upsertDeploymentImpl(index uint64, deployment *structs.Deployment, cancelPrior bool, txn *memdb.Txn) error { - // Go through and cancel any active deployment for the job. - if cancelPrior { - s.cancelPriorDeployments(index, deployment, txn) - } - +func (s *StateStore) upsertDeploymentImpl(index uint64, deployment *structs.Deployment, txn *memdb.Txn) error { // Check if the deployment already exists existing, err := txn.First("deployment", "id", deployment.ID) if err != nil { @@ -264,39 +259,6 @@ func (s *StateStore) upsertDeploymentImpl(index uint64, deployment *structs.Depl return nil } -// cancelPriorDeployments cancels any prior deployments for the job. -func (s *StateStore) cancelPriorDeployments(index uint64, deployment *structs.Deployment, txn *memdb.Txn) error { - iter, err := txn.Get("deployment", "job", deployment.JobID) - if err != nil { - return fmt.Errorf("deployment lookup failed: %v", err) - } - - for { - raw := iter.Next() - if raw == nil { - break - } - - // Ensure the deployment is active - d := raw.(*structs.Deployment) - if !d.Active() { - continue - } - - // We need to cancel so make a copy and set its status - cancelled := d.Copy() - cancelled.ModifyIndex = index - cancelled.Status = structs.DeploymentStatusCancelled - cancelled.StatusDescription = fmt.Sprintf("Cancelled in favor of deployment %q", deployment.ID) - - // Insert the cancelled deployment - if err := txn.Insert("deployment", cancelled); err != nil { - return err - } - } - return nil -} - func (s *StateStore) Deployments(ws memdb.WatchSet) (memdb.ResultIterator, error) { txn := s.db.Txn(false) @@ -2597,8 +2559,8 @@ func (s *StateStore) updateDeploymentWithAlloc(index uint64, alloc, existing *st state.HealthyAllocs += healthy state.UnhealthyAllocs += unhealthy - // Upsert the new deployment - if err := s.upsertDeploymentImpl(index, deploymentCopy, false, txn); err != nil { + // Upsert the deployment + if err := s.upsertDeploymentImpl(index, deploymentCopy, txn); err != nil { return err } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 81de030de946..41890f5ab1aa 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -99,7 +99,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { Alloc: []*structs.Allocation{alloc, alloc2}, Job: job, }, - CreatedDeployment: d, + Deployment: d, } err := state.UpsertPlanResults(1000, &res) @@ -139,98 +139,6 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { } } -// This test checks that when a new deployment is made, the old ones are -// cancelled. -func TestStateStore_UpsertPlanResults_Deployment_CancelOld(t *testing.T) { - state := testStateStore(t) - - // Create a job that applies to all - job := mock.Job() - if err := state.UpsertJob(998, job); err != nil { - t.Fatalf("err: %v", err) - } - - // Create two deployments: - // One that is already terminal and assert its modify index isn't touched - // A second that is outstanding and assert it gets cancelled. - dterminal := mock.Deployment() - dterminal.Status = structs.DeploymentStatusFailed - dterminal.JobID = job.ID - doutstanding := mock.Deployment() - doutstanding.JobID = job.ID - - if err := state.UpsertDeployment(999, dterminal, false); err != nil { - t.Fatalf("err: %v", err) - } - if err := state.UpsertDeployment(1000, doutstanding, false); err != nil { - t.Fatalf("err: %v", err) - } - - alloc := mock.Alloc() - alloc2 := mock.Alloc() - alloc.Job = nil - alloc2.Job = nil - - dnew := mock.Deployment() - dnew.JobID = job.ID - alloc.DeploymentID = dnew.ID - alloc2.DeploymentID = dnew.ID - - // Create a plan result - res := structs.ApplyPlanResultsRequest{ - AllocUpdateRequest: structs.AllocUpdateRequest{ - Alloc: []*structs.Allocation{alloc, alloc2}, - Job: job, - }, - CreatedDeployment: dnew, - } - - err := state.UpsertPlanResults(1001, &res) - if err != nil { - t.Fatalf("err: %v", err) - } - - ws := memdb.NewWatchSet() - - // Check the deployments are correctly updated. - dout, err := state.DeploymentByID(ws, dnew.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - - if dout == nil { - t.Fatalf("bad: nil deployment") - } - - tg, ok := dout.TaskGroups[alloc.TaskGroup] - if !ok { - t.Fatalf("bad: nil deployment state") - } - if tg == nil || tg.PlacedAllocs != 2 { - t.Fatalf("bad: %v", dout) - } - - dterminalout, err := state.DeploymentByID(ws, dterminal.ID) - if err != nil || dterminalout == nil { - t.Fatalf("bad: %v %v", err, dterminalout) - } - if !reflect.DeepEqual(dterminalout, dterminal) { - t.Fatalf("bad: %v %v", dterminal, dterminalout) - } - - doutstandingout, err := state.DeploymentByID(ws, doutstanding.ID) - if err != nil || doutstandingout == nil { - t.Fatalf("bad: %v %v", err, doutstandingout) - } - if doutstandingout.Status != structs.DeploymentStatusCancelled || doutstandingout.ModifyIndex != 1001 { - t.Fatalf("bad: %v", doutstandingout) - } - - if watchFired(ws) { - t.Fatalf("bad") - } -} - // This test checks that deployment updates are applied correctly func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) { state := testStateStore(t) @@ -245,7 +153,7 @@ func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) { doutstanding := mock.Deployment() doutstanding.JobID = job.ID - if err := state.UpsertDeployment(1000, doutstanding, false); err != nil { + if err := state.UpsertDeployment(1000, doutstanding); err != nil { t.Fatalf("err: %v", err) } @@ -269,7 +177,7 @@ func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) { Alloc: []*structs.Allocation{alloc}, Job: job, }, - CreatedDeployment: dnew, + Deployment: dnew, DeploymentUpdates: []*structs.DeploymentStatusUpdate{update}, } @@ -322,7 +230,7 @@ func TestStateStore_UpsertDeployment(t *testing.T) { t.Fatalf("bad: %v", err) } - err = state.UpsertDeployment(1000, deployment, false) + err = state.UpsertDeployment(1000, deployment) if err != nil { t.Fatalf("err: %v", err) } @@ -353,83 +261,16 @@ func TestStateStore_UpsertDeployment(t *testing.T) { } } -func TestStateStore_UpsertDeployment_Cancel(t *testing.T) { - state := testStateStore(t) - deployment := mock.Deployment() - deployment2 := mock.Deployment() - deployment2.JobID = deployment.JobID - - // Create a deployment that shares the job id prefix - deployment3 := mock.Deployment() - deployment3.JobID = deployment.JobID + "foo" - - if err := state.UpsertDeployment(1000, deployment, false); err != nil { - t.Fatalf("err: %v", err) - } - - if err := state.UpsertDeployment(1001, deployment2, true); err != nil { - t.Fatalf("err: %v", err) - } - - if err := state.UpsertDeployment(1002, deployment3, true); err != nil { - t.Fatalf("err: %v", err) - } - - ws := memdb.NewWatchSet() - out, err := state.DeploymentByID(ws, deployment.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - - // Check to see that the deployment was cancelled - if out.Status != structs.DeploymentStatusCancelled { - t.Fatalf("got status %v; want %v", out.Status, structs.DeploymentStatusCancelled) - } else if out.ModifyIndex != 1001 { - t.Fatalf("got modify index %v; want %v", out.ModifyIndex, 1001) - } - - // Get the deployments by job - deployments, err := state.DeploymentsByJobID(ws, deployment.JobID) - if err != nil { - t.Fatalf("err: %v", err) - } - - if l := len(deployments); l != 2 { - t.Fatalf("got %d deployments; want %v", l, 2) - } - - latest, err := state.LatestDeploymentByJobID(ws, deployment.JobID) - if err != nil { - t.Fatalf("err: %v", err) - } - - if latest == nil || latest.CreateIndex != 1001 { - t.Fatalf("bad: %+v", latest) - } - - index, err := state.Index("deployment") - if err != nil { - t.Fatalf("err: %v", err) - } - if index != 1002 { - t.Fatalf("bad: %d", index) - } - - if watchFired(ws) { - t.Fatalf("bad") - } -} - func TestStateStore_DeleteDeployment(t *testing.T) { state := testStateStore(t) d1 := mock.Deployment() d2 := mock.Deployment() - err := state.UpsertDeployment(1000, d1, false) + err := state.UpsertDeployment(1000, d1) if err != nil { t.Fatalf("err: %v", err) } - if err := state.UpsertDeployment(1001, d2, false); err != nil { + if err := state.UpsertDeployment(1001, d2); err != nil { t.Fatalf("err: %v", err) } @@ -479,7 +320,7 @@ func TestStateStore_Deployments(t *testing.T) { deployment := mock.Deployment() deployments = append(deployments, deployment) - err := state.UpsertDeployment(1000+uint64(i), deployment, false) + err := state.UpsertDeployment(1000+uint64(i), deployment) if err != nil { t.Fatalf("err: %v", err) } @@ -520,7 +361,7 @@ func TestStateStore_DeploymentsByIDPrefix(t *testing.T) { deploy := mock.Deployment() deploy.ID = "11111111-662e-d0ab-d1c9-3e434af7bdb4" - err := state.UpsertDeployment(1000, deploy, false) + err := state.UpsertDeployment(1000, deploy) if err != nil { t.Fatalf("err: %v", err) } @@ -566,7 +407,7 @@ func TestStateStore_DeploymentsByIDPrefix(t *testing.T) { deploy = mock.Deployment() deploy.ID = "11222222-662e-d0ab-d1c9-3e434af7bdb4" - err = state.UpsertDeployment(1001, deploy, false) + err = state.UpsertDeployment(1001, deploy) if err != nil { t.Fatalf("err: %v", err) } @@ -3352,7 +3193,7 @@ func TestStateStore_UpsertAlloc_Deployment(t *testing.T) { if err := state.UpsertJob(999, alloc.Job); err != nil { t.Fatalf("err: %v", err) } - if err := state.UpsertDeployment(1000, deployment, false); err != nil { + if err := state.UpsertDeployment(1000, deployment); err != nil { t.Fatalf("err: %v", err) } @@ -4872,7 +4713,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Terminal(t *testing.T) { d := mock.Deployment() d.Status = structs.DeploymentStatusFailed - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -4896,7 +4737,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) { // Insert a deployment d := mock.Deployment() - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -4974,7 +4815,7 @@ func TestStateStore_UpsertDeploymentPromotion_Terminal(t *testing.T) { d := mock.Deployment() d.Status = structs.DeploymentStatusFailed - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -5004,7 +4845,7 @@ func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) { // Create a deployment d := mock.Deployment() d.JobID = j.ID - if err := state.UpsertDeployment(2, d, false); err != nil { + if err := state.UpsertDeployment(2, d); err != nil { t.Fatalf("bad: %v", err) } @@ -5068,7 +4909,7 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { DesiredCanaries: 1, }, } - if err := state.UpsertDeployment(2, d, false); err != nil { + if err := state.UpsertDeployment(2, d); err != nil { t.Fatalf("bad: %v", err) } @@ -5180,7 +5021,7 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { DesiredCanaries: 1, }, } - if err := state.UpsertDeployment(2, d, false); err != nil { + if err := state.UpsertDeployment(2, d); err != nil { t.Fatalf("bad: %v", err) } @@ -5290,7 +5131,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Terminal(t *testing.T) { d := mock.Deployment() d.Status = structs.DeploymentStatusFailed - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -5313,7 +5154,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_NonExistant(t *testing. // Insert a deployment d := mock.Deployment() - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } @@ -5338,10 +5179,10 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_MismatchDeployment(t *t // Insert two deployment d1 := mock.Deployment() d2 := mock.Deployment() - if err := state.UpsertDeployment(1, d1, false); err != nil { + if err := state.UpsertDeployment(1, d1); err != nil { t.Fatalf("bad: %v", err) } - if err := state.UpsertDeployment(2, d2, false); err != nil { + if err := state.UpsertDeployment(2, d2); err != nil { t.Fatalf("bad: %v", err) } @@ -5371,7 +5212,7 @@ func TestStateStore_UpsertDeploymentAllocHealth(t *testing.T) { // Insert a deployment d := mock.Deployment() - if err := state.UpsertDeployment(1, d, false); err != nil { + if err := state.UpsertDeployment(1, d); err != nil { t.Fatalf("bad: %v", err) } diff --git a/nomad/structs/diff.go b/nomad/structs/diff.go index 246e18e0343a..1d8dc9ea38d5 100644 --- a/nomad/structs/diff.go +++ b/nomad/structs/diff.go @@ -62,7 +62,7 @@ func (j *Job) Diff(other *Job, contextual bool) (*JobDiff, error) { diff := &JobDiff{Type: DiffTypeNone} var oldPrimitiveFlat, newPrimitiveFlat map[string]string filter := []string{"ID", "Status", "StatusDescription", "Version", "Stable", "CreateIndex", - "ModifyIndex", "JobModifyIndex", "Update"} + "ModifyIndex", "JobModifyIndex", "Update", "SubmitTime"} if j == nil && other == nil { return diff, nil diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 5a9f7bcbf097..5cafc36e594e 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -376,10 +376,9 @@ type ApplyPlanResultsRequest struct { // scheduler. AllocUpdateRequest - // CreatedDeployment is the deployment created as a result of a scheduling - // event. Any existing deployment should be cancelled when the new - // deployment is created. - CreatedDeployment *Deployment + // Deployment is the deployment created or updated as a result of a + // scheduling event. + Deployment *Deployment // DeploymentUpdates is a set of status updates to apply to the given // deployments. This allows the scheduler to cancel any unneeded deployment @@ -4000,6 +3999,9 @@ type DeploymentState struct { // Promoted marks whether the canaries have been promoted Promoted bool + // PlacedCanaries is the set of placed canary allocations + PlacedCanaries []string + // DesiredCanaries is the number of canaries that should be created. DesiredCanaries int @@ -4020,6 +4022,7 @@ type DeploymentState struct { func (d *DeploymentState) GoString() string { base := fmt.Sprintf("Desired Total: %d", d.DesiredTotal) base += fmt.Sprintf("\nDesired Canaries: %d", d.DesiredCanaries) + base += fmt.Sprintf("\nPlaced Canaries: %#v", d.PlacedCanaries) base += fmt.Sprintf("\nPromoted: %v", d.Promoted) base += fmt.Sprintf("\nPlaced: %d", d.PlacedAllocs) base += fmt.Sprintf("\nHealthy: %d", d.HealthyAllocs) @@ -4031,6 +4034,7 @@ func (d *DeploymentState) GoString() string { func (d *DeploymentState) Copy() *DeploymentState { c := &DeploymentState{} *c = *d + c.PlacedCanaries = helper.CopySliceString(d.PlacedCanaries) return c } @@ -4797,11 +4801,9 @@ type Plan struct { // to understand the decisions made by the scheduler. Annotations *PlanAnnotations - // CreatedDeployment is the deployment created by the scheduler that should - // be applied by the planner. A created deployment will cancel all other - // deployments for a given job as there can only be a single running - // deployment. - CreatedDeployment *Deployment + // Deployment is the deployment created or updated by the scheduler that + // should be applied by the planner. + Deployment *Deployment // DeploymentUpdates is a set of status updates to apply to the given // deployments. This allows the scheduler to cancel any unneeded deployment @@ -4862,7 +4864,7 @@ func (p *Plan) AppendAlloc(alloc *Allocation) { func (p *Plan) IsNoOp() bool { return len(p.NodeUpdate) == 0 && len(p.NodeAllocation) == 0 && - p.CreatedDeployment == nil && + p.Deployment == nil && len(p.DeploymentUpdates) == 0 } diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 686f6a450559..07482573d2b2 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -396,12 +396,12 @@ func (s *GenericScheduler) computeJobAllocs() error { } // Add the deployment changes to the plan - s.plan.CreatedDeployment = results.createDeployment + s.plan.Deployment = results.deployment s.plan.DeploymentUpdates = results.deploymentUpdates // Update the stored deployment - if results.createDeployment != nil { - s.deployment = results.createDeployment + if results.deployment != nil { + s.deployment = results.deployment } // Handle the stop diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index e8df6336facf..977242c5f3e1 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -1463,10 +1463,10 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { h.AssertEvalStatus(t, structs.EvalStatusComplete) // Ensure a deployment was created - if plan.CreatedDeployment == nil { + if plan.Deployment == nil { t.Fatalf("bad: %#v", plan) } - state, ok := plan.CreatedDeployment.TaskGroups[job.TaskGroups[0].Name] + state, ok := plan.Deployment.TaskGroups[job.TaskGroups[0].Name] if !ok { t.Fatalf("bad: %#v", plan) } diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index d36904f9c4e1..6ea2b3204242 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -61,9 +61,9 @@ type allocReconciler struct { // reconcileResults contains the results of the reconciliation and should be // applied by the scheduler. type reconcileResults struct { - // createDeployment is the deployment that should be created as a result of - // scheduling - createDeployment *structs.Deployment + // deployment is the deployment that should be created or updated as a + // result of scheduling + deployment *structs.Deployment // deploymentUpdates contains a set of deployment updates that should be // applied as a result of scheduling @@ -393,7 +393,7 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // Create a new deployment if necessary if a.deployment == nil && strategy != nil && dstate.DesiredTotal != 0 { a.deployment = structs.NewDeployment(a.job) - a.result.createDeployment = a.deployment + a.result.deployment = a.deployment a.deployment.TaskGroups[group] = dstate } } diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index f3da09991163..e2dd04024b45 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -249,16 +249,16 @@ type resultExpectation struct { func assertResults(t *testing.T, r *reconcileResults, exp *resultExpectation) { - if exp.createDeployment != nil && r.createDeployment == nil { + if exp.createDeployment != nil && r.deployment == nil { t.Fatalf("Expect a created deployment got none") - } else if exp.createDeployment == nil && r.createDeployment != nil { - t.Fatalf("Expect no created deployment; got %#v", r.createDeployment) - } else if exp.createDeployment != nil && r.createDeployment != nil { + } else if exp.createDeployment == nil && r.deployment != nil { + t.Fatalf("Expect no created deployment; got %#v", r.deployment) + } else if exp.createDeployment != nil && r.deployment != nil { // Clear the deployment ID - r.createDeployment.ID, exp.createDeployment.ID = "", "" - if !reflect.DeepEqual(r.createDeployment, exp.createDeployment) { + r.deployment.ID, exp.createDeployment.ID = "", "" + if !reflect.DeepEqual(r.deployment, exp.createDeployment) { t.Fatalf("Unexpected createdDeployment; got\n %#v\nwant\n%#v\nDiff: %v", - r.createDeployment, exp.createDeployment, pretty.Diff(r.createDeployment, exp.createDeployment)) + r.deployment, exp.createDeployment, pretty.Diff(r.deployment, exp.createDeployment)) } } diff --git a/scheduler/testing.go b/scheduler/testing.go index af1411db1040..9199550290b3 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -124,7 +124,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er Job: plan.Job, Alloc: allocs, }, - CreatedDeployment: plan.CreatedDeployment, + Deployment: plan.Deployment, DeploymentUpdates: plan.DeploymentUpdates, } From c3626cd5ea0f9ec81c554e7db928c1958ee841ba Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 4 Jul 2017 14:44:03 -0700 Subject: [PATCH 067/105] vendor file --- vendor/vendor.json | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/vendor/vendor.json b/vendor/vendor.json index 6920b7b8b206..7204ee265db5 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -255,6 +255,12 @@ "revision": "7d649b46cdc2cd2ed102d350688a75a4fd7778c6", "revisionTime": "2016-11-21T13:51:53Z" }, + { + "checksumSHA1": "5qEuqnavYqfukAMm7WimC/cX0HE=", + "path": "github.com/dadgar/columnize", + "revision": "36facc107c7b0258ea8d6e96a45a46087b6d7ef9", + "revisionTime": "2017-06-30T01:36:23Z" + }, { "checksumSHA1": "/5cvgU+J4l7EhMXTK76KaCAfOuU=", "path": "github.com/davecgh/go-spew/spew", @@ -366,26 +372,26 @@ { "checksumSHA1": "iP5slJJPRZUm0rfdII8OiATAACA=", "path": "github.com/docker/docker/pkg/idtools", - "revision": "52debcd58ac91bf68503ce60561536911b74ff05", - "revisionTime": "2016-05-20T15:17:10Z" + "revision": "02caa73df411debed164f520a6a1304778f8b88c", + "revisionTime": "2016-05-28T10:48:36Z" }, { "checksumSHA1": "iP5slJJPRZUm0rfdII8OiATAACA=", "path": "github.com/docker/docker/pkg/idtools", - "revision": "02caa73df411debed164f520a6a1304778f8b88c", - "revisionTime": "2016-05-28T10:48:36Z" + "revision": "52debcd58ac91bf68503ce60561536911b74ff05", + "revisionTime": "2016-05-20T15:17:10Z" }, { "checksumSHA1": "tdhmIGUaoOMEDymMC23qTS7bt0g=", "path": "github.com/docker/docker/pkg/ioutils", - "revision": "da39e9a4f920a15683dd0f23923c302d4db6eed5", - "revisionTime": "2016-05-28T08:11:04Z" + "revision": "52debcd58ac91bf68503ce60561536911b74ff05", + "revisionTime": "2016-05-20T15:17:10Z" }, { "checksumSHA1": "tdhmIGUaoOMEDymMC23qTS7bt0g=", "path": "github.com/docker/docker/pkg/ioutils", - "revision": "52debcd58ac91bf68503ce60561536911b74ff05", - "revisionTime": "2016-05-20T15:17:10Z" + "revision": "da39e9a4f920a15683dd0f23923c302d4db6eed5", + "revisionTime": "2016-05-28T08:11:04Z" }, { "checksumSHA1": "BlFSSK7zUjPzPuxkLmM/0wpvku8=", @@ -408,14 +414,14 @@ { "checksumSHA1": "rArZ5mYIe9I1L5PRQOJu8BwafFw=", "path": "github.com/docker/docker/pkg/pools", - "revision": "da39e9a4f920a15683dd0f23923c302d4db6eed5", - "revisionTime": "2016-05-28T08:11:04Z" + "revision": "52debcd58ac91bf68503ce60561536911b74ff05", + "revisionTime": "2016-05-20T15:17:10Z" }, { "checksumSHA1": "rArZ5mYIe9I1L5PRQOJu8BwafFw=", "path": "github.com/docker/docker/pkg/pools", - "revision": "52debcd58ac91bf68503ce60561536911b74ff05", - "revisionTime": "2016-05-20T15:17:10Z" + "revision": "da39e9a4f920a15683dd0f23923c302d4db6eed5", + "revisionTime": "2016-05-28T08:11:04Z" }, { "checksumSHA1": "txf3EORYff4hO6PEvwBm2lyh1MU=", From e5b1e3171ceb1241314d0156b7f93c495422899f Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Tue, 4 Jul 2017 14:54:51 -0700 Subject: [PATCH 068/105] Remove promoted bit from allocation --- nomad/deployment_endpoint_test.go | 7 ------ nomad/fsm_test.go | 19 ---------------- nomad/state/state_store.go | 15 ------------- nomad/state/state_store_test.go | 36 ------------------------------- nomad/structs/structs.go | 13 ----------- scheduler/generic_sched_test.go | 4 ++-- scheduler/reconcile.go | 8 +++++-- scheduler/reconcile_test.go | 5 +++-- scheduler/reconcile_util.go | 17 --------------- 9 files changed, 11 insertions(+), 113 deletions(-) diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index 25a91f0d3794..995f3f9466f1 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -237,13 +237,6 @@ func TestDeploymentEndpoint_Promote(t *testing.T) { assert.Len(dout.TaskGroups, 1, "should have one group") assert.Contains(dout.TaskGroups, "web", "should have web group") assert.True(dout.TaskGroups["web"].Promoted, "web group should be promoted") - - // Lookup the allocation - aout, err := state.AllocByID(ws, a.ID) - assert.Nil(err, "AllocByID") - assert.NotNil(aout, "alloc") - assert.NotNil(aout.DeploymentStatus, "alloc deployment status") - assert.True(aout.DeploymentStatus.Promoted, "alloc deployment promoted") } func TestDeploymentEndpoint_SetAllocHealth(t *testing.T) { diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index d949df2aa80c..a4b2a0d00278 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1298,25 +1298,6 @@ func TestFSM_DeploymentPromotion(t *testing.T) { } } - // Check that the allocs were promoted - out1, err := state.AllocByID(ws, c1.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - out2, err := state.AllocByID(ws, c2.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - - for _, alloc := range []*structs.Allocation{out1, out2} { - if alloc.DeploymentStatus == nil { - t.Fatalf("bad: alloc %q has nil deployment status", alloc.ID) - } - if !alloc.DeploymentStatus.Promoted { - t.Fatalf("bad: alloc %q not promoted", alloc.ID) - } - } - // Check that the evaluation was created eout, _ := state.EvalByID(ws, e.ID) if err != nil { diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 1c4afeeda5ab..3ad6d37211bc 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -1921,7 +1921,6 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD } var unhealthyErr multierror.Error - var canaries []*structs.Allocation for { raw := iter.Next() if raw == nil { @@ -1944,26 +1943,12 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD multierror.Append(&unhealthyErr, fmt.Errorf("Canary allocation %q for group %q is not healthy", alloc.ID, alloc.TaskGroup)) continue } - - canaries = append(canaries, alloc) } if err := unhealthyErr.ErrorOrNil(); err != nil { return err } - // Mark canaries as promoted and upsert them - canariesCopy := make([]*structs.Allocation, 0, len(canaries)) - for _, c := range canaries { - copy := c.Copy() - copy.DeploymentStatus.Promoted = true - copy.DeploymentStatus.ModifyIndex = index - canariesCopy = append(canariesCopy, copy) - } - if err := s.upsertAllocsImpl(index, canariesCopy, txn); err != nil { - return err - } - // Update deployment copy := deployment.Copy() copy.ModifyIndex = index diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 41890f5ab1aa..8e9027b4caed 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -4965,25 +4965,6 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { } } - // Check that the allocs were promoted - out1, err := state.AllocByID(ws, c1.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - out2, err := state.AllocByID(ws, c2.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - - for _, alloc := range []*structs.Allocation{out1, out2} { - if alloc.DeploymentStatus == nil { - t.Fatalf("bad: alloc %q has nil deployment status", alloc.ID) - } - if !alloc.DeploymentStatus.Promoted { - t.Fatalf("bad: alloc %q not promoted", alloc.ID) - } - } - // Check that the evaluation was created eout, _ := state.EvalByID(ws, e.ID) if err != nil { @@ -5079,23 +5060,6 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { t.Fatalf("bad: task group web not promoted: %#v", stateout) } - // Check that the allocs were promoted - out1, err := state.AllocByID(ws, c1.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - out2, err := state.AllocByID(ws, c2.ID) - if err != nil { - t.Fatalf("err: %v", err) - } - - if !out1.DeploymentStatus.Promoted { - t.Fatalf("bad: alloc %q not promoted", out1.ID) - } - if out2.DeploymentStatus.Promoted { - t.Fatalf("bad: alloc %q promoted", out2.ID) - } - // Check that the evaluation was created eout, _ := state.EvalByID(ws, e.ID) if err != nil { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 5cafc36e594e..2a21667bdec7 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4423,10 +4423,6 @@ type AllocDeploymentStatus struct { // healthy or unhealthy. Healthy *bool - // Promoted marks whether the allocation is promoted. This field is only - // used if the allocation is a canary. - Promoted bool - // ModifyIndex is the raft index in which the deployment status was last // changed. ModifyIndex uint64 @@ -4452,15 +4448,6 @@ func (a *AllocDeploymentStatus) IsUnhealthy() bool { return a.Healthy != nil && !*a.Healthy } -// IsPromoted returns if the allocation is promoted as as part of a deployment -func (a *AllocDeploymentStatus) IsPromoted() bool { - if a == nil { - return false - } - - return a.Promoted -} - func (a *AllocDeploymentStatus) Copy() *AllocDeploymentStatus { if a == nil { return nil diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 977242c5f3e1..f1e43c69b606 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -2872,7 +2872,7 @@ func TestServiceSched_CancelDeployment_Stopped(t *testing.T) { d.JobID = job.ID d.JobCreateIndex = job.CreateIndex d.JobModifyIndex = job.JobModifyIndex - 1 - noErr(t, h.State.UpsertDeployment(h.NextIndex(), d, false)) + noErr(t, h.State.UpsertDeployment(h.NextIndex(), d)) // Create a mock evaluation to deregister the job eval := &structs.Evaluation{ @@ -2937,7 +2937,7 @@ func TestServiceSched_CancelDeployment_NewerJob(t *testing.T) { // Create a deployment for an old version of the job d := mock.Deployment() d.JobID = job.ID - noErr(t, h.State.UpsertDeployment(h.NextIndex(), d, false)) + noErr(t, h.State.UpsertDeployment(h.NextIndex(), d)) // Upsert again to bump job version noErr(t, h.State.UpsertJob(h.NextIndex(), job)) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 6ea2b3204242..87cd40fa0077 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -249,7 +249,9 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { if a.deployment != nil { // Stop all non-promoted canaries from older deployments current, older := canaries.filterByDeployment(a.deployment.ID) - nonPromotedOlder := older.filterByPromoted(false) + // TODO + //nonPromotedOlder := older.filterByPromoted(false) + nonPromotedOlder := older a.markStop(nonPromotedOlder, "", allocNotNeeded) desiredChanges.Stop += uint64(len(nonPromotedOlder)) @@ -264,7 +266,9 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { all = all.difference(nonPromotedOlder, migrate, lost) } else { // Stop all non-promoted canaries - nonPromoted := canaries.filterByPromoted(false) + // TODO + //nonPromoted := canaries.filterByPromoted(false) + nonPromoted := canaries a.markStop(nonPromoted, "", allocNotNeeded) desiredChanges.Stop += uint64(len(nonPromoted)) all = all.difference(nonPromoted) diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index e2dd04024b45..e293f98c73c1 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -2541,8 +2541,9 @@ func TestReconciler_CompleteDeployment(t *testing.T) { if i < 2 { alloc.Canary = true alloc.DeploymentStatus = &structs.AllocDeploymentStatus{ - Healthy: helper.BoolToPtr(true), - Promoted: true, + Healthy: helper.BoolToPtr(true), + // TODO + //Promoted: true, } } allocs = append(allocs, alloc) diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index 27e4933b8f00..35d31fc63142 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -162,23 +162,6 @@ func (a allocSet) filterByCanary() allocSet { return canaries } -// filterByPromoted filters the allocset by whether the canaries are promoted or -// not -func (a allocSet) filterByPromoted(p bool) allocSet { - promoted := make(map[string]*structs.Allocation) - for _, alloc := range a { - if !alloc.Canary { - continue - } - if p && alloc.DeploymentStatus.IsPromoted() { - promoted[alloc.ID] = alloc - } else if !p && !alloc.DeploymentStatus.IsPromoted() { - promoted[alloc.ID] = alloc - } - } - return promoted -} - // filterByDeployment filters allocations into two sets, those that match the // given deployment ID and those that don't func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { From 71c7c45cf6407b2f342d01009f953bc05cbd0d3a Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 5 Jul 2017 12:50:40 -0700 Subject: [PATCH 069/105] Change canary handling --- scheduler/generic_sched.go | 19 ++++- scheduler/reconcile.go | 133 ++++++++++++++++++++---------- scheduler/reconcile_test.go | 159 ++++++++++++++++++++++++++++++------ scheduler/reconcile_util.go | 13 +++ 4 files changed, 255 insertions(+), 69 deletions(-) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 07482573d2b2..fa0e3ff63d8f 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -410,7 +410,16 @@ func (s *GenericScheduler) computeJobAllocs() error { } // Handle the in-place updates + deploymentID := "" + if s.plan.Deployment != nil { + deploymentID = s.plan.Deployment.ID + } + for _, update := range results.inplaceUpdate { + if update.DeploymentID != deploymentID { + update.DeploymentID = deploymentID + update.DeploymentStatus = nil + } s.ctx.Plan().AppendAlloc(update) } @@ -485,7 +494,6 @@ func (s *GenericScheduler) computePlacements(place []allocPlaceResult) error { Metrics: s.ctx.Metrics(), NodeID: option.Node.ID, DeploymentID: deploymentID, - Canary: missing.canary, TaskResources: option.TaskResources, DesiredStatus: structs.AllocDesiredStatusRun, ClientStatus: structs.AllocClientStatusPending, @@ -501,6 +509,15 @@ func (s *GenericScheduler) computePlacements(place []allocPlaceResult) error { alloc.PreviousAllocation = missing.previousAlloc.ID } + // TODO test + // If we are placing a canary and we found a match, add the canary + // to the deployment state object. + if missing.canary { + if state, ok := s.deployment.TaskGroups[missing.taskGroup.Name]; ok { + state.PlacedCanaries = append(state.PlacedCanaries, alloc.ID) + } + } + s.plan.AppendAlloc(alloc) } else { // Lazy initialize the failed map diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 87cd40fa0077..a87fa35f8ebb 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -38,6 +38,9 @@ type allocReconciler struct { // being stopped so we require this seperately. jobID string + // oldDeployment is the last deployment for the job + oldDeployment *structs.Deployment + // deployment is the current deployment for the job deployment *structs.Deployment @@ -168,16 +171,31 @@ func (a *allocReconciler) cancelDeployments() { return } - // Check if the deployment is referencing an older job and cancel it - if d := a.deployment; d != nil { - if d.Active() && (d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex) { - a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ - DeploymentID: a.deployment.ID, - Status: structs.DeploymentStatusCancelled, - StatusDescription: structs.DeploymentStatusDescriptionNewerJob, - }) - a.deployment = nil - } + d := a.deployment + if d == nil { + return + } + + // Check if the deployment is active and referencing an older job and cancel it + if d.Active() && (d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex) { + a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ + DeploymentID: a.deployment.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionNewerJob, + }) + a.oldDeployment = d + a.deployment = nil + } + + // Clear it as the current deployment if it is terminal + //if !d.Active() { + //a.oldDeployment = d + //a.deployment = nil + //} + // Clear it as the current deployment if it is successful + if d.Status == structs.DeploymentStatusSuccessful { + a.oldDeployment = d + a.deployment = nil } } @@ -243,40 +261,9 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } } - // Handle stopping unneeded canaries and tracking placed canaries - canaries := all.filterByCanary() - if len(canaries) != 0 { - if a.deployment != nil { - // Stop all non-promoted canaries from older deployments - current, older := canaries.filterByDeployment(a.deployment.ID) - // TODO - //nonPromotedOlder := older.filterByPromoted(false) - nonPromotedOlder := older - a.markStop(nonPromotedOlder, "", allocNotNeeded) - desiredChanges.Stop += uint64(len(nonPromotedOlder)) - - // Handle canaries on migrating/lost nodes here by just stopping - // them - untainted, migrate, lost := current.filterByTainted(a.taintedNodes) - a.markStop(migrate, "", allocMigrating) - a.markStop(lost, structs.AllocClientStatusLost, allocLost) - canaries = untainted - - // Update the all set - all = all.difference(nonPromotedOlder, migrate, lost) - } else { - // Stop all non-promoted canaries - // TODO - //nonPromoted := canaries.filterByPromoted(false) - nonPromoted := canaries - a.markStop(nonPromoted, "", allocNotNeeded) - desiredChanges.Stop += uint64(len(nonPromoted)) - all = all.difference(nonPromoted) - canaries = nil - } - } + canaries, all := a.handleGroupCanaries(all, desiredChanges) - // Determine what set of alloations are on tainted nodes + // Determine what set of allocations are on tainted nodes untainted, migrate, lost := all.filterByTainted(a.taintedNodes) // Create a structure for choosing names. Seed with the taken names which is @@ -304,6 +291,8 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { desiredChanges.Ignore += uint64(len(ignore)) desiredChanges.InPlaceUpdate += uint64(len(inplace)) if !existingDeployment { + a.logger.Printf("inplace: %d", len(inplace)) + a.logger.Printf("destructive: %d", len(destructive)) dstate.DesiredTotal += len(destructive) + len(inplace) } @@ -340,6 +329,7 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // * If there are any canaries that they have been promoted place := a.computePlacements(tg, nameIndex, untainted, migrate) if !existingDeployment { + a.logger.Printf("place: %d", len(place)) dstate.DesiredTotal += len(place) } @@ -402,6 +392,61 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } } +// handleGroupCanaries handles the canaries for the group by stopping the +// unneeded ones and returning the current set of canaries and the updated total +// set of allocs for the group +func (a *allocReconciler) handleGroupCanaries(all allocSet, desiredChanges *structs.DesiredUpdates) (canaries, newAll allocSet) { + // Stop any canary from an older deployment or from a failed one + var stop []string + + // Cancel any non-promoted canaries from the older deployment + if a.oldDeployment != nil { + for _, s := range a.oldDeployment.TaskGroups { + if !s.Promoted { + stop = append(stop, s.PlacedCanaries...) + } + } + } + + if a.deployment != nil && a.deployment.Status == structs.DeploymentStatusFailed { + for _, s := range a.deployment.TaskGroups { + if !s.Promoted { + stop = append(stop, s.PlacedCanaries...) + } + } + } + + stopSet := all.fromKeys(stop) + a.markStop(stopSet, "", allocNotNeeded) + desiredChanges.Stop += uint64(len(stopSet)) + a.logger.Printf("canaries stopping b/c old or failed: %#v", stopSet) + all = all.difference(stopSet) + + // Capture our current set of canaries and handle any migrations that are + // needed by just stopping them. + if a.deployment != nil { + var canaryIDs []string + for _, s := range a.deployment.TaskGroups { + canaryIDs = append(canaryIDs, s.PlacedCanaries...) + } + + canaries = all.fromKeys(canaryIDs) + untainted, migrate, lost := canaries.filterByTainted(a.taintedNodes) + a.logger.Printf("canaries: %#v", canaries) + a.logger.Printf("canaries migrating: %#v", migrate) + a.logger.Printf("canaries lost %#v", lost) + + a.markStop(migrate, "", allocMigrating) + a.markStop(lost, structs.AllocClientStatusLost, allocLost) + + canaries = untainted + all = all.difference(migrate, lost) + a.logger.Printf("canaries untainted: %#v", canaries) + } + + return canaries, all +} + // computeLimit returns the placement limit for a particular group. The inputs // are the group definition, the untainted and destructive allocation set and // whether we are in a canary state. @@ -564,6 +609,8 @@ func (a *allocReconciler) computeUpdates(group *structs.TaskGroup, untainted all } else if destructiveChange { destructive[alloc.ID] = alloc } else { + // Attach the deployment ID and and clear the health if the + // deployment has changed inplace[alloc.ID] = alloc a.result.inplaceUpdate = append(a.result.inplaceUpdate, inplaceAlloc) } diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index e293f98c73c1..474a13c08ecc 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -64,6 +64,7 @@ Deployment Tests: √ Limit calculation accounts for healthy allocs on migrating/lost nodes √ Failed deployment should not place anything √ Run after canaries have been promoted, new allocs have been rolled out and there is no deployment +√ Failed deployment cancels non-promoted task groups */ var ( @@ -1403,20 +1404,23 @@ func TestReconciler_PausedOrFailedDeployment_NoMoreCanaries(t *testing.T) { cases := []struct { name string deploymentStatus string + stop uint64 }{ { name: "paused deployment", deploymentStatus: structs.DeploymentStatusPaused, + stop: 0, }, { name: "failed deployment", deploymentStatus: structs.DeploymentStatusFailed, + stop: 1, }, } for _, c := range cases { t.Run(c.name, func(t *testing.T) { - // Create a deployment that is paused and has placed some canaries + // Create a deployment that is paused/failed and has placed some canaries d := structs.NewDeployment(job) d.Status = c.deploymentStatus d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ @@ -1445,9 +1449,9 @@ func TestReconciler_PausedOrFailedDeployment_NoMoreCanaries(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, 0) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true canary.DeploymentID = d.ID allocs = append(allocs, canary) + d.TaskGroups[canary.TaskGroup].PlacedCanaries = []string{canary.ID} mockUpdateFn := allocUpdateFnMock(map[string]allocUpdateType{canary.ID: allocUpdateFnIgnore}, allocUpdateFnDestructive) reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) @@ -1459,10 +1463,11 @@ func TestReconciler_PausedOrFailedDeployment_NoMoreCanaries(t *testing.T) { deploymentUpdates: nil, place: 0, inplace: 0, - stop: 0, + stop: int(c.stop), desiredTGUpdates: map[string]*structs.DesiredUpdates{ job.TaskGroups[0].Name: { - Ignore: 11, + Ignore: 11 - c.stop, + Stop: c.stop, }, }, }) @@ -1704,12 +1709,13 @@ func TestReconciler_DrainNode_Canary(t *testing.T) { // Create a deployment that is paused and has placed some canaries d := structs.NewDeployment(job) - d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + s := &structs.DeploymentState{ Promoted: false, DesiredTotal: 10, DesiredCanaries: 2, PlacedAllocs: 2, } + d.TaskGroups[job.TaskGroups[0].Name] = s // Create 10 allocations from the old job var allocs []*structs.Allocation @@ -1733,8 +1739,8 @@ func TestReconciler_DrainNode_Canary(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true canary.DeploymentID = d.ID + s.PlacedCanaries = append(s.PlacedCanaries, canary.ID) allocs = append(allocs, canary) handled[canary.ID] = allocUpdateFnIgnore } @@ -1775,12 +1781,13 @@ func TestReconciler_LostNode_Canary(t *testing.T) { // Create a deployment that is paused and has placed some canaries d := structs.NewDeployment(job) - d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + s := &structs.DeploymentState{ Promoted: false, DesiredTotal: 10, DesiredCanaries: 2, PlacedAllocs: 2, } + d.TaskGroups[job.TaskGroups[0].Name] = s // Create 10 allocations from the old job var allocs []*structs.Allocation @@ -1804,7 +1811,7 @@ func TestReconciler_LostNode_Canary(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true + s.PlacedCanaries = append(s.PlacedCanaries, canary.ID) canary.DeploymentID = d.ID allocs = append(allocs, canary) handled[canary.ID] = allocUpdateFnIgnore @@ -1847,12 +1854,13 @@ func TestReconciler_StopOldCanaries(t *testing.T) { // Create an old deployment that has placed some canaries d := structs.NewDeployment(job) - d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + s := &structs.DeploymentState{ Promoted: false, DesiredTotal: 10, DesiredCanaries: 2, PlacedAllocs: 2, } + d.TaskGroups[job.TaskGroups[0].Name] = s // Update the job job.JobModifyIndex += 10 @@ -1878,7 +1886,7 @@ func TestReconciler_StopOldCanaries(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true + s.PlacedCanaries = append(s.PlacedCanaries, canary.ID) canary.DeploymentID = d.ID allocs = append(allocs, canary) } @@ -2070,12 +2078,13 @@ func TestReconciler_NewCanaries_FillNames(t *testing.T) { // Create an existing deployment that has placed some canaries d := structs.NewDeployment(job) - d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + s := &structs.DeploymentState{ Promoted: false, DesiredTotal: 10, DesiredCanaries: 4, PlacedAllocs: 2, } + d.TaskGroups[job.TaskGroups[0].Name] = s // Create 10 allocations from the old job var allocs []*structs.Allocation @@ -2098,7 +2107,7 @@ func TestReconciler_NewCanaries_FillNames(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true + s.PlacedCanaries = append(s.PlacedCanaries, canary.ID) canary.DeploymentID = d.ID allocs = append(allocs, canary) } @@ -2132,12 +2141,13 @@ func TestReconciler_PromoteCanaries_Unblock(t *testing.T) { // Create an existing deployment that has placed some canaries and mark them // promoted d := structs.NewDeployment(job) - d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + s := &structs.DeploymentState{ Promoted: true, DesiredTotal: 10, DesiredCanaries: 2, PlacedAllocs: 2, } + d.TaskGroups[job.TaskGroups[0].Name] = s // Create 10 allocations from the old job var allocs []*structs.Allocation @@ -2161,7 +2171,7 @@ func TestReconciler_PromoteCanaries_Unblock(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true + s.PlacedCanaries = append(s.PlacedCanaries, canary.ID) canary.DeploymentID = d.ID canary.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), @@ -2205,12 +2215,13 @@ func TestReconciler_PromoteCanaries_CanariesEqualCount(t *testing.T) { // Create an existing deployment that has placed some canaries and mark them // promoted d := structs.NewDeployment(job) - d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + s := &structs.DeploymentState{ Promoted: true, DesiredTotal: 2, DesiredCanaries: 2, PlacedAllocs: 2, } + d.TaskGroups[job.TaskGroups[0].Name] = s // Create 2 allocations from the old job var allocs []*structs.Allocation @@ -2234,7 +2245,7 @@ func TestReconciler_PromoteCanaries_CanariesEqualCount(t *testing.T) { canary.NodeID = structs.GenerateUUID() canary.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) canary.TaskGroup = job.TaskGroups[0].Name - canary.Canary = true + s.PlacedCanaries = append(s.PlacedCanaries, canary.ID) canary.DeploymentID = d.ID canary.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), @@ -2527,8 +2538,17 @@ func TestReconciler_CompleteDeployment(t *testing.T) { job := mock.Job() job.TaskGroups[0].Update = canaryUpdate + d := structs.NewDeployment(job) + d.Status = structs.DeploymentStatusSuccessful + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 10, + HealthyAllocs: 10, + } + // Create allocations from the old job - dID := structs.GenerateUUID() var allocs []*structs.Allocation for i := 0; i < 10; i++ { alloc := mock.Alloc() @@ -2537,14 +2557,9 @@ func TestReconciler_CompleteDeployment(t *testing.T) { alloc.NodeID = structs.GenerateUUID() alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) alloc.TaskGroup = job.TaskGroups[0].Name - alloc.DeploymentID = dID - if i < 2 { - alloc.Canary = true - alloc.DeploymentStatus = &structs.AllocDeploymentStatus{ - Healthy: helper.BoolToPtr(true), - // TODO - //Promoted: true, - } + alloc.DeploymentID = d.ID + alloc.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), } allocs = append(allocs, alloc) } @@ -2566,3 +2581,97 @@ func TestReconciler_CompleteDeployment(t *testing.T) { }, }) } + +// Test that a failed deployment cancels non-promoted canaries +func TestReconciler_FailedDeployment_CancelCanaries(t *testing.T) { + // Create a job with two task groups + job := mock.Job() + job.TaskGroups[0].Update = canaryUpdate + job.TaskGroups = append(job.TaskGroups, job.TaskGroups[0].Copy()) + job.TaskGroups[1].Name = "two" + + // Create an existing failed deployment that has promoted one task group + d := structs.NewDeployment(job) + d.Status = structs.DeploymentStatusFailed + s0 := &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 4, + } + s1 := &structs.DeploymentState{ + Promoted: false, + DesiredTotal: 10, + DesiredCanaries: 2, + PlacedAllocs: 2, + } + d.TaskGroups[job.TaskGroups[0].Name] = s0 + d.TaskGroups[job.TaskGroups[1].Name] = s1 + + // Create 6 allocations from the old job + var allocs []*structs.Allocation + handled := make(map[string]allocUpdateType) + for _, group := range []int{0, 1} { + replacements := 4 + state := s0 + if group == 1 { + replacements = 2 + state = s1 + } + + // Create the healthy replacements + for i := 0; i < replacements; i++ { + new := mock.Alloc() + new.Job = job + new.JobID = job.ID + new.NodeID = structs.GenerateUUID() + new.Name = structs.AllocName(job.ID, job.TaskGroups[group].Name, uint(i)) + new.TaskGroup = job.TaskGroups[group].Name + new.DeploymentID = d.ID + new.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, new) + handled[new.ID] = allocUpdateFnIgnore + + // Add the alloc to the canary list + if i < 2 { + + state.PlacedCanaries = append(state.PlacedCanaries, new.ID) + } + } + for i := replacements; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[group].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[group].Name + allocs = append(allocs, alloc) + } + } + + mockUpdateFn := allocUpdateFnMock(handled, allocUpdateFnDestructive) + reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 0, + inplace: 0, + stop: 2, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + }, + job.TaskGroups[1].Name: { + Stop: 2, + Ignore: 8, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) +} diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index 35d31fc63142..b67895442694 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -116,6 +116,19 @@ func (a allocSet) union(others ...allocSet) allocSet { return union } +// fromKeys returns an alloc set matching the passed keys +func (a allocSet) fromKeys(keys ...[]string) allocSet { + from := make(map[string]*structs.Allocation) + for _, set := range keys { + for _, k := range set { + if alloc, ok := a[k]; ok { + from[k] = alloc + } + } + } + return from +} + // fitlerByTainted takes a set of tainted nodes and filters the allocation set // into three groups: // 1. Those that exist on untainted nodes From 0c6a1c0ee83392e3849c400a29f2be0538157ea9 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 5 Jul 2017 12:55:51 -0700 Subject: [PATCH 070/105] Mark complete --- nomad/structs/structs.go | 21 ++++++---- scheduler/generic_sched.go | 1 + scheduler/reconcile.go | 77 +++++++++++++++++++++++++++---------- scheduler/reconcile_test.go | 3 +- 4 files changed, 73 insertions(+), 29 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 2a21667bdec7..1ffec565de5f 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3896,6 +3896,7 @@ const ( // deployment can be in. DeploymentStatusDescriptionRunning = "Deployment is running" DeploymentStatusDescriptionPaused = "Deployment is paused" + DeploymentStatusDescriptionSuccessful = "Deployment completed successfully" DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" DeploymentStatusDescriptionFailedAllocations = "Failed due to unhealthy allocations" @@ -3958,6 +3959,10 @@ func NewDeployment(job *Job) *Deployment { } func (d *Deployment) Copy() *Deployment { + if d == nil { + return nil + } + c := &Deployment{} *c = *d @@ -4020,14 +4025,14 @@ type DeploymentState struct { } func (d *DeploymentState) GoString() string { - base := fmt.Sprintf("Desired Total: %d", d.DesiredTotal) - base += fmt.Sprintf("\nDesired Canaries: %d", d.DesiredCanaries) - base += fmt.Sprintf("\nPlaced Canaries: %#v", d.PlacedCanaries) - base += fmt.Sprintf("\nPromoted: %v", d.Promoted) - base += fmt.Sprintf("\nPlaced: %d", d.PlacedAllocs) - base += fmt.Sprintf("\nHealthy: %d", d.HealthyAllocs) - base += fmt.Sprintf("\nUnhealthy: %d", d.UnhealthyAllocs) - base += fmt.Sprintf("\nAutoRevert: %v", d.AutoRevert) + base := fmt.Sprintf("\tDesired Total: %d", d.DesiredTotal) + base += fmt.Sprintf("\n\tDesired Canaries: %d", d.DesiredCanaries) + base += fmt.Sprintf("\n\tPlaced Canaries: %#v", d.PlacedCanaries) + base += fmt.Sprintf("\n\tPromoted: %v", d.Promoted) + base += fmt.Sprintf("\n\tPlaced: %d", d.PlacedAllocs) + base += fmt.Sprintf("\n\tHealthy: %d", d.HealthyAllocs) + base += fmt.Sprintf("\n\tUnhealthy: %d", d.UnhealthyAllocs) + base += fmt.Sprintf("\n\tAutoRevert: %v", d.AutoRevert) return base } diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index fa0e3ff63d8f..a6bc81e3422d 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -409,6 +409,7 @@ func (s *GenericScheduler) computeJobAllocs() error { s.plan.AppendUpdate(stop.alloc, structs.AllocDesiredStatusStop, stop.statusDescription, stop.clientStatus) } + // TODO test // Handle the in-place updates deploymentID := "" if s.plan.Deployment != nil { diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index a87fa35f8ebb..bd565d41e6a5 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -102,6 +102,11 @@ type allocStopResult struct { statusDescription string } +// Changes returns the number of total changes +func (r *reconcileResults) Changes() int { + return len(r.place) + len(r.inplaceUpdate) + len(r.stop) +} + // NewAllocReconciler creates a new reconciler that should be used to determine // the changes required to bring the cluster state inline with the declared jobspec func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch bool, @@ -114,7 +119,7 @@ func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch batch: batch, jobID: jobID, job: job, - deployment: deployment, + deployment: deployment.Copy(), existingAllocs: existingAllocs, taintedNodes: taintedNodes, result: &reconcileResults{ @@ -148,8 +153,16 @@ func (a *allocReconciler) Compute() *reconcileResults { } // Reconcile each group + complete := true for group, as := range m { - a.computeGroup(group, as) + groupComplete := a.computeGroup(group, as) + complete = complete && groupComplete + } + + // Mark the deployment as complete if possible + if a.deployment != nil && complete { + a.deployment.Status = structs.DeploymentStatusSuccessful + a.deployment.StatusDescription = structs.DeploymentStatusDescriptionSuccessful } return a.result @@ -171,6 +184,8 @@ func (a *allocReconciler) cancelDeployments() { return } + // TODO it doesn't like operating on a failed deployment + // Write a test d := a.deployment if d == nil { return @@ -183,15 +198,11 @@ func (a *allocReconciler) cancelDeployments() { Status: structs.DeploymentStatusCancelled, StatusDescription: structs.DeploymentStatusDescriptionNewerJob, }) + a.oldDeployment = d a.deployment = nil } - // Clear it as the current deployment if it is terminal - //if !d.Active() { - //a.oldDeployment = d - //a.deployment = nil - //} // Clear it as the current deployment if it is successful if d.Status == structs.DeploymentStatusSuccessful { a.oldDeployment = d @@ -224,8 +235,9 @@ func (a *allocReconciler) markStop(allocs allocSet, clientStatus, statusDescript } } -// computeGroup reconciles state for a particular task group. -func (a *allocReconciler) computeGroup(group string, all allocSet) { +// computeGroup reconciles state for a particular task group. It returns whether +// the deployment it is for is complete with regards to the task group. +func (a *allocReconciler) computeGroup(group string, all allocSet) bool { // Create the desired update object for the group desiredChanges := new(structs.DesiredUpdates) a.result.desiredTGUpdates[group] = desiredChanges @@ -242,12 +254,13 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { a.markStop(migrate, "", allocNotNeeded) a.markStop(lost, structs.AllocClientStatusLost, allocLost) desiredChanges.Stop = uint64(len(untainted) + len(migrate) + len(lost)) - return + return true } // Get the deployment state for the group var dstate *structs.DeploymentState var existingDeployment bool + deploymentComplete := true if a.deployment != nil { dstate, existingDeployment = a.deployment.TaskGroups[group] } @@ -290,10 +303,12 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { ignore, inplace, destructive := a.computeUpdates(tg, untainted) desiredChanges.Ignore += uint64(len(ignore)) desiredChanges.InPlaceUpdate += uint64(len(inplace)) + changes := len(destructive) + len(inplace) if !existingDeployment { - a.logger.Printf("inplace: %d", len(inplace)) - a.logger.Printf("destructive: %d", len(destructive)) - dstate.DesiredTotal += len(destructive) + len(inplace) + dstate.DesiredTotal += changes + } + if changes != 0 { + deploymentComplete = false } // The fact that we have destructive updates and have less canaries than is @@ -319,6 +334,11 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { } } + // As long as we require canaries we aren't complete + if requireCanary { + deploymentComplete = false + } + // Determine how many we can place canaryState = dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted limit := a.computeLimit(tg, untainted, destructive, canaryState) @@ -329,7 +349,6 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { // * If there are any canaries that they have been promoted place := a.computePlacements(tg, nameIndex, untainted, migrate) if !existingDeployment { - a.logger.Printf("place: %d", len(place)) dstate.DesiredTotal += len(place) } @@ -360,6 +379,11 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { desiredChanges.Ignore += uint64(len(destructive)) } + // As long as we require placements we aren't complete + if len(place) != 0 { + deploymentComplete = false + } + // TODO Migrations should be done using a stagger and max_parallel. if !a.deploymentFailed { desiredChanges.Migrate += uint64(len(migrate)) @@ -367,6 +391,11 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { desiredChanges.Stop += uint64(len(migrate)) } + // As long as we require migrations we aren't complete + if len(migrate) != 0 { + deploymentComplete = false + } + for _, alloc := range migrate { a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, @@ -390,6 +419,20 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) { a.result.deployment = a.deployment a.deployment.TaskGroups[group] = dstate } + + // Final check to see if the deployment is complete is to ensure everything + // is healthy + if deploymentComplete && a.deployment != nil { + partOf, _ := untainted.filterByDeployment(a.deployment.ID) + for _, alloc := range partOf { + if !alloc.DeploymentStatus.IsHealthy() { + deploymentComplete = false + break + } + } + } + + return deploymentComplete } // handleGroupCanaries handles the canaries for the group by stopping the @@ -419,7 +462,6 @@ func (a *allocReconciler) handleGroupCanaries(all allocSet, desiredChanges *stru stopSet := all.fromKeys(stop) a.markStop(stopSet, "", allocNotNeeded) desiredChanges.Stop += uint64(len(stopSet)) - a.logger.Printf("canaries stopping b/c old or failed: %#v", stopSet) all = all.difference(stopSet) // Capture our current set of canaries and handle any migrations that are @@ -432,16 +474,11 @@ func (a *allocReconciler) handleGroupCanaries(all allocSet, desiredChanges *stru canaries = all.fromKeys(canaryIDs) untainted, migrate, lost := canaries.filterByTainted(a.taintedNodes) - a.logger.Printf("canaries: %#v", canaries) - a.logger.Printf("canaries migrating: %#v", migrate) - a.logger.Printf("canaries lost %#v", lost) - a.markStop(migrate, "", allocMigrating) a.markStop(lost, structs.AllocClientStatusLost, allocLost) canaries = untainted all = all.difference(migrate, lost) - a.logger.Printf("canaries untainted: %#v", canaries) } return canaries, all diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 474a13c08ecc..9f3194c281ec 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -2636,7 +2636,6 @@ func TestReconciler_FailedDeployment_CancelCanaries(t *testing.T) { // Add the alloc to the canary list if i < 2 { - state.PlacedCanaries = append(state.PlacedCanaries, new.ID) } } @@ -2675,3 +2674,5 @@ func TestReconciler_FailedDeployment_CancelCanaries(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) } + +// TODO Test that a failed deployment and updated job works From 29e31af007ff6573e938e3afe5d15ba4b139ffa6 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 5 Jul 2017 17:13:45 -0700 Subject: [PATCH 071/105] Attach eval id --- nomad/structs/structs.go | 8 ++++++++ scheduler/generic_sched.go | 17 +++++++---------- scheduler/generic_sched_test.go | 5 +++++ scheduler/reconcile.go | 14 ++++++++------ scheduler/system_sched.go | 6 +++--- scheduler/util.go | 3 ++- scheduler/util_test.go | 25 ++++++++++++++++++++----- 7 files changed, 53 insertions(+), 25 deletions(-) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 1ffec565de5f..2d86afb5c9ac 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3987,6 +3987,14 @@ func (d *Deployment) Active() bool { } } +// GetID is a helper for getting the ID when the object may be nil +func (d *Deployment) GetID() string { + if d == nil { + return "" + } + return d.ID +} + func (d *Deployment) GoString() string { base := fmt.Sprintf("Deployment ID %q for job %q has status %q (%v):", d.ID, d.JobID, d.Status, d.StatusDescription) for group, state := range d.TaskGroups { diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index a6bc81e3422d..e1aa47b0b733 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -117,7 +117,8 @@ func (s *GenericScheduler) Process(eval *structs.Evaluation) error { desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", eval.TriggeredBy) return setStatus(s.logger, s.planner, s.eval, s.nextEval, s.blocked, - s.failedTGAllocs, structs.EvalStatusFailed, desc, s.queuedAllocs) + s.failedTGAllocs, structs.EvalStatusFailed, desc, s.queuedAllocs, + s.deployment.GetID()) } // Retry up to the maxScheduleAttempts and reset if progress is made. @@ -136,7 +137,7 @@ func (s *GenericScheduler) Process(eval *structs.Evaluation) error { } if err := setStatus(s.logger, s.planner, s.eval, s.nextEval, s.blocked, s.failedTGAllocs, statusErr.EvalStatus, err.Error(), - s.queuedAllocs); err != nil { + s.queuedAllocs, s.deployment.GetID()); err != nil { mErr.Errors = append(mErr.Errors, err) } return mErr.ErrorOrNil() @@ -156,7 +157,8 @@ func (s *GenericScheduler) Process(eval *structs.Evaluation) error { // Update the status to complete return setStatus(s.logger, s.planner, s.eval, s.nextEval, s.blocked, - s.failedTGAllocs, structs.EvalStatusComplete, "", s.queuedAllocs) + s.failedTGAllocs, structs.EvalStatusComplete, "", s.queuedAllocs, + s.deployment.GetID()) } // createBlockedEval creates a blocked eval and submits it to the planner. If @@ -411,14 +413,9 @@ func (s *GenericScheduler) computeJobAllocs() error { // TODO test // Handle the in-place updates - deploymentID := "" - if s.plan.Deployment != nil { - deploymentID = s.plan.Deployment.ID - } - for _, update := range results.inplaceUpdate { - if update.DeploymentID != deploymentID { - update.DeploymentID = deploymentID + if update.DeploymentID != s.deployment.GetID() { + update.DeploymentID = s.deployment.GetID() update.DeploymentStatus = nil } s.ctx.Plan().AppendAlloc(update) diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index f1e43c69b606..1d413330ca5d 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -1462,6 +1462,11 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { h.AssertEvalStatus(t, structs.EvalStatusComplete) + // Check that the deployment id is attached to the eval + if h.Evals[0].DeploymentID == "" { + t.Fatalf("Eval not annotated with deployment id") + } + // Ensure a deployment was created if plan.Deployment == nil { t.Fatalf("bad: %#v", plan) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index bd565d41e6a5..06f224e5a40b 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -192,12 +192,14 @@ func (a *allocReconciler) cancelDeployments() { } // Check if the deployment is active and referencing an older job and cancel it - if d.Active() && (d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex) { - a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ - DeploymentID: a.deployment.ID, - Status: structs.DeploymentStatusCancelled, - StatusDescription: structs.DeploymentStatusDescriptionNewerJob, - }) + if d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex { + if d.Active() { + a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ + DeploymentID: a.deployment.ID, + Status: structs.DeploymentStatusCancelled, + StatusDescription: structs.DeploymentStatusDescriptionNewerJob, + }) + } a.oldDeployment = d a.deployment = nil diff --git a/scheduler/system_sched.go b/scheduler/system_sched.go index 248deb0840ca..e3d48760c881 100644 --- a/scheduler/system_sched.go +++ b/scheduler/system_sched.go @@ -66,7 +66,7 @@ func (s *SystemScheduler) Process(eval *structs.Evaluation) error { desc := fmt.Sprintf("scheduler cannot handle '%s' evaluation reason", eval.TriggeredBy) return setStatus(s.logger, s.planner, s.eval, s.nextEval, nil, s.failedTGAllocs, structs.EvalStatusFailed, desc, - s.queuedAllocs) + s.queuedAllocs, "") } // Retry up to the maxSystemScheduleAttempts and reset if progress is made. @@ -74,14 +74,14 @@ func (s *SystemScheduler) Process(eval *structs.Evaluation) error { if err := retryMax(maxSystemScheduleAttempts, s.process, progress); err != nil { if statusErr, ok := err.(*SetStatusError); ok { return setStatus(s.logger, s.planner, s.eval, s.nextEval, nil, s.failedTGAllocs, statusErr.EvalStatus, err.Error(), - s.queuedAllocs) + s.queuedAllocs, "") } return err } // Update the status to complete return setStatus(s.logger, s.planner, s.eval, s.nextEval, nil, s.failedTGAllocs, structs.EvalStatusComplete, "", - s.queuedAllocs) + s.queuedAllocs, "") } // process is wrapped in retryMax to iteratively run the handler until we have no diff --git a/scheduler/util.go b/scheduler/util.go index 9a99d0d60091..3fcaa6893d85 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -430,12 +430,13 @@ func networkPortMap(n *structs.NetworkResource) map[string]int { func setStatus(logger *log.Logger, planner Planner, eval, nextEval, spawnedBlocked *structs.Evaluation, tgMetrics map[string]*structs.AllocMetric, status, desc string, - queuedAllocs map[string]int) error { + queuedAllocs map[string]int, deploymentID string) error { logger.Printf("[DEBUG] sched: %#v: setting status to %s", eval, status) newEval := eval.Copy() newEval.Status = status newEval.StatusDescription = desc + newEval.DeploymentID = deploymentID newEval.FailedTGAllocs = tgMetrics if nextEval != nil { newEval.NextEval = nextEval.ID diff --git a/scheduler/util_test.go b/scheduler/util_test.go index 6463e20556c8..204d10602608 100644 --- a/scheduler/util_test.go +++ b/scheduler/util_test.go @@ -627,7 +627,7 @@ func TestSetStatus(t *testing.T) { eval := mock.Eval() status := "a" desc := "b" - if err := setStatus(logger, h, eval, nil, nil, nil, status, desc, nil); err != nil { + if err := setStatus(logger, h, eval, nil, nil, nil, status, desc, nil, ""); err != nil { t.Fatalf("setStatus() failed: %v", err) } @@ -643,7 +643,7 @@ func TestSetStatus(t *testing.T) { // Test next evals h = NewHarness(t) next := mock.Eval() - if err := setStatus(logger, h, eval, next, nil, nil, status, desc, nil); err != nil { + if err := setStatus(logger, h, eval, next, nil, nil, status, desc, nil, ""); err != nil { t.Fatalf("setStatus() failed: %v", err) } @@ -659,7 +659,7 @@ func TestSetStatus(t *testing.T) { // Test blocked evals h = NewHarness(t) blocked := mock.Eval() - if err := setStatus(logger, h, eval, nil, blocked, nil, status, desc, nil); err != nil { + if err := setStatus(logger, h, eval, nil, blocked, nil, status, desc, nil, ""); err != nil { t.Fatalf("setStatus() failed: %v", err) } @@ -675,7 +675,7 @@ func TestSetStatus(t *testing.T) { // Test metrics h = NewHarness(t) metrics := map[string]*structs.AllocMetric{"foo": nil} - if err := setStatus(logger, h, eval, nil, nil, metrics, status, desc, nil); err != nil { + if err := setStatus(logger, h, eval, nil, nil, metrics, status, desc, nil, ""); err != nil { t.Fatalf("setStatus() failed: %v", err) } @@ -692,7 +692,7 @@ func TestSetStatus(t *testing.T) { h = NewHarness(t) queuedAllocs := map[string]int{"web": 1} - if err := setStatus(logger, h, eval, nil, nil, metrics, status, desc, queuedAllocs); err != nil { + if err := setStatus(logger, h, eval, nil, nil, metrics, status, desc, queuedAllocs, ""); err != nil { t.Fatalf("setStatus() failed: %v", err) } @@ -704,6 +704,21 @@ func TestSetStatus(t *testing.T) { if !reflect.DeepEqual(newEval.QueuedAllocations, queuedAllocs) { t.Fatalf("setStatus() didn't set failed task group metrics correctly: %v", newEval) } + + h = NewHarness(t) + dID := structs.GenerateUUID() + if err := setStatus(logger, h, eval, nil, nil, metrics, status, desc, queuedAllocs, dID); err != nil { + t.Fatalf("setStatus() failed: %v", err) + } + + if len(h.Evals) != 1 { + t.Fatalf("setStatus() didn't update plan: %v", h.Evals) + } + + newEval = h.Evals[0] + if newEval.DeploymentID != dID { + t.Fatalf("setStatus() didn't set deployment id correctly: %v", newEval) + } } func TestInplaceUpdate_ChangedTaskGroup(t *testing.T) { From 598748d6d5eab30e9f86736403ce01dd7474987f Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 5 Jul 2017 19:46:57 -0700 Subject: [PATCH 072/105] Fix handling of failed job --- scheduler/reconcile.go | 18 ++++------ scheduler/reconcile_test.go | 72 ++++++++++++++++++++++++++++++++++++- 2 files changed, 78 insertions(+), 12 deletions(-) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 06f224e5a40b..f44d09b8f874 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -113,7 +113,7 @@ func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch jobID string, job *structs.Job, deployment *structs.Deployment, existingAllocs []*structs.Allocation, taintedNodes map[string]*structs.Node) *allocReconciler { - a := &allocReconciler{ + return &allocReconciler{ logger: logger, allocUpdateFn: allocUpdateFn, batch: batch, @@ -126,14 +126,6 @@ func NewAllocReconciler(logger *log.Logger, allocUpdateFn allocUpdateType, batch desiredTGUpdates: make(map[string]*structs.DesiredUpdates), }, } - - // Detect if the deployment is paused - if deployment != nil { - a.deploymentPaused = deployment.Status == structs.DeploymentStatusPaused - a.deploymentFailed = deployment.Status == structs.DeploymentStatusFailed - } - - return a } // Compute reconciles the existing cluster state and returns the set of changes @@ -152,6 +144,12 @@ func (a *allocReconciler) Compute() *reconcileResults { return a.result } + // Detect if the deployment is paused + if a.deployment != nil { + a.deploymentPaused = a.deployment.Status == structs.DeploymentStatusPaused + a.deploymentFailed = a.deployment.Status == structs.DeploymentStatusFailed + } + // Reconcile each group complete := true for group, as := range m { @@ -184,8 +182,6 @@ func (a *allocReconciler) cancelDeployments() { return } - // TODO it doesn't like operating on a failed deployment - // Write a test d := a.deployment if d == nil { return diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 9f3194c281ec..e787987b4f3e 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -2675,4 +2675,74 @@ func TestReconciler_FailedDeployment_CancelCanaries(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) } -// TODO Test that a failed deployment and updated job works +// Test that a failed deployment and updated job works +func TestReconciler_FailedDeployment_NewJob(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + // Create an existing failed deployment that has some placed allocs + d := structs.NewDeployment(job) + d.Status = structs.DeploymentStatusFailed + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + PlacedAllocs: 4, + } + + // Create 6 allocations from the old job + var allocs []*structs.Allocation + for i := 4; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + allocs = append(allocs, alloc) + } + + // Create the healthy replacements + for i := 0; i < 4; i++ { + new := mock.Alloc() + new.Job = job + new.JobID = job.ID + new.NodeID = structs.GenerateUUID() + new.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + new.TaskGroup = job.TaskGroups[0].Name + new.DeploymentID = d.ID + new.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, new) + } + + // Up the job version + jobNew := job.Copy() + jobNew.JobModifyIndex += 100 + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, jobNew, d, allocs, nil) + r := reconciler.Compute() + + dnew := structs.NewDeployment(job) + dnew.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + DesiredTotal: 10, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: dnew, + deploymentUpdates: nil, + place: 4, + inplace: 0, + stop: 4, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + DestructiveUpdate: 4, + Ignore: 6, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 3), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 3), placeResultsToNames(r.place)) +} From 690fc7809195d7837abbef45e3f71a5b9522a7fa Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Wed, 5 Jul 2017 21:26:04 -0700 Subject: [PATCH 073/105] Plan apply handles canaries and success is set via update --- nomad/plan_apply.go | 56 +++++++++++++++++++++++++++++++++++++--- nomad/structs/structs.go | 22 +++++++++++++++- scheduler/reconcile.go | 10 ++++--- scheduler/util.go | 3 ++- 4 files changed, 82 insertions(+), 9 deletions(-) diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index cb5c5eb10388..a300126f41fd 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -132,8 +132,8 @@ func (s *Server) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap Job: plan.Job, Alloc: make([]*structs.Allocation, 0, minUpdates), }, - Deployment: plan.Deployment, - DeploymentUpdates: plan.DeploymentUpdates, + Deployment: result.Deployment, + DeploymentUpdates: result.DeploymentUpdates, } for _, updateList := range result.NodeUpdate { req.Alloc = append(req.Alloc, updateList...) @@ -201,8 +201,10 @@ func evaluatePlan(pool *EvaluatePool, snap *state.StateSnapshot, plan *structs.P // Create a result holder for the plan result := &structs.PlanResult{ - NodeUpdate: make(map[string][]*structs.Allocation), - NodeAllocation: make(map[string][]*structs.Allocation), + NodeUpdate: make(map[string][]*structs.Allocation), + NodeAllocation: make(map[string][]*structs.Allocation), + Deployment: plan.Deployment.Copy(), + DeploymentUpdates: plan.DeploymentUpdates, } // Collect all the nodeIDs @@ -242,6 +244,8 @@ func evaluatePlan(pool *EvaluatePool, snap *state.StateSnapshot, plan *structs.P if plan.AllAtOnce { result.NodeUpdate = nil result.NodeAllocation = nil + result.DeploymentUpdates = nil + result.Deployment = nil return true } @@ -315,10 +319,54 @@ OUTER: err := fmt.Errorf("partialCommit with RefreshIndex of 0 (%d node, %d alloc)", nodeIndex, allocIndex) mErr.Errors = append(mErr.Errors, err) } + + // If there was a partial commit and we are operating within a + // deployment correct for any canary that may have been desired to be + // placed but wasn't actually placed + correctDeploymentCanaries(result) } return result, mErr.ErrorOrNil() } +// TODO test +// correctDeploymentCanaries ensures that the deployment object doesn't list any +// canaries as placed if they didn't actually get placed. This could happen if +// the plan had a partial commit. +func correctDeploymentCanaries(result *structs.PlanResult) { + // Hot path + if result.Deployment == nil || !result.Deployment.HasPlacedCanaries() { + return + } + + // Build a set of all the allocations IDs that were placed + placedAllocs := make(map[string]struct{}, len(result.NodeAllocation)) + for _, placed := range result.NodeAllocation { + for _, alloc := range placed { + placedAllocs[alloc.ID] = struct{}{} + } + } + + // Go through all the canaries and ensure that the result list only contains + // those that have been placed + for _, group := range result.Deployment.TaskGroups { + canaries := group.PlacedCanaries + if len(canaries) == 0 { + continue + } + + // Prune the canaries in place to avoid allocating an extra slice + i := 0 + for _, canaryID := range canaries { + if _, ok := placedAllocs[canaryID]; ok { + canaries[i] = canaryID + i++ + } + } + + group.PlacedCanaries = canaries[:i] + } +} + // evaluateNodePlan is used to evalute the plan for a single node, // returning if the plan is valid or if an error is encountered func evaluateNodePlan(snap *state.StateSnapshot, plan *structs.Plan, nodeID string) (bool, error) { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 2d86afb5c9ac..d1b91e71c365 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3995,6 +3995,19 @@ func (d *Deployment) GetID() string { return d.ID } +// HasPlacedCanaries returns whether the deployment has placed canaries +func (d *Deployment) HasPlacedCanaries() bool { + if d == nil || len(d.TaskGroups) == 0 { + return false + } + for _, group := range d.TaskGroups { + if len(group.PlacedCanaries) != 0 { + return true + } + } + return false +} + func (d *Deployment) GoString() string { base := fmt.Sprintf("Deployment ID %q for job %q has status %q (%v):", d.ID, d.JobID, d.Status, d.StatusDescription) for group, state := range d.TaskGroups { @@ -4876,6 +4889,12 @@ type PlanResult struct { // NodeAllocation contains all the allocations that were committed. NodeAllocation map[string][]*Allocation + // Deployment is the deployment that was committed. + Deployment *Deployment + + // DeploymentUpdates is the set of deployment updates that were commited. + DeploymentUpdates []*DeploymentStatusUpdate + // 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, @@ -4889,7 +4908,8 @@ type PlanResult struct { // IsNoOp checks if this plan result would do nothing func (p *PlanResult) IsNoOp() bool { - return len(p.NodeUpdate) == 0 && len(p.NodeAllocation) == 0 + return len(p.NodeUpdate) == 0 && len(p.NodeAllocation) == 0 && + len(p.DeploymentUpdates) == 0 && p.Deployment == nil } // FullCommit is used to check if all the allocations in a plan diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index f44d09b8f874..6c05d646379f 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -157,10 +157,14 @@ func (a *allocReconciler) Compute() *reconcileResults { complete = complete && groupComplete } + // TODO test // Mark the deployment as complete if possible if a.deployment != nil && complete { - a.deployment.Status = structs.DeploymentStatusSuccessful - a.deployment.StatusDescription = structs.DeploymentStatusDescriptionSuccessful + a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ + DeploymentID: a.deployment.ID, + Status: structs.DeploymentStatusSuccessful, + StatusDescription: structs.DeploymentStatusDescriptionSuccessful, + }) } return a.result @@ -257,7 +261,7 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { // Get the deployment state for the group var dstate *structs.DeploymentState - var existingDeployment bool + existingDeployment := false deploymentComplete := true if a.deployment != nil { dstate, existingDeployment = a.deployment.TaskGroups[group] diff --git a/scheduler/util.go b/scheduler/util.go index 3fcaa6893d85..49368981aee4 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -290,7 +290,8 @@ func retryMax(max int, cb func() (bool, error), reset func() bool) error { // If the result is nil, false is returned. func progressMade(result *structs.PlanResult) bool { return result != nil && (len(result.NodeUpdate) != 0 || - len(result.NodeAllocation) != 0) + len(result.NodeAllocation) != 0) || result.Deployment != nil || + len(result.DeploymentUpdates) != 0 } // taintedNodes is used to scan the allocations and then check if the From 463d20e4f31711aa17c6b9591f08196ba8531cf4 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 08:39:16 -0700 Subject: [PATCH 074/105] Test marking as complete --- scheduler/reconcile.go | 1 - scheduler/reconcile_test.go | 69 +++++++++++++++++++++++++++++++++++-- 2 files changed, 67 insertions(+), 3 deletions(-) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 6c05d646379f..2bc6eaf2280d 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -157,7 +157,6 @@ func (a *allocReconciler) Compute() *reconcileResults { complete = complete && groupComplete } - // TODO test // Mark the deployment as complete if possible if a.deployment != nil && complete { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index e787987b4f3e..5cd5d6bf9d23 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -65,6 +65,8 @@ Deployment Tests: √ Failed deployment should not place anything √ Run after canaries have been promoted, new allocs have been rolled out and there is no deployment √ Failed deployment cancels non-promoted task groups +√ Failed deployment and updated job works +√ Finished deployment gets marked as complete */ var ( @@ -2258,10 +2260,18 @@ func TestReconciler_PromoteCanaries_CanariesEqualCount(t *testing.T) { reconciler := NewAllocReconciler(testLogger(), mockUpdateFn, false, job.ID, job, d, allocs, nil) r := reconciler.Compute() + updates := []*structs.DeploymentStatusUpdate{ + { + DeploymentID: d.ID, + Status: structs.DeploymentStatusSuccessful, + StatusDescription: structs.DeploymentStatusDescriptionSuccessful, + }, + } + // Assert the correct results assertResults(t, r, &resultExpectation{ createDeployment: nil, - deploymentUpdates: nil, + deploymentUpdates: updates, place: 0, inplace: 0, stop: 2, @@ -2564,7 +2574,7 @@ func TestReconciler_CompleteDeployment(t *testing.T) { allocs = append(allocs, alloc) } - reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, nil) + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, d, allocs, nil) r := reconciler.Compute() // Assert the correct results @@ -2746,3 +2756,58 @@ func TestReconciler_FailedDeployment_NewJob(t *testing.T) { assertNamesHaveIndexes(t, intRange(0, 3), stopResultsToNames(r.stop)) assertNamesHaveIndexes(t, intRange(0, 3), placeResultsToNames(r.place)) } + +// Tests the reconciler marks a deployment as complete +func TestReconciler_MarkDeploymentComplete(t *testing.T) { + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + + d := structs.NewDeployment(job) + d.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ + Promoted: true, + DesiredTotal: 10, + PlacedAllocs: 10, + HealthyAllocs: 10, + } + + // Create allocations from the old job + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[0].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[0].Name + alloc.DeploymentID = d.ID + alloc.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + allocs = append(allocs, alloc) + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, d, allocs, nil) + r := reconciler.Compute() + + updates := []*structs.DeploymentStatusUpdate{ + { + DeploymentID: d.ID, + Status: structs.DeploymentStatusSuccessful, + StatusDescription: structs.DeploymentStatusDescriptionSuccessful, + }, + } + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: updates, + place: 0, + inplace: 0, + stop: 0, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Ignore: 10, + }, + }, + }) +} From 127d43db88a0e35e450f919a3e5a11822861c16a Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 08:58:15 -0700 Subject: [PATCH 075/105] Test scheduler's handling of canaries/inplace updates --- scheduler/generic_sched.go | 2 - scheduler/generic_sched_test.go | 133 +++++++++++++++++++++++++++++++- 2 files changed, 129 insertions(+), 6 deletions(-) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index e1aa47b0b733..8db5a341bafe 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -411,7 +411,6 @@ func (s *GenericScheduler) computeJobAllocs() error { s.plan.AppendUpdate(stop.alloc, structs.AllocDesiredStatusStop, stop.statusDescription, stop.clientStatus) } - // TODO test // Handle the in-place updates for _, update := range results.inplaceUpdate { if update.DeploymentID != s.deployment.GetID() { @@ -507,7 +506,6 @@ func (s *GenericScheduler) computePlacements(place []allocPlaceResult) error { alloc.PreviousAllocation = missing.previousAlloc.ID } - // TODO test // If we are placing a canary and we found a match, add the canary // to the deployment state object. if missing.canary { diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 1d413330ca5d..4d0ee4b6b183 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -8,6 +8,7 @@ import ( "time" memdb "github.com/hashicorp/go-memdb" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" ) @@ -1480,7 +1481,7 @@ func TestServiceSched_JobModify_Rolling(t *testing.T) { } } -func TestServiceSched_JobModify_InPlace(t *testing.T) { +func TestServiceSched_JobModify_Canaries(t *testing.T) { h := NewHarness(t) // Create some nodes @@ -1509,6 +1510,124 @@ func TestServiceSched_JobModify_InPlace(t *testing.T) { // Update the job job2 := mock.Job() job2.ID = job.ID + desiredUpdates := 2 + job2.TaskGroups[0].Update = &structs.UpdateStrategy{ + MaxParallel: desiredUpdates, + Canary: desiredUpdates, + HealthCheck: structs.UpdateStrategyHealthCheck_Checks, + MinHealthyTime: 10 * time.Second, + HealthyDeadline: 10 * time.Minute, + } + + // Update the task, such that it cannot be done in-place + job2.TaskGroups[0].Tasks[0].Config["command"] = "/bin/other" + noErr(t, h.State.UpsertJob(h.NextIndex(), job2)) + + // Create a mock evaluation to deal with drain + eval := &structs.Evaluation{ + ID: structs.GenerateUUID(), + Priority: 50, + TriggeredBy: structs.EvalTriggerJobRegister, + JobID: job.ID, + } + + // Process the evaluation + err := h.Process(NewServiceScheduler, eval) + if err != nil { + t.Fatalf("err: %v", err) + } + + // Ensure a single plan + if len(h.Plans) != 1 { + t.Fatalf("bad: %#v", h.Plans) + } + plan := h.Plans[0] + + // Ensure the plan evicted nothing + var update []*structs.Allocation + for _, updateList := range plan.NodeUpdate { + update = append(update, updateList...) + } + if len(update) != 0 { + t.Fatalf("bad: got %d; want %d: %#v", len(update), 0, plan) + } + + // Ensure the plan allocated + var planned []*structs.Allocation + for _, allocList := range plan.NodeAllocation { + planned = append(planned, allocList...) + } + if len(planned) != desiredUpdates { + t.Fatalf("bad: %#v", plan) + } + + h.AssertEvalStatus(t, structs.EvalStatusComplete) + + // Check that the deployment id is attached to the eval + if h.Evals[0].DeploymentID == "" { + t.Fatalf("Eval not annotated with deployment id") + } + + // Ensure a deployment was created + if plan.Deployment == nil { + t.Fatalf("bad: %#v", plan) + } + state, ok := plan.Deployment.TaskGroups[job.TaskGroups[0].Name] + if !ok { + t.Fatalf("bad: %#v", plan) + } + if state.DesiredTotal != 10 && state.DesiredCanaries != desiredUpdates { + t.Fatalf("bad: %#v", state) + } + + // Assert the canaries were added to the placed list + if len(state.PlacedCanaries) != desiredUpdates { + t.Fatalf("bad: %#v", state) + } +} + +func TestServiceSched_JobModify_InPlace(t *testing.T) { + h := NewHarness(t) + + // Create some nodes + var nodes []*structs.Node + for i := 0; i < 10; i++ { + node := mock.Node() + nodes = append(nodes, node) + noErr(t, h.State.UpsertNode(h.NextIndex(), node)) + } + + // Generate a fake job with allocations and create an older deployment + job := mock.Job() + d := mock.Deployment() + d.JobID = job.ID + noErr(t, h.State.UpsertJob(h.NextIndex(), job)) + noErr(t, h.State.UpsertDeployment(h.NextIndex(), d)) + + // Create allocs that are part of the old deployment + var allocs []*structs.Allocation + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = nodes[i].ID + alloc.Name = fmt.Sprintf("my-job.web[%d]", i) + alloc.DeploymentID = d.ID + alloc.DeploymentStatus = &structs.AllocDeploymentStatus{Healthy: helper.BoolToPtr(true)} + allocs = append(allocs, alloc) + } + noErr(t, h.State.UpsertAllocs(h.NextIndex(), allocs)) + + // Update the job + job2 := mock.Job() + job2.ID = job.ID + desiredUpdates := 4 + job2.TaskGroups[0].Update = &structs.UpdateStrategy{ + MaxParallel: desiredUpdates, + HealthCheck: structs.UpdateStrategyHealthCheck_Checks, + MinHealthyTime: 10 * time.Second, + HealthyDeadline: 10 * time.Minute, + } noErr(t, h.State.UpsertJob(h.NextIndex(), job2)) // Create a mock evaluation to deal with drain @@ -1561,9 +1680,6 @@ func TestServiceSched_JobModify_InPlace(t *testing.T) { // Ensure all allocations placed if len(out) != 10 { - for _, alloc := range out { - t.Logf("%#v", alloc) - } t.Fatalf("bad: %#v", out) } h.AssertEvalStatus(t, structs.EvalStatusComplete) @@ -1577,6 +1693,15 @@ func TestServiceSched_JobModify_InPlace(t *testing.T) { } } } + + // Verify the deployment id was changed and health cleared + for _, alloc := range out { + if alloc.DeploymentID == d.ID { + t.Fatalf("bad: deployment id not cleared") + } else if alloc.DeploymentStatus != nil { + t.Fatalf("bad: deployment status not cleared") + } + } } func TestServiceSched_JobModify_DistinctProperty(t *testing.T) { From b7088b3d402fd556010ca52e4955ce7fe65ab873 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 09:28:08 -0700 Subject: [PATCH 076/105] plan apply tests --- nomad/plan_apply.go | 1 - nomad/plan_apply_test.go | 83 ++++++++++++++++++++++++++++++++++++++-- 2 files changed, 79 insertions(+), 5 deletions(-) diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index a300126f41fd..1035d70ab919 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -328,7 +328,6 @@ OUTER: return result, mErr.ErrorOrNil() } -// TODO test // correctDeploymentCanaries ensures that the deployment object doesn't list any // canaries as placed if they didn't actually get placed. This could happen if // the plan had a partial commit. diff --git a/nomad/plan_apply_test.go b/nomad/plan_apply_test.go index b4b16bbac6f6..d202eaab6dce 100644 --- a/nomad/plan_apply_test.go +++ b/nomad/plan_apply_test.go @@ -67,13 +67,34 @@ func TestPlanApply_applyPlan(t *testing.T) { node := mock.Node() testRegisterNode(t, s1, node) - // Register alloc + // Register a fake deployment + oldDeployment := mock.Deployment() + if err := s1.State().UpsertDeployment(900, oldDeployment); err != nil { + t.Fatalf("UpsertDeployment failed: %v", err) + } + + // Create a deployment + dnew := mock.Deployment() + + // Create a deployment update for the old deployment id + desiredStatus, desiredStatusDescription := "foo", "bar" + updates := []*structs.DeploymentStatusUpdate{ + { + DeploymentID: oldDeployment.ID, + Status: desiredStatus, + StatusDescription: desiredStatusDescription, + }, + } + + // Register alloc, deployment and deployment update alloc := mock.Alloc() s1.State().UpsertJobSummary(1000, mock.JobSummary(alloc.JobID)) planRes := &structs.PlanResult{ NodeAllocation: map[string][]*structs.Allocation{ node.ID: []*structs.Allocation{alloc}, }, + Deployment: dnew, + DeploymentUpdates: updates, } // Snapshot the state @@ -84,8 +105,9 @@ func TestPlanApply_applyPlan(t *testing.T) { // Create the plan with a deployment plan := &structs.Plan{ - Job: alloc.Job, - Deployment: mock.Deployment(), + Job: alloc.Job, + Deployment: dnew, + DeploymentUpdates: updates, } // Apply the plan @@ -123,7 +145,7 @@ func TestPlanApply_applyPlan(t *testing.T) { t.Fatalf("missing alloc") } - // Lookup the deployment + // Lookup the new deployment dout, err := fsmState.DeploymentByID(ws, plan.Deployment.ID) if err != nil { t.Fatalf("err: %v", err) @@ -132,6 +154,18 @@ func TestPlanApply_applyPlan(t *testing.T) { t.Fatalf("missing deployment") } + // Lookup the updated deployment + dout2, err := fsmState.DeploymentByID(ws, oldDeployment.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + if dout2 == nil { + t.Fatalf("missing deployment") + } + if dout2.Status != desiredStatus || dout2.StatusDescription != desiredStatusDescription { + t.Fatalf("bad status: %#v", dout2) + } + // Evict alloc, Register alloc2 allocEvict := new(structs.Allocation) *allocEvict = *alloc @@ -214,6 +248,14 @@ func TestPlanApply_EvalPlan_Simple(t *testing.T) { NodeAllocation: map[string][]*structs.Allocation{ node.ID: []*structs.Allocation{alloc}, }, + Deployment: mock.Deployment(), + DeploymentUpdates: []*structs.DeploymentStatusUpdate{ + { + DeploymentID: structs.GenerateUUID(), + Status: "foo", + StatusDescription: "bar", + }, + }, } pool := NewEvaluatePool(workerPoolSize, workerPoolBufferSize) @@ -229,6 +271,12 @@ func TestPlanApply_EvalPlan_Simple(t *testing.T) { if !reflect.DeepEqual(result.NodeAllocation, plan.NodeAllocation) { t.Fatalf("incorrect node allocations") } + if !reflect.DeepEqual(result.Deployment, plan.Deployment) { + t.Fatalf("incorrect deployment") + } + if !reflect.DeepEqual(result.DeploymentUpdates, plan.DeploymentUpdates) { + t.Fatalf("incorrect deployment updates") + } } func TestPlanApply_EvalPlan_Partial(t *testing.T) { @@ -242,11 +290,17 @@ func TestPlanApply_EvalPlan_Partial(t *testing.T) { alloc := mock.Alloc() alloc2 := mock.Alloc() // Ensure alloc2 does not fit alloc2.Resources = node2.Resources + + // Create a deployment where the allocs are markeda as canaries + d := mock.Deployment() + d.TaskGroups["web"].PlacedCanaries = []string{alloc.ID, alloc2.ID} + plan := &structs.Plan{ NodeAllocation: map[string][]*structs.Allocation{ node.ID: []*structs.Allocation{alloc}, node2.ID: []*structs.Allocation{alloc2}, }, + Deployment: d, } pool := NewEvaluatePool(workerPoolSize, workerPoolBufferSize) @@ -266,6 +320,16 @@ func TestPlanApply_EvalPlan_Partial(t *testing.T) { if _, ok := result.NodeAllocation[node2.ID]; ok { t.Fatalf("should not allow alloc2") } + + // Check the deployment was updated + if result.Deployment == nil || len(result.Deployment.TaskGroups) == 0 { + t.Fatalf("bad: %v", result.Deployment) + } + placedCanaries := result.Deployment.TaskGroups["web"].PlacedCanaries + if len(placedCanaries) != 1 || placedCanaries[0] != alloc.ID { + t.Fatalf("bad: %v", placedCanaries) + } + if result.RefreshIndex != 1001 { t.Fatalf("bad: %d", result.RefreshIndex) } @@ -288,6 +352,14 @@ func TestPlanApply_EvalPlan_Partial_AllAtOnce(t *testing.T) { node.ID: []*structs.Allocation{alloc}, node2.ID: []*structs.Allocation{alloc2}, }, + Deployment: mock.Deployment(), + DeploymentUpdates: []*structs.DeploymentStatusUpdate{ + { + DeploymentID: structs.GenerateUUID(), + Status: "foo", + StatusDescription: "bar", + }, + }, } pool := NewEvaluatePool(workerPoolSize, workerPoolBufferSize) @@ -307,6 +379,9 @@ func TestPlanApply_EvalPlan_Partial_AllAtOnce(t *testing.T) { if result.RefreshIndex != 1001 { t.Fatalf("bad: %d", result.RefreshIndex) } + if result.Deployment != nil || len(result.DeploymentUpdates) != 0 { + t.Fatalf("bad: %v", result) + } } func TestPlanApply_EvalNodePlan_Simple(t *testing.T) { From 989aa56304bed3f055e3779c032a6943a386506b Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 09:55:39 -0700 Subject: [PATCH 077/105] Remove canary --- api/allocations.go | 1 - nomad/deployment_endpoint_test.go | 2 +- .../deploymentwatcher/deployments_watcher_test.go | 4 ++-- nomad/fsm_test.go | 4 ++-- nomad/state/state_store.go | 14 +++++++++++--- nomad/state/state_store_test.go | 12 ++++++------ nomad/structs/structs.go | 3 --- scheduler/reconcile_test.go | 15 +++++++++++---- scheduler/reconcile_util.go | 11 ----------- scheduler/util.go | 4 ++-- scheduler/util_test.go | 9 ++++++++- 11 files changed, 43 insertions(+), 36 deletions(-) diff --git a/api/allocations.go b/api/allocations.go index e340b8d8f50f..a6efe4790144 100644 --- a/api/allocations.go +++ b/api/allocations.go @@ -155,7 +155,6 @@ type AllocationListStub struct { // heatlhy. type AllocDeploymentStatus struct { Healthy *bool - Promoted bool ModifyIndex uint64 } diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index 995f3f9466f1..fb244b149caa 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -194,7 +194,7 @@ func TestDeploymentEndpoint_Promote(t *testing.T) { d.TaskGroups["web"].DesiredCanaries = 2 d.JobID = j.ID a := mock.Alloc() - a.Canary = true + d.TaskGroups[a.TaskGroup].PlacedCanaries = []string{a.ID} a.DeploymentID = d.ID a.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 770e3af410f6..91993c919ba7 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -401,7 +401,7 @@ func TestWatcher_PromoteDeployment_HealthyCanaries(t *testing.T) { d := mock.Deployment() d.JobID = j.ID a := mock.Alloc() - a.Canary = true + d.TaskGroups[a.TaskGroup].PlacedCanaries = []string{a.ID} a.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), } @@ -459,7 +459,7 @@ func TestWatcher_PromoteDeployment_UnhealthyCanaries(t *testing.T) { d := mock.Deployment() d.JobID = j.ID a := mock.Alloc() - a.Canary = true + d.TaskGroups[a.TaskGroup].PlacedCanaries = []string{a.ID} a.DeploymentID = d.ID assert.Nil(m.state.UpsertJob(m.nextIndex(), j), "UpsertJob") assert.Nil(m.state.UpsertDeployment(m.nextIndex(), d), "UpsertDeployment") diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index a4b2a0d00278..b73696e9777b 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1246,14 +1246,14 @@ func TestFSM_DeploymentPromotion(t *testing.T) { c1 := mock.Alloc() c1.JobID = j.ID c1.DeploymentID = d.ID - c1.Canary = true + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) c1.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), } c2 := mock.Alloc() c2.JobID = j.ID c2.DeploymentID = d.ID - c2.Canary = true + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) c2.TaskGroup = tg2.Name c2.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 3ad6d37211bc..1a7337a1f02e 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -1920,6 +1920,13 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD groupIndex[g] = struct{}{} } + canaryIndex := make(map[string]struct{}, len(deployment.TaskGroups)) + for _, state := range deployment.TaskGroups { + for _, c := range state.PlacedCanaries { + canaryIndex[c] = struct{}{} + } + } + var unhealthyErr multierror.Error for { raw := iter.Next() @@ -1928,13 +1935,14 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD } alloc := raw.(*structs.Allocation) - if !alloc.Canary { + + // Check that the alloc is a canary + if _, ok := canaryIndex[alloc.ID]; !ok { continue } // Check that the canary is part of a group being promoted - _, ok := groupIndex[alloc.TaskGroup] - if !req.All && !ok { + if _, ok := groupIndex[alloc.TaskGroup]; !req.All && !ok { continue } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 8e9027b4caed..687c3340933a 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -4853,11 +4853,11 @@ func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) { c1 := mock.Alloc() c1.JobID = j.ID c1.DeploymentID = d.ID - c1.Canary = true + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) c2 := mock.Alloc() c2.JobID = j.ID c2.DeploymentID = d.ID - c2.Canary = true + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) if err := state.UpsertAllocs(3, []*structs.Allocation{c1, c2}); err != nil { t.Fatalf("err: %v", err) @@ -4917,14 +4917,14 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { c1 := mock.Alloc() c1.JobID = j.ID c1.DeploymentID = d.ID - c1.Canary = true + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) c1.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), } c2 := mock.Alloc() c2.JobID = j.ID c2.DeploymentID = d.ID - c2.Canary = true + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) c2.TaskGroup = tg2.Name c2.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), @@ -5010,14 +5010,14 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { c1 := mock.Alloc() c1.JobID = j.ID c1.DeploymentID = d.ID - c1.Canary = true + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) c1.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), } c2 := mock.Alloc() c2.JobID = j.ID c2.DeploymentID = d.ID - c2.Canary = true + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) c2.TaskGroup = tg2.Name c2.DeploymentStatus = &structs.AllocDeploymentStatus{ Healthy: helper.BoolToPtr(true), diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index d1b91e71c365..6586db4838a8 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4154,9 +4154,6 @@ type Allocation struct { // given deployment DeploymentStatus *AllocDeploymentStatus - // Canary marks this allocation as being a canary - Canary bool - // Raft Indexes CreateIndex uint64 ModifyIndex uint64 diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 5cd5d6bf9d23..498d0e4f5a6e 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -171,9 +171,16 @@ func assertNamesHaveIndexes(t *testing.T, indexes []int, names []string) { } } -func assertNoCanariesStopped(t *testing.T, stop []allocStopResult) { +func assertNoCanariesStopped(t *testing.T, d *structs.Deployment, stop []allocStopResult) { + canaryIndex := make(map[string]struct{}) + for _, state := range d.TaskGroups { + for _, c := range state.PlacedCanaries { + canaryIndex[c] = struct{}{} + } + } + for _, s := range stop { - if s.alloc.Canary { + if _, ok := canaryIndex[s.alloc.ID]; ok { t.Fatalf("Stopping canary alloc %q %q", s.alloc.ID, s.alloc.Name) } } @@ -2202,7 +2209,7 @@ func TestReconciler_PromoteCanaries_Unblock(t *testing.T) { }, }) - assertNoCanariesStopped(t, r.stop) + assertNoCanariesStopped(t, d, r.stop) assertNamesHaveIndexes(t, intRange(2, 3), placeResultsToNames(r.place)) assertNamesHaveIndexes(t, intRange(0, 3), stopResultsToNames(r.stop)) } @@ -2283,7 +2290,7 @@ func TestReconciler_PromoteCanaries_CanariesEqualCount(t *testing.T) { }, }) - assertNoCanariesStopped(t, r.stop) + assertNoCanariesStopped(t, d, r.stop) assertNamesHaveIndexes(t, intRange(0, 1), stopResultsToNames(r.stop)) } diff --git a/scheduler/reconcile_util.go b/scheduler/reconcile_util.go index b67895442694..34b1e811ee40 100644 --- a/scheduler/reconcile_util.go +++ b/scheduler/reconcile_util.go @@ -164,17 +164,6 @@ func (a allocSet) filterByTainted(nodes map[string]*structs.Node) (untainted, mi return } -// filterByCanary returns a new allocation set that contains only canaries -func (a allocSet) filterByCanary() allocSet { - canaries := make(map[string]*structs.Allocation) - for _, alloc := range a { - if alloc.Canary { - canaries[alloc.ID] = alloc - } - } - return canaries -} - // filterByDeployment filters allocations into two sets, those that match the // given deployment ID and those that don't func (a allocSet) filterByDeployment(id string) (match, nonmatch allocSet) { diff --git a/scheduler/util.go b/scheduler/util.go index 49368981aee4..e36df13b636b 100644 --- a/scheduler/util.go +++ b/scheduler/util.go @@ -290,8 +290,8 @@ func retryMax(max int, cb func() (bool, error), reset func() bool) error { // If the result is nil, false is returned. func progressMade(result *structs.PlanResult) bool { return result != nil && (len(result.NodeUpdate) != 0 || - len(result.NodeAllocation) != 0) || result.Deployment != nil || - len(result.DeploymentUpdates) != 0 + len(result.NodeAllocation) != 0 || result.Deployment != nil || + len(result.DeploymentUpdates) != 0) } // taintedNodes is used to scan the allocations and then check if the diff --git a/scheduler/util_test.go b/scheduler/util_test.go index 204d10602608..5d5f10045c0e 100644 --- a/scheduler/util_test.go +++ b/scheduler/util_test.go @@ -1006,7 +1006,14 @@ func TestProgressMade(t *testing.T) { } update := &structs.PlanResult{NodeUpdate: m} alloc := &structs.PlanResult{NodeAllocation: m} - if !(progressMade(both) && progressMade(update) && progressMade(alloc)) { + deployment := &structs.PlanResult{Deployment: mock.Deployment()} + deploymentUpdates := &structs.PlanResult{ + DeploymentUpdates: []*structs.DeploymentStatusUpdate{ + {DeploymentID: structs.GenerateUUID()}, + }, + } + if !(progressMade(both) && progressMade(update) && progressMade(alloc) && + progressMade(deployment) && progressMade(deploymentUpdates)) { t.Fatal("bad") } } From b3ec1466850a7d4284016c7ac986839c85a826e4 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 14:28:59 -0700 Subject: [PATCH 078/105] Respond to comments --- scheduler/reconcile.go | 29 ++++++++--------------------- 1 file changed, 8 insertions(+), 21 deletions(-) diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 2bc6eaf2280d..9bf435eff42a 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -261,7 +261,6 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { // Get the deployment state for the group var dstate *structs.DeploymentState existingDeployment := false - deploymentComplete := true if a.deployment != nil { dstate, existingDeployment = a.deployment.TaskGroups[group] } @@ -304,12 +303,8 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { ignore, inplace, destructive := a.computeUpdates(tg, untainted) desiredChanges.Ignore += uint64(len(ignore)) desiredChanges.InPlaceUpdate += uint64(len(inplace)) - changes := len(destructive) + len(inplace) if !existingDeployment { - dstate.DesiredTotal += changes - } - if changes != 0 { - deploymentComplete = false + dstate.DesiredTotal += len(destructive) + len(inplace) } // The fact that we have destructive updates and have less canaries than is @@ -335,11 +330,6 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { } } - // As long as we require canaries we aren't complete - if requireCanary { - deploymentComplete = false - } - // Determine how many we can place canaryState = dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted limit := a.computeLimit(tg, untainted, destructive, canaryState) @@ -380,11 +370,6 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { desiredChanges.Ignore += uint64(len(destructive)) } - // As long as we require placements we aren't complete - if len(place) != 0 { - deploymentComplete = false - } - // TODO Migrations should be done using a stagger and max_parallel. if !a.deploymentFailed { desiredChanges.Migrate += uint64(len(migrate)) @@ -392,11 +377,6 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { desiredChanges.Stop += uint64(len(migrate)) } - // As long as we require migrations we aren't complete - if len(migrate) != 0 { - deploymentComplete = false - } - for _, alloc := range migrate { a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, @@ -421,6 +401,10 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { a.deployment.TaskGroups[group] = dstate } + // deploymentComplete is whether the deployment is complete which largely + // means that no placements were made or desired to be made + deploymentComplete := len(destructive)+len(inplace)+len(place)+len(migrate) == 0 && !requireCanary + // Final check to see if the deployment is complete is to ensure everything // is healthy if deploymentComplete && a.deployment != nil { @@ -452,6 +436,7 @@ func (a *allocReconciler) handleGroupCanaries(all allocSet, desiredChanges *stru } } + // Cancel any non-promoted canaries from a failed deployment if a.deployment != nil && a.deployment.Status == structs.DeploymentStatusFailed { for _, s := range a.deployment.TaskGroups { if !s.Promoted { @@ -460,6 +445,8 @@ func (a *allocReconciler) handleGroupCanaries(all allocSet, desiredChanges *stru } } + // stopSet is the allocSet that contains the canaries we desire to stop from + // above. stopSet := all.fromKeys(stop) a.markStop(stopSet, "", allocNotNeeded) desiredChanges.Stop += uint64(len(stopSet)) From 62550c1f4abead6b061437d3ca7c19bc13b85d4b Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 11:13:38 -0700 Subject: [PATCH 079/105] Complete deployments mark jobs as stable This PR allows jobs to be marked as stable automatically by a successful deployment. --- nomad/state/state_store.go | 99 ++++++++++++++++++++++----------- nomad/state/state_store_test.go | 57 +++++++++++++++++++ 2 files changed, 122 insertions(+), 34 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 1a7337a1f02e..580ade621ced 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -138,32 +138,12 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR // upsertDeploymentUpdates updates the deployments given the passed status // updates. func (s *StateStore) upsertDeploymentUpdates(index uint64, updates []*structs.DeploymentStatusUpdate, txn *memdb.Txn) error { - for _, d := range updates { - raw, err := txn.First("deployment", "id", d.DeploymentID) - if err != nil { - return err - } - if raw == nil { - return fmt.Errorf("Deployment ID %q couldn't be updated as it does not exist", d.DeploymentID) - } - - copy := raw.(*structs.Deployment).Copy() - - // Apply the new status - copy.Status = d.Status - copy.StatusDescription = d.StatusDescription - copy.ModifyIndex = index - - // Insert the deployment - if err := txn.Insert("deployment", copy); err != nil { + for _, u := range updates { + if err := s.updateDeploymentStatusImpl(index, u, txn); err != nil { return err } } - if err := txn.Insert("index", &IndexEntry{"deployment", index}); err != nil { - return fmt.Errorf("index update failed: %v", err) - } - return nil } @@ -256,6 +236,13 @@ func (s *StateStore) upsertDeploymentImpl(index uint64, deployment *structs.Depl return fmt.Errorf("index update failed: %v", err) } + // If the deployment is being marked as complete, set the job to stable. + if deployment.Status == structs.DeploymentStatusSuccessful { + if err := s.updateJobStability(index, deployment, txn); err != nil { + return fmt.Errorf("failed to update job stability: %v", err) + } + } + return nil } @@ -569,7 +556,7 @@ func (s *StateStore) Nodes(ws memdb.WatchSet) (memdb.ResultIterator, error) { func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { txn := s.db.Txn(true) defer txn.Abort() - if err := s.upsertJobImpl(index, job, txn); err != nil { + if err := s.upsertJobImpl(index, job, false, txn); err != nil { return err } txn.Commit() @@ -577,7 +564,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { } // upsertJobImpl is the inplementation for registering a job or updating a job definition -func (s *StateStore) upsertJobImpl(index uint64, job *structs.Job, txn *memdb.Txn) error { +func (s *StateStore) upsertJobImpl(index uint64, job *structs.Job, keepVersion bool, txn *memdb.Txn) error { // Check if the job already exists existing, err := txn.First("jobs", "id", job.ID) if err != nil { @@ -589,7 +576,13 @@ func (s *StateStore) upsertJobImpl(index uint64, job *structs.Job, txn *memdb.Tx job.CreateIndex = existing.(*structs.Job).CreateIndex job.ModifyIndex = index job.JobModifyIndex = index - job.Version = existing.(*structs.Job).Version + 1 + + // Bump the version unless asked to keep it. This should only be done + // when changing an internal field such as Stable. A spec change should + // always come with a version bump + if !keepVersion { + job.Version = existing.(*structs.Job).Version + 1 + } // Compute the job status var err error @@ -883,12 +876,19 @@ func (s *StateStore) jobVersionByID(txn *memdb.Txn, ws *memdb.WatchSet, id strin // JobByIDAndVersion returns the job identified by its ID and Version func (s *StateStore) JobByIDAndVersion(ws memdb.WatchSet, id string, version uint64) (*structs.Job, error) { txn := s.db.Txn(false) + return s.jobByIDAndVersionImpl(ws, id, version, txn) +} + +// jobByIDAndVersionImpl returns the job identified by its ID and Version +func (s *StateStore) jobByIDAndVersionImpl(ws memdb.WatchSet, id string, version uint64, txn *memdb.Txn) (*structs.Job, error) { watchCh, existing, err := txn.FirstWatch("job_version", "id", id, version) if err != nil { return nil, err } - ws.Add(watchCh) + if ws != nil { + ws.Add(watchCh) + } if existing != nil { job := existing.(*structs.Job) @@ -1844,7 +1844,7 @@ func (s *StateStore) UpdateDeploymentStatus(index uint64, req *structs.Deploymen // Upsert the job if necessary if req.Job != nil { - if err := s.upsertJobImpl(index, req.Job, txn); err != nil { + if err := s.upsertJobImpl(index, req.Job, false, txn); err != nil { return err } } @@ -1889,9 +1889,45 @@ func (s *StateStore) updateDeploymentStatusImpl(index uint64, u *structs.Deploym return fmt.Errorf("index update failed: %v", err) } + // If the deployment is being marked as complete, set the job to stable. + if copy.Status == structs.DeploymentStatusSuccessful { + if err := s.updateJobStability(index, copy, txn); err != nil { + return fmt.Errorf("failed to update job stability: %v", err) + } + } + return nil } +// updateJobStability updates the job version referenced by a successful +// deployment to stable. +func (s *StateStore) updateJobStability(index uint64, deployment *structs.Deployment, txn *memdb.Txn) error { + // Hot-path + if deployment.Status != structs.DeploymentStatusSuccessful { + return nil + } + + // Get the job that is referenced + job, err := s.jobByIDAndVersionImpl(nil, deployment.JobID, deployment.JobVersion, txn) + if err != nil { + return err + } + + // Has already been cleared, nothing to do + if job == nil { + return nil + } + + // If the job is already stable, nothing to do + if job.Stable { + return nil + } + + copy := job.Copy() + copy.Stable = true + return s.upsertJobImpl(index, copy, true, txn) +} + // UpdateDeploymentPromotion is used to promote canaries in a deployment and // potentially make a evaluation func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyDeploymentPromoteRequest) error { @@ -1970,15 +2006,10 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD } // Insert the deployment - if err := txn.Insert("deployment", copy); err != nil { + if err := s.upsertDeploymentImpl(index, copy, txn); err != nil { return err } - // Update the index - if err := txn.Insert("index", &IndexEntry{"deployment", index}); err != nil { - return fmt.Errorf("index update failed: %v", err) - } - // Upsert the optional eval if req.Eval != nil { if err := s.nestedUpsertEval(txn, index, req.Eval); err != nil { @@ -2068,7 +2099,7 @@ func (s *StateStore) UpdateDeploymentAllocHealth(index uint64, req *structs.Appl // Upsert the job if necessary if req.Job != nil { - if err := s.upsertJobImpl(index, req.Job, txn); err != nil { + if err := s.upsertJobImpl(index, req.Job, false, txn); err != nil { return err } } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 687c3340933a..ae78483babbd 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -4790,6 +4790,63 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) { } } +// Test that when a deployment is updated to successful the job is updated to +// stable +func TestStateStore_UpsertDeploymentStatusUpdate_Successful(t *testing.T) { + state := testStateStore(t) + + // Insert a job + job := mock.Job() + if err := state.UpsertJob(1, job); err != nil { + t.Fatalf("bad: %v", err) + } + + // Insert a deployment + d := structs.NewDeployment(job) + if err := state.UpsertDeployment(2, d); err != nil { + t.Fatalf("bad: %v", err) + } + + // Update the deployment + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusSuccessful, + StatusDescription: structs.DeploymentStatusDescriptionSuccessful, + }, + } + err := state.UpdateDeploymentStatus(3, req) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the status was updated properly + ws := memdb.NewWatchSet() + dout, err := state.DeploymentByID(ws, d.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if dout.Status != structs.DeploymentStatusSuccessful || + dout.StatusDescription != structs.DeploymentStatusDescriptionSuccessful { + t.Fatalf("bad: %#v", dout) + } + + // Check that the job was created + jout, _ := state.JobByID(ws, job.ID) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } + if !jout.Stable { + t.Fatalf("job not marked stable %#v", jout) + } + if jout.Version != d.JobVersion { + t.Fatalf("job version changed; got %d; want %d", jout.Version, d.JobVersion) + } +} + // Test that non-existant deployment can't be promoted func TestStateStore_UpsertDeploymentPromotion_NonExistant(t *testing.T) { state := testStateStore(t) From aaf5ab0a21193be1780916f424c4664e99cf077c Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 12:49:13 -0700 Subject: [PATCH 080/105] Job stability --- api/jobs.go | 34 +++++++++++++++++ command/agent/job_endpoint.go | 32 ++++++++++++++++ command/agent/job_endpoint_test.go | 51 ++++++++++++++++++++++++++ nomad/fsm.go | 18 +++++++++ nomad/fsm_test.go | 37 +++++++++++++++++++ nomad/job_endpoint.go | 44 +++++++++++++++++++++- nomad/job_endpoint_test.go | 59 ++++++++++++++++++++++++++++++ nomad/state/state_store.go | 32 ++++++++++------ nomad/state/state_store_test.go | 51 ++++++++++++++++++++++++++ nomad/structs/structs.go | 18 +++++++++ 10 files changed, 362 insertions(+), 14 deletions(-) diff --git a/api/jobs.go b/api/jobs.go index 7687e9b7d14c..bbfaa1bccda9 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -266,6 +266,23 @@ func (j *Jobs) Revert(jobID string, version uint64, enforcePriorVersion *uint64, return &resp, wm, nil } +// Stable is used to mark a job version's stability. +func (j *Jobs) Stable(jobID string, version uint64, stable bool, + q *WriteOptions) (*JobStabilityResponse, *WriteMeta, error) { + + var resp JobStabilityResponse + req := &JobStabilityRequest{ + JobID: jobID, + JobVersion: version, + Stable: stable, + } + wm, err := j.client.write("/v1/job/"+jobID+"/stable", req, &resp, q) + if err != nil { + return nil, nil, err + } + return &resp, wm, nil +} + // periodicForceResponse is used to deserialize a force response type periodicForceResponse struct { EvalID string @@ -842,3 +859,20 @@ type JobVersionsResponse struct { Diffs []*JobDiff QueryMeta } + +// JobStabilityRequest is used to marked a job as stable. +type JobStabilityRequest struct { + // Job to set the stability on + JobID string + JobVersion uint64 + + // Set the stability + Stable bool + WriteRequest +} + +// JobStabilityResponse is the response when marking a job as stable. +type JobStabilityResponse struct { + JobModifyIndex uint64 + WriteMeta +} diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index df2c630e1388..4eabb91d4032 100644 --- a/command/agent/job_endpoint.go +++ b/command/agent/job_endpoint.go @@ -76,6 +76,9 @@ func (s *HTTPServer) JobSpecificRequest(resp http.ResponseWriter, req *http.Requ case strings.HasSuffix(path, "/deployment"): jobName := strings.TrimSuffix(path, "/deployment") return s.jobLatestDeployment(resp, req, jobName) + case strings.HasSuffix(path, "/stable"): + jobName := strings.TrimSuffix(path, "/stable") + return s.jobStable(resp, req, jobName) default: return s.jobCRUD(resp, req, path) } @@ -455,6 +458,35 @@ func (s *HTTPServer) jobRevert(resp http.ResponseWriter, req *http.Request, return out, nil } +func (s *HTTPServer) jobStable(resp http.ResponseWriter, req *http.Request, + jobName string) (interface{}, error) { + + if req.Method != "PUT" && req.Method != "POST" { + return nil, CodedError(405, ErrInvalidMethod) + } + + var stableRequest structs.JobStabilityRequest + if err := decodeBody(req, &stableRequest); err != nil { + return nil, CodedError(400, err.Error()) + } + if stableRequest.JobID == "" { + return nil, CodedError(400, "JobID must be specified") + } + if stableRequest.JobID != jobName { + return nil, CodedError(400, "Job ID does not match") + } + + s.parseRegion(req, &stableRequest.Region) + + var out structs.JobStabilityResponse + if err := s.agent.RPC("Job.Stable", &stableRequest, &out); err != nil { + return nil, err + } + + setIndex(resp, out.Index) + return out, nil +} + func (s *HTTPServer) jobSummaryRequest(resp http.ResponseWriter, req *http.Request, name string) (interface{}, error) { args := structs.JobSummaryRequest{ JobID: name, diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index 6e71235d67f8..af5ade693b68 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -923,6 +923,57 @@ func TestHTTP_JobRevert(t *testing.T) { }) } +func TestHTTP_JobStable(t *testing.T) { + httpTest(t, nil, func(s *TestServer) { + // Create the job and register it twice + job := mock.Job() + regReq := structs.JobRegisterRequest{ + Job: job, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + var regResp structs.JobRegisterResponse + if err := s.Agent.RPC("Job.Register", ®Req, ®Resp); err != nil { + t.Fatalf("err: %v", err) + } + + if err := s.Agent.RPC("Job.Register", ®Req, ®Resp); err != nil { + t.Fatalf("err: %v", err) + } + + args := structs.JobStabilityRequest{ + JobID: job.ID, + JobVersion: 0, + Stable: true, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + buf := encodeReq(args) + + // Make the HTTP request + req, err := http.NewRequest("PUT", "/v1/job/"+job.ID+"/stable", buf) + if err != nil { + t.Fatalf("err: %v", err) + } + respW := httptest.NewRecorder() + + // Make the request + obj, err := s.Server.JobSpecificRequest(respW, req) + if err != nil { + t.Fatalf("err: %v", err) + } + + // Check the response + stableResp := obj.(structs.JobStabilityResponse) + if stableResp.JobModifyIndex == 0 { + t.Fatalf("bad: %v", stableResp) + } + + // Check for the index + if respW.HeaderMap.Get("X-Nomad-Index") == "" { + t.Fatalf("missing index") + } + }) +} + func TestJobs_ApiJobToStructsJob(t *testing.T) { apiJob := &api.Job{ Stop: helper.BoolToPtr(true), diff --git a/nomad/fsm.go b/nomad/fsm.go index e4bc3db2db67..fd8f294e2533 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -165,6 +165,8 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { return n.applyDeploymentAllocHealth(buf[1:], log.Index) case structs.DeploymentDeleteRequestType: return n.applyDeploymentDelete(buf[1:], log.Index) + case structs.JobStabilityRequestType: + return n.applyJobStability(buf[1:], log.Index) default: if ignoreUnknown { n.logger.Printf("[WARN] nomad.fsm: ignoring unknown message type (%d), upgrade to newer version", msgType) @@ -651,6 +653,22 @@ func (n *nomadFSM) applyDeploymentDelete(buf []byte, index uint64) interface{} { return nil } +// applyJobStability is used to set the stability of a job +func (n *nomadFSM) applyJobStability(buf []byte, index uint64) interface{} { + defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_job_stability"}, time.Now()) + var req structs.JobStabilityRequest + if err := structs.Decode(buf, &req); err != nil { + panic(fmt.Errorf("failed to decode request: %v", err)) + } + + if err := n.state.UpdateJobStability(index, req.JobID, req.JobVersion, req.Stable); err != nil { + n.logger.Printf("[ERR] nomad.fsm: UpdateJobStability failed: %v", err) + return err + } + + return nil +} + func (n *nomadFSM) Snapshot() (raft.FSMSnapshot, error) { // Create a new snapshot snap, err := n.state.Snapshot() diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index b73696e9777b..a79d6aa1beb6 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -1210,6 +1210,43 @@ func TestFSM_DeploymentStatusUpdate(t *testing.T) { } } +func TestFSM_JobStabilityUpdate(t *testing.T) { + fsm := testFSM(t) + fsm.evalBroker.SetEnabled(true) + state := fsm.State() + + // Upsert a deployment + job := mock.Job() + if err := state.UpsertJob(1, job); err != nil { + t.Fatalf("bad: %v", err) + } + + // Create a request to update the job to stable + req := &structs.JobStabilityRequest{ + JobID: job.ID, + JobVersion: job.Version, + Stable: true, + } + buf, err := structs.Encode(structs.JobStabilityRequestType, req) + if err != nil { + t.Fatalf("err: %v", err) + } + resp := fsm.Apply(makeLog(buf)) + if resp != nil { + t.Fatalf("resp: %v", resp) + } + + // Check that the stability was updated properly + ws := memdb.NewWatchSet() + jout, _ := state.JobByIDAndVersion(ws, job.ID, job.Version) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil || !jout.Stable { + t.Fatalf("bad: %#v", jout) + } +} + func TestFSM_DeploymentPromotion(t *testing.T) { fsm := testFSM(t) fsm.evalBroker.SetEnabled(true) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 3d1f608c6e8d..1f6b2faf1b5d 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -340,7 +340,7 @@ func (j *Job) Revert(args *structs.JobRevertRequest, reply *structs.JobRegisterR // Validate the arguments if args.JobID == "" { - return fmt.Errorf("missing job ID for evaluation") + return fmt.Errorf("missing job ID for revert") } // Lookup the job by version @@ -389,6 +389,46 @@ func (j *Job) Revert(args *structs.JobRevertRequest, reply *structs.JobRegisterR return j.Register(reg, reply) } +// Stable is used to mark the job version as stable +func (j *Job) Stable(args *structs.JobStabilityRequest, reply *structs.JobStabilityResponse) error { + if done, err := j.srv.forward("Job.Stable", args, args, reply); done { + return err + } + defer metrics.MeasureSince([]string{"nomad", "job", "stable"}, time.Now()) + + // Validate the arguments + if args.JobID == "" { + return fmt.Errorf("missing job ID for marking job as stable") + } + + // Lookup the job by version + snap, err := j.srv.fsm.State().Snapshot() + if err != nil { + return err + } + + ws := memdb.NewWatchSet() + jobV, err := snap.JobByIDAndVersion(ws, args.JobID, args.JobVersion) + if err != nil { + return err + } + if jobV == nil { + return fmt.Errorf("job %q at version %d not found", args.JobID, args.JobVersion) + } + + // Commit this evaluation via Raft + _, modifyIndex, err := j.srv.raftApply(structs.JobStabilityRequestType, args) + if err != nil { + j.srv.logger.Printf("[ERR] nomad.job: Eval create failed: %v", err) + return err + } + + // Setup the reply + reply.JobModifyIndex = modifyIndex + reply.Index = modifyIndex + return nil +} + // Evaluate is used to force a job for re-evaluation func (j *Job) Evaluate(args *structs.JobEvaluateRequest, reply *structs.JobRegisterResponse) error { if done, err := j.srv.forward("Job.Evaluate", args, args, reply); done { @@ -460,7 +500,7 @@ func (j *Job) Deregister(args *structs.JobDeregisterRequest, reply *structs.JobD // Validate the arguments if args.JobID == "" { - return fmt.Errorf("missing job ID for evaluation") + return fmt.Errorf("missing job ID for deregistering") } // Lookup the job diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 233bf3ad58e0..ddc6f2357e9e 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -895,6 +895,65 @@ func TestJobEndpoint_Revert(t *testing.T) { } } +func TestJobEndpoint_Stable(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the initial register request + job := mock.Job() + req := &structs.JobRegisterRequest{ + Job: job, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.JobRegisterResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.Register", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + if resp.Index == 0 { + t.Fatalf("bad index: %d", resp.Index) + } + + // Create stablility request + stableReq := &structs.JobStabilityRequest{ + JobID: job.ID, + JobVersion: 0, + Stable: true, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var stableResp structs.JobStabilityResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.Stable", stableReq, &stableResp); err != nil { + t.Fatalf("err: %v", err) + } + if stableResp.Index == 0 { + t.Fatalf("bad index: %d", resp.Index) + } + + // Check that the job is marked stable + state := s1.fsm.State() + ws := memdb.NewWatchSet() + out, err := state.JobByID(ws, job.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + if out == nil { + t.Fatalf("expected job") + } + if !out.Stable { + t.Fatalf("Job is not marked stable") + } + if out.JobModifyIndex != stableResp.JobModifyIndex { + t.Fatalf("got job modify index %d; want %d", out.JobModifyIndex, stableResp.JobModifyIndex) + } +} + func TestJobEndpoint_Evaluate(t *testing.T) { s1 := testServer(t, func(c *Config) { c.NumSchedulers = 0 // Prevent automatic dequeue diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 580ade621ced..a2f4e2fb8999 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -238,7 +238,7 @@ func (s *StateStore) upsertDeploymentImpl(index uint64, deployment *structs.Depl // If the deployment is being marked as complete, set the job to stable. if deployment.Status == structs.DeploymentStatusSuccessful { - if err := s.updateJobStability(index, deployment, txn); err != nil { + if err := s.updateJobStabilityImpl(index, deployment.JobID, deployment.JobVersion, true, txn); err != nil { return fmt.Errorf("failed to update job stability: %v", err) } } @@ -1891,7 +1891,7 @@ func (s *StateStore) updateDeploymentStatusImpl(index uint64, u *structs.Deploym // If the deployment is being marked as complete, set the job to stable. if copy.Status == structs.DeploymentStatusSuccessful { - if err := s.updateJobStability(index, copy, txn); err != nil { + if err := s.updateJobStabilityImpl(index, copy.JobID, copy.JobVersion, true, txn); err != nil { return fmt.Errorf("failed to update job stability: %v", err) } } @@ -1899,16 +1899,24 @@ func (s *StateStore) updateDeploymentStatusImpl(index uint64, u *structs.Deploym return nil } -// updateJobStability updates the job version referenced by a successful -// deployment to stable. -func (s *StateStore) updateJobStability(index uint64, deployment *structs.Deployment, txn *memdb.Txn) error { - // Hot-path - if deployment.Status != structs.DeploymentStatusSuccessful { - return nil +// UpdateJobStability updates the stability of the given job and version to the +// desired status. +func (s *StateStore) UpdateJobStability(index uint64, jobID string, jobVersion uint64, stable bool) error { + txn := s.db.Txn(true) + defer txn.Abort() + + if err := s.updateJobStabilityImpl(index, jobID, jobVersion, stable, txn); err != nil { + return err } + txn.Commit() + return nil +} + +// updateJobStabilityImpl updates the stability of the given job and version +func (s *StateStore) updateJobStabilityImpl(index uint64, jobID string, jobVersion uint64, stable bool, txn *memdb.Txn) error { // Get the job that is referenced - job, err := s.jobByIDAndVersionImpl(nil, deployment.JobID, deployment.JobVersion, txn) + job, err := s.jobByIDAndVersionImpl(nil, jobID, jobVersion, txn) if err != nil { return err } @@ -1918,13 +1926,13 @@ func (s *StateStore) updateJobStability(index uint64, deployment *structs.Deploy return nil } - // If the job is already stable, nothing to do - if job.Stable { + // If the job already has the desired stability, nothing to do + if job.Stable == stable { return nil } copy := job.Copy() - copy.Stable = true + copy.Stable = stable return s.upsertJobImpl(index, copy, true, txn) } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index ae78483babbd..ac16f9c1affb 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -4847,6 +4847,57 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Successful(t *testing.T) { } } +func TestStateStore_UpdateJobStability(t *testing.T) { + state := testStateStore(t) + + // Insert a job twice to get two versions + job := mock.Job() + if err := state.UpsertJob(1, job); err != nil { + t.Fatalf("bad: %v", err) + } + + if err := state.UpsertJob(2, job); err != nil { + t.Fatalf("bad: %v", err) + } + + // Update the stability to true + err := state.UpdateJobStability(3, job.ID, 0, true) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the job was updated properly + ws := memdb.NewWatchSet() + jout, _ := state.JobByIDAndVersion(ws, job.ID, 0) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } + if !jout.Stable { + t.Fatalf("job not marked stable %#v", jout) + } + + // Update the stability to false + err = state.UpdateJobStability(3, job.ID, 0, false) + if err != nil { + t.Fatalf("bad: %v", err) + } + + // Check that the job was updated properly + jout, _ = state.JobByIDAndVersion(ws, job.ID, 0) + if err != nil { + t.Fatalf("bad: %v", err) + } + if jout == nil { + t.Fatalf("bad: %#v", jout) + } + if jout.Stable { + t.Fatalf("job marked stable %#v", jout) + } +} + // Test that non-existant deployment can't be promoted func TestStateStore_UpsertDeploymentPromotion_NonExistant(t *testing.T) { state := testStateStore(t) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 6586db4838a8..3b67f2f27d80 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -58,6 +58,7 @@ const ( DeploymentPromoteRequestType DeploymentAllocHealthRequestType DeploymentDeleteRequestType + JobStabilityRequestType ) const ( @@ -318,6 +319,23 @@ type JobRevertRequest struct { WriteRequest } +// JobStabilityRequest is used to marked a job as stable. +type JobStabilityRequest struct { + // Job to set the stability on + JobID string + JobVersion uint64 + + // Set the stability + Stable bool + WriteRequest +} + +// JobStabilityResponse is the response when marking a job as stable. +type JobStabilityResponse struct { + JobModifyIndex uint64 + WriteMeta +} + // NodeListRequest is used to parameterize a list request type NodeListRequest struct { QueryOptions From 65abc9ae41d0358abac7228fc03c3e09831d6307 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 13:21:56 -0700 Subject: [PATCH 081/105] HTTP API docs for job --- website/source/api/jobs.html.md | 286 ++++++++++++++++++++++++++++++-- 1 file changed, 270 insertions(+), 16 deletions(-) diff --git a/website/source/api/jobs.html.md b/website/source/api/jobs.html.md index 317d2f0825f6..a241887ab7ff 100644 --- a/website/source/api/jobs.html.md +++ b/website/source/api/jobs.html.md @@ -28,7 +28,7 @@ The table below shows this endpoint's support for ### Parameters -- `prefix` `(string: "")`- Specifies a string to filter jobs on based on +- `prefix` `(string: "")` - Specifies a string to filter jobs on based on an index prefix. This is specified as a querystring parameter. ### Sample Request @@ -149,7 +149,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -363,7 +363,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -535,7 +535,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -687,7 +687,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -730,6 +730,151 @@ $ curl \ ] ``` +## List Job Deployments + +This endpoint lists a single job's deployments + +| Method | Path | Produces | +| ------ | ----------------------------- | -------------------------- | +| `GET` | `/v1/job/:job_id/deployments` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `YES` | `none` | + +### Parameters + +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in + the job file during submission). This is specified as part of the path. + +### Sample Request + +```text +$ curl \ + https://nomad.rocks/v1/job/my-job/deployments +``` + +### Sample Response + +```json +[ + { + "ID": "85ee4a9a-339f-a921-a9ef-0550d20b2c61", + "JobID": "my-job", + "JobVersion": 1, + "JobModifyIndex": 19, + "JobCreateIndex": 7, + "TaskGroups": { + "cache": { + "AutoRevert": true, + "Promoted": false, + "PlacedCanaries": [ + "d0ad0808-2765-abf6-1e15-79fb7fe5a416", + "38c70cd8-81f2-1489-a328-87bb29ec0e0f" + ], + "DesiredCanaries": 2, + "DesiredTotal": 3, + "PlacedAllocs": 2, + "HealthyAllocs": 2, + "UnhealthyAllocs": 0 + } + }, + "Status": "running", + "StatusDescription": "Deployment is running", + "CreateIndex": 21, + "ModifyIndex": 25 + }, + { + "ID": "fb6070fb-4a44-e255-4e6f-8213eba3871a", + "JobID": "my-job", + "JobVersion": 0, + "JobModifyIndex": 7, + "JobCreateIndex": 7, + "TaskGroups": { + "cache": { + "AutoRevert": true, + "Promoted": false, + "PlacedCanaries": null, + "DesiredCanaries": 0, + "DesiredTotal": 3, + "PlacedAllocs": 3, + "HealthyAllocs": 3, + "UnhealthyAllocs": 0 + } + }, + "Status": "successful", + "StatusDescription": "Deployment completed successfully", + "CreateIndex": 9, + "ModifyIndex": 17 + } +] +``` + + +## Read Job's Most Recent Deployment + +This endpoint returns a single job's most recent deployment. + +| Method | Path | Produces | +| ------ | ----------------------------- | -------------------------- | +| `GET` | `/v1/job/:job_id/deployment` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `YES` | `none` | + +### Parameters + +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in + the job file during submission). This is specified as part of the path. + +### Sample Request + +```text +$ curl \ + https://nomad.rocks/v1/job/my-job/deployments +``` + +### Sample Response + +```json +{ + "ID": "85ee4a9a-339f-a921-a9ef-0550d20b2c61", + "JobID": "my-job", + "JobVersion": 1, + "JobModifyIndex": 19, + "JobCreateIndex": 7, + "TaskGroups": { + "cache": { + "AutoRevert": true, + "Promoted": false, + "PlacedCanaries": [ + "d0ad0808-2765-abf6-1e15-79fb7fe5a416", + "38c70cd8-81f2-1489-a328-87bb29ec0e0f" + ], + "DesiredCanaries": 2, + "DesiredTotal": 3, + "PlacedAllocs": 2, + "HealthyAllocs": 2, + "UnhealthyAllocs": 0 + } + }, + "Status": "running", + "StatusDescription": "Deployment is running", + "CreateIndex": 21, + "ModifyIndex": 25 +} +``` + + ## Read Job Summary This endpoint reads summary information about a job. @@ -748,7 +893,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -801,7 +946,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. - `Job` `(Job: )` - Specifies the JSON definition of the job. @@ -863,8 +1008,8 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in - the job file during submission). This is specified as part of the path. +- `:job_id` `(string: )` - Specifies the ID of the job (as specified + in the job file during submission). This is specified as part of the path. - `Payload` `(string: "")` - Specifies a base64 encoded string containing the payload. This is limited to 15 KB. @@ -889,7 +1034,7 @@ The table below shows this endpoint's support for $ curl \ --request POST \ --payload @payload.json \ - https://nomad.rocks/v1/job/my-job/summary + https://nomad.rocks/v1/job/my-job/dispatch ``` ### Sample Response @@ -904,6 +1049,115 @@ $ curl \ } ``` +## Revert to older Job Version + +This endpoint reverts the job to an older version. + +| Method | Path | Produces | +| ------- | -------------------------- | -------------------------- | +| `POST` | `/v1/job/:job_id/revert` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `none` | + +### Parameters + +- `JobID` `(string: )` - Specifies the ID of the job (as specified + in the job file during submission). This is specified as part of the path. + +- `JobVersion` `(integer: 0) - Specifies the job version to revert to. + +- `EnforcePriorVersion` `(integer: nil)` - Optional value specifying the current + job's version. This is checked and acts as a check-and-set value before + reverting to the specified job. + +### Sample Payload + +```json +{ + "JobID": "my-job", + "JobVersion": 2, +} +``` + +### Sample Request + +```text +$ curl \ + --request POST \ + --payload @payload.json \ + https://nomad.rocks/v1/job/my-job/revert +``` + +### Sample Response + +```json +{ + "EvalID": "d092fdc0-e1fd-2536-67d8-43af8ca798ac", + "EvalCreateIndex": 35, + "JobModifyIndex": 34, +} +``` + + +## Set Job Stability + +This endpoint sets the job's stability. + +| Method | Path | Produces | +| ------- | -------------------------- | -------------------------- | +| `POST` | `/v1/job/:job_id/stable` | `application/json` | + +The table below shows this endpoint's support for +[blocking queries](/api/index.html#blocking-queries) and +[required ACLs](/api/index.html#acls). + +| Blocking Queries | ACL Required | +| ---------------- | ------------ | +| `NO` | `none` | + +### Parameters + +- `JobID` `(string: )` - Specifies the ID of the job (as specified + in the job file during submission). This is specified as part of the path. + +- `JobVersion` `(integer: 0) - Specifies the job version to set the stability on. + +- `Stable` `(bool: false)` - Specifies whether the job should be marked as + stable or not. + +### Sample Payload + +```json +{ + "JobID": "my-job", + "JobVersion": 2, + "Stable": true +} +``` + +### Sample Request + +```text +$ curl \ + --request POST \ + --payload @payload.json \ + https://nomad.rocks/v1/job/my-job/stable +``` + +### Sample Response + +```json +{ + "JobModifyIndex": 34, +} +``` + ## Create Job Evaluation @@ -924,7 +1178,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -963,7 +1217,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in - the job file during submission). This is specified as part of the path. - `Job` `(string: )` - Specifies the JSON definition of the job. @@ -1141,14 +1395,14 @@ $ curl \ which in turn contain Task Diffs. Each of these objects then has Object and Field Diff structures embedded. -- `NextPeriodicLaunch`- If the job being planned is periodic, this field will +- `NextPeriodicLaunch` - If the job being planned is periodic, this field will include the next launch time for the job. - `CreatedEvals` - A set of evaluations that were created as a result of the dry-run. These evaluations can signify a follow-up rolling update evaluation or a blocked evaluation. -- `JobModifyIndex`- The `JobModifyIndex` of the server side version of this job. +- `JobModifyIndex` - The `JobModifyIndex` of the server side version of this job. - `FailedTGAllocs` - A set of metrics to understand any allocation failures that occurred for the Task Group. @@ -1178,7 +1432,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request @@ -1216,7 +1470,7 @@ The table below shows this endpoint's support for ### Parameters -- `:job_id` `(string: )`- Specifies the ID of the job (as specified in +- `:job_id` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. ### Sample Request From 4e4c4a0f46b7ef83f23e07e402bae4e1f8b0499e Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 15:19:07 -0700 Subject: [PATCH 082/105] feedback --- nomad/job_endpoint.go | 4 ++-- nomad/state/state_store.go | 6 ++++-- website/source/api/jobs.html.md | 8 ++++---- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 1f6b2faf1b5d..1c078aba9c19 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -416,10 +416,10 @@ func (j *Job) Stable(args *structs.JobStabilityRequest, reply *structs.JobStabil return fmt.Errorf("job %q at version %d not found", args.JobID, args.JobVersion) } - // Commit this evaluation via Raft + // Commit this stability request via Raft _, modifyIndex, err := j.srv.raftApply(structs.JobStabilityRequestType, args) if err != nil { - j.srv.logger.Printf("[ERR] nomad.job: Eval create failed: %v", err) + j.srv.logger.Printf("[ERR] nomad.job: Job stability request failed: %v", err) return err } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index a2f4e2fb8999..ad9b46074040 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -873,13 +873,15 @@ func (s *StateStore) jobVersionByID(txn *memdb.Txn, ws *memdb.WatchSet, id strin return all, nil } -// JobByIDAndVersion returns the job identified by its ID and Version +// JobByIDAndVersion returns the job identified by its ID and Version. The +// passed watchset may be nil. func (s *StateStore) JobByIDAndVersion(ws memdb.WatchSet, id string, version uint64) (*structs.Job, error) { txn := s.db.Txn(false) return s.jobByIDAndVersionImpl(ws, id, version, txn) } -// jobByIDAndVersionImpl returns the job identified by its ID and Version +// jobByIDAndVersionImpl returns the job identified by its ID and Version. The +// passed watchset may be nil. func (s *StateStore) jobByIDAndVersionImpl(ws memdb.WatchSet, id string, version uint64, txn *memdb.Txn) (*structs.Job, error) { watchCh, existing, err := txn.FirstWatch("job_version", "id", id, version) if err != nil { diff --git a/website/source/api/jobs.html.md b/website/source/api/jobs.html.md index a241887ab7ff..f368b9d7344f 100644 --- a/website/source/api/jobs.html.md +++ b/website/source/api/jobs.html.md @@ -821,7 +821,7 @@ This endpoint returns a single job's most recent deployment. | Method | Path | Produces | | ------ | ----------------------------- | -------------------------- | -| `GET` | `/v1/job/:job_id/deployment` | `application/json` | +| `GET` | `/v1/job/:job_id/deployment` | `application/json` | The table below shows this endpoint's support for [blocking queries](/api/index.html#blocking-queries) and @@ -1070,7 +1070,7 @@ The table below shows this endpoint's support for - `JobID` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. -- `JobVersion` `(integer: 0) - Specifies the job version to revert to. +- `JobVersion` `(integer: 0)` - Specifies the job version to revert to. - `EnforcePriorVersion` `(integer: nil)` - Optional value specifying the current job's version. This is checked and acts as a check-and-set value before @@ -1111,7 +1111,7 @@ This endpoint sets the job's stability. | Method | Path | Produces | | ------- | -------------------------- | -------------------------- | -| `POST` | `/v1/job/:job_id/stable` | `application/json` | +| `POST` | `/v1/job/:job_id/stable` | `application/json` | The table below shows this endpoint's support for [blocking queries](/api/index.html#blocking-queries) and @@ -1126,7 +1126,7 @@ The table below shows this endpoint's support for - `JobID` `(string: )` - Specifies the ID of the job (as specified in the job file during submission). This is specified as part of the path. -- `JobVersion` `(integer: 0) - Specifies the job version to set the stability on. +- `JobVersion` `(integer: 0)` - Specifies the job version to set the stability on. - `Stable` `(bool: false)` - Specifies whether the job should be marked as stable or not. From 34679cfd3669af85ca1a6d1c171da37361ad5b47 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 13:55:44 -0700 Subject: [PATCH 083/105] add reverted job version to deployment update response --- nomad/structs/structs.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 3b67f2f27d80..79c2b8028ccd 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -878,6 +878,11 @@ type DeploymentUpdateResponse struct { EvalID string EvalCreateIndex uint64 DeploymentModifyIndex uint64 + + // RevertedJobVersion is the version the job was reverted to. If unset, the + // job wasn't reverted + RevertedJobVersion *uint64 + WriteMeta } From fec6f2ef57b5464b29cc3aa77b06914b226362e5 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 15:03:27 -0700 Subject: [PATCH 084/105] Return the reverted job version --- api/deployments.go | 1 + command/deployment_fail.go | 7 +- nomad/deployment_endpoint_test.go | 160 ++++++++++++++++++ nomad/deploymentwatcher/deployment_watcher.go | 42 ++++- 4 files changed, 202 insertions(+), 8 deletions(-) diff --git a/api/deployments.go b/api/deployments.go index 5545c88d58d8..12a9a9be7ae6 100644 --- a/api/deployments.go +++ b/api/deployments.go @@ -170,6 +170,7 @@ type DeploymentUpdateResponse struct { EvalID string EvalCreateIndex uint64 DeploymentModifyIndex uint64 + RevertedJobVersion *uint64 WriteMeta } diff --git a/command/deployment_fail.go b/command/deployment_fail.go index fdd3c59ba616..543641e140f5 100644 --- a/command/deployment_fail.go +++ b/command/deployment_fail.go @@ -91,7 +91,12 @@ func (c *DeploymentFailCommand) Run(args []string) int { return 1 } - c.Ui.Output(fmt.Sprintf("Deployment %q failed", deploy.ID)) + if u.RevertedJobVersion == nil { + c.Ui.Output(fmt.Sprintf("Deployment %q failed", deploy.ID)) + } else { + c.Ui.Output(fmt.Sprintf("Deployment %q failed. Auto-reverted to job version %d.", deploy.ID, *u.RevertedJobVersion)) + } + evalCreated := u.EvalID != "" // Nothing to do diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index fb244b149caa..1a6a2a0c7f02 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -135,6 +135,80 @@ func TestDeploymentEndpoint_Fail(t *testing.T) { assert.Equal(dout.ModifyIndex, resp.DeploymentModifyIndex, "wrong modify index") } +func TestDeploymentEndpoint_Fail_Rollback(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + state := s1.fsm.State() + + // Create the original job + j := mock.Job() + j.Stable = true + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.AutoRevert = true + assert.Nil(state.UpsertJob(998, j), "UpsertJob") + + // Create the second job, deployment and alloc + j2 := j.Copy() + j2.Stable = false + + d := mock.Deployment() + d.TaskGroups["web"].AutoRevert = true + d.JobID = j2.ID + d.JobVersion = j2.Version + + a := mock.Alloc() + a.JobID = j.ID + a.DeploymentID = d.ID + + assert.Nil(state.UpsertJob(999, j2), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") + + // Mark the deployment as failed + req := &structs.DeploymentFailRequest{ + DeploymentID: d.ID, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.DeploymentUpdateResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.Fail", req, &resp), "RPC") + assert.NotEqual(resp.Index, uint64(0), "bad response index") + assert.NotNil(resp.RevertedJobVersion, "bad revert version") + assert.EqualValues(0, *resp.RevertedJobVersion, "bad revert version") + + // Lookup the evaluation + ws := memdb.NewWatchSet() + eval, err := state.EvalByID(ws, resp.EvalID) + assert.Nil(err, "EvalByID failed") + assert.NotNil(eval, "Expect eval") + assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerDeploymentWatcher, "eval trigger") + assert.Equal(eval.JobID, d.JobID, "eval job id") + assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") + assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") + + // Lookup the deployment + expectedDesc := structs.DeploymentStatusDescriptionRollback(structs.DeploymentStatusDescriptionFailedByUser, 0) + dout, err := state.DeploymentByID(ws, d.ID) + assert.Nil(err, "DeploymentByID failed") + assert.Equal(dout.Status, structs.DeploymentStatusFailed, "wrong status") + assert.Equal(dout.StatusDescription, expectedDesc, "wrong status description") + assert.Equal(resp.DeploymentModifyIndex, dout.ModifyIndex, "wrong modify index") + + // Lookup the job + jout, err := state.JobByID(ws, j.ID) + assert.Nil(err, "JobByID") + assert.NotNil(jout, "job") + assert.EqualValues(2, jout.Version, "reverted job version") +} + func TestDeploymentEndpoint_Pause(t *testing.T) { s1 := testServer(t, func(c *Config) { c.NumSchedulers = 0 // Prevent automatic dequeue @@ -305,6 +379,92 @@ func TestDeploymentEndpoint_SetAllocHealth(t *testing.T) { assert.True(*aout.DeploymentStatus.Healthy, "alloc deployment healthy") } +func TestDeploymentEndpoint_SetAllocHealth_Rollback(t *testing.T) { + s1 := testServer(t, func(c *Config) { + c.NumSchedulers = 0 // Prevent automatic dequeue + }) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + assert := assert.New(t) + state := s1.fsm.State() + + // Create the original job + j := mock.Job() + j.Stable = true + j.TaskGroups[0].Update = structs.DefaultUpdateStrategy.Copy() + j.TaskGroups[0].Update.MaxParallel = 2 + j.TaskGroups[0].Update.AutoRevert = true + assert.Nil(state.UpsertJob(998, j), "UpsertJob") + + // Create the second job, deployment and alloc + j2 := j.Copy() + j2.Stable = false + + d := mock.Deployment() + d.TaskGroups["web"].AutoRevert = true + d.JobID = j2.ID + d.JobVersion = j2.Version + + a := mock.Alloc() + a.JobID = j.ID + a.DeploymentID = d.ID + + assert.Nil(state.UpsertJob(999, j2), "UpsertJob") + assert.Nil(state.UpsertDeployment(1000, d), "UpsertDeployment") + assert.Nil(state.UpsertAllocs(1001, []*structs.Allocation{a}), "UpsertAllocs") + + // Set the alloc as unhealthy + req := &structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + UnhealthyAllocationIDs: []string{a.ID}, + WriteRequest: structs.WriteRequest{Region: "global"}, + } + + // Fetch the response + var resp structs.DeploymentUpdateResponse + assert.Nil(msgpackrpc.CallWithCodec(codec, "Deployment.SetAllocHealth", req, &resp), "RPC") + assert.NotZero(resp.Index, "bad response index") + assert.NotNil(resp.RevertedJobVersion, "bad revert version") + assert.EqualValues(0, *resp.RevertedJobVersion, "bad revert version") + + // Lookup the evaluation + ws := memdb.NewWatchSet() + eval, err := state.EvalByID(ws, resp.EvalID) + assert.Nil(err, "EvalByID failed") + assert.NotNil(eval, "Expect eval") + assert.Equal(eval.CreateIndex, resp.EvalCreateIndex, "eval index mismatch") + assert.Equal(eval.TriggeredBy, structs.EvalTriggerDeploymentWatcher, "eval trigger") + assert.Equal(eval.JobID, d.JobID, "eval job id") + assert.Equal(eval.DeploymentID, d.ID, "eval deployment id") + assert.Equal(eval.Status, structs.EvalStatusPending, "eval status") + + // Lookup the deployment + expectedDesc := structs.DeploymentStatusDescriptionRollback(structs.DeploymentStatusDescriptionFailedAllocations, 0) + dout, err := state.DeploymentByID(ws, d.ID) + assert.Nil(err, "DeploymentByID failed") + assert.Equal(dout.Status, structs.DeploymentStatusFailed, "wrong status") + assert.Equal(dout.StatusDescription, expectedDesc, "wrong status description") + assert.Equal(resp.DeploymentModifyIndex, dout.ModifyIndex, "wrong modify index") + assert.Len(dout.TaskGroups, 1, "should have one group") + assert.Contains(dout.TaskGroups, "web", "should have web group") + assert.Equal(1, dout.TaskGroups["web"].UnhealthyAllocs, "should have one healthy") + + // Lookup the allocation + aout, err := state.AllocByID(ws, a.ID) + assert.Nil(err, "AllocByID") + assert.NotNil(aout, "alloc") + assert.NotNil(aout.DeploymentStatus, "alloc deployment status") + assert.NotNil(aout.DeploymentStatus.Healthy, "alloc deployment healthy") + assert.False(*aout.DeploymentStatus.Healthy, "alloc deployment healthy") + + // Lookup the job + jout, err := state.JobByID(ws, j.ID) + assert.Nil(err, "JobByID") + assert.NotNil(jout, "job") + assert.EqualValues(2, jout.Version, "reverted job version") +} + func TestDeploymentEndpoint_List(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() diff --git a/nomad/deploymentwatcher/deployment_watcher.go b/nomad/deploymentwatcher/deployment_watcher.go index 1e751d3ca917..be7af7b13c69 100644 --- a/nomad/deploymentwatcher/deployment_watcher.go +++ b/nomad/deploymentwatcher/deployment_watcher.go @@ -2,13 +2,13 @@ package deploymentwatcher import ( "context" - "fmt" "log" "sync" "time" "golang.org/x/time/rate" + "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" ) @@ -142,7 +142,9 @@ func (w *deploymentWatcher) SetAllocHealth( return err } - desc = fmt.Sprintf("%s - rolling back to job version %d", desc, j.Version) + if j != nil { + desc = structs.DeploymentStatusDescriptionRollback(desc, j.Version) + } break } @@ -167,6 +169,9 @@ func (w *deploymentWatcher) SetAllocHealth( resp.EvalCreateIndex = index resp.DeploymentModifyIndex = index resp.Index = index + if j != nil { + resp.RevertedJobVersion = helper.Uint64ToPtr(j.Version) + } w.setLatestEval(index) return nil } @@ -231,14 +236,34 @@ func (w *deploymentWatcher) FailDeployment( req *structs.DeploymentFailRequest, resp *structs.DeploymentUpdateResponse) error { - // Determine the status we should transistion to and if we need to create an - // evaluation status, desc := structs.DeploymentStatusFailed, structs.DeploymentStatusDescriptionFailedByUser - update := w.getDeploymentStatusUpdate(status, desc) - eval := w.getEval() + + // Determine if we should rollback + rollback := false + for _, state := range w.d.TaskGroups { + if state.AutoRevert { + rollback = true + break + } + } + + var rollbackJob *structs.Job + if rollback { + var err error + rollbackJob, err = w.latestStableJob() + if err != nil { + return err + } + + if rollbackJob != nil { + desc = structs.DeploymentStatusDescriptionRollback(desc, rollbackJob.Version) + } + } // Commit the change - i, err := w.upsertDeploymentStatusUpdate(update, eval, nil) + update := w.getDeploymentStatusUpdate(status, desc) + eval := w.getEval() + i, err := w.upsertDeploymentStatusUpdate(update, eval, rollbackJob) if err != nil { return err } @@ -248,6 +273,9 @@ func (w *deploymentWatcher) FailDeployment( resp.EvalCreateIndex = i resp.DeploymentModifyIndex = i resp.Index = i + if rollbackJob != nil { + resp.RevertedJobVersion = helper.Uint64ToPtr(rollbackJob.Version) + } w.setLatestEval(i) return nil } From 6925255fe45f33f0ab13681f2572e30a1b170d26 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 15:10:29 -0700 Subject: [PATCH 085/105] add to docs --- website/source/api/deployments.html.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/source/api/deployments.html.md b/website/source/api/deployments.html.md index 8154efb604d9..4e3c71d0fd41 100644 --- a/website/source/api/deployments.html.md +++ b/website/source/api/deployments.html.md @@ -297,6 +297,7 @@ $ curl \ "EvalID": "0d834913-58a0-81ac-6e33-e452d83a0c66", "EvalCreateIndex": 20, "DeploymentModifyIndex": 20, + "RevertedJobVersion": 1, "Index": 20 } ``` @@ -475,6 +476,7 @@ $ curl \ "EvalID": "0d834913-58a0-81ac-6e33-e452d83a0c66", "EvalCreateIndex": 20, "DeploymentModifyIndex": 20, + "RevertedJobVersion": 1, "Index": 20 } ``` From 78d15809921b958158f2ab550a5fca236909f201 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 15:35:54 -0700 Subject: [PATCH 086/105] Deployment from inplace updates tracks placed properly. --- nomad/state/state_store.go | 2 +- nomad/state/state_store_test.go | 40 +++++++++++++++++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index ad9b46074040..f4d38024d6d8 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2558,7 +2558,7 @@ func (s *StateStore) updateDeploymentWithAlloc(index uint64, alloc, existing *st // the placement existingHealthSet := existing != nil && existing.DeploymentStatus != nil && existing.DeploymentStatus.Healthy != nil allocHealthSet := alloc.DeploymentStatus != nil && alloc.DeploymentStatus.Healthy != nil - if existing == nil { + if existing == nil || existing.DeploymentID != alloc.DeploymentID { placed++ } else if !existingHealthSet && allocHealthSet { if *alloc.DeploymentStatus.Healthy { diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index ac16f9c1affb..db9227fa999f 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -137,6 +137,46 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { if watchFired(ws) { t.Fatalf("bad") } + + // Update the allocs to be part of a new deployment + d2 := d.Copy() + d2.ID = structs.GenerateUUID() + + allocNew := alloc.Copy() + allocNew.DeploymentID = d2.ID + allocNew2 := alloc2.Copy() + allocNew2.DeploymentID = d2.ID + + // Create another plan + res = structs.ApplyPlanResultsRequest{ + AllocUpdateRequest: structs.AllocUpdateRequest{ + Alloc: []*structs.Allocation{allocNew, allocNew2}, + Job: job, + }, + Deployment: d2, + } + + err = state.UpsertPlanResults(1001, &res) + if err != nil { + t.Fatalf("err: %v", err) + } + + dout, err = state.DeploymentByID(ws, d2.ID) + if err != nil { + t.Fatalf("err: %v", err) + } + + if dout == nil { + t.Fatalf("bad: nil deployment") + } + + tg, ok = dout.TaskGroups[alloc.TaskGroup] + if !ok { + t.Fatalf("bad: nil deployment state") + } + if tg == nil || tg.PlacedAllocs != 2 { + t.Fatalf("bad: %v", dout) + } } // This test checks that deployment updates are applied correctly From f0a090c6e8c2a1b29208b6f47e61d9278ca6cd98 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 19:08:51 -0700 Subject: [PATCH 087/105] Disallow update stanza on batch jobs This PR: * disallows update stanzas on batch jobs * undeprecates the stagger field * changes the way warnings are returned --- api/jobs.go | 19 +-- api/jobs_test.go | 11 +- command/agent/job_endpoint.go | 7 +- command/agent/job_endpoint_test.go | 5 +- command/validate.go | 9 +- nomad/job_endpoint.go | 27 ++-- nomad/structs/funcs.go | 36 ++++++ nomad/structs/structs.go | 74 +++++++++-- nomad/structs/structs_test.go | 195 ++++++++++++++++++++++++++--- 9 files changed, 325 insertions(+), 58 deletions(-) diff --git a/api/jobs.go b/api/jobs.go index bbfaa1bccda9..f89b9a5d6518 100644 --- a/api/jobs.go +++ b/api/jobs.go @@ -290,8 +290,7 @@ type periodicForceResponse struct { // UpdateStrategy defines a task groups update strategy. type UpdateStrategy struct { - // COMPAT: Remove in 0.7.0. Stagger is deprecated in 0.6.0. - Stagger time.Duration `mapstructure:"stagger"` + Stagger *time.Duration `mapstructure:"stagger"` MaxParallel *int `mapstructure:"max_parallel"` HealthCheck *string `mapstructure:"health_check"` MinHealthyTime *time.Duration `mapstructure:"min_healthy_time"` @@ -307,8 +306,9 @@ func (u *UpdateStrategy) Copy() *UpdateStrategy { copy := new(UpdateStrategy) - // COMPAT: Remove in 0.7.0. Stagger is deprecated in 0.6.0. - copy.Stagger = u.Stagger + if u.Stagger != nil { + copy.Stagger = helper.TimeToPtr(*u.Stagger) + } if u.MaxParallel != nil { copy.MaxParallel = helper.IntToPtr(*u.MaxParallel) @@ -342,6 +342,10 @@ func (u *UpdateStrategy) Merge(o *UpdateStrategy) { return } + if o.Stagger != nil { + u.Stagger = helper.TimeToPtr(*o.Stagger) + } + if o.MaxParallel != nil { u.MaxParallel = helper.IntToPtr(*o.MaxParallel) } @@ -374,6 +378,10 @@ func (u *UpdateStrategy) Canonicalize() { d := structs.DefaultUpdateStrategy + if u.Stagger == nil { + u.Stagger = helper.TimeToPtr(d.Stagger) + } + if u.HealthCheck == nil { u.HealthCheck = helper.StringToPtr(d.HealthCheck) } @@ -530,9 +538,6 @@ func (j *Job) Canonicalize() { if j.Version == nil { j.Version = helper.Uint64ToPtr(0) } - if j.SubmitTime == nil { - j.SubmitTime = helper.Int64ToPtr(0) - } if j.CreateIndex == nil { j.CreateIndex = helper.Uint64ToPtr(0) } diff --git a/api/jobs_test.go b/api/jobs_test.go index d7af4023ba67..89d4ee313794 100644 --- a/api/jobs_test.go +++ b/api/jobs_test.go @@ -298,6 +298,7 @@ func TestJobs_Canonicalize(t *testing.T) { JobModifyIndex: helper.Uint64ToPtr(0), Datacenters: []string{"dc1"}, Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(30 * time.Second), MaxParallel: helper.IntToPtr(1), HealthCheck: helper.StringToPtr("checks"), MinHealthyTime: helper.TimeToPtr(10 * time.Second), @@ -322,6 +323,7 @@ func TestJobs_Canonicalize(t *testing.T) { }, Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(30 * time.Second), MaxParallel: helper.IntToPtr(1), HealthCheck: helper.StringToPtr("checks"), MinHealthyTime: helper.TimeToPtr(10 * time.Second), @@ -445,6 +447,7 @@ func TestJobs_Canonicalize(t *testing.T) { ID: helper.StringToPtr("bar"), ParentID: helper.StringToPtr("lol"), Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(1 * time.Second), MaxParallel: helper.IntToPtr(1), HealthCheck: helper.StringToPtr("checks"), MinHealthyTime: helper.TimeToPtr(10 * time.Second), @@ -456,6 +459,7 @@ func TestJobs_Canonicalize(t *testing.T) { { Name: helper.StringToPtr("bar"), Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(2 * time.Second), MaxParallel: helper.IntToPtr(2), HealthCheck: helper.StringToPtr("manual"), MinHealthyTime: helper.TimeToPtr(1 * time.Second), @@ -496,6 +500,7 @@ func TestJobs_Canonicalize(t *testing.T) { ModifyIndex: helper.Uint64ToPtr(0), JobModifyIndex: helper.Uint64ToPtr(0), Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(1 * time.Second), MaxParallel: helper.IntToPtr(1), HealthCheck: helper.StringToPtr("checks"), MinHealthyTime: helper.TimeToPtr(10 * time.Second), @@ -519,6 +524,7 @@ func TestJobs_Canonicalize(t *testing.T) { Mode: helper.StringToPtr("delay"), }, Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(2 * time.Second), MaxParallel: helper.IntToPtr(2), HealthCheck: helper.StringToPtr("manual"), MinHealthyTime: helper.TimeToPtr(1 * time.Second), @@ -550,6 +556,7 @@ func TestJobs_Canonicalize(t *testing.T) { Mode: helper.StringToPtr("delay"), }, Update: &UpdateStrategy{ + Stagger: helper.TimeToPtr(1 * time.Second), MaxParallel: helper.IntToPtr(1), HealthCheck: helper.StringToPtr("checks"), MinHealthyTime: helper.TimeToPtr(10 * time.Second), @@ -575,8 +582,7 @@ func TestJobs_Canonicalize(t *testing.T) { t.Run(tc.name, func(t *testing.T) { tc.input.Canonicalize() if !reflect.DeepEqual(tc.input, tc.expected) { - t.Logf("Name: %v, Diffs:\n%v", tc.name, pretty.Diff(tc.expected, tc.input)) - t.Fatalf("Name: %v, expected:\n%#v\nactual:\n%#v", tc.name, tc.expected, tc.input) + t.Fatalf("Name: %v, Diffs:\n%v", tc.name, pretty.Diff(tc.expected, tc.input)) } }) } @@ -666,6 +672,7 @@ func TestJobs_Revert(t *testing.T) { } assertWriteMeta(t, wm) + job.Meta = map[string]string{"foo": "new"} resp, wm, err = jobs.Register(job, nil) if err != nil { t.Fatalf("err: %s", err) diff --git a/command/agent/job_endpoint.go b/command/agent/job_endpoint.go index 4eabb91d4032..6cf9b03a40eb 100644 --- a/command/agent/job_endpoint.go +++ b/command/agent/job_endpoint.go @@ -562,8 +562,10 @@ func ApiJobToStructJob(job *api.Job) *structs.Job { // COMPAT: Remove in 0.7.0. Update has been pushed into the task groups if job.Update != nil { - j.Update = structs.UpdateStrategy{ - Stagger: job.Update.Stagger, + j.Update = structs.UpdateStrategy{} + + if job.Update.Stagger != nil { + j.Update.Stagger = *job.Update.Stagger } if job.Update.MaxParallel != nil { j.Update.MaxParallel = *job.Update.MaxParallel @@ -632,6 +634,7 @@ func ApiTgToStructsTG(taskGroup *api.TaskGroup, tg *structs.TaskGroup) { if taskGroup.Update != nil { tg.Update = &structs.UpdateStrategy{ + Stagger: *taskGroup.Update.Stagger, MaxParallel: *taskGroup.Update.MaxParallel, HealthCheck: *taskGroup.Update.HealthCheck, MinHealthyTime: *taskGroup.Update.MinHealthyTime, diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index af5ade693b68..5488e9568c32 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -886,6 +886,8 @@ func TestHTTP_JobRevert(t *testing.T) { t.Fatalf("err: %v", err) } + // Change the job to get a new version + job.Datacenters = append(job.Datacenters, "foo") if err := s.Agent.RPC("Job.Register", ®Req, ®Resp); err != nil { t.Fatalf("err: %v", err) } @@ -993,7 +995,7 @@ func TestJobs_ApiJobToStructsJob(t *testing.T) { }, }, Update: &api.UpdateStrategy{ - Stagger: 1 * time.Second, + Stagger: helper.TimeToPtr(1 * time.Second), MaxParallel: helper.IntToPtr(5), HealthCheck: helper.StringToPtr(structs.UpdateStrategyHealthCheck_Manual), MinHealthyTime: helper.TimeToPtr(1 * time.Minute), @@ -1227,6 +1229,7 @@ func TestJobs_ApiJobToStructsJob(t *testing.T) { Migrate: true, }, Update: &structs.UpdateStrategy{ + Stagger: 1 * time.Second, MaxParallel: 5, HealthCheck: structs.UpdateStrategyHealthCheck_Checks, MinHealthyTime: 2 * time.Minute, diff --git a/command/validate.go b/command/validate.go index 3135e1ba24d4..6bbdda396b4f 100644 --- a/command/validate.go +++ b/command/validate.go @@ -7,6 +7,7 @@ import ( multierror "github.com/hashicorp/go-multierror" "github.com/hashicorp/nomad/api" "github.com/hashicorp/nomad/command/agent" + "github.com/hashicorp/nomad/nomad/structs" ) type ValidateCommand struct { @@ -104,7 +105,7 @@ func (c *ValidateCommand) validateLocal(aj *api.Job) (*api.JobValidateResponse, var out api.JobValidateResponse job := agent.ApiJobToStructJob(aj) - job.Canonicalize() + canonicalizeWarnings := job.Canonicalize() if vErr := job.Validate(); vErr != nil { if merr, ok := vErr.(*multierror.Error); ok { @@ -118,9 +119,7 @@ func (c *ValidateCommand) validateLocal(aj *api.Job) (*api.JobValidateResponse, } } - if warnings := job.Warnings(); warnings != nil { - out.Warnings = warnings.Error() - } - + warnings := job.Warnings() + out.Warnings = structs.MergeMultierrorWarnings(warnings, canonicalizeWarnings) return &out, nil } diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 1c078aba9c19..72280eee53d5 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -57,7 +57,7 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis } // Initialize the job fields (sets defaults and any necessary init work). - args.Job.Canonicalize() + canonicalizeWarnings := args.Job.Canonicalize() // Add implicit constraints setImplicitConstraints(args.Job) @@ -66,10 +66,11 @@ func (j *Job) Register(args *structs.JobRegisterRequest, reply *structs.JobRegis err, warnings := validateJob(args.Job) if err != nil { return err - } else if warnings != nil { - reply.Warnings = warnings.Error() } + // Set the warning message + reply.Warnings = structs.MergeMultierrorWarnings(warnings, canonicalizeWarnings) + // Lookup the job snap, err := j.srv.fsm.State().Snapshot() if err != nil { @@ -310,6 +311,13 @@ func (j *Job) Summary(args *structs.JobSummaryRequest, func (j *Job) Validate(args *structs.JobValidateRequest, reply *structs.JobValidateResponse) error { defer metrics.MeasureSince([]string{"nomad", "job", "validate"}, time.Now()) + // Initialize the job fields (sets defaults and any necessary init work). + canonicalizeWarnings := args.Job.Canonicalize() + + // Add implicit constraints + setImplicitConstraints(args.Job) + + // Validate the job and capture any warnings err, warnings := validateJob(args.Job) if err != nil { if merr, ok := err.(*multierror.Error); ok { @@ -323,10 +331,8 @@ func (j *Job) Validate(args *structs.JobValidateRequest, reply *structs.JobValid } } - if warnings != nil { - reply.Warnings = warnings.Error() - } - + // Set the warning message + reply.Warnings = structs.MergeMultierrorWarnings(warnings, canonicalizeWarnings) reply.DriverConfigValidated = true return nil } @@ -871,7 +877,7 @@ func (j *Job) Plan(args *structs.JobPlanRequest, reply *structs.JobPlanResponse) } // Initialize the job fields (sets defaults and any necessary init work). - args.Job.Canonicalize() + canonicalizeWarnings := args.Job.Canonicalize() // Add implicit constraints setImplicitConstraints(args.Job) @@ -880,10 +886,11 @@ func (j *Job) Plan(args *structs.JobPlanRequest, reply *structs.JobPlanResponse) err, warnings := validateJob(args.Job) if err != nil { return err - } else if warnings != nil { - reply.Warnings = warnings.Error() } + // Set the warning message + reply.Warnings = structs.MergeMultierrorWarnings(warnings, canonicalizeWarnings) + // Acquire a snapshot of the state snap, err := j.srv.fsm.State().Snapshot() if err != nil { diff --git a/nomad/structs/funcs.go b/nomad/structs/funcs.go index b6c5ef12cae7..56b26d5106ca 100644 --- a/nomad/structs/funcs.go +++ b/nomad/structs/funcs.go @@ -4,8 +4,44 @@ import ( crand "crypto/rand" "fmt" "math" + "strings" + + multierror "github.com/hashicorp/go-multierror" ) +// MergeMultierrorWarnings takes job warnings and canonicalize warnings and +// merges them into a returnable string. Both the errors may be nil. +func MergeMultierrorWarnings(warnings, canonicalizeWarnings error) string { + if warnings == nil && canonicalizeWarnings == nil { + return "" + } + + var warningMsg multierror.Error + if canonicalizeWarnings != nil { + multierror.Append(&warningMsg, canonicalizeWarnings) + } + + if warnings != nil { + multierror.Append(&warningMsg, warnings) + } + + // Set the formatter + warningMsg.ErrorFormat = warningsFormatter + return warningMsg.Error() +} + +// warningsFormatter is used to format job warnings +func warningsFormatter(es []error) string { + points := make([]string, len(es)) + for i, err := range es { + points[i] = fmt.Sprintf("* %s", err) + } + + return fmt.Sprintf( + "%d warning(s):\n\n%s", + len(es), strings.Join(points, "\n")) +} + // RemoveAllocs is used to remove any allocs with the given IDs // from the list of allocations func RemoveAllocs(alloc []*Allocation, remove []*Allocation) []*Allocation { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 79c2b8028ccd..90459b39a0a8 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -1444,8 +1444,10 @@ type Job struct { } // Canonicalize is used to canonicalize fields in the Job. This should be called -// when registering a Job. -func (j *Job) Canonicalize() { +// when registering a Job. A set of warnings are returned if the job was changed +// in anyway that the user should be made aware of. +func (j *Job) Canonicalize() (warnings error) { + var mErr multierror.Error // Ensure that an empty and nil map are treated the same to avoid scheduling // problems since we use reflect DeepEquals. if len(j.Meta) == 0 { @@ -1466,22 +1468,26 @@ func (j *Job) Canonicalize() { // COMPAT: Remove in 0.7.0 // Rewrite any job that has an update block with pre 0.6.0 syntax. - if j.Update.Stagger > 0 && j.Update.MaxParallel > 0 { + jobHasOldUpdate := j.Update.Stagger > 0 && j.Update.MaxParallel > 0 + if jobHasOldUpdate && j.Type != JobTypeBatch { // Build an appropriate update block and copy it down to each task group base := DefaultUpdateStrategy.Copy() base.MaxParallel = j.Update.MaxParallel base.MinHealthyTime = j.Update.Stagger // Add to each task group, modifying as needed + upgraded := false l := len(j.TaskGroups) for _, tg := range j.TaskGroups { // The task group doesn't need upgrading if it has an update block with the new syntax u := tg.Update - if u != nil && u.Stagger == 0 && u.MaxParallel > 0 && + if u != nil && u.Stagger > 0 && u.MaxParallel > 0 && u.HealthCheck != "" && u.MinHealthyTime > 0 && u.HealthyDeadline > 0 { continue } + upgraded = true + // The MaxParallel for the job should be 10% of the total count // unless there is just one task group then we can infer the old // max parallel should be the new @@ -1501,7 +1507,45 @@ func (j *Job) Canonicalize() { tg.Update = tgu } + + if upgraded { + w := "A best effort conversion to new update stanza introduced in v0.6.0 applied. " + + "Please update upgrade stanza before v0.7.0." + multierror.Append(&mErr, fmt.Errorf(w)) + } + } + + // Ensure that the batch job doesn't have new style or old style update + // stanza. Unfortunately are scanning here because we have to deprecate over + // a release so we can't check in the task group since that may be new style + // but wouldn't capture the old style and we don't want to have duplicate + // warnings. + if j.Type == JobTypeBatch { + displayWarning := jobHasOldUpdate + j.Update.Stagger = 0 + j.Update.MaxParallel = 0 + j.Update.HealthCheck = "" + j.Update.MinHealthyTime = 0 + j.Update.HealthyDeadline = 0 + j.Update.AutoRevert = false + j.Update.Canary = 0 + + // Remove any update spec from the task groups + for _, tg := range j.TaskGroups { + if tg.Update != nil { + displayWarning = true + tg.Update = nil + } + } + + if displayWarning { + w := "Update stanza is disallowed for batch jobs since v0.6.0. " + + "The update block has automatically been removed" + multierror.Append(&mErr, fmt.Errorf(w)) + } } + + return mErr.ErrorOrNil() } // Copy returns a deep copy of the Job. It is expected that callers use recover. @@ -1583,7 +1627,7 @@ func (j *Job) Validate() error { // Validate the task group for _, tg := range j.TaskGroups { - if err := tg.Validate(); err != nil { + if err := tg.Validate(j); err != nil { outer := fmt.Errorf("Task group %s validation failed: %v", tg.Name, err) mErr.Errors = append(mErr.Errors, outer) } @@ -1619,11 +1663,6 @@ func (j *Job) Validate() error { // deprecation warnings. func (j *Job) Warnings() error { var mErr multierror.Error - - if j.Update.Stagger > 0 { - multierror.Append(&mErr, fmt.Errorf("Update stagger deprecated. A best effort conversion to new syntax will be applied. Please update upgrade stanza before v0.7.0")) - } - return mErr.ErrorOrNil() } @@ -1916,6 +1955,7 @@ var ( // DefaultUpdateStrategy provides a baseline that can be used to upgrade // jobs with the old policy or for populating field defaults. DefaultUpdateStrategy = &UpdateStrategy{ + Stagger: 30 * time.Second, MaxParallel: 0, HealthCheck: UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, @@ -1927,7 +1967,8 @@ var ( // UpdateStrategy is used to modify how updates are done type UpdateStrategy struct { - // COMPAT: Remove in 0.7.0. Stagger is deprecated in 0.6.0. + // Stagger is used to determine the rate at which allocations are migrated + // due to down or draining nodes. Stagger time.Duration // MaxParallel is how many updates can be done in parallel @@ -1991,6 +2032,9 @@ func (u *UpdateStrategy) Validate() error { if u.HealthyDeadline <= 0 { multierror.Append(&mErr, fmt.Errorf("Healthy deadline must be greater than zero: %v", u.HealthyDeadline)) } + if u.Stagger <= 0 { + multierror.Append(&mErr, fmt.Errorf("Stagger must be greater than zero: %v", u.Stagger)) + } return mErr.ErrorOrNil() } @@ -2427,7 +2471,7 @@ func (tg *TaskGroup) Canonicalize(job *Job) { } // Validate is used to sanity check a task group -func (tg *TaskGroup) Validate() error { +func (tg *TaskGroup) Validate(j *Job) error { var mErr multierror.Error if tg.Name == "" { mErr.Errors = append(mErr.Errors, errors.New("Missing task group name")) @@ -2463,6 +2507,12 @@ func (tg *TaskGroup) Validate() error { // Validate the update strategy if u := tg.Update; u != nil { + switch j.Type { + case JobTypeService, JobTypeSystem: + default: + // COMPAT: Enable in 0.7.0 + //mErr.Errors = append(mErr.Errors, fmt.Errorf("Job type %q does not allow update block", j.Type)) + } if err := u.Validate(); err != nil { mErr.Errors = append(mErr.Errors, err) } diff --git a/nomad/structs/structs_test.go b/nomad/structs/structs_test.go index f6e0328f16c4..00516b9d5f7c 100644 --- a/nomad/structs/structs_test.go +++ b/nomad/structs/structs_test.go @@ -101,18 +101,7 @@ func TestJob_Warnings(t *testing.T) { Name string Job *Job Expected []string - }{ - { - Name: "Old Update spec", - Job: &Job{ - Update: UpdateStrategy{ - MaxParallel: 2, - Stagger: 10 * time.Second, - }, - }, - Expected: []string{"Update stagger deprecated"}, - }, - } + }{} for _, c := range cases { t.Run(c.Name, func(t *testing.T) { @@ -140,10 +129,13 @@ func TestJob_Canonicalize_Update(t *testing.T) { Name string Job *Job Expected *Job + Warnings []string }{ { - Name: "One task group", + Name: "One task group", + Warnings: []string{"conversion to new update stanza"}, Job: &Job{ + Type: JobTypeService, Update: UpdateStrategy{ MaxParallel: 2, Stagger: 10 * time.Second, @@ -156,6 +148,7 @@ func TestJob_Canonicalize_Update(t *testing.T) { }, }, Expected: &Job{ + Type: JobTypeService, Update: UpdateStrategy{ MaxParallel: 2, Stagger: 10 * time.Second, @@ -164,8 +157,10 @@ func TestJob_Canonicalize_Update(t *testing.T) { { Name: "foo", Count: 2, + RestartPolicy: NewRestartPolicy(JobTypeService), EphemeralDisk: DefaultEphemeralDisk(), Update: &UpdateStrategy{ + Stagger: 30 * time.Second, MaxParallel: 2, HealthCheck: UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, @@ -178,8 +173,135 @@ func TestJob_Canonicalize_Update(t *testing.T) { }, }, { - Name: "One task group; too high of parallelism", + Name: "One task group batch", + Warnings: []string{"Update stanza is disallowed for batch jobs"}, Job: &Job{ + Type: JobTypeBatch, + Update: UpdateStrategy{ + MaxParallel: 2, + Stagger: 10 * time.Second, + }, + TaskGroups: []*TaskGroup{ + { + Name: "foo", + Count: 2, + }, + }, + }, + Expected: &Job{ + Type: JobTypeBatch, + Update: UpdateStrategy{}, + TaskGroups: []*TaskGroup{ + { + Name: "foo", + Count: 2, + RestartPolicy: NewRestartPolicy(JobTypeBatch), + EphemeralDisk: DefaultEphemeralDisk(), + }, + }, + }, + }, + { + Name: "One task group batch - new spec", + Warnings: []string{"Update stanza is disallowed for batch jobs"}, + Job: &Job{ + Type: JobTypeBatch, + Update: UpdateStrategy{ + Stagger: 2 * time.Second, + MaxParallel: 2, + Canary: 2, + MinHealthyTime: 2 * time.Second, + HealthyDeadline: 10 * time.Second, + HealthCheck: UpdateStrategyHealthCheck_Checks, + }, + TaskGroups: []*TaskGroup{ + { + Name: "foo", + Count: 2, + Update: &UpdateStrategy{ + Stagger: 2 * time.Second, + MaxParallel: 2, + Canary: 2, + MinHealthyTime: 2 * time.Second, + HealthyDeadline: 10 * time.Second, + HealthCheck: UpdateStrategyHealthCheck_Checks, + }, + }, + }, + }, + Expected: &Job{ + Type: JobTypeBatch, + Update: UpdateStrategy{}, + TaskGroups: []*TaskGroup{ + { + Name: "foo", + Count: 2, + RestartPolicy: NewRestartPolicy(JobTypeBatch), + EphemeralDisk: DefaultEphemeralDisk(), + }, + }, + }, + }, + { + Name: "One task group service - new spec", + Job: &Job{ + Type: JobTypeService, + Update: UpdateStrategy{ + Stagger: 2 * time.Second, + MaxParallel: 2, + Canary: 2, + MinHealthyTime: 2 * time.Second, + HealthyDeadline: 10 * time.Second, + HealthCheck: UpdateStrategyHealthCheck_Checks, + }, + TaskGroups: []*TaskGroup{ + { + Name: "foo", + Count: 2, + Update: &UpdateStrategy{ + Stagger: 2 * time.Second, + MaxParallel: 2, + Canary: 2, + MinHealthyTime: 2 * time.Second, + HealthyDeadline: 10 * time.Second, + HealthCheck: UpdateStrategyHealthCheck_Checks, + }, + }, + }, + }, + Expected: &Job{ + Type: JobTypeService, + Update: UpdateStrategy{ + Stagger: 2 * time.Second, + MaxParallel: 2, + Canary: 2, + MinHealthyTime: 2 * time.Second, + HealthyDeadline: 10 * time.Second, + HealthCheck: UpdateStrategyHealthCheck_Checks, + }, + TaskGroups: []*TaskGroup{ + { + Name: "foo", + Count: 2, + RestartPolicy: NewRestartPolicy(JobTypeService), + EphemeralDisk: DefaultEphemeralDisk(), + Update: &UpdateStrategy{ + Stagger: 2 * time.Second, + MaxParallel: 2, + Canary: 2, + MinHealthyTime: 2 * time.Second, + HealthyDeadline: 10 * time.Second, + HealthCheck: UpdateStrategyHealthCheck_Checks, + }, + }, + }, + }, + }, + { + Name: "One task group; too high of parallelism", + Warnings: []string{"conversion to new update stanza"}, + Job: &Job{ + Type: JobTypeService, Update: UpdateStrategy{ MaxParallel: 200, Stagger: 10 * time.Second, @@ -192,6 +314,7 @@ func TestJob_Canonicalize_Update(t *testing.T) { }, }, Expected: &Job{ + Type: JobTypeService, Update: UpdateStrategy{ MaxParallel: 200, Stagger: 10 * time.Second, @@ -200,8 +323,10 @@ func TestJob_Canonicalize_Update(t *testing.T) { { Name: "foo", Count: 2, + RestartPolicy: NewRestartPolicy(JobTypeService), EphemeralDisk: DefaultEphemeralDisk(), Update: &UpdateStrategy{ + Stagger: 30 * time.Second, MaxParallel: 2, HealthCheck: UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, @@ -214,8 +339,10 @@ func TestJob_Canonicalize_Update(t *testing.T) { }, }, { - Name: "Multiple task group; rounding", + Name: "Multiple task group; rounding", + Warnings: []string{"conversion to new update stanza"}, Job: &Job{ + Type: JobTypeService, Update: UpdateStrategy{ MaxParallel: 2, Stagger: 10 * time.Second, @@ -236,6 +363,7 @@ func TestJob_Canonicalize_Update(t *testing.T) { }, }, Expected: &Job{ + Type: JobTypeService, Update: UpdateStrategy{ MaxParallel: 2, Stagger: 10 * time.Second, @@ -244,8 +372,10 @@ func TestJob_Canonicalize_Update(t *testing.T) { { Name: "foo", Count: 2, + RestartPolicy: NewRestartPolicy(JobTypeService), EphemeralDisk: DefaultEphemeralDisk(), Update: &UpdateStrategy{ + Stagger: 30 * time.Second, MaxParallel: 1, HealthCheck: UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, @@ -257,8 +387,10 @@ func TestJob_Canonicalize_Update(t *testing.T) { { Name: "bar", Count: 14, + RestartPolicy: NewRestartPolicy(JobTypeService), EphemeralDisk: DefaultEphemeralDisk(), Update: &UpdateStrategy{ + Stagger: 30 * time.Second, MaxParallel: 1, HealthCheck: UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, @@ -271,7 +403,9 @@ func TestJob_Canonicalize_Update(t *testing.T) { Name: "foo", Count: 26, EphemeralDisk: DefaultEphemeralDisk(), + RestartPolicy: NewRestartPolicy(JobTypeService), Update: &UpdateStrategy{ + Stagger: 30 * time.Second, MaxParallel: 3, HealthCheck: UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, @@ -287,9 +421,23 @@ func TestJob_Canonicalize_Update(t *testing.T) { for _, c := range cases { t.Run(c.Name, func(t *testing.T) { - c.Job.Canonicalize() + warnings := c.Job.Canonicalize() if !reflect.DeepEqual(c.Job, c.Expected) { - t.Fatalf("Got %# v; want %# v", pretty.Formatter(c.Job), pretty.Formatter(c.Expected)) + t.Fatalf("Diff %#v", pretty.Diff(c.Job, c.Expected)) + } + + wErr := "" + if warnings != nil { + wErr = warnings.Error() + } + for _, w := range c.Warnings { + if !strings.Contains(wErr, w) { + t.Fatalf("Wanted warning %q: got %q", w, wErr) + } + } + + if len(c.Warnings) == 0 && warnings != nil { + t.Fatalf("Wanted no warnings: got %q", wErr) } }) } @@ -636,6 +784,7 @@ func TestJob_RequiredSignals(t *testing.T) { } func TestTaskGroup_Validate(t *testing.T) { + j := testJob() tg := &TaskGroup{ Count: -1, RestartPolicy: &RestartPolicy{ @@ -645,7 +794,7 @@ func TestTaskGroup_Validate(t *testing.T) { Mode: RestartPolicyModeDelay, }, } - err := tg.Validate() + err := tg.Validate(j) mErr := err.(*multierror.Error) if !strings.Contains(mErr.Errors[0].Error(), "group name") { t.Fatalf("err: %s", err) @@ -672,6 +821,7 @@ func TestTaskGroup_Validate(t *testing.T) { Mode: RestartPolicyModeDelay, }, Update: &UpdateStrategy{ + Stagger: 10 * time.Second, MaxParallel: 3, HealthCheck: UpdateStrategyHealthCheck_Manual, MinHealthyTime: 1 * time.Second, @@ -681,7 +831,7 @@ func TestTaskGroup_Validate(t *testing.T) { }, } - err = tg.Validate() + err = tg.Validate(j) mErr = err.(*multierror.Error) if !strings.Contains(mErr.Errors[0].Error(), "should have an ephemeral disk object") { t.Fatalf("err: %s", err) @@ -704,6 +854,13 @@ func TestTaskGroup_Validate(t *testing.T) { if !strings.Contains(mErr.Errors[6].Error(), "Task web validation failed") { t.Fatalf("err: %s", err) } + + // COMPAT: Enable in 0.7.0 + //j.Type = JobTypeBatch + //err = tg.Validate(j) + //if !strings.Contains(err.Error(), "does not allow update block") { + //t.Fatalf("err: %s", err) + //} } func TestTask_Validate(t *testing.T) { From 6639f31c4a24fc4ceb47eeb91bddbb451ac1c906 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 19:20:57 -0700 Subject: [PATCH 088/105] Fix some commands test --- command/alloc_status_test.go | 2 +- command/eval_status_test.go | 2 +- command/inspect_test.go | 2 +- command/job_revert_test.go | 2 +- command/node_status_test.go | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/command/alloc_status_test.go b/command/alloc_status_test.go index a213ce4b22f2..6eb5e7aa1685 100644 --- a/command/alloc_status_test.go +++ b/command/alloc_status_test.go @@ -70,7 +70,7 @@ func TestAllocStatusCommand_Fails(t *testing.T) { if code := cmd.Run([]string{"-address=" + url, "-json", "-t", "{{.ID}}"}); code != 1 { t.Fatalf("expected exit 1, got: %d", code) } - if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both -json and -t are not allowed") { + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both json and template formatting are not allowed") { t.Fatalf("expected getting formatter error, got: %s", out) } } diff --git a/command/eval_status_test.go b/command/eval_status_test.go index b945cf77046b..de5bd6d87cf3 100644 --- a/command/eval_status_test.go +++ b/command/eval_status_test.go @@ -49,7 +49,7 @@ func TestEvalStatusCommand_Fails(t *testing.T) { if code := cmd.Run([]string{"-address=" + url, "-json", "-t", "{{.ID}}"}); code != 1 { t.Fatalf("expected exit 1, got: %d", code) } - if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both -json and -t are not allowed") { + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both json and template formatting are not allowed") { t.Fatalf("expected getting formatter error, got: %s", out) } diff --git a/command/inspect_test.go b/command/inspect_test.go index 8d16106a3497..413e10d5c1ff 100644 --- a/command/inspect_test.go +++ b/command/inspect_test.go @@ -49,7 +49,7 @@ func TestInspectCommand_Fails(t *testing.T) { if code := cmd.Run([]string{"-address=" + url, "-json", "-t", "{{.ID}}"}); code != 1 { t.Fatalf("expected exit 1, got: %d", code) } - if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both -json and -t are not allowed") { + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both json and template formatting are not allowed") { t.Fatalf("expected getting formatter error, got: %s", out) } } diff --git a/command/job_revert_test.go b/command/job_revert_test.go index fbc665bb9f45..01d9e4397dc3 100644 --- a/command/job_revert_test.go +++ b/command/job_revert_test.go @@ -24,7 +24,7 @@ func TestJobRevertCommand_Fails(t *testing.T) { } ui.ErrorWriter.Reset() - if code := cmd.Run([]string{"-address=nope", "foo"}); code != 1 { + if code := cmd.Run([]string{"-address=nope", "foo", "1"}); code != 1 { t.Fatalf("expected exit code 1, got: %d", code) } if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error listing jobs") { diff --git a/command/node_status_test.go b/command/node_status_test.go index 8becbf01247b..0b5f398e4796 100644 --- a/command/node_status_test.go +++ b/command/node_status_test.go @@ -206,7 +206,7 @@ func TestNodeStatusCommand_Fails(t *testing.T) { if code := cmd.Run([]string{"-address=" + url, "-json", "-t", "{{.ID}}"}); code != 1 { t.Fatalf("expected exit 1, got: %d", code) } - if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both -json and -t are not allowed") { + if out := ui.ErrorWriter.String(); !strings.Contains(out, "Both json and template formatting are not allowed") { t.Fatalf("expected getting formatter error, got: %s", out) } } From 53c129493a8213ad354c174c101a4d92650932c8 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 19:29:47 -0700 Subject: [PATCH 089/105] Vendor columnize --- command/job_history.go | 2 +- vendor/github.com/ryanuber/columnize/COPYING | 20 --- .../github.com/ryanuber/columnize/README.md | 12 +- .../ryanuber/columnize/columnize.go | 150 ++++++++++-------- vendor/vendor.json | 12 +- 5 files changed, 93 insertions(+), 103 deletions(-) delete mode 100644 vendor/github.com/ryanuber/columnize/COPYING diff --git a/command/job_history.go b/command/job_history.go index 23dbceac8dfc..e8ca7d64621e 100644 --- a/command/job_history.go +++ b/command/job_history.go @@ -6,8 +6,8 @@ import ( "strings" "time" - "github.com/dadgar/columnize" "github.com/hashicorp/nomad/api" + "github.com/ryanuber/columnize" ) type JobHistoryCommand struct { diff --git a/vendor/github.com/ryanuber/columnize/COPYING b/vendor/github.com/ryanuber/columnize/COPYING deleted file mode 100644 index 86f4501489fa..000000000000 --- a/vendor/github.com/ryanuber/columnize/COPYING +++ /dev/null @@ -1,20 +0,0 @@ -MIT LICENSE - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -"Software"), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: - -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND -NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE -LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION -OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION -WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/ryanuber/columnize/README.md b/vendor/github.com/ryanuber/columnize/README.md index 6852911fcc93..9a583db8db3e 100644 --- a/vendor/github.com/ryanuber/columnize/README.md +++ b/vendor/github.com/ryanuber/columnize/README.md @@ -4,6 +4,7 @@ Columnize Easy column-formatted output for golang [![Build Status](https://travis-ci.org/ryanuber/columnize.svg)](https://travis-ci.org/ryanuber/columnize) +[![GoDoc](https://godoc.org/github.com/ryanuber/columnize?status.svg)](https://godoc.org/github.com/ryanuber/columnize) Columnize is a really small Go package that makes building CLI's a little bit easier. In some CLI designs, you want to output a number similar items in a @@ -55,21 +56,16 @@ config.Delim = "|" config.Glue = " " config.Prefix = "" config.Empty = "" +config.NoTrim = false ``` * `Delim` is the string by which columns of **input** are delimited * `Glue` is the string by which columns of **output** are delimited * `Prefix` is a string by which each line of **output** is prefixed * `Empty` is a string used to replace blank values found in output +* `NoTrim` is a boolean used to disable the automatic trimming of input values You can then pass the `Config` in using the `Format` method (signature below) to have text formatted to your liking. -Usage -===== - -```go -SimpleFormat(intput []string) string - -Format(input []string, config *Config) string -``` +See the [godoc](https://godoc.org/github.com/ryanuber/columnize) page for usage. diff --git a/vendor/github.com/ryanuber/columnize/columnize.go b/vendor/github.com/ryanuber/columnize/columnize.go index cbae439c2bc6..527c1d1b36aa 100644 --- a/vendor/github.com/ryanuber/columnize/columnize.go +++ b/vendor/github.com/ryanuber/columnize/columnize.go @@ -6,6 +6,8 @@ import ( "strings" ) +// Config can be used to tune certain parameters which affect the way +// in which Columnize will format output text. type Config struct { // The string by which the lines of input will be split. Delim string @@ -16,62 +18,60 @@ type Config struct { // The string by which columns of output will be prefixed. Prefix string - // A replacement string to replace empty fields + // A replacement string to replace empty fields. Empty string + // NoTrim disables automatic trimming of inputs. NoTrim bool } -// Returns a Config with default values. +// DefaultConfig returns a *Config with default values. func DefaultConfig() *Config { return &Config{ Delim: "|", Glue: " ", Prefix: "", + Empty: "", + NoTrim: false, } } -// Returns a list of elements, each representing a single item which will -// belong to a column of output. -func getElementsFromLine(config *Config, line string) []interface{} { - seperated := strings.Split(line, config.Delim) - elements := make([]interface{}, len(seperated)) - for i, field := range seperated { - value := field - if !config.NoTrim { - value = strings.TrimSpace(field) - } - if value == "" && config.Empty != "" { - value = config.Empty - } - elements[i] = value +// MergeConfig merges two config objects together and returns the resulting +// configuration. Values from the right take precedence over the left side. +func MergeConfig(a, b *Config) *Config { + // Return quickly if either side was nil + if a == nil { + return b + } + if b == nil { + return a } - return elements -} -// Examines a list of strings and determines how wide each column should be -// considering all of the elements that need to be printed within it. -func getWidthsFromLines(config *Config, lines []string) []int { - widths := make([]int, 0, 8) + var result Config = *a - for _, line := range lines { - elems := getElementsFromLine(config, line) - for i := 0; i < len(elems); i++ { - l := len(elems[i].(string)) - if len(widths) <= i { - widths = append(widths, l) - } else if widths[i] < l { - widths[i] = l - } - } + if b.Delim != "" { + result.Delim = b.Delim } - return widths + if b.Glue != "" { + result.Glue = b.Glue + } + if b.Prefix != "" { + result.Prefix = b.Prefix + } + if b.Empty != "" { + result.Empty = b.Empty + } + if b.NoTrim { + result.NoTrim = true + } + + return &result } -// Given a set of column widths and the number of columns in the current line, -// returns a sprintf-style format string which can be used to print output -// aligned properly with other lines using the same widths set. -func (c *Config) getStringFormat(widths []int, columns int) string { +// stringFormat, given a set of column widths and the number of columns in +// the current line, returns a sprintf-style format string which can be used +// to print output aligned properly with other lines using the same widths set. +func stringFormat(c *Config, widths []int, columns int) string { // Create the buffer with an estimate of the length buf := bytes.NewBuffer(make([]byte, 0, (6+len(c.Glue))*columns)) @@ -90,40 +90,60 @@ func (c *Config) getStringFormat(widths []int, columns int) string { return buf.String() } -// MergeConfig merges two config objects together and returns the resulting -// configuration. Values from the right take precedence over the left side. -func MergeConfig(a, b *Config) *Config { - var result Config = *a +// elementsFromLine returns a list of elements, each representing a single +// item which will belong to a column of output. +func elementsFromLine(config *Config, line string) []interface{} { + separated := strings.Split(line, config.Delim) + elements := make([]interface{}, len(separated)) + for i, field := range separated { + value := field + if !config.NoTrim { + value = strings.TrimSpace(field) + } - // Return quickly if either side was nil - if a == nil || b == nil { - return &result + // Apply the empty value, if configured. + if value == "" && config.Empty != "" { + value = config.Empty + } + elements[i] = value } + return elements +} - if b.Delim != "" { - result.Delim = b.Delim - } - if b.Glue != "" { - result.Glue = b.Glue - } - if b.Prefix != "" { - result.Prefix = b.Prefix - } - if b.Empty != "" { - result.Empty = b.Empty - } - if b.NoTrim { - result.NoTrim = true +// runeLen calculates the number of visible "characters" in a string +func runeLen(s string) int { + l := 0 + for _ = range s { + l++ } + return l +} - return &result +// widthsFromLines examines a list of strings and determines how wide each +// column should be considering all of the elements that need to be printed +// within it. +func widthsFromLines(config *Config, lines []string) []int { + widths := make([]int, 0, 8) + + for _, line := range lines { + elems := elementsFromLine(config, line) + for i := 0; i < len(elems); i++ { + l := runeLen(elems[i].(string)) + if len(widths) <= i { + widths = append(widths, l) + } else if widths[i] < l { + widths[i] = l + } + } + } + return widths } -// Format is the public-facing interface that takes either a plain string -// or a list of strings and returns nicely aligned output. +// Format is the public-facing interface that takes a list of strings and +// returns nicely aligned column-formatted text. func Format(lines []string, config *Config) string { conf := MergeConfig(DefaultConfig(), config) - widths := getWidthsFromLines(conf, lines) + widths := widthsFromLines(conf, lines) // Estimate the buffer size glueSize := len(conf.Glue) @@ -141,13 +161,13 @@ func Format(lines []string, config *Config) string { // Create the formatted output using the format string for _, line := range lines { - elems := getElementsFromLine(conf, line) + elems := elementsFromLine(conf, line) // Get the string format using cache numElems := len(elems) stringfmt, ok := fmtCache[numElems] if !ok { - stringfmt = conf.getStringFormat(widths, numElems) + stringfmt = stringFormat(conf, widths, numElems) fmtCache[numElems] = stringfmt } @@ -165,7 +185,7 @@ func Format(lines []string, config *Config) string { return result } -// Convenience function for using Columnize as easy as possible. +// SimpleFormat is a convenience function to format text with the defaults. func SimpleFormat(lines []string) string { return Format(lines, nil) } diff --git a/vendor/vendor.json b/vendor/vendor.json index 7204ee265db5..7e86fbda007d 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -255,12 +255,6 @@ "revision": "7d649b46cdc2cd2ed102d350688a75a4fd7778c6", "revisionTime": "2016-11-21T13:51:53Z" }, - { - "checksumSHA1": "5qEuqnavYqfukAMm7WimC/cX0HE=", - "path": "github.com/dadgar/columnize", - "revision": "36facc107c7b0258ea8d6e96a45a46087b6d7ef9", - "revisionTime": "2017-06-30T01:36:23Z" - }, { "checksumSHA1": "/5cvgU+J4l7EhMXTK76KaCAfOuU=", "path": "github.com/davecgh/go-spew/spew", @@ -1077,11 +1071,11 @@ "revisionTime": "2016-01-10T10:55:54Z" }, { - "checksumSHA1": "1SC2ACq72a+yfN6CYp5s5woKsR4=", + "checksumSHA1": "M57Rrfc8Z966p+IBtQ91QOcUtcg=", "comment": "v2.0.1-8-g983d3a5", "path": "github.com/ryanuber/columnize", - "revision": "50b9b539927dfe231b8be1caf81b2c81d5940276", - "revisionTime": "2016-06-28T05:25:20Z" + "revision": "abc90934186a77966e2beeac62ed966aac0561d5", + "revisionTime": "2017-07-03T20:58:27Z" }, { "checksumSHA1": "tnMZLo/kR9Kqx6GtmWwowtTLlA8=", From fe990ca3e0e42755cca7b989ec89d8b1ae4d6ebc Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 19:55:58 -0700 Subject: [PATCH 090/105] Fix some tests, eval monitor shows deployment id and deployment cancels based on version --- command/monitor.go | 21 +++++++++++------ nomad/job_endpoint_test.go | 46 ------------------------------------- scheduler/reconcile.go | 2 +- scheduler/reconcile_test.go | 8 +++---- 4 files changed, 19 insertions(+), 58 deletions(-) diff --git a/command/monitor.go b/command/monitor.go index 7e99f24df293..ef0287071d8c 100644 --- a/command/monitor.go +++ b/command/monitor.go @@ -21,13 +21,14 @@ const ( // evalState is used to store the current "state of the world" // in the context of monitoring an evaluation. type evalState struct { - status string - desc string - node string - job string - allocs map[string]*allocState - wait time.Duration - index uint64 + status string + desc string + node string + deployment string + job string + allocs map[string]*allocState + wait time.Duration + index uint64 } // newEvalState creates and initializes a new monitorState @@ -111,6 +112,11 @@ func (m *monitor) update(update *evalState) { m.ui.Output(fmt.Sprintf("Evaluation triggered by job %q", update.job)) } + // Check if the evaluation was triggered by a deployment + if existing.deployment == "" && update.deployment != "" { + m.ui.Output(fmt.Sprintf("Evaluation within deployment: %q", limit(update.deployment, m.length))) + } + // Check the allocations for allocID, alloc := range update.allocs { if existing, ok := existing.allocs[allocID]; !ok { @@ -236,6 +242,7 @@ func (m *monitor) monitor(evalID string, allowPrefix bool) int { state.desc = eval.StatusDescription state.node = eval.NodeID state.job = eval.JobID + state.deployment = eval.DeploymentID state.wait = eval.Wait state.index = eval.CreateIndex diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index ddc6f2357e9e..c811bb9f7585 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -121,36 +121,6 @@ func TestJobEndpoint_Register_InvalidDriverConfig(t *testing.T) { } } -func TestJobEndpoint_Register_UpdateWarning(t *testing.T) { - s1 := testServer(t, func(c *Config) { - c.NumSchedulers = 0 // Prevent automatic dequeue - }) - defer s1.Shutdown() - codec := rpcClient(t, s1) - testutil.WaitForLeader(t, s1.RPC) - - // Create the register request with a job containing an invalid driver - // config - job := mock.Job() - job.Update.Stagger = 1 * time.Second - job.Update.MaxParallel = 1 - req := &structs.JobRegisterRequest{ - Job: job, - WriteRequest: structs.WriteRequest{Region: "global"}, - } - - // Fetch the response - var resp structs.JobRegisterResponse - err := msgpackrpc.CallWithCodec(codec, "Job.Register", req, &resp) - if err != nil { - t.Fatalf("err: %v", err) - } - - if !strings.Contains(resp.Warnings, "Update stagger deprecated") { - t.Fatalf("expected a deprecation warning but got: %v", err) - } -} - func TestJobEndpoint_Register_Payload(t *testing.T) { s1 := testServer(t, func(c *Config) { c.NumSchedulers = 0 // Prevent automatic dequeue @@ -2629,22 +2599,6 @@ func TestJobEndpoint_ValidateJob_InvalidSignals(t *testing.T) { } } -func TestJobEndpoint_ValidateJob_UpdateWarning(t *testing.T) { - // Create a mock job with an invalid config - job := mock.Job() - job.Update.Stagger = 1 * time.Second - job.Update.MaxParallel = 1 - - err, warnings := validateJob(job) - if err != nil { - t.Fatalf("Unexpected validation error; got %v", err) - } - - if !strings.Contains(warnings.Error(), "Update stagger deprecated") { - t.Fatalf("expected a deprecation warning but got: %v", err) - } -} - func TestJobEndpoint_ValidateJobUpdate(t *testing.T) { old := mock.Job() new := mock.Job() diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 9bf435eff42a..2528d076e2f8 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -191,7 +191,7 @@ func (a *allocReconciler) cancelDeployments() { } // Check if the deployment is active and referencing an older job and cancel it - if d.JobCreateIndex != a.job.CreateIndex || d.JobModifyIndex != a.job.JobModifyIndex { + if d.JobCreateIndex != a.job.CreateIndex || d.JobVersion != a.job.Version { if d.Active() { a.result.deploymentUpdates = append(a.result.deploymentUpdates, &structs.DeploymentStatusUpdate{ DeploymentID: a.deployment.ID, diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 498d0e4f5a6e..0798c56f9fb5 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -1219,7 +1219,7 @@ func TestReconciler_CancelDeployment_JobUpdate(t *testing.T) { failed.Status = structs.DeploymentStatusFailed // Make the job newer than the deployment - job.JobModifyIndex += 10 + job.Version += 10 cases := []struct { name string @@ -1872,7 +1872,7 @@ func TestReconciler_StopOldCanaries(t *testing.T) { d.TaskGroups[job.TaskGroups[0].Name] = s // Update the job - job.JobModifyIndex += 10 + job.Version += 10 // Create 10 allocations from the old job var allocs []*structs.Allocation @@ -2735,12 +2735,12 @@ func TestReconciler_FailedDeployment_NewJob(t *testing.T) { // Up the job version jobNew := job.Copy() - jobNew.JobModifyIndex += 100 + jobNew.Version += 100 reconciler := NewAllocReconciler(testLogger(), allocUpdateFnDestructive, false, job.ID, jobNew, d, allocs, nil) r := reconciler.Compute() - dnew := structs.NewDeployment(job) + dnew := structs.NewDeployment(jobNew) dnew.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ DesiredTotal: 10, } From 09af3f98b82d4a01c137a69b66e15d4ceb46600e Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 20:02:33 -0700 Subject: [PATCH 091/105] Fix broken things on deployment promote --- command/deployment_promote.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/command/deployment_promote.go b/command/deployment_promote.go index c0743dc05c05..2112116467d0 100644 --- a/command/deployment_promote.go +++ b/command/deployment_promote.go @@ -58,7 +58,7 @@ func (c *DeploymentPromoteCommand) Run(args []string) int { flags := c.Meta.FlagSet("deployment resume", FlagSetClient) flags.Usage = func() { c.Ui.Output(c.Help()) } - flags.BoolVar(&detach, "all", false, "") + flags.BoolVar(&all, "all", false, "") flags.BoolVar(&detach, "detach", false, "") flags.BoolVar(&verbose, "verbose", false, "") flags.Var((*flaghelper.StringFlag)(&groups), "group", "") @@ -73,7 +73,10 @@ func (c *DeploymentPromoteCommand) Run(args []string) int { c.Ui.Error(c.Help()) return 1 } - + if !all && len(groups) == 0 { + c.Ui.Error("Either -all or one or more -group flags must be specified.") + return 1 + } dID := args[0] // Truncate the id unless full length is requested @@ -119,7 +122,6 @@ func (c *DeploymentPromoteCommand) Run(args []string) int { return 0 } - c.Ui.Output("") mon := newMonitor(c.Ui, client, length) return mon.monitor(u.EvalID, false) } From 5bcf2dd1f0df0586708e731ca21ce7a72e2d7dd0 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 20:06:50 -0700 Subject: [PATCH 092/105] deployment status indicates whether canaries were promoted --- command/deployment_status.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/command/deployment_status.go b/command/deployment_status.go index e75508a38c87..c38ba4b571c0 100644 --- a/command/deployment_status.go +++ b/command/deployment_status.go @@ -175,7 +175,7 @@ func formatDeployment(d *api.Deployment, uuidLength int) string { } rowString += "Desired|" if canaries { - rowString += "Canaries|" + rowString += "Canaries|Promoted|" } rowString += "Placed|Healthy|Unhealthy" @@ -190,6 +190,7 @@ func formatDeployment(d *api.Deployment, uuidLength int) string { row += fmt.Sprintf("%d|", state.DesiredTotal) if canaries { row += fmt.Sprintf("%d|", state.DesiredCanaries) + row += fmt.Sprintf("%v|", state.Promoted) } row += fmt.Sprintf("%d|%d|%d", state.PlacedAllocs, state.HealthyAllocs, state.UnhealthyAllocs) rows[i] = row From f3ffc3de4d31858da7bd22f9b848d90438051029 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 20:11:29 -0700 Subject: [PATCH 093/105] Fix JobModifyIndex changing when job is marked stable --- nomad/state/state_store.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index f4d38024d6d8..6953d1fa2fb5 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -575,12 +575,12 @@ func (s *StateStore) upsertJobImpl(index uint64, job *structs.Job, keepVersion b if existing != nil { job.CreateIndex = existing.(*structs.Job).CreateIndex job.ModifyIndex = index - job.JobModifyIndex = index // Bump the version unless asked to keep it. This should only be done // when changing an internal field such as Stable. A spec change should // always come with a version bump if !keepVersion { + job.JobModifyIndex = index job.Version = existing.(*structs.Job).Version + 1 } From c5b1e2dcc183b98d6cf6ce6921e88cd90c340f85 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 21:51:13 -0700 Subject: [PATCH 094/105] alloc-list shows version --- api/allocations.go | 1 + command/alloc_status.go | 16 +------- command/fs.go | 15 +------- command/logs.go | 15 +------- command/node_status.go | 28 ++------------ command/status.go | 80 +++++++++++++++++++++++++++++++++------- nomad/structs/structs.go | 2 + 7 files changed, 79 insertions(+), 78 deletions(-) diff --git a/api/allocations.go b/api/allocations.go index a6efe4790144..203fdc0646f1 100644 --- a/api/allocations.go +++ b/api/allocations.go @@ -138,6 +138,7 @@ type AllocationListStub struct { Name string NodeID string JobID string + JobVersion uint64 TaskGroup string DesiredStatus string DesiredDescription string diff --git a/command/alloc_status.go b/command/alloc_status.go index c9d420c37073..8ab5866fa004 100644 --- a/command/alloc_status.go +++ b/command/alloc_status.go @@ -135,20 +135,8 @@ func (c *AllocStatusCommand) Run(args []string) int { return 1 } if len(allocs) > 1 { - // Format the allocs - out := make([]string, len(allocs)+1) - out[0] = "ID|Eval ID|Job ID|Task Group|Desired Status|Client Status" - for i, alloc := range allocs { - out[i+1] = fmt.Sprintf("%s|%s|%s|%s|%s|%s", - limit(alloc.ID, length), - limit(alloc.EvalID, length), - alloc.JobID, - alloc.TaskGroup, - alloc.DesiredStatus, - alloc.ClientStatus, - ) - } - c.Ui.Output(fmt.Sprintf("Prefix matched multiple allocations\n\n%s", formatList(out))) + out := formatAllocListStubs(allocs, verbose, length) + c.Ui.Output(fmt.Sprintf("Prefix matched multiple allocations\n\n%s", out)) return 0 } // Prefix lookup matched a single allocation diff --git a/command/fs.go b/command/fs.go index 16fa5a44af5d..01c33e8fb81b 100644 --- a/command/fs.go +++ b/command/fs.go @@ -159,19 +159,8 @@ func (f *FSCommand) Run(args []string) int { } if len(allocs) > 1 { // Format the allocs - out := make([]string, len(allocs)+1) - out[0] = "ID|Eval ID|Job ID|Task Group|Desired Status|Client Status" - for i, alloc := range allocs { - out[i+1] = fmt.Sprintf("%s|%s|%s|%s|%s|%s", - limit(alloc.ID, length), - limit(alloc.EvalID, length), - alloc.JobID, - alloc.TaskGroup, - alloc.DesiredStatus, - alloc.ClientStatus, - ) - } - f.Ui.Output(fmt.Sprintf("Prefix matched multiple allocations\n\n%s", formatList(out))) + out := formatAllocListStubs(allocs, verbose, length) + f.Ui.Output(fmt.Sprintf("Prefix matched multiple allocations\n\n%s", out)) return 0 } // Prefix lookup matched a single allocation diff --git a/command/logs.go b/command/logs.go index 6b3d2bb4e81a..7a86476b222d 100644 --- a/command/logs.go +++ b/command/logs.go @@ -135,19 +135,8 @@ func (l *LogsCommand) Run(args []string) int { } if len(allocs) > 1 { // Format the allocs - out := make([]string, len(allocs)+1) - out[0] = "ID|Eval ID|Job ID|Task Group|Desired Status|Client Status" - for i, alloc := range allocs { - out[i+1] = fmt.Sprintf("%s|%s|%s|%s|%s|%s", - limit(alloc.ID, length), - limit(alloc.EvalID, length), - alloc.JobID, - alloc.TaskGroup, - alloc.DesiredStatus, - alloc.ClientStatus, - ) - } - l.Ui.Output(fmt.Sprintf("Prefix matched multiple allocations\n\n%s", formatList(out))) + out := formatAllocListStubs(allocs, verbose, length) + l.Ui.Output(fmt.Sprintf("Prefix matched multiple allocations\n\n%s", out)) return 0 } // Prefix lookup matched a single allocation diff --git a/command/node_status.go b/command/node_status.go index d3cba2bbda04..6f9a66ee22ec 100644 --- a/command/node_status.go +++ b/command/node_status.go @@ -340,16 +340,14 @@ func (c *NodeStatusCommand) formatNode(client *api.Client, node *api.Node) int { } } - allocs, err := getAllocs(client, node, c.length) + nodeAllocs, _, err := client.Nodes().Allocations(node.ID, nil) if err != nil { c.Ui.Error(fmt.Sprintf("Error querying node allocations: %s", err)) return 1 } - if len(allocs) > 1 { - c.Ui.Output(c.Colorize().Color("\n[bold]Allocations[reset]")) - c.Ui.Output(formatList(allocs)) - } + c.Ui.Output(c.Colorize().Color("\n[bold]Allocations[reset]")) + c.Ui.Output(formatAllocList(nodeAllocs, c.verbose, c.length)) if c.verbose { c.formatAttributes(node) @@ -453,26 +451,6 @@ func getRunningAllocs(client *api.Client, nodeID string) ([]*api.Allocation, err return allocs, err } -// getAllocs returns information about every running allocation on the node -func getAllocs(client *api.Client, node *api.Node, length int) ([]string, error) { - var allocs []string - // Query the node allocations - nodeAllocs, _, err := client.Nodes().Allocations(node.ID, nil) - // Format the allocations - allocs = make([]string, len(nodeAllocs)+1) - allocs[0] = "ID|Eval ID|Job ID|Task Group|Desired Status|Client Status" - for i, alloc := range nodeAllocs { - allocs[i+1] = fmt.Sprintf("%s|%s|%s|%s|%s|%s", - limit(alloc.ID, length), - limit(alloc.EvalID, length), - alloc.JobID, - alloc.TaskGroup, - alloc.DesiredStatus, - alloc.ClientStatus) - } - return allocs, err -} - // getAllocatedResources returns the resource usage of the node. func getAllocatedResources(client *api.Client, runningAllocs []*api.Allocation, node *api.Node) []string { // Compute the total diff --git a/command/status.go b/command/status.go index 9dffc6ff852a..a970abc145a0 100644 --- a/command/status.go +++ b/command/status.go @@ -279,7 +279,6 @@ func (c *StatusCommand) outputParameterizedInfo(client *api.Client, job *api.Job // outputJobInfo prints information about the passed non-periodic job. If a // request fails, an error is returned. func (c *StatusCommand) outputJobInfo(client *api.Client, job *api.Job) error { - var evals, allocs []string // Query the allocations jobAllocs, _, err := client.Jobs().Allocations(*job.ID, c.allAllocs, nil) @@ -304,7 +303,7 @@ func (c *StatusCommand) outputJobInfo(client *api.Client, job *api.Job) error { blockedEval := false // Format the evals - evals = make([]string, len(jobEvals)+1) + evals := make([]string, len(jobEvals)+1) evals[0] = "ID|Priority|Triggered By|Status|Placement Failures" for i, eval := range jobEvals { failures, _ := evalFailureStatus(eval) @@ -341,25 +340,80 @@ func (c *StatusCommand) outputJobInfo(client *api.Client, job *api.Job) error { // Format the allocs c.Ui.Output(c.Colorize().Color("\n[bold]Allocations[reset]")) - if len(jobAllocs) > 0 { - allocs = make([]string, len(jobAllocs)+1) - allocs[0] = "ID|Eval ID|Node ID|Task Group|Desired|Status|Created At" - for i, alloc := range jobAllocs { - allocs[i+1] = fmt.Sprintf("%s|%s|%s|%s|%s|%s|%s", - limit(alloc.ID, c.length), - limit(alloc.EvalID, c.length), - limit(alloc.NodeID, c.length), + c.Ui.Output(formatAllocListStubs(jobAllocs, c.verbose, c.length)) + return nil +} + +func formatAllocListStubs(stubs []*api.AllocationListStub, verbose bool, uuidLength int) string { + if len(stubs) == 0 { + return "No allocations placed" + } + + allocs := make([]string, len(stubs)+1) + if verbose { + allocs[0] = "ID|Eval ID|Node ID|Task Group|Version|Desired|Status|Created At" + for i, alloc := range stubs { + allocs[i+1] = fmt.Sprintf("%s|%s|%s|%s|%d|%s|%s|%s", + limit(alloc.ID, uuidLength), + limit(alloc.EvalID, uuidLength), + limit(alloc.NodeID, uuidLength), + alloc.TaskGroup, + alloc.JobVersion, + alloc.DesiredStatus, + alloc.ClientStatus, + formatUnixNanoTime(alloc.CreateTime)) + } + } else { + allocs[0] = "ID|Node ID|Task Group|Version|Desired|Status|Created At" + for i, alloc := range stubs { + allocs[i+1] = fmt.Sprintf("%s|%s|%s|%d|%s|%s|%s", + limit(alloc.ID, uuidLength), + limit(alloc.NodeID, uuidLength), alloc.TaskGroup, + alloc.JobVersion, alloc.DesiredStatus, alloc.ClientStatus, formatUnixNanoTime(alloc.CreateTime)) } + } + + return formatList(allocs) +} - c.Ui.Output(formatList(allocs)) +func formatAllocList(allocations []*api.Allocation, verbose bool, uuidLength int) string { + if len(allocations) == 0 { + return "No allocations placed" + } + + allocs := make([]string, len(allocations)+1) + if verbose { + allocs[0] = "ID|Eval ID|Node ID|Task Group|Version|Desired|Status|Created At" + for i, alloc := range allocations { + allocs[i+1] = fmt.Sprintf("%s|%s|%s|%s|%d|%s|%s|%s", + limit(alloc.ID, uuidLength), + limit(alloc.EvalID, uuidLength), + limit(alloc.NodeID, uuidLength), + alloc.TaskGroup, + *alloc.Job.Version, + alloc.DesiredStatus, + alloc.ClientStatus, + formatUnixNanoTime(alloc.CreateTime)) + } } else { - c.Ui.Output("No allocations placed") + allocs[0] = "ID|Node ID|Task Group|Version|Desired|Status|Created At" + for i, alloc := range allocations { + allocs[i+1] = fmt.Sprintf("%s|%s|%s|%d|%s|%s|%s", + limit(alloc.ID, uuidLength), + limit(alloc.NodeID, uuidLength), + alloc.TaskGroup, + *alloc.Job.Version, + alloc.DesiredStatus, + alloc.ClientStatus, + formatUnixNanoTime(alloc.CreateTime)) + } } - return nil + + return formatList(allocs) } // outputJobSummary displays the given jobs summary and children job summary diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 90459b39a0a8..01f53a041946 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -4375,6 +4375,7 @@ func (a *Allocation) Stub() *AllocListStub { Name: a.Name, NodeID: a.NodeID, JobID: a.JobID, + JobVersion: a.Job.Version, TaskGroup: a.TaskGroup, DesiredStatus: a.DesiredStatus, DesiredDescription: a.DesiredDescription, @@ -4395,6 +4396,7 @@ type AllocListStub struct { Name string NodeID string JobID string + JobVersion uint64 TaskGroup string DesiredStatus string DesiredDescription string From e5d8c0888fc9bff9c4ab3b951078457ea129987f Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 22:18:44 -0700 Subject: [PATCH 095/105] alloc-status --- api/deployments.go | 1 + command/alloc_status.go | 105 ++++++++++++++++++++++++++++++++-------- 2 files changed, 85 insertions(+), 21 deletions(-) diff --git a/api/deployments.go b/api/deployments.go index 12a9a9be7ae6..6895c360b345 100644 --- a/api/deployments.go +++ b/api/deployments.go @@ -138,6 +138,7 @@ type Deployment struct { // DeploymentState tracks the state of a deployment for a given task group. type DeploymentState struct { + PlacedCanaries []string AutoRevert bool Promoted bool DesiredCanaries int diff --git a/command/alloc_status.go b/command/alloc_status.go index 8ab5866fa004..5a023557f40c 100644 --- a/command/alloc_status.go +++ b/command/alloc_status.go @@ -159,28 +159,12 @@ func (c *AllocStatusCommand) Run(args []string) int { } // Format the allocation data - basic := []string{ - fmt.Sprintf("ID|%s", limit(alloc.ID, length)), - fmt.Sprintf("Eval ID|%s", limit(alloc.EvalID, length)), - fmt.Sprintf("Name|%s", alloc.Name), - fmt.Sprintf("Node ID|%s", limit(alloc.NodeID, length)), - fmt.Sprintf("Job ID|%s", alloc.JobID), - fmt.Sprintf("Client Status|%s", alloc.ClientStatus), - fmt.Sprintf("Client Description|%s", alloc.ClientDescription), - fmt.Sprintf("Desired Status|%s", alloc.DesiredStatus), - fmt.Sprintf("Desired Description|%s", alloc.DesiredDescription), - fmt.Sprintf("Created At|%s", formatUnixNanoTime(alloc.CreateTime)), - } - - if verbose { - basic = append(basic, - fmt.Sprintf("Evaluated Nodes|%d", alloc.Metrics.NodesEvaluated), - fmt.Sprintf("Filtered Nodes|%d", alloc.Metrics.NodesFiltered), - fmt.Sprintf("Exhausted Nodes|%d", alloc.Metrics.NodesExhausted), - fmt.Sprintf("Allocation Time|%s", alloc.Metrics.AllocationTime), - fmt.Sprintf("Failures|%d", alloc.Metrics.CoalescedFailures)) + output, err := formatAllocBasicInfo(alloc, client, length, verbose) + if err != nil { + c.Ui.Error(err.Error()) + return 1 } - c.Ui.Output(formatKV(basic)) + c.Ui.Output(output) if short { c.shortTaskStatus(alloc) @@ -208,6 +192,68 @@ func (c *AllocStatusCommand) Run(args []string) int { return 0 } +func formatAllocBasicInfo(alloc *api.Allocation, client *api.Client, uuidLength int, verbose bool) (string, error) { + basic := []string{ + fmt.Sprintf("ID|%s", limit(alloc.ID, uuidLength)), + fmt.Sprintf("Eval ID|%s", limit(alloc.EvalID, uuidLength)), + fmt.Sprintf("Name|%s", alloc.Name), + fmt.Sprintf("Node ID|%s", limit(alloc.NodeID, uuidLength)), + fmt.Sprintf("Job ID|%s", alloc.JobID), + fmt.Sprintf("Job Version|%d", *alloc.Job.Version), + fmt.Sprintf("Client Status|%s", alloc.ClientStatus), + fmt.Sprintf("Client Description|%s", alloc.ClientDescription), + fmt.Sprintf("Desired Status|%s", alloc.DesiredStatus), + fmt.Sprintf("Desired Description|%s", alloc.DesiredDescription), + fmt.Sprintf("Created At|%s", formatUnixNanoTime(alloc.CreateTime)), + } + + if alloc.DeploymentID != "" { + health := "unset" + if alloc.DeploymentStatus != nil && alloc.DeploymentStatus.Healthy != nil { + if *alloc.DeploymentStatus.Healthy { + health = "healthy" + } else { + health = "unhealthy" + } + } + + basic = append(basic, + fmt.Sprintf("Deployment ID|%s", limit(alloc.DeploymentID, uuidLength)), + fmt.Sprintf("Deployment Health|%s", health)) + + // Check if this allocation is a canary + deployment, _, err := client.Deployments().Info(alloc.DeploymentID, nil) + if err != nil { + return "", fmt.Errorf("Error querying deployment %q: %s", alloc.DeploymentID, err) + } + + canary := false + if state, ok := deployment.TaskGroups[alloc.TaskGroup]; ok { + for _, id := range state.PlacedCanaries { + if id == alloc.ID { + canary = true + break + } + } + } + + if canary { + basic = append(basic, fmt.Sprintf("Canary|%v", true)) + } + } + + if verbose { + basic = append(basic, + fmt.Sprintf("Evaluated Nodes|%d", alloc.Metrics.NodesEvaluated), + fmt.Sprintf("Filtered Nodes|%d", alloc.Metrics.NodesFiltered), + fmt.Sprintf("Exhausted Nodes|%d", alloc.Metrics.NodesExhausted), + fmt.Sprintf("Allocation Time|%s", alloc.Metrics.AllocationTime), + fmt.Sprintf("Failures|%d", alloc.Metrics.CoalescedFailures)) + } + + return formatKV(basic), nil +} + // outputTaskDetails prints task details for each task in the allocation, // optionally printing verbose statistics if displayStats is set func (c *AllocStatusCommand) outputTaskDetails(alloc *api.Allocation, stats *api.AllocResourceUsage, displayStats bool) { @@ -220,9 +266,26 @@ func (c *AllocStatusCommand) outputTaskDetails(alloc *api.Allocation, stats *api } } +func formatTaskTimes(t time.Time) string { + if t.IsZero() { + return "N/A" + } + + return formatTime(t) +} + // outputTaskStatus prints out a list of the most recent events for the given // task state. func (c *AllocStatusCommand) outputTaskStatus(state *api.TaskState) { + basic := []string{ + fmt.Sprintf("Started At|%s", formatTaskTimes(state.StartedAt)), + fmt.Sprintf("Finished At|%s", formatTaskTimes(state.FinishedAt)), + fmt.Sprintf("Total Restarts|%d", state.Restarts)} + + c.Ui.Output("Task Events:") + c.Ui.Output(formatKV(basic)) + c.Ui.Output("") + c.Ui.Output("Recent Events:") events := make([]string, len(state.Events)+1) events[0] = "Time|Type|Description" From 466702a2db92f31372d53cb941c6b7349132c077 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 22:44:05 -0700 Subject: [PATCH 096/105] deployment status in job status view --- command/deployment_status.go | 7 +++++-- command/status.go | 27 +++++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/command/deployment_status.go b/command/deployment_status.go index c38ba4b571c0..736bf5087fa2 100644 --- a/command/deployment_status.go +++ b/command/deployment_status.go @@ -156,7 +156,11 @@ func formatDeployment(d *api.Deployment, uuidLength int) string { return base } base += "\n\n[bold]Deployed[reset]\n" + base += formatDeploymentGroups(d, uuidLength) + return base +} +func formatDeploymentGroups(d *api.Deployment, uuidLength int) string { // Detect if we need to add these columns canaries, autorevert := false, false for _, state := range d.TaskGroups { @@ -197,6 +201,5 @@ func formatDeployment(d *api.Deployment, uuidLength int) string { i++ } - base += formatList(rows) - return base + return formatList(rows) } diff --git a/command/status.go b/command/status.go index a970abc145a0..9a77f9110d35 100644 --- a/command/status.go +++ b/command/status.go @@ -292,6 +292,11 @@ func (c *StatusCommand) outputJobInfo(client *api.Client, job *api.Job) error { return fmt.Errorf("Error querying job evaluations: %s", err) } + latestDeployment, _, err := client.Jobs().LatestDeployment(*job.ID, nil) + if err != nil { + return fmt.Errorf("Error querying latest job deployment: %s", err) + } + // Output the summary if err := c.outputJobSummary(client, job); err != nil { return err @@ -338,12 +343,34 @@ func (c *StatusCommand) outputJobInfo(client *api.Client, job *api.Job) error { c.outputFailedPlacements(latestFailedPlacement) } + if latestDeployment != nil && latestDeployment.Status != "successful" { + c.Ui.Output(c.Colorize().Color("\n[bold]Latest Deployment[reset]")) + c.Ui.Output(c.Colorize().Color(c.formatDeployment(latestDeployment))) + } + // Format the allocs c.Ui.Output(c.Colorize().Color("\n[bold]Allocations[reset]")) c.Ui.Output(formatAllocListStubs(jobAllocs, c.verbose, c.length)) return nil } +func (c *StatusCommand) formatDeployment(d *api.Deployment) string { + // Format the high-level elements + high := []string{ + fmt.Sprintf("ID|%s", limit(d.ID, c.length)), + fmt.Sprintf("Status|%s", d.Status), + fmt.Sprintf("Description|%s", d.StatusDescription), + } + + base := formatKV(high) + if len(d.TaskGroups) == 0 { + return base + } + base += "\n\n[bold]Deployed[reset]\n" + base += formatDeploymentGroups(d, c.length) + return base +} + func formatAllocListStubs(stubs []*api.AllocationListStub, verbose bool, uuidLength int) string { if len(stubs) == 0 { return "No allocations placed" From e1c631064a3b336c2bbab7a829c943452df9ce05 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 23:04:32 -0700 Subject: [PATCH 097/105] @jippi Changed my mind! Good suggestion --- api/tasks.go | 13 +++++++------ client/alloc_runner.go | 1 + command/alloc_status.go | 3 ++- command/status.go | 2 +- nomad/structs/structs.go | 4 ++++ 5 files changed, 15 insertions(+), 8 deletions(-) diff --git a/api/tasks.go b/api/tasks.go index 1659dd43bf28..2b59a89fc6c9 100644 --- a/api/tasks.go +++ b/api/tasks.go @@ -473,12 +473,13 @@ func (t *Task) SetLogConfig(l *LogConfig) *Task { // TaskState tracks the current state of a task and events that caused state // transitions. type TaskState struct { - State string - Failed bool - Restarts uint64 - StartedAt time.Time - FinishedAt time.Time - Events []*TaskEvent + State string + Failed bool + Restarts uint64 + LastRestart time.Time + StartedAt time.Time + FinishedAt time.Time + Events []*TaskEvent } const ( diff --git a/client/alloc_runner.go b/client/alloc_runner.go index efd122627ba4..a545444395d1 100644 --- a/client/alloc_runner.go +++ b/client/alloc_runner.go @@ -584,6 +584,7 @@ func (r *AllocRunner) setTaskState(taskName, state string, event *structs.TaskEv } if event.Type == structs.TaskRestarting { taskState.Restarts++ + taskState.LastRestart = time.Unix(0, event.Time) } r.appendTaskEvent(taskState, event) } diff --git a/command/alloc_status.go b/command/alloc_status.go index 5a023557f40c..caef33743aad 100644 --- a/command/alloc_status.go +++ b/command/alloc_status.go @@ -280,7 +280,8 @@ func (c *AllocStatusCommand) outputTaskStatus(state *api.TaskState) { basic := []string{ fmt.Sprintf("Started At|%s", formatTaskTimes(state.StartedAt)), fmt.Sprintf("Finished At|%s", formatTaskTimes(state.FinishedAt)), - fmt.Sprintf("Total Restarts|%d", state.Restarts)} + fmt.Sprintf("Total Restarts|%d", state.Restarts), + fmt.Sprintf("Last Restart|%s", formatTaskTimes(state.LastRestart))} c.Ui.Output("Task Events:") c.Ui.Output(formatKV(basic)) diff --git a/command/status.go b/command/status.go index 9a77f9110d35..a8bc009cc98f 100644 --- a/command/status.go +++ b/command/status.go @@ -343,7 +343,7 @@ func (c *StatusCommand) outputJobInfo(client *api.Client, job *api.Job) error { c.outputFailedPlacements(latestFailedPlacement) } - if latestDeployment != nil && latestDeployment.Status != "successful" { + if latestDeployment != nil { c.Ui.Output(c.Colorize().Color("\n[bold]Latest Deployment[reset]")) c.Ui.Output(c.Colorize().Color(c.formatDeployment(latestDeployment))) } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 01f53a041946..01a96284340b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3337,6 +3337,10 @@ type TaskState struct { // Restarts is the number of times the task has restarted Restarts uint64 + // LastRestart is the time the task last restarted. It is updated each time the + // task restarts + LastRestart time.Time + // StartedAt is the time the task is started. It is updated each time the // task starts StartedAt time.Time From 27048bb7fadefd9daf1b704c40ed9e340e63dc07 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 23:04:42 -0700 Subject: [PATCH 098/105] events --- api/tasks.go | 1 + website/source/api/allocations.html.md | 83 ++++++++++++++------------ 2 files changed, 47 insertions(+), 37 deletions(-) diff --git a/api/tasks.go b/api/tasks.go index 2b59a89fc6c9..c859850c2a99 100644 --- a/api/tasks.go +++ b/api/tasks.go @@ -501,6 +501,7 @@ const ( TaskSignaling = "Signaling" TaskRestartSignal = "Restart Signaled" TaskLeaderDead = "Leader Task Dead" + TaskBuildingTaskDir = "Building Task Directory" ) // TaskEvent is an event that effects the state of a task and contains meta-data diff --git a/website/source/api/allocations.html.md b/website/source/api/allocations.html.md index ee321f8b4749..82c75490a82f 100644 --- a/website/source/api/allocations.html.md +++ b/website/source/api/allocations.html.md @@ -460,65 +460,74 @@ $ curl \ #### Field Reference - `TaskStates` - A map of tasks to their current state and the latest events - that have effected the state. + that have effected the state. `TaskState` objects contain the following + fields: - A task can be in the following states: + - `State`: The task's current state. It can have one of the following + values: - - `TaskStatePending` - The task is waiting to be run, either for the first - time or due to a restart. + - `TaskStatePending` - The task is waiting to be run, either for the first + time or due to a restart. - - `TaskStateRunning` - The task is currently running. + - `TaskStateRunning` - The task is currently running. - - `TaskStateDead` - The task is dead and will not run again. + - `TaskStateDead` - The task is dead and will not run again. - Further the state contains the `StartedAt` and `FinishedAt` times of the - task. `StartedAt` can be updated multiple times if the task restarts but - `FinishedAt` is set only when the task transitions to `TaskStateDead` + - `StartedAt`: The time the task was last started at. Can be updated through + restarts. -- `Events` - An event contains metadata about the event. The latest 10 events - are stored per task. Each event is timestamped (unix nano-seconds) and has one - of the following types: + - `FinishedAt`: The time the task was finished at. - - `Setup Failure` - The task could not be started because there was a - failure setting up the task prior to it running. + - `LastRestart`: The last time the task was restarted. - - `Driver Failure` - The task could not be started due to a failure in the - driver. + - `Restarts`: The number of times the task has restarted. - - `Started` - The task was started; either for the first time or due to a - restart. + - `Events` - An event contains metadata about the event. The latest 10 events + are stored per task. Each event is timestamped (unix nano-seconds) and has one + of the following types: - - `Terminated` - The task was started and exited. + - `Setup Failure` - The task could not be started because there was a + failure setting up the task prior to it running. - - `Killing` - The task has been sent the kill signal. + - `Driver Failure` - The task could not be started due to a failure in the + driver. - - `Killed` - The task was killed by an user. + - `Started` - The task was started; either for the first time or due to a + restart. - - `Received` - The task has been pulled by the client at the given timestamp. + - `Terminated` - The task was started and exited. - - `Failed Validation` - The task was invalid and as such it didn't run. + - `Killing` - The task has been sent the kill signal. - - `Restarting` - The task terminated and is being restarted. + - `Killed` - The task was killed by an user. - - `Not Restarting` - the task has failed and is not being restarted because - it has exceeded its restart policy. + - `Received` - The task has been pulled by the client at the given timestamp. - - `Downloading Artifacts` - The task is downloading the artifact(s) - - specified in the task. + - `Failed Validation` - The task was invalid and as such it didn't run. - - `Failed Artifact Download` - Artifact(s) specified in the task failed to - download. + - `Restarting` - The task terminated and is being restarted. - - `Restart Signaled` - The task was singled to be restarted. + - `Not Restarting` - the task has failed and is not being restarted because + it has exceeded its restart policy. - - `Signaling` - The task was is being sent a signal. + - `Downloading Artifacts` - The task is downloading the artifact(s) + - specified in the task. - - `Sibling Task Failed` - A task in the same task group failed. + - `Failed Artifact Download` - Artifact(s) specified in the task failed to + download. - - `Leader Task Dead` - The group's leader task is dead. + - `Restart Signaled` - The task was singled to be restarted. - - `Driver` - A message from the driver. + - `Signaling` - The task was is being sent a signal. - - `Task Setup` - Task setup messages. + - `Sibling Task Failed` - A task in the same task group failed. - Depending on the type the event will have applicable annotations. + - `Leader Task Dead` - The group's leader task is dead. + + - `Driver` - A message from the driver. + + - `Task Setup` - Task setup messages. + + - `Building Task Directory` - Task is building its file system. + + Depending on the type the event will have applicable annotations. From 9eb2c8f848fa1b61f904a4b1f2f46d2372c89732 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Thu, 6 Jul 2017 23:30:46 -0700 Subject: [PATCH 099/105] Status description shows requiring promotion --- nomad/state/state_store.go | 5 +++++ nomad/state/state_store_test.go | 4 ++++ nomad/structs/structs.go | 29 ++++++++++++++++++++++------- scheduler/reconcile.go | 9 +++++++++ scheduler/reconcile_test.go | 4 ++++ 5 files changed, 44 insertions(+), 7 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 6953d1fa2fb5..cd7ee3f0badf 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2015,6 +2015,11 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD status.Promoted = true } + // If the deployment no longer needs promotion, update its status + if !copy.RequiresPromotion() && copy.Status == structs.DeploymentStatusRunning { + copy.StatusDescription = structs.DeploymentStatusDescriptionRunning + } + // Insert the deployment if err := s.upsertDeploymentImpl(index, copy, txn); err != nil { return err diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index db9227fa999f..bd168129f5c4 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -5046,6 +5046,7 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { // Create a deployment d := mock.Deployment() + d.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion d.JobID = j.ID d.TaskGroups = map[string]*structs.DeploymentState{ "web": &structs.DeploymentState{ @@ -5104,6 +5105,9 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { if err != nil { t.Fatalf("bad: %v", err) } + if dout.StatusDescription != structs.DeploymentStatusDescriptionRunning { + t.Fatalf("status description not updated: got %v; want %v", dout.StatusDescription, structs.DeploymentStatusDescriptionRunning) + } if len(dout.TaskGroups) != 2 { t.Fatalf("bad: %#v", dout.TaskGroups) } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 01a96284340b..d76f70d4ec3d 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -3971,13 +3971,14 @@ const ( // DeploymentStatusDescriptions are the various descriptions of the states a // deployment can be in. - DeploymentStatusDescriptionRunning = "Deployment is running" - DeploymentStatusDescriptionPaused = "Deployment is paused" - DeploymentStatusDescriptionSuccessful = "Deployment completed successfully" - DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" - DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" - DeploymentStatusDescriptionFailedAllocations = "Failed due to unhealthy allocations" - DeploymentStatusDescriptionFailedByUser = "Deployment marked as failed" + DeploymentStatusDescriptionRunning = "Deployment is running" + DeploymentStatusDescriptionRunningNeedsPromotion = "Deployment is running but requires promotion" + DeploymentStatusDescriptionPaused = "Deployment is paused" + DeploymentStatusDescriptionSuccessful = "Deployment completed successfully" + DeploymentStatusDescriptionStoppedJob = "Cancelled because job is stopped" + DeploymentStatusDescriptionNewerJob = "Cancelled due to newer version of job" + DeploymentStatusDescriptionFailedAllocations = "Failed due to unhealthy allocations" + DeploymentStatusDescriptionFailedByUser = "Deployment marked as failed" ) // DeploymentStatusDescriptionRollback is used to get the status description of @@ -4085,6 +4086,20 @@ func (d *Deployment) HasPlacedCanaries() bool { return false } +// RequiresPromotion returns whether the deployment requires promotion to +// continue +func (d *Deployment) RequiresPromotion() bool { + if d == nil || len(d.TaskGroups) == 0 || d.Status != DeploymentStatusRunning { + return false + } + for _, group := range d.TaskGroups { + if group.DesiredCanaries > 0 && !group.Promoted { + return true + } + } + return false +} + func (d *Deployment) GoString() string { base := fmt.Sprintf("Deployment ID %q for job %q has status %q (%v):", d.ID, d.JobID, d.Status, d.StatusDescription) for group, state := range d.TaskGroups { diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 2528d076e2f8..678df50b36bb 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -166,6 +166,13 @@ func (a *allocReconciler) Compute() *reconcileResults { }) } + // Set the description of a created deployment + if d := a.result.deployment; d != nil { + if d.RequiresPromotion() { + d.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion + } + } + return a.result } @@ -182,6 +189,8 @@ func (a *allocReconciler) cancelDeployments() { } // Nothing else to do + a.oldDeployment = a.deployment + a.deployment = nil return } diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 0798c56f9fb5..6f53851f2aca 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -1904,6 +1904,7 @@ func TestReconciler_StopOldCanaries(t *testing.T) { r := reconciler.Compute() newD := structs.NewDeployment(job) + newD.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ DesiredCanaries: 2, DesiredTotal: 10, @@ -1956,6 +1957,7 @@ func TestReconciler_NewCanaries(t *testing.T) { r := reconciler.Compute() newD := structs.NewDeployment(job) + newD.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ DesiredCanaries: 2, DesiredTotal: 10, @@ -2002,6 +2004,7 @@ func TestReconciler_NewCanaries_ScaleUp(t *testing.T) { r := reconciler.Compute() newD := structs.NewDeployment(job) + newD.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ DesiredCanaries: 2, DesiredTotal: 15, @@ -2049,6 +2052,7 @@ func TestReconciler_NewCanaries_ScaleDown(t *testing.T) { r := reconciler.Compute() newD := structs.NewDeployment(job) + newD.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion newD.TaskGroups[job.TaskGroups[0].Name] = &structs.DeploymentState{ DesiredCanaries: 2, DesiredTotal: 5, From 02447214d5618a294b4554a38d215a44ad4765b1 Mon Sep 17 00:00:00 2001 From: Michael Schurter Date: Fri, 7 Jul 2017 11:04:18 -0700 Subject: [PATCH 100/105] an user -> a user --- website/source/api/allocations.html.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/source/api/allocations.html.md b/website/source/api/allocations.html.md index 82c75490a82f..0a14fe4d6658 100644 --- a/website/source/api/allocations.html.md +++ b/website/source/api/allocations.html.md @@ -499,7 +499,7 @@ $ curl \ - `Killing` - The task has been sent the kill signal. - - `Killed` - The task was killed by an user. + - `Killed` - The task was killed by a user. - `Received` - The task has been pulled by the client at the given timestamp. From 0129455876b45a41617ee719b82d9c03070ddad0 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 7 Jul 2017 11:42:51 -0700 Subject: [PATCH 101/105] Rolling node drains using max_parallel and stagger This PR adds rolling node drains done at max_parallel and stagger of the update spec. It brings it inline with old behavior. --- scheduler/generic_sched.go | 21 +++--- scheduler/generic_sched_test.go | 9 +-- scheduler/reconcile.go | 62 +++++++++++++----- scheduler/reconcile_test.go | 113 ++++++++++++++++++++++++++------ scheduler/system_sched.go | 2 +- 5 files changed, 156 insertions(+), 51 deletions(-) diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go index 8db5a341bafe..d6ee9deb36ca 100644 --- a/scheduler/generic_sched.go +++ b/scheduler/generic_sched.go @@ -3,6 +3,7 @@ package scheduler import ( "fmt" "log" + "time" memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/go-multierror" @@ -70,8 +71,8 @@ type GenericScheduler struct { ctx *EvalContext stack *GenericStack - limitReached bool - nextEval *structs.Evaluation + followupEvalWait time.Duration + nextEval *structs.Evaluation deployment *structs.Deployment @@ -248,16 +249,14 @@ func (s *GenericScheduler) process() (bool, error) { return true, nil } - // XXX Don't need a next rolling update eval - // If the limit of placements was reached we need to create an evaluation - // to pickup from here after the stagger period. - if s.limitReached && s.nextEval == nil { - s.nextEval = s.eval.NextRollingEval(s.job.Update.Stagger) + // If we need a followup eval and we haven't created one, do so. + if s.followupEvalWait != 0 && s.nextEval == nil { + s.nextEval = s.eval.NextRollingEval(s.followupEvalWait) if err := s.planner.CreateEval(s.nextEval); err != nil { - s.logger.Printf("[ERR] sched: %#v failed to make next eval for rolling update: %v", s.eval, err) + s.logger.Printf("[ERR] sched: %#v failed to make next eval for rolling migration: %v", s.eval, err) return false, err } - s.logger.Printf("[DEBUG] sched: %#v: rolling update limit reached, next eval '%s' created", s.eval, s.nextEval.ID) + s.logger.Printf("[DEBUG] sched: %#v: rolling migration limit reached, next eval '%s' created", s.eval, s.nextEval.ID) } // Submit the plan and store the results. @@ -401,6 +400,10 @@ func (s *GenericScheduler) computeJobAllocs() error { s.plan.Deployment = results.deployment s.plan.DeploymentUpdates = results.deploymentUpdates + // Store the the follow up eval wait duration. If set this will trigger a + // follow up eval to handle node draining. + s.followupEvalWait = results.followupEvalWait + // Update the stored deployment if results.deployment != nil { s.deployment = results.deployment diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 4d0ee4b6b183..37ad32f79f42 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -2308,10 +2308,11 @@ func TestServiceSched_NodeDrain_UpdateStrategy(t *testing.T) { // Generate a fake job with allocations and an update policy. job := mock.Job() mp := 5 - job.Update = structs.UpdateStrategy{ - Stagger: time.Second, - MaxParallel: mp, - } + u := structs.DefaultUpdateStrategy.Copy() + u.MaxParallel = mp + u.Stagger = time.Second + job.TaskGroups[0].Update = u + noErr(t, h.State.UpsertJob(h.NextIndex(), job)) var allocs []*structs.Allocation diff --git a/scheduler/reconcile.go b/scheduler/reconcile.go index 678df50b36bb..d01427395d12 100644 --- a/scheduler/reconcile.go +++ b/scheduler/reconcile.go @@ -2,6 +2,7 @@ package scheduler import ( "log" + "time" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" @@ -84,6 +85,10 @@ type reconcileResults struct { // desiredTGUpdates captures the desired set of changes to make for each // task group. desiredTGUpdates map[string]*structs.DesiredUpdates + + // followupEvalWait is set if there should be a followup eval run after the + // given duration + followupEvalWait time.Duration } // allocPlaceResult contains the information required to place a single @@ -341,7 +346,7 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { // Determine how many we can place canaryState = dstate != nil && dstate.DesiredCanaries != 0 && !dstate.Promoted - limit := a.computeLimit(tg, untainted, destructive, canaryState) + limit := a.computeLimit(tg, untainted, destructive, migrate, canaryState) // Place if: // * The deployment is not paused or failed @@ -379,28 +384,49 @@ func (a *allocReconciler) computeGroup(group string, all allocSet) bool { desiredChanges.Ignore += uint64(len(destructive)) } - // TODO Migrations should be done using a stagger and max_parallel. - if !a.deploymentFailed { - desiredChanges.Migrate += uint64(len(migrate)) + // Calculate the allowed number of changes and set the desired changes + // accordingly. + min := helper.IntMin(len(migrate), limit) + if !a.deploymentFailed && !a.deploymentPaused { + desiredChanges.Migrate += uint64(min) + desiredChanges.Ignore += uint64(len(migrate) - min) } else { desiredChanges.Stop += uint64(len(migrate)) } - for _, alloc := range migrate { + followup := false + migrated := 0 + for _, alloc := range migrate.nameOrder() { + // If the deployment is failed or paused, don't replace it, just mark as stop. + if a.deploymentFailed || a.deploymentPaused { + a.result.stop = append(a.result.stop, allocStopResult{ + alloc: alloc, + statusDescription: allocNodeTainted, + }) + continue + } + + if migrated >= limit { + followup = true + break + } + + migrated++ a.result.stop = append(a.result.stop, allocStopResult{ alloc: alloc, statusDescription: allocMigrating, }) + a.result.place = append(a.result.place, allocPlaceResult{ + name: alloc.Name, + canary: false, + taskGroup: tg, + previousAlloc: alloc, + }) + } - // If the deployment is failed, just stop the allocation - if !a.deploymentFailed { - a.result.place = append(a.result.place, allocPlaceResult{ - name: alloc.Name, - canary: false, - taskGroup: tg, - previousAlloc: alloc, - }) - } + // We need to create a followup evaluation. + if followup && strategy != nil && a.result.followupEvalWait < strategy.Stagger { + a.result.followupEvalWait = strategy.Stagger } // Create a new deployment if necessary @@ -482,12 +508,12 @@ func (a *allocReconciler) handleGroupCanaries(all allocSet, desiredChanges *stru } // computeLimit returns the placement limit for a particular group. The inputs -// are the group definition, the untainted and destructive allocation set and -// whether we are in a canary state. -func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, destructive allocSet, canaryState bool) int { +// are the group definition, the untainted, destructive, and migrate allocation +// set and whether we are in a canary state. +func (a *allocReconciler) computeLimit(group *structs.TaskGroup, untainted, destructive, migrate allocSet, canaryState bool) int { // If there is no update stategy or deployment for the group we can deploy // as many as the group has - if group.Update == nil || len(destructive) == 0 { + if group.Update == nil || len(destructive)+len(migrate) == 0 { return group.Count } else if a.deploymentPaused || a.deploymentFailed { // If the deployment is paused or failed, do not create anything else diff --git a/scheduler/reconcile_test.go b/scheduler/reconcile_test.go index 6f53851f2aca..e9d7957a4f93 100644 --- a/scheduler/reconcile_test.go +++ b/scheduler/reconcile_test.go @@ -38,7 +38,7 @@ Basic Tests: √ Handle job being stopped both as .Stopped and nil √ Place more that one group -Deployment Tests: +Update stanza Tests: √ Stopped job cancels any active deployment √ Stopped job doesn't cancel terminal deployment √ JobIndex change cancels any active deployment @@ -67,6 +67,7 @@ Deployment Tests: √ Failed deployment cancels non-promoted task groups √ Failed deployment and updated job works √ Finished deployment gets marked as complete +√ The stagger is correctly calculated when it is applied across multiple task groups. */ var ( @@ -76,6 +77,7 @@ var ( HealthCheck: structs.UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, HealthyDeadline: 10 * time.Minute, + Stagger: 31 * time.Second, } noCanaryUpdate = &structs.UpdateStrategy{ @@ -83,6 +85,7 @@ var ( HealthCheck: structs.UpdateStrategyHealthCheck_Checks, MinHealthyTime: 10 * time.Second, HealthyDeadline: 10 * time.Minute, + Stagger: 31 * time.Second, } ) @@ -255,6 +258,7 @@ type resultExpectation struct { inplace int stop int desiredTGUpdates map[string]*structs.DesiredUpdates + followupEvalWait time.Duration } func assertResults(t *testing.T, r *reconcileResults, exp *resultExpectation) { @@ -287,6 +291,9 @@ func assertResults(t *testing.T, r *reconcileResults, exp *resultExpectation) { if l := len(r.desiredTGUpdates); l != len(exp.desiredTGUpdates) { t.Fatalf("Expected %d task group desired tg updates annotations; got %d", len(exp.desiredTGUpdates), l) } + if r.followupEvalWait != exp.followupEvalWait { + t.Fatalf("Unexpected followup eval wait time. Got %v; want %v", r.followupEvalWait, exp.followupEvalWait) + } // Check the desired updates happened for group, desired := range exp.desiredTGUpdates { @@ -1638,12 +1645,12 @@ func TestReconciler_PausedOrFailedDeployment_Migrations(t *testing.T) { stopAnnotation uint64 }{ { - name: "paused deployment", - deploymentStatus: structs.DeploymentStatusPaused, - place: 3, - stop: 3, - ignoreAnnotation: 5, - migrateAnnotation: 3, + name: "paused deployment", + deploymentStatus: structs.DeploymentStatusPaused, + place: 0, + stop: 3, + ignoreAnnotation: 5, + stopAnnotation: 3, }, { name: "failed deployment", @@ -2407,9 +2414,9 @@ func TestReconciler_TaintedNode_RollingUpgrade(t *testing.T) { PlacedAllocs: 4, } - // Create 6 allocations from the old job + // Create 3 allocations from the old job var allocs []*structs.Allocation - for i := 4; i < 10; i++ { + for i := 7; i < 10; i++ { alloc := mock.Alloc() alloc.Job = job alloc.JobID = job.ID @@ -2421,7 +2428,7 @@ func TestReconciler_TaintedNode_RollingUpgrade(t *testing.T) { // Create the healthy replacements handled := make(map[string]allocUpdateType) - for i := 0; i < 4; i++ { + for i := 0; i < 7; i++ { new := mock.Alloc() new.Job = job new.JobID = job.ID @@ -2437,10 +2444,10 @@ func TestReconciler_TaintedNode_RollingUpgrade(t *testing.T) { } // Build a map of tainted nodes - tainted := make(map[string]*structs.Node, 2) - for i := 0; i < 2; i++ { + tainted := make(map[string]*structs.Node, 3) + for i := 0; i < 3; i++ { n := mock.Node() - n.ID = allocs[6+i].NodeID + n.ID = allocs[3+i].NodeID if i == 0 { n.Status = structs.NodeStatusDown } else { @@ -2457,22 +2464,23 @@ func TestReconciler_TaintedNode_RollingUpgrade(t *testing.T) { assertResults(t, r, &resultExpectation{ createDeployment: nil, deploymentUpdates: nil, - place: 6, + place: 5, inplace: 0, - stop: 6, + stop: 5, + followupEvalWait: 31 * time.Second, desiredTGUpdates: map[string]*structs.DesiredUpdates{ job.TaskGroups[0].Name: { Place: 1, // Place the lost Stop: 1, // Stop the lost Migrate: 1, // Migrate the tainted - DestructiveUpdate: 4, - Ignore: 4, + DestructiveUpdate: 3, + Ignore: 5, }, }, }) - assertNamesHaveIndexes(t, intRange(0, 1, 4, 7), placeResultsToNames(r.place)) - assertNamesHaveIndexes(t, intRange(0, 1, 4, 7), stopResultsToNames(r.stop)) + assertNamesHaveIndexes(t, intRange(0, 1, 7, 9), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(0, 1, 7, 9), stopResultsToNames(r.stop)) } // Tests the reconciler handles a failed deployment and does no placements @@ -2542,6 +2550,7 @@ func TestReconciler_FailedDeployment_NoPlacements(t *testing.T) { place: 0, inplace: 0, stop: 2, + followupEvalWait: 0, // Since the deployment is failed, there should be no followup desiredTGUpdates: map[string]*structs.DesiredUpdates{ job.TaskGroups[0].Name: { Stop: 2, @@ -2822,3 +2831,69 @@ func TestReconciler_MarkDeploymentComplete(t *testing.T) { }, }) } + +// Tests the reconciler picks the maximum of the staggers when multiple task +// groups are under going node drains. +func TestReconciler_TaintedNode_MultiGroups(t *testing.T) { + // Create a job with two task groups + job := mock.Job() + job.TaskGroups[0].Update = noCanaryUpdate + job.TaskGroups = append(job.TaskGroups, job.TaskGroups[0].Copy()) + job.TaskGroups[1].Name = "two" + job.TaskGroups[1].Update.Stagger = 100 * time.Second + + // Create the allocations + var allocs []*structs.Allocation + for j := 0; j < 2; j++ { + for i := 0; i < 10; i++ { + alloc := mock.Alloc() + alloc.Job = job + alloc.JobID = job.ID + alloc.NodeID = structs.GenerateUUID() + alloc.Name = structs.AllocName(job.ID, job.TaskGroups[j].Name, uint(i)) + alloc.TaskGroup = job.TaskGroups[j].Name + allocs = append(allocs, alloc) + } + } + + // Build a map of tainted nodes + tainted := make(map[string]*structs.Node, 15) + for i := 0; i < 15; i++ { + n := mock.Node() + n.ID = allocs[i].NodeID + n.Drain = true + tainted[n.ID] = n + } + + reconciler := NewAllocReconciler(testLogger(), allocUpdateFnIgnore, false, job.ID, job, nil, allocs, tainted) + r := reconciler.Compute() + + // Assert the correct results + assertResults(t, r, &resultExpectation{ + createDeployment: nil, + deploymentUpdates: nil, + place: 8, + inplace: 0, + stop: 8, + followupEvalWait: 100 * time.Second, + desiredTGUpdates: map[string]*structs.DesiredUpdates{ + job.TaskGroups[0].Name: { + Place: 0, + Stop: 0, + Migrate: 4, + DestructiveUpdate: 0, + Ignore: 6, + }, + job.TaskGroups[1].Name: { + Place: 0, + Stop: 0, + Migrate: 4, + DestructiveUpdate: 0, + Ignore: 6, + }, + }, + }) + + assertNamesHaveIndexes(t, intRange(0, 3, 0, 3), placeResultsToNames(r.place)) + assertNamesHaveIndexes(t, intRange(0, 3, 0, 3), stopResultsToNames(r.stop)) +} diff --git a/scheduler/system_sched.go b/scheduler/system_sched.go index e3d48760c881..db5373197478 100644 --- a/scheduler/system_sched.go +++ b/scheduler/system_sched.go @@ -16,7 +16,7 @@ const ( // allocNodeTainted is the status used when stopping an alloc because it's // node is tainted. - allocNodeTainted = "system alloc not needed as node is tainted" + allocNodeTainted = "alloc not needed as node is tainted" ) // SystemScheduler is used for 'system' jobs. This scheduler is From 38e50eedb1f604857c2f574f3a9cec758a90ce27 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 7 Jul 2017 12:15:09 -0700 Subject: [PATCH 102/105] check id method name changed --- command/agent/consul/client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/command/agent/consul/client.go b/command/agent/consul/client.go index 1c1d762aad0a..733f899a3f17 100644 --- a/command/agent/consul/client.go +++ b/command/agent/consul/client.go @@ -620,7 +620,7 @@ func (c *ServiceClient) Checks(a *structs.Allocation) ([]*api.AgentCheck, error) for _, service := range task.Services { id := makeTaskServiceID(a.ID, task.Name, service) for _, check := range service.Checks { - relevant[createCheckID(id, check)] = struct{}{} + relevant[makeCheckID(id, check)] = struct{}{} } } } From 50e3178d2db6651bde4f52fea398f0e41c79d8a9 Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 7 Jul 2017 12:18:14 -0700 Subject: [PATCH 103/105] vendor --- vendor/github.com/davecgh/go-spew/LICENSE | 15 + .../github.com/davecgh/go-spew/spew/bypass.go | 152 + .../github.com/davecgh/go-spew/spew/common.go | 341 ++ .../github.com/davecgh/go-spew/spew/config.go | 306 ++ vendor/github.com/davecgh/go-spew/spew/doc.go | 211 ++ .../github.com/davecgh/go-spew/spew/dump.go | 509 +++ .../github.com/davecgh/go-spew/spew/format.go | 419 +++ .../github.com/davecgh/go-spew/spew/spew.go | 148 + vendor/github.com/pmezard/go-difflib/LICENSE | 27 + .../pmezard/go-difflib/difflib/difflib.go | 772 +++++ vendor/github.com/stretchr/objx/LICENSE.md | 23 + vendor/github.com/stretchr/objx/README.md | 3 + vendor/github.com/stretchr/objx/accessors.go | 179 + vendor/github.com/stretchr/objx/constants.go | 13 + .../github.com/stretchr/objx/conversions.go | 117 + vendor/github.com/stretchr/objx/doc.go | 72 + vendor/github.com/stretchr/objx/map.go | 222 ++ vendor/github.com/stretchr/objx/mutations.go | 81 + vendor/github.com/stretchr/objx/security.go | 14 + vendor/github.com/stretchr/objx/tests.go | 17 + .../stretchr/objx/type_specific_codegen.go | 2881 +++++++++++++++++ vendor/github.com/stretchr/objx/value.go | 56 + vendor/github.com/stretchr/testify/LICENSE | 22 + .../testify/assert/assertion_format.go | 379 +++ .../testify/assert/assertion_format.go.tmpl | 4 + .../testify/assert/assertion_forward.go | 746 +++++ .../testify/assert/assertion_forward.go.tmpl | 4 + .../stretchr/testify/assert/assertions.go | 1208 +++++++ .../github.com/stretchr/testify/assert/doc.go | 45 + .../stretchr/testify/assert/errors.go | 10 + .../testify/assert/forward_assertions.go | 16 + .../testify/assert/http_assertions.go | 127 + .../github.com/stretchr/testify/mock/doc.go | 44 + .../github.com/stretchr/testify/mock/mock.go | 776 +++++ 34 files changed, 9959 insertions(+) create mode 100644 vendor/github.com/davecgh/go-spew/LICENSE create mode 100644 vendor/github.com/davecgh/go-spew/spew/bypass.go create mode 100644 vendor/github.com/davecgh/go-spew/spew/common.go create mode 100644 vendor/github.com/davecgh/go-spew/spew/config.go create mode 100644 vendor/github.com/davecgh/go-spew/spew/doc.go create mode 100644 vendor/github.com/davecgh/go-spew/spew/dump.go create mode 100644 vendor/github.com/davecgh/go-spew/spew/format.go create mode 100644 vendor/github.com/davecgh/go-spew/spew/spew.go create mode 100644 vendor/github.com/pmezard/go-difflib/LICENSE create mode 100644 vendor/github.com/pmezard/go-difflib/difflib/difflib.go create mode 100644 vendor/github.com/stretchr/objx/LICENSE.md create mode 100644 vendor/github.com/stretchr/objx/README.md create mode 100644 vendor/github.com/stretchr/objx/accessors.go create mode 100644 vendor/github.com/stretchr/objx/constants.go create mode 100644 vendor/github.com/stretchr/objx/conversions.go create mode 100644 vendor/github.com/stretchr/objx/doc.go create mode 100644 vendor/github.com/stretchr/objx/map.go create mode 100644 vendor/github.com/stretchr/objx/mutations.go create mode 100644 vendor/github.com/stretchr/objx/security.go create mode 100644 vendor/github.com/stretchr/objx/tests.go create mode 100644 vendor/github.com/stretchr/objx/type_specific_codegen.go create mode 100644 vendor/github.com/stretchr/objx/value.go create mode 100644 vendor/github.com/stretchr/testify/LICENSE create mode 100644 vendor/github.com/stretchr/testify/assert/assertion_format.go create mode 100644 vendor/github.com/stretchr/testify/assert/assertion_format.go.tmpl create mode 100644 vendor/github.com/stretchr/testify/assert/assertion_forward.go create mode 100644 vendor/github.com/stretchr/testify/assert/assertion_forward.go.tmpl create mode 100644 vendor/github.com/stretchr/testify/assert/assertions.go create mode 100644 vendor/github.com/stretchr/testify/assert/doc.go create mode 100644 vendor/github.com/stretchr/testify/assert/errors.go create mode 100644 vendor/github.com/stretchr/testify/assert/forward_assertions.go create mode 100644 vendor/github.com/stretchr/testify/assert/http_assertions.go create mode 100644 vendor/github.com/stretchr/testify/mock/doc.go create mode 100644 vendor/github.com/stretchr/testify/mock/mock.go diff --git a/vendor/github.com/davecgh/go-spew/LICENSE b/vendor/github.com/davecgh/go-spew/LICENSE new file mode 100644 index 000000000000..c836416192da --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/LICENSE @@ -0,0 +1,15 @@ +ISC License + +Copyright (c) 2012-2016 Dave Collins + +Permission to use, copy, modify, and distribute this software for any +purpose with or without fee is hereby granted, provided that the above +copyright notice and this permission notice appear in all copies. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES +WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR +ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF +OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. diff --git a/vendor/github.com/davecgh/go-spew/spew/bypass.go b/vendor/github.com/davecgh/go-spew/spew/bypass.go new file mode 100644 index 000000000000..8a4a6589a2d4 --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/bypass.go @@ -0,0 +1,152 @@ +// Copyright (c) 2015-2016 Dave Collins +// +// Permission to use, copy, modify, and distribute this software for any +// purpose with or without fee is hereby granted, provided that the above +// copyright notice and this permission notice appear in all copies. +// +// THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES +// WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR +// ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +// WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +// ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF +// OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + +// NOTE: Due to the following build constraints, this file will only be compiled +// when the code is not running on Google App Engine, compiled by GopherJS, and +// "-tags safe" is not added to the go build command line. The "disableunsafe" +// tag is deprecated and thus should not be used. +// +build !js,!appengine,!safe,!disableunsafe + +package spew + +import ( + "reflect" + "unsafe" +) + +const ( + // UnsafeDisabled is a build-time constant which specifies whether or + // not access to the unsafe package is available. + UnsafeDisabled = false + + // ptrSize is the size of a pointer on the current arch. + ptrSize = unsafe.Sizeof((*byte)(nil)) +) + +var ( + // offsetPtr, offsetScalar, and offsetFlag are the offsets for the + // internal reflect.Value fields. These values are valid before golang + // commit ecccf07e7f9d which changed the format. The are also valid + // after commit 82f48826c6c7 which changed the format again to mirror + // the original format. Code in the init function updates these offsets + // as necessary. + offsetPtr = uintptr(ptrSize) + offsetScalar = uintptr(0) + offsetFlag = uintptr(ptrSize * 2) + + // flagKindWidth and flagKindShift indicate various bits that the + // reflect package uses internally to track kind information. + // + // flagRO indicates whether or not the value field of a reflect.Value is + // read-only. + // + // flagIndir indicates whether the value field of a reflect.Value is + // the actual data or a pointer to the data. + // + // These values are valid before golang commit 90a7c3c86944 which + // changed their positions. Code in the init function updates these + // flags as necessary. + flagKindWidth = uintptr(5) + flagKindShift = uintptr(flagKindWidth - 1) + flagRO = uintptr(1 << 0) + flagIndir = uintptr(1 << 1) +) + +func init() { + // Older versions of reflect.Value stored small integers directly in the + // ptr field (which is named val in the older versions). Versions + // between commits ecccf07e7f9d and 82f48826c6c7 added a new field named + // scalar for this purpose which unfortunately came before the flag + // field, so the offset of the flag field is different for those + // versions. + // + // This code constructs a new reflect.Value from a known small integer + // and checks if the size of the reflect.Value struct indicates it has + // the scalar field. When it does, the offsets are updated accordingly. + vv := reflect.ValueOf(0xf00) + if unsafe.Sizeof(vv) == (ptrSize * 4) { + offsetScalar = ptrSize * 2 + offsetFlag = ptrSize * 3 + } + + // Commit 90a7c3c86944 changed the flag positions such that the low + // order bits are the kind. This code extracts the kind from the flags + // field and ensures it's the correct type. When it's not, the flag + // order has been changed to the newer format, so the flags are updated + // accordingly. + upf := unsafe.Pointer(uintptr(unsafe.Pointer(&vv)) + offsetFlag) + upfv := *(*uintptr)(upf) + flagKindMask := uintptr((1<>flagKindShift != uintptr(reflect.Int) { + flagKindShift = 0 + flagRO = 1 << 5 + flagIndir = 1 << 6 + + // Commit adf9b30e5594 modified the flags to separate the + // flagRO flag into two bits which specifies whether or not the + // field is embedded. This causes flagIndir to move over a bit + // and means that flagRO is the combination of either of the + // original flagRO bit and the new bit. + // + // This code detects the change by extracting what used to be + // the indirect bit to ensure it's set. When it's not, the flag + // order has been changed to the newer format, so the flags are + // updated accordingly. + if upfv&flagIndir == 0 { + flagRO = 3 << 5 + flagIndir = 1 << 7 + } + } +} + +// unsafeReflectValue converts the passed reflect.Value into a one that bypasses +// the typical safety restrictions preventing access to unaddressable and +// unexported data. It works by digging the raw pointer to the underlying +// value out of the protected value and generating a new unprotected (unsafe) +// reflect.Value to it. +// +// This allows us to check for implementations of the Stringer and error +// interfaces to be used for pretty printing ordinarily unaddressable and +// inaccessible values such as unexported struct fields. +func unsafeReflectValue(v reflect.Value) (rv reflect.Value) { + indirects := 1 + vt := v.Type() + upv := unsafe.Pointer(uintptr(unsafe.Pointer(&v)) + offsetPtr) + rvf := *(*uintptr)(unsafe.Pointer(uintptr(unsafe.Pointer(&v)) + offsetFlag)) + if rvf&flagIndir != 0 { + vt = reflect.PtrTo(v.Type()) + indirects++ + } else if offsetScalar != 0 { + // The value is in the scalar field when it's not one of the + // reference types. + switch vt.Kind() { + case reflect.Uintptr: + case reflect.Chan: + case reflect.Func: + case reflect.Map: + case reflect.Ptr: + case reflect.UnsafePointer: + default: + upv = unsafe.Pointer(uintptr(unsafe.Pointer(&v)) + + offsetScalar) + } + } + + pv := reflect.NewAt(vt, upv) + rv = pv + for i := 0; i < indirects; i++ { + rv = rv.Elem() + } + return rv +} diff --git a/vendor/github.com/davecgh/go-spew/spew/common.go b/vendor/github.com/davecgh/go-spew/spew/common.go new file mode 100644 index 000000000000..7c519ff47ac3 --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/common.go @@ -0,0 +1,341 @@ +/* + * Copyright (c) 2013-2016 Dave Collins + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + */ + +package spew + +import ( + "bytes" + "fmt" + "io" + "reflect" + "sort" + "strconv" +) + +// Some constants in the form of bytes to avoid string overhead. This mirrors +// the technique used in the fmt package. +var ( + panicBytes = []byte("(PANIC=") + plusBytes = []byte("+") + iBytes = []byte("i") + trueBytes = []byte("true") + falseBytes = []byte("false") + interfaceBytes = []byte("(interface {})") + commaNewlineBytes = []byte(",\n") + newlineBytes = []byte("\n") + openBraceBytes = []byte("{") + openBraceNewlineBytes = []byte("{\n") + closeBraceBytes = []byte("}") + asteriskBytes = []byte("*") + colonBytes = []byte(":") + colonSpaceBytes = []byte(": ") + openParenBytes = []byte("(") + closeParenBytes = []byte(")") + spaceBytes = []byte(" ") + pointerChainBytes = []byte("->") + nilAngleBytes = []byte("") + maxNewlineBytes = []byte("\n") + maxShortBytes = []byte("") + circularBytes = []byte("") + circularShortBytes = []byte("") + invalidAngleBytes = []byte("") + openBracketBytes = []byte("[") + closeBracketBytes = []byte("]") + percentBytes = []byte("%") + precisionBytes = []byte(".") + openAngleBytes = []byte("<") + closeAngleBytes = []byte(">") + openMapBytes = []byte("map[") + closeMapBytes = []byte("]") + lenEqualsBytes = []byte("len=") + capEqualsBytes = []byte("cap=") +) + +// hexDigits is used to map a decimal value to a hex digit. +var hexDigits = "0123456789abcdef" + +// catchPanic handles any panics that might occur during the handleMethods +// calls. +func catchPanic(w io.Writer, v reflect.Value) { + if err := recover(); err != nil { + w.Write(panicBytes) + fmt.Fprintf(w, "%v", err) + w.Write(closeParenBytes) + } +} + +// handleMethods attempts to call the Error and String methods on the underlying +// type the passed reflect.Value represents and outputes the result to Writer w. +// +// It handles panics in any called methods by catching and displaying the error +// as the formatted value. +func handleMethods(cs *ConfigState, w io.Writer, v reflect.Value) (handled bool) { + // We need an interface to check if the type implements the error or + // Stringer interface. However, the reflect package won't give us an + // interface on certain things like unexported struct fields in order + // to enforce visibility rules. We use unsafe, when it's available, + // to bypass these restrictions since this package does not mutate the + // values. + if !v.CanInterface() { + if UnsafeDisabled { + return false + } + + v = unsafeReflectValue(v) + } + + // Choose whether or not to do error and Stringer interface lookups against + // the base type or a pointer to the base type depending on settings. + // Technically calling one of these methods with a pointer receiver can + // mutate the value, however, types which choose to satisify an error or + // Stringer interface with a pointer receiver should not be mutating their + // state inside these interface methods. + if !cs.DisablePointerMethods && !UnsafeDisabled && !v.CanAddr() { + v = unsafeReflectValue(v) + } + if v.CanAddr() { + v = v.Addr() + } + + // Is it an error or Stringer? + switch iface := v.Interface().(type) { + case error: + defer catchPanic(w, v) + if cs.ContinueOnMethod { + w.Write(openParenBytes) + w.Write([]byte(iface.Error())) + w.Write(closeParenBytes) + w.Write(spaceBytes) + return false + } + + w.Write([]byte(iface.Error())) + return true + + case fmt.Stringer: + defer catchPanic(w, v) + if cs.ContinueOnMethod { + w.Write(openParenBytes) + w.Write([]byte(iface.String())) + w.Write(closeParenBytes) + w.Write(spaceBytes) + return false + } + w.Write([]byte(iface.String())) + return true + } + return false +} + +// printBool outputs a boolean value as true or false to Writer w. +func printBool(w io.Writer, val bool) { + if val { + w.Write(trueBytes) + } else { + w.Write(falseBytes) + } +} + +// printInt outputs a signed integer value to Writer w. +func printInt(w io.Writer, val int64, base int) { + w.Write([]byte(strconv.FormatInt(val, base))) +} + +// printUint outputs an unsigned integer value to Writer w. +func printUint(w io.Writer, val uint64, base int) { + w.Write([]byte(strconv.FormatUint(val, base))) +} + +// printFloat outputs a floating point value using the specified precision, +// which is expected to be 32 or 64bit, to Writer w. +func printFloat(w io.Writer, val float64, precision int) { + w.Write([]byte(strconv.FormatFloat(val, 'g', -1, precision))) +} + +// printComplex outputs a complex value using the specified float precision +// for the real and imaginary parts to Writer w. +func printComplex(w io.Writer, c complex128, floatPrecision int) { + r := real(c) + w.Write(openParenBytes) + w.Write([]byte(strconv.FormatFloat(r, 'g', -1, floatPrecision))) + i := imag(c) + if i >= 0 { + w.Write(plusBytes) + } + w.Write([]byte(strconv.FormatFloat(i, 'g', -1, floatPrecision))) + w.Write(iBytes) + w.Write(closeParenBytes) +} + +// printHexPtr outputs a uintptr formatted as hexidecimal with a leading '0x' +// prefix to Writer w. +func printHexPtr(w io.Writer, p uintptr) { + // Null pointer. + num := uint64(p) + if num == 0 { + w.Write(nilAngleBytes) + return + } + + // Max uint64 is 16 bytes in hex + 2 bytes for '0x' prefix + buf := make([]byte, 18) + + // It's simpler to construct the hex string right to left. + base := uint64(16) + i := len(buf) - 1 + for num >= base { + buf[i] = hexDigits[num%base] + num /= base + i-- + } + buf[i] = hexDigits[num] + + // Add '0x' prefix. + i-- + buf[i] = 'x' + i-- + buf[i] = '0' + + // Strip unused leading bytes. + buf = buf[i:] + w.Write(buf) +} + +// valuesSorter implements sort.Interface to allow a slice of reflect.Value +// elements to be sorted. +type valuesSorter struct { + values []reflect.Value + strings []string // either nil or same len and values + cs *ConfigState +} + +// newValuesSorter initializes a valuesSorter instance, which holds a set of +// surrogate keys on which the data should be sorted. It uses flags in +// ConfigState to decide if and how to populate those surrogate keys. +func newValuesSorter(values []reflect.Value, cs *ConfigState) sort.Interface { + vs := &valuesSorter{values: values, cs: cs} + if canSortSimply(vs.values[0].Kind()) { + return vs + } + if !cs.DisableMethods { + vs.strings = make([]string, len(values)) + for i := range vs.values { + b := bytes.Buffer{} + if !handleMethods(cs, &b, vs.values[i]) { + vs.strings = nil + break + } + vs.strings[i] = b.String() + } + } + if vs.strings == nil && cs.SpewKeys { + vs.strings = make([]string, len(values)) + for i := range vs.values { + vs.strings[i] = Sprintf("%#v", vs.values[i].Interface()) + } + } + return vs +} + +// canSortSimply tests whether a reflect.Kind is a primitive that can be sorted +// directly, or whether it should be considered for sorting by surrogate keys +// (if the ConfigState allows it). +func canSortSimply(kind reflect.Kind) bool { + // This switch parallels valueSortLess, except for the default case. + switch kind { + case reflect.Bool: + return true + case reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, reflect.Int: + return true + case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uint: + return true + case reflect.Float32, reflect.Float64: + return true + case reflect.String: + return true + case reflect.Uintptr: + return true + case reflect.Array: + return true + } + return false +} + +// Len returns the number of values in the slice. It is part of the +// sort.Interface implementation. +func (s *valuesSorter) Len() int { + return len(s.values) +} + +// Swap swaps the values at the passed indices. It is part of the +// sort.Interface implementation. +func (s *valuesSorter) Swap(i, j int) { + s.values[i], s.values[j] = s.values[j], s.values[i] + if s.strings != nil { + s.strings[i], s.strings[j] = s.strings[j], s.strings[i] + } +} + +// valueSortLess returns whether the first value should sort before the second +// value. It is used by valueSorter.Less as part of the sort.Interface +// implementation. +func valueSortLess(a, b reflect.Value) bool { + switch a.Kind() { + case reflect.Bool: + return !a.Bool() && b.Bool() + case reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, reflect.Int: + return a.Int() < b.Int() + case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uint: + return a.Uint() < b.Uint() + case reflect.Float32, reflect.Float64: + return a.Float() < b.Float() + case reflect.String: + return a.String() < b.String() + case reflect.Uintptr: + return a.Uint() < b.Uint() + case reflect.Array: + // Compare the contents of both arrays. + l := a.Len() + for i := 0; i < l; i++ { + av := a.Index(i) + bv := b.Index(i) + if av.Interface() == bv.Interface() { + continue + } + return valueSortLess(av, bv) + } + } + return a.String() < b.String() +} + +// Less returns whether the value at index i should sort before the +// value at index j. It is part of the sort.Interface implementation. +func (s *valuesSorter) Less(i, j int) bool { + if s.strings == nil { + return valueSortLess(s.values[i], s.values[j]) + } + return s.strings[i] < s.strings[j] +} + +// sortValues is a sort function that handles both native types and any type that +// can be converted to error or Stringer. Other inputs are sorted according to +// their Value.String() value to ensure display stability. +func sortValues(values []reflect.Value, cs *ConfigState) { + if len(values) == 0 { + return + } + sort.Sort(newValuesSorter(values, cs)) +} diff --git a/vendor/github.com/davecgh/go-spew/spew/config.go b/vendor/github.com/davecgh/go-spew/spew/config.go new file mode 100644 index 000000000000..2e3d22f31202 --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/config.go @@ -0,0 +1,306 @@ +/* + * Copyright (c) 2013-2016 Dave Collins + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + */ + +package spew + +import ( + "bytes" + "fmt" + "io" + "os" +) + +// ConfigState houses the configuration options used by spew to format and +// display values. There is a global instance, Config, that is used to control +// all top-level Formatter and Dump functionality. Each ConfigState instance +// provides methods equivalent to the top-level functions. +// +// The zero value for ConfigState provides no indentation. You would typically +// want to set it to a space or a tab. +// +// Alternatively, you can use NewDefaultConfig to get a ConfigState instance +// with default settings. See the documentation of NewDefaultConfig for default +// values. +type ConfigState struct { + // Indent specifies the string to use for each indentation level. The + // global config instance that all top-level functions use set this to a + // single space by default. If you would like more indentation, you might + // set this to a tab with "\t" or perhaps two spaces with " ". + Indent string + + // MaxDepth controls the maximum number of levels to descend into nested + // data structures. The default, 0, means there is no limit. + // + // NOTE: Circular data structures are properly detected, so it is not + // necessary to set this value unless you specifically want to limit deeply + // nested data structures. + MaxDepth int + + // DisableMethods specifies whether or not error and Stringer interfaces are + // invoked for types that implement them. + DisableMethods bool + + // DisablePointerMethods specifies whether or not to check for and invoke + // error and Stringer interfaces on types which only accept a pointer + // receiver when the current type is not a pointer. + // + // NOTE: This might be an unsafe action since calling one of these methods + // with a pointer receiver could technically mutate the value, however, + // in practice, types which choose to satisify an error or Stringer + // interface with a pointer receiver should not be mutating their state + // inside these interface methods. As a result, this option relies on + // access to the unsafe package, so it will not have any effect when + // running in environments without access to the unsafe package such as + // Google App Engine or with the "safe" build tag specified. + DisablePointerMethods bool + + // DisablePointerAddresses specifies whether to disable the printing of + // pointer addresses. This is useful when diffing data structures in tests. + DisablePointerAddresses bool + + // DisableCapacities specifies whether to disable the printing of capacities + // for arrays, slices, maps and channels. This is useful when diffing + // data structures in tests. + DisableCapacities bool + + // ContinueOnMethod specifies whether or not recursion should continue once + // a custom error or Stringer interface is invoked. The default, false, + // means it will print the results of invoking the custom error or Stringer + // interface and return immediately instead of continuing to recurse into + // the internals of the data type. + // + // NOTE: This flag does not have any effect if method invocation is disabled + // via the DisableMethods or DisablePointerMethods options. + ContinueOnMethod bool + + // SortKeys specifies map keys should be sorted before being printed. Use + // this to have a more deterministic, diffable output. Note that only + // native types (bool, int, uint, floats, uintptr and string) and types + // that support the error or Stringer interfaces (if methods are + // enabled) are supported, with other types sorted according to the + // reflect.Value.String() output which guarantees display stability. + SortKeys bool + + // SpewKeys specifies that, as a last resort attempt, map keys should + // be spewed to strings and sorted by those strings. This is only + // considered if SortKeys is true. + SpewKeys bool +} + +// Config is the active configuration of the top-level functions. +// The configuration can be changed by modifying the contents of spew.Config. +var Config = ConfigState{Indent: " "} + +// Errorf is a wrapper for fmt.Errorf that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the formatted string as a value that satisfies error. See NewFormatter +// for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Errorf(format, c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Errorf(format string, a ...interface{}) (err error) { + return fmt.Errorf(format, c.convertArgs(a)...) +} + +// Fprint is a wrapper for fmt.Fprint that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Fprint(w, c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Fprint(w io.Writer, a ...interface{}) (n int, err error) { + return fmt.Fprint(w, c.convertArgs(a)...) +} + +// Fprintf is a wrapper for fmt.Fprintf that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Fprintf(w, format, c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Fprintf(w io.Writer, format string, a ...interface{}) (n int, err error) { + return fmt.Fprintf(w, format, c.convertArgs(a)...) +} + +// Fprintln is a wrapper for fmt.Fprintln that treats each argument as if it +// passed with a Formatter interface returned by c.NewFormatter. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Fprintln(w, c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Fprintln(w io.Writer, a ...interface{}) (n int, err error) { + return fmt.Fprintln(w, c.convertArgs(a)...) +} + +// Print is a wrapper for fmt.Print that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Print(c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Print(a ...interface{}) (n int, err error) { + return fmt.Print(c.convertArgs(a)...) +} + +// Printf is a wrapper for fmt.Printf that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Printf(format, c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Printf(format string, a ...interface{}) (n int, err error) { + return fmt.Printf(format, c.convertArgs(a)...) +} + +// Println is a wrapper for fmt.Println that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Println(c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Println(a ...interface{}) (n int, err error) { + return fmt.Println(c.convertArgs(a)...) +} + +// Sprint is a wrapper for fmt.Sprint that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the resulting string. See NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Sprint(c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Sprint(a ...interface{}) string { + return fmt.Sprint(c.convertArgs(a)...) +} + +// Sprintf is a wrapper for fmt.Sprintf that treats each argument as if it were +// passed with a Formatter interface returned by c.NewFormatter. It returns +// the resulting string. See NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Sprintf(format, c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Sprintf(format string, a ...interface{}) string { + return fmt.Sprintf(format, c.convertArgs(a)...) +} + +// Sprintln is a wrapper for fmt.Sprintln that treats each argument as if it +// were passed with a Formatter interface returned by c.NewFormatter. It +// returns the resulting string. See NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Sprintln(c.NewFormatter(a), c.NewFormatter(b)) +func (c *ConfigState) Sprintln(a ...interface{}) string { + return fmt.Sprintln(c.convertArgs(a)...) +} + +/* +NewFormatter returns a custom formatter that satisfies the fmt.Formatter +interface. As a result, it integrates cleanly with standard fmt package +printing functions. The formatter is useful for inline printing of smaller data +types similar to the standard %v format specifier. + +The custom formatter only responds to the %v (most compact), %+v (adds pointer +addresses), %#v (adds types), and %#+v (adds types and pointer addresses) verb +combinations. Any other verbs such as %x and %q will be sent to the the +standard fmt package for formatting. In addition, the custom formatter ignores +the width and precision arguments (however they will still work on the format +specifiers not handled by the custom formatter). + +Typically this function shouldn't be called directly. It is much easier to make +use of the custom formatter by calling one of the convenience functions such as +c.Printf, c.Println, or c.Printf. +*/ +func (c *ConfigState) NewFormatter(v interface{}) fmt.Formatter { + return newFormatter(c, v) +} + +// Fdump formats and displays the passed arguments to io.Writer w. It formats +// exactly the same as Dump. +func (c *ConfigState) Fdump(w io.Writer, a ...interface{}) { + fdump(c, w, a...) +} + +/* +Dump displays the passed parameters to standard out with newlines, customizable +indentation, and additional debug information such as complete types and all +pointer addresses used to indirect to the final value. It provides the +following features over the built-in printing facilities provided by the fmt +package: + + * Pointers are dereferenced and followed + * Circular data structures are detected and handled properly + * Custom Stringer/error interfaces are optionally invoked, including + on unexported types + * Custom types which only implement the Stringer/error interfaces via + a pointer receiver are optionally invoked when passing non-pointer + variables + * Byte arrays and slices are dumped like the hexdump -C command which + includes offsets, byte values in hex, and ASCII output + +The configuration options are controlled by modifying the public members +of c. See ConfigState for options documentation. + +See Fdump if you would prefer dumping to an arbitrary io.Writer or Sdump to +get the formatted result as a string. +*/ +func (c *ConfigState) Dump(a ...interface{}) { + fdump(c, os.Stdout, a...) +} + +// Sdump returns a string with the passed arguments formatted exactly the same +// as Dump. +func (c *ConfigState) Sdump(a ...interface{}) string { + var buf bytes.Buffer + fdump(c, &buf, a...) + return buf.String() +} + +// convertArgs accepts a slice of arguments and returns a slice of the same +// length with each argument converted to a spew Formatter interface using +// the ConfigState associated with s. +func (c *ConfigState) convertArgs(args []interface{}) (formatters []interface{}) { + formatters = make([]interface{}, len(args)) + for index, arg := range args { + formatters[index] = newFormatter(c, arg) + } + return formatters +} + +// NewDefaultConfig returns a ConfigState with the following default settings. +// +// Indent: " " +// MaxDepth: 0 +// DisableMethods: false +// DisablePointerMethods: false +// ContinueOnMethod: false +// SortKeys: false +func NewDefaultConfig() *ConfigState { + return &ConfigState{Indent: " "} +} diff --git a/vendor/github.com/davecgh/go-spew/spew/doc.go b/vendor/github.com/davecgh/go-spew/spew/doc.go new file mode 100644 index 000000000000..aacaac6f1e1e --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/doc.go @@ -0,0 +1,211 @@ +/* + * Copyright (c) 2013-2016 Dave Collins + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + */ + +/* +Package spew implements a deep pretty printer for Go data structures to aid in +debugging. + +A quick overview of the additional features spew provides over the built-in +printing facilities for Go data types are as follows: + + * Pointers are dereferenced and followed + * Circular data structures are detected and handled properly + * Custom Stringer/error interfaces are optionally invoked, including + on unexported types + * Custom types which only implement the Stringer/error interfaces via + a pointer receiver are optionally invoked when passing non-pointer + variables + * Byte arrays and slices are dumped like the hexdump -C command which + includes offsets, byte values in hex, and ASCII output (only when using + Dump style) + +There are two different approaches spew allows for dumping Go data structures: + + * Dump style which prints with newlines, customizable indentation, + and additional debug information such as types and all pointer addresses + used to indirect to the final value + * A custom Formatter interface that integrates cleanly with the standard fmt + package and replaces %v, %+v, %#v, and %#+v to provide inline printing + similar to the default %v while providing the additional functionality + outlined above and passing unsupported format verbs such as %x and %q + along to fmt + +Quick Start + +This section demonstrates how to quickly get started with spew. See the +sections below for further details on formatting and configuration options. + +To dump a variable with full newlines, indentation, type, and pointer +information use Dump, Fdump, or Sdump: + spew.Dump(myVar1, myVar2, ...) + spew.Fdump(someWriter, myVar1, myVar2, ...) + str := spew.Sdump(myVar1, myVar2, ...) + +Alternatively, if you would prefer to use format strings with a compacted inline +printing style, use the convenience wrappers Printf, Fprintf, etc with +%v (most compact), %+v (adds pointer addresses), %#v (adds types), or +%#+v (adds types and pointer addresses): + spew.Printf("myVar1: %v -- myVar2: %+v", myVar1, myVar2) + spew.Printf("myVar3: %#v -- myVar4: %#+v", myVar3, myVar4) + spew.Fprintf(someWriter, "myVar1: %v -- myVar2: %+v", myVar1, myVar2) + spew.Fprintf(someWriter, "myVar3: %#v -- myVar4: %#+v", myVar3, myVar4) + +Configuration Options + +Configuration of spew is handled by fields in the ConfigState type. For +convenience, all of the top-level functions use a global state available +via the spew.Config global. + +It is also possible to create a ConfigState instance that provides methods +equivalent to the top-level functions. This allows concurrent configuration +options. See the ConfigState documentation for more details. + +The following configuration options are available: + * Indent + String to use for each indentation level for Dump functions. + It is a single space by default. A popular alternative is "\t". + + * MaxDepth + Maximum number of levels to descend into nested data structures. + There is no limit by default. + + * DisableMethods + Disables invocation of error and Stringer interface methods. + Method invocation is enabled by default. + + * DisablePointerMethods + Disables invocation of error and Stringer interface methods on types + which only accept pointer receivers from non-pointer variables. + Pointer method invocation is enabled by default. + + * DisablePointerAddresses + DisablePointerAddresses specifies whether to disable the printing of + pointer addresses. This is useful when diffing data structures in tests. + + * DisableCapacities + DisableCapacities specifies whether to disable the printing of + capacities for arrays, slices, maps and channels. This is useful when + diffing data structures in tests. + + * ContinueOnMethod + Enables recursion into types after invoking error and Stringer interface + methods. Recursion after method invocation is disabled by default. + + * SortKeys + Specifies map keys should be sorted before being printed. Use + this to have a more deterministic, diffable output. Note that + only native types (bool, int, uint, floats, uintptr and string) + and types which implement error or Stringer interfaces are + supported with other types sorted according to the + reflect.Value.String() output which guarantees display + stability. Natural map order is used by default. + + * SpewKeys + Specifies that, as a last resort attempt, map keys should be + spewed to strings and sorted by those strings. This is only + considered if SortKeys is true. + +Dump Usage + +Simply call spew.Dump with a list of variables you want to dump: + + spew.Dump(myVar1, myVar2, ...) + +You may also call spew.Fdump if you would prefer to output to an arbitrary +io.Writer. For example, to dump to standard error: + + spew.Fdump(os.Stderr, myVar1, myVar2, ...) + +A third option is to call spew.Sdump to get the formatted output as a string: + + str := spew.Sdump(myVar1, myVar2, ...) + +Sample Dump Output + +See the Dump example for details on the setup of the types and variables being +shown here. + + (main.Foo) { + unexportedField: (*main.Bar)(0xf84002e210)({ + flag: (main.Flag) flagTwo, + data: (uintptr) + }), + ExportedField: (map[interface {}]interface {}) (len=1) { + (string) (len=3) "one": (bool) true + } + } + +Byte (and uint8) arrays and slices are displayed uniquely like the hexdump -C +command as shown. + ([]uint8) (len=32 cap=32) { + 00000000 11 12 13 14 15 16 17 18 19 1a 1b 1c 1d 1e 1f 20 |............... | + 00000010 21 22 23 24 25 26 27 28 29 2a 2b 2c 2d 2e 2f 30 |!"#$%&'()*+,-./0| + 00000020 31 32 |12| + } + +Custom Formatter + +Spew provides a custom formatter that implements the fmt.Formatter interface +so that it integrates cleanly with standard fmt package printing functions. The +formatter is useful for inline printing of smaller data types similar to the +standard %v format specifier. + +The custom formatter only responds to the %v (most compact), %+v (adds pointer +addresses), %#v (adds types), or %#+v (adds types and pointer addresses) verb +combinations. Any other verbs such as %x and %q will be sent to the the +standard fmt package for formatting. In addition, the custom formatter ignores +the width and precision arguments (however they will still work on the format +specifiers not handled by the custom formatter). + +Custom Formatter Usage + +The simplest way to make use of the spew custom formatter is to call one of the +convenience functions such as spew.Printf, spew.Println, or spew.Printf. The +functions have syntax you are most likely already familiar with: + + spew.Printf("myVar1: %v -- myVar2: %+v", myVar1, myVar2) + spew.Printf("myVar3: %#v -- myVar4: %#+v", myVar3, myVar4) + spew.Println(myVar, myVar2) + spew.Fprintf(os.Stderr, "myVar1: %v -- myVar2: %+v", myVar1, myVar2) + spew.Fprintf(os.Stderr, "myVar3: %#v -- myVar4: %#+v", myVar3, myVar4) + +See the Index for the full list convenience functions. + +Sample Formatter Output + +Double pointer to a uint8: + %v: <**>5 + %+v: <**>(0xf8400420d0->0xf8400420c8)5 + %#v: (**uint8)5 + %#+v: (**uint8)(0xf8400420d0->0xf8400420c8)5 + +Pointer to circular struct with a uint8 field and a pointer to itself: + %v: <*>{1 <*>} + %+v: <*>(0xf84003e260){ui8:1 c:<*>(0xf84003e260)} + %#v: (*main.circular){ui8:(uint8)1 c:(*main.circular)} + %#+v: (*main.circular)(0xf84003e260){ui8:(uint8)1 c:(*main.circular)(0xf84003e260)} + +See the Printf example for details on the setup of variables being shown +here. + +Errors + +Since it is possible for custom Stringer/error interfaces to panic, spew +detects them and handles them internally by printing the panic information +inline with the output. Since spew is intended to provide deep pretty printing +capabilities on structures, it intentionally does not return any errors. +*/ +package spew diff --git a/vendor/github.com/davecgh/go-spew/spew/dump.go b/vendor/github.com/davecgh/go-spew/spew/dump.go new file mode 100644 index 000000000000..df1d582a728a --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/dump.go @@ -0,0 +1,509 @@ +/* + * Copyright (c) 2013-2016 Dave Collins + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + */ + +package spew + +import ( + "bytes" + "encoding/hex" + "fmt" + "io" + "os" + "reflect" + "regexp" + "strconv" + "strings" +) + +var ( + // uint8Type is a reflect.Type representing a uint8. It is used to + // convert cgo types to uint8 slices for hexdumping. + uint8Type = reflect.TypeOf(uint8(0)) + + // cCharRE is a regular expression that matches a cgo char. + // It is used to detect character arrays to hexdump them. + cCharRE = regexp.MustCompile("^.*\\._Ctype_char$") + + // cUnsignedCharRE is a regular expression that matches a cgo unsigned + // char. It is used to detect unsigned character arrays to hexdump + // them. + cUnsignedCharRE = regexp.MustCompile("^.*\\._Ctype_unsignedchar$") + + // cUint8tCharRE is a regular expression that matches a cgo uint8_t. + // It is used to detect uint8_t arrays to hexdump them. + cUint8tCharRE = regexp.MustCompile("^.*\\._Ctype_uint8_t$") +) + +// dumpState contains information about the state of a dump operation. +type dumpState struct { + w io.Writer + depth int + pointers map[uintptr]int + ignoreNextType bool + ignoreNextIndent bool + cs *ConfigState +} + +// indent performs indentation according to the depth level and cs.Indent +// option. +func (d *dumpState) indent() { + if d.ignoreNextIndent { + d.ignoreNextIndent = false + return + } + d.w.Write(bytes.Repeat([]byte(d.cs.Indent), d.depth)) +} + +// unpackValue returns values inside of non-nil interfaces when possible. +// This is useful for data types like structs, arrays, slices, and maps which +// can contain varying types packed inside an interface. +func (d *dumpState) unpackValue(v reflect.Value) reflect.Value { + if v.Kind() == reflect.Interface && !v.IsNil() { + v = v.Elem() + } + return v +} + +// dumpPtr handles formatting of pointers by indirecting them as necessary. +func (d *dumpState) dumpPtr(v reflect.Value) { + // Remove pointers at or below the current depth from map used to detect + // circular refs. + for k, depth := range d.pointers { + if depth >= d.depth { + delete(d.pointers, k) + } + } + + // Keep list of all dereferenced pointers to show later. + pointerChain := make([]uintptr, 0) + + // Figure out how many levels of indirection there are by dereferencing + // pointers and unpacking interfaces down the chain while detecting circular + // references. + nilFound := false + cycleFound := false + indirects := 0 + ve := v + for ve.Kind() == reflect.Ptr { + if ve.IsNil() { + nilFound = true + break + } + indirects++ + addr := ve.Pointer() + pointerChain = append(pointerChain, addr) + if pd, ok := d.pointers[addr]; ok && pd < d.depth { + cycleFound = true + indirects-- + break + } + d.pointers[addr] = d.depth + + ve = ve.Elem() + if ve.Kind() == reflect.Interface { + if ve.IsNil() { + nilFound = true + break + } + ve = ve.Elem() + } + } + + // Display type information. + d.w.Write(openParenBytes) + d.w.Write(bytes.Repeat(asteriskBytes, indirects)) + d.w.Write([]byte(ve.Type().String())) + d.w.Write(closeParenBytes) + + // Display pointer information. + if !d.cs.DisablePointerAddresses && len(pointerChain) > 0 { + d.w.Write(openParenBytes) + for i, addr := range pointerChain { + if i > 0 { + d.w.Write(pointerChainBytes) + } + printHexPtr(d.w, addr) + } + d.w.Write(closeParenBytes) + } + + // Display dereferenced value. + d.w.Write(openParenBytes) + switch { + case nilFound == true: + d.w.Write(nilAngleBytes) + + case cycleFound == true: + d.w.Write(circularBytes) + + default: + d.ignoreNextType = true + d.dump(ve) + } + d.w.Write(closeParenBytes) +} + +// dumpSlice handles formatting of arrays and slices. Byte (uint8 under +// reflection) arrays and slices are dumped in hexdump -C fashion. +func (d *dumpState) dumpSlice(v reflect.Value) { + // Determine whether this type should be hex dumped or not. Also, + // for types which should be hexdumped, try to use the underlying data + // first, then fall back to trying to convert them to a uint8 slice. + var buf []uint8 + doConvert := false + doHexDump := false + numEntries := v.Len() + if numEntries > 0 { + vt := v.Index(0).Type() + vts := vt.String() + switch { + // C types that need to be converted. + case cCharRE.MatchString(vts): + fallthrough + case cUnsignedCharRE.MatchString(vts): + fallthrough + case cUint8tCharRE.MatchString(vts): + doConvert = true + + // Try to use existing uint8 slices and fall back to converting + // and copying if that fails. + case vt.Kind() == reflect.Uint8: + // We need an addressable interface to convert the type + // to a byte slice. However, the reflect package won't + // give us an interface on certain things like + // unexported struct fields in order to enforce + // visibility rules. We use unsafe, when available, to + // bypass these restrictions since this package does not + // mutate the values. + vs := v + if !vs.CanInterface() || !vs.CanAddr() { + vs = unsafeReflectValue(vs) + } + if !UnsafeDisabled { + vs = vs.Slice(0, numEntries) + + // Use the existing uint8 slice if it can be + // type asserted. + iface := vs.Interface() + if slice, ok := iface.([]uint8); ok { + buf = slice + doHexDump = true + break + } + } + + // The underlying data needs to be converted if it can't + // be type asserted to a uint8 slice. + doConvert = true + } + + // Copy and convert the underlying type if needed. + if doConvert && vt.ConvertibleTo(uint8Type) { + // Convert and copy each element into a uint8 byte + // slice. + buf = make([]uint8, numEntries) + for i := 0; i < numEntries; i++ { + vv := v.Index(i) + buf[i] = uint8(vv.Convert(uint8Type).Uint()) + } + doHexDump = true + } + } + + // Hexdump the entire slice as needed. + if doHexDump { + indent := strings.Repeat(d.cs.Indent, d.depth) + str := indent + hex.Dump(buf) + str = strings.Replace(str, "\n", "\n"+indent, -1) + str = strings.TrimRight(str, d.cs.Indent) + d.w.Write([]byte(str)) + return + } + + // Recursively call dump for each item. + for i := 0; i < numEntries; i++ { + d.dump(d.unpackValue(v.Index(i))) + if i < (numEntries - 1) { + d.w.Write(commaNewlineBytes) + } else { + d.w.Write(newlineBytes) + } + } +} + +// dump is the main workhorse for dumping a value. It uses the passed reflect +// value to figure out what kind of object we are dealing with and formats it +// appropriately. It is a recursive function, however circular data structures +// are detected and handled properly. +func (d *dumpState) dump(v reflect.Value) { + // Handle invalid reflect values immediately. + kind := v.Kind() + if kind == reflect.Invalid { + d.w.Write(invalidAngleBytes) + return + } + + // Handle pointers specially. + if kind == reflect.Ptr { + d.indent() + d.dumpPtr(v) + return + } + + // Print type information unless already handled elsewhere. + if !d.ignoreNextType { + d.indent() + d.w.Write(openParenBytes) + d.w.Write([]byte(v.Type().String())) + d.w.Write(closeParenBytes) + d.w.Write(spaceBytes) + } + d.ignoreNextType = false + + // Display length and capacity if the built-in len and cap functions + // work with the value's kind and the len/cap itself is non-zero. + valueLen, valueCap := 0, 0 + switch v.Kind() { + case reflect.Array, reflect.Slice, reflect.Chan: + valueLen, valueCap = v.Len(), v.Cap() + case reflect.Map, reflect.String: + valueLen = v.Len() + } + if valueLen != 0 || !d.cs.DisableCapacities && valueCap != 0 { + d.w.Write(openParenBytes) + if valueLen != 0 { + d.w.Write(lenEqualsBytes) + printInt(d.w, int64(valueLen), 10) + } + if !d.cs.DisableCapacities && valueCap != 0 { + if valueLen != 0 { + d.w.Write(spaceBytes) + } + d.w.Write(capEqualsBytes) + printInt(d.w, int64(valueCap), 10) + } + d.w.Write(closeParenBytes) + d.w.Write(spaceBytes) + } + + // Call Stringer/error interfaces if they exist and the handle methods flag + // is enabled + if !d.cs.DisableMethods { + if (kind != reflect.Invalid) && (kind != reflect.Interface) { + if handled := handleMethods(d.cs, d.w, v); handled { + return + } + } + } + + switch kind { + case reflect.Invalid: + // Do nothing. We should never get here since invalid has already + // been handled above. + + case reflect.Bool: + printBool(d.w, v.Bool()) + + case reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, reflect.Int: + printInt(d.w, v.Int(), 10) + + case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uint: + printUint(d.w, v.Uint(), 10) + + case reflect.Float32: + printFloat(d.w, v.Float(), 32) + + case reflect.Float64: + printFloat(d.w, v.Float(), 64) + + case reflect.Complex64: + printComplex(d.w, v.Complex(), 32) + + case reflect.Complex128: + printComplex(d.w, v.Complex(), 64) + + case reflect.Slice: + if v.IsNil() { + d.w.Write(nilAngleBytes) + break + } + fallthrough + + case reflect.Array: + d.w.Write(openBraceNewlineBytes) + d.depth++ + if (d.cs.MaxDepth != 0) && (d.depth > d.cs.MaxDepth) { + d.indent() + d.w.Write(maxNewlineBytes) + } else { + d.dumpSlice(v) + } + d.depth-- + d.indent() + d.w.Write(closeBraceBytes) + + case reflect.String: + d.w.Write([]byte(strconv.Quote(v.String()))) + + case reflect.Interface: + // The only time we should get here is for nil interfaces due to + // unpackValue calls. + if v.IsNil() { + d.w.Write(nilAngleBytes) + } + + case reflect.Ptr: + // Do nothing. We should never get here since pointers have already + // been handled above. + + case reflect.Map: + // nil maps should be indicated as different than empty maps + if v.IsNil() { + d.w.Write(nilAngleBytes) + break + } + + d.w.Write(openBraceNewlineBytes) + d.depth++ + if (d.cs.MaxDepth != 0) && (d.depth > d.cs.MaxDepth) { + d.indent() + d.w.Write(maxNewlineBytes) + } else { + numEntries := v.Len() + keys := v.MapKeys() + if d.cs.SortKeys { + sortValues(keys, d.cs) + } + for i, key := range keys { + d.dump(d.unpackValue(key)) + d.w.Write(colonSpaceBytes) + d.ignoreNextIndent = true + d.dump(d.unpackValue(v.MapIndex(key))) + if i < (numEntries - 1) { + d.w.Write(commaNewlineBytes) + } else { + d.w.Write(newlineBytes) + } + } + } + d.depth-- + d.indent() + d.w.Write(closeBraceBytes) + + case reflect.Struct: + d.w.Write(openBraceNewlineBytes) + d.depth++ + if (d.cs.MaxDepth != 0) && (d.depth > d.cs.MaxDepth) { + d.indent() + d.w.Write(maxNewlineBytes) + } else { + vt := v.Type() + numFields := v.NumField() + for i := 0; i < numFields; i++ { + d.indent() + vtf := vt.Field(i) + d.w.Write([]byte(vtf.Name)) + d.w.Write(colonSpaceBytes) + d.ignoreNextIndent = true + d.dump(d.unpackValue(v.Field(i))) + if i < (numFields - 1) { + d.w.Write(commaNewlineBytes) + } else { + d.w.Write(newlineBytes) + } + } + } + d.depth-- + d.indent() + d.w.Write(closeBraceBytes) + + case reflect.Uintptr: + printHexPtr(d.w, uintptr(v.Uint())) + + case reflect.UnsafePointer, reflect.Chan, reflect.Func: + printHexPtr(d.w, v.Pointer()) + + // There were not any other types at the time this code was written, but + // fall back to letting the default fmt package handle it in case any new + // types are added. + default: + if v.CanInterface() { + fmt.Fprintf(d.w, "%v", v.Interface()) + } else { + fmt.Fprintf(d.w, "%v", v.String()) + } + } +} + +// fdump is a helper function to consolidate the logic from the various public +// methods which take varying writers and config states. +func fdump(cs *ConfigState, w io.Writer, a ...interface{}) { + for _, arg := range a { + if arg == nil { + w.Write(interfaceBytes) + w.Write(spaceBytes) + w.Write(nilAngleBytes) + w.Write(newlineBytes) + continue + } + + d := dumpState{w: w, cs: cs} + d.pointers = make(map[uintptr]int) + d.dump(reflect.ValueOf(arg)) + d.w.Write(newlineBytes) + } +} + +// Fdump formats and displays the passed arguments to io.Writer w. It formats +// exactly the same as Dump. +func Fdump(w io.Writer, a ...interface{}) { + fdump(&Config, w, a...) +} + +// Sdump returns a string with the passed arguments formatted exactly the same +// as Dump. +func Sdump(a ...interface{}) string { + var buf bytes.Buffer + fdump(&Config, &buf, a...) + return buf.String() +} + +/* +Dump displays the passed parameters to standard out with newlines, customizable +indentation, and additional debug information such as complete types and all +pointer addresses used to indirect to the final value. It provides the +following features over the built-in printing facilities provided by the fmt +package: + + * Pointers are dereferenced and followed + * Circular data structures are detected and handled properly + * Custom Stringer/error interfaces are optionally invoked, including + on unexported types + * Custom types which only implement the Stringer/error interfaces via + a pointer receiver are optionally invoked when passing non-pointer + variables + * Byte arrays and slices are dumped like the hexdump -C command which + includes offsets, byte values in hex, and ASCII output + +The configuration options are controlled by an exported package global, +spew.Config. See ConfigState for options documentation. + +See Fdump if you would prefer dumping to an arbitrary io.Writer or Sdump to +get the formatted result as a string. +*/ +func Dump(a ...interface{}) { + fdump(&Config, os.Stdout, a...) +} diff --git a/vendor/github.com/davecgh/go-spew/spew/format.go b/vendor/github.com/davecgh/go-spew/spew/format.go new file mode 100644 index 000000000000..c49875bacbb8 --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/format.go @@ -0,0 +1,419 @@ +/* + * Copyright (c) 2013-2016 Dave Collins + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + */ + +package spew + +import ( + "bytes" + "fmt" + "reflect" + "strconv" + "strings" +) + +// supportedFlags is a list of all the character flags supported by fmt package. +const supportedFlags = "0-+# " + +// formatState implements the fmt.Formatter interface and contains information +// about the state of a formatting operation. The NewFormatter function can +// be used to get a new Formatter which can be used directly as arguments +// in standard fmt package printing calls. +type formatState struct { + value interface{} + fs fmt.State + depth int + pointers map[uintptr]int + ignoreNextType bool + cs *ConfigState +} + +// buildDefaultFormat recreates the original format string without precision +// and width information to pass in to fmt.Sprintf in the case of an +// unrecognized type. Unless new types are added to the language, this +// function won't ever be called. +func (f *formatState) buildDefaultFormat() (format string) { + buf := bytes.NewBuffer(percentBytes) + + for _, flag := range supportedFlags { + if f.fs.Flag(int(flag)) { + buf.WriteRune(flag) + } + } + + buf.WriteRune('v') + + format = buf.String() + return format +} + +// constructOrigFormat recreates the original format string including precision +// and width information to pass along to the standard fmt package. This allows +// automatic deferral of all format strings this package doesn't support. +func (f *formatState) constructOrigFormat(verb rune) (format string) { + buf := bytes.NewBuffer(percentBytes) + + for _, flag := range supportedFlags { + if f.fs.Flag(int(flag)) { + buf.WriteRune(flag) + } + } + + if width, ok := f.fs.Width(); ok { + buf.WriteString(strconv.Itoa(width)) + } + + if precision, ok := f.fs.Precision(); ok { + buf.Write(precisionBytes) + buf.WriteString(strconv.Itoa(precision)) + } + + buf.WriteRune(verb) + + format = buf.String() + return format +} + +// unpackValue returns values inside of non-nil interfaces when possible and +// ensures that types for values which have been unpacked from an interface +// are displayed when the show types flag is also set. +// This is useful for data types like structs, arrays, slices, and maps which +// can contain varying types packed inside an interface. +func (f *formatState) unpackValue(v reflect.Value) reflect.Value { + if v.Kind() == reflect.Interface { + f.ignoreNextType = false + if !v.IsNil() { + v = v.Elem() + } + } + return v +} + +// formatPtr handles formatting of pointers by indirecting them as necessary. +func (f *formatState) formatPtr(v reflect.Value) { + // Display nil if top level pointer is nil. + showTypes := f.fs.Flag('#') + if v.IsNil() && (!showTypes || f.ignoreNextType) { + f.fs.Write(nilAngleBytes) + return + } + + // Remove pointers at or below the current depth from map used to detect + // circular refs. + for k, depth := range f.pointers { + if depth >= f.depth { + delete(f.pointers, k) + } + } + + // Keep list of all dereferenced pointers to possibly show later. + pointerChain := make([]uintptr, 0) + + // Figure out how many levels of indirection there are by derferencing + // pointers and unpacking interfaces down the chain while detecting circular + // references. + nilFound := false + cycleFound := false + indirects := 0 + ve := v + for ve.Kind() == reflect.Ptr { + if ve.IsNil() { + nilFound = true + break + } + indirects++ + addr := ve.Pointer() + pointerChain = append(pointerChain, addr) + if pd, ok := f.pointers[addr]; ok && pd < f.depth { + cycleFound = true + indirects-- + break + } + f.pointers[addr] = f.depth + + ve = ve.Elem() + if ve.Kind() == reflect.Interface { + if ve.IsNil() { + nilFound = true + break + } + ve = ve.Elem() + } + } + + // Display type or indirection level depending on flags. + if showTypes && !f.ignoreNextType { + f.fs.Write(openParenBytes) + f.fs.Write(bytes.Repeat(asteriskBytes, indirects)) + f.fs.Write([]byte(ve.Type().String())) + f.fs.Write(closeParenBytes) + } else { + if nilFound || cycleFound { + indirects += strings.Count(ve.Type().String(), "*") + } + f.fs.Write(openAngleBytes) + f.fs.Write([]byte(strings.Repeat("*", indirects))) + f.fs.Write(closeAngleBytes) + } + + // Display pointer information depending on flags. + if f.fs.Flag('+') && (len(pointerChain) > 0) { + f.fs.Write(openParenBytes) + for i, addr := range pointerChain { + if i > 0 { + f.fs.Write(pointerChainBytes) + } + printHexPtr(f.fs, addr) + } + f.fs.Write(closeParenBytes) + } + + // Display dereferenced value. + switch { + case nilFound == true: + f.fs.Write(nilAngleBytes) + + case cycleFound == true: + f.fs.Write(circularShortBytes) + + default: + f.ignoreNextType = true + f.format(ve) + } +} + +// format is the main workhorse for providing the Formatter interface. It +// uses the passed reflect value to figure out what kind of object we are +// dealing with and formats it appropriately. It is a recursive function, +// however circular data structures are detected and handled properly. +func (f *formatState) format(v reflect.Value) { + // Handle invalid reflect values immediately. + kind := v.Kind() + if kind == reflect.Invalid { + f.fs.Write(invalidAngleBytes) + return + } + + // Handle pointers specially. + if kind == reflect.Ptr { + f.formatPtr(v) + return + } + + // Print type information unless already handled elsewhere. + if !f.ignoreNextType && f.fs.Flag('#') { + f.fs.Write(openParenBytes) + f.fs.Write([]byte(v.Type().String())) + f.fs.Write(closeParenBytes) + } + f.ignoreNextType = false + + // Call Stringer/error interfaces if they exist and the handle methods + // flag is enabled. + if !f.cs.DisableMethods { + if (kind != reflect.Invalid) && (kind != reflect.Interface) { + if handled := handleMethods(f.cs, f.fs, v); handled { + return + } + } + } + + switch kind { + case reflect.Invalid: + // Do nothing. We should never get here since invalid has already + // been handled above. + + case reflect.Bool: + printBool(f.fs, v.Bool()) + + case reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, reflect.Int: + printInt(f.fs, v.Int(), 10) + + case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uint: + printUint(f.fs, v.Uint(), 10) + + case reflect.Float32: + printFloat(f.fs, v.Float(), 32) + + case reflect.Float64: + printFloat(f.fs, v.Float(), 64) + + case reflect.Complex64: + printComplex(f.fs, v.Complex(), 32) + + case reflect.Complex128: + printComplex(f.fs, v.Complex(), 64) + + case reflect.Slice: + if v.IsNil() { + f.fs.Write(nilAngleBytes) + break + } + fallthrough + + case reflect.Array: + f.fs.Write(openBracketBytes) + f.depth++ + if (f.cs.MaxDepth != 0) && (f.depth > f.cs.MaxDepth) { + f.fs.Write(maxShortBytes) + } else { + numEntries := v.Len() + for i := 0; i < numEntries; i++ { + if i > 0 { + f.fs.Write(spaceBytes) + } + f.ignoreNextType = true + f.format(f.unpackValue(v.Index(i))) + } + } + f.depth-- + f.fs.Write(closeBracketBytes) + + case reflect.String: + f.fs.Write([]byte(v.String())) + + case reflect.Interface: + // The only time we should get here is for nil interfaces due to + // unpackValue calls. + if v.IsNil() { + f.fs.Write(nilAngleBytes) + } + + case reflect.Ptr: + // Do nothing. We should never get here since pointers have already + // been handled above. + + case reflect.Map: + // nil maps should be indicated as different than empty maps + if v.IsNil() { + f.fs.Write(nilAngleBytes) + break + } + + f.fs.Write(openMapBytes) + f.depth++ + if (f.cs.MaxDepth != 0) && (f.depth > f.cs.MaxDepth) { + f.fs.Write(maxShortBytes) + } else { + keys := v.MapKeys() + if f.cs.SortKeys { + sortValues(keys, f.cs) + } + for i, key := range keys { + if i > 0 { + f.fs.Write(spaceBytes) + } + f.ignoreNextType = true + f.format(f.unpackValue(key)) + f.fs.Write(colonBytes) + f.ignoreNextType = true + f.format(f.unpackValue(v.MapIndex(key))) + } + } + f.depth-- + f.fs.Write(closeMapBytes) + + case reflect.Struct: + numFields := v.NumField() + f.fs.Write(openBraceBytes) + f.depth++ + if (f.cs.MaxDepth != 0) && (f.depth > f.cs.MaxDepth) { + f.fs.Write(maxShortBytes) + } else { + vt := v.Type() + for i := 0; i < numFields; i++ { + if i > 0 { + f.fs.Write(spaceBytes) + } + vtf := vt.Field(i) + if f.fs.Flag('+') || f.fs.Flag('#') { + f.fs.Write([]byte(vtf.Name)) + f.fs.Write(colonBytes) + } + f.format(f.unpackValue(v.Field(i))) + } + } + f.depth-- + f.fs.Write(closeBraceBytes) + + case reflect.Uintptr: + printHexPtr(f.fs, uintptr(v.Uint())) + + case reflect.UnsafePointer, reflect.Chan, reflect.Func: + printHexPtr(f.fs, v.Pointer()) + + // There were not any other types at the time this code was written, but + // fall back to letting the default fmt package handle it if any get added. + default: + format := f.buildDefaultFormat() + if v.CanInterface() { + fmt.Fprintf(f.fs, format, v.Interface()) + } else { + fmt.Fprintf(f.fs, format, v.String()) + } + } +} + +// Format satisfies the fmt.Formatter interface. See NewFormatter for usage +// details. +func (f *formatState) Format(fs fmt.State, verb rune) { + f.fs = fs + + // Use standard formatting for verbs that are not v. + if verb != 'v' { + format := f.constructOrigFormat(verb) + fmt.Fprintf(fs, format, f.value) + return + } + + if f.value == nil { + if fs.Flag('#') { + fs.Write(interfaceBytes) + } + fs.Write(nilAngleBytes) + return + } + + f.format(reflect.ValueOf(f.value)) +} + +// newFormatter is a helper function to consolidate the logic from the various +// public methods which take varying config states. +func newFormatter(cs *ConfigState, v interface{}) fmt.Formatter { + fs := &formatState{value: v, cs: cs} + fs.pointers = make(map[uintptr]int) + return fs +} + +/* +NewFormatter returns a custom formatter that satisfies the fmt.Formatter +interface. As a result, it integrates cleanly with standard fmt package +printing functions. The formatter is useful for inline printing of smaller data +types similar to the standard %v format specifier. + +The custom formatter only responds to the %v (most compact), %+v (adds pointer +addresses), %#v (adds types), or %#+v (adds types and pointer addresses) verb +combinations. Any other verbs such as %x and %q will be sent to the the +standard fmt package for formatting. In addition, the custom formatter ignores +the width and precision arguments (however they will still work on the format +specifiers not handled by the custom formatter). + +Typically this function shouldn't be called directly. It is much easier to make +use of the custom formatter by calling one of the convenience functions such as +Printf, Println, or Fprintf. +*/ +func NewFormatter(v interface{}) fmt.Formatter { + return newFormatter(&Config, v) +} diff --git a/vendor/github.com/davecgh/go-spew/spew/spew.go b/vendor/github.com/davecgh/go-spew/spew/spew.go new file mode 100644 index 000000000000..32c0e3388253 --- /dev/null +++ b/vendor/github.com/davecgh/go-spew/spew/spew.go @@ -0,0 +1,148 @@ +/* + * Copyright (c) 2013-2016 Dave Collins + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + */ + +package spew + +import ( + "fmt" + "io" +) + +// Errorf is a wrapper for fmt.Errorf that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the formatted string as a value that satisfies error. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Errorf(format, spew.NewFormatter(a), spew.NewFormatter(b)) +func Errorf(format string, a ...interface{}) (err error) { + return fmt.Errorf(format, convertArgs(a)...) +} + +// Fprint is a wrapper for fmt.Fprint that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Fprint(w, spew.NewFormatter(a), spew.NewFormatter(b)) +func Fprint(w io.Writer, a ...interface{}) (n int, err error) { + return fmt.Fprint(w, convertArgs(a)...) +} + +// Fprintf is a wrapper for fmt.Fprintf that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Fprintf(w, format, spew.NewFormatter(a), spew.NewFormatter(b)) +func Fprintf(w io.Writer, format string, a ...interface{}) (n int, err error) { + return fmt.Fprintf(w, format, convertArgs(a)...) +} + +// Fprintln is a wrapper for fmt.Fprintln that treats each argument as if it +// passed with a default Formatter interface returned by NewFormatter. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Fprintln(w, spew.NewFormatter(a), spew.NewFormatter(b)) +func Fprintln(w io.Writer, a ...interface{}) (n int, err error) { + return fmt.Fprintln(w, convertArgs(a)...) +} + +// Print is a wrapper for fmt.Print that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Print(spew.NewFormatter(a), spew.NewFormatter(b)) +func Print(a ...interface{}) (n int, err error) { + return fmt.Print(convertArgs(a)...) +} + +// Printf is a wrapper for fmt.Printf that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Printf(format, spew.NewFormatter(a), spew.NewFormatter(b)) +func Printf(format string, a ...interface{}) (n int, err error) { + return fmt.Printf(format, convertArgs(a)...) +} + +// Println is a wrapper for fmt.Println that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the number of bytes written and any write error encountered. See +// NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Println(spew.NewFormatter(a), spew.NewFormatter(b)) +func Println(a ...interface{}) (n int, err error) { + return fmt.Println(convertArgs(a)...) +} + +// Sprint is a wrapper for fmt.Sprint that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the resulting string. See NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Sprint(spew.NewFormatter(a), spew.NewFormatter(b)) +func Sprint(a ...interface{}) string { + return fmt.Sprint(convertArgs(a)...) +} + +// Sprintf is a wrapper for fmt.Sprintf that treats each argument as if it were +// passed with a default Formatter interface returned by NewFormatter. It +// returns the resulting string. See NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Sprintf(format, spew.NewFormatter(a), spew.NewFormatter(b)) +func Sprintf(format string, a ...interface{}) string { + return fmt.Sprintf(format, convertArgs(a)...) +} + +// Sprintln is a wrapper for fmt.Sprintln that treats each argument as if it +// were passed with a default Formatter interface returned by NewFormatter. It +// returns the resulting string. See NewFormatter for formatting details. +// +// This function is shorthand for the following syntax: +// +// fmt.Sprintln(spew.NewFormatter(a), spew.NewFormatter(b)) +func Sprintln(a ...interface{}) string { + return fmt.Sprintln(convertArgs(a)...) +} + +// convertArgs accepts a slice of arguments and returns a slice of the same +// length with each argument converted to a default spew Formatter interface. +func convertArgs(args []interface{}) (formatters []interface{}) { + formatters = make([]interface{}, len(args)) + for index, arg := range args { + formatters[index] = NewFormatter(arg) + } + return formatters +} diff --git a/vendor/github.com/pmezard/go-difflib/LICENSE b/vendor/github.com/pmezard/go-difflib/LICENSE new file mode 100644 index 000000000000..c67dad612a3d --- /dev/null +++ b/vendor/github.com/pmezard/go-difflib/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2013, Patrick Mezard +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + The names of its contributors may not be used to endorse or promote +products derived from this software without specific prior written +permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS +IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED +TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A +PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/pmezard/go-difflib/difflib/difflib.go b/vendor/github.com/pmezard/go-difflib/difflib/difflib.go new file mode 100644 index 000000000000..003e99fadb4f --- /dev/null +++ b/vendor/github.com/pmezard/go-difflib/difflib/difflib.go @@ -0,0 +1,772 @@ +// Package difflib is a partial port of Python difflib module. +// +// It provides tools to compare sequences of strings and generate textual diffs. +// +// The following class and functions have been ported: +// +// - SequenceMatcher +// +// - unified_diff +// +// - context_diff +// +// Getting unified diffs was the main goal of the port. Keep in mind this code +// is mostly suitable to output text differences in a human friendly way, there +// are no guarantees generated diffs are consumable by patch(1). +package difflib + +import ( + "bufio" + "bytes" + "fmt" + "io" + "strings" +) + +func min(a, b int) int { + if a < b { + return a + } + return b +} + +func max(a, b int) int { + if a > b { + return a + } + return b +} + +func calculateRatio(matches, length int) float64 { + if length > 0 { + return 2.0 * float64(matches) / float64(length) + } + return 1.0 +} + +type Match struct { + A int + B int + Size int +} + +type OpCode struct { + Tag byte + I1 int + I2 int + J1 int + J2 int +} + +// SequenceMatcher compares sequence of strings. The basic +// algorithm predates, and is a little fancier than, an algorithm +// published in the late 1980's by Ratcliff and Obershelp under the +// hyperbolic name "gestalt pattern matching". The basic idea is to find +// the longest contiguous matching subsequence that contains no "junk" +// elements (R-O doesn't address junk). The same idea is then applied +// recursively to the pieces of the sequences to the left and to the right +// of the matching subsequence. This does not yield minimal edit +// sequences, but does tend to yield matches that "look right" to people. +// +// SequenceMatcher tries to compute a "human-friendly diff" between two +// sequences. Unlike e.g. UNIX(tm) diff, the fundamental notion is the +// longest *contiguous* & junk-free matching subsequence. That's what +// catches peoples' eyes. The Windows(tm) windiff has another interesting +// notion, pairing up elements that appear uniquely in each sequence. +// That, and the method here, appear to yield more intuitive difference +// reports than does diff. This method appears to be the least vulnerable +// to synching up on blocks of "junk lines", though (like blank lines in +// ordinary text files, or maybe "

" lines in HTML files). That may be +// because this is the only method of the 3 that has a *concept* of +// "junk" . +// +// Timing: Basic R-O is cubic time worst case and quadratic time expected +// case. SequenceMatcher is quadratic time for the worst case and has +// expected-case behavior dependent in a complicated way on how many +// elements the sequences have in common; best case time is linear. +type SequenceMatcher struct { + a []string + b []string + b2j map[string][]int + IsJunk func(string) bool + autoJunk bool + bJunk map[string]struct{} + matchingBlocks []Match + fullBCount map[string]int + bPopular map[string]struct{} + opCodes []OpCode +} + +func NewMatcher(a, b []string) *SequenceMatcher { + m := SequenceMatcher{autoJunk: true} + m.SetSeqs(a, b) + return &m +} + +func NewMatcherWithJunk(a, b []string, autoJunk bool, + isJunk func(string) bool) *SequenceMatcher { + + m := SequenceMatcher{IsJunk: isJunk, autoJunk: autoJunk} + m.SetSeqs(a, b) + return &m +} + +// Set two sequences to be compared. +func (m *SequenceMatcher) SetSeqs(a, b []string) { + m.SetSeq1(a) + m.SetSeq2(b) +} + +// Set the first sequence to be compared. The second sequence to be compared is +// not changed. +// +// SequenceMatcher computes and caches detailed information about the second +// sequence, so if you want to compare one sequence S against many sequences, +// use .SetSeq2(s) once and call .SetSeq1(x) repeatedly for each of the other +// sequences. +// +// See also SetSeqs() and SetSeq2(). +func (m *SequenceMatcher) SetSeq1(a []string) { + if &a == &m.a { + return + } + m.a = a + m.matchingBlocks = nil + m.opCodes = nil +} + +// Set the second sequence to be compared. The first sequence to be compared is +// not changed. +func (m *SequenceMatcher) SetSeq2(b []string) { + if &b == &m.b { + return + } + m.b = b + m.matchingBlocks = nil + m.opCodes = nil + m.fullBCount = nil + m.chainB() +} + +func (m *SequenceMatcher) chainB() { + // Populate line -> index mapping + b2j := map[string][]int{} + for i, s := range m.b { + indices := b2j[s] + indices = append(indices, i) + b2j[s] = indices + } + + // Purge junk elements + m.bJunk = map[string]struct{}{} + if m.IsJunk != nil { + junk := m.bJunk + for s, _ := range b2j { + if m.IsJunk(s) { + junk[s] = struct{}{} + } + } + for s, _ := range junk { + delete(b2j, s) + } + } + + // Purge remaining popular elements + popular := map[string]struct{}{} + n := len(m.b) + if m.autoJunk && n >= 200 { + ntest := n/100 + 1 + for s, indices := range b2j { + if len(indices) > ntest { + popular[s] = struct{}{} + } + } + for s, _ := range popular { + delete(b2j, s) + } + } + m.bPopular = popular + m.b2j = b2j +} + +func (m *SequenceMatcher) isBJunk(s string) bool { + _, ok := m.bJunk[s] + return ok +} + +// Find longest matching block in a[alo:ahi] and b[blo:bhi]. +// +// If IsJunk is not defined: +// +// Return (i,j,k) such that a[i:i+k] is equal to b[j:j+k], where +// alo <= i <= i+k <= ahi +// blo <= j <= j+k <= bhi +// and for all (i',j',k') meeting those conditions, +// k >= k' +// i <= i' +// and if i == i', j <= j' +// +// In other words, of all maximal matching blocks, return one that +// starts earliest in a, and of all those maximal matching blocks that +// start earliest in a, return the one that starts earliest in b. +// +// If IsJunk is defined, first the longest matching block is +// determined as above, but with the additional restriction that no +// junk element appears in the block. Then that block is extended as +// far as possible by matching (only) junk elements on both sides. So +// the resulting block never matches on junk except as identical junk +// happens to be adjacent to an "interesting" match. +// +// If no blocks match, return (alo, blo, 0). +func (m *SequenceMatcher) findLongestMatch(alo, ahi, blo, bhi int) Match { + // CAUTION: stripping common prefix or suffix would be incorrect. + // E.g., + // ab + // acab + // Longest matching block is "ab", but if common prefix is + // stripped, it's "a" (tied with "b"). UNIX(tm) diff does so + // strip, so ends up claiming that ab is changed to acab by + // inserting "ca" in the middle. That's minimal but unintuitive: + // "it's obvious" that someone inserted "ac" at the front. + // Windiff ends up at the same place as diff, but by pairing up + // the unique 'b's and then matching the first two 'a's. + besti, bestj, bestsize := alo, blo, 0 + + // find longest junk-free match + // during an iteration of the loop, j2len[j] = length of longest + // junk-free match ending with a[i-1] and b[j] + j2len := map[int]int{} + for i := alo; i != ahi; i++ { + // look at all instances of a[i] in b; note that because + // b2j has no junk keys, the loop is skipped if a[i] is junk + newj2len := map[int]int{} + for _, j := range m.b2j[m.a[i]] { + // a[i] matches b[j] + if j < blo { + continue + } + if j >= bhi { + break + } + k := j2len[j-1] + 1 + newj2len[j] = k + if k > bestsize { + besti, bestj, bestsize = i-k+1, j-k+1, k + } + } + j2len = newj2len + } + + // Extend the best by non-junk elements on each end. In particular, + // "popular" non-junk elements aren't in b2j, which greatly speeds + // the inner loop above, but also means "the best" match so far + // doesn't contain any junk *or* popular non-junk elements. + for besti > alo && bestj > blo && !m.isBJunk(m.b[bestj-1]) && + m.a[besti-1] == m.b[bestj-1] { + besti, bestj, bestsize = besti-1, bestj-1, bestsize+1 + } + for besti+bestsize < ahi && bestj+bestsize < bhi && + !m.isBJunk(m.b[bestj+bestsize]) && + m.a[besti+bestsize] == m.b[bestj+bestsize] { + bestsize += 1 + } + + // Now that we have a wholly interesting match (albeit possibly + // empty!), we may as well suck up the matching junk on each + // side of it too. Can't think of a good reason not to, and it + // saves post-processing the (possibly considerable) expense of + // figuring out what to do with it. In the case of an empty + // interesting match, this is clearly the right thing to do, + // because no other kind of match is possible in the regions. + for besti > alo && bestj > blo && m.isBJunk(m.b[bestj-1]) && + m.a[besti-1] == m.b[bestj-1] { + besti, bestj, bestsize = besti-1, bestj-1, bestsize+1 + } + for besti+bestsize < ahi && bestj+bestsize < bhi && + m.isBJunk(m.b[bestj+bestsize]) && + m.a[besti+bestsize] == m.b[bestj+bestsize] { + bestsize += 1 + } + + return Match{A: besti, B: bestj, Size: bestsize} +} + +// Return list of triples describing matching subsequences. +// +// Each triple is of the form (i, j, n), and means that +// a[i:i+n] == b[j:j+n]. The triples are monotonically increasing in +// i and in j. It's also guaranteed that if (i, j, n) and (i', j', n') are +// adjacent triples in the list, and the second is not the last triple in the +// list, then i+n != i' or j+n != j'. IOW, adjacent triples never describe +// adjacent equal blocks. +// +// The last triple is a dummy, (len(a), len(b), 0), and is the only +// triple with n==0. +func (m *SequenceMatcher) GetMatchingBlocks() []Match { + if m.matchingBlocks != nil { + return m.matchingBlocks + } + + var matchBlocks func(alo, ahi, blo, bhi int, matched []Match) []Match + matchBlocks = func(alo, ahi, blo, bhi int, matched []Match) []Match { + match := m.findLongestMatch(alo, ahi, blo, bhi) + i, j, k := match.A, match.B, match.Size + if match.Size > 0 { + if alo < i && blo < j { + matched = matchBlocks(alo, i, blo, j, matched) + } + matched = append(matched, match) + if i+k < ahi && j+k < bhi { + matched = matchBlocks(i+k, ahi, j+k, bhi, matched) + } + } + return matched + } + matched := matchBlocks(0, len(m.a), 0, len(m.b), nil) + + // It's possible that we have adjacent equal blocks in the + // matching_blocks list now. + nonAdjacent := []Match{} + i1, j1, k1 := 0, 0, 0 + for _, b := range matched { + // Is this block adjacent to i1, j1, k1? + i2, j2, k2 := b.A, b.B, b.Size + if i1+k1 == i2 && j1+k1 == j2 { + // Yes, so collapse them -- this just increases the length of + // the first block by the length of the second, and the first + // block so lengthened remains the block to compare against. + k1 += k2 + } else { + // Not adjacent. Remember the first block (k1==0 means it's + // the dummy we started with), and make the second block the + // new block to compare against. + if k1 > 0 { + nonAdjacent = append(nonAdjacent, Match{i1, j1, k1}) + } + i1, j1, k1 = i2, j2, k2 + } + } + if k1 > 0 { + nonAdjacent = append(nonAdjacent, Match{i1, j1, k1}) + } + + nonAdjacent = append(nonAdjacent, Match{len(m.a), len(m.b), 0}) + m.matchingBlocks = nonAdjacent + return m.matchingBlocks +} + +// Return list of 5-tuples describing how to turn a into b. +// +// Each tuple is of the form (tag, i1, i2, j1, j2). The first tuple +// has i1 == j1 == 0, and remaining tuples have i1 == the i2 from the +// tuple preceding it, and likewise for j1 == the previous j2. +// +// The tags are characters, with these meanings: +// +// 'r' (replace): a[i1:i2] should be replaced by b[j1:j2] +// +// 'd' (delete): a[i1:i2] should be deleted, j1==j2 in this case. +// +// 'i' (insert): b[j1:j2] should be inserted at a[i1:i1], i1==i2 in this case. +// +// 'e' (equal): a[i1:i2] == b[j1:j2] +func (m *SequenceMatcher) GetOpCodes() []OpCode { + if m.opCodes != nil { + return m.opCodes + } + i, j := 0, 0 + matching := m.GetMatchingBlocks() + opCodes := make([]OpCode, 0, len(matching)) + for _, m := range matching { + // invariant: we've pumped out correct diffs to change + // a[:i] into b[:j], and the next matching block is + // a[ai:ai+size] == b[bj:bj+size]. So we need to pump + // out a diff to change a[i:ai] into b[j:bj], pump out + // the matching block, and move (i,j) beyond the match + ai, bj, size := m.A, m.B, m.Size + tag := byte(0) + if i < ai && j < bj { + tag = 'r' + } else if i < ai { + tag = 'd' + } else if j < bj { + tag = 'i' + } + if tag > 0 { + opCodes = append(opCodes, OpCode{tag, i, ai, j, bj}) + } + i, j = ai+size, bj+size + // the list of matching blocks is terminated by a + // sentinel with size 0 + if size > 0 { + opCodes = append(opCodes, OpCode{'e', ai, i, bj, j}) + } + } + m.opCodes = opCodes + return m.opCodes +} + +// Isolate change clusters by eliminating ranges with no changes. +// +// Return a generator of groups with up to n lines of context. +// Each group is in the same format as returned by GetOpCodes(). +func (m *SequenceMatcher) GetGroupedOpCodes(n int) [][]OpCode { + if n < 0 { + n = 3 + } + codes := m.GetOpCodes() + if len(codes) == 0 { + codes = []OpCode{OpCode{'e', 0, 1, 0, 1}} + } + // Fixup leading and trailing groups if they show no changes. + if codes[0].Tag == 'e' { + c := codes[0] + i1, i2, j1, j2 := c.I1, c.I2, c.J1, c.J2 + codes[0] = OpCode{c.Tag, max(i1, i2-n), i2, max(j1, j2-n), j2} + } + if codes[len(codes)-1].Tag == 'e' { + c := codes[len(codes)-1] + i1, i2, j1, j2 := c.I1, c.I2, c.J1, c.J2 + codes[len(codes)-1] = OpCode{c.Tag, i1, min(i2, i1+n), j1, min(j2, j1+n)} + } + nn := n + n + groups := [][]OpCode{} + group := []OpCode{} + for _, c := range codes { + i1, i2, j1, j2 := c.I1, c.I2, c.J1, c.J2 + // End the current group and start a new one whenever + // there is a large range with no changes. + if c.Tag == 'e' && i2-i1 > nn { + group = append(group, OpCode{c.Tag, i1, min(i2, i1+n), + j1, min(j2, j1+n)}) + groups = append(groups, group) + group = []OpCode{} + i1, j1 = max(i1, i2-n), max(j1, j2-n) + } + group = append(group, OpCode{c.Tag, i1, i2, j1, j2}) + } + if len(group) > 0 && !(len(group) == 1 && group[0].Tag == 'e') { + groups = append(groups, group) + } + return groups +} + +// Return a measure of the sequences' similarity (float in [0,1]). +// +// Where T is the total number of elements in both sequences, and +// M is the number of matches, this is 2.0*M / T. +// Note that this is 1 if the sequences are identical, and 0 if +// they have nothing in common. +// +// .Ratio() is expensive to compute if you haven't already computed +// .GetMatchingBlocks() or .GetOpCodes(), in which case you may +// want to try .QuickRatio() or .RealQuickRation() first to get an +// upper bound. +func (m *SequenceMatcher) Ratio() float64 { + matches := 0 + for _, m := range m.GetMatchingBlocks() { + matches += m.Size + } + return calculateRatio(matches, len(m.a)+len(m.b)) +} + +// Return an upper bound on ratio() relatively quickly. +// +// This isn't defined beyond that it is an upper bound on .Ratio(), and +// is faster to compute. +func (m *SequenceMatcher) QuickRatio() float64 { + // viewing a and b as multisets, set matches to the cardinality + // of their intersection; this counts the number of matches + // without regard to order, so is clearly an upper bound + if m.fullBCount == nil { + m.fullBCount = map[string]int{} + for _, s := range m.b { + m.fullBCount[s] = m.fullBCount[s] + 1 + } + } + + // avail[x] is the number of times x appears in 'b' less the + // number of times we've seen it in 'a' so far ... kinda + avail := map[string]int{} + matches := 0 + for _, s := range m.a { + n, ok := avail[s] + if !ok { + n = m.fullBCount[s] + } + avail[s] = n - 1 + if n > 0 { + matches += 1 + } + } + return calculateRatio(matches, len(m.a)+len(m.b)) +} + +// Return an upper bound on ratio() very quickly. +// +// This isn't defined beyond that it is an upper bound on .Ratio(), and +// is faster to compute than either .Ratio() or .QuickRatio(). +func (m *SequenceMatcher) RealQuickRatio() float64 { + la, lb := len(m.a), len(m.b) + return calculateRatio(min(la, lb), la+lb) +} + +// Convert range to the "ed" format +func formatRangeUnified(start, stop int) string { + // Per the diff spec at http://www.unix.org/single_unix_specification/ + beginning := start + 1 // lines start numbering with one + length := stop - start + if length == 1 { + return fmt.Sprintf("%d", beginning) + } + if length == 0 { + beginning -= 1 // empty ranges begin at line just before the range + } + return fmt.Sprintf("%d,%d", beginning, length) +} + +// Unified diff parameters +type UnifiedDiff struct { + A []string // First sequence lines + FromFile string // First file name + FromDate string // First file time + B []string // Second sequence lines + ToFile string // Second file name + ToDate string // Second file time + Eol string // Headers end of line, defaults to LF + Context int // Number of context lines +} + +// Compare two sequences of lines; generate the delta as a unified diff. +// +// Unified diffs are a compact way of showing line changes and a few +// lines of context. The number of context lines is set by 'n' which +// defaults to three. +// +// By default, the diff control lines (those with ---, +++, or @@) are +// created with a trailing newline. This is helpful so that inputs +// created from file.readlines() result in diffs that are suitable for +// file.writelines() since both the inputs and outputs have trailing +// newlines. +// +// For inputs that do not have trailing newlines, set the lineterm +// argument to "" so that the output will be uniformly newline free. +// +// The unidiff format normally has a header for filenames and modification +// times. Any or all of these may be specified using strings for +// 'fromfile', 'tofile', 'fromfiledate', and 'tofiledate'. +// The modification times are normally expressed in the ISO 8601 format. +func WriteUnifiedDiff(writer io.Writer, diff UnifiedDiff) error { + buf := bufio.NewWriter(writer) + defer buf.Flush() + wf := func(format string, args ...interface{}) error { + _, err := buf.WriteString(fmt.Sprintf(format, args...)) + return err + } + ws := func(s string) error { + _, err := buf.WriteString(s) + return err + } + + if len(diff.Eol) == 0 { + diff.Eol = "\n" + } + + started := false + m := NewMatcher(diff.A, diff.B) + for _, g := range m.GetGroupedOpCodes(diff.Context) { + if !started { + started = true + fromDate := "" + if len(diff.FromDate) > 0 { + fromDate = "\t" + diff.FromDate + } + toDate := "" + if len(diff.ToDate) > 0 { + toDate = "\t" + diff.ToDate + } + if diff.FromFile != "" || diff.ToFile != "" { + err := wf("--- %s%s%s", diff.FromFile, fromDate, diff.Eol) + if err != nil { + return err + } + err = wf("+++ %s%s%s", diff.ToFile, toDate, diff.Eol) + if err != nil { + return err + } + } + } + first, last := g[0], g[len(g)-1] + range1 := formatRangeUnified(first.I1, last.I2) + range2 := formatRangeUnified(first.J1, last.J2) + if err := wf("@@ -%s +%s @@%s", range1, range2, diff.Eol); err != nil { + return err + } + for _, c := range g { + i1, i2, j1, j2 := c.I1, c.I2, c.J1, c.J2 + if c.Tag == 'e' { + for _, line := range diff.A[i1:i2] { + if err := ws(" " + line); err != nil { + return err + } + } + continue + } + if c.Tag == 'r' || c.Tag == 'd' { + for _, line := range diff.A[i1:i2] { + if err := ws("-" + line); err != nil { + return err + } + } + } + if c.Tag == 'r' || c.Tag == 'i' { + for _, line := range diff.B[j1:j2] { + if err := ws("+" + line); err != nil { + return err + } + } + } + } + } + return nil +} + +// Like WriteUnifiedDiff but returns the diff a string. +func GetUnifiedDiffString(diff UnifiedDiff) (string, error) { + w := &bytes.Buffer{} + err := WriteUnifiedDiff(w, diff) + return string(w.Bytes()), err +} + +// Convert range to the "ed" format. +func formatRangeContext(start, stop int) string { + // Per the diff spec at http://www.unix.org/single_unix_specification/ + beginning := start + 1 // lines start numbering with one + length := stop - start + if length == 0 { + beginning -= 1 // empty ranges begin at line just before the range + } + if length <= 1 { + return fmt.Sprintf("%d", beginning) + } + return fmt.Sprintf("%d,%d", beginning, beginning+length-1) +} + +type ContextDiff UnifiedDiff + +// Compare two sequences of lines; generate the delta as a context diff. +// +// Context diffs are a compact way of showing line changes and a few +// lines of context. The number of context lines is set by diff.Context +// which defaults to three. +// +// By default, the diff control lines (those with *** or ---) are +// created with a trailing newline. +// +// For inputs that do not have trailing newlines, set the diff.Eol +// argument to "" so that the output will be uniformly newline free. +// +// The context diff format normally has a header for filenames and +// modification times. Any or all of these may be specified using +// strings for diff.FromFile, diff.ToFile, diff.FromDate, diff.ToDate. +// The modification times are normally expressed in the ISO 8601 format. +// If not specified, the strings default to blanks. +func WriteContextDiff(writer io.Writer, diff ContextDiff) error { + buf := bufio.NewWriter(writer) + defer buf.Flush() + var diffErr error + wf := func(format string, args ...interface{}) { + _, err := buf.WriteString(fmt.Sprintf(format, args...)) + if diffErr == nil && err != nil { + diffErr = err + } + } + ws := func(s string) { + _, err := buf.WriteString(s) + if diffErr == nil && err != nil { + diffErr = err + } + } + + if len(diff.Eol) == 0 { + diff.Eol = "\n" + } + + prefix := map[byte]string{ + 'i': "+ ", + 'd': "- ", + 'r': "! ", + 'e': " ", + } + + started := false + m := NewMatcher(diff.A, diff.B) + for _, g := range m.GetGroupedOpCodes(diff.Context) { + if !started { + started = true + fromDate := "" + if len(diff.FromDate) > 0 { + fromDate = "\t" + diff.FromDate + } + toDate := "" + if len(diff.ToDate) > 0 { + toDate = "\t" + diff.ToDate + } + if diff.FromFile != "" || diff.ToFile != "" { + wf("*** %s%s%s", diff.FromFile, fromDate, diff.Eol) + wf("--- %s%s%s", diff.ToFile, toDate, diff.Eol) + } + } + + first, last := g[0], g[len(g)-1] + ws("***************" + diff.Eol) + + range1 := formatRangeContext(first.I1, last.I2) + wf("*** %s ****%s", range1, diff.Eol) + for _, c := range g { + if c.Tag == 'r' || c.Tag == 'd' { + for _, cc := range g { + if cc.Tag == 'i' { + continue + } + for _, line := range diff.A[cc.I1:cc.I2] { + ws(prefix[cc.Tag] + line) + } + } + break + } + } + + range2 := formatRangeContext(first.J1, last.J2) + wf("--- %s ----%s", range2, diff.Eol) + for _, c := range g { + if c.Tag == 'r' || c.Tag == 'i' { + for _, cc := range g { + if cc.Tag == 'd' { + continue + } + for _, line := range diff.B[cc.J1:cc.J2] { + ws(prefix[cc.Tag] + line) + } + } + break + } + } + } + return diffErr +} + +// Like WriteContextDiff but returns the diff a string. +func GetContextDiffString(diff ContextDiff) (string, error) { + w := &bytes.Buffer{} + err := WriteContextDiff(w, diff) + return string(w.Bytes()), err +} + +// Split a string on "\n" while preserving them. The output can be used +// as input for UnifiedDiff and ContextDiff structures. +func SplitLines(s string) []string { + lines := strings.SplitAfter(s, "\n") + lines[len(lines)-1] += "\n" + return lines +} diff --git a/vendor/github.com/stretchr/objx/LICENSE.md b/vendor/github.com/stretchr/objx/LICENSE.md new file mode 100644 index 000000000000..2199945813c8 --- /dev/null +++ b/vendor/github.com/stretchr/objx/LICENSE.md @@ -0,0 +1,23 @@ +objx - by Mat Ryer and Tyler Bunnell + +The MIT License (MIT) + +Copyright (c) 2014 Stretchr, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/vendor/github.com/stretchr/objx/README.md b/vendor/github.com/stretchr/objx/README.md new file mode 100644 index 000000000000..4aa180687a7a --- /dev/null +++ b/vendor/github.com/stretchr/objx/README.md @@ -0,0 +1,3 @@ +# objx + + * Jump into the [API Documentation](http://godoc.org/github.com/stretchr/objx) diff --git a/vendor/github.com/stretchr/objx/accessors.go b/vendor/github.com/stretchr/objx/accessors.go new file mode 100644 index 000000000000..721bcac79939 --- /dev/null +++ b/vendor/github.com/stretchr/objx/accessors.go @@ -0,0 +1,179 @@ +package objx + +import ( + "fmt" + "regexp" + "strconv" + "strings" +) + +// arrayAccesRegexString is the regex used to extract the array number +// from the access path +const arrayAccesRegexString = `^(.+)\[([0-9]+)\]$` + +// arrayAccesRegex is the compiled arrayAccesRegexString +var arrayAccesRegex = regexp.MustCompile(arrayAccesRegexString) + +// Get gets the value using the specified selector and +// returns it inside a new Obj object. +// +// If it cannot find the value, Get will return a nil +// value inside an instance of Obj. +// +// Get can only operate directly on map[string]interface{} and []interface. +// +// Example +// +// To access the title of the third chapter of the second book, do: +// +// o.Get("books[1].chapters[2].title") +func (m Map) Get(selector string) *Value { + rawObj := access(m, selector, nil, false, false) + return &Value{data: rawObj} +} + +// Set sets the value using the specified selector and +// returns the object on which Set was called. +// +// Set can only operate directly on map[string]interface{} and []interface +// +// Example +// +// To set the title of the third chapter of the second book, do: +// +// o.Set("books[1].chapters[2].title","Time to Go") +func (m Map) Set(selector string, value interface{}) Map { + access(m, selector, value, true, false) + return m +} + +// access accesses the object using the selector and performs the +// appropriate action. +func access(current, selector, value interface{}, isSet, panics bool) interface{} { + + switch selector.(type) { + case int, int8, int16, int32, int64, uint, uint8, uint16, uint32, uint64: + + if array, ok := current.([]interface{}); ok { + index := intFromInterface(selector) + + if index >= len(array) { + if panics { + panic(fmt.Sprintf("objx: Index %d is out of range. Slice only contains %d items.", index, len(array))) + } + return nil + } + + return array[index] + } + + return nil + + case string: + + selStr := selector.(string) + selSegs := strings.SplitN(selStr, PathSeparator, 2) + thisSel := selSegs[0] + index := -1 + var err error + + // https://github.com/stretchr/objx/issues/12 + if strings.Contains(thisSel, "[") { + + arrayMatches := arrayAccesRegex.FindStringSubmatch(thisSel) + + if len(arrayMatches) > 0 { + + // Get the key into the map + thisSel = arrayMatches[1] + + // Get the index into the array at the key + index, err = strconv.Atoi(arrayMatches[2]) + + if err != nil { + // This should never happen. If it does, something has gone + // seriously wrong. Panic. + panic("objx: Array index is not an integer. Must use array[int].") + } + + } + } + + if curMap, ok := current.(Map); ok { + current = map[string]interface{}(curMap) + } + + // get the object in question + switch current.(type) { + case map[string]interface{}: + curMSI := current.(map[string]interface{}) + if len(selSegs) <= 1 && isSet { + curMSI[thisSel] = value + return nil + } else { + current = curMSI[thisSel] + } + default: + current = nil + } + + if current == nil && panics { + panic(fmt.Sprintf("objx: '%v' invalid on object.", selector)) + } + + // do we need to access the item of an array? + if index > -1 { + if array, ok := current.([]interface{}); ok { + if index < len(array) { + current = array[index] + } else { + if panics { + panic(fmt.Sprintf("objx: Index %d is out of range. Slice only contains %d items.", index, len(array))) + } + current = nil + } + } + } + + if len(selSegs) > 1 { + current = access(current, selSegs[1], value, isSet, panics) + } + + } + + return current + +} + +// intFromInterface converts an interface object to the largest +// representation of an unsigned integer using a type switch and +// assertions +func intFromInterface(selector interface{}) int { + var value int + switch selector.(type) { + case int: + value = selector.(int) + case int8: + value = int(selector.(int8)) + case int16: + value = int(selector.(int16)) + case int32: + value = int(selector.(int32)) + case int64: + value = int(selector.(int64)) + case uint: + value = int(selector.(uint)) + case uint8: + value = int(selector.(uint8)) + case uint16: + value = int(selector.(uint16)) + case uint32: + value = int(selector.(uint32)) + case uint64: + value = int(selector.(uint64)) + default: + panic("objx: array access argument is not an integer type (this should never happen)") + } + + return value +} diff --git a/vendor/github.com/stretchr/objx/constants.go b/vendor/github.com/stretchr/objx/constants.go new file mode 100644 index 000000000000..f9eb42a25e0b --- /dev/null +++ b/vendor/github.com/stretchr/objx/constants.go @@ -0,0 +1,13 @@ +package objx + +const ( + // PathSeparator is the character used to separate the elements + // of the keypath. + // + // For example, `location.address.city` + PathSeparator string = "." + + // SignatureSeparator is the character that is used to + // separate the Base64 string from the security signature. + SignatureSeparator = "_" +) diff --git a/vendor/github.com/stretchr/objx/conversions.go b/vendor/github.com/stretchr/objx/conversions.go new file mode 100644 index 000000000000..9cdfa9f9f617 --- /dev/null +++ b/vendor/github.com/stretchr/objx/conversions.go @@ -0,0 +1,117 @@ +package objx + +import ( + "bytes" + "encoding/base64" + "encoding/json" + "errors" + "fmt" + "net/url" +) + +// JSON converts the contained object to a JSON string +// representation +func (m Map) JSON() (string, error) { + + result, err := json.Marshal(m) + + if err != nil { + err = errors.New("objx: JSON encode failed with: " + err.Error()) + } + + return string(result), err + +} + +// MustJSON converts the contained object to a JSON string +// representation and panics if there is an error +func (m Map) MustJSON() string { + result, err := m.JSON() + if err != nil { + panic(err.Error()) + } + return result +} + +// Base64 converts the contained object to a Base64 string +// representation of the JSON string representation +func (m Map) Base64() (string, error) { + + var buf bytes.Buffer + + jsonData, err := m.JSON() + if err != nil { + return "", err + } + + encoder := base64.NewEncoder(base64.StdEncoding, &buf) + encoder.Write([]byte(jsonData)) + encoder.Close() + + return buf.String(), nil + +} + +// MustBase64 converts the contained object to a Base64 string +// representation of the JSON string representation and panics +// if there is an error +func (m Map) MustBase64() string { + result, err := m.Base64() + if err != nil { + panic(err.Error()) + } + return result +} + +// SignedBase64 converts the contained object to a Base64 string +// representation of the JSON string representation and signs it +// using the provided key. +func (m Map) SignedBase64(key string) (string, error) { + + base64, err := m.Base64() + if err != nil { + return "", err + } + + sig := HashWithKey(base64, key) + + return base64 + SignatureSeparator + sig, nil + +} + +// MustSignedBase64 converts the contained object to a Base64 string +// representation of the JSON string representation and signs it +// using the provided key and panics if there is an error +func (m Map) MustSignedBase64(key string) string { + result, err := m.SignedBase64(key) + if err != nil { + panic(err.Error()) + } + return result +} + +/* + URL Query + ------------------------------------------------ +*/ + +// URLValues creates a url.Values object from an Obj. This +// function requires that the wrapped object be a map[string]interface{} +func (m Map) URLValues() url.Values { + + vals := make(url.Values) + + for k, v := range m { + //TODO: can this be done without sprintf? + vals.Set(k, fmt.Sprintf("%v", v)) + } + + return vals +} + +// URLQuery gets an encoded URL query representing the given +// Obj. This function requires that the wrapped object be a +// map[string]interface{} +func (m Map) URLQuery() (string, error) { + return m.URLValues().Encode(), nil +} diff --git a/vendor/github.com/stretchr/objx/doc.go b/vendor/github.com/stretchr/objx/doc.go new file mode 100644 index 000000000000..47bf85e46349 --- /dev/null +++ b/vendor/github.com/stretchr/objx/doc.go @@ -0,0 +1,72 @@ +// objx - Go package for dealing with maps, slices, JSON and other data. +// +// Overview +// +// Objx provides the `objx.Map` type, which is a `map[string]interface{}` that exposes +// a powerful `Get` method (among others) that allows you to easily and quickly get +// access to data within the map, without having to worry too much about type assertions, +// missing data, default values etc. +// +// Pattern +// +// Objx uses a preditable pattern to make access data from within `map[string]interface{}'s +// easy. +// +// Call one of the `objx.` functions to create your `objx.Map` to get going: +// +// m, err := objx.FromJSON(json) +// +// NOTE: Any methods or functions with the `Must` prefix will panic if something goes wrong, +// the rest will be optimistic and try to figure things out without panicking. +// +// Use `Get` to access the value you're interested in. You can use dot and array +// notation too: +// +// m.Get("places[0].latlng") +// +// Once you have saught the `Value` you're interested in, you can use the `Is*` methods +// to determine its type. +// +// if m.Get("code").IsStr() { /* ... */ } +// +// Or you can just assume the type, and use one of the strong type methods to +// extract the real value: +// +// m.Get("code").Int() +// +// If there's no value there (or if it's the wrong type) then a default value +// will be returned, or you can be explicit about the default value. +// +// Get("code").Int(-1) +// +// If you're dealing with a slice of data as a value, Objx provides many useful +// methods for iterating, manipulating and selecting that data. You can find out more +// by exploring the index below. +// +// Reading data +// +// A simple example of how to use Objx: +// +// // use MustFromJSON to make an objx.Map from some JSON +// m := objx.MustFromJSON(`{"name": "Mat", "age": 30}`) +// +// // get the details +// name := m.Get("name").Str() +// age := m.Get("age").Int() +// +// // get their nickname (or use their name if they +// // don't have one) +// nickname := m.Get("nickname").Str(name) +// +// Ranging +// +// Since `objx.Map` is a `map[string]interface{}` you can treat it as such. For +// example, to `range` the data, do what you would expect: +// +// m := objx.MustFromJSON(json) +// for key, value := range m { +// +// /* ... do your magic ... */ +// +// } +package objx diff --git a/vendor/github.com/stretchr/objx/map.go b/vendor/github.com/stretchr/objx/map.go new file mode 100644 index 000000000000..eb6ed8e285cc --- /dev/null +++ b/vendor/github.com/stretchr/objx/map.go @@ -0,0 +1,222 @@ +package objx + +import ( + "encoding/base64" + "encoding/json" + "errors" + "io/ioutil" + "net/url" + "strings" +) + +// MSIConvertable is an interface that defines methods for converting your +// custom types to a map[string]interface{} representation. +type MSIConvertable interface { + // MSI gets a map[string]interface{} (msi) representing the + // object. + MSI() map[string]interface{} +} + +// Map provides extended functionality for working with +// untyped data, in particular map[string]interface (msi). +type Map map[string]interface{} + +// Value returns the internal value instance +func (m Map) Value() *Value { + return &Value{data: m} +} + +// Nil represents a nil Map. +var Nil Map = New(nil) + +// New creates a new Map containing the map[string]interface{} in the data argument. +// If the data argument is not a map[string]interface, New attempts to call the +// MSI() method on the MSIConvertable interface to create one. +func New(data interface{}) Map { + if _, ok := data.(map[string]interface{}); !ok { + if converter, ok := data.(MSIConvertable); ok { + data = converter.MSI() + } else { + return nil + } + } + return Map(data.(map[string]interface{})) +} + +// MSI creates a map[string]interface{} and puts it inside a new Map. +// +// The arguments follow a key, value pattern. +// +// Panics +// +// Panics if any key arugment is non-string or if there are an odd number of arguments. +// +// Example +// +// To easily create Maps: +// +// m := objx.MSI("name", "Mat", "age", 29, "subobj", objx.MSI("active", true)) +// +// // creates an Map equivalent to +// m := objx.New(map[string]interface{}{"name": "Mat", "age": 29, "subobj": map[string]interface{}{"active": true}}) +func MSI(keyAndValuePairs ...interface{}) Map { + + newMap := make(map[string]interface{}) + keyAndValuePairsLen := len(keyAndValuePairs) + + if keyAndValuePairsLen%2 != 0 { + panic("objx: MSI must have an even number of arguments following the 'key, value' pattern.") + } + + for i := 0; i < keyAndValuePairsLen; i = i + 2 { + + key := keyAndValuePairs[i] + value := keyAndValuePairs[i+1] + + // make sure the key is a string + keyString, keyStringOK := key.(string) + if !keyStringOK { + panic("objx: MSI must follow 'string, interface{}' pattern. " + keyString + " is not a valid key.") + } + + newMap[keyString] = value + + } + + return New(newMap) +} + +// ****** Conversion Constructors + +// MustFromJSON creates a new Map containing the data specified in the +// jsonString. +// +// Panics if the JSON is invalid. +func MustFromJSON(jsonString string) Map { + o, err := FromJSON(jsonString) + + if err != nil { + panic("objx: MustFromJSON failed with error: " + err.Error()) + } + + return o +} + +// FromJSON creates a new Map containing the data specified in the +// jsonString. +// +// Returns an error if the JSON is invalid. +func FromJSON(jsonString string) (Map, error) { + + var data interface{} + err := json.Unmarshal([]byte(jsonString), &data) + + if err != nil { + return Nil, err + } + + return New(data), nil + +} + +// FromBase64 creates a new Obj containing the data specified +// in the Base64 string. +// +// The string is an encoded JSON string returned by Base64 +func FromBase64(base64String string) (Map, error) { + + decoder := base64.NewDecoder(base64.StdEncoding, strings.NewReader(base64String)) + + decoded, err := ioutil.ReadAll(decoder) + if err != nil { + return nil, err + } + + return FromJSON(string(decoded)) +} + +// MustFromBase64 creates a new Obj containing the data specified +// in the Base64 string and panics if there is an error. +// +// The string is an encoded JSON string returned by Base64 +func MustFromBase64(base64String string) Map { + + result, err := FromBase64(base64String) + + if err != nil { + panic("objx: MustFromBase64 failed with error: " + err.Error()) + } + + return result +} + +// FromSignedBase64 creates a new Obj containing the data specified +// in the Base64 string. +// +// The string is an encoded JSON string returned by SignedBase64 +func FromSignedBase64(base64String, key string) (Map, error) { + parts := strings.Split(base64String, SignatureSeparator) + if len(parts) != 2 { + return nil, errors.New("objx: Signed base64 string is malformed.") + } + + sig := HashWithKey(parts[0], key) + if parts[1] != sig { + return nil, errors.New("objx: Signature for base64 data does not match.") + } + + return FromBase64(parts[0]) +} + +// MustFromSignedBase64 creates a new Obj containing the data specified +// in the Base64 string and panics if there is an error. +// +// The string is an encoded JSON string returned by Base64 +func MustFromSignedBase64(base64String, key string) Map { + + result, err := FromSignedBase64(base64String, key) + + if err != nil { + panic("objx: MustFromSignedBase64 failed with error: " + err.Error()) + } + + return result +} + +// FromURLQuery generates a new Obj by parsing the specified +// query. +// +// For queries with multiple values, the first value is selected. +func FromURLQuery(query string) (Map, error) { + + vals, err := url.ParseQuery(query) + + if err != nil { + return nil, err + } + + m := make(map[string]interface{}) + for k, vals := range vals { + m[k] = vals[0] + } + + return New(m), nil +} + +// MustFromURLQuery generates a new Obj by parsing the specified +// query. +// +// For queries with multiple values, the first value is selected. +// +// Panics if it encounters an error +func MustFromURLQuery(query string) Map { + + o, err := FromURLQuery(query) + + if err != nil { + panic("objx: MustFromURLQuery failed with error: " + err.Error()) + } + + return o + +} diff --git a/vendor/github.com/stretchr/objx/mutations.go b/vendor/github.com/stretchr/objx/mutations.go new file mode 100644 index 000000000000..b35c86392bf8 --- /dev/null +++ b/vendor/github.com/stretchr/objx/mutations.go @@ -0,0 +1,81 @@ +package objx + +// Exclude returns a new Map with the keys in the specified []string +// excluded. +func (d Map) Exclude(exclude []string) Map { + + excluded := make(Map) + for k, v := range d { + var shouldInclude bool = true + for _, toExclude := range exclude { + if k == toExclude { + shouldInclude = false + break + } + } + if shouldInclude { + excluded[k] = v + } + } + + return excluded +} + +// Copy creates a shallow copy of the Obj. +func (m Map) Copy() Map { + copied := make(map[string]interface{}) + for k, v := range m { + copied[k] = v + } + return New(copied) +} + +// Merge blends the specified map with a copy of this map and returns the result. +// +// Keys that appear in both will be selected from the specified map. +// This method requires that the wrapped object be a map[string]interface{} +func (m Map) Merge(merge Map) Map { + return m.Copy().MergeHere(merge) +} + +// Merge blends the specified map with this map and returns the current map. +// +// Keys that appear in both will be selected from the specified map. The original map +// will be modified. This method requires that +// the wrapped object be a map[string]interface{} +func (m Map) MergeHere(merge Map) Map { + + for k, v := range merge { + m[k] = v + } + + return m + +} + +// Transform builds a new Obj giving the transformer a chance +// to change the keys and values as it goes. This method requires that +// the wrapped object be a map[string]interface{} +func (m Map) Transform(transformer func(key string, value interface{}) (string, interface{})) Map { + newMap := make(map[string]interface{}) + for k, v := range m { + modifiedKey, modifiedVal := transformer(k, v) + newMap[modifiedKey] = modifiedVal + } + return New(newMap) +} + +// TransformKeys builds a new map using the specified key mapping. +// +// Unspecified keys will be unaltered. +// This method requires that the wrapped object be a map[string]interface{} +func (m Map) TransformKeys(mapping map[string]string) Map { + return m.Transform(func(key string, value interface{}) (string, interface{}) { + + if newKey, ok := mapping[key]; ok { + return newKey, value + } + + return key, value + }) +} diff --git a/vendor/github.com/stretchr/objx/security.go b/vendor/github.com/stretchr/objx/security.go new file mode 100644 index 000000000000..fdd6be9cfb35 --- /dev/null +++ b/vendor/github.com/stretchr/objx/security.go @@ -0,0 +1,14 @@ +package objx + +import ( + "crypto/sha1" + "encoding/hex" +) + +// HashWithKey hashes the specified string using the security +// key. +func HashWithKey(data, key string) string { + hash := sha1.New() + hash.Write([]byte(data + ":" + key)) + return hex.EncodeToString(hash.Sum(nil)) +} diff --git a/vendor/github.com/stretchr/objx/tests.go b/vendor/github.com/stretchr/objx/tests.go new file mode 100644 index 000000000000..d9e0b479a4c0 --- /dev/null +++ b/vendor/github.com/stretchr/objx/tests.go @@ -0,0 +1,17 @@ +package objx + +// Has gets whether there is something at the specified selector +// or not. +// +// If m is nil, Has will always return false. +func (m Map) Has(selector string) bool { + if m == nil { + return false + } + return !m.Get(selector).IsNil() +} + +// IsNil gets whether the data is nil or not. +func (v *Value) IsNil() bool { + return v == nil || v.data == nil +} diff --git a/vendor/github.com/stretchr/objx/type_specific_codegen.go b/vendor/github.com/stretchr/objx/type_specific_codegen.go new file mode 100644 index 000000000000..f3ecb29b9503 --- /dev/null +++ b/vendor/github.com/stretchr/objx/type_specific_codegen.go @@ -0,0 +1,2881 @@ +package objx + +/* + Inter (interface{} and []interface{}) + -------------------------------------------------- +*/ + +// Inter gets the value as a interface{}, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Inter(optionalDefault ...interface{}) interface{} { + if s, ok := v.data.(interface{}); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustInter gets the value as a interface{}. +// +// Panics if the object is not a interface{}. +func (v *Value) MustInter() interface{} { + return v.data.(interface{}) +} + +// InterSlice gets the value as a []interface{}, returns the optionalDefault +// value or nil if the value is not a []interface{}. +func (v *Value) InterSlice(optionalDefault ...[]interface{}) []interface{} { + if s, ok := v.data.([]interface{}); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustInterSlice gets the value as a []interface{}. +// +// Panics if the object is not a []interface{}. +func (v *Value) MustInterSlice() []interface{} { + return v.data.([]interface{}) +} + +// IsInter gets whether the object contained is a interface{} or not. +func (v *Value) IsInter() bool { + _, ok := v.data.(interface{}) + return ok +} + +// IsInterSlice gets whether the object contained is a []interface{} or not. +func (v *Value) IsInterSlice() bool { + _, ok := v.data.([]interface{}) + return ok +} + +// EachInter calls the specified callback for each object +// in the []interface{}. +// +// Panics if the object is the wrong type. +func (v *Value) EachInter(callback func(int, interface{}) bool) *Value { + + for index, val := range v.MustInterSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereInter uses the specified decider function to select items +// from the []interface{}. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereInter(decider func(int, interface{}) bool) *Value { + + var selected []interface{} + + v.EachInter(func(index int, val interface{}) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupInter uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]interface{}. +func (v *Value) GroupInter(grouper func(int, interface{}) string) *Value { + + groups := make(map[string][]interface{}) + + v.EachInter(func(index int, val interface{}) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]interface{}, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceInter uses the specified function to replace each interface{}s +// by iterating each item. The data in the returned result will be a +// []interface{} containing the replaced items. +func (v *Value) ReplaceInter(replacer func(int, interface{}) interface{}) *Value { + + arr := v.MustInterSlice() + replaced := make([]interface{}, len(arr)) + + v.EachInter(func(index int, val interface{}) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectInter uses the specified collector function to collect a value +// for each of the interface{}s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectInter(collector func(int, interface{}) interface{}) *Value { + + arr := v.MustInterSlice() + collected := make([]interface{}, len(arr)) + + v.EachInter(func(index int, val interface{}) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + MSI (map[string]interface{} and []map[string]interface{}) + -------------------------------------------------- +*/ + +// MSI gets the value as a map[string]interface{}, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) MSI(optionalDefault ...map[string]interface{}) map[string]interface{} { + if s, ok := v.data.(map[string]interface{}); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustMSI gets the value as a map[string]interface{}. +// +// Panics if the object is not a map[string]interface{}. +func (v *Value) MustMSI() map[string]interface{} { + return v.data.(map[string]interface{}) +} + +// MSISlice gets the value as a []map[string]interface{}, returns the optionalDefault +// value or nil if the value is not a []map[string]interface{}. +func (v *Value) MSISlice(optionalDefault ...[]map[string]interface{}) []map[string]interface{} { + if s, ok := v.data.([]map[string]interface{}); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustMSISlice gets the value as a []map[string]interface{}. +// +// Panics if the object is not a []map[string]interface{}. +func (v *Value) MustMSISlice() []map[string]interface{} { + return v.data.([]map[string]interface{}) +} + +// IsMSI gets whether the object contained is a map[string]interface{} or not. +func (v *Value) IsMSI() bool { + _, ok := v.data.(map[string]interface{}) + return ok +} + +// IsMSISlice gets whether the object contained is a []map[string]interface{} or not. +func (v *Value) IsMSISlice() bool { + _, ok := v.data.([]map[string]interface{}) + return ok +} + +// EachMSI calls the specified callback for each object +// in the []map[string]interface{}. +// +// Panics if the object is the wrong type. +func (v *Value) EachMSI(callback func(int, map[string]interface{}) bool) *Value { + + for index, val := range v.MustMSISlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereMSI uses the specified decider function to select items +// from the []map[string]interface{}. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereMSI(decider func(int, map[string]interface{}) bool) *Value { + + var selected []map[string]interface{} + + v.EachMSI(func(index int, val map[string]interface{}) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupMSI uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]map[string]interface{}. +func (v *Value) GroupMSI(grouper func(int, map[string]interface{}) string) *Value { + + groups := make(map[string][]map[string]interface{}) + + v.EachMSI(func(index int, val map[string]interface{}) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]map[string]interface{}, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceMSI uses the specified function to replace each map[string]interface{}s +// by iterating each item. The data in the returned result will be a +// []map[string]interface{} containing the replaced items. +func (v *Value) ReplaceMSI(replacer func(int, map[string]interface{}) map[string]interface{}) *Value { + + arr := v.MustMSISlice() + replaced := make([]map[string]interface{}, len(arr)) + + v.EachMSI(func(index int, val map[string]interface{}) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectMSI uses the specified collector function to collect a value +// for each of the map[string]interface{}s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectMSI(collector func(int, map[string]interface{}) interface{}) *Value { + + arr := v.MustMSISlice() + collected := make([]interface{}, len(arr)) + + v.EachMSI(func(index int, val map[string]interface{}) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + ObjxMap ((Map) and [](Map)) + -------------------------------------------------- +*/ + +// ObjxMap gets the value as a (Map), returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) ObjxMap(optionalDefault ...(Map)) Map { + if s, ok := v.data.((Map)); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return New(nil) +} + +// MustObjxMap gets the value as a (Map). +// +// Panics if the object is not a (Map). +func (v *Value) MustObjxMap() Map { + return v.data.((Map)) +} + +// ObjxMapSlice gets the value as a [](Map), returns the optionalDefault +// value or nil if the value is not a [](Map). +func (v *Value) ObjxMapSlice(optionalDefault ...[](Map)) [](Map) { + if s, ok := v.data.([](Map)); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustObjxMapSlice gets the value as a [](Map). +// +// Panics if the object is not a [](Map). +func (v *Value) MustObjxMapSlice() [](Map) { + return v.data.([](Map)) +} + +// IsObjxMap gets whether the object contained is a (Map) or not. +func (v *Value) IsObjxMap() bool { + _, ok := v.data.((Map)) + return ok +} + +// IsObjxMapSlice gets whether the object contained is a [](Map) or not. +func (v *Value) IsObjxMapSlice() bool { + _, ok := v.data.([](Map)) + return ok +} + +// EachObjxMap calls the specified callback for each object +// in the [](Map). +// +// Panics if the object is the wrong type. +func (v *Value) EachObjxMap(callback func(int, Map) bool) *Value { + + for index, val := range v.MustObjxMapSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereObjxMap uses the specified decider function to select items +// from the [](Map). The object contained in the result will contain +// only the selected items. +func (v *Value) WhereObjxMap(decider func(int, Map) bool) *Value { + + var selected [](Map) + + v.EachObjxMap(func(index int, val Map) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupObjxMap uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][](Map). +func (v *Value) GroupObjxMap(grouper func(int, Map) string) *Value { + + groups := make(map[string][](Map)) + + v.EachObjxMap(func(index int, val Map) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([](Map), 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceObjxMap uses the specified function to replace each (Map)s +// by iterating each item. The data in the returned result will be a +// [](Map) containing the replaced items. +func (v *Value) ReplaceObjxMap(replacer func(int, Map) Map) *Value { + + arr := v.MustObjxMapSlice() + replaced := make([](Map), len(arr)) + + v.EachObjxMap(func(index int, val Map) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectObjxMap uses the specified collector function to collect a value +// for each of the (Map)s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectObjxMap(collector func(int, Map) interface{}) *Value { + + arr := v.MustObjxMapSlice() + collected := make([]interface{}, len(arr)) + + v.EachObjxMap(func(index int, val Map) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Bool (bool and []bool) + -------------------------------------------------- +*/ + +// Bool gets the value as a bool, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Bool(optionalDefault ...bool) bool { + if s, ok := v.data.(bool); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return false +} + +// MustBool gets the value as a bool. +// +// Panics if the object is not a bool. +func (v *Value) MustBool() bool { + return v.data.(bool) +} + +// BoolSlice gets the value as a []bool, returns the optionalDefault +// value or nil if the value is not a []bool. +func (v *Value) BoolSlice(optionalDefault ...[]bool) []bool { + if s, ok := v.data.([]bool); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustBoolSlice gets the value as a []bool. +// +// Panics if the object is not a []bool. +func (v *Value) MustBoolSlice() []bool { + return v.data.([]bool) +} + +// IsBool gets whether the object contained is a bool or not. +func (v *Value) IsBool() bool { + _, ok := v.data.(bool) + return ok +} + +// IsBoolSlice gets whether the object contained is a []bool or not. +func (v *Value) IsBoolSlice() bool { + _, ok := v.data.([]bool) + return ok +} + +// EachBool calls the specified callback for each object +// in the []bool. +// +// Panics if the object is the wrong type. +func (v *Value) EachBool(callback func(int, bool) bool) *Value { + + for index, val := range v.MustBoolSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereBool uses the specified decider function to select items +// from the []bool. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereBool(decider func(int, bool) bool) *Value { + + var selected []bool + + v.EachBool(func(index int, val bool) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupBool uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]bool. +func (v *Value) GroupBool(grouper func(int, bool) string) *Value { + + groups := make(map[string][]bool) + + v.EachBool(func(index int, val bool) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]bool, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceBool uses the specified function to replace each bools +// by iterating each item. The data in the returned result will be a +// []bool containing the replaced items. +func (v *Value) ReplaceBool(replacer func(int, bool) bool) *Value { + + arr := v.MustBoolSlice() + replaced := make([]bool, len(arr)) + + v.EachBool(func(index int, val bool) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectBool uses the specified collector function to collect a value +// for each of the bools in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectBool(collector func(int, bool) interface{}) *Value { + + arr := v.MustBoolSlice() + collected := make([]interface{}, len(arr)) + + v.EachBool(func(index int, val bool) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Str (string and []string) + -------------------------------------------------- +*/ + +// Str gets the value as a string, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Str(optionalDefault ...string) string { + if s, ok := v.data.(string); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return "" +} + +// MustStr gets the value as a string. +// +// Panics if the object is not a string. +func (v *Value) MustStr() string { + return v.data.(string) +} + +// StrSlice gets the value as a []string, returns the optionalDefault +// value or nil if the value is not a []string. +func (v *Value) StrSlice(optionalDefault ...[]string) []string { + if s, ok := v.data.([]string); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustStrSlice gets the value as a []string. +// +// Panics if the object is not a []string. +func (v *Value) MustStrSlice() []string { + return v.data.([]string) +} + +// IsStr gets whether the object contained is a string or not. +func (v *Value) IsStr() bool { + _, ok := v.data.(string) + return ok +} + +// IsStrSlice gets whether the object contained is a []string or not. +func (v *Value) IsStrSlice() bool { + _, ok := v.data.([]string) + return ok +} + +// EachStr calls the specified callback for each object +// in the []string. +// +// Panics if the object is the wrong type. +func (v *Value) EachStr(callback func(int, string) bool) *Value { + + for index, val := range v.MustStrSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereStr uses the specified decider function to select items +// from the []string. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereStr(decider func(int, string) bool) *Value { + + var selected []string + + v.EachStr(func(index int, val string) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupStr uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]string. +func (v *Value) GroupStr(grouper func(int, string) string) *Value { + + groups := make(map[string][]string) + + v.EachStr(func(index int, val string) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]string, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceStr uses the specified function to replace each strings +// by iterating each item. The data in the returned result will be a +// []string containing the replaced items. +func (v *Value) ReplaceStr(replacer func(int, string) string) *Value { + + arr := v.MustStrSlice() + replaced := make([]string, len(arr)) + + v.EachStr(func(index int, val string) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectStr uses the specified collector function to collect a value +// for each of the strings in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectStr(collector func(int, string) interface{}) *Value { + + arr := v.MustStrSlice() + collected := make([]interface{}, len(arr)) + + v.EachStr(func(index int, val string) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Int (int and []int) + -------------------------------------------------- +*/ + +// Int gets the value as a int, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Int(optionalDefault ...int) int { + if s, ok := v.data.(int); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustInt gets the value as a int. +// +// Panics if the object is not a int. +func (v *Value) MustInt() int { + return v.data.(int) +} + +// IntSlice gets the value as a []int, returns the optionalDefault +// value or nil if the value is not a []int. +func (v *Value) IntSlice(optionalDefault ...[]int) []int { + if s, ok := v.data.([]int); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustIntSlice gets the value as a []int. +// +// Panics if the object is not a []int. +func (v *Value) MustIntSlice() []int { + return v.data.([]int) +} + +// IsInt gets whether the object contained is a int or not. +func (v *Value) IsInt() bool { + _, ok := v.data.(int) + return ok +} + +// IsIntSlice gets whether the object contained is a []int or not. +func (v *Value) IsIntSlice() bool { + _, ok := v.data.([]int) + return ok +} + +// EachInt calls the specified callback for each object +// in the []int. +// +// Panics if the object is the wrong type. +func (v *Value) EachInt(callback func(int, int) bool) *Value { + + for index, val := range v.MustIntSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereInt uses the specified decider function to select items +// from the []int. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereInt(decider func(int, int) bool) *Value { + + var selected []int + + v.EachInt(func(index int, val int) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupInt uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]int. +func (v *Value) GroupInt(grouper func(int, int) string) *Value { + + groups := make(map[string][]int) + + v.EachInt(func(index int, val int) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]int, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceInt uses the specified function to replace each ints +// by iterating each item. The data in the returned result will be a +// []int containing the replaced items. +func (v *Value) ReplaceInt(replacer func(int, int) int) *Value { + + arr := v.MustIntSlice() + replaced := make([]int, len(arr)) + + v.EachInt(func(index int, val int) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectInt uses the specified collector function to collect a value +// for each of the ints in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectInt(collector func(int, int) interface{}) *Value { + + arr := v.MustIntSlice() + collected := make([]interface{}, len(arr)) + + v.EachInt(func(index int, val int) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Int8 (int8 and []int8) + -------------------------------------------------- +*/ + +// Int8 gets the value as a int8, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Int8(optionalDefault ...int8) int8 { + if s, ok := v.data.(int8); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustInt8 gets the value as a int8. +// +// Panics if the object is not a int8. +func (v *Value) MustInt8() int8 { + return v.data.(int8) +} + +// Int8Slice gets the value as a []int8, returns the optionalDefault +// value or nil if the value is not a []int8. +func (v *Value) Int8Slice(optionalDefault ...[]int8) []int8 { + if s, ok := v.data.([]int8); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustInt8Slice gets the value as a []int8. +// +// Panics if the object is not a []int8. +func (v *Value) MustInt8Slice() []int8 { + return v.data.([]int8) +} + +// IsInt8 gets whether the object contained is a int8 or not. +func (v *Value) IsInt8() bool { + _, ok := v.data.(int8) + return ok +} + +// IsInt8Slice gets whether the object contained is a []int8 or not. +func (v *Value) IsInt8Slice() bool { + _, ok := v.data.([]int8) + return ok +} + +// EachInt8 calls the specified callback for each object +// in the []int8. +// +// Panics if the object is the wrong type. +func (v *Value) EachInt8(callback func(int, int8) bool) *Value { + + for index, val := range v.MustInt8Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereInt8 uses the specified decider function to select items +// from the []int8. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereInt8(decider func(int, int8) bool) *Value { + + var selected []int8 + + v.EachInt8(func(index int, val int8) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupInt8 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]int8. +func (v *Value) GroupInt8(grouper func(int, int8) string) *Value { + + groups := make(map[string][]int8) + + v.EachInt8(func(index int, val int8) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]int8, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceInt8 uses the specified function to replace each int8s +// by iterating each item. The data in the returned result will be a +// []int8 containing the replaced items. +func (v *Value) ReplaceInt8(replacer func(int, int8) int8) *Value { + + arr := v.MustInt8Slice() + replaced := make([]int8, len(arr)) + + v.EachInt8(func(index int, val int8) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectInt8 uses the specified collector function to collect a value +// for each of the int8s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectInt8(collector func(int, int8) interface{}) *Value { + + arr := v.MustInt8Slice() + collected := make([]interface{}, len(arr)) + + v.EachInt8(func(index int, val int8) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Int16 (int16 and []int16) + -------------------------------------------------- +*/ + +// Int16 gets the value as a int16, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Int16(optionalDefault ...int16) int16 { + if s, ok := v.data.(int16); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustInt16 gets the value as a int16. +// +// Panics if the object is not a int16. +func (v *Value) MustInt16() int16 { + return v.data.(int16) +} + +// Int16Slice gets the value as a []int16, returns the optionalDefault +// value or nil if the value is not a []int16. +func (v *Value) Int16Slice(optionalDefault ...[]int16) []int16 { + if s, ok := v.data.([]int16); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustInt16Slice gets the value as a []int16. +// +// Panics if the object is not a []int16. +func (v *Value) MustInt16Slice() []int16 { + return v.data.([]int16) +} + +// IsInt16 gets whether the object contained is a int16 or not. +func (v *Value) IsInt16() bool { + _, ok := v.data.(int16) + return ok +} + +// IsInt16Slice gets whether the object contained is a []int16 or not. +func (v *Value) IsInt16Slice() bool { + _, ok := v.data.([]int16) + return ok +} + +// EachInt16 calls the specified callback for each object +// in the []int16. +// +// Panics if the object is the wrong type. +func (v *Value) EachInt16(callback func(int, int16) bool) *Value { + + for index, val := range v.MustInt16Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereInt16 uses the specified decider function to select items +// from the []int16. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereInt16(decider func(int, int16) bool) *Value { + + var selected []int16 + + v.EachInt16(func(index int, val int16) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupInt16 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]int16. +func (v *Value) GroupInt16(grouper func(int, int16) string) *Value { + + groups := make(map[string][]int16) + + v.EachInt16(func(index int, val int16) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]int16, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceInt16 uses the specified function to replace each int16s +// by iterating each item. The data in the returned result will be a +// []int16 containing the replaced items. +func (v *Value) ReplaceInt16(replacer func(int, int16) int16) *Value { + + arr := v.MustInt16Slice() + replaced := make([]int16, len(arr)) + + v.EachInt16(func(index int, val int16) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectInt16 uses the specified collector function to collect a value +// for each of the int16s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectInt16(collector func(int, int16) interface{}) *Value { + + arr := v.MustInt16Slice() + collected := make([]interface{}, len(arr)) + + v.EachInt16(func(index int, val int16) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Int32 (int32 and []int32) + -------------------------------------------------- +*/ + +// Int32 gets the value as a int32, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Int32(optionalDefault ...int32) int32 { + if s, ok := v.data.(int32); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustInt32 gets the value as a int32. +// +// Panics if the object is not a int32. +func (v *Value) MustInt32() int32 { + return v.data.(int32) +} + +// Int32Slice gets the value as a []int32, returns the optionalDefault +// value or nil if the value is not a []int32. +func (v *Value) Int32Slice(optionalDefault ...[]int32) []int32 { + if s, ok := v.data.([]int32); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustInt32Slice gets the value as a []int32. +// +// Panics if the object is not a []int32. +func (v *Value) MustInt32Slice() []int32 { + return v.data.([]int32) +} + +// IsInt32 gets whether the object contained is a int32 or not. +func (v *Value) IsInt32() bool { + _, ok := v.data.(int32) + return ok +} + +// IsInt32Slice gets whether the object contained is a []int32 or not. +func (v *Value) IsInt32Slice() bool { + _, ok := v.data.([]int32) + return ok +} + +// EachInt32 calls the specified callback for each object +// in the []int32. +// +// Panics if the object is the wrong type. +func (v *Value) EachInt32(callback func(int, int32) bool) *Value { + + for index, val := range v.MustInt32Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereInt32 uses the specified decider function to select items +// from the []int32. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereInt32(decider func(int, int32) bool) *Value { + + var selected []int32 + + v.EachInt32(func(index int, val int32) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupInt32 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]int32. +func (v *Value) GroupInt32(grouper func(int, int32) string) *Value { + + groups := make(map[string][]int32) + + v.EachInt32(func(index int, val int32) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]int32, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceInt32 uses the specified function to replace each int32s +// by iterating each item. The data in the returned result will be a +// []int32 containing the replaced items. +func (v *Value) ReplaceInt32(replacer func(int, int32) int32) *Value { + + arr := v.MustInt32Slice() + replaced := make([]int32, len(arr)) + + v.EachInt32(func(index int, val int32) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectInt32 uses the specified collector function to collect a value +// for each of the int32s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectInt32(collector func(int, int32) interface{}) *Value { + + arr := v.MustInt32Slice() + collected := make([]interface{}, len(arr)) + + v.EachInt32(func(index int, val int32) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Int64 (int64 and []int64) + -------------------------------------------------- +*/ + +// Int64 gets the value as a int64, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Int64(optionalDefault ...int64) int64 { + if s, ok := v.data.(int64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustInt64 gets the value as a int64. +// +// Panics if the object is not a int64. +func (v *Value) MustInt64() int64 { + return v.data.(int64) +} + +// Int64Slice gets the value as a []int64, returns the optionalDefault +// value or nil if the value is not a []int64. +func (v *Value) Int64Slice(optionalDefault ...[]int64) []int64 { + if s, ok := v.data.([]int64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustInt64Slice gets the value as a []int64. +// +// Panics if the object is not a []int64. +func (v *Value) MustInt64Slice() []int64 { + return v.data.([]int64) +} + +// IsInt64 gets whether the object contained is a int64 or not. +func (v *Value) IsInt64() bool { + _, ok := v.data.(int64) + return ok +} + +// IsInt64Slice gets whether the object contained is a []int64 or not. +func (v *Value) IsInt64Slice() bool { + _, ok := v.data.([]int64) + return ok +} + +// EachInt64 calls the specified callback for each object +// in the []int64. +// +// Panics if the object is the wrong type. +func (v *Value) EachInt64(callback func(int, int64) bool) *Value { + + for index, val := range v.MustInt64Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereInt64 uses the specified decider function to select items +// from the []int64. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereInt64(decider func(int, int64) bool) *Value { + + var selected []int64 + + v.EachInt64(func(index int, val int64) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupInt64 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]int64. +func (v *Value) GroupInt64(grouper func(int, int64) string) *Value { + + groups := make(map[string][]int64) + + v.EachInt64(func(index int, val int64) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]int64, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceInt64 uses the specified function to replace each int64s +// by iterating each item. The data in the returned result will be a +// []int64 containing the replaced items. +func (v *Value) ReplaceInt64(replacer func(int, int64) int64) *Value { + + arr := v.MustInt64Slice() + replaced := make([]int64, len(arr)) + + v.EachInt64(func(index int, val int64) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectInt64 uses the specified collector function to collect a value +// for each of the int64s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectInt64(collector func(int, int64) interface{}) *Value { + + arr := v.MustInt64Slice() + collected := make([]interface{}, len(arr)) + + v.EachInt64(func(index int, val int64) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Uint (uint and []uint) + -------------------------------------------------- +*/ + +// Uint gets the value as a uint, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Uint(optionalDefault ...uint) uint { + if s, ok := v.data.(uint); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustUint gets the value as a uint. +// +// Panics if the object is not a uint. +func (v *Value) MustUint() uint { + return v.data.(uint) +} + +// UintSlice gets the value as a []uint, returns the optionalDefault +// value or nil if the value is not a []uint. +func (v *Value) UintSlice(optionalDefault ...[]uint) []uint { + if s, ok := v.data.([]uint); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustUintSlice gets the value as a []uint. +// +// Panics if the object is not a []uint. +func (v *Value) MustUintSlice() []uint { + return v.data.([]uint) +} + +// IsUint gets whether the object contained is a uint or not. +func (v *Value) IsUint() bool { + _, ok := v.data.(uint) + return ok +} + +// IsUintSlice gets whether the object contained is a []uint or not. +func (v *Value) IsUintSlice() bool { + _, ok := v.data.([]uint) + return ok +} + +// EachUint calls the specified callback for each object +// in the []uint. +// +// Panics if the object is the wrong type. +func (v *Value) EachUint(callback func(int, uint) bool) *Value { + + for index, val := range v.MustUintSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereUint uses the specified decider function to select items +// from the []uint. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereUint(decider func(int, uint) bool) *Value { + + var selected []uint + + v.EachUint(func(index int, val uint) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupUint uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]uint. +func (v *Value) GroupUint(grouper func(int, uint) string) *Value { + + groups := make(map[string][]uint) + + v.EachUint(func(index int, val uint) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]uint, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceUint uses the specified function to replace each uints +// by iterating each item. The data in the returned result will be a +// []uint containing the replaced items. +func (v *Value) ReplaceUint(replacer func(int, uint) uint) *Value { + + arr := v.MustUintSlice() + replaced := make([]uint, len(arr)) + + v.EachUint(func(index int, val uint) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectUint uses the specified collector function to collect a value +// for each of the uints in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectUint(collector func(int, uint) interface{}) *Value { + + arr := v.MustUintSlice() + collected := make([]interface{}, len(arr)) + + v.EachUint(func(index int, val uint) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Uint8 (uint8 and []uint8) + -------------------------------------------------- +*/ + +// Uint8 gets the value as a uint8, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Uint8(optionalDefault ...uint8) uint8 { + if s, ok := v.data.(uint8); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustUint8 gets the value as a uint8. +// +// Panics if the object is not a uint8. +func (v *Value) MustUint8() uint8 { + return v.data.(uint8) +} + +// Uint8Slice gets the value as a []uint8, returns the optionalDefault +// value or nil if the value is not a []uint8. +func (v *Value) Uint8Slice(optionalDefault ...[]uint8) []uint8 { + if s, ok := v.data.([]uint8); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustUint8Slice gets the value as a []uint8. +// +// Panics if the object is not a []uint8. +func (v *Value) MustUint8Slice() []uint8 { + return v.data.([]uint8) +} + +// IsUint8 gets whether the object contained is a uint8 or not. +func (v *Value) IsUint8() bool { + _, ok := v.data.(uint8) + return ok +} + +// IsUint8Slice gets whether the object contained is a []uint8 or not. +func (v *Value) IsUint8Slice() bool { + _, ok := v.data.([]uint8) + return ok +} + +// EachUint8 calls the specified callback for each object +// in the []uint8. +// +// Panics if the object is the wrong type. +func (v *Value) EachUint8(callback func(int, uint8) bool) *Value { + + for index, val := range v.MustUint8Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereUint8 uses the specified decider function to select items +// from the []uint8. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereUint8(decider func(int, uint8) bool) *Value { + + var selected []uint8 + + v.EachUint8(func(index int, val uint8) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupUint8 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]uint8. +func (v *Value) GroupUint8(grouper func(int, uint8) string) *Value { + + groups := make(map[string][]uint8) + + v.EachUint8(func(index int, val uint8) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]uint8, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceUint8 uses the specified function to replace each uint8s +// by iterating each item. The data in the returned result will be a +// []uint8 containing the replaced items. +func (v *Value) ReplaceUint8(replacer func(int, uint8) uint8) *Value { + + arr := v.MustUint8Slice() + replaced := make([]uint8, len(arr)) + + v.EachUint8(func(index int, val uint8) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectUint8 uses the specified collector function to collect a value +// for each of the uint8s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectUint8(collector func(int, uint8) interface{}) *Value { + + arr := v.MustUint8Slice() + collected := make([]interface{}, len(arr)) + + v.EachUint8(func(index int, val uint8) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Uint16 (uint16 and []uint16) + -------------------------------------------------- +*/ + +// Uint16 gets the value as a uint16, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Uint16(optionalDefault ...uint16) uint16 { + if s, ok := v.data.(uint16); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustUint16 gets the value as a uint16. +// +// Panics if the object is not a uint16. +func (v *Value) MustUint16() uint16 { + return v.data.(uint16) +} + +// Uint16Slice gets the value as a []uint16, returns the optionalDefault +// value or nil if the value is not a []uint16. +func (v *Value) Uint16Slice(optionalDefault ...[]uint16) []uint16 { + if s, ok := v.data.([]uint16); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustUint16Slice gets the value as a []uint16. +// +// Panics if the object is not a []uint16. +func (v *Value) MustUint16Slice() []uint16 { + return v.data.([]uint16) +} + +// IsUint16 gets whether the object contained is a uint16 or not. +func (v *Value) IsUint16() bool { + _, ok := v.data.(uint16) + return ok +} + +// IsUint16Slice gets whether the object contained is a []uint16 or not. +func (v *Value) IsUint16Slice() bool { + _, ok := v.data.([]uint16) + return ok +} + +// EachUint16 calls the specified callback for each object +// in the []uint16. +// +// Panics if the object is the wrong type. +func (v *Value) EachUint16(callback func(int, uint16) bool) *Value { + + for index, val := range v.MustUint16Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereUint16 uses the specified decider function to select items +// from the []uint16. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereUint16(decider func(int, uint16) bool) *Value { + + var selected []uint16 + + v.EachUint16(func(index int, val uint16) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupUint16 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]uint16. +func (v *Value) GroupUint16(grouper func(int, uint16) string) *Value { + + groups := make(map[string][]uint16) + + v.EachUint16(func(index int, val uint16) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]uint16, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceUint16 uses the specified function to replace each uint16s +// by iterating each item. The data in the returned result will be a +// []uint16 containing the replaced items. +func (v *Value) ReplaceUint16(replacer func(int, uint16) uint16) *Value { + + arr := v.MustUint16Slice() + replaced := make([]uint16, len(arr)) + + v.EachUint16(func(index int, val uint16) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectUint16 uses the specified collector function to collect a value +// for each of the uint16s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectUint16(collector func(int, uint16) interface{}) *Value { + + arr := v.MustUint16Slice() + collected := make([]interface{}, len(arr)) + + v.EachUint16(func(index int, val uint16) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Uint32 (uint32 and []uint32) + -------------------------------------------------- +*/ + +// Uint32 gets the value as a uint32, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Uint32(optionalDefault ...uint32) uint32 { + if s, ok := v.data.(uint32); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustUint32 gets the value as a uint32. +// +// Panics if the object is not a uint32. +func (v *Value) MustUint32() uint32 { + return v.data.(uint32) +} + +// Uint32Slice gets the value as a []uint32, returns the optionalDefault +// value or nil if the value is not a []uint32. +func (v *Value) Uint32Slice(optionalDefault ...[]uint32) []uint32 { + if s, ok := v.data.([]uint32); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustUint32Slice gets the value as a []uint32. +// +// Panics if the object is not a []uint32. +func (v *Value) MustUint32Slice() []uint32 { + return v.data.([]uint32) +} + +// IsUint32 gets whether the object contained is a uint32 or not. +func (v *Value) IsUint32() bool { + _, ok := v.data.(uint32) + return ok +} + +// IsUint32Slice gets whether the object contained is a []uint32 or not. +func (v *Value) IsUint32Slice() bool { + _, ok := v.data.([]uint32) + return ok +} + +// EachUint32 calls the specified callback for each object +// in the []uint32. +// +// Panics if the object is the wrong type. +func (v *Value) EachUint32(callback func(int, uint32) bool) *Value { + + for index, val := range v.MustUint32Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereUint32 uses the specified decider function to select items +// from the []uint32. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereUint32(decider func(int, uint32) bool) *Value { + + var selected []uint32 + + v.EachUint32(func(index int, val uint32) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupUint32 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]uint32. +func (v *Value) GroupUint32(grouper func(int, uint32) string) *Value { + + groups := make(map[string][]uint32) + + v.EachUint32(func(index int, val uint32) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]uint32, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceUint32 uses the specified function to replace each uint32s +// by iterating each item. The data in the returned result will be a +// []uint32 containing the replaced items. +func (v *Value) ReplaceUint32(replacer func(int, uint32) uint32) *Value { + + arr := v.MustUint32Slice() + replaced := make([]uint32, len(arr)) + + v.EachUint32(func(index int, val uint32) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectUint32 uses the specified collector function to collect a value +// for each of the uint32s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectUint32(collector func(int, uint32) interface{}) *Value { + + arr := v.MustUint32Slice() + collected := make([]interface{}, len(arr)) + + v.EachUint32(func(index int, val uint32) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Uint64 (uint64 and []uint64) + -------------------------------------------------- +*/ + +// Uint64 gets the value as a uint64, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Uint64(optionalDefault ...uint64) uint64 { + if s, ok := v.data.(uint64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustUint64 gets the value as a uint64. +// +// Panics if the object is not a uint64. +func (v *Value) MustUint64() uint64 { + return v.data.(uint64) +} + +// Uint64Slice gets the value as a []uint64, returns the optionalDefault +// value or nil if the value is not a []uint64. +func (v *Value) Uint64Slice(optionalDefault ...[]uint64) []uint64 { + if s, ok := v.data.([]uint64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustUint64Slice gets the value as a []uint64. +// +// Panics if the object is not a []uint64. +func (v *Value) MustUint64Slice() []uint64 { + return v.data.([]uint64) +} + +// IsUint64 gets whether the object contained is a uint64 or not. +func (v *Value) IsUint64() bool { + _, ok := v.data.(uint64) + return ok +} + +// IsUint64Slice gets whether the object contained is a []uint64 or not. +func (v *Value) IsUint64Slice() bool { + _, ok := v.data.([]uint64) + return ok +} + +// EachUint64 calls the specified callback for each object +// in the []uint64. +// +// Panics if the object is the wrong type. +func (v *Value) EachUint64(callback func(int, uint64) bool) *Value { + + for index, val := range v.MustUint64Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereUint64 uses the specified decider function to select items +// from the []uint64. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereUint64(decider func(int, uint64) bool) *Value { + + var selected []uint64 + + v.EachUint64(func(index int, val uint64) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupUint64 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]uint64. +func (v *Value) GroupUint64(grouper func(int, uint64) string) *Value { + + groups := make(map[string][]uint64) + + v.EachUint64(func(index int, val uint64) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]uint64, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceUint64 uses the specified function to replace each uint64s +// by iterating each item. The data in the returned result will be a +// []uint64 containing the replaced items. +func (v *Value) ReplaceUint64(replacer func(int, uint64) uint64) *Value { + + arr := v.MustUint64Slice() + replaced := make([]uint64, len(arr)) + + v.EachUint64(func(index int, val uint64) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectUint64 uses the specified collector function to collect a value +// for each of the uint64s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectUint64(collector func(int, uint64) interface{}) *Value { + + arr := v.MustUint64Slice() + collected := make([]interface{}, len(arr)) + + v.EachUint64(func(index int, val uint64) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Uintptr (uintptr and []uintptr) + -------------------------------------------------- +*/ + +// Uintptr gets the value as a uintptr, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Uintptr(optionalDefault ...uintptr) uintptr { + if s, ok := v.data.(uintptr); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustUintptr gets the value as a uintptr. +// +// Panics if the object is not a uintptr. +func (v *Value) MustUintptr() uintptr { + return v.data.(uintptr) +} + +// UintptrSlice gets the value as a []uintptr, returns the optionalDefault +// value or nil if the value is not a []uintptr. +func (v *Value) UintptrSlice(optionalDefault ...[]uintptr) []uintptr { + if s, ok := v.data.([]uintptr); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustUintptrSlice gets the value as a []uintptr. +// +// Panics if the object is not a []uintptr. +func (v *Value) MustUintptrSlice() []uintptr { + return v.data.([]uintptr) +} + +// IsUintptr gets whether the object contained is a uintptr or not. +func (v *Value) IsUintptr() bool { + _, ok := v.data.(uintptr) + return ok +} + +// IsUintptrSlice gets whether the object contained is a []uintptr or not. +func (v *Value) IsUintptrSlice() bool { + _, ok := v.data.([]uintptr) + return ok +} + +// EachUintptr calls the specified callback for each object +// in the []uintptr. +// +// Panics if the object is the wrong type. +func (v *Value) EachUintptr(callback func(int, uintptr) bool) *Value { + + for index, val := range v.MustUintptrSlice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereUintptr uses the specified decider function to select items +// from the []uintptr. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereUintptr(decider func(int, uintptr) bool) *Value { + + var selected []uintptr + + v.EachUintptr(func(index int, val uintptr) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupUintptr uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]uintptr. +func (v *Value) GroupUintptr(grouper func(int, uintptr) string) *Value { + + groups := make(map[string][]uintptr) + + v.EachUintptr(func(index int, val uintptr) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]uintptr, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceUintptr uses the specified function to replace each uintptrs +// by iterating each item. The data in the returned result will be a +// []uintptr containing the replaced items. +func (v *Value) ReplaceUintptr(replacer func(int, uintptr) uintptr) *Value { + + arr := v.MustUintptrSlice() + replaced := make([]uintptr, len(arr)) + + v.EachUintptr(func(index int, val uintptr) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectUintptr uses the specified collector function to collect a value +// for each of the uintptrs in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectUintptr(collector func(int, uintptr) interface{}) *Value { + + arr := v.MustUintptrSlice() + collected := make([]interface{}, len(arr)) + + v.EachUintptr(func(index int, val uintptr) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Float32 (float32 and []float32) + -------------------------------------------------- +*/ + +// Float32 gets the value as a float32, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Float32(optionalDefault ...float32) float32 { + if s, ok := v.data.(float32); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustFloat32 gets the value as a float32. +// +// Panics if the object is not a float32. +func (v *Value) MustFloat32() float32 { + return v.data.(float32) +} + +// Float32Slice gets the value as a []float32, returns the optionalDefault +// value or nil if the value is not a []float32. +func (v *Value) Float32Slice(optionalDefault ...[]float32) []float32 { + if s, ok := v.data.([]float32); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustFloat32Slice gets the value as a []float32. +// +// Panics if the object is not a []float32. +func (v *Value) MustFloat32Slice() []float32 { + return v.data.([]float32) +} + +// IsFloat32 gets whether the object contained is a float32 or not. +func (v *Value) IsFloat32() bool { + _, ok := v.data.(float32) + return ok +} + +// IsFloat32Slice gets whether the object contained is a []float32 or not. +func (v *Value) IsFloat32Slice() bool { + _, ok := v.data.([]float32) + return ok +} + +// EachFloat32 calls the specified callback for each object +// in the []float32. +// +// Panics if the object is the wrong type. +func (v *Value) EachFloat32(callback func(int, float32) bool) *Value { + + for index, val := range v.MustFloat32Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereFloat32 uses the specified decider function to select items +// from the []float32. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereFloat32(decider func(int, float32) bool) *Value { + + var selected []float32 + + v.EachFloat32(func(index int, val float32) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupFloat32 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]float32. +func (v *Value) GroupFloat32(grouper func(int, float32) string) *Value { + + groups := make(map[string][]float32) + + v.EachFloat32(func(index int, val float32) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]float32, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceFloat32 uses the specified function to replace each float32s +// by iterating each item. The data in the returned result will be a +// []float32 containing the replaced items. +func (v *Value) ReplaceFloat32(replacer func(int, float32) float32) *Value { + + arr := v.MustFloat32Slice() + replaced := make([]float32, len(arr)) + + v.EachFloat32(func(index int, val float32) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectFloat32 uses the specified collector function to collect a value +// for each of the float32s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectFloat32(collector func(int, float32) interface{}) *Value { + + arr := v.MustFloat32Slice() + collected := make([]interface{}, len(arr)) + + v.EachFloat32(func(index int, val float32) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Float64 (float64 and []float64) + -------------------------------------------------- +*/ + +// Float64 gets the value as a float64, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Float64(optionalDefault ...float64) float64 { + if s, ok := v.data.(float64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustFloat64 gets the value as a float64. +// +// Panics if the object is not a float64. +func (v *Value) MustFloat64() float64 { + return v.data.(float64) +} + +// Float64Slice gets the value as a []float64, returns the optionalDefault +// value or nil if the value is not a []float64. +func (v *Value) Float64Slice(optionalDefault ...[]float64) []float64 { + if s, ok := v.data.([]float64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustFloat64Slice gets the value as a []float64. +// +// Panics if the object is not a []float64. +func (v *Value) MustFloat64Slice() []float64 { + return v.data.([]float64) +} + +// IsFloat64 gets whether the object contained is a float64 or not. +func (v *Value) IsFloat64() bool { + _, ok := v.data.(float64) + return ok +} + +// IsFloat64Slice gets whether the object contained is a []float64 or not. +func (v *Value) IsFloat64Slice() bool { + _, ok := v.data.([]float64) + return ok +} + +// EachFloat64 calls the specified callback for each object +// in the []float64. +// +// Panics if the object is the wrong type. +func (v *Value) EachFloat64(callback func(int, float64) bool) *Value { + + for index, val := range v.MustFloat64Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereFloat64 uses the specified decider function to select items +// from the []float64. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereFloat64(decider func(int, float64) bool) *Value { + + var selected []float64 + + v.EachFloat64(func(index int, val float64) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupFloat64 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]float64. +func (v *Value) GroupFloat64(grouper func(int, float64) string) *Value { + + groups := make(map[string][]float64) + + v.EachFloat64(func(index int, val float64) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]float64, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceFloat64 uses the specified function to replace each float64s +// by iterating each item. The data in the returned result will be a +// []float64 containing the replaced items. +func (v *Value) ReplaceFloat64(replacer func(int, float64) float64) *Value { + + arr := v.MustFloat64Slice() + replaced := make([]float64, len(arr)) + + v.EachFloat64(func(index int, val float64) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectFloat64 uses the specified collector function to collect a value +// for each of the float64s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectFloat64(collector func(int, float64) interface{}) *Value { + + arr := v.MustFloat64Slice() + collected := make([]interface{}, len(arr)) + + v.EachFloat64(func(index int, val float64) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Complex64 (complex64 and []complex64) + -------------------------------------------------- +*/ + +// Complex64 gets the value as a complex64, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Complex64(optionalDefault ...complex64) complex64 { + if s, ok := v.data.(complex64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustComplex64 gets the value as a complex64. +// +// Panics if the object is not a complex64. +func (v *Value) MustComplex64() complex64 { + return v.data.(complex64) +} + +// Complex64Slice gets the value as a []complex64, returns the optionalDefault +// value or nil if the value is not a []complex64. +func (v *Value) Complex64Slice(optionalDefault ...[]complex64) []complex64 { + if s, ok := v.data.([]complex64); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustComplex64Slice gets the value as a []complex64. +// +// Panics if the object is not a []complex64. +func (v *Value) MustComplex64Slice() []complex64 { + return v.data.([]complex64) +} + +// IsComplex64 gets whether the object contained is a complex64 or not. +func (v *Value) IsComplex64() bool { + _, ok := v.data.(complex64) + return ok +} + +// IsComplex64Slice gets whether the object contained is a []complex64 or not. +func (v *Value) IsComplex64Slice() bool { + _, ok := v.data.([]complex64) + return ok +} + +// EachComplex64 calls the specified callback for each object +// in the []complex64. +// +// Panics if the object is the wrong type. +func (v *Value) EachComplex64(callback func(int, complex64) bool) *Value { + + for index, val := range v.MustComplex64Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereComplex64 uses the specified decider function to select items +// from the []complex64. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereComplex64(decider func(int, complex64) bool) *Value { + + var selected []complex64 + + v.EachComplex64(func(index int, val complex64) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupComplex64 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]complex64. +func (v *Value) GroupComplex64(grouper func(int, complex64) string) *Value { + + groups := make(map[string][]complex64) + + v.EachComplex64(func(index int, val complex64) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]complex64, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceComplex64 uses the specified function to replace each complex64s +// by iterating each item. The data in the returned result will be a +// []complex64 containing the replaced items. +func (v *Value) ReplaceComplex64(replacer func(int, complex64) complex64) *Value { + + arr := v.MustComplex64Slice() + replaced := make([]complex64, len(arr)) + + v.EachComplex64(func(index int, val complex64) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectComplex64 uses the specified collector function to collect a value +// for each of the complex64s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectComplex64(collector func(int, complex64) interface{}) *Value { + + arr := v.MustComplex64Slice() + collected := make([]interface{}, len(arr)) + + v.EachComplex64(func(index int, val complex64) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} + +/* + Complex128 (complex128 and []complex128) + -------------------------------------------------- +*/ + +// Complex128 gets the value as a complex128, returns the optionalDefault +// value or a system default object if the value is the wrong type. +func (v *Value) Complex128(optionalDefault ...complex128) complex128 { + if s, ok := v.data.(complex128); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return 0 +} + +// MustComplex128 gets the value as a complex128. +// +// Panics if the object is not a complex128. +func (v *Value) MustComplex128() complex128 { + return v.data.(complex128) +} + +// Complex128Slice gets the value as a []complex128, returns the optionalDefault +// value or nil if the value is not a []complex128. +func (v *Value) Complex128Slice(optionalDefault ...[]complex128) []complex128 { + if s, ok := v.data.([]complex128); ok { + return s + } + if len(optionalDefault) == 1 { + return optionalDefault[0] + } + return nil +} + +// MustComplex128Slice gets the value as a []complex128. +// +// Panics if the object is not a []complex128. +func (v *Value) MustComplex128Slice() []complex128 { + return v.data.([]complex128) +} + +// IsComplex128 gets whether the object contained is a complex128 or not. +func (v *Value) IsComplex128() bool { + _, ok := v.data.(complex128) + return ok +} + +// IsComplex128Slice gets whether the object contained is a []complex128 or not. +func (v *Value) IsComplex128Slice() bool { + _, ok := v.data.([]complex128) + return ok +} + +// EachComplex128 calls the specified callback for each object +// in the []complex128. +// +// Panics if the object is the wrong type. +func (v *Value) EachComplex128(callback func(int, complex128) bool) *Value { + + for index, val := range v.MustComplex128Slice() { + carryon := callback(index, val) + if carryon == false { + break + } + } + + return v + +} + +// WhereComplex128 uses the specified decider function to select items +// from the []complex128. The object contained in the result will contain +// only the selected items. +func (v *Value) WhereComplex128(decider func(int, complex128) bool) *Value { + + var selected []complex128 + + v.EachComplex128(func(index int, val complex128) bool { + shouldSelect := decider(index, val) + if shouldSelect == false { + selected = append(selected, val) + } + return true + }) + + return &Value{data: selected} + +} + +// GroupComplex128 uses the specified grouper function to group the items +// keyed by the return of the grouper. The object contained in the +// result will contain a map[string][]complex128. +func (v *Value) GroupComplex128(grouper func(int, complex128) string) *Value { + + groups := make(map[string][]complex128) + + v.EachComplex128(func(index int, val complex128) bool { + group := grouper(index, val) + if _, ok := groups[group]; !ok { + groups[group] = make([]complex128, 0) + } + groups[group] = append(groups[group], val) + return true + }) + + return &Value{data: groups} + +} + +// ReplaceComplex128 uses the specified function to replace each complex128s +// by iterating each item. The data in the returned result will be a +// []complex128 containing the replaced items. +func (v *Value) ReplaceComplex128(replacer func(int, complex128) complex128) *Value { + + arr := v.MustComplex128Slice() + replaced := make([]complex128, len(arr)) + + v.EachComplex128(func(index int, val complex128) bool { + replaced[index] = replacer(index, val) + return true + }) + + return &Value{data: replaced} + +} + +// CollectComplex128 uses the specified collector function to collect a value +// for each of the complex128s in the slice. The data returned will be a +// []interface{}. +func (v *Value) CollectComplex128(collector func(int, complex128) interface{}) *Value { + + arr := v.MustComplex128Slice() + collected := make([]interface{}, len(arr)) + + v.EachComplex128(func(index int, val complex128) bool { + collected[index] = collector(index, val) + return true + }) + + return &Value{data: collected} +} diff --git a/vendor/github.com/stretchr/objx/value.go b/vendor/github.com/stretchr/objx/value.go new file mode 100644 index 000000000000..956a2211d4d5 --- /dev/null +++ b/vendor/github.com/stretchr/objx/value.go @@ -0,0 +1,56 @@ +package objx + +import ( + "fmt" + "strconv" +) + +// Value provides methods for extracting interface{} data in various +// types. +type Value struct { + // data contains the raw data being managed by this Value + data interface{} +} + +// Data returns the raw data contained by this Value +func (v *Value) Data() interface{} { + return v.data +} + +// String returns the value always as a string +func (v *Value) String() string { + switch { + case v.IsStr(): + return v.Str() + case v.IsBool(): + return strconv.FormatBool(v.Bool()) + case v.IsFloat32(): + return strconv.FormatFloat(float64(v.Float32()), 'f', -1, 32) + case v.IsFloat64(): + return strconv.FormatFloat(v.Float64(), 'f', -1, 64) + case v.IsInt(): + return strconv.FormatInt(int64(v.Int()), 10) + case v.IsInt(): + return strconv.FormatInt(int64(v.Int()), 10) + case v.IsInt8(): + return strconv.FormatInt(int64(v.Int8()), 10) + case v.IsInt16(): + return strconv.FormatInt(int64(v.Int16()), 10) + case v.IsInt32(): + return strconv.FormatInt(int64(v.Int32()), 10) + case v.IsInt64(): + return strconv.FormatInt(v.Int64(), 10) + case v.IsUint(): + return strconv.FormatUint(uint64(v.Uint()), 10) + case v.IsUint8(): + return strconv.FormatUint(uint64(v.Uint8()), 10) + case v.IsUint16(): + return strconv.FormatUint(uint64(v.Uint16()), 10) + case v.IsUint32(): + return strconv.FormatUint(uint64(v.Uint32()), 10) + case v.IsUint64(): + return strconv.FormatUint(v.Uint64(), 10) + } + + return fmt.Sprintf("%#v", v.Data()) +} diff --git a/vendor/github.com/stretchr/testify/LICENSE b/vendor/github.com/stretchr/testify/LICENSE new file mode 100644 index 000000000000..473b670a7c61 --- /dev/null +++ b/vendor/github.com/stretchr/testify/LICENSE @@ -0,0 +1,22 @@ +Copyright (c) 2012 - 2013 Mat Ryer and Tyler Bunnell + +Please consider promoting this project if you find it useful. + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without restriction, +including without limitation the rights to use, copy, modify, merge, +publish, distribute, sublicense, and/or sell copies of the Software, +and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice shall be included +in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT +OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE +OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/stretchr/testify/assert/assertion_format.go b/vendor/github.com/stretchr/testify/assert/assertion_format.go new file mode 100644 index 000000000000..23838c4ceea7 --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/assertion_format.go @@ -0,0 +1,379 @@ +/* +* CODE GENERATED AUTOMATICALLY WITH github.com/stretchr/testify/_codegen +* THIS FILE MUST NOT BE EDITED BY HAND + */ + +package assert + +import ( + http "net/http" + url "net/url" + time "time" +) + +// Conditionf uses a Comparison to assert a complex condition. +func Conditionf(t TestingT, comp Comparison, msg string, args ...interface{}) bool { + return Condition(t, comp, append([]interface{}{msg}, args...)...) +} + +// Containsf asserts that the specified string, list(array, slice...) or map contains the +// specified substring or element. +// +// assert.Containsf(t, "Hello World", "World", "error message %s", "formatted") +// assert.Containsf(t, ["Hello", "World"], "World", "error message %s", "formatted") +// assert.Containsf(t, {"Hello": "World"}, "Hello", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Containsf(t TestingT, s interface{}, contains interface{}, msg string, args ...interface{}) bool { + return Contains(t, s, contains, append([]interface{}{msg}, args...)...) +} + +// Emptyf asserts that the specified object is empty. I.e. nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// assert.Emptyf(t, obj, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Emptyf(t TestingT, object interface{}, msg string, args ...interface{}) bool { + return Empty(t, object, append([]interface{}{msg}, args...)...) +} + +// Equalf asserts that two objects are equal. +// +// assert.Equalf(t, 123, 123, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). Function equality +// cannot be determined and will always fail. +func Equalf(t TestingT, expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return Equal(t, expected, actual, append([]interface{}{msg}, args...)...) +} + +// EqualErrorf asserts that a function returned an error (i.e. not `nil`) +// and that it is equal to the provided error. +// +// actualObj, err := SomeFunction() +// assert.EqualErrorf(t, err, expectedErrorString, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func EqualErrorf(t TestingT, theError error, errString string, msg string, args ...interface{}) bool { + return EqualError(t, theError, errString, append([]interface{}{msg}, args...)...) +} + +// EqualValuesf asserts that two objects are equal or convertable to the same types +// and equal. +// +// assert.EqualValuesf(t, uint32(123, "error message %s", "formatted"), int32(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func EqualValuesf(t TestingT, expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return EqualValues(t, expected, actual, append([]interface{}{msg}, args...)...) +} + +// Errorf asserts that a function returned an error (i.e. not `nil`). +// +// actualObj, err := SomeFunction() +// if assert.Errorf(t, err, "error message %s", "formatted") { +// assert.Equal(t, expectedErrorf, err) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func Errorf(t TestingT, err error, msg string, args ...interface{}) bool { + return Error(t, err, append([]interface{}{msg}, args...)...) +} + +// Exactlyf asserts that two objects are equal is value and type. +// +// assert.Exactlyf(t, int32(123, "error message %s", "formatted"), int64(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func Exactlyf(t TestingT, expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return Exactly(t, expected, actual, append([]interface{}{msg}, args...)...) +} + +// Failf reports a failure through +func Failf(t TestingT, failureMessage string, msg string, args ...interface{}) bool { + return Fail(t, failureMessage, append([]interface{}{msg}, args...)...) +} + +// FailNowf fails test +func FailNowf(t TestingT, failureMessage string, msg string, args ...interface{}) bool { + return FailNow(t, failureMessage, append([]interface{}{msg}, args...)...) +} + +// Falsef asserts that the specified value is false. +// +// assert.Falsef(t, myBool, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Falsef(t TestingT, value bool, msg string, args ...interface{}) bool { + return False(t, value, append([]interface{}{msg}, args...)...) +} + +// HTTPBodyContainsf asserts that a specified handler returns a +// body that contains a string. +// +// assert.HTTPBodyContainsf(t, myHandler, "www.google.com", nil, "I'm Feeling Lucky", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPBodyContainsf(t TestingT, handler http.HandlerFunc, method string, url string, values url.Values, str interface{}) bool { + return HTTPBodyContains(t, handler, method, url, values, str) +} + +// HTTPBodyNotContainsf asserts that a specified handler returns a +// body that does not contain a string. +// +// assert.HTTPBodyNotContainsf(t, myHandler, "www.google.com", nil, "I'm Feeling Lucky", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPBodyNotContainsf(t TestingT, handler http.HandlerFunc, method string, url string, values url.Values, str interface{}) bool { + return HTTPBodyNotContains(t, handler, method, url, values, str) +} + +// HTTPErrorf asserts that a specified handler returns an error status code. +// +// assert.HTTPErrorf(t, myHandler, "POST", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true, "error message %s", "formatted") or not (false). +func HTTPErrorf(t TestingT, handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPError(t, handler, method, url, values) +} + +// HTTPRedirectf asserts that a specified handler returns a redirect status code. +// +// assert.HTTPRedirectf(t, myHandler, "GET", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true, "error message %s", "formatted") or not (false). +func HTTPRedirectf(t TestingT, handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPRedirect(t, handler, method, url, values) +} + +// HTTPSuccessf asserts that a specified handler returns a success status code. +// +// assert.HTTPSuccessf(t, myHandler, "POST", "http://www.google.com", nil, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPSuccessf(t TestingT, handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPSuccess(t, handler, method, url, values) +} + +// Implementsf asserts that an object is implemented by the specified interface. +// +// assert.Implementsf(t, (*MyInterface, "error message %s", "formatted")(nil), new(MyObject)) +func Implementsf(t TestingT, interfaceObject interface{}, object interface{}, msg string, args ...interface{}) bool { + return Implements(t, interfaceObject, object, append([]interface{}{msg}, args...)...) +} + +// InDeltaf asserts that the two numerals are within delta of each other. +// +// assert.InDeltaf(t, math.Pi, (22 / 7.0, "error message %s", "formatted"), 0.01) +// +// Returns whether the assertion was successful (true) or not (false). +func InDeltaf(t TestingT, expected interface{}, actual interface{}, delta float64, msg string, args ...interface{}) bool { + return InDelta(t, expected, actual, delta, append([]interface{}{msg}, args...)...) +} + +// InDeltaSlicef is the same as InDelta, except it compares two slices. +func InDeltaSlicef(t TestingT, expected interface{}, actual interface{}, delta float64, msg string, args ...interface{}) bool { + return InDeltaSlice(t, expected, actual, delta, append([]interface{}{msg}, args...)...) +} + +// InEpsilonf asserts that expected and actual have a relative error less than epsilon +// +// Returns whether the assertion was successful (true) or not (false). +func InEpsilonf(t TestingT, expected interface{}, actual interface{}, epsilon float64, msg string, args ...interface{}) bool { + return InEpsilon(t, expected, actual, epsilon, append([]interface{}{msg}, args...)...) +} + +// InEpsilonSlicef is the same as InEpsilon, except it compares each value from two slices. +func InEpsilonSlicef(t TestingT, expected interface{}, actual interface{}, epsilon float64, msg string, args ...interface{}) bool { + return InEpsilonSlice(t, expected, actual, epsilon, append([]interface{}{msg}, args...)...) +} + +// IsTypef asserts that the specified objects are of the same type. +func IsTypef(t TestingT, expectedType interface{}, object interface{}, msg string, args ...interface{}) bool { + return IsType(t, expectedType, object, append([]interface{}{msg}, args...)...) +} + +// JSONEqf asserts that two JSON strings are equivalent. +// +// assert.JSONEqf(t, `{"hello": "world", "foo": "bar"}`, `{"foo": "bar", "hello": "world"}`, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func JSONEqf(t TestingT, expected string, actual string, msg string, args ...interface{}) bool { + return JSONEq(t, expected, actual, append([]interface{}{msg}, args...)...) +} + +// Lenf asserts that the specified object has specific length. +// Lenf also fails if the object has a type that len() not accept. +// +// assert.Lenf(t, mySlice, 3, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Lenf(t TestingT, object interface{}, length int, msg string, args ...interface{}) bool { + return Len(t, object, length, append([]interface{}{msg}, args...)...) +} + +// Nilf asserts that the specified object is nil. +// +// assert.Nilf(t, err, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Nilf(t TestingT, object interface{}, msg string, args ...interface{}) bool { + return Nil(t, object, append([]interface{}{msg}, args...)...) +} + +// NoErrorf asserts that a function returned no error (i.e. `nil`). +// +// actualObj, err := SomeFunction() +// if assert.NoErrorf(t, err, "error message %s", "formatted") { +// assert.Equal(t, expectedObj, actualObj) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func NoErrorf(t TestingT, err error, msg string, args ...interface{}) bool { + return NoError(t, err, append([]interface{}{msg}, args...)...) +} + +// NotContainsf asserts that the specified string, list(array, slice...) or map does NOT contain the +// specified substring or element. +// +// assert.NotContainsf(t, "Hello World", "Earth", "error message %s", "formatted") +// assert.NotContainsf(t, ["Hello", "World"], "Earth", "error message %s", "formatted") +// assert.NotContainsf(t, {"Hello": "World"}, "Earth", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func NotContainsf(t TestingT, s interface{}, contains interface{}, msg string, args ...interface{}) bool { + return NotContains(t, s, contains, append([]interface{}{msg}, args...)...) +} + +// NotEmptyf asserts that the specified object is NOT empty. I.e. not nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// if assert.NotEmptyf(t, obj, "error message %s", "formatted") { +// assert.Equal(t, "two", obj[1]) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func NotEmptyf(t TestingT, object interface{}, msg string, args ...interface{}) bool { + return NotEmpty(t, object, append([]interface{}{msg}, args...)...) +} + +// NotEqualf asserts that the specified values are NOT equal. +// +// assert.NotEqualf(t, obj1, obj2, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). +func NotEqualf(t TestingT, expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return NotEqual(t, expected, actual, append([]interface{}{msg}, args...)...) +} + +// NotNilf asserts that the specified object is not nil. +// +// assert.NotNilf(t, err, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func NotNilf(t TestingT, object interface{}, msg string, args ...interface{}) bool { + return NotNil(t, object, append([]interface{}{msg}, args...)...) +} + +// NotPanicsf asserts that the code inside the specified PanicTestFunc does NOT panic. +// +// assert.NotPanicsf(t, func(){ RemainCalm() }, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func NotPanicsf(t TestingT, f PanicTestFunc, msg string, args ...interface{}) bool { + return NotPanics(t, f, append([]interface{}{msg}, args...)...) +} + +// NotRegexpf asserts that a specified regexp does not match a string. +// +// assert.NotRegexpf(t, regexp.MustCompile("starts", "error message %s", "formatted"), "it's starting") +// assert.NotRegexpf(t, "^start", "it's not starting", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func NotRegexpf(t TestingT, rx interface{}, str interface{}, msg string, args ...interface{}) bool { + return NotRegexp(t, rx, str, append([]interface{}{msg}, args...)...) +} + +// NotSubsetf asserts that the specified list(array, slice...) contains not all +// elements given in the specified subset(array, slice...). +// +// assert.NotSubsetf(t, [1, 3, 4], [1, 2], "But [1, 3, 4] does not contain [1, 2]", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func NotSubsetf(t TestingT, list interface{}, subset interface{}, msg string, args ...interface{}) bool { + return NotSubset(t, list, subset, append([]interface{}{msg}, args...)...) +} + +// NotZerof asserts that i is not the zero value for its type and returns the truth. +func NotZerof(t TestingT, i interface{}, msg string, args ...interface{}) bool { + return NotZero(t, i, append([]interface{}{msg}, args...)...) +} + +// Panicsf asserts that the code inside the specified PanicTestFunc panics. +// +// assert.Panicsf(t, func(){ GoCrazy() }, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Panicsf(t TestingT, f PanicTestFunc, msg string, args ...interface{}) bool { + return Panics(t, f, append([]interface{}{msg}, args...)...) +} + +// PanicsWithValuef asserts that the code inside the specified PanicTestFunc panics, and that +// the recovered panic value equals the expected panic value. +// +// assert.PanicsWithValuef(t, "crazy error", func(){ GoCrazy() }, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func PanicsWithValuef(t TestingT, expected interface{}, f PanicTestFunc, msg string, args ...interface{}) bool { + return PanicsWithValue(t, expected, f, append([]interface{}{msg}, args...)...) +} + +// Regexpf asserts that a specified regexp matches a string. +// +// assert.Regexpf(t, regexp.MustCompile("start", "error message %s", "formatted"), "it's starting") +// assert.Regexpf(t, "start...$", "it's not starting", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Regexpf(t TestingT, rx interface{}, str interface{}, msg string, args ...interface{}) bool { + return Regexp(t, rx, str, append([]interface{}{msg}, args...)...) +} + +// Subsetf asserts that the specified list(array, slice...) contains all +// elements given in the specified subset(array, slice...). +// +// assert.Subsetf(t, [1, 2, 3], [1, 2], "But [1, 2, 3] does contain [1, 2]", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Subsetf(t TestingT, list interface{}, subset interface{}, msg string, args ...interface{}) bool { + return Subset(t, list, subset, append([]interface{}{msg}, args...)...) +} + +// Truef asserts that the specified value is true. +// +// assert.Truef(t, myBool, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func Truef(t TestingT, value bool, msg string, args ...interface{}) bool { + return True(t, value, append([]interface{}{msg}, args...)...) +} + +// WithinDurationf asserts that the two times are within duration delta of each other. +// +// assert.WithinDurationf(t, time.Now(), time.Now(), 10*time.Second, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func WithinDurationf(t TestingT, expected time.Time, actual time.Time, delta time.Duration, msg string, args ...interface{}) bool { + return WithinDuration(t, expected, actual, delta, append([]interface{}{msg}, args...)...) +} + +// Zerof asserts that i is the zero value for its type and returns the truth. +func Zerof(t TestingT, i interface{}, msg string, args ...interface{}) bool { + return Zero(t, i, append([]interface{}{msg}, args...)...) +} diff --git a/vendor/github.com/stretchr/testify/assert/assertion_format.go.tmpl b/vendor/github.com/stretchr/testify/assert/assertion_format.go.tmpl new file mode 100644 index 000000000000..c5cc66f4305f --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/assertion_format.go.tmpl @@ -0,0 +1,4 @@ +{{.CommentFormat}} +func {{.DocInfo.Name}}f(t TestingT, {{.ParamsFormat}}) bool { + return {{.DocInfo.Name}}(t, {{.ForwardedParamsFormat}}) +} diff --git a/vendor/github.com/stretchr/testify/assert/assertion_forward.go b/vendor/github.com/stretchr/testify/assert/assertion_forward.go new file mode 100644 index 000000000000..fcccbd01c8da --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/assertion_forward.go @@ -0,0 +1,746 @@ +/* +* CODE GENERATED AUTOMATICALLY WITH github.com/stretchr/testify/_codegen +* THIS FILE MUST NOT BE EDITED BY HAND + */ + +package assert + +import ( + http "net/http" + url "net/url" + time "time" +) + +// Condition uses a Comparison to assert a complex condition. +func (a *Assertions) Condition(comp Comparison, msgAndArgs ...interface{}) bool { + return Condition(a.t, comp, msgAndArgs...) +} + +// Conditionf uses a Comparison to assert a complex condition. +func (a *Assertions) Conditionf(comp Comparison, msg string, args ...interface{}) bool { + return Conditionf(a.t, comp, msg, args...) +} + +// Contains asserts that the specified string, list(array, slice...) or map contains the +// specified substring or element. +// +// a.Contains("Hello World", "World") +// a.Contains(["Hello", "World"], "World") +// a.Contains({"Hello": "World"}, "Hello") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Contains(s interface{}, contains interface{}, msgAndArgs ...interface{}) bool { + return Contains(a.t, s, contains, msgAndArgs...) +} + +// Containsf asserts that the specified string, list(array, slice...) or map contains the +// specified substring or element. +// +// a.Containsf("Hello World", "World", "error message %s", "formatted") +// a.Containsf(["Hello", "World"], "World", "error message %s", "formatted") +// a.Containsf({"Hello": "World"}, "Hello", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Containsf(s interface{}, contains interface{}, msg string, args ...interface{}) bool { + return Containsf(a.t, s, contains, msg, args...) +} + +// Empty asserts that the specified object is empty. I.e. nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// a.Empty(obj) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Empty(object interface{}, msgAndArgs ...interface{}) bool { + return Empty(a.t, object, msgAndArgs...) +} + +// Emptyf asserts that the specified object is empty. I.e. nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// a.Emptyf(obj, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Emptyf(object interface{}, msg string, args ...interface{}) bool { + return Emptyf(a.t, object, msg, args...) +} + +// Equal asserts that two objects are equal. +// +// a.Equal(123, 123) +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). Function equality +// cannot be determined and will always fail. +func (a *Assertions) Equal(expected interface{}, actual interface{}, msgAndArgs ...interface{}) bool { + return Equal(a.t, expected, actual, msgAndArgs...) +} + +// EqualError asserts that a function returned an error (i.e. not `nil`) +// and that it is equal to the provided error. +// +// actualObj, err := SomeFunction() +// a.EqualError(err, expectedErrorString) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) EqualError(theError error, errString string, msgAndArgs ...interface{}) bool { + return EqualError(a.t, theError, errString, msgAndArgs...) +} + +// EqualErrorf asserts that a function returned an error (i.e. not `nil`) +// and that it is equal to the provided error. +// +// actualObj, err := SomeFunction() +// a.EqualErrorf(err, expectedErrorString, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) EqualErrorf(theError error, errString string, msg string, args ...interface{}) bool { + return EqualErrorf(a.t, theError, errString, msg, args...) +} + +// EqualValues asserts that two objects are equal or convertable to the same types +// and equal. +// +// a.EqualValues(uint32(123), int32(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) EqualValues(expected interface{}, actual interface{}, msgAndArgs ...interface{}) bool { + return EqualValues(a.t, expected, actual, msgAndArgs...) +} + +// EqualValuesf asserts that two objects are equal or convertable to the same types +// and equal. +// +// a.EqualValuesf(uint32(123, "error message %s", "formatted"), int32(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) EqualValuesf(expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return EqualValuesf(a.t, expected, actual, msg, args...) +} + +// Equalf asserts that two objects are equal. +// +// a.Equalf(123, 123, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). Function equality +// cannot be determined and will always fail. +func (a *Assertions) Equalf(expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return Equalf(a.t, expected, actual, msg, args...) +} + +// Error asserts that a function returned an error (i.e. not `nil`). +// +// actualObj, err := SomeFunction() +// if a.Error(err) { +// assert.Equal(t, expectedError, err) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Error(err error, msgAndArgs ...interface{}) bool { + return Error(a.t, err, msgAndArgs...) +} + +// Errorf asserts that a function returned an error (i.e. not `nil`). +// +// actualObj, err := SomeFunction() +// if a.Errorf(err, "error message %s", "formatted") { +// assert.Equal(t, expectedErrorf, err) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Errorf(err error, msg string, args ...interface{}) bool { + return Errorf(a.t, err, msg, args...) +} + +// Exactly asserts that two objects are equal is value and type. +// +// a.Exactly(int32(123), int64(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Exactly(expected interface{}, actual interface{}, msgAndArgs ...interface{}) bool { + return Exactly(a.t, expected, actual, msgAndArgs...) +} + +// Exactlyf asserts that two objects are equal is value and type. +// +// a.Exactlyf(int32(123, "error message %s", "formatted"), int64(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Exactlyf(expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return Exactlyf(a.t, expected, actual, msg, args...) +} + +// Fail reports a failure through +func (a *Assertions) Fail(failureMessage string, msgAndArgs ...interface{}) bool { + return Fail(a.t, failureMessage, msgAndArgs...) +} + +// FailNow fails test +func (a *Assertions) FailNow(failureMessage string, msgAndArgs ...interface{}) bool { + return FailNow(a.t, failureMessage, msgAndArgs...) +} + +// FailNowf fails test +func (a *Assertions) FailNowf(failureMessage string, msg string, args ...interface{}) bool { + return FailNowf(a.t, failureMessage, msg, args...) +} + +// Failf reports a failure through +func (a *Assertions) Failf(failureMessage string, msg string, args ...interface{}) bool { + return Failf(a.t, failureMessage, msg, args...) +} + +// False asserts that the specified value is false. +// +// a.False(myBool) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) False(value bool, msgAndArgs ...interface{}) bool { + return False(a.t, value, msgAndArgs...) +} + +// Falsef asserts that the specified value is false. +// +// a.Falsef(myBool, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Falsef(value bool, msg string, args ...interface{}) bool { + return Falsef(a.t, value, msg, args...) +} + +// HTTPBodyContains asserts that a specified handler returns a +// body that contains a string. +// +// a.HTTPBodyContains(myHandler, "www.google.com", nil, "I'm Feeling Lucky") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPBodyContains(handler http.HandlerFunc, method string, url string, values url.Values, str interface{}) bool { + return HTTPBodyContains(a.t, handler, method, url, values, str) +} + +// HTTPBodyContainsf asserts that a specified handler returns a +// body that contains a string. +// +// a.HTTPBodyContainsf(myHandler, "www.google.com", nil, "I'm Feeling Lucky", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPBodyContainsf(handler http.HandlerFunc, method string, url string, values url.Values, str interface{}) bool { + return HTTPBodyContainsf(a.t, handler, method, url, values, str) +} + +// HTTPBodyNotContains asserts that a specified handler returns a +// body that does not contain a string. +// +// a.HTTPBodyNotContains(myHandler, "www.google.com", nil, "I'm Feeling Lucky") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPBodyNotContains(handler http.HandlerFunc, method string, url string, values url.Values, str interface{}) bool { + return HTTPBodyNotContains(a.t, handler, method, url, values, str) +} + +// HTTPBodyNotContainsf asserts that a specified handler returns a +// body that does not contain a string. +// +// a.HTTPBodyNotContainsf(myHandler, "www.google.com", nil, "I'm Feeling Lucky", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPBodyNotContainsf(handler http.HandlerFunc, method string, url string, values url.Values, str interface{}) bool { + return HTTPBodyNotContainsf(a.t, handler, method, url, values, str) +} + +// HTTPError asserts that a specified handler returns an error status code. +// +// a.HTTPError(myHandler, "POST", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPError(handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPError(a.t, handler, method, url, values) +} + +// HTTPErrorf asserts that a specified handler returns an error status code. +// +// a.HTTPErrorf(myHandler, "POST", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true, "error message %s", "formatted") or not (false). +func (a *Assertions) HTTPErrorf(handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPErrorf(a.t, handler, method, url, values) +} + +// HTTPRedirect asserts that a specified handler returns a redirect status code. +// +// a.HTTPRedirect(myHandler, "GET", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPRedirect(handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPRedirect(a.t, handler, method, url, values) +} + +// HTTPRedirectf asserts that a specified handler returns a redirect status code. +// +// a.HTTPRedirectf(myHandler, "GET", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true, "error message %s", "formatted") or not (false). +func (a *Assertions) HTTPRedirectf(handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPRedirectf(a.t, handler, method, url, values) +} + +// HTTPSuccess asserts that a specified handler returns a success status code. +// +// a.HTTPSuccess(myHandler, "POST", "http://www.google.com", nil) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPSuccess(handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPSuccess(a.t, handler, method, url, values) +} + +// HTTPSuccessf asserts that a specified handler returns a success status code. +// +// a.HTTPSuccessf(myHandler, "POST", "http://www.google.com", nil, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) HTTPSuccessf(handler http.HandlerFunc, method string, url string, values url.Values) bool { + return HTTPSuccessf(a.t, handler, method, url, values) +} + +// Implements asserts that an object is implemented by the specified interface. +// +// a.Implements((*MyInterface)(nil), new(MyObject)) +func (a *Assertions) Implements(interfaceObject interface{}, object interface{}, msgAndArgs ...interface{}) bool { + return Implements(a.t, interfaceObject, object, msgAndArgs...) +} + +// Implementsf asserts that an object is implemented by the specified interface. +// +// a.Implementsf((*MyInterface, "error message %s", "formatted")(nil), new(MyObject)) +func (a *Assertions) Implementsf(interfaceObject interface{}, object interface{}, msg string, args ...interface{}) bool { + return Implementsf(a.t, interfaceObject, object, msg, args...) +} + +// InDelta asserts that the two numerals are within delta of each other. +// +// a.InDelta(math.Pi, (22 / 7.0), 0.01) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) InDelta(expected interface{}, actual interface{}, delta float64, msgAndArgs ...interface{}) bool { + return InDelta(a.t, expected, actual, delta, msgAndArgs...) +} + +// InDeltaSlice is the same as InDelta, except it compares two slices. +func (a *Assertions) InDeltaSlice(expected interface{}, actual interface{}, delta float64, msgAndArgs ...interface{}) bool { + return InDeltaSlice(a.t, expected, actual, delta, msgAndArgs...) +} + +// InDeltaSlicef is the same as InDelta, except it compares two slices. +func (a *Assertions) InDeltaSlicef(expected interface{}, actual interface{}, delta float64, msg string, args ...interface{}) bool { + return InDeltaSlicef(a.t, expected, actual, delta, msg, args...) +} + +// InDeltaf asserts that the two numerals are within delta of each other. +// +// a.InDeltaf(math.Pi, (22 / 7.0, "error message %s", "formatted"), 0.01) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) InDeltaf(expected interface{}, actual interface{}, delta float64, msg string, args ...interface{}) bool { + return InDeltaf(a.t, expected, actual, delta, msg, args...) +} + +// InEpsilon asserts that expected and actual have a relative error less than epsilon +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) InEpsilon(expected interface{}, actual interface{}, epsilon float64, msgAndArgs ...interface{}) bool { + return InEpsilon(a.t, expected, actual, epsilon, msgAndArgs...) +} + +// InEpsilonSlice is the same as InEpsilon, except it compares each value from two slices. +func (a *Assertions) InEpsilonSlice(expected interface{}, actual interface{}, epsilon float64, msgAndArgs ...interface{}) bool { + return InEpsilonSlice(a.t, expected, actual, epsilon, msgAndArgs...) +} + +// InEpsilonSlicef is the same as InEpsilon, except it compares each value from two slices. +func (a *Assertions) InEpsilonSlicef(expected interface{}, actual interface{}, epsilon float64, msg string, args ...interface{}) bool { + return InEpsilonSlicef(a.t, expected, actual, epsilon, msg, args...) +} + +// InEpsilonf asserts that expected and actual have a relative error less than epsilon +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) InEpsilonf(expected interface{}, actual interface{}, epsilon float64, msg string, args ...interface{}) bool { + return InEpsilonf(a.t, expected, actual, epsilon, msg, args...) +} + +// IsType asserts that the specified objects are of the same type. +func (a *Assertions) IsType(expectedType interface{}, object interface{}, msgAndArgs ...interface{}) bool { + return IsType(a.t, expectedType, object, msgAndArgs...) +} + +// IsTypef asserts that the specified objects are of the same type. +func (a *Assertions) IsTypef(expectedType interface{}, object interface{}, msg string, args ...interface{}) bool { + return IsTypef(a.t, expectedType, object, msg, args...) +} + +// JSONEq asserts that two JSON strings are equivalent. +// +// a.JSONEq(`{"hello": "world", "foo": "bar"}`, `{"foo": "bar", "hello": "world"}`) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) JSONEq(expected string, actual string, msgAndArgs ...interface{}) bool { + return JSONEq(a.t, expected, actual, msgAndArgs...) +} + +// JSONEqf asserts that two JSON strings are equivalent. +// +// a.JSONEqf(`{"hello": "world", "foo": "bar"}`, `{"foo": "bar", "hello": "world"}`, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) JSONEqf(expected string, actual string, msg string, args ...interface{}) bool { + return JSONEqf(a.t, expected, actual, msg, args...) +} + +// Len asserts that the specified object has specific length. +// Len also fails if the object has a type that len() not accept. +// +// a.Len(mySlice, 3) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Len(object interface{}, length int, msgAndArgs ...interface{}) bool { + return Len(a.t, object, length, msgAndArgs...) +} + +// Lenf asserts that the specified object has specific length. +// Lenf also fails if the object has a type that len() not accept. +// +// a.Lenf(mySlice, 3, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Lenf(object interface{}, length int, msg string, args ...interface{}) bool { + return Lenf(a.t, object, length, msg, args...) +} + +// Nil asserts that the specified object is nil. +// +// a.Nil(err) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Nil(object interface{}, msgAndArgs ...interface{}) bool { + return Nil(a.t, object, msgAndArgs...) +} + +// Nilf asserts that the specified object is nil. +// +// a.Nilf(err, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Nilf(object interface{}, msg string, args ...interface{}) bool { + return Nilf(a.t, object, msg, args...) +} + +// NoError asserts that a function returned no error (i.e. `nil`). +// +// actualObj, err := SomeFunction() +// if a.NoError(err) { +// assert.Equal(t, expectedObj, actualObj) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NoError(err error, msgAndArgs ...interface{}) bool { + return NoError(a.t, err, msgAndArgs...) +} + +// NoErrorf asserts that a function returned no error (i.e. `nil`). +// +// actualObj, err := SomeFunction() +// if a.NoErrorf(err, "error message %s", "formatted") { +// assert.Equal(t, expectedObj, actualObj) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NoErrorf(err error, msg string, args ...interface{}) bool { + return NoErrorf(a.t, err, msg, args...) +} + +// NotContains asserts that the specified string, list(array, slice...) or map does NOT contain the +// specified substring or element. +// +// a.NotContains("Hello World", "Earth") +// a.NotContains(["Hello", "World"], "Earth") +// a.NotContains({"Hello": "World"}, "Earth") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotContains(s interface{}, contains interface{}, msgAndArgs ...interface{}) bool { + return NotContains(a.t, s, contains, msgAndArgs...) +} + +// NotContainsf asserts that the specified string, list(array, slice...) or map does NOT contain the +// specified substring or element. +// +// a.NotContainsf("Hello World", "Earth", "error message %s", "formatted") +// a.NotContainsf(["Hello", "World"], "Earth", "error message %s", "formatted") +// a.NotContainsf({"Hello": "World"}, "Earth", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotContainsf(s interface{}, contains interface{}, msg string, args ...interface{}) bool { + return NotContainsf(a.t, s, contains, msg, args...) +} + +// NotEmpty asserts that the specified object is NOT empty. I.e. not nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// if a.NotEmpty(obj) { +// assert.Equal(t, "two", obj[1]) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotEmpty(object interface{}, msgAndArgs ...interface{}) bool { + return NotEmpty(a.t, object, msgAndArgs...) +} + +// NotEmptyf asserts that the specified object is NOT empty. I.e. not nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// if a.NotEmptyf(obj, "error message %s", "formatted") { +// assert.Equal(t, "two", obj[1]) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotEmptyf(object interface{}, msg string, args ...interface{}) bool { + return NotEmptyf(a.t, object, msg, args...) +} + +// NotEqual asserts that the specified values are NOT equal. +// +// a.NotEqual(obj1, obj2) +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). +func (a *Assertions) NotEqual(expected interface{}, actual interface{}, msgAndArgs ...interface{}) bool { + return NotEqual(a.t, expected, actual, msgAndArgs...) +} + +// NotEqualf asserts that the specified values are NOT equal. +// +// a.NotEqualf(obj1, obj2, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). +func (a *Assertions) NotEqualf(expected interface{}, actual interface{}, msg string, args ...interface{}) bool { + return NotEqualf(a.t, expected, actual, msg, args...) +} + +// NotNil asserts that the specified object is not nil. +// +// a.NotNil(err) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotNil(object interface{}, msgAndArgs ...interface{}) bool { + return NotNil(a.t, object, msgAndArgs...) +} + +// NotNilf asserts that the specified object is not nil. +// +// a.NotNilf(err, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotNilf(object interface{}, msg string, args ...interface{}) bool { + return NotNilf(a.t, object, msg, args...) +} + +// NotPanics asserts that the code inside the specified PanicTestFunc does NOT panic. +// +// a.NotPanics(func(){ RemainCalm() }) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotPanics(f PanicTestFunc, msgAndArgs ...interface{}) bool { + return NotPanics(a.t, f, msgAndArgs...) +} + +// NotPanicsf asserts that the code inside the specified PanicTestFunc does NOT panic. +// +// a.NotPanicsf(func(){ RemainCalm() }, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotPanicsf(f PanicTestFunc, msg string, args ...interface{}) bool { + return NotPanicsf(a.t, f, msg, args...) +} + +// NotRegexp asserts that a specified regexp does not match a string. +// +// a.NotRegexp(regexp.MustCompile("starts"), "it's starting") +// a.NotRegexp("^start", "it's not starting") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotRegexp(rx interface{}, str interface{}, msgAndArgs ...interface{}) bool { + return NotRegexp(a.t, rx, str, msgAndArgs...) +} + +// NotRegexpf asserts that a specified regexp does not match a string. +// +// a.NotRegexpf(regexp.MustCompile("starts", "error message %s", "formatted"), "it's starting") +// a.NotRegexpf("^start", "it's not starting", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotRegexpf(rx interface{}, str interface{}, msg string, args ...interface{}) bool { + return NotRegexpf(a.t, rx, str, msg, args...) +} + +// NotSubset asserts that the specified list(array, slice...) contains not all +// elements given in the specified subset(array, slice...). +// +// a.NotSubset([1, 3, 4], [1, 2], "But [1, 3, 4] does not contain [1, 2]") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotSubset(list interface{}, subset interface{}, msgAndArgs ...interface{}) bool { + return NotSubset(a.t, list, subset, msgAndArgs...) +} + +// NotSubsetf asserts that the specified list(array, slice...) contains not all +// elements given in the specified subset(array, slice...). +// +// a.NotSubsetf([1, 3, 4], [1, 2], "But [1, 3, 4] does not contain [1, 2]", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) NotSubsetf(list interface{}, subset interface{}, msg string, args ...interface{}) bool { + return NotSubsetf(a.t, list, subset, msg, args...) +} + +// NotZero asserts that i is not the zero value for its type and returns the truth. +func (a *Assertions) NotZero(i interface{}, msgAndArgs ...interface{}) bool { + return NotZero(a.t, i, msgAndArgs...) +} + +// NotZerof asserts that i is not the zero value for its type and returns the truth. +func (a *Assertions) NotZerof(i interface{}, msg string, args ...interface{}) bool { + return NotZerof(a.t, i, msg, args...) +} + +// Panics asserts that the code inside the specified PanicTestFunc panics. +// +// a.Panics(func(){ GoCrazy() }) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Panics(f PanicTestFunc, msgAndArgs ...interface{}) bool { + return Panics(a.t, f, msgAndArgs...) +} + +// PanicsWithValue asserts that the code inside the specified PanicTestFunc panics, and that +// the recovered panic value equals the expected panic value. +// +// a.PanicsWithValue("crazy error", func(){ GoCrazy() }) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) PanicsWithValue(expected interface{}, f PanicTestFunc, msgAndArgs ...interface{}) bool { + return PanicsWithValue(a.t, expected, f, msgAndArgs...) +} + +// PanicsWithValuef asserts that the code inside the specified PanicTestFunc panics, and that +// the recovered panic value equals the expected panic value. +// +// a.PanicsWithValuef("crazy error", func(){ GoCrazy() }, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) PanicsWithValuef(expected interface{}, f PanicTestFunc, msg string, args ...interface{}) bool { + return PanicsWithValuef(a.t, expected, f, msg, args...) +} + +// Panicsf asserts that the code inside the specified PanicTestFunc panics. +// +// a.Panicsf(func(){ GoCrazy() }, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Panicsf(f PanicTestFunc, msg string, args ...interface{}) bool { + return Panicsf(a.t, f, msg, args...) +} + +// Regexp asserts that a specified regexp matches a string. +// +// a.Regexp(regexp.MustCompile("start"), "it's starting") +// a.Regexp("start...$", "it's not starting") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Regexp(rx interface{}, str interface{}, msgAndArgs ...interface{}) bool { + return Regexp(a.t, rx, str, msgAndArgs...) +} + +// Regexpf asserts that a specified regexp matches a string. +// +// a.Regexpf(regexp.MustCompile("start", "error message %s", "formatted"), "it's starting") +// a.Regexpf("start...$", "it's not starting", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Regexpf(rx interface{}, str interface{}, msg string, args ...interface{}) bool { + return Regexpf(a.t, rx, str, msg, args...) +} + +// Subset asserts that the specified list(array, slice...) contains all +// elements given in the specified subset(array, slice...). +// +// a.Subset([1, 2, 3], [1, 2], "But [1, 2, 3] does contain [1, 2]") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Subset(list interface{}, subset interface{}, msgAndArgs ...interface{}) bool { + return Subset(a.t, list, subset, msgAndArgs...) +} + +// Subsetf asserts that the specified list(array, slice...) contains all +// elements given in the specified subset(array, slice...). +// +// a.Subsetf([1, 2, 3], [1, 2], "But [1, 2, 3] does contain [1, 2]", "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Subsetf(list interface{}, subset interface{}, msg string, args ...interface{}) bool { + return Subsetf(a.t, list, subset, msg, args...) +} + +// True asserts that the specified value is true. +// +// a.True(myBool) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) True(value bool, msgAndArgs ...interface{}) bool { + return True(a.t, value, msgAndArgs...) +} + +// Truef asserts that the specified value is true. +// +// a.Truef(myBool, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) Truef(value bool, msg string, args ...interface{}) bool { + return Truef(a.t, value, msg, args...) +} + +// WithinDuration asserts that the two times are within duration delta of each other. +// +// a.WithinDuration(time.Now(), time.Now(), 10*time.Second) +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) WithinDuration(expected time.Time, actual time.Time, delta time.Duration, msgAndArgs ...interface{}) bool { + return WithinDuration(a.t, expected, actual, delta, msgAndArgs...) +} + +// WithinDurationf asserts that the two times are within duration delta of each other. +// +// a.WithinDurationf(time.Now(), time.Now(), 10*time.Second, "error message %s", "formatted") +// +// Returns whether the assertion was successful (true) or not (false). +func (a *Assertions) WithinDurationf(expected time.Time, actual time.Time, delta time.Duration, msg string, args ...interface{}) bool { + return WithinDurationf(a.t, expected, actual, delta, msg, args...) +} + +// Zero asserts that i is the zero value for its type and returns the truth. +func (a *Assertions) Zero(i interface{}, msgAndArgs ...interface{}) bool { + return Zero(a.t, i, msgAndArgs...) +} + +// Zerof asserts that i is the zero value for its type and returns the truth. +func (a *Assertions) Zerof(i interface{}, msg string, args ...interface{}) bool { + return Zerof(a.t, i, msg, args...) +} diff --git a/vendor/github.com/stretchr/testify/assert/assertion_forward.go.tmpl b/vendor/github.com/stretchr/testify/assert/assertion_forward.go.tmpl new file mode 100644 index 000000000000..99f9acfbba5f --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/assertion_forward.go.tmpl @@ -0,0 +1,4 @@ +{{.CommentWithoutT "a"}} +func (a *Assertions) {{.DocInfo.Name}}({{.Params}}) bool { + return {{.DocInfo.Name}}(a.t, {{.ForwardedParams}}) +} diff --git a/vendor/github.com/stretchr/testify/assert/assertions.go b/vendor/github.com/stretchr/testify/assert/assertions.go new file mode 100644 index 000000000000..c8034f68ae49 --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/assertions.go @@ -0,0 +1,1208 @@ +package assert + +import ( + "bufio" + "bytes" + "encoding/json" + "errors" + "fmt" + "math" + "reflect" + "regexp" + "runtime" + "strings" + "time" + "unicode" + "unicode/utf8" + + "github.com/davecgh/go-spew/spew" + "github.com/pmezard/go-difflib/difflib" +) + +//go:generate go run ../_codegen/main.go -output-package=assert -template=assertion_format.go.tmpl + +// TestingT is an interface wrapper around *testing.T +type TestingT interface { + Errorf(format string, args ...interface{}) +} + +// Comparison a custom function that returns true on success and false on failure +type Comparison func() (success bool) + +/* + Helper functions +*/ + +// ObjectsAreEqual determines if two objects are considered equal. +// +// This function does no assertion of any kind. +func ObjectsAreEqual(expected, actual interface{}) bool { + + if expected == nil || actual == nil { + return expected == actual + } + if exp, ok := expected.([]byte); ok { + act, ok := actual.([]byte) + if !ok { + return false + } else if exp == nil || act == nil { + return exp == nil && act == nil + } + return bytes.Equal(exp, act) + } + return reflect.DeepEqual(expected, actual) + +} + +// ObjectsAreEqualValues gets whether two objects are equal, or if their +// values are equal. +func ObjectsAreEqualValues(expected, actual interface{}) bool { + if ObjectsAreEqual(expected, actual) { + return true + } + + actualType := reflect.TypeOf(actual) + if actualType == nil { + return false + } + expectedValue := reflect.ValueOf(expected) + if expectedValue.IsValid() && expectedValue.Type().ConvertibleTo(actualType) { + // Attempt comparison after type conversion + return reflect.DeepEqual(expectedValue.Convert(actualType).Interface(), actual) + } + + return false +} + +/* CallerInfo is necessary because the assert functions use the testing object +internally, causing it to print the file:line of the assert method, rather than where +the problem actually occurred in calling code.*/ + +// CallerInfo returns an array of strings containing the file and line number +// of each stack frame leading from the current test to the assert call that +// failed. +func CallerInfo() []string { + + pc := uintptr(0) + file := "" + line := 0 + ok := false + name := "" + + callers := []string{} + for i := 0; ; i++ { + pc, file, line, ok = runtime.Caller(i) + if !ok { + // The breaks below failed to terminate the loop, and we ran off the + // end of the call stack. + break + } + + // This is a huge edge case, but it will panic if this is the case, see #180 + if file == "" { + break + } + + f := runtime.FuncForPC(pc) + if f == nil { + break + } + name = f.Name() + + // testing.tRunner is the standard library function that calls + // tests. Subtests are called directly by tRunner, without going through + // the Test/Benchmark/Example function that contains the t.Run calls, so + // with subtests we should break when we hit tRunner, without adding it + // to the list of callers. + if name == "testing.tRunner" { + break + } + + parts := strings.Split(file, "/") + file = parts[len(parts)-1] + if len(parts) > 1 { + dir := parts[len(parts)-2] + if (dir != "assert" && dir != "mock" && dir != "require") || file == "mock_test.go" { + callers = append(callers, fmt.Sprintf("%s:%d", file, line)) + } + } + + // Drop the package + segments := strings.Split(name, ".") + name = segments[len(segments)-1] + if isTest(name, "Test") || + isTest(name, "Benchmark") || + isTest(name, "Example") { + break + } + } + + return callers +} + +// Stolen from the `go test` tool. +// isTest tells whether name looks like a test (or benchmark, according to prefix). +// It is a Test (say) if there is a character after Test that is not a lower-case letter. +// We don't want TesticularCancer. +func isTest(name, prefix string) bool { + if !strings.HasPrefix(name, prefix) { + return false + } + if len(name) == len(prefix) { // "Test" is ok + return true + } + rune, _ := utf8.DecodeRuneInString(name[len(prefix):]) + return !unicode.IsLower(rune) +} + +// getWhitespaceString returns a string that is long enough to overwrite the default +// output from the go testing framework. +func getWhitespaceString() string { + + _, file, line, ok := runtime.Caller(1) + if !ok { + return "" + } + parts := strings.Split(file, "/") + file = parts[len(parts)-1] + + return strings.Repeat(" ", len(fmt.Sprintf("%s:%d: ", file, line))) + +} + +func messageFromMsgAndArgs(msgAndArgs ...interface{}) string { + if len(msgAndArgs) == 0 || msgAndArgs == nil { + return "" + } + if len(msgAndArgs) == 1 { + return msgAndArgs[0].(string) + } + if len(msgAndArgs) > 1 { + return fmt.Sprintf(msgAndArgs[0].(string), msgAndArgs[1:]...) + } + return "" +} + +// Aligns the provided message so that all lines after the first line start at the same location as the first line. +// Assumes that the first line starts at the correct location (after carriage return, tab, label, spacer and tab). +// The longestLabelLen parameter specifies the length of the longest label in the output (required becaues this is the +// basis on which the alignment occurs). +func indentMessageLines(message string, longestLabelLen int) string { + outBuf := new(bytes.Buffer) + + for i, scanner := 0, bufio.NewScanner(strings.NewReader(message)); scanner.Scan(); i++ { + // no need to align first line because it starts at the correct location (after the label) + if i != 0 { + // append alignLen+1 spaces to align with "{{longestLabel}}:" before adding tab + outBuf.WriteString("\n\r\t" + strings.Repeat(" ", longestLabelLen+1) + "\t") + } + outBuf.WriteString(scanner.Text()) + } + + return outBuf.String() +} + +type failNower interface { + FailNow() +} + +// FailNow fails test +func FailNow(t TestingT, failureMessage string, msgAndArgs ...interface{}) bool { + Fail(t, failureMessage, msgAndArgs...) + + // We cannot extend TestingT with FailNow() and + // maintain backwards compatibility, so we fallback + // to panicking when FailNow is not available in + // TestingT. + // See issue #263 + + if t, ok := t.(failNower); ok { + t.FailNow() + } else { + panic("test failed and t is missing `FailNow()`") + } + return false +} + +// Fail reports a failure through +func Fail(t TestingT, failureMessage string, msgAndArgs ...interface{}) bool { + content := []labeledContent{ + {"Error Trace", strings.Join(CallerInfo(), "\n\r\t\t\t")}, + {"Error", failureMessage}, + } + + message := messageFromMsgAndArgs(msgAndArgs...) + if len(message) > 0 { + content = append(content, labeledContent{"Messages", message}) + } + + t.Errorf("%s", "\r"+getWhitespaceString()+labeledOutput(content...)) + + return false +} + +type labeledContent struct { + label string + content string +} + +// labeledOutput returns a string consisting of the provided labeledContent. Each labeled output is appended in the following manner: +// +// \r\t{{label}}:{{align_spaces}}\t{{content}}\n +// +// The initial carriage return is required to undo/erase any padding added by testing.T.Errorf. The "\t{{label}}:" is for the label. +// If a label is shorter than the longest label provided, padding spaces are added to make all the labels match in length. Once this +// alignment is achieved, "\t{{content}}\n" is added for the output. +// +// If the content of the labeledOutput contains line breaks, the subsequent lines are aligned so that they start at the same location as the first line. +func labeledOutput(content ...labeledContent) string { + longestLabel := 0 + for _, v := range content { + if len(v.label) > longestLabel { + longestLabel = len(v.label) + } + } + var output string + for _, v := range content { + output += "\r\t" + v.label + ":" + strings.Repeat(" ", longestLabel-len(v.label)) + "\t" + indentMessageLines(v.content, longestLabel) + "\n" + } + return output +} + +// Implements asserts that an object is implemented by the specified interface. +// +// assert.Implements(t, (*MyInterface)(nil), new(MyObject)) +func Implements(t TestingT, interfaceObject interface{}, object interface{}, msgAndArgs ...interface{}) bool { + + interfaceType := reflect.TypeOf(interfaceObject).Elem() + + if !reflect.TypeOf(object).Implements(interfaceType) { + return Fail(t, fmt.Sprintf("%T must implement %v", object, interfaceType), msgAndArgs...) + } + + return true + +} + +// IsType asserts that the specified objects are of the same type. +func IsType(t TestingT, expectedType interface{}, object interface{}, msgAndArgs ...interface{}) bool { + + if !ObjectsAreEqual(reflect.TypeOf(object), reflect.TypeOf(expectedType)) { + return Fail(t, fmt.Sprintf("Object expected to be of type %v, but was %v", reflect.TypeOf(expectedType), reflect.TypeOf(object)), msgAndArgs...) + } + + return true +} + +// Equal asserts that two objects are equal. +// +// assert.Equal(t, 123, 123) +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). Function equality +// cannot be determined and will always fail. +func Equal(t TestingT, expected, actual interface{}, msgAndArgs ...interface{}) bool { + if err := validateEqualArgs(expected, actual); err != nil { + return Fail(t, fmt.Sprintf("Invalid operation: %#v == %#v (%s)", + expected, actual, err), msgAndArgs...) + } + + if !ObjectsAreEqual(expected, actual) { + diff := diff(expected, actual) + expected, actual = formatUnequalValues(expected, actual) + return Fail(t, fmt.Sprintf("Not equal: \n"+ + "expected: %s\n"+ + "actual: %s%s", expected, actual, diff), msgAndArgs...) + } + + return true + +} + +// formatUnequalValues takes two values of arbitrary types and returns string +// representations appropriate to be presented to the user. +// +// If the values are not of like type, the returned strings will be prefixed +// with the type name, and the value will be enclosed in parenthesis similar +// to a type conversion in the Go grammar. +func formatUnequalValues(expected, actual interface{}) (e string, a string) { + if reflect.TypeOf(expected) != reflect.TypeOf(actual) { + return fmt.Sprintf("%T(%#v)", expected, expected), + fmt.Sprintf("%T(%#v)", actual, actual) + } + + return fmt.Sprintf("%#v", expected), + fmt.Sprintf("%#v", actual) +} + +// EqualValues asserts that two objects are equal or convertable to the same types +// and equal. +// +// assert.EqualValues(t, uint32(123), int32(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func EqualValues(t TestingT, expected, actual interface{}, msgAndArgs ...interface{}) bool { + + if !ObjectsAreEqualValues(expected, actual) { + diff := diff(expected, actual) + expected, actual = formatUnequalValues(expected, actual) + return Fail(t, fmt.Sprintf("Not equal: \n"+ + "expected: %s\n"+ + "actual: %s%s", expected, actual, diff), msgAndArgs...) + } + + return true + +} + +// Exactly asserts that two objects are equal is value and type. +// +// assert.Exactly(t, int32(123), int64(123)) +// +// Returns whether the assertion was successful (true) or not (false). +func Exactly(t TestingT, expected, actual interface{}, msgAndArgs ...interface{}) bool { + + aType := reflect.TypeOf(expected) + bType := reflect.TypeOf(actual) + + if aType != bType { + return Fail(t, fmt.Sprintf("Types expected to match exactly\n\r\t%v != %v", aType, bType), msgAndArgs...) + } + + return Equal(t, expected, actual, msgAndArgs...) + +} + +// NotNil asserts that the specified object is not nil. +// +// assert.NotNil(t, err) +// +// Returns whether the assertion was successful (true) or not (false). +func NotNil(t TestingT, object interface{}, msgAndArgs ...interface{}) bool { + if !isNil(object) { + return true + } + return Fail(t, "Expected value not to be nil.", msgAndArgs...) +} + +// isNil checks if a specified object is nil or not, without Failing. +func isNil(object interface{}) bool { + if object == nil { + return true + } + + value := reflect.ValueOf(object) + kind := value.Kind() + if kind >= reflect.Chan && kind <= reflect.Slice && value.IsNil() { + return true + } + + return false +} + +// Nil asserts that the specified object is nil. +// +// assert.Nil(t, err) +// +// Returns whether the assertion was successful (true) or not (false). +func Nil(t TestingT, object interface{}, msgAndArgs ...interface{}) bool { + if isNil(object) { + return true + } + return Fail(t, fmt.Sprintf("Expected nil, but got: %#v", object), msgAndArgs...) +} + +var numericZeros = []interface{}{ + int(0), + int8(0), + int16(0), + int32(0), + int64(0), + uint(0), + uint8(0), + uint16(0), + uint32(0), + uint64(0), + float32(0), + float64(0), +} + +// isEmpty gets whether the specified object is considered empty or not. +func isEmpty(object interface{}) bool { + + if object == nil { + return true + } else if object == "" { + return true + } else if object == false { + return true + } + + for _, v := range numericZeros { + if object == v { + return true + } + } + + objValue := reflect.ValueOf(object) + + switch objValue.Kind() { + case reflect.Map: + fallthrough + case reflect.Slice, reflect.Chan: + { + return (objValue.Len() == 0) + } + case reflect.Struct: + switch object.(type) { + case time.Time: + return object.(time.Time).IsZero() + } + case reflect.Ptr: + { + if objValue.IsNil() { + return true + } + switch object.(type) { + case *time.Time: + return object.(*time.Time).IsZero() + default: + return false + } + } + } + return false +} + +// Empty asserts that the specified object is empty. I.e. nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// assert.Empty(t, obj) +// +// Returns whether the assertion was successful (true) or not (false). +func Empty(t TestingT, object interface{}, msgAndArgs ...interface{}) bool { + + pass := isEmpty(object) + if !pass { + Fail(t, fmt.Sprintf("Should be empty, but was %v", object), msgAndArgs...) + } + + return pass + +} + +// NotEmpty asserts that the specified object is NOT empty. I.e. not nil, "", false, 0 or either +// a slice or a channel with len == 0. +// +// if assert.NotEmpty(t, obj) { +// assert.Equal(t, "two", obj[1]) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func NotEmpty(t TestingT, object interface{}, msgAndArgs ...interface{}) bool { + + pass := !isEmpty(object) + if !pass { + Fail(t, fmt.Sprintf("Should NOT be empty, but was %v", object), msgAndArgs...) + } + + return pass + +} + +// getLen try to get length of object. +// return (false, 0) if impossible. +func getLen(x interface{}) (ok bool, length int) { + v := reflect.ValueOf(x) + defer func() { + if e := recover(); e != nil { + ok = false + } + }() + return true, v.Len() +} + +// Len asserts that the specified object has specific length. +// Len also fails if the object has a type that len() not accept. +// +// assert.Len(t, mySlice, 3) +// +// Returns whether the assertion was successful (true) or not (false). +func Len(t TestingT, object interface{}, length int, msgAndArgs ...interface{}) bool { + ok, l := getLen(object) + if !ok { + return Fail(t, fmt.Sprintf("\"%s\" could not be applied builtin len()", object), msgAndArgs...) + } + + if l != length { + return Fail(t, fmt.Sprintf("\"%s\" should have %d item(s), but has %d", object, length, l), msgAndArgs...) + } + return true +} + +// True asserts that the specified value is true. +// +// assert.True(t, myBool) +// +// Returns whether the assertion was successful (true) or not (false). +func True(t TestingT, value bool, msgAndArgs ...interface{}) bool { + + if value != true { + return Fail(t, "Should be true", msgAndArgs...) + } + + return true + +} + +// False asserts that the specified value is false. +// +// assert.False(t, myBool) +// +// Returns whether the assertion was successful (true) or not (false). +func False(t TestingT, value bool, msgAndArgs ...interface{}) bool { + + if value != false { + return Fail(t, "Should be false", msgAndArgs...) + } + + return true + +} + +// NotEqual asserts that the specified values are NOT equal. +// +// assert.NotEqual(t, obj1, obj2) +// +// Returns whether the assertion was successful (true) or not (false). +// +// Pointer variable equality is determined based on the equality of the +// referenced values (as opposed to the memory addresses). +func NotEqual(t TestingT, expected, actual interface{}, msgAndArgs ...interface{}) bool { + if err := validateEqualArgs(expected, actual); err != nil { + return Fail(t, fmt.Sprintf("Invalid operation: %#v != %#v (%s)", + expected, actual, err), msgAndArgs...) + } + + if ObjectsAreEqual(expected, actual) { + return Fail(t, fmt.Sprintf("Should not be: %#v\n", actual), msgAndArgs...) + } + + return true + +} + +// containsElement try loop over the list check if the list includes the element. +// return (false, false) if impossible. +// return (true, false) if element was not found. +// return (true, true) if element was found. +func includeElement(list interface{}, element interface{}) (ok, found bool) { + + listValue := reflect.ValueOf(list) + elementValue := reflect.ValueOf(element) + defer func() { + if e := recover(); e != nil { + ok = false + found = false + } + }() + + if reflect.TypeOf(list).Kind() == reflect.String { + return true, strings.Contains(listValue.String(), elementValue.String()) + } + + if reflect.TypeOf(list).Kind() == reflect.Map { + mapKeys := listValue.MapKeys() + for i := 0; i < len(mapKeys); i++ { + if ObjectsAreEqual(mapKeys[i].Interface(), element) { + return true, true + } + } + return true, false + } + + for i := 0; i < listValue.Len(); i++ { + if ObjectsAreEqual(listValue.Index(i).Interface(), element) { + return true, true + } + } + return true, false + +} + +// Contains asserts that the specified string, list(array, slice...) or map contains the +// specified substring or element. +// +// assert.Contains(t, "Hello World", "World") +// assert.Contains(t, ["Hello", "World"], "World") +// assert.Contains(t, {"Hello": "World"}, "Hello") +// +// Returns whether the assertion was successful (true) or not (false). +func Contains(t TestingT, s, contains interface{}, msgAndArgs ...interface{}) bool { + + ok, found := includeElement(s, contains) + if !ok { + return Fail(t, fmt.Sprintf("\"%s\" could not be applied builtin len()", s), msgAndArgs...) + } + if !found { + return Fail(t, fmt.Sprintf("\"%s\" does not contain \"%s\"", s, contains), msgAndArgs...) + } + + return true + +} + +// NotContains asserts that the specified string, list(array, slice...) or map does NOT contain the +// specified substring or element. +// +// assert.NotContains(t, "Hello World", "Earth") +// assert.NotContains(t, ["Hello", "World"], "Earth") +// assert.NotContains(t, {"Hello": "World"}, "Earth") +// +// Returns whether the assertion was successful (true) or not (false). +func NotContains(t TestingT, s, contains interface{}, msgAndArgs ...interface{}) bool { + + ok, found := includeElement(s, contains) + if !ok { + return Fail(t, fmt.Sprintf("\"%s\" could not be applied builtin len()", s), msgAndArgs...) + } + if found { + return Fail(t, fmt.Sprintf("\"%s\" should not contain \"%s\"", s, contains), msgAndArgs...) + } + + return true + +} + +// Subset asserts that the specified list(array, slice...) contains all +// elements given in the specified subset(array, slice...). +// +// assert.Subset(t, [1, 2, 3], [1, 2], "But [1, 2, 3] does contain [1, 2]") +// +// Returns whether the assertion was successful (true) or not (false). +func Subset(t TestingT, list, subset interface{}, msgAndArgs ...interface{}) (ok bool) { + if subset == nil { + return true // we consider nil to be equal to the nil set + } + + subsetValue := reflect.ValueOf(subset) + defer func() { + if e := recover(); e != nil { + ok = false + } + }() + + listKind := reflect.TypeOf(list).Kind() + subsetKind := reflect.TypeOf(subset).Kind() + + if listKind != reflect.Array && listKind != reflect.Slice { + return Fail(t, fmt.Sprintf("%q has an unsupported type %s", list, listKind), msgAndArgs...) + } + + if subsetKind != reflect.Array && subsetKind != reflect.Slice { + return Fail(t, fmt.Sprintf("%q has an unsupported type %s", subset, subsetKind), msgAndArgs...) + } + + for i := 0; i < subsetValue.Len(); i++ { + element := subsetValue.Index(i).Interface() + ok, found := includeElement(list, element) + if !ok { + return Fail(t, fmt.Sprintf("\"%s\" could not be applied builtin len()", list), msgAndArgs...) + } + if !found { + return Fail(t, fmt.Sprintf("\"%s\" does not contain \"%s\"", list, element), msgAndArgs...) + } + } + + return true +} + +// NotSubset asserts that the specified list(array, slice...) contains not all +// elements given in the specified subset(array, slice...). +// +// assert.NotSubset(t, [1, 3, 4], [1, 2], "But [1, 3, 4] does not contain [1, 2]") +// +// Returns whether the assertion was successful (true) or not (false). +func NotSubset(t TestingT, list, subset interface{}, msgAndArgs ...interface{}) (ok bool) { + if subset == nil { + return false // we consider nil to be equal to the nil set + } + + subsetValue := reflect.ValueOf(subset) + defer func() { + if e := recover(); e != nil { + ok = false + } + }() + + listKind := reflect.TypeOf(list).Kind() + subsetKind := reflect.TypeOf(subset).Kind() + + if listKind != reflect.Array && listKind != reflect.Slice { + return Fail(t, fmt.Sprintf("%q has an unsupported type %s", list, listKind), msgAndArgs...) + } + + if subsetKind != reflect.Array && subsetKind != reflect.Slice { + return Fail(t, fmt.Sprintf("%q has an unsupported type %s", subset, subsetKind), msgAndArgs...) + } + + for i := 0; i < subsetValue.Len(); i++ { + element := subsetValue.Index(i).Interface() + ok, found := includeElement(list, element) + if !ok { + return Fail(t, fmt.Sprintf("\"%s\" could not be applied builtin len()", list), msgAndArgs...) + } + if !found { + return true + } + } + + return Fail(t, fmt.Sprintf("%q is a subset of %q", subset, list), msgAndArgs...) +} + +// Condition uses a Comparison to assert a complex condition. +func Condition(t TestingT, comp Comparison, msgAndArgs ...interface{}) bool { + result := comp() + if !result { + Fail(t, "Condition failed!", msgAndArgs...) + } + return result +} + +// PanicTestFunc defines a func that should be passed to the assert.Panics and assert.NotPanics +// methods, and represents a simple func that takes no arguments, and returns nothing. +type PanicTestFunc func() + +// didPanic returns true if the function passed to it panics. Otherwise, it returns false. +func didPanic(f PanicTestFunc) (bool, interface{}) { + + didPanic := false + var message interface{} + func() { + + defer func() { + if message = recover(); message != nil { + didPanic = true + } + }() + + // call the target function + f() + + }() + + return didPanic, message + +} + +// Panics asserts that the code inside the specified PanicTestFunc panics. +// +// assert.Panics(t, func(){ GoCrazy() }) +// +// Returns whether the assertion was successful (true) or not (false). +func Panics(t TestingT, f PanicTestFunc, msgAndArgs ...interface{}) bool { + + if funcDidPanic, panicValue := didPanic(f); !funcDidPanic { + return Fail(t, fmt.Sprintf("func %#v should panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...) + } + + return true +} + +// PanicsWithValue asserts that the code inside the specified PanicTestFunc panics, and that +// the recovered panic value equals the expected panic value. +// +// assert.PanicsWithValue(t, "crazy error", func(){ GoCrazy() }) +// +// Returns whether the assertion was successful (true) or not (false). +func PanicsWithValue(t TestingT, expected interface{}, f PanicTestFunc, msgAndArgs ...interface{}) bool { + + funcDidPanic, panicValue := didPanic(f) + if !funcDidPanic { + return Fail(t, fmt.Sprintf("func %#v should panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...) + } + if panicValue != expected { + return Fail(t, fmt.Sprintf("func %#v should panic with value:\t%v\n\r\tPanic value:\t%v", f, expected, panicValue), msgAndArgs...) + } + + return true +} + +// NotPanics asserts that the code inside the specified PanicTestFunc does NOT panic. +// +// assert.NotPanics(t, func(){ RemainCalm() }) +// +// Returns whether the assertion was successful (true) or not (false). +func NotPanics(t TestingT, f PanicTestFunc, msgAndArgs ...interface{}) bool { + + if funcDidPanic, panicValue := didPanic(f); funcDidPanic { + return Fail(t, fmt.Sprintf("func %#v should not panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...) + } + + return true +} + +// WithinDuration asserts that the two times are within duration delta of each other. +// +// assert.WithinDuration(t, time.Now(), time.Now(), 10*time.Second) +// +// Returns whether the assertion was successful (true) or not (false). +func WithinDuration(t TestingT, expected, actual time.Time, delta time.Duration, msgAndArgs ...interface{}) bool { + + dt := expected.Sub(actual) + if dt < -delta || dt > delta { + return Fail(t, fmt.Sprintf("Max difference between %v and %v allowed is %v, but difference was %v", expected, actual, delta, dt), msgAndArgs...) + } + + return true +} + +func toFloat(x interface{}) (float64, bool) { + var xf float64 + xok := true + + switch xn := x.(type) { + case uint8: + xf = float64(xn) + case uint16: + xf = float64(xn) + case uint32: + xf = float64(xn) + case uint64: + xf = float64(xn) + case int: + xf = float64(xn) + case int8: + xf = float64(xn) + case int16: + xf = float64(xn) + case int32: + xf = float64(xn) + case int64: + xf = float64(xn) + case float32: + xf = float64(xn) + case float64: + xf = float64(xn) + default: + xok = false + } + + return xf, xok +} + +// InDelta asserts that the two numerals are within delta of each other. +// +// assert.InDelta(t, math.Pi, (22 / 7.0), 0.01) +// +// Returns whether the assertion was successful (true) or not (false). +func InDelta(t TestingT, expected, actual interface{}, delta float64, msgAndArgs ...interface{}) bool { + + af, aok := toFloat(expected) + bf, bok := toFloat(actual) + + if !aok || !bok { + return Fail(t, fmt.Sprintf("Parameters must be numerical"), msgAndArgs...) + } + + if math.IsNaN(af) { + return Fail(t, fmt.Sprintf("Actual must not be NaN"), msgAndArgs...) + } + + if math.IsNaN(bf) { + return Fail(t, fmt.Sprintf("Expected %v with delta %v, but was NaN", expected, delta), msgAndArgs...) + } + + dt := af - bf + if dt < -delta || dt > delta { + return Fail(t, fmt.Sprintf("Max difference between %v and %v allowed is %v, but difference was %v", expected, actual, delta, dt), msgAndArgs...) + } + + return true +} + +// InDeltaSlice is the same as InDelta, except it compares two slices. +func InDeltaSlice(t TestingT, expected, actual interface{}, delta float64, msgAndArgs ...interface{}) bool { + if expected == nil || actual == nil || + reflect.TypeOf(actual).Kind() != reflect.Slice || + reflect.TypeOf(expected).Kind() != reflect.Slice { + return Fail(t, fmt.Sprintf("Parameters must be slice"), msgAndArgs...) + } + + actualSlice := reflect.ValueOf(actual) + expectedSlice := reflect.ValueOf(expected) + + for i := 0; i < actualSlice.Len(); i++ { + result := InDelta(t, actualSlice.Index(i).Interface(), expectedSlice.Index(i).Interface(), delta, msgAndArgs...) + if !result { + return result + } + } + + return true +} + +func calcRelativeError(expected, actual interface{}) (float64, error) { + af, aok := toFloat(expected) + if !aok { + return 0, fmt.Errorf("expected value %q cannot be converted to float", expected) + } + if af == 0 { + return 0, fmt.Errorf("expected value must have a value other than zero to calculate the relative error") + } + bf, bok := toFloat(actual) + if !bok { + return 0, fmt.Errorf("expected value %q cannot be converted to float", actual) + } + + return math.Abs(af-bf) / math.Abs(af), nil +} + +// InEpsilon asserts that expected and actual have a relative error less than epsilon +// +// Returns whether the assertion was successful (true) or not (false). +func InEpsilon(t TestingT, expected, actual interface{}, epsilon float64, msgAndArgs ...interface{}) bool { + actualEpsilon, err := calcRelativeError(expected, actual) + if err != nil { + return Fail(t, err.Error(), msgAndArgs...) + } + if actualEpsilon > epsilon { + return Fail(t, fmt.Sprintf("Relative error is too high: %#v (expected)\n"+ + " < %#v (actual)", actualEpsilon, epsilon), msgAndArgs...) + } + + return true +} + +// InEpsilonSlice is the same as InEpsilon, except it compares each value from two slices. +func InEpsilonSlice(t TestingT, expected, actual interface{}, epsilon float64, msgAndArgs ...interface{}) bool { + if expected == nil || actual == nil || + reflect.TypeOf(actual).Kind() != reflect.Slice || + reflect.TypeOf(expected).Kind() != reflect.Slice { + return Fail(t, fmt.Sprintf("Parameters must be slice"), msgAndArgs...) + } + + actualSlice := reflect.ValueOf(actual) + expectedSlice := reflect.ValueOf(expected) + + for i := 0; i < actualSlice.Len(); i++ { + result := InEpsilon(t, actualSlice.Index(i).Interface(), expectedSlice.Index(i).Interface(), epsilon) + if !result { + return result + } + } + + return true +} + +/* + Errors +*/ + +// NoError asserts that a function returned no error (i.e. `nil`). +// +// actualObj, err := SomeFunction() +// if assert.NoError(t, err) { +// assert.Equal(t, expectedObj, actualObj) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func NoError(t TestingT, err error, msgAndArgs ...interface{}) bool { + if err != nil { + return Fail(t, fmt.Sprintf("Received unexpected error:\n%+v", err), msgAndArgs...) + } + + return true +} + +// Error asserts that a function returned an error (i.e. not `nil`). +// +// actualObj, err := SomeFunction() +// if assert.Error(t, err) { +// assert.Equal(t, expectedError, err) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func Error(t TestingT, err error, msgAndArgs ...interface{}) bool { + + if err == nil { + return Fail(t, "An error is expected but got nil.", msgAndArgs...) + } + + return true +} + +// EqualError asserts that a function returned an error (i.e. not `nil`) +// and that it is equal to the provided error. +// +// actualObj, err := SomeFunction() +// assert.EqualError(t, err, expectedErrorString) +// +// Returns whether the assertion was successful (true) or not (false). +func EqualError(t TestingT, theError error, errString string, msgAndArgs ...interface{}) bool { + if !Error(t, theError, msgAndArgs...) { + return false + } + expected := errString + actual := theError.Error() + // don't need to use deep equals here, we know they are both strings + if expected != actual { + return Fail(t, fmt.Sprintf("Error message not equal:\n"+ + "expected: %q\n"+ + "actual: %q", expected, actual), msgAndArgs...) + } + return true +} + +// matchRegexp return true if a specified regexp matches a string. +func matchRegexp(rx interface{}, str interface{}) bool { + + var r *regexp.Regexp + if rr, ok := rx.(*regexp.Regexp); ok { + r = rr + } else { + r = regexp.MustCompile(fmt.Sprint(rx)) + } + + return (r.FindStringIndex(fmt.Sprint(str)) != nil) + +} + +// Regexp asserts that a specified regexp matches a string. +// +// assert.Regexp(t, regexp.MustCompile("start"), "it's starting") +// assert.Regexp(t, "start...$", "it's not starting") +// +// Returns whether the assertion was successful (true) or not (false). +func Regexp(t TestingT, rx interface{}, str interface{}, msgAndArgs ...interface{}) bool { + + match := matchRegexp(rx, str) + + if !match { + Fail(t, fmt.Sprintf("Expect \"%v\" to match \"%v\"", str, rx), msgAndArgs...) + } + + return match +} + +// NotRegexp asserts that a specified regexp does not match a string. +// +// assert.NotRegexp(t, regexp.MustCompile("starts"), "it's starting") +// assert.NotRegexp(t, "^start", "it's not starting") +// +// Returns whether the assertion was successful (true) or not (false). +func NotRegexp(t TestingT, rx interface{}, str interface{}, msgAndArgs ...interface{}) bool { + match := matchRegexp(rx, str) + + if match { + Fail(t, fmt.Sprintf("Expect \"%v\" to NOT match \"%v\"", str, rx), msgAndArgs...) + } + + return !match + +} + +// Zero asserts that i is the zero value for its type and returns the truth. +func Zero(t TestingT, i interface{}, msgAndArgs ...interface{}) bool { + if i != nil && !reflect.DeepEqual(i, reflect.Zero(reflect.TypeOf(i)).Interface()) { + return Fail(t, fmt.Sprintf("Should be zero, but was %v", i), msgAndArgs...) + } + return true +} + +// NotZero asserts that i is not the zero value for its type and returns the truth. +func NotZero(t TestingT, i interface{}, msgAndArgs ...interface{}) bool { + if i == nil || reflect.DeepEqual(i, reflect.Zero(reflect.TypeOf(i)).Interface()) { + return Fail(t, fmt.Sprintf("Should not be zero, but was %v", i), msgAndArgs...) + } + return true +} + +// JSONEq asserts that two JSON strings are equivalent. +// +// assert.JSONEq(t, `{"hello": "world", "foo": "bar"}`, `{"foo": "bar", "hello": "world"}`) +// +// Returns whether the assertion was successful (true) or not (false). +func JSONEq(t TestingT, expected string, actual string, msgAndArgs ...interface{}) bool { + var expectedJSONAsInterface, actualJSONAsInterface interface{} + + if err := json.Unmarshal([]byte(expected), &expectedJSONAsInterface); err != nil { + return Fail(t, fmt.Sprintf("Expected value ('%s') is not valid json.\nJSON parsing error: '%s'", expected, err.Error()), msgAndArgs...) + } + + if err := json.Unmarshal([]byte(actual), &actualJSONAsInterface); err != nil { + return Fail(t, fmt.Sprintf("Input ('%s') needs to be valid json.\nJSON parsing error: '%s'", actual, err.Error()), msgAndArgs...) + } + + return Equal(t, expectedJSONAsInterface, actualJSONAsInterface, msgAndArgs...) +} + +func typeAndKind(v interface{}) (reflect.Type, reflect.Kind) { + t := reflect.TypeOf(v) + k := t.Kind() + + if k == reflect.Ptr { + t = t.Elem() + k = t.Kind() + } + return t, k +} + +// diff returns a diff of both values as long as both are of the same type and +// are a struct, map, slice or array. Otherwise it returns an empty string. +func diff(expected interface{}, actual interface{}) string { + if expected == nil || actual == nil { + return "" + } + + et, ek := typeAndKind(expected) + at, _ := typeAndKind(actual) + + if et != at { + return "" + } + + if ek != reflect.Struct && ek != reflect.Map && ek != reflect.Slice && ek != reflect.Array { + return "" + } + + e := spewConfig.Sdump(expected) + a := spewConfig.Sdump(actual) + + diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{ + A: difflib.SplitLines(e), + B: difflib.SplitLines(a), + FromFile: "Expected", + FromDate: "", + ToFile: "Actual", + ToDate: "", + Context: 1, + }) + + return "\n\nDiff:\n" + diff +} + +// validateEqualArgs checks whether provided arguments can be safely used in the +// Equal/NotEqual functions. +func validateEqualArgs(expected, actual interface{}) error { + if isFunction(expected) || isFunction(actual) { + return errors.New("cannot take func type as argument") + } + return nil +} + +func isFunction(arg interface{}) bool { + if arg == nil { + return false + } + return reflect.TypeOf(arg).Kind() == reflect.Func +} + +var spewConfig = spew.ConfigState{ + Indent: " ", + DisablePointerAddresses: true, + DisableCapacities: true, + SortKeys: true, +} diff --git a/vendor/github.com/stretchr/testify/assert/doc.go b/vendor/github.com/stretchr/testify/assert/doc.go new file mode 100644 index 000000000000..c9dccc4d6cd0 --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/doc.go @@ -0,0 +1,45 @@ +// Package assert provides a set of comprehensive testing tools for use with the normal Go testing system. +// +// Example Usage +// +// The following is a complete example using assert in a standard test function: +// import ( +// "testing" +// "github.com/stretchr/testify/assert" +// ) +// +// func TestSomething(t *testing.T) { +// +// var a string = "Hello" +// var b string = "Hello" +// +// assert.Equal(t, a, b, "The two words should be the same.") +// +// } +// +// if you assert many times, use the format below: +// +// import ( +// "testing" +// "github.com/stretchr/testify/assert" +// ) +// +// func TestSomething(t *testing.T) { +// assert := assert.New(t) +// +// var a string = "Hello" +// var b string = "Hello" +// +// assert.Equal(a, b, "The two words should be the same.") +// } +// +// Assertions +// +// Assertions allow you to easily write test code, and are global funcs in the `assert` package. +// All assertion functions take, as the first argument, the `*testing.T` object provided by the +// testing framework. This allows the assertion funcs to write the failings and other details to +// the correct place. +// +// Every assertion function also takes an optional string message as the final argument, +// allowing custom error messages to be appended to the message the assertion method outputs. +package assert diff --git a/vendor/github.com/stretchr/testify/assert/errors.go b/vendor/github.com/stretchr/testify/assert/errors.go new file mode 100644 index 000000000000..ac9dc9d1d615 --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/errors.go @@ -0,0 +1,10 @@ +package assert + +import ( + "errors" +) + +// AnError is an error instance useful for testing. If the code does not care +// about error specifics, and only needs to return the error for example, this +// error should be used to make the test code more readable. +var AnError = errors.New("assert.AnError general error for testing") diff --git a/vendor/github.com/stretchr/testify/assert/forward_assertions.go b/vendor/github.com/stretchr/testify/assert/forward_assertions.go new file mode 100644 index 000000000000..9ad56851d971 --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/forward_assertions.go @@ -0,0 +1,16 @@ +package assert + +// Assertions provides assertion methods around the +// TestingT interface. +type Assertions struct { + t TestingT +} + +// New makes a new Assertions object for the specified TestingT. +func New(t TestingT) *Assertions { + return &Assertions{ + t: t, + } +} + +//go:generate go run ../_codegen/main.go -output-package=assert -template=assertion_forward.go.tmpl -include-format-funcs diff --git a/vendor/github.com/stretchr/testify/assert/http_assertions.go b/vendor/github.com/stretchr/testify/assert/http_assertions.go new file mode 100644 index 000000000000..ba811c04dd5f --- /dev/null +++ b/vendor/github.com/stretchr/testify/assert/http_assertions.go @@ -0,0 +1,127 @@ +package assert + +import ( + "fmt" + "net/http" + "net/http/httptest" + "net/url" + "strings" +) + +// httpCode is a helper that returns HTTP code of the response. It returns -1 and +// an error if building a new request fails. +func httpCode(handler http.HandlerFunc, method, url string, values url.Values) (int, error) { + w := httptest.NewRecorder() + req, err := http.NewRequest(method, url+"?"+values.Encode(), nil) + if err != nil { + return -1, err + } + handler(w, req) + return w.Code, nil +} + +// HTTPSuccess asserts that a specified handler returns a success status code. +// +// assert.HTTPSuccess(t, myHandler, "POST", "http://www.google.com", nil) +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPSuccess(t TestingT, handler http.HandlerFunc, method, url string, values url.Values) bool { + code, err := httpCode(handler, method, url, values) + if err != nil { + Fail(t, fmt.Sprintf("Failed to build test request, got error: %s", err)) + return false + } + + isSuccessCode := code >= http.StatusOK && code <= http.StatusPartialContent + if !isSuccessCode { + Fail(t, fmt.Sprintf("Expected HTTP success status code for %q but received %d", url+"?"+values.Encode(), code)) + } + + return isSuccessCode +} + +// HTTPRedirect asserts that a specified handler returns a redirect status code. +// +// assert.HTTPRedirect(t, myHandler, "GET", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPRedirect(t TestingT, handler http.HandlerFunc, method, url string, values url.Values) bool { + code, err := httpCode(handler, method, url, values) + if err != nil { + Fail(t, fmt.Sprintf("Failed to build test request, got error: %s", err)) + return false + } + + isRedirectCode := code >= http.StatusMultipleChoices && code <= http.StatusTemporaryRedirect + if !isRedirectCode { + Fail(t, fmt.Sprintf("Expected HTTP redirect status code for %q but received %d", url+"?"+values.Encode(), code)) + } + + return isRedirectCode +} + +// HTTPError asserts that a specified handler returns an error status code. +// +// assert.HTTPError(t, myHandler, "POST", "/a/b/c", url.Values{"a": []string{"b", "c"}} +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPError(t TestingT, handler http.HandlerFunc, method, url string, values url.Values) bool { + code, err := httpCode(handler, method, url, values) + if err != nil { + Fail(t, fmt.Sprintf("Failed to build test request, got error: %s", err)) + return false + } + + isErrorCode := code >= http.StatusBadRequest + if !isErrorCode { + Fail(t, fmt.Sprintf("Expected HTTP error status code for %q but received %d", url+"?"+values.Encode(), code)) + } + + return isErrorCode +} + +// HTTPBody is a helper that returns HTTP body of the response. It returns +// empty string if building a new request fails. +func HTTPBody(handler http.HandlerFunc, method, url string, values url.Values) string { + w := httptest.NewRecorder() + req, err := http.NewRequest(method, url+"?"+values.Encode(), nil) + if err != nil { + return "" + } + handler(w, req) + return w.Body.String() +} + +// HTTPBodyContains asserts that a specified handler returns a +// body that contains a string. +// +// assert.HTTPBodyContains(t, myHandler, "www.google.com", nil, "I'm Feeling Lucky") +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPBodyContains(t TestingT, handler http.HandlerFunc, method, url string, values url.Values, str interface{}) bool { + body := HTTPBody(handler, method, url, values) + + contains := strings.Contains(body, fmt.Sprint(str)) + if !contains { + Fail(t, fmt.Sprintf("Expected response body for \"%s\" to contain \"%s\" but found \"%s\"", url+"?"+values.Encode(), str, body)) + } + + return contains +} + +// HTTPBodyNotContains asserts that a specified handler returns a +// body that does not contain a string. +// +// assert.HTTPBodyNotContains(t, myHandler, "www.google.com", nil, "I'm Feeling Lucky") +// +// Returns whether the assertion was successful (true) or not (false). +func HTTPBodyNotContains(t TestingT, handler http.HandlerFunc, method, url string, values url.Values, str interface{}) bool { + body := HTTPBody(handler, method, url, values) + + contains := strings.Contains(body, fmt.Sprint(str)) + if contains { + Fail(t, fmt.Sprintf("Expected response body for \"%s\" to NOT contain \"%s\" but found \"%s\"", url+"?"+values.Encode(), str, body)) + } + + return !contains +} diff --git a/vendor/github.com/stretchr/testify/mock/doc.go b/vendor/github.com/stretchr/testify/mock/doc.go new file mode 100644 index 000000000000..7324128ef19c --- /dev/null +++ b/vendor/github.com/stretchr/testify/mock/doc.go @@ -0,0 +1,44 @@ +// Package mock provides a system by which it is possible to mock your objects +// and verify calls are happening as expected. +// +// Example Usage +// +// The mock package provides an object, Mock, that tracks activity on another object. It is usually +// embedded into a test object as shown below: +// +// type MyTestObject struct { +// // add a Mock object instance +// mock.Mock +// +// // other fields go here as normal +// } +// +// When implementing the methods of an interface, you wire your functions up +// to call the Mock.Called(args...) method, and return the appropriate values. +// +// For example, to mock a method that saves the name and age of a person and returns +// the year of their birth or an error, you might write this: +// +// func (o *MyTestObject) SavePersonDetails(firstname, lastname string, age int) (int, error) { +// args := o.Called(firstname, lastname, age) +// return args.Int(0), args.Error(1) +// } +// +// The Int, Error and Bool methods are examples of strongly typed getters that take the argument +// index position. Given this argument list: +// +// (12, true, "Something") +// +// You could read them out strongly typed like this: +// +// args.Int(0) +// args.Bool(1) +// args.String(2) +// +// For objects of your own type, use the generic Arguments.Get(index) method and make a type assertion: +// +// return args.Get(0).(*MyObject), args.Get(1).(*AnotherObjectOfMine) +// +// This may cause a panic if the object you are getting is nil (the type assertion will fail), in those +// cases you should check for nil first. +package mock diff --git a/vendor/github.com/stretchr/testify/mock/mock.go b/vendor/github.com/stretchr/testify/mock/mock.go new file mode 100644 index 000000000000..fc63571d4e8b --- /dev/null +++ b/vendor/github.com/stretchr/testify/mock/mock.go @@ -0,0 +1,776 @@ +package mock + +import ( + "fmt" + "reflect" + "regexp" + "runtime" + "strings" + "sync" + "time" + + "github.com/davecgh/go-spew/spew" + "github.com/pmezard/go-difflib/difflib" + "github.com/stretchr/objx" + "github.com/stretchr/testify/assert" +) + +// TestingT is an interface wrapper around *testing.T +type TestingT interface { + Logf(format string, args ...interface{}) + Errorf(format string, args ...interface{}) + FailNow() +} + +/* + Call +*/ + +// Call represents a method call and is used for setting expectations, +// as well as recording activity. +type Call struct { + Parent *Mock + + // The name of the method that was or will be called. + Method string + + // Holds the arguments of the method. + Arguments Arguments + + // Holds the arguments that should be returned when + // this method is called. + ReturnArguments Arguments + + // The number of times to return the return arguments when setting + // expectations. 0 means to always return the value. + Repeatability int + + // Amount of times this call has been called + totalCalls int + + // Holds a channel that will be used to block the Return until it either + // receives a message or is closed. nil means it returns immediately. + WaitFor <-chan time.Time + + // Holds a handler used to manipulate arguments content that are passed by + // reference. It's useful when mocking methods such as unmarshalers or + // decoders. + RunFn func(Arguments) +} + +func newCall(parent *Mock, methodName string, methodArguments ...interface{}) *Call { + return &Call{ + Parent: parent, + Method: methodName, + Arguments: methodArguments, + ReturnArguments: make([]interface{}, 0), + Repeatability: 0, + WaitFor: nil, + RunFn: nil, + } +} + +func (c *Call) lock() { + c.Parent.mutex.Lock() +} + +func (c *Call) unlock() { + c.Parent.mutex.Unlock() +} + +// Return specifies the return arguments for the expectation. +// +// Mock.On("DoSomething").Return(errors.New("failed")) +func (c *Call) Return(returnArguments ...interface{}) *Call { + c.lock() + defer c.unlock() + + c.ReturnArguments = returnArguments + + return c +} + +// Once indicates that that the mock should only return the value once. +// +// Mock.On("MyMethod", arg1, arg2).Return(returnArg1, returnArg2).Once() +func (c *Call) Once() *Call { + return c.Times(1) +} + +// Twice indicates that that the mock should only return the value twice. +// +// Mock.On("MyMethod", arg1, arg2).Return(returnArg1, returnArg2).Twice() +func (c *Call) Twice() *Call { + return c.Times(2) +} + +// Times indicates that that the mock should only return the indicated number +// of times. +// +// Mock.On("MyMethod", arg1, arg2).Return(returnArg1, returnArg2).Times(5) +func (c *Call) Times(i int) *Call { + c.lock() + defer c.unlock() + c.Repeatability = i + return c +} + +// WaitUntil sets the channel that will block the mock's return until its closed +// or a message is received. +// +// Mock.On("MyMethod", arg1, arg2).WaitUntil(time.After(time.Second)) +func (c *Call) WaitUntil(w <-chan time.Time) *Call { + c.lock() + defer c.unlock() + c.WaitFor = w + return c +} + +// After sets how long to block until the call returns +// +// Mock.On("MyMethod", arg1, arg2).After(time.Second) +func (c *Call) After(d time.Duration) *Call { + return c.WaitUntil(time.After(d)) +} + +// Run sets a handler to be called before returning. It can be used when +// mocking a method such as unmarshalers that takes a pointer to a struct and +// sets properties in such struct +// +// Mock.On("Unmarshal", AnythingOfType("*map[string]interface{}").Return().Run(func(args Arguments) { +// arg := args.Get(0).(*map[string]interface{}) +// arg["foo"] = "bar" +// }) +func (c *Call) Run(fn func(args Arguments)) *Call { + c.lock() + defer c.unlock() + c.RunFn = fn + return c +} + +// On chains a new expectation description onto the mocked interface. This +// allows syntax like. +// +// Mock. +// On("MyMethod", 1).Return(nil). +// On("MyOtherMethod", 'a', 'b', 'c').Return(errors.New("Some Error")) +func (c *Call) On(methodName string, arguments ...interface{}) *Call { + return c.Parent.On(methodName, arguments...) +} + +// Mock is the workhorse used to track activity on another object. +// For an example of its usage, refer to the "Example Usage" section at the top +// of this document. +type Mock struct { + // Represents the calls that are expected of + // an object. + ExpectedCalls []*Call + + // Holds the calls that were made to this mocked object. + Calls []Call + + // TestData holds any data that might be useful for testing. Testify ignores + // this data completely allowing you to do whatever you like with it. + testData objx.Map + + mutex sync.Mutex +} + +// TestData holds any data that might be useful for testing. Testify ignores +// this data completely allowing you to do whatever you like with it. +func (m *Mock) TestData() objx.Map { + + if m.testData == nil { + m.testData = make(objx.Map) + } + + return m.testData +} + +/* + Setting expectations +*/ + +// On starts a description of an expectation of the specified method +// being called. +// +// Mock.On("MyMethod", arg1, arg2) +func (m *Mock) On(methodName string, arguments ...interface{}) *Call { + for _, arg := range arguments { + if v := reflect.ValueOf(arg); v.Kind() == reflect.Func { + panic(fmt.Sprintf("cannot use Func in expectations. Use mock.AnythingOfType(\"%T\")", arg)) + } + } + + m.mutex.Lock() + defer m.mutex.Unlock() + c := newCall(m, methodName, arguments...) + m.ExpectedCalls = append(m.ExpectedCalls, c) + return c +} + +// /* +// Recording and responding to activity +// */ + +func (m *Mock) findExpectedCall(method string, arguments ...interface{}) (int, *Call) { + for i, call := range m.ExpectedCalls { + if call.Method == method && call.Repeatability > -1 { + + _, diffCount := call.Arguments.Diff(arguments) + if diffCount == 0 { + return i, call + } + + } + } + return -1, nil +} + +func (m *Mock) findClosestCall(method string, arguments ...interface{}) (bool, *Call) { + diffCount := 0 + var closestCall *Call + + for _, call := range m.expectedCalls() { + if call.Method == method { + + _, tempDiffCount := call.Arguments.Diff(arguments) + if tempDiffCount < diffCount || diffCount == 0 { + diffCount = tempDiffCount + closestCall = call + } + + } + } + + if closestCall == nil { + return false, nil + } + + return true, closestCall +} + +func callString(method string, arguments Arguments, includeArgumentValues bool) string { + + var argValsString string + if includeArgumentValues { + var argVals []string + for argIndex, arg := range arguments { + argVals = append(argVals, fmt.Sprintf("%d: %#v", argIndex, arg)) + } + argValsString = fmt.Sprintf("\n\t\t%s", strings.Join(argVals, "\n\t\t")) + } + + return fmt.Sprintf("%s(%s)%s", method, arguments.String(), argValsString) +} + +// Called tells the mock object that a method has been called, and gets an array +// of arguments to return. Panics if the call is unexpected (i.e. not preceded by +// appropriate .On .Return() calls) +// If Call.WaitFor is set, blocks until the channel is closed or receives a message. +func (m *Mock) Called(arguments ...interface{}) Arguments { + // get the calling function's name + pc, _, _, ok := runtime.Caller(1) + if !ok { + panic("Couldn't get the caller information") + } + functionPath := runtime.FuncForPC(pc).Name() + //Next four lines are required to use GCCGO function naming conventions. + //For Ex: github_com_docker_libkv_store_mock.WatchTree.pN39_github_com_docker_libkv_store_mock.Mock + //uses interface information unlike golang github.com/docker/libkv/store/mock.(*Mock).WatchTree + //With GCCGO we need to remove interface information starting from pN

. + re := regexp.MustCompile("\\.pN\\d+_") + if re.MatchString(functionPath) { + functionPath = re.Split(functionPath, -1)[0] + } + parts := strings.Split(functionPath, ".") + functionName := parts[len(parts)-1] + return m.MethodCalled(functionName, arguments...) +} + +// MethodCalled tells the mock object that the given method has been called, and gets +// an array of arguments to return. Panics if the call is unexpected (i.e. not preceded +// by appropriate .On .Return() calls) +// If Call.WaitFor is set, blocks until the channel is closed or receives a message. +func (m *Mock) MethodCalled(methodName string, arguments ...interface{}) Arguments { + m.mutex.Lock() + found, call := m.findExpectedCall(methodName, arguments...) + + if found < 0 { + // we have to fail here - because we don't know what to do + // as the return arguments. This is because: + // + // a) this is a totally unexpected call to this method, + // b) the arguments are not what was expected, or + // c) the developer has forgotten to add an accompanying On...Return pair. + + closestFound, closestCall := m.findClosestCall(methodName, arguments...) + m.mutex.Unlock() + + if closestFound { + panic(fmt.Sprintf("\n\nmock: Unexpected Method Call\n-----------------------------\n\n%s\n\nThe closest call I have is: \n\n%s\n\n%s\n", callString(methodName, arguments, true), callString(methodName, closestCall.Arguments, true), diffArguments(arguments, closestCall.Arguments))) + } else { + panic(fmt.Sprintf("\nassert: mock: I don't know what to return because the method call was unexpected.\n\tEither do Mock.On(\"%s\").Return(...) first, or remove the %s() call.\n\tThis method was unexpected:\n\t\t%s\n\tat: %s", methodName, methodName, callString(methodName, arguments, true), assert.CallerInfo())) + } + } + + switch { + case call.Repeatability == 1: + call.Repeatability = -1 + call.totalCalls++ + + case call.Repeatability > 1: + call.Repeatability-- + call.totalCalls++ + + case call.Repeatability == 0: + call.totalCalls++ + } + + // add the call + m.Calls = append(m.Calls, *newCall(m, methodName, arguments...)) + m.mutex.Unlock() + + // block if specified + if call.WaitFor != nil { + <-call.WaitFor + } + + if call.RunFn != nil { + call.RunFn(arguments) + } + + return call.ReturnArguments +} + +/* + Assertions +*/ + +type assertExpectationser interface { + AssertExpectations(TestingT) bool +} + +// AssertExpectationsForObjects asserts that everything specified with On and Return +// of the specified objects was in fact called as expected. +// +// Calls may have occurred in any order. +func AssertExpectationsForObjects(t TestingT, testObjects ...interface{}) bool { + for _, obj := range testObjects { + if m, ok := obj.(Mock); ok { + t.Logf("Deprecated mock.AssertExpectationsForObjects(myMock.Mock) use mock.AssertExpectationsForObjects(myMock)") + obj = &m + } + m := obj.(assertExpectationser) + if !m.AssertExpectations(t) { + return false + } + } + return true +} + +// AssertExpectations asserts that everything specified with On and Return was +// in fact called as expected. Calls may have occurred in any order. +func (m *Mock) AssertExpectations(t TestingT) bool { + m.mutex.Lock() + defer m.mutex.Unlock() + var somethingMissing bool + var failedExpectations int + + // iterate through each expectation + expectedCalls := m.expectedCalls() + for _, expectedCall := range expectedCalls { + if !m.methodWasCalled(expectedCall.Method, expectedCall.Arguments) && expectedCall.totalCalls == 0 { + somethingMissing = true + failedExpectations++ + t.Logf("\u274C\t%s(%s)", expectedCall.Method, expectedCall.Arguments.String()) + } else { + if expectedCall.Repeatability > 0 { + somethingMissing = true + failedExpectations++ + } else { + t.Logf("\u2705\t%s(%s)", expectedCall.Method, expectedCall.Arguments.String()) + } + } + } + + if somethingMissing { + t.Errorf("FAIL: %d out of %d expectation(s) were met.\n\tThe code you are testing needs to make %d more call(s).\n\tat: %s", len(expectedCalls)-failedExpectations, len(expectedCalls), failedExpectations, assert.CallerInfo()) + } + + return !somethingMissing +} + +// AssertNumberOfCalls asserts that the method was called expectedCalls times. +func (m *Mock) AssertNumberOfCalls(t TestingT, methodName string, expectedCalls int) bool { + m.mutex.Lock() + defer m.mutex.Unlock() + var actualCalls int + for _, call := range m.calls() { + if call.Method == methodName { + actualCalls++ + } + } + return assert.Equal(t, expectedCalls, actualCalls, fmt.Sprintf("Expected number of calls (%d) does not match the actual number of calls (%d).", expectedCalls, actualCalls)) +} + +// AssertCalled asserts that the method was called. +// It can produce a false result when an argument is a pointer type and the underlying value changed after calling the mocked method. +func (m *Mock) AssertCalled(t TestingT, methodName string, arguments ...interface{}) bool { + m.mutex.Lock() + defer m.mutex.Unlock() + if !assert.True(t, m.methodWasCalled(methodName, arguments), fmt.Sprintf("The \"%s\" method should have been called with %d argument(s), but was not.", methodName, len(arguments))) { + t.Logf("%v", m.expectedCalls()) + return false + } + return true +} + +// AssertNotCalled asserts that the method was not called. +// It can produce a false result when an argument is a pointer type and the underlying value changed after calling the mocked method. +func (m *Mock) AssertNotCalled(t TestingT, methodName string, arguments ...interface{}) bool { + m.mutex.Lock() + defer m.mutex.Unlock() + if !assert.False(t, m.methodWasCalled(methodName, arguments), fmt.Sprintf("The \"%s\" method was called with %d argument(s), but should NOT have been.", methodName, len(arguments))) { + t.Logf("%v", m.expectedCalls()) + return false + } + return true +} + +func (m *Mock) methodWasCalled(methodName string, expected []interface{}) bool { + for _, call := range m.calls() { + if call.Method == methodName { + + _, differences := Arguments(expected).Diff(call.Arguments) + + if differences == 0 { + // found the expected call + return true + } + + } + } + // we didn't find the expected call + return false +} + +func (m *Mock) expectedCalls() []*Call { + return append([]*Call{}, m.ExpectedCalls...) +} + +func (m *Mock) calls() []Call { + return append([]Call{}, m.Calls...) +} + +/* + Arguments +*/ + +// Arguments holds an array of method arguments or return values. +type Arguments []interface{} + +const ( + // Anything is used in Diff and Assert when the argument being tested + // shouldn't be taken into consideration. + Anything string = "mock.Anything" +) + +// AnythingOfTypeArgument is a string that contains the type of an argument +// for use when type checking. Used in Diff and Assert. +type AnythingOfTypeArgument string + +// AnythingOfType returns an AnythingOfTypeArgument object containing the +// name of the type to check for. Used in Diff and Assert. +// +// For example: +// Assert(t, AnythingOfType("string"), AnythingOfType("int")) +func AnythingOfType(t string) AnythingOfTypeArgument { + return AnythingOfTypeArgument(t) +} + +// argumentMatcher performs custom argument matching, returning whether or +// not the argument is matched by the expectation fixture function. +type argumentMatcher struct { + // fn is a function which accepts one argument, and returns a bool. + fn reflect.Value +} + +func (f argumentMatcher) Matches(argument interface{}) bool { + expectType := f.fn.Type().In(0) + + if reflect.TypeOf(argument).AssignableTo(expectType) { + result := f.fn.Call([]reflect.Value{reflect.ValueOf(argument)}) + return result[0].Bool() + } + return false +} + +func (f argumentMatcher) String() string { + return fmt.Sprintf("func(%s) bool", f.fn.Type().In(0).Name()) +} + +// MatchedBy can be used to match a mock call based on only certain properties +// from a complex struct or some calculation. It takes a function that will be +// evaluated with the called argument and will return true when there's a match +// and false otherwise. +// +// Example: +// m.On("Do", MatchedBy(func(req *http.Request) bool { return req.Host == "example.com" })) +// +// |fn|, must be a function accepting a single argument (of the expected type) +// which returns a bool. If |fn| doesn't match the required signature, +// MatchedBy() panics. +func MatchedBy(fn interface{}) argumentMatcher { + fnType := reflect.TypeOf(fn) + + if fnType.Kind() != reflect.Func { + panic(fmt.Sprintf("assert: arguments: %s is not a func", fn)) + } + if fnType.NumIn() != 1 { + panic(fmt.Sprintf("assert: arguments: %s does not take exactly one argument", fn)) + } + if fnType.NumOut() != 1 || fnType.Out(0).Kind() != reflect.Bool { + panic(fmt.Sprintf("assert: arguments: %s does not return a bool", fn)) + } + + return argumentMatcher{fn: reflect.ValueOf(fn)} +} + +// Get Returns the argument at the specified index. +func (args Arguments) Get(index int) interface{} { + if index+1 > len(args) { + panic(fmt.Sprintf("assert: arguments: Cannot call Get(%d) because there are %d argument(s).", index, len(args))) + } + return args[index] +} + +// Is gets whether the objects match the arguments specified. +func (args Arguments) Is(objects ...interface{}) bool { + for i, obj := range args { + if obj != objects[i] { + return false + } + } + return true +} + +// Diff gets a string describing the differences between the arguments +// and the specified objects. +// +// Returns the diff string and number of differences found. +func (args Arguments) Diff(objects []interface{}) (string, int) { + + var output = "\n" + var differences int + + var maxArgCount = len(args) + if len(objects) > maxArgCount { + maxArgCount = len(objects) + } + + for i := 0; i < maxArgCount; i++ { + var actual, expected interface{} + + if len(objects) <= i { + actual = "(Missing)" + } else { + actual = objects[i] + } + + if len(args) <= i { + expected = "(Missing)" + } else { + expected = args[i] + } + + if matcher, ok := expected.(argumentMatcher); ok { + if matcher.Matches(actual) { + output = fmt.Sprintf("%s\t%d: \u2705 %s matched by %s\n", output, i, actual, matcher) + } else { + differences++ + output = fmt.Sprintf("%s\t%d: \u2705 %s not matched by %s\n", output, i, actual, matcher) + } + } else if reflect.TypeOf(expected) == reflect.TypeOf((*AnythingOfTypeArgument)(nil)).Elem() { + + // type checking + if reflect.TypeOf(actual).Name() != string(expected.(AnythingOfTypeArgument)) && reflect.TypeOf(actual).String() != string(expected.(AnythingOfTypeArgument)) { + // not match + differences++ + output = fmt.Sprintf("%s\t%d: \u274C type %s != type %s - %s\n", output, i, expected, reflect.TypeOf(actual).Name(), actual) + } + + } else { + + // normal checking + + if assert.ObjectsAreEqual(expected, Anything) || assert.ObjectsAreEqual(actual, Anything) || assert.ObjectsAreEqual(actual, expected) { + // match + output = fmt.Sprintf("%s\t%d: \u2705 %s == %s\n", output, i, actual, expected) + } else { + // not match + differences++ + output = fmt.Sprintf("%s\t%d: \u274C %s != %s\n", output, i, actual, expected) + } + } + + } + + if differences == 0 { + return "No differences.", differences + } + + return output, differences + +} + +// Assert compares the arguments with the specified objects and fails if +// they do not exactly match. +func (args Arguments) Assert(t TestingT, objects ...interface{}) bool { + + // get the differences + diff, diffCount := args.Diff(objects) + + if diffCount == 0 { + return true + } + + // there are differences... report them... + t.Logf(diff) + t.Errorf("%sArguments do not match.", assert.CallerInfo()) + + return false + +} + +// String gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +// +// If no index is provided, String() returns a complete string representation +// of the arguments. +func (args Arguments) String(indexOrNil ...int) string { + + if len(indexOrNil) == 0 { + // normal String() method - return a string representation of the args + var argsStr []string + for _, arg := range args { + argsStr = append(argsStr, fmt.Sprintf("%s", reflect.TypeOf(arg))) + } + return strings.Join(argsStr, ",") + } else if len(indexOrNil) == 1 { + // Index has been specified - get the argument at that index + var index = indexOrNil[0] + var s string + var ok bool + if s, ok = args.Get(index).(string); !ok { + panic(fmt.Sprintf("assert: arguments: String(%d) failed because object wasn't correct type: %s", index, args.Get(index))) + } + return s + } + + panic(fmt.Sprintf("assert: arguments: Wrong number of arguments passed to String. Must be 0 or 1, not %d", len(indexOrNil))) + +} + +// Int gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +func (args Arguments) Int(index int) int { + var s int + var ok bool + if s, ok = args.Get(index).(int); !ok { + panic(fmt.Sprintf("assert: arguments: Int(%d) failed because object wasn't correct type: %v", index, args.Get(index))) + } + return s +} + +// Error gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +func (args Arguments) Error(index int) error { + obj := args.Get(index) + var s error + var ok bool + if obj == nil { + return nil + } + if s, ok = obj.(error); !ok { + panic(fmt.Sprintf("assert: arguments: Error(%d) failed because object wasn't correct type: %v", index, args.Get(index))) + } + return s +} + +// Bool gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +func (args Arguments) Bool(index int) bool { + var s bool + var ok bool + if s, ok = args.Get(index).(bool); !ok { + panic(fmt.Sprintf("assert: arguments: Bool(%d) failed because object wasn't correct type: %v", index, args.Get(index))) + } + return s +} + +func typeAndKind(v interface{}) (reflect.Type, reflect.Kind) { + t := reflect.TypeOf(v) + k := t.Kind() + + if k == reflect.Ptr { + t = t.Elem() + k = t.Kind() + } + return t, k +} + +func diffArguments(expected Arguments, actual Arguments) string { + if len(expected) != len(actual) { + return fmt.Sprintf("Provided %v arguments, mocked for %v arguments", len(expected), len(actual)) + } + + for x := range expected { + if diffString := diff(expected[x], actual[x]); diffString != "" { + return fmt.Sprintf("Difference found in argument %v:\n\n%s", x, diffString) + } + } + + return "" +} + +// diff returns a diff of both values as long as both are of the same type and +// are a struct, map, slice or array. Otherwise it returns an empty string. +func diff(expected interface{}, actual interface{}) string { + if expected == nil || actual == nil { + return "" + } + + et, ek := typeAndKind(expected) + at, _ := typeAndKind(actual) + + if et != at { + return "" + } + + if ek != reflect.Struct && ek != reflect.Map && ek != reflect.Slice && ek != reflect.Array { + return "" + } + + e := spewConfig.Sdump(expected) + a := spewConfig.Sdump(actual) + + diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{ + A: difflib.SplitLines(e), + B: difflib.SplitLines(a), + FromFile: "Expected", + FromDate: "", + ToFile: "Actual", + ToDate: "", + Context: 1, + }) + + return diff +} + +var spewConfig = spew.ConfigState{ + Indent: " ", + DisablePointerAddresses: true, + DisableCapacities: true, + SortKeys: true, +} From beb01f175476701032f4b080f03e31a9e0fa319f Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 7 Jul 2017 13:55:39 -0700 Subject: [PATCH 104/105] test fixes --- client/driver/driver_test.go | 1 + client/driver/env/env_test.go | 1 + command/agent/job_endpoint_test.go | 2 +- command/deployment_promote_test.go | 8 ++++++++ command/node_status_test.go | 4 ++-- jobspec/parse_test.go | 2 +- nomad/job_endpoint.go | 1 - nomad/job_endpoint_test.go | 3 --- nomad/structs/structs.go | 1 - 9 files changed, 14 insertions(+), 9 deletions(-) diff --git a/client/driver/driver_test.go b/client/driver/driver_test.go index 9bda35f0d1de..53d839e957b8 100644 --- a/client/driver/driver_test.go +++ b/client/driver/driver_test.go @@ -206,6 +206,7 @@ func setupTaskEnv(t *testing.T, driver string) (*allocdir.TaskDir, map[string]st "HELLO": "world", "lorem": "ipsum", "NOMAD_ALLOC_ID": alloc.ID, + "NOMAD_ALLOC_INDEX": "0", "NOMAD_ALLOC_NAME": alloc.Name, "NOMAD_TASK_NAME": task.Name, "NOMAD_JOB_NAME": alloc.Job.Name, diff --git a/client/driver/env/env_test.go b/client/driver/env/env_test.go index 7846ad4e8e03..ce5262ebd48f 100644 --- a/client/driver/env/env_test.go +++ b/client/driver/env/env_test.go @@ -203,6 +203,7 @@ func TestEnvironment_AsList(t *testing.T) { "NOMAD_META_owner=armon", "NOMAD_JOB_NAME=my-job", fmt.Sprintf("NOMAD_ALLOC_ID=%s", a.ID), + "NOMAD_ALLOC_INDEX=0", } sort.Strings(act) sort.Strings(exp) diff --git a/command/agent/job_endpoint_test.go b/command/agent/job_endpoint_test.go index 5488e9568c32..173da2acb10e 100644 --- a/command/agent/job_endpoint_test.go +++ b/command/agent/job_endpoint_test.go @@ -965,7 +965,7 @@ func TestHTTP_JobStable(t *testing.T) { // Check the response stableResp := obj.(structs.JobStabilityResponse) - if stableResp.JobModifyIndex == 0 { + if stableResp.Index == 0 { t.Fatalf("bad: %v", stableResp) } diff --git a/command/deployment_promote_test.go b/command/deployment_promote_test.go index b3ce0c85a7ab..c05390d6c70c 100644 --- a/command/deployment_promote_test.go +++ b/command/deployment_promote_test.go @@ -27,6 +27,14 @@ func TestDeploymentPromoteCommand_Fails(t *testing.T) { if code := cmd.Run([]string{"-address=nope", "12"}); code != 1 { t.Fatalf("expected exit code 1, got: %d", code) } + if out := ui.ErrorWriter.String(); !strings.Contains(out, "flags must be specified") { + t.Fatalf("expected missing flags error, got: %s", out) + } + ui.ErrorWriter.Reset() + + if code := cmd.Run([]string{"-address=nope", "-all", "12"}); code != 1 { + t.Fatalf("expected exit code 1, got: %d", code) + } if out := ui.ErrorWriter.String(); !strings.Contains(out, "Error retrieving deployment") { t.Fatalf("expected failed query error, got: %s", out) } diff --git a/command/node_status_test.go b/command/node_status_test.go index 0b5f398e4796..3fba4c88cab6 100644 --- a/command/node_status_test.go +++ b/command/node_status_test.go @@ -48,7 +48,7 @@ func TestNodeStatusCommand_Self(t *testing.T) { if !strings.Contains(out, "mynode") { t.Fatalf("expect to find mynode, got: %s", out) } - if strings.Contains(out, "Allocations") { + if !strings.Contains(out, "No allocations placed") { t.Fatalf("should not dump allocations") } ui.OutputWriter.Reset() @@ -119,7 +119,7 @@ func TestNodeStatusCommand_Run(t *testing.T) { if !strings.Contains(out, "mynode") { t.Fatalf("expect to find mynode, got: %s", out) } - if strings.Contains(out, "Allocations") { + if !strings.Contains(out, "No allocations placed") { t.Fatalf("should not dump allocations") } diff --git a/jobspec/parse_test.go b/jobspec/parse_test.go index 012108be5217..340ed76e2de0 100644 --- a/jobspec/parse_test.go +++ b/jobspec/parse_test.go @@ -46,7 +46,7 @@ func TestParse(t *testing.T) { }, Update: &api.UpdateStrategy{ - Stagger: 60 * time.Second, + Stagger: helper.TimeToPtr(60 * time.Second), MaxParallel: helper.IntToPtr(2), HealthCheck: helper.StringToPtr("manual"), MinHealthyTime: helper.TimeToPtr(10 * time.Second), diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 72280eee53d5..b37ea308d393 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -430,7 +430,6 @@ func (j *Job) Stable(args *structs.JobStabilityRequest, reply *structs.JobStabil } // Setup the reply - reply.JobModifyIndex = modifyIndex reply.Index = modifyIndex return nil } diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index c811bb9f7585..677896fb3ee7 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -919,9 +919,6 @@ func TestJobEndpoint_Stable(t *testing.T) { if !out.Stable { t.Fatalf("Job is not marked stable") } - if out.JobModifyIndex != stableResp.JobModifyIndex { - t.Fatalf("got job modify index %d; want %d", out.JobModifyIndex, stableResp.JobModifyIndex) - } } func TestJobEndpoint_Evaluate(t *testing.T) { diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index d76f70d4ec3d..2a92c4b44586 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -332,7 +332,6 @@ type JobStabilityRequest struct { // JobStabilityResponse is the response when marking a job as stable. type JobStabilityResponse struct { - JobModifyIndex uint64 WriteMeta } From 3beaafca9af4e24348d02f90b7d9a64a830741fa Mon Sep 17 00:00:00 2001 From: Alex Dadgar Date: Fri, 7 Jul 2017 14:53:01 -0700 Subject: [PATCH 105/105] Vet and small improvement on watcher failure detection --- client/alloc_runner.go | 23 +++++++++++++++++++---- client/alloc_runner_health_watcher.go | 8 ++++++++ 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/client/alloc_runner.go b/client/alloc_runner.go index a545444395d1..ebd21e95ec68 100644 --- a/client/alloc_runner.go +++ b/client/alloc_runner.go @@ -497,6 +497,17 @@ func (r *AllocRunner) Alloc() *structs.Allocation { alloc.ClientStatus = getClientStatus(r.taskStates) r.taskStatusLock.RUnlock() + // If the client status is failed and we are part of a deployment, mark the + // alloc as unhealthy. This guards against the watcher not be started. + r.allocLock.Lock() + if alloc.ClientStatus == structs.AllocClientStatusFailed && + alloc.DeploymentID != "" && !alloc.DeploymentStatus.IsUnhealthy() { + alloc.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(false), + } + } + r.allocLock.Unlock() + return alloc } @@ -670,10 +681,6 @@ func (r *AllocRunner) Run() { defer close(r.waitCh) go r.dirtySyncState() - // Start the watcher - wCtx, watcherCancel := context.WithCancel(r.ctx) - go r.watchHealth(wCtx) - // Find the task group to run in the allocation alloc := r.Alloc() tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup) @@ -716,6 +723,10 @@ func (r *AllocRunner) Run() { return } + // Start the watcher + wCtx, watcherCancel := context.WithCancel(r.ctx) + go r.watchHealth(wCtx) + // Start the task runners r.logger.Printf("[DEBUG] client: starting task runners for alloc '%s'", r.alloc.ID) r.taskLock.Lock() @@ -787,6 +798,10 @@ OUTER: // Block until we should destroy the state of the alloc r.handleDestroy() + + // Free up the context. It has likely exited already + watcherCancel() + r.logger.Printf("[DEBUG] client: terminating runner for alloc '%s'", r.alloc.ID) } diff --git a/client/alloc_runner_health_watcher.go b/client/alloc_runner_health_watcher.go index 92e5c4149357..c27f6a16d5f3 100644 --- a/client/alloc_runner_health_watcher.go +++ b/client/alloc_runner_health_watcher.go @@ -133,6 +133,14 @@ OUTER: return } + // If the alloc is marked as failed by the client set the status to + // unhealthy + if alloc.ClientStatus == structs.AllocClientStatusFailed { + r.logger.Printf("[TRACE] client.alloc_watcher: client status failed for alloc %q", alloc.ID) + setHealth(false) + return + } + if len(alloc.TaskStates) != len(tg.Tasks) { r.logger.Printf("[TRACE] client.alloc_watcher: all task runners haven't started") continue OUTER