summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorJon Marius Venstad <jvenstad@yahoo-inc.com>2018-04-13 10:58:14 +0200
committerJon Marius Venstad <jvenstad@yahoo-inc.com>2018-04-13 12:08:10 +0200
commit5163c7ddde2f98866e6b922f6d514e4cbf137e2c (patch)
tree889bd6a9fb392c585b42c8e98ee105b969947732
parenta4073d0e1f5078cdeebd2c1f23134da20ca62dec (diff)
BuildService.isRunning and remove faulty data on exceptions
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java8
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/stubs/MockBuildService.java17
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/DeploymentJobs.java19
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobList.java5
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobStatus.java10
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java153
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java14
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java5
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java13
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java45
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/FailureRedeployerTest.java39
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java31
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerControllerTester.java13
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiTest.java12
15 files changed, 143 insertions, 243 deletions
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java
index e32ad47b180..d853443bf5d 100644
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java
@@ -9,11 +9,11 @@ import java.util.Objects;
public interface BuildService {
/**
- * Enqueues a job defined by buildJob in an external build system, and returns the outcome of the enqueue request.
- * This method should return false only when a retry is in order, and true otherwise, e.g., on success, or for
- * invalid jobs.
+ * Enqueues a job defined by buildJob in an external build system.
+ *
+ * Implementations should throw an exception if the triggering fails.
*/
- boolean trigger(BuildJob buildJob);
+ void trigger(BuildJob buildJob);
/**
* Returns whether the given job is currently running.
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/stubs/MockBuildService.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/stubs/MockBuildService.java
deleted file mode 100644
index a75236e6f67..00000000000
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/stubs/MockBuildService.java
+++ /dev/null
@@ -1,17 +0,0 @@
-package com.yahoo.vespa.hosted.controller.api.integration.stubs;
-
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
-
-public class MockBuildService implements BuildService {
-
- @Override
- public boolean trigger(BuildJob buildJob) {
- return true;
- }
-
- @Override
- public boolean isRunning(BuildJob buildJob) {
- return false;
- }
-
-}
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 f31360451cc..dca1736d2c5 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
@@ -96,18 +96,6 @@ public class DeploymentJobs {
return ! JobList.from(status.values()).failing().isEmpty();
}
- /** Returns whether any job is currently in progress */
- public boolean isRunning(Instant timeoutLimit) {
- return ! JobList.from(status.values()).running(timeoutLimit).isEmpty();
- }
-
- /** Returns whether the given job type is currently running and was started after timeoutLimit */
- public boolean isRunning(JobType jobType, Instant timeoutLimit) {
- JobStatus jobStatus = status.get(jobType);
- if ( jobStatus == null) return false;
- return jobStatus.isRunning(timeoutLimit);
- }
-
/** Returns whether change can be deployed to the given environment */
public boolean isDeployableTo(Environment environment, Change change) {
// TODO jvenstad: Rewrite to verify versions when deployment is already decided.
@@ -127,13 +115,6 @@ public class DeploymentJobs {
return Optional.ofNullable(jobStatus().get(jobType));
}
- /** Returns the last successful application version for the given job */
- public Optional<ApplicationVersion> lastSuccessfulApplicationVersionFor(JobType jobType) {
- return statusOf(jobType)
- .flatMap(JobStatus::lastSuccess)
- .map(JobStatus.JobRun::applicationVersion);
- }
-
/**
* Returns the id of the Screwdriver project running these deployment jobs
* - or empty when this is not known or does not exist.
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobList.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobList.java
index c19c8a31ec1..bb90370f6c0 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobList.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobList.java
@@ -73,11 +73,6 @@ public class JobList {
&& job.lastSuccess().get().version().isBefore(job.lastTriggered().get().version()));
}
- /** Returns the subset of jobs which are currently running, according to the given timeout */
- public JobList running(Instant timeoutLimit) {
- return filter(job -> job.isRunning(timeoutLimit));
- }
-
/** Returns the subset of jobs which are currently failing */
public JobList failing() {
return filter(job -> ! job.isSuccess());
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobStatus.java
index 325af87a21a..febaac8b18f 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/JobStatus.java
@@ -77,7 +77,7 @@ public class JobStatus {
if (type == DeploymentJobs.JobType.component) { // not triggered by us
version = controller.systemVersion(); // TODO jvenstad: Get rid of this, and perhaps all of component info?
reason = "Application commit";
- } else if (! lastTriggered.isPresent()) {
+ } else if ( ! lastTriggered.isPresent()) {
throw new IllegalStateException("Got notified about completion of " + this +
", but that has neither been triggered nor deployed");
@@ -109,14 +109,6 @@ public class JobStatus {
return lastCompleted().isPresent() && ! jobError.isPresent();
}
- /** Returns true if last triggered is newer than last completed and was started after timeoutLimit */
- public boolean isRunning(Instant timeoutLimit) {
- if ( ! lastTriggered.isPresent()) return false;
- if (lastTriggered.get().at().isBefore(timeoutLimit)) return false;
- if ( ! lastCompleted.isPresent()) return true;
- return ! lastTriggered.get().at().isBefore(lastCompleted.get().at());
- }
-
/** The error of the last completion, or empty if the last run succeeded */
public Optional<DeploymentJobs.JobError> jobError() { return jobError; }
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 55f16999cce..c78f5289d3c 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
@@ -5,7 +5,6 @@ import com.yahoo.component.Version;
import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.Environment;
-import com.yahoo.config.provision.SystemName;
import com.yahoo.log.LogLevel;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.ApplicationController;
@@ -19,7 +18,6 @@ import com.yahoo.vespa.hosted.controller.application.Deployment;
import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobError;
import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobReport;
import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
-import com.yahoo.vespa.hosted.controller.application.JobList;
import com.yahoo.vespa.hosted.controller.application.JobStatus;
import com.yahoo.vespa.hosted.controller.persistence.CuratorDb;
@@ -36,6 +34,7 @@ import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import java.util.logging.Logger;
+import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.Comparator.comparing;
@@ -57,11 +56,6 @@ import static java.util.stream.Collectors.toSet;
*/
public class DeploymentTrigger {
- /**
- * The max duration a job may run before we consider it dead/hanging
- */
- private final Duration jobTimeout;
-
private final static Logger log = Logger.getLogger(DeploymentTrigger.class.getName());
private final Controller controller;
@@ -77,14 +71,6 @@ public class DeploymentTrigger {
this.clock = clock;
this.order = new DeploymentOrder(controller::system);
this.buildService = buildService;
- this.jobTimeout = controller.system().equals(SystemName.main) ? Duration.ofHours(12) : Duration.ofHours(1);
- }
-
- /**
- * Returns the time in the past before which jobs are at this moment considered unresponsive
- */
- public Instant jobTimeoutLimit() {
- return clock.instant().minus(jobTimeout);
}
public DeploymentOrder deploymentOrder() {
@@ -97,6 +83,11 @@ public class DeploymentTrigger {
* Called each time a job completes (successfully or not) to record information used when deciding what to trigger.
*/
public void notifyOfCompletion(JobReport report) {
+ log.log(LogLevel.INFO, String.format("Got notified of %s for %s of %s (%d).",
+ report.jobError().map(JobError::toString).orElse("success"),
+ report.jobType(),
+ report.applicationId(),
+ report.projectId()));
if ( ! applications().get(report.applicationId()).isPresent()) {
log.log(LogLevel.WARNING, "Ignoring completion of job of project '" + report.projectId() +
"': Unknown application '" + report.applicationId() + "'");
@@ -151,23 +142,20 @@ public class DeploymentTrigger {
* Triggers the given job for the given application.
*/
public boolean trigger(Job job) {
- log.log(LogLevel.INFO, String.format("Attempting to trigger %s for %s, deploying %s: %s", job.jobType, job.id, job.change, job.reason));
+ log.log(LogLevel.INFO, String.format("Attempting to trigger %s for %s, deploying %s: %s (platform: %s, application: %s)", job.jobType, job.id, job.change, job.reason, job.platformVersion, job.applicationVersion.id()));
- BuildService.BuildJob buildJob = new BuildService.BuildJob(job.projectId, job.jobType.jobName());
try {
- if (buildService.trigger(buildJob)) {
- applications().lockOrThrow(job.id, application -> applications().store(application.withJobTriggering(
- job.jobType, new JobStatus.JobRun(-1, job.platformVersion, job.applicationVersion, job.reason, clock.instant()))));
- return true;
- }
+ buildService.trigger(new BuildService.BuildJob(job.projectId, job.jobType.jobName()));
+ applications().lockOrThrow(job.id, application -> applications().store(application.withJobTriggering(
+ job.jobType, new JobStatus.JobRun(-1, job.platformVersion, job.applicationVersion, job.reason, clock.instant()))));
+ return true;
}
- catch (NoSuchElementException | IllegalArgumentException e) {
+ catch (RuntimeException e) {
+ if (e instanceof NoSuchElementException || e instanceof IllegalArgumentException)
applications().lockOrThrow(job.id, application -> applications().store(application.withProjectId(Optional.empty())));
- log.log(LogLevel.WARNING, "Removing projectId " + job.projectId + " from " + job.id
- + " because of exception trying to trigger " + buildJob + ": " + e.getMessage());
+ log.log(LogLevel.WARNING, String.format("Exception triggering %s for %s (%s): %s", job.jobType, job.id, job.projectId, e));
+ return false;
}
- log.log(LogLevel.INFO, "Failed to trigger " + buildJob + " for " + job.id);
- return false;
}
/**
@@ -204,9 +192,59 @@ public class DeploymentTrigger {
}
/**
- * Finds the next step to trigger for the given application, if any, and triggers it
+ * Returns the set of all jobs which have changes to propagate from the upstream steps, sorted by job.
+ */
+ public Stream<Job> computeReadyJobs() {
+ return ApplicationList.from(applications().asList())
+ .notPullRequest()
+ .withProjectId()
+ .deploying()
+ .idList().stream()
+ .map(this::computeReadyJobs)
+ .flatMap(List::stream);
+ }
+
+ public 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)
+ && buildService.isRunning(new BuildService.BuildJob(application.deploymentJobs().projectId().get(), jobType.jobName()));
+ }
+
+ public Job forcedDeploymentJob(Application application, JobType jobType, String reason) {
+ return deploymentJob(application, jobType, reason, clock.instant(), Collections.emptySet());
+ }
+
+ // TODO JVENSTAD: CHECK ALL TESTS WITH LOW VERSIONS!!!!!
+
+ private Job deploymentJob(Application application, JobType jobType, String reason, Instant availableSince, Collection<JobType> concurrentlyWith) {
+ boolean isRetry = application.deploymentJobs().statusOf(jobType).flatMap(JobStatus::jobError)
+ .filter(JobError.outOfCapacity::equals).isPresent();
+ if (isRetry) reason += "; retrying on out of capacity";
+
+ Change change = application.change();
+ // For both versions, use the newer of the change's and the currently deployed versions, or a fallback if none of these exist.
+ Version platform = jobType == JobType.component
+ ? Version.emptyVersion
+ : deploymentFor(application, jobType).map(Deployment::version)
+ .filter(version -> ! change.upgrades(version))
+ .orElse(change.platform()
+ .orElse(application.oldestDeployedPlatform()
+ .orElse(controller.systemVersion())));
+ ApplicationVersion applicationVersion = jobType == JobType.component
+ ? ApplicationVersion.unknown
+ : deploymentFor(application, jobType).map(Deployment::applicationVersion)
+ .filter(version -> ! change.upgrades(version))
+ .orElse(change.application()
+ .orElseGet(() -> application.oldestDeployedApplication()
+ .orElseThrow(() -> new IllegalArgumentException("Cannot determine application version to use for " + jobType))));
+
+ return new Job(application, jobType, reason, availableSince, concurrentlyWith, isRetry, change, platform, applicationVersion);
+ }
+
+ /**
+ * Finds the next step to trigger for the given application, if any, and returns these as a list.
*/
- public List<Job> computeReadyJobs(ApplicationId id) {
+ private List<Job> computeReadyJobs(ApplicationId id) {
List<Job> jobs = new ArrayList<>();
applications().lockIfPresent(id, application -> {
List<DeploymentSpec.Step> steps = application.deploymentSpec().steps().isEmpty()
@@ -245,19 +283,6 @@ public class DeploymentTrigger {
}
/**
- * Returns the set of all jobs which have changes to propagate from the upstream steps, sorted by job.
- */
- public Stream<Job> computeReadyJobs() {
- return ApplicationList.from(applications().asList())
- .notPullRequest()
- .withProjectId()
- .deploying()
- .idList().stream()
- .map(this::computeReadyJobs)
- .flatMap(List::stream);
- }
-
- /**
* Returns the instant when the given change is complete for the given application for the given job.
*
* Any job is complete if the given change is already successful on that job.
@@ -284,16 +309,13 @@ public class DeploymentTrigger {
if ( ! application.deploymentJobs().isDeployableTo(job.jobType.environment(), application.change()))
return false;
- if (application.deploymentJobs().isRunning(job.jobType, jobTimeoutLimit()))
+ if (isRunning(application, job.jobType))
return false;
if ( ! job.jobType.isProduction())
return true;
- if ( ! job.concurrentlyWith.containsAll(JobList.from(application)
- .production()
- .running(jobTimeoutLimit())
- .mapToList(JobStatus::type)))
+ if ( ! job.concurrentlyWith.containsAll(runningProductionJobsFor(application)))
return false;
if ( ! application.changeAt(clock.instant()).isPresent())
@@ -302,6 +324,13 @@ public class DeploymentTrigger {
return true;
}
+ private List<JobType> runningProductionJobsFor(Application application) {
+ return application.deploymentJobs().jobStatus().keySet().parallelStream()
+ .filter(job -> job.isProduction())
+ .filter(job -> isRunning(application, job))
+ .collect(Collectors.toList());
+ }
+
private ApplicationController applications() {
return controller.applications();
}
@@ -317,36 +346,6 @@ public class DeploymentTrigger {
return Optional.ofNullable(application.deployments().get(jobType.zone(controller.system()).get()));
}
- public Job forcedDeploymentJob(Application application, JobType jobType, String reason) {
- return deploymentJob(application, jobType, reason, clock.instant(), Collections.emptySet());
- }
-
- public Job deploymentJob(Application application, JobType jobType, String reason, Instant availableSince, Collection<JobType> concurrentlyWith) {
- boolean isRetry = application.deploymentJobs().statusOf(jobType).flatMap(JobStatus::jobError)
- .filter(JobError.outOfCapacity::equals).isPresent();
- if (isRetry) reason += "; retrying on out of capacity";
-
- Change change = application.change();
- // For both versions, use the newer of the change's and the currently deployed versions, or a fallback if none of these exist.
- Version platform = jobType == JobType.component
- ? Version.emptyVersion
- : deploymentFor(application, jobType).map(Deployment::version)
- .filter(version -> ! change.upgrades(version))
- .orElse(change.platform()
- .orElse(application.oldestDeployedPlatform()
- .orElse(controller.systemVersion())));
- ApplicationVersion applicationVersion = jobType == JobType.component
- ? ApplicationVersion.unknown
- : deploymentFor(application, jobType).map(Deployment::applicationVersion)
- .filter(version -> ! change.upgrades(version))
- .orElse(change.application()
- .orElseGet(() -> application.oldestDeployedApplication()
- .orElseThrow(() -> new IllegalArgumentException("Cannot determine application version to use for " + jobType))));
-
- return new Job(application, jobType, reason, availableSince, concurrentlyWith, isRetry, change, platform, applicationVersion);
- }
-
-
public static class Job {
private final ApplicationId id;
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java
index 3195bff1325..79dd6156220 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java
@@ -31,7 +31,7 @@ import com.yahoo.vespa.hosted.controller.application.JobStatus;
import com.yahoo.vespa.hosted.controller.application.SourceRevision;
import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder;
import com.yahoo.vespa.hosted.controller.deployment.BuildJob;
-import com.yahoo.vespa.hosted.controller.deployment.MockBuildService;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester;
import com.yahoo.vespa.hosted.controller.persistence.ApplicationSerializer;
import com.yahoo.vespa.hosted.controller.rotation.RotationId;
@@ -436,7 +436,9 @@ public class ControllerTest {
assertEquals(3, mockBuildService.jobs().size());
// Abort all running jobs, so we have three candidate jobs, of which only one should be triggered at a time.
- tester.buildService().takeJobsToRun();
+ synchronized (tester.buildService()) {
+ tester.buildService().clear();
+ }
tester.clock().advance(Duration.ofHours(13));
List<BuildService.BuildJob> jobs = new ArrayList<>();
@@ -456,8 +458,8 @@ public class ControllerTest {
// Remove the jobs for app1 and app2, and then let app3 fail with outOfCapacity.
// All three jobs are now eligible, but the one for app3 should trigger first as an outOfCapacity-retry.
- tester.buildService().removeJob(app1.deploymentJobs().projectId().get(), stagingTest);
- tester.buildService().removeJob(app2.deploymentJobs().projectId().get(), stagingTest);
+ tester.buildService().removeJob(app1.deploymentJobs().projectId().get(), stagingTest.jobName());
+ tester.buildService().removeJob(app2.deploymentJobs().projectId().get(), stagingTest.jobName());
tester.clock().advance(Duration.ofHours(13));
jobs.remove(new BuildService.BuildJob(app1.deploymentJobs().projectId().get(), stagingTest.jobName()));
jobs.remove(new BuildService.BuildJob(app2.deploymentJobs().projectId().get(), stagingTest.jobName()));
@@ -489,8 +491,8 @@ public class ControllerTest {
// Let the last system test job start, then remove the ones for apps 1 and 2, and let app3 fail with outOfCapacity again.
tester.readyJobTrigger().maintain();
- tester.buildService().removeJob(app1.deploymentJobs().projectId().get(), systemTest);
- tester.buildService().removeJob(app2.deploymentJobs().projectId().get(), systemTest);
+ tester.buildService().removeJob(app1.deploymentJobs().projectId().get(), systemTest.jobName());
+ tester.buildService().removeJob(app2.deploymentJobs().projectId().get(), systemTest.jobName());
tester.clock().advance(Duration.ofHours(13));
jobs.clear();
jobs.add(new BuildService.BuildJob(app1.deploymentJobs().projectId().get(), stagingTest.jobName()));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java
index 27b77cfc901..097ee6ef057 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java
@@ -34,7 +34,7 @@ import com.yahoo.vespa.hosted.controller.tenant.AthenzTenant;
import com.yahoo.vespa.hosted.controller.tenant.Tenant;
import com.yahoo.vespa.hosted.controller.athenz.mock.AthenzClientFactoryMock;
import com.yahoo.vespa.hosted.controller.athenz.mock.AthenzDbMock;
-import com.yahoo.vespa.hosted.controller.deployment.MockBuildService;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService;
import com.yahoo.vespa.hosted.controller.integration.MockMetricsService;
import com.yahoo.vespa.hosted.controller.persistence.ApplicationSerializer;
import com.yahoo.vespa.hosted.controller.persistence.ControllerDb;
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java
index f6a4578f70b..4caf4645233 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java
@@ -13,6 +13,7 @@ import com.yahoo.vespa.hosted.controller.ConfigServerClientMock;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService;
import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.Change;
import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
@@ -258,10 +259,10 @@ public class DeploymentTester {
}
private void notifyJobCompletion(DeploymentJobs.JobReport report) {
- if (report.jobType() != JobType.component && ! buildService().removeJob(report.projectId(), report.jobType()))
+ if (report.jobType() != JobType.component && ! buildService().removeJob(report.projectId(), report.jobType().jobName()))
throw new IllegalArgumentException(report.jobType() + " is not running for " + report.applicationId());
assertFalse("Unexpected entry '" + report.jobType() + "@" + report.projectId() + " in: " + buildService().jobs(),
- buildService().removeJob(report.projectId(), report.jobType()));
+ buildService().removeJob(report.projectId(), report.jobType().jobName()));
clock().advance(Duration.ofMillis(1));
applications().deploymentTrigger().notifyOfCompletion(report);
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 701d5ed69db..097066dc847 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
@@ -3,12 +3,12 @@ package com.yahoo.vespa.hosted.controller.deployment;
import com.yahoo.component.Version;
import com.yahoo.config.provision.Environment;
-import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.TenantName;
import com.yahoo.test.ManualClock;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService;
import com.yahoo.vespa.hosted.controller.api.integration.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.ApplicationVersion;
@@ -22,7 +22,6 @@ import org.junit.Test;
import java.time.Duration;
import java.time.Instant;
-import java.util.Collections;
import java.util.Optional;
import java.util.function.Supplier;
@@ -34,6 +33,7 @@ import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobTy
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.productionUsWest1;
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.singletonList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
@@ -78,7 +78,9 @@ public class DeploymentTriggerTest {
tester.deployAndNotify(app, applicationPackage, true, JobType.systemTest);
// staging-test times out and is retried
- tester.buildService().takeJobsToRun();
+ synchronized (tester.buildService()) {
+ tester.buildService().clear();
+ }
tester.clock().advance(Duration.ofHours(12).plus(Duration.ofSeconds(1)));
tester.readyJobTrigger().maintain();
assertEquals("Retried dead job", 1, tester.buildService().jobs().size());
@@ -324,9 +326,8 @@ public class DeploymentTriggerTest {
tester.clock().advance(Duration.ofHours(2)); // ---------------- Exit block window: 20:30
tester.deploymentTrigger().triggerReadyJobs(); // Schedules the blocked production job(s)
- assertEquals(1, tester.buildService().jobs().size());
- BuildService.BuildJob productionJob = tester.buildService().takeJobsToRun().get(0);
- assertEquals("production-us-west-1", productionJob.jobName());
+ assertEquals(singletonList(new BuildService.BuildJob(app.deploymentJobs().projectId().get(), "production-us-west-1")),
+ tester.buildService().jobs());
}
@Test
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java
deleted file mode 100644
index e99e86f6c92..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java
+++ /dev/null
@@ -1,45 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.controller.deployment;
-
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * @author jvenstad
- */
-public class MockBuildService implements BuildService {
-
- private final List<BuildJob> jobs = new ArrayList<>();
-
- @Override
- public boolean trigger(BuildJob buildJob) {
- return jobs.add(buildJob);
- }
-
- @Override
- public boolean isRunning(BuildJob buildJob) {
- return jobs.contains(buildJob);
- }
-
- /** List all running jobs. */
- public List<BuildJob> jobs() {
- return new ArrayList<>(jobs);
- }
-
- /** List and remove all running jobs. */
- public List<BuildJob> takeJobsToRun() {
- List<BuildJob> jobsToRun = jobs();
- jobs.clear();
- return jobsToRun;
- }
-
- /** Remove all running jobs for the given project. */
- public boolean removeJob(long projectId, JobType jobType) {
- return jobs.remove(new BuildJob(projectId, jobType.jobName()));
- }
-
-}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/FailureRedeployerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/FailureRedeployerTest.java
index 59df3cd4553..2d76d395804 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/FailureRedeployerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/FailureRedeployerTest.java
@@ -8,7 +8,6 @@ import com.yahoo.slime.Slime;
import com.yahoo.vespa.config.SlimeUtils;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
-import com.yahoo.vespa.hosted.controller.api.integration.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder;
@@ -21,9 +20,7 @@ import java.time.Duration;
import java.util.Collections;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.component;
-import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.productionCdUsCentral1;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.productionUsEast3;
-import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.stagingTest;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.systemTest;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -95,36 +92,6 @@ public class FailureRedeployerTest {
assertTrue("All jobs consumed", tester.buildService().jobs().isEmpty());
assertFalse("No failures", tester.application(app.id()).deploymentJobs().hasFailures());
}
-
- @Test
- public void testRetriesDeploymentWithStuckJobs() {
- DeploymentTester tester = new DeploymentTester();
- ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .upgradePolicy("canary")
- .environment(Environment.prod)
- .region("us-east-3")
- .build();
-
- Application app = tester.createApplication("app1", "tenant1", 1, 11L);
- tester.jobCompletion(component).application(app).uploadArtifact(applicationPackage).submit();
- tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.systemTest);
-
- // staging-test starts, but does not complete
- assertEquals(DeploymentJobs.JobType.stagingTest.jobName(), tester.buildService().takeJobsToRun().get(0).jobName());
- tester.readyJobTrigger().maintain();
- assertTrue("No jobs retried", tester.buildService().jobs().isEmpty());
-
- // Just over 12 hours pass, job is retried
- tester.clock().advance(Duration.ofHours(12).plus(Duration.ofSeconds(1)));
- tester.readyJobTrigger().maintain();
- tester.assertRunning(app.id(), stagingTest);
-
- // Deployment completes
- tester.deployAndNotify(app, applicationPackage, true, stagingTest);
- tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsEast3);
- assertTrue("All jobs consumed", tester.buildService().jobs().isEmpty());
- }
-
@Test
public void testRetriesJobsFailingForCurrentChange() {
DeploymentTester tester = new DeploymentTester();
@@ -158,8 +125,10 @@ public class FailureRedeployerTest {
tester.updateVersionStatus(version);
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
- // Job is left "running", so needs to time out before it can be retried.
- tester.clock().advance(Duration.ofHours(13));
+ boolean result;
+ synchronized (tester.buildService()) {
+ result = tester.buildService().removeJob((long) 1, systemTest.jobName());
+ }
tester.upgrader().maintain();
tester.readyJobTrigger().maintain();
assertEquals("Application has pending upgrade to " + version, version, tester.application(app.id()).change().platform().get());
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 02672f06ea6..7de53d47fe4 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
@@ -123,8 +123,14 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
tester.upgrader().maintain();
- tester.buildService().removeJob(canary0.deploymentJobs().projectId().get(), stagingTest);
- tester.buildService().removeJob(canary1.deploymentJobs().projectId().get(), systemTest);
+ boolean result1;
+ synchronized (tester.buildService()) {
+ result1 = tester.buildService().removeJob(canary0.deploymentJobs().projectId().get(), stagingTest.jobName());
+ }
+ boolean result;
+ synchronized (tester.buildService()) {
+ result = tester.buildService().removeJob(canary1.deploymentJobs().projectId().get(), systemTest.jobName());
+ }
tester.readyJobTrigger().maintain();
tester.readyJobTrigger().maintain();
@@ -254,7 +260,9 @@ public class UpgraderTest {
tester.jobCompletion(DeploymentJobs.JobType.productionUsWest1).application(default3).unsuccessful().submit();
tester.upgrader().maintain();
- tester.buildService().takeJobsToRun();
+ synchronized (tester.buildService()) {
+ tester.buildService().clear();
+ }
tester.readyJobTrigger().maintain();
assertEquals("Upgrade of defaults are scheduled on 5.4 instead, since 5.5 broken: " +
"This is default3 since it failed upgrade on both 5.4 and 5.5",
@@ -339,7 +347,9 @@ public class UpgraderTest {
// > 40% and at least 4 failed - version is broken
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- tester.buildService().takeJobsToRun();
+ synchronized (tester.buildService()) {
+ tester.buildService().clear();
+ }
tester.readyJobTrigger().maintain();
assertEquals(VespaVersion.Confidence.broken, tester.controller().versionStatus().systemVersion().get().confidence());
assertEquals("Upgrades are cancelled", 0, tester.buildService().jobs().size());
@@ -393,12 +403,11 @@ public class UpgraderTest {
tester.jobCompletion(stagingTest).application(app).unsuccessful().submit();
assertTrue("Application still has failures", tester.application(app.id()).deploymentJobs().hasFailures());
assertEquals(1, tester.buildService().jobs().size());
- tester.buildService().takeJobsToRun();
// Upgrader runs again, nothing happens as there's already a job in progress for this change
tester.upgrader().maintain();
tester.readyJobTrigger().maintain();
- assertTrue("No more jobs triggered at this time", tester.buildService().jobs().isEmpty());
+ assertEquals(1, tester.buildService().jobs().size());
}
@Test
@@ -518,7 +527,9 @@ public class UpgraderTest {
tester.deploymentTrigger().cancelChange(default1.id(), false);
tester.deploymentTrigger().cancelChange(default2.id(), false);
tester.deploymentTrigger().cancelChange(default3.id(), false);
- tester.buildService().takeJobsToRun();
+ synchronized (tester.buildService()) {
+ tester.buildService().clear();
+ }
tester.clock().advance(Duration.ofHours(13)); // Currently we don't cancel running jobs, so this is necessary to allow a new triggering below
// Applications with default policy start upgrading to V2
@@ -542,7 +553,9 @@ public class UpgraderTest {
assertEquals(v2, tester.application("default0").deployments().get(ZoneId.from("prod.us-west-1")).version());
assertEquals(v0, tester.application("default0").deployments().get(ZoneId.from("prod.us-east-3")).version());
tester.upgrader().maintain();
- tester.buildService().takeJobsToRun();
+ synchronized (tester.buildService()) {
+ tester.buildService().clear();
+ }
tester.clock().advance(Duration.ofHours(13)); // TODO jvenstad: Reduce all these when build service is polled for status.
tester.readyJobTrigger().maintain();
tester.readyJobTrigger().maintain();
@@ -859,7 +872,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", deadLocked.deploymentJobs().isRunning(tester.controller().applications().deploymentTrigger().jobTimeoutLimit()));
+ assertTrue("Jobs in progress", tester.deploymentTrigger().isRunning(deadLocked, 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/ContainerControllerTester.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerControllerTester.java
index fc5e3e72f57..5184eeacc33 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerControllerTester.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerControllerTester.java
@@ -22,6 +22,7 @@ import com.yahoo.vespa.hosted.controller.api.identifiers.PropertyId;
import com.yahoo.vespa.hosted.controller.api.identifiers.ScrewdriverId;
import com.yahoo.vespa.hosted.controller.api.integration.athenz.ApplicationAction;
import com.yahoo.vespa.hosted.controller.api.integration.athenz.HostedAthenzIdentities;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService;
import com.yahoo.vespa.hosted.controller.api.integration.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.tenant.AthenzTenant;
@@ -39,6 +40,8 @@ import java.io.IOException;
import java.time.Duration;
import java.util.Optional;
+import static org.junit.Assert.assertFalse;
+
/**
* Provides testing of controller functionality accessed through the container
*
@@ -119,13 +122,19 @@ public class ContainerControllerTester {
.addRoleMember(action, HostedAthenzIdentities.from(screwdriverId));
}
- private void notifyJobCompletion(DeploymentJobs.JobReport jobReport) {
+ private void notifyJobCompletion(DeploymentJobs.JobReport report) {
+
+ MockBuildService buildService = (MockBuildService) containerTester.container().components().getComponent(MockBuildService.class.getName());
+ if (report.jobType() != DeploymentJobs.JobType.component && ! buildService.removeJob(report.projectId(), report.jobType().jobName()))
+ throw new IllegalArgumentException(report.jobType() + " is not running for " + report.applicationId());
+ assertFalse("Unexpected entry '" + report.jobType() + "@" + report.projectId() + " in: " + buildService.jobs(),
+ buildService.removeJob(report.projectId(), report.jobType().jobName()));
try {
Thread.sleep(1);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
- controller().applications().deploymentTrigger().notifyOfCompletion(jobReport);
+ controller().applications().deploymentTrigger().notifyOfCompletion(report);
controller().applications().deploymentTrigger().triggerReadyJobs();
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiTest.java
index 743baf76759..f52cc9e9e11 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiTest.java
@@ -42,7 +42,6 @@ public class DeploymentApiTest extends ControllerContainerTest {
ContainerControllerTester tester = new ContainerControllerTester(container, responseFiles);
Version version = Version.fromString("5.0");
tester.containerTester().updateSystemVersion(version);
- long projectId = 11;
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
.environment(Environment.prod)
.region("corp-us-east-1")
@@ -55,11 +54,11 @@ public class DeploymentApiTest extends ControllerContainerTest {
"application2");
Application applicationWithoutDeployment = tester.createApplication("domain3", "tenant3",
"application3");
- deployCompletely(failingApplication, applicationPackage, projectId, true);
- deployCompletely(productionApplication, applicationPackage, projectId, true);
+ deployCompletely(failingApplication, applicationPackage, 1L, true);
+ deployCompletely(productionApplication, applicationPackage, 2L, true);
// Deploy once so that job information is stored, then remove the deployment
- deployCompletely(applicationWithoutDeployment, applicationPackage, projectId, true);
+ deployCompletely(applicationWithoutDeployment, applicationPackage, 3L, true);
tester.controller().applications().deactivate(applicationWithoutDeployment,
ZoneId.from("prod", "corp-us-east-1"));
@@ -70,8 +69,9 @@ public class DeploymentApiTest extends ControllerContainerTest {
// Applications upgrade, 1/2 succeed
tester.upgrader().maintain();
tester.controller().applications().deploymentTrigger().triggerReadyJobs();
- deployCompletely(failingApplication, applicationPackage, projectId, false);
- deployCompletely(productionApplication, applicationPackage, projectId, true);
+ tester.controller().applications().deploymentTrigger().triggerReadyJobs();
+ deployCompletely(failingApplication, applicationPackage, 1L, false);
+ deployCompletely(productionApplication, applicationPackage, 2L, true);
tester.controller().updateVersionStatus(censorConfigServers(VersionStatus.compute(tester.controller()),
tester.controller()));