summaryrefslogtreecommitdiffstats
path: root/controller-server
diff options
context:
space:
mode:
authorMartin Polden <mpolden@mpolden.no>2018-04-27 13:03:54 +0200
committerGitHub <noreply@github.com>2018-04-27 13:03:54 +0200
commitd7ec114976a4e5816a272aea25dbb2e13717fb92 (patch)
treea500c17acb70bf2bdcb689bff850c6e55fc2ffe8 /controller-server
parentd83814043998817d404b922e3050ce2006e8ec19 (diff)
parent57b64df221bec89715a80aa515a945c236c36f6e (diff)
Merge pull request #5726 from vespa-engine/jvenstad/DO-unified
Refactoring
Diffstat (limited to 'controller-server')
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java1
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/DeploymentJobs.java8
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java241
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java3
5 files changed, 102 insertions, 153 deletions
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java
index e37feae4225..1f4c1412515 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java
@@ -360,6 +360,7 @@ public class ApplicationController {
}
private ActivateResult unexpectedDeployment(ApplicationId applicationId, ZoneId zone) {
+
Log logEntry = new Log();
logEntry.level = "WARNING";
logEntry.time = clock.instant().toEpochMilli();
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/DeploymentJobs.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/DeploymentJobs.java
index d0cff526690..fb8a4051649 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/DeploymentJobs.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/DeploymentJobs.java
@@ -2,17 +2,14 @@
package com.yahoo.vespa.hosted.controller.application;
import com.google.common.collect.ImmutableMap;
-import com.yahoo.component.Version;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.RegionName;
import com.yahoo.config.provision.SystemName;
-import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
import com.yahoo.vespa.hosted.controller.api.integration.organization.IssueId;
import com.yahoo.vespa.hosted.controller.api.integration.zone.ZoneId;
-import java.time.Instant;
import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedHashMap;
@@ -94,7 +91,10 @@ public class DeploymentJobs {
/** Returns whether this has some job status which is not a success */
public boolean hasFailures() {
- return ! JobList.from(status.values()).failing().isEmpty();
+ return ! JobList.from(status.values())
+ .failing()
+ .not().failingBecause(JobError.outOfCapacity)
+ .isEmpty();
}
/** Returns the JobStatus of the given JobType, or empty. */
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 403c82dd63e..78ea99b523d 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
@@ -26,8 +26,6 @@ import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
@@ -44,13 +42,12 @@ import static com.yahoo.config.provision.Environment.prod;
import static com.yahoo.config.provision.Environment.staging;
import static com.yahoo.config.provision.Environment.test;
import static com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
-import static com.yahoo.vespa.hosted.controller.api.integration.BuildService.JobState.disabled;
+import static com.yahoo.vespa.hosted.controller.api.integration.BuildService.JobState.idle;
import static com.yahoo.vespa.hosted.controller.api.integration.BuildService.JobState.queued;
import static com.yahoo.vespa.hosted.controller.api.integration.BuildService.JobState.running;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.component;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.stagingTest;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.systemTest;
-import static java.util.Collections.emptyList;
import static java.util.Collections.emptySet;
import static java.util.Collections.singletonList;
import static java.util.Comparator.comparing;
@@ -126,10 +123,14 @@ public class DeploymentTrigger {
else triggering = application.deploymentJobs().statusOf(report.jobType()).flatMap(JobStatus::lastTriggered)
.orElseThrow(() -> new IllegalStateException("Got notified about completion of " + report.jobType().jobName() + " for " +
report.applicationId() + ", but that has neither been triggered nor deployed"));
- applications().store(application.withJobCompletion(report.projectId(),
- report.jobType(),
- triggering.completion(report.buildNumber(), clock.instant()),
- report.jobError()));
+ application = application.withJobCompletion(report.projectId(),
+ report.jobType(),
+ triggering.completion(report.buildNumber(), clock.instant()),
+ report.jobError());
+
+ application = application.withChange(remainingChange(application));
+
+ applications().store(application);
});
}
@@ -154,7 +155,7 @@ public class DeploymentTrigger {
.collect(groupingBy(Job::applicationId)))
.values().stream()
.map(jobs -> (Supplier<Long>) jobs.stream()
- .filter(job -> canTrigger(job) && trigger(job))
+ .filter(this::trigger)
.limit(entry.getKey() ? 1 : Long.MAX_VALUE)::count))
.parallel().map(Supplier::get).reduce(0L, Long::sum);
}
@@ -166,14 +167,12 @@ public class DeploymentTrigger {
* the project id is removed from the application owning the job, to prevent further trigger attemps.
*/
public boolean trigger(Job job) {
- log.log(LogLevel.INFO, String.format("Attempting to trigger %s: %s (%s)", job, job.reason, job.target));
+ log.log(LogLevel.INFO, String.format("Attempting to trigger %s: %s", job, job.triggering));
try {
buildService.trigger(job);
- applications().lockOrThrow(job.applicationId(), application -> applications().store(application.withJobTriggering(
- job.jobType, JobRun.triggering(job.target.targetPlatform, job.target.targetApplication,
- job.target.sourcePlatform, job.target.sourceApplication,
- job.reason, clock.instant()))));
+ applications().lockOrThrow(job.applicationId(), application ->
+ applications().store(application.withJobTriggering(job.jobType, job.triggering)));
return true;
}
catch (RuntimeException e) {
@@ -218,19 +217,23 @@ public class DeploymentTrigger {
return computeReadyJobs().collect(groupingBy(Job::jobType));
}
- /** Returns the set of all jobs which have changes to propagate from the upstream steps. */
- private Stream<Job> computeReadyJobs() {
- return ApplicationList.from(applications().asList())
- .notPullRequest()
- .withProjectId()
- .deploying()
- .idList().stream()
- .map(this::computeReadyJobs)
- .flatMap(List::stream);
+ public List<JobType> forceTrigger(ApplicationId applicationId, JobType jobType) {
+ Application application = applications().require(applicationId);
+ if (jobType == component) {
+ buildService.trigger(BuildJob.of(applicationId, application.deploymentJobs().projectId().getAsLong(), jobType.jobName()));
+ return singletonList(component);
+ }
+ Versions versions = versionsFor(application, application.change(), deploymentFor(application, jobType));
+ String reason = ">:o:< Triggered by force! (-o-) |-o-| (=oo=)";
+ return (jobType.isProduction() && ! isTested(application, versions)
+ ? testJobsFor(application, versions, reason, clock.instant()).stream()
+ : Stream.of(deploymentJob(application, versions, application.change(), jobType, reason, clock.instant())))
+ .peek(this::trigger)
+ .map(Job::jobType).collect(toList());
}
/** Returns whether the given job is currently running; false if completed since last triggered, asking the build service otherwise. */
- public boolean isRunning(Application application, JobType jobType) {
+ private boolean isRunning(Application application, JobType jobType) {
return ! application.deploymentJobs().statusOf(jobType)
.flatMap(job -> job.lastCompleted().map(run -> run.at().isAfter(job.lastTriggered().get().at())))
.orElse(false)
@@ -243,29 +246,13 @@ public class DeploymentTrigger {
jobType.jobName())));
}
- public List<JobType> forceTrigger(ApplicationId applicationId, JobType jobType) {
- Application application = applications().require(applicationId);
- if (jobType == component) {
- buildService.trigger(BuildJob.of(applicationId, application.deploymentJobs().projectId().getAsLong(), jobType.jobName()));
- return singletonList(component);
- }
- State target = targetFor(application, application.change(), deploymentFor(application, jobType));
- String reason = ">:o:< Triggered by force! (-o-) |-o-| (=oo=)";
- if ( ! jobType.isProduction() || isTested(application, target)) {
- trigger(deploymentJob(application, target, application.change(), jobType, reason, clock.instant(), Collections.emptySet()));
- return singletonList(jobType);
- }
- List<Job> testJobs = testJobsFor(application, target, reason, clock.instant());
- testJobs.forEach(this::trigger);
- return testJobs.stream().map(Job::jobType).collect(toList());
- }
-
- private Job deploymentJob(Application application, State target, Change change, JobType jobType, String reason, Instant availableSince, Collection<JobType> concurrentlyWith) {
+ private Job deploymentJob(Application application, Versions versions, Change change, JobType jobType, String reason, Instant availableSince) {
boolean isRetry = application.deploymentJobs().statusOf(jobType).flatMap(JobStatus::jobError)
.filter(JobError.outOfCapacity::equals).isPresent();
if (isRetry) reason += "; retrying on out of capacity";
- return new Job(application, target, change, jobType, reason, availableSince, concurrentlyWith, isRetry);
+ JobRun triggering = JobRun.triggering(versions.targetPlatform, versions.targetApplication, versions.sourcePlatform, versions.sourceApplication, reason, clock.instant());
+ return new Job(application, triggering, jobType, availableSince, isRetry, change.application().isPresent());
}
private Version targetPlatform(Application application, Change change, Optional<Deployment> deployment) {
@@ -284,6 +271,17 @@ public class DeploymentTrigger {
return ! o1.isPresent() ? o2 : ! o2.isPresent() ? o1 : o1.get().compareTo(o2.get()) >= 0 ? o1 : o2;
}
+ /** Returns the set of all jobs which have changes to propagate from the upstream steps. */
+ private Stream<Job> computeReadyJobs() {
+ return ApplicationList.from(applications().asList())
+ .notPullRequest()
+ .withProjectId()
+ .deploying()
+ .idList().stream()
+ .map(this::computeReadyJobs)
+ .flatMap(List::stream);
+ }
+
/**
* Finds the next step to trigger for the given application, if any, and returns these as a list.
*/
@@ -307,20 +305,20 @@ public class DeploymentTrigger {
for (Step step : productionSteps) {
Set<JobType> stepJobs = step.zones().stream().map(order::toJob).collect(toSet());
Map<Optional<Instant>, List<JobType>> jobsByCompletion = stepJobs.stream().collect(groupingBy(job -> completedAt(change, application, job)));
- if (jobsByCompletion.containsKey(empty())) { // Step not complete, because some jobs remain -- trigger these if the previous step was done.
+ if (jobsByCompletion.containsKey(empty())) { // Step not complete, because some jobs remain -- trigger these if the previous step was done, or theirs tests if needed.
for (JobType job : jobsByCompletion.get(empty())) {
- State target = targetFor(application, change, deploymentFor(application, job));
- if (isTested(application, target)) {
- if (completedAt.isPresent())
- jobs.add(deploymentJob(application, target, change, job, reason, completedAt.get(), stepJobs));
+ Versions versions = versionsFor(application, change, deploymentFor(application, job));
+ if (isTested(application, versions)) {
+ if ( completedAt.isPresent()
+ && jobStateIsAmong(application, job, idle)
+ && stepJobs.containsAll(runningProductionJobsFor(application)))
+ jobs.add(deploymentJob(application, versions, change, job, reason, completedAt.get()));
}
else if (testJobs == null) {
- if ( ! alreadyTriggered(application, target)) // TODO jvenstad: This is always true now ...
- testJobs = testJobsFor(application, target, "Testing deployment for " + job.jobName(), completedAt.orElse(clock.instant()));
- else
- testJobs = emptyList();
+ testJobs = testJobsFor(application, versions, "Testing deployment for " + job.jobName(), completedAt.orElse(clock.instant()));
}
}
+ completedAt = empty();
}
else { // All jobs are complete -- find the time of completion of this step.
if (stepJobs.isEmpty()) { // No jobs means this is delay step.
@@ -335,15 +333,13 @@ public class DeploymentTrigger {
}
}
if (testJobs == null)
- testJobs = testJobsFor(application, targetFor(application, application.change(), empty()), "Testing last changes outside prod", clock.instant());
+ testJobs = testJobsFor(application, versionsFor(application, application.change(), empty()), "Testing last changes outside prod", clock.instant());
jobs.addAll(testJobs);
-
- removeCompletedChange(application);
});
return jobs;
}
- private void removeCompletedChange(Application application) {
+ private Change remainingChange(Application application) {
List<JobType> jobs = (application.deploymentSpec().steps().isEmpty()
? singletonList(new DeploymentSpec.DeclaredZone(test))
: application.deploymentSpec().steps()).stream()
@@ -359,67 +355,61 @@ public class DeploymentTrigger {
.map(change -> jobs.stream().allMatch(job -> completedAt(change, application, job).isPresent()))
.orElse(false);
- if (platformComplete || applicationComplete)
- applications().lockIfPresent(application.id(), lockedApplication -> {
- if ( ! application.change().equals(lockedApplication.change()))
- return; // If new changes were added after we verified completion, we can't remove those.
-
- Change change = application.change();
- if (platformComplete) change = change.withoutPlatform();
- if (applicationComplete) change = change.withoutApplication();
- applications().store(lockedApplication.withChange(change));
- });
+ Change change = application.change();
+ if (platformComplete) change = change.withoutPlatform();
+ if (applicationComplete) change = change.withoutApplication();
+ return change;
}
/**
- * Returns the list of test jobs that should run now, and that need to succeed on the given target for it to be considered tested.
+ * Returns the list of test jobs that should run now, and that need to succeed on the given versions for it to be considered tested.
*/
- private List<Job> testJobsFor(Application application, State target, String reason, Instant availableSince) {
+ private List<Job> testJobsFor(Application application, Versions versions, String reason, Instant availableSince) {
List<Step> steps = application.deploymentSpec().steps();
if (steps.isEmpty()) steps = singletonList(new DeploymentSpec.DeclaredZone(test));
List<Job> jobs = new ArrayList<>();
for (Step step : steps.stream().filter(step -> step.deploysTo(test) || step.deploysTo(staging)).collect(toList())) {
for (JobType jobType : step.zones().stream().map(order::toJob).collect(toList())) {
- Optional<JobRun> completion = successOn(application, jobType, target)
- .filter(run -> jobType != stagingTest || sourcesMatchIfPresent(target, run));
- if ( ! completion.isPresent())
- jobs.add(deploymentJob(application, target, application.change(), jobType, reason, availableSince, emptySet()));
+ Optional<JobRun> completion = successOn(application, jobType, versions)
+ .filter(run -> jobType != stagingTest || sourcesMatchIfPresent(versions, run));
+ if ( ! completion.isPresent() && jobStateIsAmong(application, jobType, idle))
+ jobs.add(deploymentJob(application, versions, application.change(), jobType, reason, availableSince));
}
}
return jobs;
}
- private boolean isTested(Application application, State target) {
- return testedAt(application, target).isPresent()
- || alreadyTriggered(application, target);
+ private boolean isTested(Application application, Versions versions) {
+ return testedAt(application, versions).isPresent()
+ || alreadyTriggered(application, versions);
}
/** If the given state's sources are present and differ from its targets, returns whether they are equal to those of the given job run. */
- private static boolean sourcesMatchIfPresent(State target, JobRun jobRun) {
- return ( ! target.sourcePlatform.filter(version -> ! version.equals(target.targetPlatform)).isPresent()
- || target.sourcePlatform.equals(jobRun.sourcePlatform()))
- && ( ! target.sourceApplication.filter(version -> ! version.equals(target.targetApplication)).isPresent()
- || target.sourceApplication.equals(jobRun.sourceApplication()));
+ private static boolean sourcesMatchIfPresent(Versions versions, JobRun jobRun) {
+ return ( ! versions.sourcePlatform.filter(version -> ! version.equals(versions.targetPlatform)).isPresent()
+ || versions.sourcePlatform.equals(jobRun.sourcePlatform()))
+ && ( ! versions.sourceApplication.filter(version -> ! version.equals(versions.targetApplication)).isPresent()
+ || versions.sourceApplication.equals(jobRun.sourceApplication()));
}
- private static boolean targetsMatch(State target, JobRun jobRun) {
- return target.targetPlatform.equals(jobRun.platform()) && target.targetApplication.equals(jobRun.application());
+ private static boolean targetsMatch(Versions versions, JobRun jobRun) {
+ return versions.targetPlatform.equals(jobRun.platform()) && versions.targetApplication.equals(jobRun.application());
}
- private Optional<Instant> testedAt(Application application, State target) {
- Optional<JobRun> testRun = successOn(application, systemTest, target);
- Optional<JobRun> stagingRun = successOn(application, stagingTest, target)
- .filter(run -> sourcesMatchIfPresent(target, run));
+ private Optional<Instant> testedAt(Application application, Versions versions) {
+ Optional<JobRun> testRun = successOn(application, systemTest, versions);
+ Optional<JobRun> stagingRun = successOn(application, stagingTest, versions)
+ .filter(run -> sourcesMatchIfPresent(versions, run));
return max(testRun.map(JobRun::at), stagingRun.map(JobRun::at))
.filter(__ -> testRun.isPresent() && stagingRun.isPresent());
}
- private boolean alreadyTriggered(Application application, State target) {
+ private boolean alreadyTriggered(Application application, Versions versions) {
return application.deploymentJobs().jobStatus().values().stream()
.filter(job -> job.type().isProduction())
.anyMatch(job -> job.lastTriggered()
- .filter(run -> targetsMatch(target, run))
- .filter(run -> sourcesMatchIfPresent(target, run))
+ .filter(run -> targetsMatch(versions, run))
+ .filter(run -> sourcesMatchIfPresent(versions, run))
.isPresent());
}
@@ -432,10 +422,9 @@ public class DeploymentTrigger {
* change for the application downgrades the deployment, which is an acknowledgement that the deployed
* version is broken somehow, such that the job may be locked in failure until a new version is released.
*/
- // TODO jvenstad: This is only used for production jobs now.
private Optional<Instant> completedAt(Change change, Application application, JobType jobType) {
- State target = targetFor(application, change, deploymentFor(application, jobType));
- Optional<JobRun> lastSuccess = successOn(application, jobType, target);
+ Versions versions = versionsFor(application, change, deploymentFor(application, jobType));
+ Optional<JobRun> lastSuccess = successOn(application, jobType, versions);
if (lastSuccess.isPresent() || ! jobType.isProduction())
return lastSuccess.map(JobRun::at);
@@ -447,26 +436,9 @@ public class DeploymentTrigger {
.map(Deployment::at);
}
- private Optional<JobRun> successOn(Application application, JobType jobType, State target) {
+ private Optional<JobRun> successOn(Application application, JobType jobType, Versions versions) {
return application.deploymentJobs().statusOf(jobType).flatMap(JobStatus::lastSuccess)
- .filter(run -> targetsMatch(target, run));
- }
-
- private boolean canTrigger(Job job) {
- Application application = applications().require(job.applicationId());
- if (isRunning(application, job.jobType) || jobStateIsAmong(application, job.jobType, disabled))
- return false;
-
- if (successOn(application, job.jobType, job.target).filter(run -> sourcesMatchIfPresent(job.target, run)).isPresent())
- return false; // Job may have completed since it was computed.
-
- if ( ! job.jobType.isProduction())
- return true;
-
- if ( ! job.concurrentlyWith.containsAll(runningProductionJobsFor(application)))
- return false;
-
- return true;
+ .filter(run -> targetsMatch(versions, run));
}
private List<JobType> runningProductionJobsFor(Application application) {
@@ -483,7 +455,6 @@ public class DeploymentTrigger {
private boolean acceptNewApplicationVersion(Application application) {
if (application.change().application().isPresent()) return true; // More application changes are ok.
if (application.deploymentJobs().hasFailures()) return true; // Allow changes to fix upgrade problems.
- // Otherwise, allow an application change if not currently upgrading.
return ! application.changeAt(clock.instant()).platform().isPresent();
}
@@ -491,75 +462,53 @@ public class DeploymentTrigger {
return Optional.ofNullable(application.deployments().get(jobType.zone(controller.system()).get()));
}
- private State targetFor(Application application, Change change, Optional<Deployment> deployment) {
- return new State(targetPlatform(application, change, deployment),
- targetApplication(application, change, deployment),
- deployment.map(Deployment::version),
- deployment.map(Deployment::applicationVersion));
+ private Versions versionsFor(Application application, Change change, Optional<Deployment> deployment) {
+ return new Versions(targetPlatform(application, change, deployment),
+ targetApplication(application, change, deployment),
+ deployment.map(Deployment::version),
+ deployment.map(Deployment::applicationVersion));
}
private static class Job extends BuildJob {
- private final Change change;
private final JobType jobType;
- private final String reason;
+ private final JobRun triggering;
private final Instant availableSince;
- private final Collection<JobType> concurrentlyWith;
private final boolean isRetry;
private final boolean isApplicationUpgrade;
- private final State target;
- private Job(Application application, State target, Change change, JobType jobType, String reason, Instant availableSince, Collection<JobType> concurrentlyWith, boolean isRetry) {
+ private Job(Application application, JobRun triggering, JobType jobType, Instant availableSince, boolean isRetry, boolean isApplicationUpgrade) {
super(application.id(), application.deploymentJobs().projectId().getAsLong(), jobType.jobName());
- this.change = change;
this.jobType = jobType;
+ this.triggering = triggering;
this.availableSince = availableSince;
- this.concurrentlyWith = concurrentlyWith;
- this.reason = reason;
this.isRetry = isRetry;
- this.isApplicationUpgrade = change.application().isPresent();
- this.target = target;
+ this.isApplicationUpgrade = isApplicationUpgrade;
}
JobType jobType() { return jobType; }
- Instant availableSince() { return availableSince; }
+ Instant availableSince() { return availableSince; } // TODO jvenstad: This is 95% broken now. Change.at() can restore it.
boolean isRetry() { return isRetry; }
boolean applicationUpgrade() { return isApplicationUpgrade; }
}
- public static class State {
+ private static class Versions {
private final Version targetPlatform;
private final ApplicationVersion targetApplication;
private final Optional<Version> sourcePlatform;
private final Optional<ApplicationVersion> sourceApplication;
- public State(Version targetPlatform, ApplicationVersion targetApplication, Optional<Version> sourcePlatform, Optional<ApplicationVersion> sourceApplication) {
+ private Versions(Version targetPlatform, ApplicationVersion targetApplication, Optional<Version> sourcePlatform, Optional<ApplicationVersion> sourceApplication) {
this.targetPlatform = targetPlatform;
this.targetApplication = targetApplication;
this.sourcePlatform = sourcePlatform;
this.sourceApplication = sourceApplication;
}
- public Version targetPlatform() { return targetPlatform; }
- public ApplicationVersion targetApplication() { return targetApplication; }
- public Optional<Version> sourcePlatform() { return sourcePlatform; }
- public Optional<ApplicationVersion> sourceApplication() { return sourceApplication; }
-
- @Override
- public String toString() {
- return String.format("platform %s%s, application %s%s",
- targetPlatform,
- sourcePlatform.filter(version -> ! version.equals(targetPlatform))
- .map(v -> " (from " + v + ")").orElse(""),
- targetApplication.id(),
- sourceApplication.filter(version -> ! version.equals(targetApplication))
- .map(v -> " (from " + v.id() + ")").orElse(""));
- }
-
}
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java
index afd53e7bd3c..4d2e64d66c6 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java
@@ -804,7 +804,7 @@ public class UpgraderTest {
// 5th app never reports back and has a dead job, but no ongoing change
Application deadLocked = tester.applications().require(default4.id());
- assertTrue("Jobs in progress", tester.deploymentTrigger().isRunning(deadLocked, systemTest));
+ tester.assertRunning(deadLocked.id(), systemTest);
assertFalse("No change present", deadLocked.change().isPresent());
// 4 out of 5 applications are repaired and confidence is restored
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
index 3bfcf6f194c..196794df068 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
@@ -885,8 +885,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
response = container.handleRequest(request("/screwdriver/v1/jobsToRun", GET).get());
Inspector jobs = SlimeUtils.jsonToSlime(response.getBody()).get();
- assertTrue("Response contains staging-test", jobs.field("staging-test").valid());
- assertEquals("Response contains only this item", 1, jobs.children());
+ assertEquals("Response contains no items, as all jobs are triggered", 0, jobs.children());
}