summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorJon Marius Venstad <jonmv@users.noreply.github.com>2022-02-14 16:02:41 +0100
committerGitHub <noreply@github.com>2022-02-14 16:02:41 +0100
commit7cfaed8c720105c47cdd9fcc18e6d4d687156654 (patch)
tree3291d4f3faec7ca260fc1c5c4338900f6514fc9f
parentadfb8ded72e194875ed9347569b169132e8b725e (diff)
parente99e17aac939e61a0258325bb1cb6b9087e41713 (diff)
Merge pull request #21173 from vespa-engine/jonmv/long-deployment-pipelines-2
Jonmv/long deployment pipelines 2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java90
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java250
3 files changed, 303 insertions, 39 deletions
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java
index 7f7449fb38b..d3271a4abb1 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java
@@ -7,6 +7,7 @@ import com.yahoo.config.application.api.DeploymentInstanceSpec;
import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.application.api.DeploymentSpec.DeclaredTest;
import com.yahoo.config.application.api.DeploymentSpec.DeclaredZone;
+import com.yahoo.config.application.api.DeploymentSpec.UpgradeRollout;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.InstanceName;
import com.yahoo.config.provision.SystemName;
@@ -313,7 +314,7 @@ public class DeploymentStatus {
if (job.application().instance().equals(instance) && job.type().isProduction()) {
List<Job> toRun = new ArrayList<>();
- List<Change> changes = changes(job, step, change, deployment);
+ List<Change> changes = changes(job, step, change);
if (changes.isEmpty()) return;
for (Change partial : changes) {
toRun.add(new Job(Versions.from(partial, application, deployment, systemVersion),
@@ -340,36 +341,75 @@ public class DeploymentStatus {
}
/** Changes to deploy with the given job, possibly split in two steps. */
- private List<Change> changes(JobId job, StepStatus step, Change change, Optional<Deployment> deployment) {
+ private List<Change> changes(JobId job, StepStatus step, Change change) {
// Signal strict completion criterion by depending on job itself.
if (step.completedAt(change, Optional.of(job)).isPresent())
return List.of();
+ if (change.platform().isEmpty() || change.application().isEmpty() || change.isPinned())
+ return List.of(change);
+
if ( step.completedAt(change.withoutApplication(), Optional.of(job)).isPresent()
|| step.completedAt(change.withoutPlatform(), Optional.of(job)).isPresent())
return List.of(change);
- if (change.platform().isEmpty() || change.application().isEmpty() || change.isPinned())
- return List.of(change);
+ // For a dual change, where both target remain, we determine what to run by looking at when the two parts became ready:
+ // for deployments, we look at dependencies; for tests, this may be overridden by what is already deployed.
+ JobId deployment = new JobId(job.application(), JobType.from(system, job.type().zone(system)).get());
+ UpgradeRollout rollout = application.deploymentSpec().requireInstance(job.application().instance()).upgradeRollout();
+ if (job.type().isTest()) {
+ Optional<Instant> platformDeployedAt = jobSteps.get(deployment).completedAt(change.withoutApplication(), Optional.of(deployment));
+ Optional<Instant> revisionDeployedAt = jobSteps.get(deployment).completedAt(change.withoutPlatform(), Optional.of(deployment));
+
+ // If only the revision has deployed, then we expect to test that first.
+ if (platformDeployedAt.isEmpty() && revisionDeployedAt.isPresent()) return List.of(change.withoutPlatform(), change);
+
+ // If only the upgrade has deployed, then we expect to test that first, with one exception:
+ // The revision has caught up to the upgrade at the deployment job; and either
+ // the upgrade is failing between deployment and here, or
+ // the specified rollout is leading or simultaneous; and
+ // the revision is now blocked by waiting for the production test to verify the upgrade.
+ // In this case we must abandon the production test on the pure upgrade, so the revision can be deployed.
+ if (platformDeployedAt.isPresent() && revisionDeployedAt.isEmpty()) {
+ if (jobSteps.get(deployment).readyAt(change, Optional.of(deployment))
+ .map(ready -> ! now.isBefore(ready)).orElse(false)) {
+ switch (rollout) {
+ // If separate rollout, this test should keep blocking the revision, unless there are failures.
+ case separate: return hasFailures(jobSteps.get(deployment), jobSteps.get(job)) ? List.of(change) : List.of(change.withoutApplication(), change);
+ // If leading rollout, this test should now expect the two changes to fuse and roll together.
+ case leading: return List.of(change);
+ // If simultaneous rollout, this test should now expect the revision to run ahead.
+ case simultaneous: return List.of(change.withoutPlatform(), change);
+ }
+ }
+ return List.of(change.withoutApplication(), change);
+ }
+ // If neither is deployed, then neither is ready, and we guess like for deployments.
+ // If both are deployed, then we need to follow normal logic for whatever is ready.
+ }
Optional<Instant> platformReadyAt = step.dependenciesCompletedAt(change.withoutApplication(), Optional.of(job));
Optional<Instant> revisionReadyAt = step.dependenciesCompletedAt(change.withoutPlatform(), Optional.of(job));
- // If the revision is not ready for this job, we prioritise the upgrade, assuming it was triggered first,
- // unless this is a production test job, and the upgrade is already failing between deployment and here,
- // in which case we must abandon the production test on the pure upgrade, so the revision can be deployed.
+ // If neither change is ready, we guess based on the specified rollout.
+ if (platformReadyAt.isEmpty() && revisionReadyAt.isEmpty()) switch (rollout) {
+ case separate: return List.of(change.withoutApplication(), change); // Platform should stay ahead.
+ case leading: return List.of(change); // They should eventually join.
+ case simultaneous: return List.of(change.withoutPlatform(), change); // Revision should get ahead.
+ }
+
+ // If only the revision is ready, we run that first.
+ if (platformReadyAt.isEmpty()) return List.of(change.withoutPlatform(), change);
+
+ // If only the platform is ready, we run that first.
if (revisionReadyAt.isEmpty()) {
- if (isTestAndIsFailingBetweenDeploymentAndThis(job)) return List.of(change);
return List.of(change.withoutApplication(), change);
}
- // If only the revision is ready, we prioritise that.
- if (platformReadyAt.isEmpty()) return List.of(change.withoutPlatform(), change);
- // Both changes are ready for this step, so we use the policy to decide.
- // TODO jonmv adding change.application.at and change.platform.at makes this better
+ // Both changes are ready for this step, and we look to the specified rollout to decide.
boolean platformReadyFirst = platformReadyAt.get().isBefore(revisionReadyAt.get());
boolean revisionReadyFirst = revisionReadyAt.get().isBefore(platformReadyAt.get());
- switch (application.deploymentSpec().requireInstance(job.application().instance()).upgradeRollout()) {
+ switch (rollout) {
case separate: // Let whichever change rolled out first, keep rolling first, unless upgrade alone is failing.
return (platformReadyFirst || platformReadyAt.get().equals(Instant.EPOCH)) // Assume platform was first if no jobs have run yet.
? step.job().flatMap(jobs()::get).flatMap(JobStatus::firstFailing).isPresent()
@@ -386,12 +426,6 @@ public class DeploymentStatus {
}
}
- private boolean isTestAndIsFailingBetweenDeploymentAndThis(JobId job) {
- if ( ! job.type().isTest()) return false;
- JobId deployment = new JobId(job.application(), JobType.from(system, job.type().zone(system)).get());
- return hasFailures(jobSteps.get(deployment), jobSteps.get(job));
- }
-
/** The test jobs that need to run prior to the given production deployment jobs. */
public Map<JobId, List<Job>> testJobs(Map<JobId, List<Job>> jobs) {
Map<JobId, List<Job>> testJobs = new LinkedHashMap<>();
@@ -498,7 +532,7 @@ public class DeploymentStatus {
if (step instanceof DeploymentInstanceSpec) {
DeploymentInstanceSpec spec = ((DeploymentInstanceSpec) step);
- StepStatus instanceStatus = new InstanceStatus(spec, previous, now, application.require(spec.name()), this);
+ StepStatus instanceStatus = new InstanceStatus(spec, previous, now, application.require(spec.name()));
instance = spec.name();
allSteps.add(instanceStatus);
previous = List.of(instanceStatus);
@@ -603,11 +637,13 @@ public class DeploymentStatus {
/** The time at which all dependencies completed on the given change and / or versions. */
Optional<Instant> dependenciesCompletedAt(Change change, Optional<JobId> dependent) {
- return dependencies.stream().allMatch(step -> step.completedAt(change, dependent).isPresent())
- ? dependencies.stream().map(step -> step.completedAt(change, dependent).get())
- .max(naturalOrder())
- .or(() -> Optional.of(Instant.EPOCH))
- : Optional.empty();
+ Instant latest = Instant.EPOCH;
+ for (StepStatus step : dependencies) {
+ Optional<Instant> completedAt = step.completedAt(change, dependent);
+ if (completedAt.isEmpty()) return Optional.empty();
+ latest = latest.isBefore(completedAt.get()) ? completedAt.get() : latest;
+ }
+ return Optional.of(latest);
}
/** The time until which this step is blocked by a change blocker. */
@@ -644,15 +680,13 @@ public class DeploymentStatus {
private final DeploymentInstanceSpec spec;
private final Instant now;
private final Instance instance;
- private final DeploymentStatus status;
private InstanceStatus(DeploymentInstanceSpec spec, List<StepStatus> dependencies, Instant now,
- Instance instance, DeploymentStatus status) {
+ Instance instance) {
super(StepType.instance, spec, dependencies, spec.name());
this.spec = spec;
this.now = now;
this.instance = instance;
- this.status = status;
}
/**
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java
index b944593bbab..3be21aec608 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java
@@ -373,7 +373,7 @@ public class DeploymentTrigger {
if (productionTest.isPresent()) {
abortIfOutdated(status, jobs, productionTest.get().id());
// Production deployments are also blocked by their declared tests, if the next versions to run
- // for those are not the same as the versions we're considering running in the deployment job now.
+ // for those are not the same as the versions we're considering running in the deployment job now.
if (productionTest.map(JobStatus::id).map(jobs::get)
.map(versions -> ! versions.get(0).versions().targetsMatch(jobs.get(job).get(0).versions()))
.orElse(false))
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java
index 0cc5b479304..fb34d57ba0d 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java
@@ -1263,28 +1263,258 @@ public class DeploymentTriggerTest {
assertEquals(Change.empty(), app3.instance().change());
}
- // TODO test new revision allowed exactly when dependent instance is failing
// TODO test multi-tier pipeline with various policies
// TODO test multi-tier pipeline with upgrade after new version is the candidate
@Test
public void testRevisionJoinsUpgradeWithSeparateRollout() {
- var app = tester.newDeploymentContext().submit().deploy();
- var version = new Version("7.1");
- tester.controllerTester().upgradeSystem(version);
+ var appPackage = new ApplicationPackageBuilder().region("us-central-1")
+ .region("us-east-3")
+ .region("us-west-1")
+ .upgradeRollout("separate")
+ .build();
+ var app = tester.newDeploymentContext().submit(appPackage).deploy();
+
+ // Platform rolls through first production zone.
+ var version0 = tester.controller().readSystemVersion();
+ var version1 = new Version("7.1");
+ tester.controllerTester().upgradeSystem(version1);
tester.upgrader().maintain();
app.runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
tester.clock().advance(Duration.ofMinutes(1));
- app.submit();
- assertEquals(Change.of(version).with(app.lastSubmission().get()), app.instance().change());
+ // Revision starts rolling, but stays behind.
+ var revision0 = app.lastSubmission();
+ app.submit(appPackage);
+ var revision1 = app.lastSubmission();
+ assertEquals(Change.of(version1).with(revision1.get()), app.instance().change());
app.runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
- app.runJob(productionUsEast3).runJob(productionUsWest1);
- tester.triggerJobs();
- assertEquals(Change.of(app.lastSubmission().get()), app.instance().change());
+ // Upgrade got here first, so attempts to proceed alone, but the upgrade fails.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision0.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+ app.timeOutConvergence(productionUsEast3);
+
+ // Revision is allowed to join.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version1), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+ app.runJob(productionUsEast3);
+
+ // Platform and revision now proceed together.
+ app.runJob(stagingTest);
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsWest1).get().versions());
+ app.runJob(productionUsWest1);
+ assertEquals(Change.empty(), app.instance().change());
+ }
+
+ @Test
+ public void testProductionTestBlockingDeploymentWithSeparateRollout() {
+ var appPackage = new ApplicationPackageBuilder().region("us-east-3")
+ .region("us-west-1")
+ .delay(Duration.ofHours(1))
+ .test("us-east-3")
+ .upgradeRollout("separate")
+ .build();
+ var app = tester.newDeploymentContext().submit(appPackage)
+ .runJob(systemTest).runJob(stagingTest)
+ .runJob(productionUsEast3).runJob(productionUsWest1);
+ tester.clock().advance(Duration.ofHours(1));
+ app.runJob(testUsEast3);
+ assertEquals(Change.empty(), app.instance().change());
+
+ // Platform rolls through first production zone.
+ var version0 = tester.controller().readSystemVersion();
+ var version1 = new Version("7.1");
+ tester.controllerTester().upgradeSystem(version1);
+ tester.upgrader().maintain();
+ app.runJob(systemTest).runJob(stagingTest).runJob(productionUsEast3);
+
+ // Revision starts rolling, but waits for production test to verify the upgrade.
+ var revision0 = app.lastSubmission();
+ app.submit(appPackage);
+ var revision1 = app.lastSubmission();
+ assertEquals(Change.of(version1).with(revision1.get()), app.instance().change());
+ app.runJob(systemTest).runJob(stagingTest).triggerJobs();
+ app.assertRunning(productionUsWest1);
+ app.assertNotRunning(productionUsEast3);
+
+ // Upgrade got here first, so attempts to proceed alone, but the upgrade fails.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision0.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsWest1).get().versions());
+ app.timeOutConvergence(productionUsWest1).triggerJobs();
+
+ // Upgrade now fails between us-east-3 deployment and test, so test is abandoned, and revision unblocked.
+ app.assertRunning(productionUsEast3);
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version1), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+ app.runJob(productionUsEast3).triggerJobs()
+ .jobAborted(productionUsWest1).runJob(productionUsWest1);
+ tester.clock().advance(Duration.ofHours(1));
+ app.runJob(testUsEast3);
+ assertEquals(Change.empty(), app.instance().change());
+ }
+
+ @Test
+ public void testProductionTestNotBlockingDeploymentWithSimultaneousRollout() {
+ var appPackage = new ApplicationPackageBuilder().region("us-east-3")
+ .region("us-central-1")
+ .region("us-west-1")
+ .delay(Duration.ofHours(1))
+ .test("us-east-3")
+ .test("us-west-1")
+ .upgradeRollout("simultaneous")
+ .build();
+ var app = tester.newDeploymentContext().submit(appPackage)
+ .runJob(systemTest).runJob(stagingTest)
+ .runJob(productionUsEast3).runJob(productionUsCentral1).runJob(productionUsWest1);
+ tester.clock().advance(Duration.ofHours(1));
+ app.runJob(testUsEast3).runJob(testUsWest1);
+ assertEquals(Change.empty(), app.instance().change());
+
+ // Platform rolls through first production zone.
+ var version0 = tester.controller().readSystemVersion();
+ var version1 = new Version("7.1");
+ tester.controllerTester().upgradeSystem(version1);
+ tester.upgrader().maintain();
+ app.runJob(systemTest).runJob(stagingTest).runJob(productionUsEast3);
- app.runJob(productionUsEast3).runJob(productionUsWest1);
+ // Revision starts rolling, and causes production test to abort when it reaches deployment.
+ var revision0 = app.lastSubmission();
+ app.submit(appPackage);
+ var revision1 = app.lastSubmission();
+ assertEquals(Change.of(version1).with(revision1.get()), app.instance().change());
+ app.runJob(systemTest).runJob(stagingTest).triggerJobs();
+ app.assertRunning(productionUsCentral1);
+ app.assertRunning(productionUsEast3);
+
+ // Revision deploys to first prod zone.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version1), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+ tester.clock().advance(Duration.ofSeconds(1));
+ app.runJob(productionUsEast3);
+
+ // Revision catches up in second prod zone.
+ app.runJob(systemTest).runJob(stagingTest).runJob(stagingTest).triggerJobs();
+ app.jobAborted(productionUsCentral1).triggerJobs();
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsCentral1).get().versions());
+ app.runJob(productionUsCentral1).triggerJobs();
+
+ // Revision proceeds alone in third prod zone, making test targets different for the two prod tests.
+ assertEquals(new Versions(version0, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsWest1).get().versions());
+ app.runJob(productionUsWest1);
+ app.triggerJobs();
+ app.assertNotRunning(testUsEast3);
+ tester.clock().advance(Duration.ofHours(1));
+
+ // Test lets revision proceed alone, and us-west-1 is blocked until tested.
+ app.runJob(testUsEast3).triggerJobs();
+ app.assertNotRunning(productionUsWest1);
+ app.runJob(testUsWest1).runJob(productionUsWest1).runJob(testUsWest1); // Test for us-east-3 is not re-run.
+ assertEquals(Change.empty(), app.instance().change());
+ }
+
+ @Test
+ public void testRevisionJoinsUpgradeWithLeadingRollout() {
+ var appPackage = new ApplicationPackageBuilder().region("us-central-1")
+ .region("us-east-3")
+ .region("us-west-1")
+ .upgradeRollout("leading")
+ .build();
+ var app = tester.newDeploymentContext().submit(appPackage).deploy();
+
+ // Platform rolls through first production zone.
+ var version0 = tester.controller().readSystemVersion();
+ var version1 = new Version("7.1");
+ tester.controllerTester().upgradeSystem(version1);
+ tester.upgrader().maintain();
+ app.runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
+ tester.clock().advance(Duration.ofMinutes(1));
+
+ // Revision starts rolling, and catches up.
+ var revision0 = app.lastSubmission();
+ app.submit(appPackage);
+ var revision1 = app.lastSubmission();
+ assertEquals(Change.of(version1).with(revision1.get()), app.instance().change());
+ app.runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
+
+ // Upgrade got here first, and has triggered, but is now obsolete.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision0.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+ assertEquals(RunStatus.running, tester.jobs().last(app.instanceId(), productionUsEast3).get().status());
+
+ // Once staging tests verify the joint upgrade, the job is replaced with that.
+ app.runJob(stagingTest);
+ app.triggerJobs();
+ app.jobAborted(productionUsEast3).runJob(productionUsEast3);
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+
+ // Platform and revision now proceed together.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsWest1).get().versions());
+ app.runJob(productionUsWest1);
+ assertEquals(Change.empty(), app.instance().change());
+ }
+
+ @Test
+ public void testRevisionPassesUpgradeWithSimultaneousRollout() {
+ var appPackage = new ApplicationPackageBuilder().region("us-central-1")
+ .region("us-east-3")
+ .region("us-west-1")
+ .upgradeRollout("simultaneous")
+ .build();
+ var app = tester.newDeploymentContext().submit(appPackage).deploy();
+
+ // Platform rolls through first production zone.
+ var version0 = tester.controller().readSystemVersion();
+ var version1 = new Version("7.1");
+ tester.controllerTester().upgradeSystem(version1);
+ tester.upgrader().maintain();
+ app.runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
+ tester.clock().advance(Duration.ofMinutes(1));
+
+ // Revision starts rolling, and catches up.
+ var revision0 = app.lastSubmission();
+ app.submit(appPackage);
+ var revision1 = app.lastSubmission();
+ assertEquals(Change.of(version1).with(revision1.get()), app.instance().change());
+ app.runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
+
+ // Upgrade got here first, and has triggered, but is now obsolete.
+ app.triggerJobs();
+ app.assertRunning(productionUsEast3);
+ assertEquals(new Versions(version1, revision0.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+ assertEquals(RunStatus.running, tester.jobs().last(app.instanceId(), productionUsEast3).get().status());
+
+ // Once staging tests verify the joint upgrade, the job is replaced with that.
+ app.runJob(systemTest).runJob(stagingTest).runJob(stagingTest);
+ app.triggerJobs();
+ app.jobAborted(productionUsEast3).runJob(productionUsEast3);
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsEast3).get().versions());
+
+ // Revision now proceeds alone.
+ app.triggerJobs();
+ assertEquals(new Versions(version0, revision1.get(), Optional.of(version0), revision0),
+ tester.jobs().last(app.instanceId(), productionUsWest1).get().versions());
+ app.runJob(productionUsWest1);
+
+ // Upgrade follows.
+ app.triggerJobs();
+ assertEquals(new Versions(version1, revision1.get(), Optional.of(version0), revision1),
+ tester.jobs().last(app.instanceId(), productionUsWest1).get().versions());
+ app.runJob(productionUsWest1);
assertEquals(Change.empty(), app.instance().change());
}