From fbedab8dfb0c2a0c04faa0da73197a622d77b8f3 Mon Sep 17 00:00:00 2001 From: Jon Marius Venstad Date: Thu, 31 Oct 2019 14:51:35 +0100 Subject: Remove DeploymentTester and rename InternalDeploymentTester --- .../vespa/hosted/controller/ControllerTest.java | 4 +- .../controller/deployment/DeploymentContext.java | 6 +- .../controller/deployment/DeploymentTester.java | 415 +++++++++------------ .../deployment/DeploymentTriggerTest.java | 13 +- .../deployment/InternalDeploymentTester.java | 297 --------------- .../deployment/InternalStepRunnerTest.java | 11 +- .../deployment/TestConfigSerializerTest.java | 2 +- .../ApplicationOwnershipConfirmerTest.java | 8 +- .../maintenance/DeploymentExpirerTest.java | 4 +- .../maintenance/DeploymentIssueReporterTest.java | 6 +- .../DeploymentMetricsMaintainerTest.java | 4 +- .../controller/maintenance/JobRunnerTest.java | 12 +- .../maintenance/MetricsReporterTest.java | 14 +- .../maintenance/OutstandingChangeDeployerTest.java | 4 +- .../maintenance/RotationStatusUpdaterTest.java | 4 +- .../maintenance/RoutingPoliciesTest.java | 6 +- .../controller/maintenance/UpgraderTest.java | 4 +- .../JobControllerApiHandlerHelperTest.java | 12 +- .../rotation/RotationRepositoryTest.java | 8 +- .../controller/versions/VersionStatusTest.java | 12 +- 20 files changed, 235 insertions(+), 611 deletions(-) delete mode 100644 controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalDeploymentTester.java 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 192c71094a5..c374787aaa4 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 @@ -32,7 +32,7 @@ import com.yahoo.vespa.hosted.controller.application.JobStatus; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; import com.yahoo.vespa.hosted.controller.deployment.BuildJob; import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.integration.ZoneApiMock; import com.yahoo.vespa.hosted.controller.rotation.RotationId; import com.yahoo.vespa.hosted.controller.rotation.RotationLock; @@ -68,7 +68,7 @@ import static org.junit.Assert.fail; */ public class ControllerTest { - private final InternalDeploymentTester tester = new InternalDeploymentTester(); + private final DeploymentTester tester = new DeploymentTester(); @Test public void testDeployment() { diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentContext.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentContext.java index b0adebfbb19..a5840cea3bd 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentContext.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentContext.java @@ -57,10 +57,10 @@ import static org.junit.Assert.assertTrue; * A deployment context for an application. This allows fine-grained control of the deployment of an application's * instances. * - * References to this should be acquired through {@link InternalDeploymentTester#deploymentContext}. + * References to this should be acquired through {@link DeploymentTester#deploymentContext}. * * Tester code that is not specific to deployments should be added to either {@link ControllerTester} or - * {@link InternalDeploymentTester} instead of this class. + * {@link DeploymentTester} instead of this class. * * @author mpolden */ @@ -99,7 +99,7 @@ public class DeploymentContext { private ApplicationVersion lastSubmission = null; private boolean deferDnsUpdates = false; - public DeploymentContext(ApplicationId instanceId, InternalDeploymentTester tester) { + public DeploymentContext(ApplicationId instanceId, DeploymentTester tester) { this.applicationId = TenantAndApplicationId.from(instanceId); this.instanceId = instanceId; 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 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 failOnJob) { - assertTrue(application.id() + " has pending changes to deploy", applications().requireApplication(application.id()).change().hasTargets()); - DeploymentSteps steps = controller().applications().deploymentTrigger().steps(applicationPackage.deploymentSpec()); - List 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 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, - 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, 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 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)); } } 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 38c9b6e3e49..07cddbe2c7e 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java @@ -3,28 +3,20 @@ package com.yahoo.vespa.hosted.controller.deployment; import com.yahoo.component.Version; import com.yahoo.config.provision.Environment; -import com.yahoo.config.provision.InstanceName; import com.yahoo.config.provision.zone.ZoneId; -import com.yahoo.vespa.hosted.controller.Application; -import com.yahoo.vespa.hosted.controller.Instance; import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions; -import com.yahoo.vespa.hosted.controller.api.integration.BuildService; 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.deployment.RunId; -import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockBuildService; import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.Change; -import com.yahoo.vespa.hosted.controller.application.DeploymentJobs; import com.yahoo.vespa.hosted.controller.versions.VespaVersion; import org.junit.Ignore; import org.junit.Test; import java.time.Duration; import java.time.Instant; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Optional; @@ -32,8 +24,6 @@ import java.util.OptionalLong; import java.util.stream.Collectors; import static com.yahoo.config.provision.SystemName.main; -import static com.yahoo.vespa.hosted.controller.ControllerTester.buildJob; -import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.component; import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.productionEuWest1; import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.productionUsCentral1; import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.productionUsEast3; @@ -42,7 +32,6 @@ import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobTy import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.systemTest; import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger.ChangesToCancel.ALL; import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger.ChangesToCancel.PLATFORM; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.appId; import static java.time.temporal.ChronoUnit.MILLIS; import static java.util.Collections.emptyList; import static org.junit.Assert.assertEquals; @@ -59,7 +48,7 @@ import static org.junit.Assert.assertTrue; */ public class DeploymentTriggerTest { - private final InternalDeploymentTester tester = new InternalDeploymentTester(); + private final DeploymentTester tester = new DeploymentTester(); @Test public void testTriggerFailing() { diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalDeploymentTester.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalDeploymentTester.java deleted file mode 100644 index d257392298e..00000000000 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalDeploymentTester.java +++ /dev/null @@ -1,297 +0,0 @@ -// 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.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.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.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; -import com.yahoo.vespa.hosted.controller.maintenance.Upgrader; - -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.Collections; -import java.util.Optional; -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; - -/** - * @author jonmv - */ -public class InternalDeploymentTester { - - // 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); - public static final String athenzDomain = "domain"; - - 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 NameServiceDispatcher nameServiceDispatcher; - - 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 InternalDeploymentTester() { - this(new ControllerTester()); - } - - public InternalDeploymentTester(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 OutstandingChangeDeployer outstandingChangeDeployer() { return outstandingChangeDeployer; } - - public NameServiceDispatcher nameServiceDispatcher() { - return nameServiceDispatcher; - } - - public InternalDeploymentTester atHourOfDay(int hour) { - var dateTime = tester.clock().instant().atZone(ZoneOffset.UTC); - return at(LocalDateTime.of(dateTime.getYear(), dateTime.getMonth(), dateTime.getDayOfMonth(), hour, - dateTime.getMinute(), dateTime.getSecond()) - .toInstant(ZoneOffset.UTC)); - } - - public InternalDeploymentTester at(Instant instant) { - tester.clock().setInstant(instant); - return this; - } - - /** Returns the default deployment context owned by this */ - public DeploymentContext deploymentContext() { - return defaultContext; - } - - /** Create a new deployment context for given application */ - public DeploymentContext newDeploymentContext(String tenantName, String applicationName, String instanceName) { - return newDeploymentContext(ApplicationId.from(tenantName, applicationName, instanceName)); - } - - /** Create a new deployment context for given application */ - public DeploymentContext newDeploymentContext(ApplicationId instance) { - return new DeploymentContext(instance, this); - } - - /** 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(); - } - - /** 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(); - } - - public ApplicationVersion newSubmission(TenantAndApplicationId id, ApplicationPackage applicationPackage) { - return newSubmission(id, applicationPackage, DeploymentContext.defaultSourceRevision); - } - - /** - * Submits a new application package, and returns the version of the new submission. - */ - public ApplicationVersion newSubmission(ApplicationPackage applicationPackage) { - return newSubmission(appId, applicationPackage); - } - - /** - * Submits a new application, and returns the version of the new submission. - */ - public ApplicationVersion newSubmission() { - return defaultContext.submit().lastSubmission().get(); - } - - /** - * 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); - } - - /** Completely deploys the given application version, assuming it is the last to be submitted. */ - public void deployNewSubmission(ApplicationVersion version) { - deployNewSubmission(appId, version); - } - - /** 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()); - } - - /** Completely deploys the given, new platform. */ - public void deployNewPlatform(Version version) { - deployNewPlatform(appId, version); - } - - /** Completely deploys the given, new platform. */ - public void deployNewPlatform(TenantAndApplicationId id, Version version) { - newDeploymentContext(id.defaultInstance()).deployPlatform(version); - } - - /** 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()); - } - } - - /** 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; - } - - /** 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); - } - - /** Pulls the ready job trigger, and then runs the whole of the given job, successfully. */ - public void runJob(JobType type) { - defaultContext.runJob(type); - } - - /** 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 failDeployment(JobType type) { - defaultContext.failDeployment(type); - } - - public void failDeployment(ApplicationId instanceId, JobType type) { - newDeploymentContext(instanceId).failDeployment(type); - } - - public void timeOutUpgrade(JobType type) { - defaultContext.timeOutUpgrade(type); - } - - public void timeOutUpgrade(ApplicationId instanceId, JobType type) { - newDeploymentContext(instanceId).timeOutConvergence(type); - } - - public void timeOutConvergence(JobType type) { - defaultContext.timeOutConvergence(type); - } - - public void timeOutConvergence(ApplicationId instanceId, JobType type) { - newDeploymentContext(instanceId).timeOutConvergence(type); - } - - public RunId startSystemTestTests() { - return defaultContext.startSystemTestTests(); - } - - /** 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 assertRunning(JobType type) { - assertRunning(instanceId, type); - } - - public void assertRunning(ApplicationId id, JobType type) { - assertTrue(jobs.active().stream().anyMatch(run -> run.id().application().equals(id) && run.id().type() == type)); - } - -} diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunnerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunnerTest.java index 57ec2d3130d..7be2b6a9797 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunnerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunnerTest.java @@ -50,11 +50,10 @@ import java.util.concurrent.Future; import static com.yahoo.vespa.hosted.controller.api.integration.LogEntry.Type.error; import static com.yahoo.vespa.hosted.controller.api.integration.LogEntry.Type.info; import static com.yahoo.vespa.hosted.controller.api.integration.LogEntry.Type.warning; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.appId; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.instanceId; +import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTester.instanceId; import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.applicationPackage; import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.publicCdApplicationPackage; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.testerId; +import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTester.testerId; import static com.yahoo.vespa.hosted.controller.deployment.Step.Status.failed; import static com.yahoo.vespa.hosted.controller.deployment.Step.Status.succeeded; import static com.yahoo.vespa.hosted.controller.deployment.Step.Status.unfinished; @@ -70,11 +69,11 @@ import static org.junit.Assert.fail; */ public class InternalStepRunnerTest { - private InternalDeploymentTester tester; + private DeploymentTester tester; @Before public void setup() { - tester = new InternalDeploymentTester(); + tester = new DeploymentTester(); } private SystemName system() { @@ -91,7 +90,7 @@ public class InternalStepRunnerTest { tester.newRun(JobType.stagingTest); tester.runner().run(); DeploymentSpec spec = tester.configServer() - .application(InternalDeploymentTester.testerId.id(), JobType.stagingTest.zone(system())).get() + .application(DeploymentTester.testerId.id(), JobType.stagingTest.zone(system())).get() .applicationPackage().deploymentSpec(); assertEquals("domain", spec.athenzDomain().get().value()); ZoneId zone = JobType.stagingTest.zone(system()); diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializerTest.java index e28e5dab555..2fc49c3dc21 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializerTest.java @@ -14,7 +14,7 @@ import java.nio.file.Paths; import java.util.List; import java.util.Map; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.instanceId; +import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTester.instanceId; import static org.junit.Assert.assertEquals; /** diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ApplicationOwnershipConfirmerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ApplicationOwnershipConfirmerTest.java index 18acd91969b..88d1626b834 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ApplicationOwnershipConfirmerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ApplicationOwnershipConfirmerTest.java @@ -11,7 +11,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.organization.OwnershipI import com.yahoo.vespa.hosted.controller.api.integration.organization.User; import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId; import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb; import com.yahoo.vespa.hosted.controller.tenant.UserTenant; import org.junit.Before; @@ -22,7 +22,7 @@ import java.util.List; import java.util.Optional; import java.util.function.Supplier; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.appId; +import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTester.appId; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -34,11 +34,11 @@ public class ApplicationOwnershipConfirmerTest { private MockOwnershipIssues issues; private ApplicationOwnershipConfirmer confirmer; - private InternalDeploymentTester tester; + private DeploymentTester tester; @Before public void setup() { - tester = new InternalDeploymentTester(); + tester = new DeploymentTester(); issues = new MockOwnershipIssues(); confirmer = new ApplicationOwnershipConfirmer(tester.controller(), Duration.ofDays(1), new JobControl(new MockCuratorDb()), issues); } diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java index 222ec9dfb66..4df1336ac1f 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java @@ -10,7 +10,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType; import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.Deployment; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb; import org.junit.Test; @@ -25,7 +25,7 @@ import static org.junit.Assert.assertEquals; */ public class DeploymentExpirerTest { - private final InternalDeploymentTester tester = new InternalDeploymentTester(); + private final DeploymentTester tester = new DeploymentTester(); @Test public void testDeploymentExpiry() { diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentIssueReporterTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentIssueReporterTest.java index fdc8185a45f..070171ad399 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentIssueReporterTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentIssueReporterTest.java @@ -13,7 +13,7 @@ import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.Change; import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb; import com.yahoo.vespa.hosted.controller.versions.VespaVersion; import org.junit.Before; @@ -49,13 +49,13 @@ public class DeploymentIssueReporterTest { .upgradePolicy("canary") .build(); - private InternalDeploymentTester tester; + private DeploymentTester tester; private DeploymentIssueReporter reporter; private MockDeploymentIssues issues; @Before public void setup() { - tester = new InternalDeploymentTester(); + tester = new DeploymentTester(); issues = new MockDeploymentIssues(); reporter = new DeploymentIssueReporter(tester.controller(), issues, Duration.ofDays(1), new JobControl(new MockCuratorDb())); } diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainerTest.java index d0ba13cf5bc..4e4fbe00bb7 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentMetricsMaintainerTest.java @@ -12,7 +12,7 @@ import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId; import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType; import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.Deployment; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import org.junit.Test; import java.time.Duration; @@ -31,7 +31,7 @@ import static org.junit.Assert.assertFalse; */ public class DeploymentMetricsMaintainerTest { - private final InternalDeploymentTester tester = new InternalDeploymentTester(); + private final DeploymentTester tester = new DeploymentTester(); @Test public void updates_metrics() { diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/JobRunnerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/JobRunnerTest.java index 7a341dc7441..0f6aec804e2 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/JobRunnerTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/JobRunnerTest.java @@ -8,7 +8,7 @@ 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.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.deployment.JobController; import com.yahoo.vespa.hosted.controller.deployment.Run; import com.yahoo.vespa.hosted.controller.deployment.RunStatus; @@ -76,7 +76,7 @@ public class JobRunnerTest { @Test public void multiThreadedExecutionFinishes() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); JobController jobs = tester.controller().jobController(); StepRunner stepRunner = (step, id) -> id.type() == stagingTest && step.get() == startTests? Optional.of(error) : Optional.of(running); Phaser phaser = new Phaser(1); @@ -109,7 +109,7 @@ public class JobRunnerTest { @Test public void stepLogic() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); JobController jobs = tester.controller().jobController(); Map outcomes = new EnumMap<>(Step.class); JobRunner runner = new JobRunner(tester.controller(), Duration.ofDays(1), new JobControl(tester.controller().curator()), @@ -197,7 +197,7 @@ public class JobRunnerTest { @Test public void locksAndGarbage() throws InterruptedException, BrokenBarrierException { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); JobController jobs = tester.controller().jobController(); // Hang during tester deployment, until notified. CyclicBarrier barrier = new CyclicBarrier(2); @@ -236,7 +236,7 @@ public class JobRunnerTest { @Test public void historyPruning() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); JobController jobs = tester.controller().jobController(); JobRunner runner = new JobRunner(tester.controller(), Duration.ofDays(1), new JobControl(tester.controller().curator()), inThreadExecutor(), (id, step) -> Optional.of(running)); @@ -264,7 +264,7 @@ public class JobRunnerTest { @Test public void timeout() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); JobController jobs = tester.controller().jobController(); Map outcomes = new EnumMap<>(Step.class); JobRunner runner = new JobRunner(tester.controller(), Duration.ofDays(1), new JobControl(tester.controller().curator()), diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java index c2cb427aaab..31f55be92a9 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java @@ -12,7 +12,7 @@ import com.yahoo.vespa.hosted.controller.ControllerTester; import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.SystemApplication; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.integration.MetricsMock; import com.yahoo.vespa.hosted.controller.integration.ZoneApiMock; import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb; @@ -36,7 +36,7 @@ public class MetricsReporterTest { @Test public void deployment_fail_ratio() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() .environment(Environment.prod) .region("us-west-1") @@ -70,7 +70,7 @@ public class MetricsReporterTest { @Test public void deployment_average_duration() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() .environment(Environment.prod) .region("us-west-1") @@ -115,7 +115,7 @@ public class MetricsReporterTest { @Test public void deployments_failing_upgrade() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() .environment(Environment.prod) .region("us-west-1") @@ -167,7 +167,7 @@ public class MetricsReporterTest { @Test public void deployment_warnings_metric() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() .environment(Environment.prod) .region("us-west-1") @@ -184,7 +184,7 @@ public class MetricsReporterTest { @Test public void build_time_reporting() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); var applicationPackage = new ApplicationPackageBuilder().region("us-west-1").build(); var context = tester.deploymentContext() .submit(applicationPackage) @@ -199,7 +199,7 @@ public class MetricsReporterTest { @Test public void name_service_queue_size_metric() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() .environment(Environment.prod) .globalServiceId("default") 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 0a6d5d7698d..0be873f80ed 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 @@ -10,7 +10,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.deployment.SourceRevisi import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.Change; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.deployment.Run; import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb; import org.junit.Test; @@ -29,7 +29,7 @@ public class OutstandingChangeDeployerTest { @Test public void testChangeDeployer() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); OutstandingChangeDeployer deployer = new OutstandingChangeDeployer(tester.controller(), Duration.ofMinutes(10), new JobControl(new MockCuratorDb())); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RotationStatusUpdaterTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RotationStatusUpdaterTest.java index fde6db37d62..d7a28708049 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RotationStatusUpdaterTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RotationStatusUpdaterTest.java @@ -6,7 +6,7 @@ import com.yahoo.config.provision.Environment; import com.yahoo.config.provision.zone.ZoneId; import com.yahoo.vespa.hosted.controller.api.integration.routing.RotationStatus; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.rotation.RotationState; import org.junit.Test; @@ -21,7 +21,7 @@ public class RotationStatusUpdaterTest { @Test public void updates_rotation_status() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); var globalRotationService = tester.controllerTester().serviceRegistry().globalRoutingServiceMock(); var updater = new RotationStatusUpdater(tester.controller(), Duration.ofDays(1), new JobControl(tester.controller().curator())); diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RoutingPoliciesTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RoutingPoliciesTest.java index b7671cf47c2..f5f1605a699 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RoutingPoliciesTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RoutingPoliciesTest.java @@ -6,9 +6,7 @@ import com.yahoo.config.provision.ApplicationId; import com.yahoo.config.provision.ClusterSpec; import com.yahoo.config.provision.HostName; import com.yahoo.config.provision.zone.ZoneId; -import com.yahoo.vespa.hosted.controller.Application; import com.yahoo.vespa.hosted.controller.Instance; -import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId; import com.yahoo.vespa.hosted.controller.api.integration.configserver.LoadBalancer; import com.yahoo.vespa.hosted.controller.api.integration.dns.Record; import com.yahoo.vespa.hosted.controller.api.integration.dns.RecordData; @@ -18,7 +16,7 @@ import com.yahoo.vespa.hosted.controller.application.RoutingPolicy; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; import com.yahoo.vespa.hosted.controller.deployment.BuildJob; import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import org.junit.Test; import java.net.URI; @@ -39,7 +37,7 @@ import static org.junit.Assert.assertTrue; */ public class RoutingPoliciesTest { - private final InternalDeploymentTester tester = new InternalDeploymentTester(); + private final DeploymentTester tester = new DeploymentTester(); private final DeploymentContext context1 = tester.newDeploymentContext(ApplicationId.from("tenant1", "app1", "default")); private final DeploymentContext context2 = tester.newDeploymentContext(ApplicationId.from("tenant1", "app2", "default")); 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 4e910fe2d80..58a49307733 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 @@ -8,7 +8,7 @@ import com.yahoo.vespa.hosted.controller.application.Change; import com.yahoo.vespa.hosted.controller.application.Deployment; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.versions.VespaVersion; import org.junit.Test; @@ -35,7 +35,7 @@ import static org.junit.Assert.assertTrue; */ public class UpgraderTest { - private final InternalDeploymentTester tester = new InternalDeploymentTester().atHourOfDay(5); + private final DeploymentTester tester = new DeploymentTester().atHourOfDay(5); @Test public void testUpgrading() { diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/JobControllerApiHandlerHelperTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/JobControllerApiHandlerHelperTest.java index a60c195519a..daf1a4c2ea7 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/JobControllerApiHandlerHelperTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/JobControllerApiHandlerHelperTest.java @@ -9,7 +9,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.configserver.ConfigServ 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.deployment.ApplicationPackageBuilder; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import org.json.JSONException; import org.json.JSONObject; import org.junit.Test; @@ -33,9 +33,9 @@ import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobTy import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.stagingTest; import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.systemTest; import static com.yahoo.vespa.hosted.controller.api.integration.deployment.TesterCloud.Status.FAILURE; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.instanceId; +import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTester.instanceId; import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.applicationPackage; -import static com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester.testerId; +import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTester.testerId; import static com.yahoo.vespa.hosted.controller.deployment.RunStatus.deploymentFailed; import static com.yahoo.vespa.hosted.controller.deployment.RunStatus.installationFailed; import static com.yahoo.vespa.hosted.controller.deployment.RunStatus.running; @@ -50,7 +50,7 @@ public class JobControllerApiHandlerHelperTest { @Test public void testResponses() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); tester.clock().setInstant(Instant.EPOCH); // Revision 1 gets deployed everywhere. @@ -127,7 +127,7 @@ public class JobControllerApiHandlerHelperTest { @Test public void testDevResponses() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); tester.clock().setInstant(Instant.EPOCH); ZoneId zone = JobType.devUsEast1.zone(tester.controller().system()); @@ -146,7 +146,7 @@ public class JobControllerApiHandlerHelperTest { @Test public void testResponsesWithDirectDeployment() { - var tester = new InternalDeploymentTester(); + var tester = new DeploymentTester(); tester.clock().setInstant(Instant.EPOCH); var region = "us-west-1"; var applicationPackage = new ApplicationPackageBuilder().region(region).build(); diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/rotation/RotationRepositoryTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/rotation/RotationRepositoryTest.java index 70775bc9c00..c76046b3f67 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/rotation/RotationRepositoryTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/rotation/RotationRepositoryTest.java @@ -7,7 +7,7 @@ import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.AssignedRotation; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.rotation.config.RotationsConfig; import org.junit.Before; import org.junit.Rule; @@ -48,13 +48,13 @@ public class RotationRepositoryTest { .region("us-west-1") .build(); - private InternalDeploymentTester tester; + private DeploymentTester tester; private RotationRepository repository; private DeploymentContext application; @Before public void before() { - tester = new InternalDeploymentTester(new ControllerTester(rotationsConfig)); + tester = new DeploymentTester(new ControllerTester(rotationsConfig)); repository = tester.applications().rotationRepository(); application = tester.newDeploymentContext("tenant1", "app1", "default"); } @@ -82,7 +82,7 @@ public class RotationRepositoryTest { @Test public void strips_whitespace_in_rotation_fqdn() { - tester = new InternalDeploymentTester(new ControllerTester(rotationsConfigWhitespaces)); + tester = new DeploymentTester(new ControllerTester(rotationsConfigWhitespaces)); RotationRepository repository = tester.controller().applications().rotationRepository(); var application2 = tester.newDeploymentContext("tenant1", "app2", "default"); diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/versions/VersionStatusTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/versions/VersionStatusTest.java index bd4fd3d29f7..5c68fd2e370 100644 --- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/versions/VersionStatusTest.java +++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/versions/VersionStatusTest.java @@ -14,7 +14,7 @@ import com.yahoo.vespa.hosted.controller.application.ApplicationPackage; import com.yahoo.vespa.hosted.controller.application.SystemApplication; import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder; import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext; -import com.yahoo.vespa.hosted.controller.deployment.InternalDeploymentTester; +import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester; import com.yahoo.vespa.hosted.controller.persistence.CuratorDb; import com.yahoo.vespa.hosted.controller.persistence.MockCuratorDb; import com.yahoo.vespa.hosted.controller.versions.VespaVersion.Confidence; @@ -122,7 +122,7 @@ public class VersionStatusTest { @Test public void testVersionStatusAfterApplicationUpdates() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); ApplicationPackage applicationPackage = new ApplicationPackageBuilder() .upgradePolicy("default") .environment(Environment.prod) @@ -177,7 +177,7 @@ public class VersionStatusTest { @Test public void testVersionConfidence() { - InternalDeploymentTester tester = new InternalDeploymentTester().atHourOfDay(5); + DeploymentTester tester = new DeploymentTester().atHourOfDay(5); Version version0 = new Version("6.2"); tester.controllerTester().upgradeSystem(version0); tester.upgrader().maintain(); @@ -352,7 +352,7 @@ public class VersionStatusTest { @Test public void testConfidenceOverride() { - InternalDeploymentTester tester = new InternalDeploymentTester(); + DeploymentTester tester = new DeploymentTester(); Version version0 = new Version("6.2"); tester.controllerTester().upgradeSystem(version0); @@ -389,7 +389,7 @@ public class VersionStatusTest { MockCuratorDb db = new MockCuratorDb(Stream.of(controller1, controller2, controller3) .map(hostName -> hostName.value() + ":2222") .collect(Collectors.joining(","))); - InternalDeploymentTester tester = new InternalDeploymentTester(new ControllerTester(db)); + DeploymentTester tester = new DeploymentTester(new ControllerTester(db)); // Commit details are set for initial version var version0 = tester.controllerTester().nextVersion(); @@ -420,7 +420,7 @@ public class VersionStatusTest { @Test public void testConfidenceChangeRespectsTimeWindow() { - InternalDeploymentTester tester = new InternalDeploymentTester().atHourOfDay(5); + DeploymentTester tester = new DeploymentTester().atHourOfDay(5); // Canaries and normal application deploys on initial version Version version0 = Version.fromString("7.1"); tester.controllerTester().upgradeSystem(version0); -- cgit v1.2.3