summaryrefslogtreecommitdiffstats
path: root/controller-server/src/main/java/com/yahoo
diff options
context:
space:
mode:
Diffstat (limited to 'controller-server/src/main/java/com/yahoo')
-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
10 files changed, 171 insertions, 188 deletions
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
deleted file mode 100644
index 15b3ef7fb83..00000000000
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/BuildSystem.java
+++ /dev/null
@@ -1,34 +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.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
new file mode 100644
index 00000000000..06d7d72a2f3
--- /dev/null
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentQueue.java
@@ -0,0 +1,96 @@
+// 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 c768aea8248..ee448775cbf 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 BuildSystem buildSystem;
+ private final DeploymentQueue deploymentQueue;
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.buildSystem = new PolledBuildSystem(controller, curator);
+ this.deploymentQueue = new DeploymentQueue(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 BuildSystem buildSystem() { return buildSystem; }
+ public DeploymentQueue deploymentQueue() { return deploymentQueue; }
public DeploymentOrder deploymentOrder() { return order; }
@@ -270,7 +270,7 @@ public class DeploymentTrigger {
*/
public void cancelChange(ApplicationId applicationId) {
applications().lockOrThrow(applicationId, application -> {
- buildSystem.removeJobs(application.id());
+ deploymentQueue.removeJobs(application.id());
applications().store(application.withChange(Change.empty()));
});
}
@@ -345,7 +345,7 @@ public class DeploymentTrigger {
log.info(String.format("Triggering %s for %s, %s: %s", jobType, application,
application.change().isPresent() ? "deploying " + application.change() : "restarted deployment",
reason));
- buildSystem.addJob(application.id(), jobType, first);
+ deploymentQueue.addJob(application.id(), jobType, first);
return application.withJobTriggering(jobType,
application.change(),
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
new file mode 100644
index 00000000000..42824073f1c
--- /dev/null
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerer.java
@@ -0,0 +1,53 @@
+// 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
deleted file mode 100644
index e25db10a8cd..00000000000
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PolledBuildSystem.java
+++ /dev/null
@@ -1,117 +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.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 f6dc1326d5e..c63b283ac0a 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,9 +5,11 @@ 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;
@@ -36,12 +38,13 @@ 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) {
+ NameService nameService, BuildService buildService) {
Duration maintenanceInterval = Duration.ofMinutes(maintainerConfig.intervalMinutes());
this.jobControl = jobControl;
deploymentExpirer = new DeploymentExpirer(controller, maintenanceInterval, jobControl);
@@ -56,6 +59,7 @@ 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(12), jobControl, nameService);
+ deploymentTriggerer = new DeploymentTriggerer(controller, Duration.ofSeconds(30), jobControl, buildService);
}
public Upgrader upgrader() { return upgrader; }
@@ -77,6 +81,7 @@ 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 ebab2054d4f..40563c4cf95 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,8 +41,6 @@ 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 a3bb191fc38..caf2af5114c 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,33 +123,29 @@ 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, deleting inactive state");
- writeInactiveJobs(Collections.emptySet());
+ log.log(Level.WARNING, "Error reading job queue of type '" + jobType.jobName() + "'; deleting it.");
+ writeJobQueue(jobType, Collections::emptyIterator);
return new ArrayDeque<>();
}
}
- public void writeJobQueue(DeploymentJobs.JobType jobType, Deque<ApplicationId> queue) {
- NestedTransaction transaction = new NestedTransaction();
+ public void writeJobQueue(DeploymentJobs.JobType jobType, Iterable<ApplicationId> queue) {
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();
@@ -159,39 +155,33 @@ 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 5017624f286..f6d6406a820 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,18 +12,19 @@ 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 set is serialized as an array of string.
+ * The queue is serialized as an array of strings.
*
* @author bratseth
*/
public class JobQueueSerializer {
- public byte[] toJson(Deque<ApplicationId> queue) {
+ public byte[] toJson(Iterable<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 021672248c9..50b810afeed 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
@@ -50,7 +50,6 @@ 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) {
@@ -67,7 +66,7 @@ public class ScrewdriverApiHandler extends LoggingRequestHandler {
return vespaVersion();
}
if (path.matches("/screwdriver/v1/jobsToRun")) {
- return buildJobs(controller.applications().deploymentTrigger().buildSystem().jobs());
+ return buildJobs(controller.applications().deploymentTrigger().deploymentQueue().jobs());
}
return notFound(request);
}
@@ -80,14 +79,6 @@ 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())