aboutsummaryrefslogtreecommitdiffstats
path: root/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java
diff options
context:
space:
mode:
Diffstat (limited to 'controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java')
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTester.java415
1 files changed, 175 insertions, 240 deletions
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 582333154a2..0ca035b85b2 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
@@ -1,27 +1,31 @@
-// Copyright 2019 Oath Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+// Copyright 2018 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.config.provision.ApplicationId;
-import com.yahoo.config.provision.Environment;
-import com.yahoo.config.provision.InstanceName;
-import com.yahoo.config.provision.TenantName;
+import com.yahoo.config.provision.zone.ZoneId;
+import com.yahoo.log.LogLevel;
import com.yahoo.test.ManualClock;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.ApplicationController;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.vespa.hosted.controller.Instance;
+import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
+import com.yahoo.vespa.hosted.controller.api.integration.athenz.AthenzDbMock;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationVersion;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
-import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.RunId;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.SourceRevision;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.TesterId;
+import com.yahoo.vespa.hosted.controller.api.integration.routing.RoutingGeneratorMock;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockTesterCloud;
import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
-import com.yahoo.vespa.hosted.controller.application.Change;
-import com.yahoo.vespa.hosted.controller.application.DeploymentJobs;
-import com.yahoo.vespa.hosted.controller.application.JobStatus;
-import com.yahoo.vespa.hosted.controller.application.SystemApplication;
import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId;
import com.yahoo.vespa.hosted.controller.integration.ConfigServerMock;
import com.yahoo.vespa.hosted.controller.maintenance.JobControl;
+import com.yahoo.vespa.hosted.controller.maintenance.JobRunner;
+import com.yahoo.vespa.hosted.controller.maintenance.JobRunnerTest;
import com.yahoo.vespa.hosted.controller.maintenance.NameServiceDispatcher;
import com.yahoo.vespa.hosted.controller.maintenance.OutstandingChangeDeployer;
import com.yahoo.vespa.hosted.controller.maintenance.ReadyJobsTrigger;
@@ -31,54 +35,96 @@ import java.time.Duration;
import java.time.Instant;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
-import java.util.List;
+import java.util.Collections;
import java.util.Optional;
-import java.util.UUID;
+import java.util.logging.Logger;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
/**
- * This class provides convenience methods for testing deployments
- *
- * @author bratseth
- * @author mpolden
+ * @author jonmv
*/
public class DeploymentTester {
// Set a long interval so that maintainers never do scheduled runs during tests
private static final Duration maintenanceInterval = Duration.ofDays(1);
+ private static final String ATHENZ_DOMAIN = "domain";
+ private static final String ATHENZ_SERVICE = "service";
+
+ public static final TenantAndApplicationId appId = TenantAndApplicationId.from("tenant", "application");
+ public static final ApplicationId instanceId = appId.defaultInstance();
+ public static final TesterId testerId = TesterId.of(instanceId);
+
+ private final DeploymentContext defaultContext;
private final ControllerTester tester;
+ private final JobController jobs;
+ private final RoutingGeneratorMock routing;
+ private final MockTesterCloud cloud;
+ private final JobRunner runner;
private final Upgrader upgrader;
+ private final ReadyJobsTrigger readyJobsTrigger;
private final OutstandingChangeDeployer outstandingChangeDeployer;
- private final ReadyJobsTrigger readyJobTrigger;
private final NameServiceDispatcher nameServiceDispatcher;
- private final boolean updateDnsAutomatically;
+
+ public JobController jobs() { return jobs; }
+ public RoutingGeneratorMock routing() { return routing; }
+ public MockTesterCloud cloud() { return cloud; }
+ public JobRunner runner() { return runner; }
+ public ConfigServerMock configServer() { return tester.configServer(); }
+ public Controller controller() { return tester.controller(); }
+ public DeploymentTrigger deploymentTrigger() { return applications().deploymentTrigger(); }
+ public ControllerTester controllerTester() { return tester; }
+ public Upgrader upgrader() { return upgrader; }
+ public ApplicationController applications() { return tester.controller().applications(); }
+ public ManualClock clock() { return tester.clock(); }
+ public Application application() { return application(appId); }
+ public Application application(TenantAndApplicationId id ) { return applications().requireApplication(id); }
+ public Instance instance() { return instance(instanceId); }
+ public Instance instance(ApplicationId id) { return applications().requireInstance(id); }
public DeploymentTester() {
this(new ControllerTester());
}
- public DeploymentTester(ControllerTester tester) {
- this(tester, true);
+ public DeploymentTester(ControllerTester controllerTester) {
+ tester = controllerTester;
+ jobs = tester.controller().jobController();
+ routing = tester.serviceRegistry().routingGeneratorMock();
+ cloud = (MockTesterCloud) tester.controller().jobController().cloud();
+ var jobControl = new JobControl(tester.controller().curator());
+ runner = new JobRunner(tester.controller(), Duration.ofDays(1), jobControl,
+ JobRunnerTest.inThreadExecutor(), new InternalStepRunner(tester.controller()));
+ upgrader = new Upgrader(tester.controller(), maintenanceInterval, jobControl, tester.curator());
+ upgrader.setUpgradesPerMinute(1); // Anything that makes it at least one for any maintenance period is fine.
+ readyJobsTrigger = new ReadyJobsTrigger(tester.controller(), maintenanceInterval, jobControl);
+ outstandingChangeDeployer = new OutstandingChangeDeployer(tester.controller(), maintenanceInterval, jobControl);
+ nameServiceDispatcher = new NameServiceDispatcher(tester.controller(), maintenanceInterval, jobControl,
+ Integer.MAX_VALUE);
+ defaultContext = newDeploymentContext(instanceId);
+ routing.putEndpoints(new DeploymentId(null, null), Collections.emptyList()); // Turn off default behaviour for the mock.
+
+ // Get deployment job logs to stderr.
+ Logger.getLogger("").setLevel(LogLevel.DEBUG);
+ Logger.getLogger(InternalStepRunner.class.getName()).setLevel(LogLevel.DEBUG);
+ tester.configureDefaultLogHandler(handler -> handler.setLevel(LogLevel.DEBUG));
+
+ // Mock Athenz domain to allow launch of service
+ AthenzDbMock.Domain domain = tester.athenzDb().getOrCreateDomain(new com.yahoo.vespa.athenz.api.AthenzDomain(ATHENZ_DOMAIN));
+ domain.services.put(ATHENZ_SERVICE, new AthenzDbMock.Service(true));
+ }
+
+ public ReadyJobsTrigger readyJobsTrigger() {
+ return readyJobsTrigger;
}
- public DeploymentTester(ControllerTester tester, boolean updateDnsAutomatically) {
- this.tester = tester;
- this.updateDnsAutomatically = updateDnsAutomatically;
- tester.curator().writeUpgradesPerMinute(100);
+ public OutstandingChangeDeployer outstandingChangeDeployer() { return outstandingChangeDeployer; }
- JobControl jobControl = new JobControl(tester.curator());
- this.upgrader = new Upgrader(tester.controller(), maintenanceInterval, jobControl, tester.curator());
- this.upgrader.setUpgradesPerMinute(1); // Anything that makes it at least one for any maintenance period is fine.
- this.outstandingChangeDeployer = new OutstandingChangeDeployer(tester.controller(), maintenanceInterval, jobControl);
- this.readyJobTrigger = new ReadyJobsTrigger(tester.controller(), maintenanceInterval, jobControl);
- this.nameServiceDispatcher = new NameServiceDispatcher(tester.controller(), Duration.ofHours(12),
- new JobControl(tester.controller().curator()),
- Integer.MAX_VALUE);
- atHourOfDay(5); // Set hour of day which always allows confidence to change
+ public NameServiceDispatcher nameServiceDispatcher() {
+ return nameServiceDispatcher;
}
public DeploymentTester atHourOfDay(int hour) {
@@ -93,269 +139,158 @@ public class DeploymentTester {
return this;
}
- public Upgrader upgrader() { return upgrader; }
-
- public OutstandingChangeDeployer outstandingChangeDeployer() { return outstandingChangeDeployer; }
-
- public ReadyJobsTrigger readyJobTrigger() { return readyJobTrigger; }
-
- public Controller controller() { return tester.controller(); }
-
- public ApplicationController applications() { return tester.controller().applications(); }
-
- public MockBuildService buildService() { return tester.serviceRegistry().buildServiceMock(); }
-
- public DeploymentTrigger deploymentTrigger() { return tester.controller().applications().deploymentTrigger(); }
-
- public ManualClock clock() { return tester.clock(); }
-
- public ControllerTester controllerTester() { return tester; }
-
- public ConfigServerMock configServer() { return tester.serviceRegistry().configServerMock(); }
-
- public Application application(TenantAndApplicationId id) {
- return controller().applications().requireApplication(id);
- }
-
- public Instance defaultInstance(String name) {
- return instance(ApplicationId.from("tenant1", name, "default"));
- }
-
- public Instance defaultInstance(TenantAndApplicationId application) {
- return controller().applications().requireApplication(application).require(InstanceName.defaultName());
- }
-
- public Instance instance(ApplicationId application) {
- return controller().applications().requireInstance(application);
+ /** Returns the default deployment context owned by this */
+ public DeploymentContext deploymentContext() {
+ return defaultContext;
}
- // TODO(mpolden): Change callers to use ControllerTester#computeVersionStatus and remove this
- public void computeVersionStatus() {
- tester.computeVersionStatus();
+ /** Create a new deployment context for given application */
+ public DeploymentContext newDeploymentContext(String tenantName, String applicationName, String instanceName) {
+ return newDeploymentContext(ApplicationId.from(tenantName, applicationName, instanceName));
}
- // TODO(mpolden): Change callers to use ControllerTester#upgradeController and remove this
- public void upgradeController(Version version) {
- tester.upgradeController(version);
+ /** Create a new deployment context for given application */
+ public DeploymentContext newDeploymentContext(ApplicationId instance) {
+ return new DeploymentContext(instance, this);
}
- // TODO(mpolden): Change callers to use ControllerTester#upgradeController and remove this
- public void upgradeController(Version version, String commitSha, Instant commitDate) {
- tester.upgradeController(version, commitSha, commitDate);
+ /** Create a new application with given tenant and application name */
+ public Application createApplication(String tenantName, String applicationName, String instanceName) {
+ return newDeploymentContext(tenantName, applicationName, instanceName).application();
}
- // TODO(mpolden): Change callers to use ControllerTester#upgradeSystemApplications and remove this
- public void upgradeSystemApplications(Version version) {
- tester.upgradeSystemApplications(version);
+ /** Submits a new application, and returns the version of the new submission. */
+ public ApplicationVersion newSubmission(TenantAndApplicationId id, ApplicationPackage applicationPackage, SourceRevision sourceRevision) {
+ return newDeploymentContext(id.defaultInstance()).submit(applicationPackage, sourceRevision).lastSubmission().get();
}
- // TODO(mpolden): Change callers to use ControllerTester#upgradeSystemApplications and remove this
- public void upgradeSystemApplications(Version version, List<SystemApplication> systemApplications) {
- tester.upgradeSystemApplications(version, systemApplications);
+ public ApplicationVersion newSubmission(TenantAndApplicationId id, ApplicationPackage applicationPackage) {
+ return newSubmission(id, applicationPackage, DeploymentContext.defaultSourceRevision);
}
- // TODO(mpolden): Change callers to use ControllerTester#upgradeSystem and remove this
- public void upgradeSystem(Version version) {
- tester.upgradeSystem(version);
- upgrader().maintain();
- readyJobTrigger().maintain();
+ /**
+ * Submits a new application package, and returns the version of the new submission.
+ */
+ public ApplicationVersion newSubmission(ApplicationPackage applicationPackage) {
+ return newSubmission(appId, applicationPackage);
}
- // TODO(mpolden): Change callers to use InternalDeploymentTester#flushDnsRequests and remove this
- public void flushDnsRequests() {
- nameServiceDispatcher.run();
- assertTrue("All name service requests dispatched",
- controller().curator().readNameServiceQueue().requests().isEmpty());
+ /**
+ * Submits a new application, and returns the version of the new submission.
+ */
+ public ApplicationVersion newSubmission() {
+ return defaultContext.submit().lastSubmission().get();
}
- /** Triggers jobs until nothing more triggers, and returns the number of triggered jobs. */
- public int triggerUntilQuiescence() {
- int triggered = 0;
- while (triggered != (triggered += deploymentTrigger().triggerReadyJobs()));
- return triggered;
- }
-
- public Application createApplication(String applicationName, String tenantName, long projectId, long propertyId) {
- return createApplication("default", applicationName, tenantName, projectId, propertyId);
- }
-
- public Application createApplication(String instanceName, String applicationName, String tenantName, long projectId, long propertyId) {
- TenantName tenant = tester.createTenant(tenantName, UUID.randomUUID().toString(), propertyId);
- return tester.createApplication(tenant, applicationName, instanceName, projectId);
- }
-
- public void restartController() { tester.createNewController(); }
-
- /** Notify the controller about a job completing */
- public BuildJob jobCompletion(JobType job) {
- return new BuildJob(this::notifyJobCompletion, tester.serviceRegistry().artifactRepositoryMock()).type(job);
+ /**
+ * Sets a single endpoint in the routing mock; this matches that required for the tester.
+ */
+ public void setEndpoints(ApplicationId id, ZoneId zone) {
+ newDeploymentContext(id).setEndpoints(zone);
}
- /** Simulate the full lifecycle of an application deployment as declared in given application package */
- public Application createAndDeploy(String applicationName, int projectId, ApplicationPackage applicationPackage) {
- TenantName tenant = tester.createTenant("tenant1", "domain1", 1L);
- return createAndDeploy(tenant, applicationName, projectId, applicationPackage);
+ /** Completely deploys the given application version, assuming it is the last to be submitted. */
+ public void deployNewSubmission(ApplicationVersion version) {
+ deployNewSubmission(appId, version);
}
- /** Simulate the full lifecycle of an application deployment as declared in given application package */
- public Application createAndDeploy(TenantName tenant, String applicationName, int projectId, ApplicationPackage applicationPackage) {
- Application application = tester.createApplication(tenant, applicationName, "default", projectId);
- deployCompletely(application, applicationPackage);
- return applications().requireApplication(application.id());
+ /** Completely deploys the given application version, assuming it is the last to be submitted. */
+ public void deployNewSubmission(TenantAndApplicationId id, ApplicationVersion version) {
+ var context = newDeploymentContext(id.defaultInstance());
+ var application = context.application();
+ assertFalse(application.instances().values().stream()
+ .anyMatch(instance -> instance.deployments().values().stream()
+ .anyMatch(deployment -> deployment.applicationVersion().equals(version))));
+ assertEquals(version, application.change().application().get());
+ assertFalse(application.change().platform().isPresent());
+ context.completeRollout();
+ assertFalse(context.application().change().hasTargets());
}
- /** Simulate the full lifecycle of an application deployment to prod.us-west-1 with the given upgrade policy */
- public Application createAndDeploy(String applicationName, int projectId, String upgradePolicy) {
- return createAndDeploy(applicationName, projectId, applicationPackage(upgradePolicy));
+ /** Completely deploys the given, new platform. */
+ public void deployNewPlatform(Version version) {
+ deployNewPlatform(appId, version);
}
- /** Simulate the full lifecycle of an application deployment to prod.us-west-1 with the given upgrade policy */
- public void createAndDeploy(TenantName tenant, String applicationName, int projectId, String upgradePolicy) {
- createAndDeploy(tenant, applicationName, projectId, applicationPackage(upgradePolicy));
+ /** Completely deploys the given, new platform. */
+ public void deployNewPlatform(TenantAndApplicationId id, Version version) {
+ newDeploymentContext(id.defaultInstance()).deployPlatform(version);
}
- /** Deploy application completely using the given application package */
- public void deployCompletely(Application application, ApplicationPackage applicationPackage) {
- deployCompletely(application, applicationPackage, BuildJob.defaultBuildNumber);
- }
-
- public void completeDeploymentWithError(Application application, ApplicationPackage applicationPackage, long buildNumber, JobType failOnJob) {
- jobCompletion(JobType.component).application(application)
- .buildNumber(buildNumber)
- .uploadArtifact(applicationPackage)
- .submit();
- completeDeployment(application, applicationPackage, Optional.ofNullable(failOnJob));
- }
-
- public void deployCompletely(Application application, ApplicationPackage applicationPackage, long buildNumber) {
- completeDeploymentWithError(application, applicationPackage, buildNumber, null);
- }
-
- private void completeDeployment(Application application, ApplicationPackage applicationPackage, Optional<JobType> failOnJob) {
- assertTrue(application.id() + " has pending changes to deploy", applications().requireApplication(application.id()).change().hasTargets());
- DeploymentSteps steps = controller().applications().deploymentTrigger().steps(applicationPackage.deploymentSpec());
- List<JobType> jobs = steps.jobs();
- // TODO jonmv: Change to list instances here.
- for (JobType job : jobs) {
- boolean failJob = failOnJob.map(j -> j.equals(job)).orElse(false);
- deployAndNotify(application.id().defaultInstance(), applicationPackage, ! failJob, job);
- if (failJob) {
- break;
- }
- }
- if (failOnJob.isPresent()) {
- assertTrue(applications().requireApplication(application.id()).change().hasTargets());
- assertTrue(defaultInstance(application.id()).deploymentJobs().hasFailures());
- } else {
- assertFalse(applications().requireApplication(application.id()).change().hasTargets());
+ /** Aborts and finishes all running jobs. */
+ public void abortAll() {
+ triggerJobs();
+ for (Run run : jobs.active()) {
+ jobs.abort(run.id());
+ runner.advance(jobs.run(run.id()).get());
+ assertTrue(jobs.run(run.id()).get().hasEnded());
}
- if (updateDnsAutomatically) {
- flushDnsRequests();
- }
- }
-
- public void completeUpgrade(Application application, Version version, String upgradePolicy) {
- completeUpgrade(application, version, applicationPackage(upgradePolicy));
- }
-
- public void completeUpgrade(Application application, Version version, ApplicationPackage applicationPackage) {
- assertTrue(application + " has a change", applications().requireApplication(application.id()).change().hasTargets());
- assertEquals(Change.of(version), applications().requireApplication(application.id()).change());
- completeDeployment(application, applicationPackage, Optional.empty());
- }
-
- public void completeUpgradeWithError(Application application, Version version, String upgradePolicy, JobType failOnJob) {
- completeUpgradeWithError(application, version, applicationPackage(upgradePolicy), Optional.of(failOnJob));
- }
-
- public void completeUpgradeWithError(Application application, Version version, ApplicationPackage applicationPackage, JobType failOnJob) {
- completeUpgradeWithError(application, version, applicationPackage, Optional.of(failOnJob));
}
- private void completeUpgradeWithError(Application application, Version version, ApplicationPackage applicationPackage, Optional<JobType> failOnJob) {
- assertTrue(applications().requireApplication(application.id()).change().hasTargets());
- assertEquals(Change.of(version), applications().requireApplication(application.id()).change());
- completeDeployment(application, applicationPackage, failOnJob);
+ /** Triggers jobs until nothing more triggers, and returns the number of triggered jobs. */
+ public int triggerJobs() {
+ int triggered = 0;
+ while (triggered != (triggered += deploymentTrigger().triggerReadyJobs()));
+ return triggered;
}
- public void deploy(JobType job, ApplicationId id, ApplicationPackage applicationPackage) {
- deploy(job, id, Optional.of(applicationPackage), false);
+ /** Starts a manual deployment of the given package, and then runs the whole of the given job, successfully. */
+ public void runJob(ApplicationId instanceId, JobType type, ApplicationPackage applicationPackage) {
+ jobs.deploy(instanceId, type, Optional.empty(), applicationPackage);
+ newDeploymentContext(instanceId).runJob(type);
}
- public void deploy(JobType job, ApplicationId id, ApplicationPackage applicationPackage,
- boolean deployCurrentVersion) {
- deploy(job, id, Optional.of(applicationPackage), deployCurrentVersion);
+ /** Pulls the ready job trigger, and then runs the whole of the given job, successfully. */
+ public void runJob(JobType type) {
+ defaultContext.runJob(type);
}
- public void deploy(JobType job, ApplicationId id, Optional<ApplicationPackage> applicationPackage,
- boolean deployCurrentVersion) {
- tester.deploy(id, job.zone(controller().system()), applicationPackage, deployCurrentVersion);
+ /** Pulls the ready job trigger, and then runs the whole of the given job, successfully. */
+ public void runJob(ApplicationId instanceId, JobType type) {
+ if (type.environment().isManuallyDeployed())
+ throw new IllegalArgumentException("Use overload with application package for dev/perf jobs");
+ newDeploymentContext(instanceId).runJob(type);
}
- public void deployAndNotify(Instance i, String upgradePolicy, boolean success, JobType job) {
- deployAndNotify(i.id(), applicationPackage(upgradePolicy), success, job);
+ public void failDeployment(JobType type) {
+ defaultContext.failDeployment(type);
}
- public void deployAndNotify(ApplicationId id, ApplicationPackage applicationPackage, boolean success, JobType job) {
- deployAndNotify(id, Optional.of(applicationPackage), success, job);
+ public void failDeployment(ApplicationId instanceId, JobType type) {
+ newDeploymentContext(instanceId).failDeployment(type);
}
- public void deployAndNotify(Instance i, boolean success, JobType job) {
- deployAndNotify(i.id(), Optional.empty(), success, job);
- }
- public void deployAndNotify(ApplicationId id, boolean success, JobType job) {
- deployAndNotify(id, Optional.empty(), success, job);
+ public void timeOutUpgrade(JobType type) {
+ defaultContext.timeOutUpgrade(type);
}
- public void deployAndNotify(ApplicationId id, Optional<ApplicationPackage> applicationPackage, boolean success, JobType job) {
- if (success) {
- // Staging deploys twice, once with current version and once with new version
- if (job == JobType.stagingTest) {
- deploy(job, id, applicationPackage, true);
- }
- deploy(job, id, applicationPackage, false);
- }
- // Deactivate test deployments after deploy. This replicates the behaviour of the tenant pipeline
- if (job.isTest()) {
- controller().applications().deactivate(id, job.zone(controller().system()));
- }
- jobCompletion(job).application(id).success(success).submit();
+ public void timeOutUpgrade(ApplicationId instanceId, JobType type) {
+ newDeploymentContext(instanceId).timeOutConvergence(type);
}
- public Optional<JobStatus.JobRun> firstFailing(Instance instance, JobType job) {
- return tester.controller().applications().requireInstance(instance.id())
- .deploymentJobs().jobStatus().get(job).firstFailing();
+ public void timeOutConvergence(JobType type) {
+ defaultContext.timeOutConvergence(type);
}
- private void notifyJobCompletion(DeploymentJobs.JobReport report) {
- if (report.jobType() != JobType.component && ! buildService().remove(report.buildJob()))
- throw new IllegalArgumentException(report.jobType() + " is not running for " + report.applicationId());
- assertFalse("Unexpected entry '" + report.jobType() + "@" + report.projectId() + " in: " + buildService().jobs(),
- buildService().remove(report.buildJob()));
-
- applications().deploymentTrigger().notifyOfCompletion(report);
- applications().deploymentTrigger().triggerReadyJobs();
+ public void timeOutConvergence(ApplicationId instanceId, JobType type) {
+ newDeploymentContext(instanceId).timeOutConvergence(type);
}
- public static ApplicationPackage applicationPackage(String upgradePolicy) {
- return new ApplicationPackageBuilder()
- .upgradePolicy(upgradePolicy)
- .environment(Environment.prod)
- .region("us-west-1")
- .region("us-east-3")
- .build();
+ public RunId startSystemTestTests() {
+ return defaultContext.startSystemTestTests();
}
- public void assertRunning(JobType job, ApplicationId application) {
- assertTrue(String.format("Job %s for %s is running", job, application), isRunning(job, application));
+ /** Creates and submits a new application, and then starts the job of the given type. Use only once per test. */
+ public RunId newRun(JobType type) {
+ return defaultContext.newRun(type);
}
- public void assertNotRunning(JobType job, ApplicationId application) {
- assertFalse(String.format("Job %s for %s is not running", job, application), isRunning(job, application));
+ public void assertRunning(JobType type) {
+ assertRunning(instanceId, type);
}
- private boolean isRunning(JobType job, ApplicationId application) {
- return buildService().jobs().contains(ControllerTester.buildJob(instance(application).id(), job));
+ public void assertRunning(ApplicationId id, JobType type) {
+ assertTrue(jobs.active().stream().anyMatch(run -> run.id().application().equals(id) && run.id().type() == type));
}
}