Skip to content

Commit

Permalink
feat(monitoredDeploy): Completed event should include details of succ…
Browse files Browse the repository at this point in the history
…ess/failure (#3214)

* feat(monitoredDeploy): Completed event should include details of success/failure

* addressing comments

* adding tests

* Update orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/monitoreddeploy/NotifyDeployCompletedTaskSpec.groovy

Co-Authored-By: Mark Vulfson <markvu@live.com>

* Update orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/monitoreddeploy/NotifyDeployCompletedTask.java

Co-Authored-By: Mark Vulfson <markvu@live.com>

* removing null check
  • Loading branch information
gal-yardeni authored and mergify[bot] committed Oct 28, 2019
1 parent 871358f commit 3483221
Show file tree
Hide file tree
Showing 10 changed files with 227 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ class MonitoredDeployStrategy implements Strategy {
stage.execution,
NotifyDeployCompletedStage.PIPELINE_CONFIG_TYPE,
"Notify monitored deploy complete",
evalContext,
evalContext + [hasDeploymentFailed: false],
stage,
SyntheticStageOwner.STAGE_AFTER
)
Expand Down Expand Up @@ -399,7 +399,7 @@ class MonitoredDeployStrategy implements Strategy {
internalStageData.oldServerGroup = source?.serverGroupName
internalStageData.newServerGroup = createServerStageData.getServerGroup()
internalStageData.parameters = stageData.deploymentMonitor.parameters

internalStageData.hasDeploymentFailed = true;
evalContext += internalStageData.toContextMap()
stages << newStage(
parent.execution,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ class ClusterSizePreconditionTask extends AbstractCloudProviderAwareTask impleme
int actual = serverGroups.size()
boolean acceptable = config.getOp().evaluate(actual, config.expected)
if (!acceptable) {
failures << "expected $config.comparison $config.expected ${config.onlyEnabledServerGroups ? 'enabled ' : ''}server groups in $region but found $actual: ${serverGroups*.name}"
failures << "expected $config.comparison $config.expected ${config.onlyEnabledServerGroups ? 'enabled ' : ''}server groups in $region but found $actual: ${serverGroups*.name}. Please clean up the cluster to only have the specific number of server groups, or opt out of this check in your deploy stage."
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import com.netflix.spinnaker.config.DeploymentMonitorServiceProvider;
import com.netflix.spinnaker.orca.ExecutionStatus;
import com.netflix.spinnaker.orca.TaskResult;
import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies.MonitoredDeployStageData;
import com.netflix.spinnaker.orca.deploymentmonitor.models.EvaluateHealthRequest;
import com.netflix.spinnaker.orca.deploymentmonitor.models.EvaluateHealthResponse;
import com.netflix.spinnaker.orca.pipeline.model.Stage;
Expand All @@ -44,9 +43,7 @@ public class EvaluateDeploymentHealthTask extends MonitoredDeployBaseTask {

@Override
public @Nonnull TaskResult executeInternal(
Stage stage,
MonitoredDeployStageData context,
DeploymentMonitorDefinition monitorDefinition) {
Stage stage, DeploymentMonitorDefinition monitorDefinition) {
EvaluateHealthRequest request = new EvaluateHealthRequest(stage);

log.info(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ public long getDynamicTimeout(Stage stage) {
DeploymentMonitorDefinition monitorDefinition = getDeploymentMonitorDefinition(stage);

try {
return executeInternal(stage, context, monitorDefinition);
return executeInternal(stage, monitorDefinition);
} catch (RetrofitError e) {
log.warn(
"HTTP Error encountered while talking to {}->{}, {}}",
Expand All @@ -151,9 +151,7 @@ public long getDynamicTimeout(Stage stage) {
}

public @Nonnull TaskResult executeInternal(
Stage stage,
MonitoredDeployStageData context,
DeploymentMonitorDefinition monitorDefinition) {
Stage stage, DeploymentMonitorDefinition monitorDefinition) {
throw new UnsupportedOperationException("Must implement executeInternal method");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import com.netflix.spinnaker.config.DeploymentMonitorServiceProvider;
import com.netflix.spinnaker.orca.ExecutionStatus;
import com.netflix.spinnaker.orca.TaskResult;
import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies.MonitoredDeployStageData;
import com.netflix.spinnaker.orca.clouddriver.pipeline.cluster.RollbackClusterStage;
import com.netflix.spinnaker.orca.deploymentmonitor.models.DeploymentCompletedRequest;
import com.netflix.spinnaker.orca.pipeline.model.Stage;
import javax.annotation.Nonnull;
Expand All @@ -40,13 +40,33 @@ public class NotifyDeployCompletedTask extends MonitoredDeployBaseTask {

@Override
public @Nonnull TaskResult executeInternal(
Stage stage,
MonitoredDeployStageData context,
DeploymentMonitorDefinition monitorDefinition) {
// TODO(mvulfson): actually populate the request data
Stage stage, DeploymentMonitorDefinition monitorDefinition) {
DeploymentCompletedRequest request = new DeploymentCompletedRequest(stage);
monitorDefinition.getService().notifyCompleted(request);

request.setStatus(
convertStageStatus(
(boolean) stage.getContext().getOrDefault("hasDeploymentFailed", false)));
request.setRollback(DeploymentCompletedRequest.DeploymentStatus.ROLLBACK_NOT_PERFORMED);

// check whether rollback was initiated and successful
if (stage.getParent() != null) {
stage.getParent().directChildren().stream()
.filter(s -> s.getType().equals(RollbackClusterStage.PIPELINE_CONFIG_TYPE))
.findFirst()
.ifPresent(
foundRollbackStage ->
request.setRollback(
convertStageStatus(
foundRollbackStage.getStatus() != ExecutionStatus.SUCCEEDED)));
}

monitorDefinition.getService().notifyCompleted(request);
return TaskResult.ofStatus(ExecutionStatus.SUCCEEDED);
}

private DeploymentCompletedRequest.DeploymentStatus convertStageStatus(Boolean failedStatus) {
return failedStatus
? DeploymentCompletedRequest.DeploymentStatus.FAILURE
: DeploymentCompletedRequest.DeploymentStatus.SUCCESS;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import com.netflix.spinnaker.config.DeploymentMonitorServiceProvider;
import com.netflix.spinnaker.orca.ExecutionStatus;
import com.netflix.spinnaker.orca.TaskResult;
import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies.MonitoredDeployStageData;
import com.netflix.spinnaker.orca.deploymentmonitor.models.EvaluateHealthResponse;
import com.netflix.spinnaker.orca.deploymentmonitor.models.RequestBase;
import com.netflix.spinnaker.orca.pipeline.model.Stage;
Expand All @@ -41,9 +40,7 @@ public class NotifyDeployStartingTask extends MonitoredDeployBaseTask {

@Override
public @Nonnull TaskResult executeInternal(
Stage stage,
MonitoredDeployStageData context,
DeploymentMonitorDefinition monitorDefinition) {
Stage stage, DeploymentMonitorDefinition monitorDefinition) {
RequestBase request = new RequestBase(stage);
EvaluateHealthResponse response = monitorDefinition.getService().notifyStarting(request);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
/*
* Copyright 2019 Netflix, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.netflix.spinnaker.orca.clouddriver.tasks.monitoreddeploy

import com.fasterxml.jackson.databind.ObjectMapper
import com.netflix.spectator.api.NoopRegistry
import com.netflix.spinnaker.config.DeploymentMonitorDefinition
import com.netflix.spinnaker.config.DeploymentMonitorServiceProvider
import com.netflix.spinnaker.orca.ExecutionStatus
import com.netflix.spinnaker.orca.TaskResult
import com.netflix.spinnaker.orca.clouddriver.pipeline.cluster.RollbackClusterStage
import com.netflix.spinnaker.orca.clouddriver.pipeline.monitoreddeploy.NotifyDeployCompletedStage
import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.CreateServerGroupStage
import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies.DeploymentMonitor
import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies.MonitoredDeployStageData
import com.netflix.spinnaker.orca.deploymentmonitor.DeploymentMonitorService
import com.netflix.spinnaker.orca.deploymentmonitor.models.DeploymentCompletedRequest
import com.netflix.spinnaker.orca.pipeline.model.Execution
import com.netflix.spinnaker.orca.pipeline.model.Stage
import retrofit.client.Response
import spock.lang.Specification

import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.pipeline

class NotifyDeployCompletedTaskSpec extends Specification {
ObjectMapper mapper = new ObjectMapper()
Execution pipe = pipeline {
}

def "should indicate deployment success or failure when no rollback is performed"() {
given:
def monitorServiceStub = Stub(DeploymentMonitorService) {
notifyCompleted(_) >> {
return new Response("", 200, "OK", [], null)
}
}

def serviceProviderStub = Stub(DeploymentMonitorServiceProvider) {
getDefinitionById(_) >> {
def deploymentMonitor = new DeploymentMonitorDefinition()
deploymentMonitor.id = "LogMonitorId"
deploymentMonitor.name = "LogMonitor"
deploymentMonitor.failOnError = true
deploymentMonitor.service = monitorServiceStub

return deploymentMonitor
}
}

def task = new NotifyDeployCompletedTask(serviceProviderStub, new NoopRegistry())

MonitoredDeployStageData stageData = new MonitoredDeployStageData()
stageData.deploymentMonitor = new DeploymentMonitor()
stageData.deploymentMonitor.id = "LogMonitorId"
stageData.application = pipe.application

def stage = new Stage(pipe, NotifyDeployCompletedStage.PIPELINE_CONFIG_TYPE, mapper.convertValue(stageData, Map))
stage.context.put("hasDeploymentFailed", false)

when:
TaskResult result = task.execute(stage)

then:
monitorServiceStub.notifyCompleted({
it != null
} as DeploymentCompletedRequest) >> { DeploymentCompletedRequest request ->
assert request.status == DeploymentCompletedRequest.DeploymentStatus.SUCCESS &&
request.rollback == DeploymentCompletedRequest.DeploymentStatus.ROLLBACK_NOT_PERFORMED
new Response('', 200, 'OK', [], null)
}


when: 'deployment has failed'

stage.context.put("hasDeploymentFailed", true)
result = task.execute(stage)

then:
monitorServiceStub.notifyCompleted({
it != null
} as DeploymentCompletedRequest) >> { DeploymentCompletedRequest request ->
assert request.status == DeploymentCompletedRequest.DeploymentStatus.FAILURE &&
request.rollback == DeploymentCompletedRequest.DeploymentStatus.ROLLBACK_NOT_PERFORMED
new Response('', 200, 'OK', [], null)
}

result.status == ExecutionStatus.SUCCEEDED
}

def "rollback stage was initiated"() {
given:
def monitorServiceStub = Stub(DeploymentMonitorService) {

notifyCompleted(_) >> {
return new Response("", 200, "OK", [], null)
}
}

def serviceProviderStub = Stub(DeploymentMonitorServiceProvider) {
getDefinitionById(_) >> {
def deploymentMonitor = new DeploymentMonitorDefinition()
deploymentMonitor.id = "LogMonitorId"
deploymentMonitor.name = "LogMonitor"
deploymentMonitor.failOnError = true
deploymentMonitor.service = monitorServiceStub

return deploymentMonitor
}
}

def task = new NotifyDeployCompletedTask(serviceProviderStub, new NoopRegistry())

MonitoredDeployStageData stageData = new MonitoredDeployStageData()
stageData.deploymentMonitor = new DeploymentMonitor()
stageData.deploymentMonitor.id = "LogMonitorId"
stageData.application = pipe.application


def notifyCompleteStage = new Stage(pipe, NotifyDeployCompletedStage.PIPELINE_CONFIG_TYPE, mapper.convertValue(stageData, Map))
def rollbackStage = new Stage(pipe, RollbackClusterStage.PIPELINE_CONFIG_TYPE, mapper.convertValue(stageData, Map))
def createServerGroupStage = new Stage(pipe, CreateServerGroupStage.PIPELINE_CONFIG_TYPE, mapper.convertValue(stageData, Map))
rollbackStage.status = ExecutionStatus.SUCCEEDED

notifyCompleteStage.setParentStageId(createServerGroupStage.getId())
rollbackStage.setParentStageId(createServerGroupStage.getId())
pipe.stages.addAll([notifyCompleteStage, rollbackStage, createServerGroupStage])


when: 'rollback was initiated and successful'
TaskResult result = task.execute(notifyCompleteStage)

then:

monitorServiceStub.notifyCompleted({
it != null
} as DeploymentCompletedRequest) >> { DeploymentCompletedRequest request ->
assert request.rollback == DeploymentCompletedRequest.DeploymentStatus.SUCCESS
new Response('', 200, 'OK', [], null)
}

when: 'rollback was initiated and failed'

rollbackStage.status = ExecutionStatus.FAILED_CONTINUE
result = task.execute(notifyCompleteStage)

then:

monitorServiceStub.notifyCompleted({
it != null
} as DeploymentCompletedRequest) >> { DeploymentCompletedRequest request ->
assert request.rollback == DeploymentCompletedRequest.DeploymentStatus.FAILURE
new Response('', 200, 'OK', [], null)
}

result.status == ExecutionStatus.SUCCEEDED
}
}


Original file line number Diff line number Diff line change
Expand Up @@ -615,6 +615,21 @@ public List<Stage> allDownstreamStages() {
return children;
}

/**
* Gets all direct children of the current stage. This is not a recursive method and will return
* only the children in the first level of the stage.
*/
public List<Stage> directChildren() {

if (execution != null) {
return getExecution().getStages().stream()
.filter(
stage -> stage.getParentStageId() != null && stage.getParentStageId().equals(getId()))
.collect(toList());
}
return emptyList();
}

/** Maps the stage's context to a typed object */
public <O> O mapTo(Class<O> type) {
return mapTo(null, type);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,14 @@ public enum DeploymentStatus {
SUCCESS,

@JsonProperty("failure")
FAILURE
FAILURE,

@JsonProperty("not_performed")
ROLLBACK_NOT_PERFORMED;
}

private DeploymentStatus status;
private Boolean rolledback;
private DeploymentStatus rollback;

public DeploymentCompletedRequest(Stage stage) {
super(stage);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ public class MonitoredDeployInternalStageData {
private String cloudProvider;
private int currentProgress;
private Map<String, Object> parameters;
private boolean hasDeploymentFailed;

public Map toContextMap() {
return new ObjectMapper().convertValue(this, Map.class);
Expand Down

0 comments on commit 3483221

Please sign in to comment.