aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorArnstein Ressem <aressem@gmail.com>2018-01-05 15:13:20 +0100
committerGitHub <noreply@github.com>2018-01-05 15:13:20 +0100
commit2d8e42a5cf3e81eb94439f5fb26a52b52de6da03 (patch)
treee7ad98feb133e69f728f4ad856ec43516ed00491
parentc1c186a34500e3eacd9d813eba4138b6cdd3e428 (diff)
parente65ca655fd3b8a6293fba56d031a973874452412 (diff)
Merge pull request #4560 from vespa-engine/revert-4556-jvenstad/pushing-build-system
Revert "Jvenstad/pushing build system"
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java25
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/stubs/MockBuildService.java15
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/BuildSystem.java34
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueue.java96
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java10
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerer.java53
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystem.java117
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java7
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Maintainer.java2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java24
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/JobQueueSerializer.java5
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiHandler.java11
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java36
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueueTest.java65
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java10
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java72
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggererTest.java86
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java178
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimeline.java106
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystemTest.java68
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/FailureRedeployerTest.java46
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OutstandingChangeDeployerTest.java8
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java100
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ControllerContainerTest.java1
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/controller/responses/maintenance.json3
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiTest.java25
26 files changed, 699 insertions, 504 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 5e9c8d73b38..7933a23c45f 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
@@ -1,21 +1,18 @@
// 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.api.integration;
-import java.util.Objects;
-
/**
* @author jvenstad
*/
public interface BuildService {
/**
- * Enqueue a job defined by buildJob in an external build system, and return 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
+ * Enqueue a job defined by "buildJob in an external build system, and return 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.
*/
boolean trigger(BuildJob buildJob);
-
class BuildJob {
private final long projectId;
@@ -30,23 +27,7 @@ public interface BuildService {
public String jobName() { return jobName; }
@Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if ( ! (o instanceof BuildJob)) return false;
- BuildJob buildJob = (BuildJob) o;
- return projectId == buildJob.projectId &&
- Objects.equals(jobName, buildJob.jobName);
- }
-
- @Override
- public String toString() {
- return jobName + "@" + projectId;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(projectId, jobName);
- }
+ public String toString() { return jobName + "@" + projectId; }
}
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 786a26a2330..00000000000
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/stubs/MockBuildService.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package com.yahoo.vespa.hosted.controller.api.integration.stubs;
-
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
-
-/**
- * @author jvenstad
- */
-public class MockBuildService implements BuildService {
-
- @Override
- public boolean trigger(BuildJob buildJob) {
- return true;
- }
-
-}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/BuildSystem.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/BuildSystem.java
new file mode 100644
index 00000000000..15b3ef7fb83
--- /dev/null
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/BuildSystem.java
@@ -0,0 +1,34 @@
+// 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.config.provision.ApplicationId;
+import com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
+import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
+
+import java.util.List;
+
+/**
+ * @author jvenstad
+ * @author mpolden
+ */
+public interface BuildSystem {
+
+ /**
+ * Add a job for the given application to the build system
+ *
+ * @param application the application owning the job
+ * @param jobType the job type to be queued
+ * @param first whether the job should be added to the front of the queue
+ */
+ void addJob(ApplicationId application, JobType jobType, boolean first);
+
+ /** Remove and return a list of jobs which should be run now */
+ List<BuildJob> takeJobsToRun();
+
+ /** Get a list of all jobs currently waiting to run */
+ List<BuildJob> jobs();
+
+ /** Removes all queued jobs for the given application */
+ void removeJobs(ApplicationId applicationId);
+
+}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueue.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueue.java
deleted file mode 100644
index 06d7d72a2f3..00000000000
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueue.java
+++ /dev/null
@@ -1,96 +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.google.common.collect.ImmutableList;
-import com.yahoo.config.provision.ApplicationId;
-import com.yahoo.searchlib.rankingexpression.rule.Function;
-import com.yahoo.vespa.curator.Lock;
-import com.yahoo.vespa.hosted.controller.Controller;
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
-import com.yahoo.vespa.hosted.controller.persistence.CuratorDb;
-
-import java.util.Deque;
-import java.util.List;
-import java.util.Optional;
-import java.util.function.Consumer;
-
-/**
- * Stores a queue for each type of job, and offers jobs from each of these to a periodic
- * polling mechanism which is responsible for triggering the offered jobs in an external build service.
- *
- * @author jvenstad
- */
-public class DeploymentQueue {
-
- private final Controller controller;
- private final CuratorDb curator;
-
- public DeploymentQueue(Controller controller, CuratorDb curator) {
- this.controller = controller;
- this.curator = curator;
- }
-
- /** Add the given application to the queue of the given job type -- in front if first, at the back otherwise. */
- public void addJob(ApplicationId applicationId, JobType jobType, boolean first) {
- locked(jobType, queue -> {
- if ( ! queue.contains(applicationId)) {
- if (first)
- queue.addFirst(applicationId);
- else
- queue.addLast(applicationId);
- }
- });
- }
-
- /** List all jobs currently enqueued. */
- public List<BuildJob> jobs() {
- ImmutableList.Builder<BuildJob> builder = ImmutableList.builder();
- for (JobType jobType : JobType.values())
- for (ApplicationId id : curator.readJobQueue(jobType))
- toBuildJob(id, jobType).ifPresent(builder::add);
-
- return builder.build();
- }
-
- /** Remove and return a set of jobs to run. This set will contain only one of each job type for capacity constrained zones. */
- public List<BuildJob> takeJobsToRun() {
- ImmutableList.Builder<BuildJob> builder = ImmutableList.builder();
- for (JobType jobType : JobType.values())
- locked(jobType, queue ->
- queue.stream()
- .limit(isCapacityConstrained(jobType) ? 1 : 1 << 30)
- .peek(id -> toBuildJob(id, jobType).ifPresent(builder::add))
- .forEach(queue::remove));
-
- return builder.build();
- }
-
- /** Remove all enqueued jobs for the given application. */
- public void removeJobs(ApplicationId applicationId) {
- for (JobType jobType : JobType.values())
- locked(jobType, queue -> {
- while (queue.remove(applicationId)); // Keep removing until not found.
- });
- }
-
- /** Lock the job queues and read, modify, and store the queue for the given job type. */
- private void locked(JobType jobType, Consumer<Deque<ApplicationId>> modifications) {
- try (Lock lock = curator.lockJobQueues()) {
- Deque<ApplicationId> queue = curator.readJobQueue(jobType);
- modifications.accept(queue);
- curator.writeJobQueue(jobType, queue);
- }
- }
-
- private static boolean isCapacityConstrained(JobType jobType) {
- return jobType == JobType.stagingTest || jobType == JobType.systemTest;
- }
-
- private Optional<BuildJob> toBuildJob(ApplicationId applicationId, JobType jobType) {
- return controller.applications().get(applicationId)
- .flatMap(application -> application.deploymentJobs().projectId())
- .map(projectId -> new BuildJob(projectId, jobType.jobName()));
- }
-
-}
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 5cfeb155a43..90237a17fb9 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
@@ -47,7 +47,7 @@ public class DeploymentTrigger {
private final Controller controller;
private final Clock clock;
- private final DeploymentQueue deploymentQueue;
+ private final BuildSystem buildSystem;
private final DeploymentOrder order;
public DeploymentTrigger(Controller controller, CuratorDb curator, Clock clock) {
@@ -56,7 +56,7 @@ public class DeploymentTrigger {
Objects.requireNonNull(clock,"clock cannot be null");
this.controller = controller;
this.clock = clock;
- this.deploymentQueue = new DeploymentQueue(controller, curator);
+ this.buildSystem = new PolledBuildSystem(controller, curator);
this.order = new DeploymentOrder(controller);
this.jobTimeout = controller.system().equals(SystemName.main) ? Duration.ofHours(12) : Duration.ofHours(1);
}
@@ -64,7 +64,7 @@ public class DeploymentTrigger {
/** Returns the time in the past before which jobs are at this moment considered unresponsive */
public Instant jobTimeoutLimit() { return clock.instant().minus(jobTimeout); }
- public DeploymentQueue deploymentQueue() { return deploymentQueue; }
+ public BuildSystem buildSystem() { return buildSystem; }
public DeploymentOrder deploymentOrder() { return order; }
@@ -238,7 +238,7 @@ public class DeploymentTrigger {
*/
public void cancelChange(ApplicationId applicationId) {
applications().lockOrThrow(applicationId, application -> {
- deploymentQueue.removeJobs(application.id());
+ buildSystem.removeJobs(application.id());
applications().store(application.withDeploying(Optional.empty()));
});
}
@@ -313,7 +313,7 @@ public class DeploymentTrigger {
log.info(String.format("Triggering %s for %s, %s: %s", jobType, application,
application.deploying().map(d -> "deploying " + d).orElse("restarted deployment"),
reason));
- deploymentQueue.addJob(application.id(), jobType, first);
+ buildSystem.addJob(application.id(), jobType, first);
return application.withJobTriggering(jobType,
application.deploying(),
clock.instant(),
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerer.java
deleted file mode 100644
index 42824073f1c..00000000000
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerer.java
+++ /dev/null
@@ -1,53 +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.Controller;
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
-import com.yahoo.vespa.hosted.controller.maintenance.JobControl;
-import com.yahoo.vespa.hosted.controller.maintenance.Maintainer;
-
-import java.time.Duration;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Triggers deployment jobs in an external BuildService.
- *
- * Triggering is performed by an Executor, as there is no guarantee the BuildService provides a timely response.
- *
- * @author jvenstad
- */
-public class DeploymentTriggerer extends Maintainer {
-
- private static final Logger log = Logger.getLogger(DeploymentTriggerer.class.getName());
- static final int triggeringRetries = 5;
-
- private final BuildService buildService;
- private final Executor executor;
-
- public DeploymentTriggerer(Controller controller, Duration triggeringInterval, JobControl jobControl, BuildService buildService) {
- this(controller, triggeringInterval, jobControl, buildService, Executors.newFixedThreadPool(20));
- }
-
- DeploymentTriggerer(Controller controller, Duration triggeringInterval, JobControl jobControl,
- BuildService buildService, Executor executor) {
- super(controller, triggeringInterval, jobControl);
- this.buildService = buildService;
- this.executor = executor;
- }
-
- @Override
- protected void maintain() {
- controller().applications().deploymentTrigger().deploymentQueue().takeJobsToRun()
- .forEach(buildJob -> executor.execute(() -> {
- for (int i = 0; i < triggeringRetries; i++)
- if (buildService.trigger(buildJob))
- return;
-
- log.log(Level.WARNING, "Exhausted all " + triggeringRetries + " retries for " + buildJob + " without success.");
- }));
- }
-
-}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystem.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystem.java
new file mode 100644
index 00000000000..e25db10a8cd
--- /dev/null
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystem.java
@@ -0,0 +1,117 @@
+// 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.config.provision.ApplicationId;
+import com.yahoo.vespa.curator.Lock;
+import com.yahoo.vespa.hosted.controller.Controller;
+import com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
+import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
+import com.yahoo.vespa.hosted.controller.persistence.CuratorDb;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import java.util.Optional;
+import java.util.logging.Logger;
+
+/**
+ * Stores a queue for each type of job, and offers jobs from each of these to a periodic
+ * polling mechanism which is responsible for triggering the offered jobs in an external build service.
+ *
+ * @author jvenstad
+ * @author mpolden
+ */
+public class PolledBuildSystem implements BuildSystem {
+
+ private static final Logger log = Logger.getLogger(PolledBuildSystem.class.getName());
+
+ // The number of jobs to offer, on each poll, for zones that have limited capacity
+ private static final int maxCapacityConstrainedJobsToOffer = 2;
+
+ private final Controller controller;
+ private final CuratorDb curator;
+
+ public PolledBuildSystem(Controller controller, CuratorDb curator) {
+ this.controller = controller;
+ this.curator = curator;
+ }
+
+ @Override
+ public void addJob(ApplicationId application, JobType jobType, boolean first) {
+ try (Lock lock = curator.lockJobQueues()) {
+ Deque<ApplicationId> queue = curator.readJobQueue(jobType);
+ if ( ! queue.contains(application)) {
+ if (first) {
+ queue.addFirst(application);
+ } else {
+ queue.add(application);
+ }
+ }
+ curator.writeJobQueue(jobType, queue);
+ }
+ }
+
+ @Override
+ public List<BuildJob> jobs() {
+ return getJobs(false);
+ }
+
+ @Override
+ public List<BuildJob> takeJobsToRun() {
+ return getJobs(true);
+ }
+
+
+ @Override
+ public void removeJobs(ApplicationId application) {
+ try (Lock lock = curator.lockJobQueues()) {
+ for (JobType jobType : JobType.values()) {
+ Deque<ApplicationId> queue = curator.readJobQueue(jobType);
+ while (queue.remove(application)) {
+ // keep removing until not found
+ }
+ curator.writeJobQueue(jobType, queue);
+ }
+ }
+ }
+
+ private List<BuildJob> getJobs(boolean removeFromQueue) {
+ int capacityConstrainedJobsOffered = 0;
+ try (Lock lock = curator.lockJobQueues()) {
+ List<BuildJob> jobsToRun = new ArrayList<>();
+ for (JobType jobType : JobType.values()) {
+ Deque<ApplicationId> queue = curator.readJobQueue(jobType);
+ for (ApplicationId a : queue) {
+ ApplicationId application = removeFromQueue ? queue.poll() : a;
+
+ Optional<Long> projectId = projectId(application);
+ if (projectId.isPresent()) {
+ jobsToRun.add(new BuildJob(projectId.get(), jobType.jobName()));
+ } else {
+ log.warning("Not queuing " + jobType.jobName() + " for " + application.toShortString() +
+ " because project ID is missing");
+ }
+
+ // Return a limited number of jobs at a time for capacity constrained zones
+ if (removeFromQueue && isCapacityConstrained(jobType) &&
+ ++capacityConstrainedJobsOffered >= maxCapacityConstrainedJobsToOffer) {
+ break;
+ }
+ }
+ if (removeFromQueue)
+ curator.writeJobQueue(jobType, queue);
+ }
+ return Collections.unmodifiableList(jobsToRun);
+ }
+ }
+
+ private Optional<Long> projectId(ApplicationId applicationId) {
+ return controller.applications().require(applicationId).deploymentJobs().projectId();
+ }
+
+ private static boolean isCapacityConstrained(JobType jobType) {
+ return jobType == JobType.stagingTest || jobType == JobType.systemTest;
+ }
+
+}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java
index bac388d3155..4e9dd94d8e5 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java
@@ -5,11 +5,9 @@ import com.yahoo.component.AbstractComponent;
import com.yahoo.jdisc.Metric;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.api.integration.dns.NameService;
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
import com.yahoo.vespa.hosted.controller.api.integration.organization.OwnershipIssues;
import com.yahoo.vespa.hosted.controller.api.integration.organization.DeploymentIssues;
import com.yahoo.vespa.hosted.controller.api.integration.chef.Chef;
-import com.yahoo.vespa.hosted.controller.deployment.DeploymentTriggerer;
import com.yahoo.vespa.hosted.controller.maintenance.config.MaintainerConfig;
import com.yahoo.vespa.hosted.controller.persistence.CuratorDb;
@@ -38,13 +36,12 @@ public class ControllerMaintenance extends AbstractComponent {
private final DeploymentMetricsMaintainer deploymentMetricsMaintainer;
private final ApplicationOwnershipConfirmer applicationOwnershipConfirmer;
private final DnsMaintainer dnsMaintainer;
- private final DeploymentTriggerer deploymentTriggerer;
@SuppressWarnings("unused") // instantiated by Dependency Injection
public ControllerMaintenance(MaintainerConfig maintainerConfig, Controller controller, CuratorDb curator,
JobControl jobControl, Metric metric, Chef chefClient,
DeploymentIssues deploymentIssues, OwnershipIssues ownershipIssues,
- NameService nameService, BuildService buildService) {
+ NameService nameService) {
Duration maintenanceInterval = Duration.ofMinutes(maintainerConfig.intervalMinutes());
this.jobControl = jobControl;
deploymentExpirer = new DeploymentExpirer(controller, maintenanceInterval, jobControl);
@@ -59,7 +56,6 @@ public class ControllerMaintenance extends AbstractComponent {
deploymentMetricsMaintainer = new DeploymentMetricsMaintainer(controller, Duration.ofMinutes(10), jobControl);
applicationOwnershipConfirmer = new ApplicationOwnershipConfirmer(controller, Duration.ofHours(12), jobControl, ownershipIssues);
dnsMaintainer = new DnsMaintainer(controller, Duration.ofHours(1), jobControl, nameService);
- deploymentTriggerer = new DeploymentTriggerer(controller, Duration.ofSeconds(30), jobControl, buildService);
}
public Upgrader upgrader() { return upgrader; }
@@ -81,7 +77,6 @@ public class ControllerMaintenance extends AbstractComponent {
deploymentMetricsMaintainer.deconstruct();
applicationOwnershipConfirmer.deconstruct();
dnsMaintainer.deconstruct();
- deploymentTriggerer.deconstruct();
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Maintainer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Maintainer.java
index 40563c4cf95..ebab2054d4f 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Maintainer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Maintainer.java
@@ -41,6 +41,8 @@ public abstract class Maintainer extends AbstractComponent implements Runnable {
protected Controller controller() { return controller; }
+ protected CuratorDb curator() { return jobControl.curator(); }
+
@Override
public void run() {
try {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java
index caf2af5114c..a3bb191fc38 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java
@@ -123,29 +123,33 @@ public class CuratorDb {
}
public void writeInactiveJobs(Set<String> inactiveJobs) {
+ NestedTransaction transaction = new NestedTransaction();
curator.set(inactiveJobsPath(), stringSetSerializer.toJson(inactiveJobs));
+ transaction.commit();
}
public Deque<ApplicationId> readJobQueue(DeploymentJobs.JobType jobType) {
try {
Optional<byte[]> data = curator.getData(jobQueuePath(jobType));
- if ( ! data.isPresent() || data.get().length == 0) return new ArrayDeque<>(); // job queue has never been written
+ if (! data.isPresent() || data.get().length == 0) return new ArrayDeque<>(); // job queue has never been written
return jobQueueSerializer.fromJson(data.get());
}
catch (RuntimeException e) {
- log.log(Level.WARNING, "Error reading job queue of type '" + jobType.jobName() + "'; deleting it.");
- writeJobQueue(jobType, Collections::emptyIterator);
+ log.log(Level.WARNING, "Error reading job queue, deleting inactive state");
+ writeInactiveJobs(Collections.emptySet());
return new ArrayDeque<>();
}
}
- public void writeJobQueue(DeploymentJobs.JobType jobType, Iterable<ApplicationId> queue) {
+ public void writeJobQueue(DeploymentJobs.JobType jobType, Deque<ApplicationId> queue) {
+ NestedTransaction transaction = new NestedTransaction();
curator.set(jobQueuePath(jobType), jobQueueSerializer.toJson(queue));
+ transaction.commit();
}
public double readUpgradesPerMinute() {
Optional<byte[]> n = curator.getData(upgradesPerMinutePath());
- if ( ! n.isPresent() || n.get().length == 0) {
+ if (!n.isPresent() || n.get().length == 0) {
return 0.5; // Default if value has never been written
}
return ByteBuffer.wrap(n.get()).getDouble();
@@ -155,33 +159,39 @@ public class CuratorDb {
if (n < 0) {
throw new IllegalArgumentException("Upgrades per minute must be >= 0");
}
+ NestedTransaction transaction = new NestedTransaction();
curator.set(upgradesPerMinutePath(), ByteBuffer.allocate(Double.BYTES).putDouble(n).array());
+ transaction.commit();
}
public boolean readIgnoreConfidence() {
Optional<byte[]> value = curator.getData(ignoreConfidencePath());
- if ( ! value.isPresent() || value.get().length == 0) {
+ if (! value.isPresent() || value.get().length == 0) {
return false; // Default if value has never been written
}
return ByteBuffer.wrap(value.get()).getInt() == 1;
}
public void writeIgnoreConfidence(boolean value) {
+ NestedTransaction transaction = new NestedTransaction();
curator.set(ignoreConfidencePath(), ByteBuffer.allocate(Integer.BYTES).putInt(value ? 1 : 0).array());
+ transaction.commit();
}
public void writeVersionStatus(VersionStatus status) {
VersionStatusSerializer serializer = new VersionStatusSerializer();
+ NestedTransaction transaction = new NestedTransaction();
try {
curator.set(versionStatusPath(), SlimeUtils.toJsonBytes(serializer.toSlime(status)));
} catch (IOException e) {
throw new UncheckedIOException("Failed to serialize version status", e);
}
+ transaction.commit();
}
public VersionStatus readVersionStatus() {
Optional<byte[]> data = curator.getData(versionStatusPath());
- if ( ! data.isPresent() || data.get().length == 0) {
+ if (!data.isPresent() || data.get().length == 0) {
return VersionStatus.empty(); // Default if status has never been written
}
VersionStatusSerializer serializer = new VersionStatusSerializer();
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/JobQueueSerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/JobQueueSerializer.java
index f6d6406a820..5017624f286 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/JobQueueSerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/JobQueueSerializer.java
@@ -12,19 +12,18 @@ import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Deque;
import java.util.HashSet;
-import java.util.List;
import java.util.Set;
/**
* Serialization of a queue of ApplicationIds to/from Json bytes using Slime.
*
- * The queue is serialized as an array of strings.
+ * The set is serialized as an array of string.
*
* @author bratseth
*/
public class JobQueueSerializer {
- public byte[] toJson(Iterable<ApplicationId> queue) {
+ public byte[] toJson(Deque<ApplicationId> queue) {
try {
Slime slime = new Slime();
Cursor array = slime.setArray();
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiHandler.java
index 56f84189092..4d4f01bc1a6 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiHandler.java
@@ -59,6 +59,7 @@ public class ScrewdriverApiHandler extends LoggingRequestHandler {
switch (method) {
case GET: return get(request);
case POST: return post(request);
+ case DELETE: return delete(request);
default: return ErrorResponse.methodNotAllowed("Method '" + method + "' is unsupported");
}
} catch (IllegalArgumentException|IllegalStateException e) {
@@ -75,7 +76,7 @@ public class ScrewdriverApiHandler extends LoggingRequestHandler {
return vespaVersion();
}
if (path.matches("/screwdriver/v1/jobsToRun")) {
- return buildJobs(controller.applications().deploymentTrigger().deploymentQueue().jobs());
+ return buildJobs(controller.applications().deploymentTrigger().buildSystem().jobs());
}
return notFound(request);
}
@@ -91,6 +92,14 @@ public class ScrewdriverApiHandler extends LoggingRequestHandler {
return notFound(request);
}
+ private HttpResponse delete(HttpRequest request) {
+ Path path = new Path(request.getUri().getPath());
+ if (path.matches("/screwdriver/v1/jobsToRun")) {
+ return buildJobs(controller.applications().deploymentTrigger().buildSystem().takeJobsToRun());
+ }
+ return notFound(request);
+ }
+
private HttpResponse trigger(HttpRequest request, String tenantName, String applicationName) {
JobType jobType = Optional.of(asString(request.getData()))
.filter(s -> !s.isEmpty())
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 a803832011e..17801bde546 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
@@ -34,7 +34,7 @@ import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
import com.yahoo.vespa.hosted.controller.application.JobStatus;
import com.yahoo.vespa.hosted.controller.athenz.mock.AthenzDbMock;
import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder;
-import com.yahoo.vespa.hosted.controller.deployment.DeploymentQueue;
+import com.yahoo.vespa.hosted.controller.deployment.BuildSystem;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester;
import com.yahoo.vespa.hosted.controller.persistence.ApplicationSerializer;
import com.yahoo.vespa.hosted.controller.rotation.RotationId;
@@ -434,7 +434,7 @@ public class ControllerTest {
Application app1 = tester.createApplication("app1", "tenant1", project1, 1L);
Application app2 = tester.createApplication("app2", "tenant2", project2, 1L);
Application app3 = tester.createApplication("app3", "tenant3", project3, 1L);
- DeploymentQueue deploymentQueue = tester.controller().applications().deploymentTrigger().deploymentQueue();
+ BuildSystem buildSystem = tester.controller().applications().deploymentTrigger().buildSystem();
// all applications: system-test completes successfully
tester.notifyJobCompletion(component, app1, true);
@@ -447,13 +447,13 @@ public class ControllerTest {
tester.deployAndNotify(app3, applicationPackage, true, systemTest);
// all applications: staging test jobs queued
- assertEquals(3, deploymentQueue.jobs().size());
+ assertEquals(3, buildSystem.jobs().size());
// app1: staging-test job fails with out of capacity and is added to the front of the queue
tester.deploy(stagingTest, app1, applicationPackage);
tester.notifyJobCompletion(stagingTest, app1, Optional.of(JobError.outOfCapacity));
- assertEquals(stagingTest.jobName(), deploymentQueue.jobs().get(0).jobName());
- assertEquals(project1, deploymentQueue.jobs().get(0).projectId());
+ assertEquals(stagingTest.jobName(), buildSystem.jobs().get(0).jobName());
+ assertEquals(project1, buildSystem.jobs().get(0).projectId());
// app2 and app3: Completes deployment
tester.deployAndNotify(app2, applicationPackage, true, stagingTest);
@@ -468,9 +468,9 @@ public class ControllerTest {
tester.notifyJobCompletion(component, app1, true);
tester.deployAndNotify(app1, applicationPackage, true, false, systemTest);
tester.deploy(stagingTest, app1, applicationPackage);
- assertEquals(1, deploymentQueue.takeJobsToRun().size());
+ assertEquals(1, buildSystem.takeJobsToRun().size());
tester.notifyJobCompletion(stagingTest, app1, Optional.of(JobError.outOfCapacity));
- assertTrue("No jobs queued", deploymentQueue.jobs().isEmpty());
+ assertTrue("No jobs queued", buildSystem.jobs().isEmpty());
// app2 and app3: New change triggers system-test jobs
// Provide a changed application package, too, or the deployment is a no-op.
@@ -480,18 +480,26 @@ public class ControllerTest {
tester.notifyJobCompletion(component, app3, true);
tester.deployAndNotify(app3, applicationPackage2, true, systemTest);
- assertEquals(2, deploymentQueue.jobs().size());
+ assertEquals(2, buildSystem.jobs().size());
- // app1: 4 hours pass in total, staging-test job for app1 is re-queued by periodic trigger mechanism and added at the
+ // app1: 4 hours pass in total, staging-test job is re-queued by periodic trigger mechanism and added at the
// back of the queue
tester.clock().advance(Duration.ofHours(3));
tester.clock().advance(Duration.ofMinutes(50));
tester.readyJobTrigger().maintain();
- assertEquals(Collections.singletonList(new BuildJob(project2, stagingTest.jobName())), deploymentQueue.takeJobsToRun());
- assertEquals(Collections.singletonList(new BuildJob(project3, stagingTest.jobName())), deploymentQueue.takeJobsToRun());
- assertEquals(Collections.singletonList(new BuildJob(project1, stagingTest.jobName())), deploymentQueue.takeJobsToRun());
- assertEquals(Collections.emptyList(), deploymentQueue.takeJobsToRun());
+ List<BuildJob> nextJobs = buildSystem.takeJobsToRun();
+ assertEquals(2, nextJobs.size());
+ assertEquals(stagingTest.jobName(), nextJobs.get(0).jobName());
+ assertEquals(project2, nextJobs.get(0).projectId());
+ assertEquals(stagingTest.jobName(), nextJobs.get(1).jobName());
+ assertEquals(project3, nextJobs.get(1).projectId());
+
+ // And finally the requeued job for app1
+ nextJobs = buildSystem.takeJobsToRun();
+ assertEquals(1, nextJobs.size());
+ assertEquals(stagingTest.jobName(), nextJobs.get(0).jobName());
+ assertEquals(project1, nextJobs.get(0).projectId());
}
private void assertStatus(JobStatus expectedStatus, ApplicationId id, Controller controller) {
@@ -589,7 +597,7 @@ public class ControllerTest {
// Test environments pass
tester.deploy(DeploymentJobs.JobType.systemTest, application, applicationPackage);
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofMinutes(10));
tester.notifyJobCompletion(DeploymentJobs.JobType.systemTest, application, true);
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueueTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueueTest.java
deleted file mode 100644
index bd2250b4402..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueueTest.java
+++ /dev/null
@@ -1,65 +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.config.provision.ApplicationId;
-import com.yahoo.vespa.hosted.controller.ControllerTester;
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
-import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
-import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * @author jvenstad
- */
-public class DeploymentQueueTest {
-
- @Test
- public void testJobOffering() {
- DeploymentTester tester = new DeploymentTester();
- DeploymentQueue deploymentQueue = new DeploymentQueue(tester.controller(), tester.controller().curator());
-
- int project1 = 1;
- int project2 = 2;
- int project3 = 3;
-
- ApplicationId app1 = tester.createApplication("app1", "tenant", project1, null).id();
- ApplicationId app2 = tester.createApplication("app2", "tenant", project2, null).id();
- ApplicationId app3 = tester.createApplication("app3", "tenant", project3, null).id();
-
- // Trigger jobs in capacity constrained environment
- deploymentQueue.addJob(app1, JobType.systemTest, false);
- deploymentQueue.addJob(app2, JobType.systemTest, true);
- deploymentQueue.addJob(app3, JobType.stagingTest, false);
-
- // Trigger jobs in non-capacity constrained environment
- deploymentQueue.addJob(app1, JobType.productionUsWest1, false);
- deploymentQueue.addJob(app2, JobType.productionUsWest1, false);
- deploymentQueue.addJob(app3, JobType.productionUsWest1, false);
-
- assertEquals("Each offer contains a single job from each capacity constrained environment, and all other jobs.",
- Arrays.asList(new BuildJob(project2, JobType.systemTest.jobName()),
- new BuildJob(project3, JobType.stagingTest.jobName()),
- new BuildJob(project1, JobType.productionUsWest1.jobName()),
- new BuildJob(project2, JobType.productionUsWest1.jobName()),
- new BuildJob(project3, JobType.productionUsWest1.jobName())),
- deploymentQueue.takeJobsToRun());
-
- assertEquals("The system test job for project 1 was pushed back in the queue by that for project 2.",
- Collections.singletonList(new BuildJob(project1, JobType.systemTest.jobName())),
- deploymentQueue.takeJobsToRun());
-
- assertEquals("No jobs are left.",
- Collections.emptyList(),
- deploymentQueue.takeJobsToRun());
- }
-
-}
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 f3b3b8b0a0c..2b0e953c12c 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
@@ -69,7 +69,9 @@ public class DeploymentTester {
public ApplicationController applications() { return tester.controller().applications(); }
- public DeploymentQueue deploymentQueue() { return tester.controller().applications().deploymentTrigger().deploymentQueue(); }
+ // TODO: This thing simulates the wrong thing: the build system won't hold the jobs that are running,
+ // and so these should be consumed immediately upon triggering, and be "somewhere else" while running.
+ public BuildSystem buildSystem() { return tester.controller().applications().deploymentTrigger().buildSystem(); }
public DeploymentTrigger deploymentTrigger() { return tester.controller().applications().deploymentTrigger(); }
@@ -258,18 +260,18 @@ public class DeploymentTester {
}
if (expectOnlyTheseJobs)
assertEquals(jobs.length, countJobsOf(application));
- deploymentQueue().removeJobs(application.id());
+ buildSystem().removeJobs(application.id());
}
private BuildService.BuildJob findJob(Application application, JobType jobType) {
- for (BuildService.BuildJob job : deploymentQueue().jobs())
+ for (BuildService.BuildJob job : buildSystem().jobs())
if (job.projectId() == application.deploymentJobs().projectId().get() && job.jobName().equals(jobType.jobName()))
return job;
throw new NoSuchElementException(jobType + " is not scheduled for " + application);
}
private int countJobsOf(Application application) {
- return (int) deploymentQueue().jobs().stream()
+ return (int)buildSystem().jobs().stream()
.filter(job -> job.projectId() == application.deploymentJobs().projectId().get())
.count();
}
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 a0faf6403d8..10f8e80f318 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
@@ -58,28 +58,28 @@ public class DeploymentTriggerTest {
// system-test fails and is retried
tester.deployAndNotify(app, applicationPackage, false, JobType.systemTest);
- assertEquals("Retried immediately", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Retried immediately", 1, tester.buildSystem().jobs().size());
tester.clock().advance(Duration.ofHours(1));
tester.deployAndNotify(app, applicationPackage, false, JobType.systemTest);
tester.clock().advance(Duration.ofHours(1));
- assertEquals("Nothing scheduled", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("Nothing scheduled", 0, tester.buildSystem().jobs().size());
tester.readyJobTrigger().maintain(); // Causes retry of systemTests
- assertEquals("Scheduled retry", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Scheduled retry", 1, tester.buildSystem().jobs().size());
tester.deployAndNotify(app, applicationPackage, true, JobType.systemTest);
// staging-test times out and is retried
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofHours(12).plus(Duration.ofSeconds(1)));
tester.readyJobTrigger().maintain();
- assertEquals("Retried dead job", 1, tester.deploymentQueue().jobs().size());
- assertEquals(JobType.stagingTest.jobName(), tester.deploymentQueue().jobs().get(0).jobName());
+ assertEquals("Retried dead job", 1, tester.buildSystem().jobs().size());
+ assertEquals(JobType.stagingTest.jobName(), tester.buildSystem().jobs().get(0).jobName());
}
@Test
public void deploymentSpecDecidesTriggerOrder() {
DeploymentTester tester = new DeploymentTester();
- DeploymentQueue deploymentQueue = tester.deploymentQueue();
+ BuildSystem buildSystem = tester.buildSystem();
TenantId tenant = tester.controllerTester().createTenant("tenant1", "domain1", 1L);
Application application = tester.controllerTester().createApplication(tenant, "app1", "default", 1L);
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
@@ -98,13 +98,13 @@ public class DeploymentTriggerTest {
tester.deployAndNotify(application, applicationPackage, true, JobType.productionCorpUsEast1);
tester.deployAndNotify(application, applicationPackage, true, JobType.productionUsCentral1);
tester.deployAndNotify(application, applicationPackage, true, JobType.productionUsWest1);
- assertTrue("All jobs consumed", deploymentQueue.jobs().isEmpty());
+ assertTrue("All jobs consumed", buildSystem.jobs().isEmpty());
}
@Test
public void deploymentsSpecWithDelays() {
DeploymentTester tester = new DeploymentTester();
- DeploymentQueue deploymentQueue = tester.deploymentQueue();
+ BuildSystem buildSystem = tester.buildSystem();
Application application = tester.createApplication("app1", "tenant1", 1, 1L);
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
@@ -124,21 +124,21 @@ public class DeploymentTriggerTest {
tester.deployAndNotify(application, applicationPackage, true, JobType.systemTest);
tester.clock().advance(Duration.ofSeconds(1)); // Make staging test sort as the last successful job
tester.deployAndNotify(application, applicationPackage, true, JobType.stagingTest);
- assertTrue("No more jobs triggered at this time", deploymentQueue.jobs().isEmpty());
+ assertTrue("No more jobs triggered at this time", buildSystem.jobs().isEmpty());
// 30 seconds pass, us-west-1 is triggered
tester.clock().advance(Duration.ofSeconds(30));
tester.deploymentTrigger().triggerReadyJobs();
// Consume us-west-1 job without reporting completion
- assertEquals(1, deploymentQueue.jobs().size());
- assertEquals(JobType.productionUsWest1.jobName(), deploymentQueue.jobs().get(0).jobName());
- deploymentQueue.takeJobsToRun();
+ assertEquals(1, buildSystem.jobs().size());
+ assertEquals(JobType.productionUsWest1.jobName(), buildSystem.jobs().get(0).jobName());
+ buildSystem.takeJobsToRun();
// 3 minutes pass, delayed trigger does nothing as us-west-1 is still in progress
tester.clock().advance(Duration.ofMinutes(3));
tester.deploymentTrigger().triggerReadyJobs();
- assertTrue("No more jobs triggered at this time", deploymentQueue.jobs().isEmpty());
+ assertTrue("No more jobs triggered at this time", buildSystem.jobs().isEmpty());
// us-west-1 completes
tester.deploy(JobType.productionUsWest1, application, applicationPackage);
@@ -146,18 +146,18 @@ public class DeploymentTriggerTest {
// Delayed trigger does nothing as not enough time has passed after us-west-1 completion
tester.deploymentTrigger().triggerReadyJobs();
- assertTrue("No more jobs triggered at this time", deploymentQueue.jobs().isEmpty());
+ assertTrue("No more jobs triggered at this time", buildSystem.jobs().isEmpty());
// 3 minutes pass, us-central-1 is triggered
tester.clock().advance(Duration.ofMinutes(3));
tester.deploymentTrigger().triggerReadyJobs();
tester.deployAndNotify(application, applicationPackage, true, JobType.productionUsCentral1);
- assertTrue("All jobs consumed", deploymentQueue.jobs().isEmpty());
+ assertTrue("All jobs consumed", buildSystem.jobs().isEmpty());
// Delayed trigger job runs again, with nothing to trigger
tester.clock().advance(Duration.ofMinutes(10));
tester.deploymentTrigger().triggerReadyJobs();
- assertTrue("All jobs consumed", deploymentQueue.jobs().isEmpty());
+ assertTrue("All jobs consumed", buildSystem.jobs().isEmpty());
}
@Test
@@ -180,26 +180,26 @@ public class DeploymentTriggerTest {
tester.deployAndNotify(application, applicationPackage, true, JobType.stagingTest);
// Deploys in first region
- assertEquals(1, tester.deploymentQueue().jobs().size());
+ assertEquals(1, tester.buildSystem().jobs().size());
tester.deployAndNotify(application, applicationPackage, true, JobType.productionUsCentral1);
// Deploys in two regions in parallel
- assertEquals(2, tester.deploymentQueue().jobs().size());
- assertEquals(JobType.productionUsEast3.jobName(), tester.deploymentQueue().jobs().get(0).jobName());
- assertEquals(JobType.productionUsWest1.jobName(), tester.deploymentQueue().jobs().get(1).jobName());
- tester.deploymentQueue().takeJobsToRun();
+ assertEquals(2, tester.buildSystem().jobs().size());
+ assertEquals(JobType.productionUsEast3.jobName(), tester.buildSystem().jobs().get(0).jobName());
+ assertEquals(JobType.productionUsWest1.jobName(), tester.buildSystem().jobs().get(1).jobName());
+ tester.buildSystem().takeJobsToRun();
tester.deploy(JobType.productionUsWest1, application, applicationPackage, false);
tester.notifyJobCompletion(JobType.productionUsWest1, application, true);
- assertTrue("No more jobs triggered at this time", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No more jobs triggered at this time", tester.buildSystem().jobs().isEmpty());
tester.deploy(JobType.productionUsEast3, application, applicationPackage, false);
tester.notifyJobCompletion(JobType.productionUsEast3, application, true);
// Last region completes
- assertEquals(1, tester.deploymentQueue().jobs().size());
+ assertEquals(1, tester.buildSystem().jobs().size());
tester.deployAndNotify(application, applicationPackage, true, JobType.productionEuWest1);
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -235,7 +235,7 @@ public class DeploymentTriggerTest {
@Test
public void testSuccessfulDeploymentApplicationPackageChanged() {
DeploymentTester tester = new DeploymentTester();
- DeploymentQueue deploymentQueue = tester.deploymentQueue();
+ BuildSystem buildSystem = tester.buildSystem();
TenantId tenant = tester.controllerTester().createTenant("tenant1", "domain1", 1L);
Application application = tester.controllerTester().createApplication(tenant, "app1", "default", 1L);
ApplicationPackage previousApplicationPackage = new ApplicationPackageBuilder()
@@ -263,7 +263,7 @@ public class DeploymentTriggerTest {
tester.deployAndNotify(application, newApplicationPackage, true, JobType.productionUsCentral1);
tester.deployAndNotify(application, newApplicationPackage, true, JobType.productionUsWest1);
tester.deployAndNotify(application, newApplicationPackage, true, JobType.productionApNortheast1);
- assertTrue("All jobs consumed", deploymentQueue.jobs().isEmpty());
+ assertTrue("All jobs consumed", buildSystem.jobs().isEmpty());
}
@Test
@@ -292,7 +292,7 @@ public class DeploymentTriggerTest {
tester.clock().advance(Duration.ofHours(1)); // --------------- Enter block window: 18:30
readyJobsTrigger.run();
- assertEquals(0, tester.deploymentQueue().jobs().size());
+ assertEquals(0, tester.buildSystem().jobs().size());
String searchDefinition =
"search test {\n" +
@@ -306,12 +306,12 @@ public class DeploymentTriggerTest {
tester.deployTestOnly(app, changedApplication);
readyJobsTrigger.run();
- assertEquals(0, tester.deploymentQueue().jobs().size());
+ assertEquals(0, tester.buildSystem().jobs().size());
tester.clock().advance(Duration.ofHours(2)); // ---------------- Exit block window: 20:30
tester.deploymentTrigger().triggerReadyJobs(); // Schedules the blocked production job(s)
- assertEquals(1, tester.deploymentQueue().jobs().size());
- BuildService.BuildJob productionJob = tester.deploymentQueue().takeJobsToRun().get(0);
+ assertEquals(1, tester.buildSystem().jobs().size());
+ BuildService.BuildJob productionJob = tester.buildSystem().takeJobsToRun().get(0);
assertEquals("production-us-west-1", productionJob.jobName());
}
@@ -324,16 +324,16 @@ public class DeploymentTriggerTest {
LockedApplication app = (LockedApplication)tester.createAndDeploy("default0", 3, "default");
// Store that we are upgrading but don't start the system-tests job
tester.controller().applications().store(app.withDeploying(Optional.of(new Change.VersionChange(Version.fromString("6.2")))));
- assertEquals(0, tester.deploymentQueue().jobs().size());
+ assertEquals(0, tester.buildSystem().jobs().size());
readyJobsTrigger.run();
- assertEquals(1, tester.deploymentQueue().jobs().size());
- assertEquals("system-test", tester.deploymentQueue().jobs().get(0).jobName());
+ assertEquals(1, tester.buildSystem().jobs().size());
+ assertEquals("system-test", tester.buildSystem().jobs().get(0).jobName());
}
@Test
public void testHandleMultipleNotificationsFromLastJob() {
DeploymentTester tester = new DeploymentTester();
- DeploymentQueue deploymentQueue = tester.deploymentQueue();
+ BuildSystem buildSystem = tester.buildSystem();
TenantId tenant = tester.controllerTester().createTenant("tenant1", "domain1", 1L);
Application application = tester.controllerTester().createApplication(tenant, "app1", "default", 1L);
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
@@ -353,7 +353,7 @@ public class DeploymentTriggerTest {
tester.notifyJobCompletion(JobType.productionCorpUsEast1, application, true);
assertFalse("Change has been deployed",
tester.applications().require(application.id()).deploying().isPresent());
- assertTrue("All jobs consumed", deploymentQueue.jobs().isEmpty());
+ assertTrue("All jobs consumed", buildSystem.jobs().isEmpty());
}
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggererTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggererTest.java
deleted file mode 100644
index c93e1375db0..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggererTest.java
+++ /dev/null
@@ -1,86 +0,0 @@
-package com.yahoo.vespa.hosted.controller.deployment;
-
-import com.yahoo.config.provision.ApplicationId;
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService;
-import com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
-import com.yahoo.vespa.hosted.controller.maintenance.JobControl;
-import org.junit.Test;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * @author jvenstad
- */
-public class DeploymentTriggererTest {
-
- @Test
- public void testMaintenance() {
- DeploymentTester tester = new DeploymentTester();
- JobControl jobControl = new JobControl(tester.controller().curator());
-
- int project1 = 1;
- int project2 = 2;
- int project3 = 3;
-
- ApplicationId app1 = tester.createApplication("app1", "tenant", project1, null).id();
- ApplicationId app2 = tester.createApplication("app2", "tenant", project2, null).id();
- ApplicationId app3 = tester.createApplication("app3", "tenant", project3, null).id();
-
- // Create a BuildService which always rejects jobs from project2, but accepts and runs all others.
- ArrayList<BuildJob> buildJobs = new ArrayList<>();
- BuildService buildService = buildJob -> buildJob.projectId() == project2 ? false : buildJobs.add(buildJob);
-
- DeploymentTriggerer triggerer = new DeploymentTriggerer(tester.controller(),
- Duration.ofDays(1),
- jobControl,
- buildService,
- Runnable::run);
-
- triggerer.maintain();
- assertEquals("No jobs are triggered initially.",
- Collections.emptyList(),
- buildJobs);
-
- // Trigger jobs in capacity constrained environment
- tester.deploymentQueue().addJob(app1, DeploymentJobs.JobType.systemTest, false);
- tester.deploymentQueue().addJob(app2, DeploymentJobs.JobType.systemTest, false);
- tester.deploymentQueue().addJob(app3, DeploymentJobs.JobType.systemTest, false);
-
- // Trigger jobs in non-capacity constrained environment
- tester.deploymentQueue().addJob(app1, DeploymentJobs.JobType.productionUsWest1, false);
- tester.deploymentQueue().addJob(app2, DeploymentJobs.JobType.productionUsWest1, false);
- tester.deploymentQueue().addJob(app3, DeploymentJobs.JobType.productionUsWest1, false);
-
- triggerer.maintain();
- assertEquals("One system test job and all production jobs not for app2 are triggered after one maintenance run.",
- Arrays.asList(new BuildJob(project1, DeploymentJobs.JobType.systemTest.jobName()),
- new BuildJob(project1, DeploymentJobs.JobType.productionUsWest1.jobName()),
- new BuildJob(project3, DeploymentJobs.JobType.productionUsWest1.jobName())),
- buildJobs);
-
- buildJobs.clear();
- triggerer.maintain();
- assertEquals("Next job in line fails to trigger in the build service.",
- Collections.emptyList(),
- buildJobs);
-
- buildJobs.clear();
- triggerer.maintain();
- assertEquals("Next job which was waiting for capacity is triggered on next run.",
- Collections.singletonList(new BuildJob(project3, DeploymentJobs.JobType.systemTest.jobName())),
- buildJobs);
-
- buildJobs.clear();
- triggerer.maintain();
- assertEquals("No jobs are left.",
- Collections.emptyList(),
- tester.deploymentQueue().takeJobsToRun());
- }
-
-}
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
new file mode 100644
index 00000000000..1b1a4feaa4e
--- /dev/null
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java
@@ -0,0 +1,178 @@
+// 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.component.Version;
+import com.yahoo.vespa.hosted.controller.ControllerTester;
+import com.yahoo.config.provision.ApplicationId;
+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.JobError;
+import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
+
+import java.time.Duration;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+import static com.yahoo.vespa.hosted.controller.deployment.MockBuildService.JobStatus.QUEUED;
+import static com.yahoo.vespa.hosted.controller.deployment.MockBuildService.JobStatus.RUNNING;
+
+/**
+ * Simulates polling of build jobs from the controller and triggering and execution of
+ * these in Screwdriver.
+ *
+ * @author jvenstad
+ */
+public class MockBuildService implements BuildService {
+
+ private final ControllerTester tester;
+ private final MockTimeline timeline;
+ private final Map<String, Job> jobs;
+ private final Map<String, JobStatus> jobStatuses;
+ private Version version;
+
+ public MockBuildService(ControllerTester tester, MockTimeline timeline) {
+ this.tester = tester;
+ this.timeline = timeline;
+ jobs = new HashMap<>();
+ jobStatuses = new HashMap<>();
+ version = new Version(6, 86);
+ }
+
+ /** Simulates the triggering of a Screwdriver job, where jobs are queued if already running. */
+ @Override
+ public boolean trigger(BuildJob buildJob) {
+ String key = buildJob.toString();
+ System.err.println(timeline.now() + ": Asked to trigger " + key);
+
+ if ( ! jobStatuses.containsKey(key))
+ startJob(key);
+ else
+ jobStatuses.put(key, QUEUED);
+
+ return true;
+ }
+
+ /** Simulates the internal triggering of Screwdriver, where only one instance is run at a time. */
+ private void startJob(String key) {
+ jobStatuses.put(key, RUNNING);
+ Job job = jobs.get(key);
+ if (job == null)
+ return;
+
+ timeline.in(job.duration, () -> {
+ job.outcome();
+ if (jobStatuses.get(key) == QUEUED)
+ startJob(key);
+ else
+ jobStatuses.remove(key);
+ });
+ System.err.println(timeline.now() + ": Triggered " + key + "; it will finish at " + timeline.now().plus(job.duration));
+ }
+
+ public void incrementVersion() {
+ version = new Version(version.getMajor(), version.getMinor() + 1);
+ }
+
+ public Version version() { return version; }
+
+ /** Add @job to the set of @Job objects we have information about. */
+ private void add(Job job) {
+ jobs.put(job.buildJob().toString(), job);
+ }
+
+ /** Add @project to the set of @Project objects we have information about. */
+ private void add(Project project) {
+ project.jobs.values().forEach(this::add);
+ }
+
+ /** Make a @Project with the given settings, modify it if desired, and @add() it its jobs to the pool of known ones. */
+ public Project project(ApplicationId applicationId, Long projectId, Duration duration, Supplier<Boolean> success) {
+ return new Project(applicationId, projectId, duration, success);
+ }
+
+
+ /** Convenience creator for many jobs, belonging to the same project. Jobs can be modified independently after creation. */
+ class Project {
+
+ private final ApplicationId applicationId;
+ private final Long projectId;
+ private final Duration duration;
+ private final Supplier<Boolean> success;
+ private final Map<JobType, Job> jobs;
+
+ private Project(ApplicationId applicationId, Long projectId, Duration duration, Supplier<Boolean> success) {
+ this.applicationId = applicationId;
+ this.projectId = projectId;
+ this.duration = duration;
+ this.success = success;
+
+ jobs = new EnumMap<>(JobType.class);
+
+ for (JobType jobType : JobType.values())
+ jobs.put(jobType, new Job(applicationId, projectId, jobType, duration, success));
+ }
+
+ /** Set @duration for @jobType of this @Project. */
+ public Project set(Duration duration, JobType jobType) {
+ jobs.compute(jobType, (type, job) -> new Job(applicationId, projectId, jobType, duration, job.success));
+ return this;
+ }
+
+ /** Set @success for @jobType of this @Project. */
+ public Project set(Supplier<Boolean> success, JobType jobType) {
+ jobs.compute(jobType, (type, job) -> new Job(applicationId, projectId, jobType, job.duration, success));
+ return this;
+ }
+
+ /** Add the @Job objects of this @Project to the pool of known jobs for this @MockBuildService. */
+ public void add() {
+ MockBuildService.this.add(this);
+ }
+
+ }
+
+
+ /** Representation of a simulated job -- most noteworthy is the @outcome(), which is used to simulate a job completing. */
+ private class Job {
+
+ private final ApplicationId applicationId;
+ private final Long projectId;
+ private final JobType jobType;
+ private final Duration duration;
+ private final Supplier<Boolean> success;
+
+ private Job(ApplicationId applicationId, Long projectId, JobType jobType, Duration duration, Supplier<Boolean> success) {
+ this.applicationId = applicationId;
+ this.projectId = projectId;
+ this.jobType = jobType;
+ this.duration = duration;
+ this.success = success;
+ }
+
+ private void outcome() {
+ Boolean success = this.success.get();
+ System.err.println(timeline.now() + ": Job " + projectId + ":" + jobType + " reports " + success);
+ if (success != null)
+ tester.controller().applications().notifyJobCompletion(
+ new DeploymentJobs.JobReport(
+ applicationId,
+ jobType,
+ projectId,
+ 42,
+ Optional.ofNullable(success ? null : JobError.unknown)
+ ));
+ }
+
+ private BuildJob buildJob() { return new BuildJob(projectId, jobType.jobName()); }
+
+ }
+
+ enum JobStatus {
+ QUEUED,
+ RUNNING
+ }
+
+}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimeline.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimeline.java
new file mode 100644
index 00000000000..878c25bf6bd
--- /dev/null
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimeline.java
@@ -0,0 +1,106 @@
+// 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.test.ManualClock;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.PriorityQueue;
+
+/**
+ * @author jvenstad
+ */
+public class MockTimeline {
+
+ private final ManualClock clock;
+ private final PriorityQueue<Event> events;
+
+ public MockTimeline(ManualClock clock) {
+ this.events = new PriorityQueue<>();
+ this.clock = clock;
+ }
+
+ /** Make @event happen at time @at, as measured by the internal clock. */
+ public void at(Instant at, Runnable event) {
+ if (at.isBefore(now()))
+ throw new IllegalArgumentException("The flow of time runs only one way, my friend.");
+ events.add(new Event(at, event));
+ }
+
+ /** Make @event happen in @in time, as measured by the internal clock. */
+ public void in(Duration in, Runnable event) {
+ at(now().plus(in), event);
+ }
+
+ /** Make @event happen every @period time, starting @offset time from @now(), as measured by the internal clock. */
+ public void every(Duration period, Duration offset, Runnable event) {
+ in(offset, () -> {
+ every(period, event);
+ event.run();
+ });
+ }
+
+ /** Make @event happen every @period time, starting @period time from @now(), as measured by the internal clock. */
+ public void every(Duration period, Runnable event) {
+ every(period, period, event);
+ }
+
+ /** Returns the current time, as measured by the internal clock. */
+ public Instant now() {
+ return clock.instant();
+ }
+
+ /** Returns whether there are more events in the timeline, or not. */
+ public boolean hasNext() {
+ return ! events.isEmpty();
+ }
+
+ /** Advance time to the next event, let it happen, and return the time of this event. */
+ public Instant next() {
+ Event event = events.poll();
+ clock.advance(Duration.ofMillis(now().until(event.at(), ChronoUnit.MILLIS)));
+ event.happen();
+ return event.at();
+ }
+
+ /** Advance the time until @until, letting all events from now to then happen. */
+ public void advance(Instant until) {
+ at(until, () -> {});
+ while (next() != until);
+ }
+
+ /** Advance the time by @duration, letting all events from now to then happen. */
+ public void advance(Duration duration) {
+ advance(now().plus(duration));
+ }
+
+ /** Let the timeline unfold! Careful about those @every-s, though... */
+ public void unfold() {
+ while (hasNext())
+ next();
+ }
+
+
+ private static class Event implements Comparable<Event> {
+
+ private final Instant at;
+ private final Runnable event;
+
+ private Event(Instant at, Runnable event) {
+ this.at = at;
+ this.event = event;
+ }
+
+ public Instant at() { return at; }
+ public void happen() { event.run(); }
+
+
+ @Override
+ public int compareTo(Event other) {
+ return at().compareTo(other.at());
+ }
+
+ }
+
+}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystemTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystemTest.java
new file mode 100644
index 00000000000..e66d7e9168d
--- /dev/null
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystemTest.java
@@ -0,0 +1,68 @@
+// 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.config.provision.ApplicationId;
+import com.yahoo.vespa.hosted.controller.api.integration.BuildService.BuildJob;
+import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
+import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
+import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * @author mpolden
+ */
+@RunWith(Parameterized.class)
+public class PolledBuildSystemTest {
+
+ @Parameterized.Parameters(name = "jobType={0}")
+ public static Iterable<?> capacityConstrainedJobs() {
+ return Arrays.asList(JobType.systemTest, JobType.stagingTest);
+ }
+
+ private final JobType jobType;
+
+ public PolledBuildSystemTest(JobType jobType) {
+ this.jobType = jobType;
+ }
+
+ @Test
+ public void throttle_capacity_constrained_jobs() {
+ DeploymentTester tester = new DeploymentTester();
+ BuildSystem buildSystem = new PolledBuildSystem(tester.controller(), new MockCuratorDb());
+
+ int project1 = 1;
+ int project2 = 2;
+ int project3 = 3;
+ ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
+ .region("us-west-1")
+ .build();
+ ApplicationId app1 = tester.createAndDeploy("app1", project1, applicationPackage).id();
+ ApplicationId app2 = tester.createAndDeploy("app2", project2, applicationPackage).id();
+ ApplicationId app3 = tester.createAndDeploy("app3", project3, applicationPackage).id();
+
+ // Trigger jobs in capacity constrained environment
+ buildSystem.addJob(app1, jobType, false);
+ buildSystem.addJob(app2, jobType, false);
+ buildSystem.addJob(app3, jobType, false);
+
+ // A limited number of jobs are offered at a time:
+ // First offer
+ List<BuildJob> nextJobs = buildSystem.takeJobsToRun();
+ assertEquals(2, nextJobs.size());
+ assertEquals(project1, nextJobs.get(0).projectId());
+ assertEquals(project2, nextJobs.get(1).projectId());
+
+ // Second offer
+ nextJobs = buildSystem.takeJobsToRun();
+ assertEquals(1, nextJobs.size());
+ assertEquals(project3, nextJobs.get(0).projectId());
+ }
+
+}
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 598c3b0bb3d..fd00123c697 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
@@ -56,7 +56,7 @@ public class FailureRedeployerTest {
// Production job fails and is retried
tester.clock().advance(Duration.ofSeconds(1)); // Advance time so that we can detect jobs in progress
tester.deployAndNotify(app, applicationPackage, false, DeploymentJobs.JobType.productionUsEast3);
- assertEquals("Production job is retried", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Production job is retried", 1, tester.buildSystem().jobs().size());
assertEquals("Application has pending upgrade to " + version, version, tester.versionChange(app.id()).get().version());
// Another version is released, which cancels any pending upgrades to lower versions
@@ -64,13 +64,13 @@ public class FailureRedeployerTest {
tester.updateVersionStatus(version);
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsEast3); // Finish previous production job.
tester.upgrader().maintain();
- assertEquals("Application starts upgrading to new version", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Application starts upgrading to new version", 1, tester.buildSystem().jobs().size());
assertEquals("Application has pending upgrade to " + version, version, tester.versionChange(app.id()).get().version());
// Failure redeployer does not retry failing job for prod.us-east-3 as there's an ongoing deployment
tester.clock().advance(Duration.ofMinutes(1));
tester.readyJobTrigger().maintain();
- assertFalse("Job is not retried", tester.deploymentQueue().jobs().stream()
+ assertFalse("Job is not retried", tester.buildSystem().jobs().stream()
.anyMatch(j -> j.jobName().equals(DeploymentJobs.JobType.productionUsEast3.jobName())));
// Test environments pass
@@ -79,20 +79,20 @@ public class FailureRedeployerTest {
// Production job fails again and exhausts all immediate retries
tester.deployAndNotify(app, applicationPackage, false, DeploymentJobs.JobType.productionUsEast3);
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofMinutes(10));
tester.notifyJobCompletion(DeploymentJobs.JobType.productionUsEast3, app, false);
- assertTrue("Retries exhausted", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("Retries exhausted", tester.buildSystem().jobs().isEmpty());
assertTrue("Failure is recorded", tester.application(app.id()).deploymentJobs().hasFailures());
// Failure redeployer retries job
tester.clock().advance(Duration.ofMinutes(5));
tester.readyJobTrigger().maintain();
- assertEquals("Job is retried", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Job is retried", 1, tester.buildSystem().jobs().size());
// Production job finally succeeds
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsEast3);
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
assertFalse("No failures", tester.application(app.id()).deploymentJobs().hasFailures());
}
@@ -110,20 +110,20 @@ public class FailureRedeployerTest {
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.systemTest);
// staging-test starts, but does not complete
- assertEquals(DeploymentJobs.JobType.stagingTest.jobName(), tester.deploymentQueue().takeJobsToRun().get(0).jobName());
+ assertEquals(DeploymentJobs.JobType.stagingTest.jobName(), tester.buildSystem().takeJobsToRun().get(0).jobName());
tester.readyJobTrigger().maintain();
- assertTrue("No jobs retried", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs retried", tester.buildSystem().jobs().isEmpty());
// Just over 12 hours pass, job is retried
tester.clock().advance(Duration.ofHours(12).plus(Duration.ofSeconds(1)));
tester.readyJobTrigger().maintain();
- assertEquals(DeploymentJobs.JobType.stagingTest.jobName(), tester.deploymentQueue().takeJobsToRun().get(0).jobName());
+ assertEquals(DeploymentJobs.JobType.stagingTest.jobName(), tester.buildSystem().takeJobsToRun().get(0).jobName());
// Deployment completes
tester.deploy(DeploymentJobs.JobType.stagingTest, app, applicationPackage, true);
tester.notifyJobCompletion(DeploymentJobs.JobType.stagingTest, app, true);
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsEast3);
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -152,10 +152,10 @@ public class FailureRedeployerTest {
// system-test fails and exhausts all immediate retries
tester.deployAndNotify(app, applicationPackage, false, DeploymentJobs.JobType.systemTest);
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofMinutes(10));
tester.notifyJobCompletion(DeploymentJobs.JobType.systemTest, app, false);
- assertTrue("Retries exhausted", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("Retries exhausted", tester.buildSystem().jobs().isEmpty());
// Another version is released
version = Version.fromString("5.2");
@@ -165,12 +165,12 @@ public class FailureRedeployerTest {
assertEquals("Application has pending upgrade to " + version, version, tester.versionChange(app.id()).get().version());
// Consume system-test job for 5.2
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
// Failure re-deployer does not retry failing system-test job as it failed for an older change
tester.clock().advance(Duration.ofMinutes(5));
tester.readyJobTrigger().maintain();
- assertTrue("No jobs retried", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs retried", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -202,23 +202,23 @@ public class FailureRedeployerTest {
// Test environments pass
tester.deploy(DeploymentJobs.JobType.systemTest, application, applicationPackage);
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofMinutes(10));
tester.notifyJobCompletion(DeploymentJobs.JobType.systemTest, application, true);
tester.deploy(DeploymentJobs.JobType.stagingTest, application, applicationPackage);
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofMinutes(10));
tester.notifyJobCompletion(DeploymentJobs.JobType.stagingTest, application, true);
// Production job starts, but does not complete
- assertEquals(1, tester.deploymentQueue().jobs().size());
- assertEquals("Production job triggered", DeploymentJobs.JobType.productionCdUsCentral1.jobName(), tester.deploymentQueue().jobs().get(0).jobName());
- tester.deploymentQueue().takeJobsToRun();
+ assertEquals(1, tester.buildSystem().jobs().size());
+ assertEquals("Production job triggered", DeploymentJobs.JobType.productionCdUsCentral1.jobName(), tester.buildSystem().jobs().get(0).jobName());
+ tester.buildSystem().takeJobsToRun();
// Failure re-deployer runs
tester.readyJobTrigger().maintain();
- assertTrue("No jobs retried", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs retried", tester.buildSystem().jobs().isEmpty());
// Deployment completes
tester.notifyJobCompletion(DeploymentJobs.JobType.productionCdUsCentral1, application, true);
@@ -243,7 +243,7 @@ public class FailureRedeployerTest {
// Failure redeployer does not restart deployment
tester.readyJobTrigger().maintain();
- assertTrue("No jobs scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs scheduled", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -263,7 +263,7 @@ public class FailureRedeployerTest {
// Failure redeployer does not restart deployment
tester.readyJobTrigger().maintain();
- assertTrue("No jobs scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs scheduled", tester.buildSystem().jobs().isEmpty());
}
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OutstandingChangeDeployerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OutstandingChangeDeployerTest.java
index 65a90de1a16..13636122cfd 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OutstandingChangeDeployerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OutstandingChangeDeployerTest.java
@@ -37,16 +37,16 @@ public class OutstandingChangeDeployerTest {
assertFalse(tester.application("app1").hasOutstandingChange());
tester.notifyJobCompletion(DeploymentJobs.JobType.component, tester.application("app1"), true);
assertTrue(tester.application("app1").hasOutstandingChange());
- assertEquals(1, tester.deploymentQueue().jobs().size());
+ assertEquals(1, tester.buildSystem().jobs().size());
deployer.maintain();
- assertEquals("No effect as job is in progress", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("No effect as job is in progress", 1, tester.buildSystem().jobs().size());
tester.deployCompletely("app1");
- assertEquals("Upgrade done", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade done", 0, tester.buildSystem().jobs().size());
deployer.maintain();
- List<BuildService.BuildJob> jobs = tester.deploymentQueue().jobs();
+ List<BuildService.BuildJob> jobs = tester.buildSystem().jobs();
assertEquals(1, jobs.size());
assertEquals(11, jobs.get(0).projectId());
assertEquals(DeploymentJobs.JobType.systemTest.jobName(), jobs.get(0).jobName());
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 e3636974e9a..90721e7be6b 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
@@ -38,7 +38,7 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("No applications: Nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("No applications: Nothing to do", 0, tester.buildSystem().jobs().size());
// Setup applications
Application canary0 = tester.createAndDeploy("canary0", 1, "canary");
@@ -49,7 +49,7 @@ public class UpgraderTest {
Application conservative0 = tester.createAndDeploy("conservative0", 6, "conservative");
tester.upgrader().maintain();
- assertEquals("All already on the right version: Nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("All already on the right version: Nothing to do", 0, tester.buildSystem().jobs().size());
// --- 5.1 is released - everything goes smoothly
version = Version.fromString("5.1");
@@ -57,20 +57,20 @@ public class UpgraderTest {
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
tester.upgrader().maintain();
- assertEquals("New system version: Should upgrade Canaries", 2, tester.deploymentQueue().jobs().size());
+ assertEquals("New system version: Should upgrade Canaries", 2, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary0, version, "canary");
assertEquals(version, tester.configServer().lastPrepareVersion().get());
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("One canary pending; nothing else", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("One canary pending; nothing else", 1, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary1, version, "canary");
tester.updateVersionStatus(version);
assertEquals(VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Canaries done: Should upgrade defaults", 3, tester.deploymentQueue().jobs().size());
+ assertEquals("Canaries done: Should upgrade defaults", 3, tester.buildSystem().jobs().size());
tester.completeUpgrade(default0, version, "default");
tester.completeUpgrade(default1, version, "default");
@@ -79,12 +79,12 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
assertEquals(VespaVersion.Confidence.high, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Normals done: Should upgrade conservatives", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Normals done: Should upgrade conservatives", 1, tester.buildSystem().jobs().size());
tester.completeUpgrade(conservative0, version, "conservative");
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("Nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("Nothing to do", 0, tester.buildSystem().jobs().size());
// --- 5.2 is released - which fails a Canary
version = Version.fromString("5.2");
@@ -92,9 +92,9 @@ public class UpgraderTest {
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
tester.upgrader().maintain();
- assertEquals("New system version: Should upgrade Canaries", 2, tester.deploymentQueue().jobs().size());
+ assertEquals("New system version: Should upgrade Canaries", 2, tester.buildSystem().jobs().size());
tester.completeUpgradeWithError(canary0, version, "canary", DeploymentJobs.JobType.stagingTest);
- assertEquals("Other Canary was cancelled", 2, tester.deploymentQueue().jobs().size());
+ assertEquals("Other Canary was cancelled", 2, tester.buildSystem().jobs().size());
// TODO: Cancelled would mean it was triggerd, removed from the build system, but never reported in.
// Thus, the expected number of jobs should be 1, above: the retrying canary0.
// Further, canary1 should be retried after the timeout period of 12 hours, but verifying this is
@@ -103,7 +103,7 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
assertEquals(VespaVersion.Confidence.broken, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Version broken, but Canaries should keep trying", 2, tester.deploymentQueue().jobs().size());
+ assertEquals("Version broken, but Canaries should keep trying", 2, tester.buildSystem().jobs().size());
// Exhaust canary retries.
tester.notifyJobCompletion(DeploymentJobs.JobType.systemTest, canary1, false);
@@ -118,13 +118,13 @@ public class UpgraderTest {
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
tester.upgrader().maintain();
- assertEquals("New system version: Should upgrade Canaries", 2, tester.deploymentQueue().jobs().size());
+ assertEquals("New system version: Should upgrade Canaries", 2, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary0, version, "canary");
assertEquals(version, tester.configServer().lastPrepareVersion().get());
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("One canary pending; nothing else", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("One canary pending; nothing else", 1, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary1, version, "canary");
@@ -132,7 +132,7 @@ public class UpgraderTest {
assertEquals(VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Canaries done: Should upgrade defaults", 3, tester.deploymentQueue().jobs().size());
+ assertEquals("Canaries done: Should upgrade defaults", 3, tester.buildSystem().jobs().size());
tester.completeUpgradeWithError(default0, version, "default", DeploymentJobs.JobType.stagingTest);
tester.completeUpgrade(default1, version, "default");
@@ -142,7 +142,7 @@ public class UpgraderTest {
assertEquals("Not enough evidence to mark this as neither broken nor high",
VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
- assertEquals("Upgrade with error should retry", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade with error should retry", 1, tester.buildSystem().jobs().size());
// Finish previous run, with exhausted retry.
tester.clock().advance(Duration.ofHours(1));
@@ -155,12 +155,12 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
assertEquals(VespaVersion.Confidence.high, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Normals done: Should upgrade conservatives", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("Normals done: Should upgrade conservatives", 1, tester.buildSystem().jobs().size());
tester.completeUpgrade(conservative0, version, "conservative");
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("Applications are on 5.3 - nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("Applications are on 5.3 - nothing to do", 0, tester.buildSystem().jobs().size());
// --- Starting upgrading to a new version which breaks, causing upgrades to commence on the previous version
Version version54 = Version.fromString("5.4");
@@ -173,7 +173,7 @@ public class UpgraderTest {
tester.updateVersionStatus(version54);
assertEquals(VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Upgrade of defaults are scheduled", 5, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade of defaults are scheduled", 5, tester.buildSystem().jobs().size());
assertEquals(version54, ((Change.VersionChange)tester.application(default0.id()).deploying().get()).version());
assertEquals(version54, ((Change.VersionChange)tester.application(default1.id()).deploying().get()).version());
assertEquals(version54, ((Change.VersionChange)tester.application(default2.id()).deploying().get()).version());
@@ -189,7 +189,7 @@ public class UpgraderTest {
tester.updateVersionStatus(version55);
assertEquals(VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Upgrade of defaults are scheduled", 5, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade of defaults are scheduled", 5, tester.buildSystem().jobs().size());
assertEquals(version55, ((Change.VersionChange)tester.application(default0.id()).deploying().get()).version());
assertEquals(version54, ((Change.VersionChange)tester.application(default1.id()).deploying().get()).version());
assertEquals(version54, ((Change.VersionChange)tester.application(default2.id()).deploying().get()).version());
@@ -215,7 +215,7 @@ public class UpgraderTest {
tester.upgrader().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",
- 1, tester.deploymentQueue().jobs().size());
+ 1, tester.buildSystem().jobs().size());
assertEquals("5.4", ((Change.VersionChange)tester.application(default3.id()).deploying().get()).version().toString());
}
@@ -224,13 +224,13 @@ public class UpgraderTest {
// --- Setup
DeploymentTester tester = new DeploymentTester();
tester.upgrader().maintain();
- assertEquals("No system version: Nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("No system version: Nothing to do", 0, tester.buildSystem().jobs().size());
Version version = Version.fromString("5.0"); // (lower than the hardcoded version in the config server client)
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("No applications: Nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("No applications: Nothing to do", 0, tester.buildSystem().jobs().size());
// Setup applications
Application canary0 = tester.createAndDeploy("canary0", 1, "canary");
@@ -247,7 +247,7 @@ public class UpgraderTest {
Application default9 = tester.createAndDeploy("default9", 12, "default");
tester.upgrader().maintain();
- assertEquals("All already on the right version: Nothing to do", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("All already on the right version: Nothing to do", 0, tester.buildSystem().jobs().size());
// --- A new version is released
version = Version.fromString("5.1");
@@ -255,20 +255,20 @@ public class UpgraderTest {
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
tester.upgrader().maintain();
- assertEquals("New system version: Should upgrade Canaries", 2, tester.deploymentQueue().jobs().size());
+ assertEquals("New system version: Should upgrade Canaries", 2, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary0, version, "canary");
assertEquals(version, tester.configServer().lastPrepareVersion().get());
tester.updateVersionStatus(version);
tester.upgrader().maintain();
- assertEquals("One canary pending; nothing else", 1, tester.deploymentQueue().jobs().size());
+ assertEquals("One canary pending; nothing else", 1, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary1, version, "canary");
tester.updateVersionStatus(version);
assertEquals(VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Canaries done: Should upgrade defaults", 10, tester.deploymentQueue().jobs().size());
+ assertEquals("Canaries done: Should upgrade defaults", 10, tester.buildSystem().jobs().size());
tester.completeUpgrade(default0, version, "default");
tester.completeUpgradeWithError(default1, version, "default", DeploymentJobs.JobType.systemTest);
@@ -280,7 +280,7 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
tester.upgrader().maintain();
assertEquals(VespaVersion.Confidence.broken, tester.controller().versionStatus().systemVersion().get().confidence());
- assertEquals("Upgrades are cancelled", 0, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrades are cancelled", 0, tester.buildSystem().jobs().size());
}
@Test
@@ -302,7 +302,7 @@ public class UpgraderTest {
tester.upgrader().maintain();
assertEquals("Application is on expected version: Nothing to do", 0,
- tester.deploymentQueue().jobs().size());
+ tester.buildSystem().jobs().size());
// New version is released
version = Version.fromString("5.1");
@@ -315,10 +315,10 @@ public class UpgraderTest {
// staging-test fails multiple times, exhausts retries and failure is recorded
tester.deployAndNotify(app, applicationPackage, false, DeploymentJobs.JobType.stagingTest);
- tester.deploymentQueue().takeJobsToRun();
+ tester.buildSystem().takeJobsToRun();
tester.clock().advance(Duration.ofMinutes(10));
tester.notifyJobCompletion(DeploymentJobs.JobType.stagingTest, app, false);
- assertTrue("Retries exhausted", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("Retries exhausted", tester.buildSystem().jobs().isEmpty());
assertTrue("Failure is recorded", tester.application(app.id()).deploymentJobs().hasFailures());
assertTrue("Application has pending change", tester.application(app.id()).deploying().isPresent());
@@ -330,12 +330,12 @@ public class UpgraderTest {
// Upgrade is scheduled. system-tests starts, but does not complete
tester.upgrader().maintain();
assertTrue("Application still has failures", tester.application(app.id()).deploymentJobs().hasFailures());
- assertEquals(1, tester.deploymentQueue().jobs().size());
- tester.deploymentQueue().takeJobsToRun();
+ assertEquals(1, tester.buildSystem().jobs().size());
+ tester.buildSystem().takeJobsToRun();
// Upgrader runs again, nothing happens as there's already a job in progress for this change
tester.upgrader().maintain();
- assertTrue("No more jobs triggered at this time", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No more jobs triggered at this time", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -367,7 +367,7 @@ public class UpgraderTest {
// Applications with default policy start upgrading
tester.upgrader().maintain();
- assertEquals("Upgrade scheduled for remaining apps", 5, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade scheduled for remaining apps", 5, tester.buildSystem().jobs().size());
// 4/5 applications fail and lowers confidence
tester.completeUpgradeWithError(default0, version, "default", DeploymentJobs.JobType.systemTest);
@@ -381,7 +381,7 @@ public class UpgraderTest {
// 5th app passes system-test, but does not trigger next job as upgrade is cancelled
assertFalse("No change present", tester.applications().require(default4.id()).deploying().isPresent());
tester.notifyJobCompletion(DeploymentJobs.JobType.systemTest, default4, true);
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -455,19 +455,19 @@ public class UpgraderTest {
// Application is not upgraded at this time
tester.upgrader().maintain();
- assertTrue("No jobs scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs scheduled", tester.buildSystem().jobs().isEmpty());
// One hour passes, time is 19:00, still no upgrade
tester.clock().advance(Duration.ofHours(1));
tester.upgrader().maintain();
- assertTrue("No jobs scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs scheduled", tester.buildSystem().jobs().isEmpty());
// Two hours pass in total, time is 20:00 and application upgrades
tester.clock().advance(Duration.ofHours(1));
tester.upgrader().maintain();
- assertFalse("Job is scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertFalse("Job is scheduled", tester.buildSystem().jobs().isEmpty());
tester.completeUpgrade(app, version, "canary");
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
}
@Test
@@ -502,19 +502,19 @@ public class UpgraderTest {
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.stagingTest);
clock.advance(Duration.ofHours(1)); // Entering block window after prod job is triggered
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsWest1);
- assertTrue(tester.deploymentQueue().jobs().isEmpty()); // Next job not triggered due to being in the block window
+ assertTrue(tester.buildSystem().jobs().isEmpty()); // Next job not triggered due to being in the block window
// One hour passes, time is 19:00, still no upgrade
tester.clock().advance(Duration.ofHours(1));
readyJobsTrigger.maintain();
- assertTrue("No jobs scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("No jobs scheduled", tester.buildSystem().jobs().isEmpty());
// Another hour pass, time is 20:00 and application upgrades
tester.clock().advance(Duration.ofHours(1));
readyJobsTrigger.maintain();
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsCentral1);
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsEast3);
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
}
/**
@@ -558,7 +558,7 @@ public class UpgraderTest {
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsWest1);
clock.advance(Duration.ofHours(1)); // Entering block window after prod job is triggered
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsCentral1);
- assertTrue(tester.deploymentQueue().jobs().isEmpty()); // Next job not triggered due to being in the block window
+ assertTrue(tester.buildSystem().jobs().isEmpty()); // Next job not triggered due to being in the block window
// A day passes and we get a new version
tester.clock().advance(Duration.ofDays(1));
@@ -566,7 +566,7 @@ public class UpgraderTest {
tester.updateVersionStatus(version);
tester.upgrader().maintain();
readyJobsTrigger.maintain();
- assertTrue("Nothing is scheduled", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("Nothing is scheduled", tester.buildSystem().jobs().isEmpty());
// Monday morning: We are not blocked
tester.clock().advance(Duration.ofDays(1)); // Sunday, 17:00
@@ -580,7 +580,7 @@ public class UpgraderTest {
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsWest1);
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsCentral1);
tester.deployAndNotify(app, applicationPackage, true, DeploymentJobs.JobType.productionUsEast3);
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
// App is completely upgraded to the latest version
for (Deployment deployment : tester.applications().require(app.id()).deployments().values())
@@ -624,7 +624,7 @@ public class UpgraderTest {
// Applications with default policy start upgrading
tester.clock().advance(Duration.ofMinutes(1));
tester.upgrader().maintain();
- assertEquals("Upgrade scheduled for remaining apps", 5, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade scheduled for remaining apps", 5, tester.buildSystem().jobs().size());
// 4/5 applications fail, confidence is lowered and upgrade is cancelled
tester.completeUpgradeWithError(default0, version, "default", DeploymentJobs.JobType.systemTest);
@@ -658,7 +658,7 @@ public class UpgraderTest {
assertEquals(VespaVersion.Confidence.normal, tester.controller().versionStatus().systemVersion().get().confidence());
tester.upgrader().maintain();
- assertEquals("Upgrade scheduled for previously failing apps", 4, tester.deploymentQueue().jobs().size());
+ assertEquals("Upgrade scheduled for previously failing apps", 4, tester.buildSystem().jobs().size());
tester.completeUpgrade(default0, version, "default");
tester.completeUpgrade(default1, version, "default");
tester.completeUpgrade(default2, version, "default");
@@ -700,24 +700,24 @@ public class UpgraderTest {
assertEquals(version, tester.controller().versionStatus().systemVersion().get().versionNumber());
upgrader.maintain();
- assertEquals(2, tester.deploymentQueue().jobs().size());
+ assertEquals(2, tester.buildSystem().jobs().size());
tester.completeUpgrade(canary0, version, "canary");
tester.completeUpgrade(canary1, version, "canary");
tester.updateVersionStatus(version);
// Next run upgrades a subset
upgrader.maintain();
- assertEquals(2, tester.deploymentQueue().jobs().size());
+ assertEquals(2, tester.buildSystem().jobs().size());
tester.completeUpgrade(default0, version, "default");
tester.completeUpgrade(default2, version, "default");
// Remaining applications upgraded
upgrader.maintain();
- assertEquals(2, tester.deploymentQueue().jobs().size());
+ assertEquals(2, tester.buildSystem().jobs().size());
tester.completeUpgrade(default1, version, "default");
tester.completeUpgrade(default3, version, "default");
upgrader.maintain();
- assertTrue("All jobs consumed", tester.deploymentQueue().jobs().isEmpty());
+ assertTrue("All jobs consumed", tester.buildSystem().jobs().isEmpty());
}
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ControllerContainerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ControllerContainerTest.java
index 70564a1e85b..631ceab98a5 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ControllerContainerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ControllerContainerTest.java
@@ -59,7 +59,6 @@ public class ControllerContainerTest {
" <component id='com.yahoo.vespa.hosted.controller.ConfigServerClientMock'/>\n" +
" <component id='com.yahoo.vespa.hosted.controller.ZoneRegistryMock'/>\n" +
" <component id='com.yahoo.vespa.hosted.controller.Controller'/>\n" +
- " <component id='com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService'/>\n" +
" <component id='com.yahoo.vespa.hosted.controller.ConfigServerProxyMock'/>\n" +
" <component id='com.yahoo.vespa.hosted.controller.integration.MockMetricsService'/>\n" +
" <component id='com.yahoo.vespa.hosted.controller.maintenance.ControllerMaintenance'/>\n" +
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/controller/responses/maintenance.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/controller/responses/maintenance.json
index baafca7cf34..8f8b76c83c6 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/controller/responses/maintenance.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/controller/responses/maintenance.json
@@ -19,9 +19,6 @@
"name": "DeploymentMetricsMaintainer"
},
{
- "name": "DeploymentTriggerer"
- },
- {
"name": "DnsMaintainer"
},
{
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiTest.java
index 46082f10b74..d680d943f84 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/screwdriver/ScrewdriverApiTest.java
@@ -15,7 +15,7 @@ import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobError;
import com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType;
import com.yahoo.vespa.hosted.controller.application.JobStatus;
import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder;
-import com.yahoo.vespa.hosted.controller.deployment.DeploymentQueue;
+import com.yahoo.vespa.hosted.controller.deployment.BuildSystem;
import com.yahoo.vespa.hosted.controller.restapi.ContainerControllerTester;
import com.yahoo.vespa.hosted.controller.restapi.ControllerContainerTest;
import com.yahoo.vespa.hosted.controller.versions.VersionStatus;
@@ -100,10 +100,15 @@ public class ScrewdriverApiTest extends ControllerContainerTest {
assertEquals("Response contains only two items", 2, SlimeUtils.jsonToSlime(response.getBody()).get().entries());
// Check that GET didn't affect the enqueued jobs.
- response = container.handleRequest(new Request("http://localhost:8080/screwdriver/v1/jobsToRun", "", Request.Method.GET));
+ response = container.handleRequest(new Request("http://localhost:8080/screwdriver/v1/jobsToRun", "", Request.Method.DELETE));
assertTrue("Response contains system-test", response.getBodyAsString().contains(JobType.systemTest.jobName()));
assertTrue("Response contains staging-test", response.getBodyAsString().contains(JobType.stagingTest.jobName()));
assertEquals("Response contains only two items", 2, SlimeUtils.jsonToSlime(response.getBody()).get().entries());
+
+ Thread.sleep(50);
+ // Check that the *first* DELETE has removed the enqueued jobs.
+ assertResponse(new Request("http://localhost:8080/screwdriver/v1/jobsToRun", "", Request.Method.DELETE),
+ 200, "[]");
}
@Test
@@ -138,7 +143,7 @@ public class ScrewdriverApiTest extends ControllerContainerTest {
@Test
public void testTriggerJobForApplication() throws Exception {
ContainerControllerTester tester = new ContainerControllerTester(container, responseFiles);
- DeploymentQueue deploymentQueue = tester.controller().applications().deploymentTrigger().deploymentQueue();
+ BuildSystem buildSystem = tester.controller().applications().deploymentTrigger().buildSystem();
tester.containerTester().updateSystemVersion();
Application app = tester.createApplication();
@@ -162,19 +167,19 @@ public class ScrewdriverApiTest extends ControllerContainerTest {
new byte[0], Request.Method.POST),
200, "{\"message\":\"Triggered component for tenant1.application1\"}");
- assertFalse(deploymentQueue.jobs().isEmpty());
- assertEquals(JobType.component.jobName(), deploymentQueue.jobs().get(0).jobName());
- assertEquals(1L, deploymentQueue.jobs().get(0).projectId());
- deploymentQueue.takeJobsToRun();
+ assertFalse(buildSystem.jobs().isEmpty());
+ assertEquals(JobType.component.jobName(), buildSystem.jobs().get(0).jobName());
+ assertEquals(1L, buildSystem.jobs().get(0).projectId());
+ buildSystem.takeJobsToRun();
// Triggers specific job when given
assertResponse(new Request("http://localhost:8080/screwdriver/v1/trigger/tenant/" +
app.id().tenant().value() + "/application/" + app.id().application().value(),
"staging-test".getBytes(StandardCharsets.UTF_8), Request.Method.POST),
200, "{\"message\":\"Triggered staging-test for tenant1.application1\"}");
- assertFalse(deploymentQueue.jobs().isEmpty());
- assertEquals(JobType.stagingTest.jobName(), deploymentQueue.jobs().get(0).jobName());
- assertEquals(1L, deploymentQueue.jobs().get(0).projectId());
+ assertFalse(buildSystem.jobs().isEmpty());
+ assertEquals(JobType.stagingTest.jobName(), buildSystem.jobs().get(0).jobName());
+ assertEquals(1L, buildSystem.jobs().get(0).projectId());
}
private void notifyCompletion(ApplicationId app, long projectId, JobType jobType, Optional<JobError> error) throws IOException {