summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorJon Marius Venstad <jvenstad@yahoo-inc.com>2017-12-05 11:16:22 +0100
committerJon Marius Venstad <jvenstad@yahoo-inc.com>2018-01-03 15:56:34 +0100
commit23ed1db9ff118a45fdd2b3636599a0e7ec8521cf (patch)
treee4a2722040bdec733b749146a8101f345508a905
parent5f5505006c8cf6a2011a19977bef2b1be5f058ce (diff)
Stash...
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/BuildService.java3
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystem.java17
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java3
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimeline.java60
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimelineBuildService.java (renamed from controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockBuildService.java)54
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystemTest.java57
7 files changed, 108 insertions, 90 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 0d9c316d4d8..a721b8e142a 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
@@ -26,9 +26,6 @@ public interface BuildService {
public long projectId() { return projectId; }
public String jobName() { return jobName; }
- @Override
- public String toString() { return jobName + "@" + projectId; }
-
}
}
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 6a9db3ae917..2e66ff97fc6 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
@@ -50,13 +50,13 @@ public class DeploymentTrigger {
private final BuildSystem buildSystem;
private final DeploymentOrder order;
- public DeploymentTrigger(Controller controller, CuratorDb curator, Clock clock) {
+ public DeploymentTrigger(Controller controller, CuratorDb curator, Clock clock, BuildSystem buildSystem) {
Objects.requireNonNull(controller,"controller cannot be null");
Objects.requireNonNull(curator,"curator cannot be null");
Objects.requireNonNull(clock,"clock cannot be null");
this.controller = controller;
this.clock = clock;
- this.buildSystem = new PolledBuildSystem(controller, curator);
+ this.buildSystem = buildSystem;
this.order = new DeploymentOrder(controller);
this.jobTimeout = controller.system().equals(SystemName.main) ? Duration.ofHours(12) : Duration.ofHours(1);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystem.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystem.java
index 7e0175296c9..f95d5f949e6 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystem.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystem.java
@@ -15,11 +15,11 @@ import java.time.Duration;
import java.util.Deque;
import java.util.List;
import java.util.Optional;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.Executor;
import java.util.function.Consumer;
import java.util.logging.Level;
import java.util.logging.Logger;
+import java.util.stream.Stream;
/**
* Stores and triggers build jobs in an external BuildService.
@@ -30,24 +30,23 @@ import java.util.logging.Logger;
*
* All triggering (constrained and otherwise) can be turned off in the given JobControl.
*
- * Triggering is performed by an ExecutorService, as there is no guarantee the BuildService provides a timely response.
+ * Triggering is performed by an Executor, as there is no guarantee the BuildService provides a timely response.
*
* @author jvenstad
*/
public class PushingBuildSystem extends Maintainer implements BuildSystem {
private static final Logger log = Logger.getLogger(PushingBuildSystem.class.getName());
- static final Duration triggeringInterval = Duration.ofSeconds(30);
static final int triggeringRetries = 5;
- private final ExecutorService executors;
+ private final Executor executor;
private final BuildService buildService;
- @SuppressWarnings("unused") // Used by DI.
- public PushingBuildSystem(Controller controller, JobControl jobControl, BuildService buildService) {
+ public PushingBuildSystem(Controller controller, Duration triggeringInterval, JobControl jobControl,
+ BuildService buildService, Executor executor) {
super(controller, triggeringInterval, jobControl);
this.buildService = buildService;
- this.executors = Executors.newFixedThreadPool(20);
+ this.executor = executor;
}
@Override
@@ -102,7 +101,7 @@ public class PushingBuildSystem extends Maintainer implements BuildSystem {
private void triggerWithRetries(ApplicationId applicationId, JobType jobType) {
projectId(applicationId).ifPresent(projectId -> {
- executors.submit(() -> {
+ executor.execute(() -> {
for (int i = 0; i < triggeringRetries; i++)
if (buildService.trigger(new BuildJob(projectId, jobType.jobName())))
return;
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 4e9dd94d8e5..7d4af84c3a6 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,6 +5,7 @@ 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;
@@ -41,7 +42,7 @@ public class ControllerMaintenance extends AbstractComponent {
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);
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
index 878c25bf6bd..8746c9b8c06 100644
--- 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
@@ -6,9 +6,23 @@ import com.yahoo.test.ManualClock;
import java.time.Duration;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
import java.util.PriorityQueue;
/**
+ * Simulates concurrent series of events where time is a significant factor.
+ *
+ * Each event is modelled as a runnable and a time at which to run.
+ * A priority queue keeps all pending events, sorted on when they will run.
+ * A manual clock is used to keep the time, and is advanced to the point in
+ * time where an event will happen, right before running the event.
+ * For events with exactly the same time of happening, the order is undefined.
+ *
+ * An event may be added with a fixed time at which to run, or with a delay
+ * relative to the current time in the timeline. The latter can be used to
+ * chain several events together, by letting each event add its successor.
+ * The time may similarly be advanced to a given instant, or by a given duration.
+ *
* @author jvenstad
*/
public class MockTimeline {
@@ -17,31 +31,31 @@ public class MockTimeline {
private final PriorityQueue<Event> events;
public MockTimeline(ManualClock clock) {
- this.events = new PriorityQueue<>();
+ this.events = new PriorityQueue<>(Comparator.comparing(Event::at));
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()))
+ /** Makes the given event happen at the given instant. */
+ public void at(Instant instant, Runnable event) {
+ if (instant.isBefore(now()))
throw new IllegalArgumentException("The flow of time runs only one way, my friend.");
- events.add(new Event(at, event));
+ events.add(new Event(instant, 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);
+ /** Makes the given event happen after the given delay. */
+ public void in(Duration delay, Runnable event) {
+ at(now().plus(delay), 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, () -> {
+ /** Makes the given event happen with the given period, starting with the given delay from now. */
+ public void every(Duration period, Duration delay, Runnable event) {
+ in(delay, () -> {
every(period, event);
event.run();
});
}
- /** Make @event happen every @period time, starting @period time from @now(), as measured by the internal clock. */
+ /** Makes the given event happen with the given period, starting with one period delay from now. */
public void every(Duration period, Runnable event) {
every(period, period, event);
}
@@ -51,12 +65,12 @@ public class MockTimeline {
return clock.instant();
}
- /** Returns whether there are more events in the timeline, or not. */
+ /** Returns whether pending events remain in the timeline. */
public boolean hasNext() {
return ! events.isEmpty();
}
- /** Advance time to the next event, let it happen, and return the time of this event. */
+ /** Advances 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)));
@@ -64,25 +78,19 @@ public class MockTimeline {
return event.at();
}
- /** Advance the time until @until, letting all events from now to then happen. */
+ /** Advances the time until the given instant, 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. */
+ /** Advances the time by the given 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 static class Event {
private final Instant at;
private final Runnable event;
@@ -95,12 +103,6 @@ public class MockTimeline {
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/MockBuildService.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/MockTimelineBuildService.java
index 91f0f7be612..3bc015d9580 100644
--- 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/MockTimelineBuildService.java
@@ -1,7 +1,6 @@
// 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.ApplicationController;
import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.config.provision.ApplicationId;
@@ -17,45 +16,48 @@ import java.util.Map;
import java.util.Optional;
import java.util.function.Supplier;
-import static com.yahoo.vespa.hosted.controller.deployment.MockBuildService.JobRunStatus.QUEUED;
-import static com.yahoo.vespa.hosted.controller.deployment.MockBuildService.JobRunStatus.RUNNING;
+import static com.yahoo.vespa.hosted.controller.deployment.MockTimelineBuildService.JobRunStatus.ENQUEUED;
+import static com.yahoo.vespa.hosted.controller.deployment.MockTimelineBuildService.JobRunStatus.RUNNING;
/**
- * Simulates polling of build jobs from the controller and triggering and execution of
- * these in Screwdriver.
+ * Simulates enqueueing of build jobs in an external build service.
+ *
+ * The external build service simulated here allows only one concurrent execution of each job,
+ * and enqueueing a job which is currently running makes this job run one more time when finished,
+ * regardless of the number of times it is enqueued.
*
* @author jvenstad
*/
-public class MockBuildService implements BuildService {
+public class MockTimelineBuildService implements BuildService {
private final ApplicationController applications;
private final MockTimeline timeline;
private final Map<String, Job> jobs;
private final Map<String, JobRunStatus> jobStatuses;
- public MockBuildService(ControllerTester tester, MockTimeline timeline) {
+ public MockTimelineBuildService(ControllerTester tester, MockTimeline timeline) {
this.applications = tester.controller().applications();
this.timeline = timeline;
jobs = new HashMap<>();
jobStatuses = new HashMap<>();
}
- /** Simulates the triggering of a Screwdriver job, where jobs are queued if already running. */
+ /** Simulates the triggering of a Screwdriver job, where jobs are enqueued if already running. */
@Override
public boolean trigger(BuildJob buildJob) {
- String key = buildJob.toString();
+ String key = buildJob.jobName() + "@" + buildJob.projectId();
System.err.println(timeline.now() + ": Asked to trigger " + key);
if ( ! jobStatuses.containsKey(key))
- startJob(key);
+ start(key);
else
- jobStatuses.put(key, QUEUED);
+ jobStatuses.put(key, ENQUEUED);
return true;
}
/** Simulates the internal triggering of Screwdriver, where only one instance is run at a time. */
- private void startJob(String key) {
+ private void start(String key) {
jobStatuses.put(key, RUNNING);
Job job = jobs.get(key);
if (job == null)
@@ -63,32 +65,32 @@ public class MockBuildService implements BuildService {
timeline.in(job.duration, () -> {
job.outcome();
- if (jobStatuses.get(key) == QUEUED)
- startJob(key);
+ if (jobStatuses.get(key) == ENQUEUED)
+ start(key);
else
jobStatuses.remove(key);
});
System.err.println(timeline.now() + ": Triggered " + key + "; it will finish at " + timeline.now().plus(job.duration));
}
- /** Add @job to the set of @Job objects we have information about. */
+ /** Add job to the set of Job objects we have information about. */
private void add(Job job) {
- jobs.put(job.buildJob().toString(), job);
+ jobs.put(job.buildJob().jobName() + "@" + job.buildJob().projectId(), job);
}
- /** Add @project to the set of @Project objects we have information about. */
+ /** Add project to the set of Project objects we have information about. */
private void add(Project project) {
project.jobs.values().forEach(this::add);
}
// TODO: Replace with something that relies on ApplicationPackage.
- /** Make a @Project with the given settings, modify it if desired, and @add() it its jobs to the pool of known ones. */
+ /** 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<JobError> error) {
return new Project(applicationId, projectId, duration, error);
}
- /** Convenience creator for many jobs, belonging to the same project. Jobs can be modified independently after creation. */
+ /** Convenience creator for many jobs belonging to the same project. Jobs can be modified independently after creation. */
class Project {
private final ApplicationId applicationId;
@@ -105,27 +107,27 @@ public class MockBuildService implements BuildService {
jobs.put(jobType, new Job(applicationId, projectId, jobType, duration, error));
}
- /** Set @duration for @jobType of this @Project. */
+ /** Set duration for jobType of this Project. */
public Project set(Duration duration, JobType jobType) {
jobs.compute(jobType, (__, job) -> new Job(applicationId, projectId, jobType, duration, job.error));
return this;
}
- /** Set @success for @jobType of this @Project. */
+ /** Set success for jobType of this Project. */
public Project set(Supplier<JobError> error, JobType jobType) {
jobs.compute(jobType, (__, job) -> new Job(applicationId, projectId, jobType, job.duration, error));
return this;
}
- /** Add the @Job objects of this @Project to the pool of known jobs for this @MockBuildService. */
+ /** Add the Job objects of this Project to the pool of known jobs for this MockBuildService. */
public void add() {
- MockBuildService.this.add(this);
+ MockTimelineBuildService.this.add(this);
}
}
- /** Representation of a simulated job -- most noteworthy is the @outcome(), which is used to simulate a job completing. */
+ /** 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;
@@ -152,7 +154,7 @@ public class MockBuildService implements BuildService {
applications.notifyJobCompletion(new DeploymentJobs.JobReport(applicationId,
jobType,
projectId,
- ++buildNumber,
+ ++buildNumber, // TODO: Increase this on triggering instead.
Optional.ofNullable(jobError)));
}
@@ -161,7 +163,7 @@ public class MockBuildService implements BuildService {
}
enum JobRunStatus {
- QUEUED,
+ ENQUEUED,
RUNNING
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystemTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystemTest.java
index b5227423991..015d714c23d 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystemTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/PushingBuildSystemTest.java
@@ -2,13 +2,15 @@ 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.application.ApplicationPackage;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
import com.yahoo.vespa.hosted.controller.maintenance.JobControl;
-import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb;
+import org.junit.Test;
+import java.time.Duration;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
+import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.productionUsWest1;
import static com.yahoo.vespa.hosted.controller.application.DeploymentJobs.JobType.systemTest;
import static org.junit.Assert.assertEquals;
@@ -17,38 +19,53 @@ import static org.junit.Assert.assertEquals;
*/
public class PushingBuildSystemTest {
+ @Test
public void testTriggering() {
DeploymentTester tester = new DeploymentTester();
- MockTimeline timeline = new MockTimeline(tester.clock());
JobControl jobControl = new JobControl(tester.controller().curator());
- BuildSystem buildSystem = new PushingBuildSystem(tester.controller(), jobControl, new MockBuildService(tester.controllerTester(), timeline));
-
- 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 buildSystem = new PushingBuildSystem(tester.controller(),
+ Duration.ofDays(1),
+ jobControl,
+ new MockBuildService(),
+ Runnable::run);
+
+ // Make sure the applications exist in the controller, as the build system uses this information.
+ ApplicationId app1 = tester.createAndDeploy("app1", 1, "default-policy").id();
+ ApplicationId app2 = tester.createAndDeploy("app2", 2, "default-policy").id();
+ ApplicationId app3 = tester.createAndDeploy("app3", 3, "default-policy").id();
+
+ // Trigger jobs in a capacity-constrained environment.
buildSystem.addJob(app1, systemTest, false);
buildSystem.addJob(app2, systemTest, false);
buildSystem.addJob(app3, systemTest, false);
- // A limited number of jobs are offered at a time:
- // First offer
+ // Trigger jobs in a non-constrained environment.
+ buildSystem.addJob(app1, productionUsWest1, false);
+ buildSystem.addJob(app2, productionUsWest1, false);
+ buildSystem.addJob(app3, productionUsWest1, false);
+
+ // A single capacity-constrained job is triggered each run.
List<BuildService.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());
}
+
+ private static class MockBuildService implements BuildService {
+
+ private final Set<String> jobs = new HashSet<>();
+
+ @Override
+ public boolean trigger(BuildJob buildJob) {
+ jobs.add(buildJob.jobName() + "@" + buildJob.projectId());
+ return true;
+ }
+
+ }
+
}