summaryrefslogtreecommitdiffstats
path: root/controller-server
diff options
context:
space:
mode:
authorJon Marius Venstad <jonmv@users.noreply.github.com>2021-03-18 18:05:25 +0100
committerGitHub <noreply@github.com>2021-03-18 18:05:25 +0100
commit87281511a8f8a9f35a96a1ea557ce26933981c7c (patch)
treec4aa057c71def8a86e515730b66ead9d8b9562a1 /controller-server
parentd4a82f387a4ed5104b0c6201aae0dbc4492c24f1 (diff)
parent36eb3e232b50eaa3fb687e6a049214633759b5c6 (diff)
Merge pull request #17037 from vespa-engine/mpolden/deploy-cleanup
Remove legacy deployment path
Diffstat (limited to 'controller-server')
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java123
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/SystemApplication.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java7
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgrader.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java108
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/OsVersionStatus.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java50
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java17
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentContext.java7
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java8
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java26
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java84
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/JobControllerApiHandlerHelperTest.java6
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/cost-report.json20
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/create-user-response.json3
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-activation-conflict.json4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-failure.json4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-internal-server-error.json4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-no-deployment.json1
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-out-of-capacity.json4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-overview.json1
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list-with-user.json18
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list.json11
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java14
26 files changed, 81 insertions, 455 deletions
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java
index 0a1af8817d7..51b7a24b5e4 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java
@@ -22,9 +22,7 @@ import com.yahoo.vespa.flags.FlagSource;
import com.yahoo.vespa.flags.PermanentFlags;
import com.yahoo.vespa.flags.StringFlag;
import com.yahoo.vespa.hosted.controller.api.ActivateResult;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeploymentData;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.configserverbindings.ConfigChangeActions;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
import com.yahoo.vespa.hosted.controller.api.identifiers.InstanceId;
import com.yahoo.vespa.hosted.controller.api.identifiers.RevisionId;
@@ -39,12 +37,10 @@ import com.yahoo.vespa.hosted.controller.api.integration.configserver.ContainerE
import com.yahoo.vespa.hosted.controller.api.integration.configserver.Log;
import com.yahoo.vespa.hosted.controller.api.integration.configserver.Node;
import com.yahoo.vespa.hosted.controller.api.integration.configserver.NotFoundException;
-import com.yahoo.vespa.hosted.controller.api.integration.configserver.PrepareResponse;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationStore;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationVersion;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.ArtifactRepository;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobId;
-import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.TesterId;
import com.yahoo.vespa.hosted.controller.api.integration.noderepository.RestartFilter;
import com.yahoo.vespa.hosted.controller.api.integration.secrets.TenantSecretStore;
@@ -63,7 +59,6 @@ import com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger;
import com.yahoo.vespa.hosted.controller.deployment.JobStatus;
import com.yahoo.vespa.hosted.controller.deployment.Run;
import com.yahoo.vespa.hosted.controller.deployment.RunStatus;
-import com.yahoo.vespa.hosted.controller.deployment.Versions;
import com.yahoo.vespa.hosted.controller.persistence.CuratorDb;
import com.yahoo.vespa.hosted.controller.security.AccessControl;
import com.yahoo.vespa.hosted.controller.security.Credentials;
@@ -348,14 +343,8 @@ public class ApplicationController {
return application.withNewInstance(id.instance());
}
- public ActivateResult deploy(ApplicationId applicationId, ZoneId zone,
- Optional<ApplicationPackage> applicationPackageFromDeployer,
- DeployOptions options) {
- return deploy(applicationId, zone, applicationPackageFromDeployer, Optional.empty(), options);
- }
-
/** Deploys an application package for an existing application instance. */
- public ActivateResult deploy2(JobId job, boolean deploySourceVersions) { // TODO jonmv: make it number one!
+ public ActivateResult deploy(JobId job, boolean deploySourceVersions) {
if (job.application().instance().isTester())
throw new IllegalArgumentException("'" + job.application() + "' is a tester application!");
@@ -424,93 +413,6 @@ public class ApplicationController {
: applicationStore.get(application.tenant(), application.application(), revision));
}
- public ActivateResult deploy(ApplicationId instanceId, ZoneId zone,
- Optional<ApplicationPackage> applicationPackageFromDeployer,
- Optional<ApplicationVersion> applicationVersionFromDeployer,
- DeployOptions options) {
- if (instanceId.instance().isTester())
- throw new IllegalArgumentException("'" + instanceId + "' is a tester application!");
-
- TenantAndApplicationId applicationId = TenantAndApplicationId.from(instanceId);
- if (getInstance(instanceId).isEmpty())
- createInstance(instanceId);
-
- try (Lock deploymentLock = lockForDeployment(instanceId, zone)) {
- Version platformVersion;
- ApplicationVersion applicationVersion;
- ApplicationPackage applicationPackage;
- Set<ContainerEndpoint> endpoints;
- Optional<EndpointCertificateMetadata> endpointCertificateMetadata;
-
- try (Lock lock = lock(applicationId)) {
- LockedApplication application = new LockedApplication(requireApplication(applicationId), lock);
- InstanceName instance = instanceId.instance();
-
- boolean manuallyDeployed = options.deployDirectly || zone.environment().isManuallyDeployed();
- boolean preferOldestVersion = options.deployCurrentVersion;
-
- // Determine versions to use.
- if (manuallyDeployed) {
- applicationVersion = applicationVersionFromDeployer.orElse(ApplicationVersion.unknown);
- applicationPackage = applicationPackageFromDeployer.orElseThrow(
- () -> new IllegalArgumentException("Application package must be given when deploying to " + zone));
- platformVersion = options.vespaVersion.map(Version::new)
- .orElse(applicationPackage.deploymentSpec().majorVersion()
- .flatMap(this::lastCompatibleVersion)
- .orElseGet(controller::readSystemVersion));
- }
- else {
- JobType jobType = JobType.from(controller.system(), zone)
- .orElseThrow(() -> new IllegalArgumentException("No job is known for " + zone + "."));
- var run = controller.jobController().last(instanceId, jobType);
- if (run.map(Run::hasEnded).orElse(true))
- return unexpectedDeployment(instanceId, zone);
- Versions versions = run.get().versions();
- platformVersion = preferOldestVersion ? versions.sourcePlatform().orElse(versions.targetPlatform())
- : versions.targetPlatform();
- applicationVersion = preferOldestVersion ? versions.sourceApplication().orElse(versions.targetApplication())
- : versions.targetApplication();
-
- applicationPackage = getApplicationPackage(instanceId, applicationVersion);
- applicationPackage = withTesterCertificate(applicationPackage, instanceId, jobType);
- validateRun(application.get().require(instance), zone, platformVersion, applicationVersion);
- }
-
- endpointCertificateMetadata = endpointCertificateManager.getEndpointCertificateMetadata(
- application.get().require(instance), zone, applicationPackage.deploymentSpec().instance(instance));
-
- endpoints = controller.routing().registerEndpointsInDns(application.get(), instance, zone);
- } // Release application lock while doing the deployment, which is a lengthy task.
-
- // Carry out deployment without holding the application lock.
- ActivateResult result = deploy(instanceId, applicationPackage, zone, platformVersion,
- endpoints, endpointCertificateMetadata, Optional.empty());
-
- // Record the quota usage for this application
- var quotaUsage = deploymentQuotaUsage(zone, instanceId);
-
- lockApplicationOrThrow(applicationId, application ->
- store(application.with(instanceId.instance(),
- instance -> instance.withNewDeployment(zone, applicationVersion, platformVersion,
- clock.instant(), warningsFrom(result),
- quotaUsage))));
- return result;
- }
- }
-
- private ApplicationPackage withTesterCertificate(ApplicationPackage applicationPackage, ApplicationId id, JobType type) {
- if (applicationPackage.trustedCertificates().isEmpty())
- return applicationPackage;
-
- // TODO jonmv: move this to the caller, when external build service is removed.
- Run run = controller.jobController().last(id, type)
- .orElseThrow(() -> new IllegalStateException("Last run of " + type + " for " + id + " not found"));
- if (run.testerCertificate().isEmpty())
- return applicationPackage;
-
- return applicationPackage.withTrustedCertificate(run.testerCertificate().get());
- }
-
/** Fetches the requested application package from the artifact store(s). */
public ApplicationPackage getApplicationPackage(ApplicationId id, ApplicationVersion version) {
return new ApplicationPackage(applicationStore.get(id.tenant(), id.application(), version));
@@ -616,18 +518,6 @@ public class ApplicationController {
}
}
- private ActivateResult unexpectedDeployment(ApplicationId application, ZoneId zone) {
- Log logEntry = new Log();
- logEntry.level = "WARNING";
- logEntry.time = clock.instant().toEpochMilli();
- logEntry.message = "Ignoring deployment of application '" + application + "' to " + zone +
- " as a deployment is not currently expected";
- PrepareResponse prepareResponse = new PrepareResponse();
- prepareResponse.log = List.of(logEntry);
- prepareResponse.configChangeActions = new ConfigChangeActions(List.of(), List.of(), List.of());
- return new ActivateResult(new RevisionId("0"), prepareResponse, 0);
- }
-
private LockedApplication withoutDeletedDeployments(LockedApplication application, InstanceName instance) {
DeploymentSpec deploymentSpec = application.get().deploymentSpec();
List<ZoneId> deploymentsToRemove = application.get().require(instance).productionDeployments().values().stream()
@@ -844,17 +734,6 @@ public class ApplicationController {
return curator.lockForDeployment(application, zone);
}
- /** Verify that we don't downgrade an existing production deployment. */
- private void validateRun(Instance instance, ZoneId zone, Version platformVersion, ApplicationVersion applicationVersion) {
- Deployment deployment = instance.deployments().get(zone);
- if ( zone.environment().isProduction() && deployment != null
- && ( platformVersion.compareTo(deployment.version()) < 0 && ! instance.change().isPinned()
- || applicationVersion.compareTo(deployment.applicationVersion()) < 0))
- throw new IllegalArgumentException(String.format("Rejecting deployment of application %s to %s, as the requested versions (platform: %s, application: %s)" +
- " are older than the currently deployed (platform: %s, application: %s).",
- instance.id(), zone, platformVersion, applicationVersion, deployment.version(), deployment.applicationVersion()));
- }
-
/**
* Verifies that the application can be deployed to the tenant, following these rules:
*
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/SystemApplication.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/SystemApplication.java
index c994f0ae4fd..1a1b6988a96 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/SystemApplication.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/SystemApplication.java
@@ -65,8 +65,8 @@ public enum SystemApplication {
.orElse(false);
}
- /** Returns whether this should receive OS upgrades in given zone */
- public boolean shouldUpgradeOsIn() {
+ /** Returns whether this should receive OS upgrades */
+ public boolean shouldUpgradeOs() {
return nodeType.isHost();
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java
index e1587d250ec..f2df1cce15b 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java
@@ -184,7 +184,7 @@ public class InternalStepRunner implements StepRunner {
}
private Optional<RunStatus> deployReal(RunId id, boolean setTheStage, DualLogger logger) {
- return deploy(() -> controller.applications().deploy2(id.job(), setTheStage),
+ return deploy(() -> controller.applications().deploy(id.job(), setTheStage),
controller.jobController().run(id).get()
.stepInfo(setTheStage ? deployInitialReal : deployReal).get()
.startTime().get(),
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java
index f19c07e4650..65d3f666309 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java
@@ -41,10 +41,8 @@ import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.Optional;
-import java.util.Queue;
import java.util.Set;
import java.util.SortedMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
@@ -95,7 +93,7 @@ public class JobController {
this.logs = new BufferedLogStore(curator, controller.serviceRegistry().runDataStore());
this.cloud = controller.serviceRegistry().testerCloud();
this.badges = new Badges(controller.zoneRegistry().badgeUrl());
- this.metric = new JobMetrics(controller.metric(), controller.system());
+ this.metric = new JobMetrics(controller.metric(), controller::system);
}
public TesterCloud cloud() { return cloud; }
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java
index 80924c3c0aa..49da987ea18 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java
@@ -6,6 +6,7 @@ import com.yahoo.jdisc.Metric;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobId;
import java.util.Map;
+import java.util.function.Supplier;
/**
* Records metrics related to deployment jobs.
@@ -25,9 +26,9 @@ public class JobMetrics {
public static final String success = "deployment.success";
private final Metric metric;
- private final SystemName system;
+ private final Supplier<SystemName> system;
- public JobMetrics(Metric metric, SystemName system) {
+ public JobMetrics(Metric metric, Supplier<SystemName> system) {
this.metric = metric;
this.system = system;
}
@@ -45,7 +46,7 @@ public class JobMetrics {
"tenantName", id.application().tenant().value(),
"app", id.application().application().value() + "." + id.application().instance().value(),
"test", Boolean.toString(id.type().isTest()),
- "zone", id.type().zone(system).value());
+ "zone", id.type().zone(system.get()).value());
}
static String valueOf(RunStatus status) {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgrader.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgrader.java
index a20043d35c0..43e9ce51040 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgrader.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgrader.java
@@ -62,7 +62,7 @@ public class OsUpgrader extends InfrastructureUpgrader<OsVersionTarget> {
@Override
protected boolean expectUpgradeOf(Node node, SystemApplication application, ZoneApi zone) {
return cloud.equals(zone.getCloudName()) && // Cloud is managed by this upgrader
- application.shouldUpgradeOsIn() && // Application should upgrade in this cloud
+ application.shouldUpgradeOs() && // Application should upgrade in this cloud
canUpgrade(node); // Node is in an upgradable state
}
@@ -76,7 +76,7 @@ public class OsUpgrader extends InfrastructureUpgrader<OsVersionTarget> {
@Override
protected boolean changeTargetTo(OsVersionTarget target, SystemApplication application, ZoneApi zone) {
- if (!application.shouldUpgradeOsIn()) return false;
+ if (!application.shouldUpgradeOs()) return false;
return controller().serviceRegistry().configServer().nodeRepository()
.targetVersionsOf(zone.getId())
.osVersion(application.nodeType())
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
index df9491c5d22..e5ada9365f0 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
@@ -43,7 +43,6 @@ import com.yahoo.vespa.hosted.controller.LockedTenant;
import com.yahoo.vespa.hosted.controller.NotExistsException;
import com.yahoo.vespa.hosted.controller.api.ActivateResult;
import com.yahoo.vespa.hosted.controller.api.application.v4.EnvironmentResource;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.EndpointStatus;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.ProtonMetrics;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.configserverbindings.RefeedAction;
@@ -1841,99 +1840,30 @@ public class ApplicationApiHandler extends LoggingRequestHandler {
return ErrorResponse.badRequest("Missing required form part 'deployOptions'");
Inspector deployOptions = SlimeUtils.jsonToSlime(dataParts.get("deployOptions")).get();
- /*
- * Special handling of the proxy application (the only system application with an application package)
- * Setting any other deployOptions here is not supported for now (e.g. specifying version), but
- * this might be handy later to handle emergency downgrades.
- */
- boolean isZoneApplication = SystemApplication.proxy.id().equals(applicationId);
- if (isZoneApplication) { // TODO jvenstad: Separate out.
- // Make it explicit that version is not yet supported here
- String versionStr = deployOptions.field("vespaVersion").asString();
- boolean versionPresent = !versionStr.isEmpty() && !versionStr.equals("null");
- if (versionPresent) {
- throw new RuntimeException("Version not supported for system applications");
- }
- // To avoid second guessing the orchestrated upgrades of system applications
- // we don't allow to deploy these during an system upgrade (i.e when new vespa is being rolled out)
- VersionStatus versionStatus = controller.readVersionStatus();
- if (versionStatus.isUpgrading()) {
- throw new IllegalArgumentException("Deployment of system applications during a system upgrade is not allowed");
- }
- Optional<VespaVersion> systemVersion = versionStatus.systemVersion();
- if (systemVersion.isEmpty()) {
- throw new IllegalArgumentException("Deployment of system applications is not permitted until system version is determined");
- }
- ActivateResult result = controller.applications()
- .deploySystemApplicationPackage(SystemApplication.proxy, zone, systemVersion.get().versionNumber());
- return new SlimeJsonResponse(toSlime(result));
+ // Resolve system application
+ Optional<SystemApplication> systemApplication = SystemApplication.matching(applicationId);
+ if (systemApplication.isEmpty() || !systemApplication.get().hasApplicationPackage()) {
+ return ErrorResponse.badRequest("Deployment of " + applicationId + " is not supported through this API");
}
- /*
- * Normal applications from here
- */
-
- Optional<ApplicationPackage> applicationPackage = Optional.ofNullable(dataParts.get("applicationZip"))
- .map(ApplicationPackage::new);
- Optional<Application> application = controller.applications().getApplication(TenantAndApplicationId.from(applicationId));
-
- Inspector sourceRevision = deployOptions.field("sourceRevision");
- Inspector buildNumber = deployOptions.field("buildNumber");
- if (sourceRevision.valid() != buildNumber.valid())
- throw new IllegalArgumentException("Source revision and build number must both be provided, or not");
-
- Optional<ApplicationVersion> applicationVersion = Optional.empty();
- if (sourceRevision.valid()) {
- if (applicationPackage.isPresent())
- throw new IllegalArgumentException("Application version and application package can't both be provided.");
-
- applicationVersion = Optional.of(ApplicationVersion.from(toSourceRevision(sourceRevision),
- buildNumber.asLong()));
- applicationPackage = Optional.of(controller.applications().getApplicationPackage(applicationId,
- applicationVersion.get()));
+ // Make it explicit that version is not yet supported here
+ String vespaVersion = deployOptions.field("vespaVersion").asString();
+ if (!vespaVersion.isEmpty() && !vespaVersion.equals("null")) {
+ return ErrorResponse.badRequest("Specifying version for " + applicationId + " is not permitted");
}
- boolean deployDirectly = deployOptions.field("deployDirectly").asBool();
- Optional<Version> vespaVersion = optional("vespaVersion", deployOptions).map(Version::new);
-
- if (deployDirectly && applicationPackage.isEmpty() && applicationVersion.isEmpty() && vespaVersion.isEmpty()) {
-
- // Redeploy the existing deployment with the same versions.
- Optional<Deployment> deployment = controller.applications().getInstance(applicationId)
- .map(Instance::deployments)
- .flatMap(deployments -> Optional.ofNullable(deployments.get(zone)));
-
- if(deployment.isEmpty())
- throw new IllegalArgumentException("Can't redeploy application, no deployment currently exist");
-
- ApplicationVersion version = deployment.get().applicationVersion();
- if(version.isUnknown())
- throw new IllegalArgumentException("Can't redeploy application, application version is unknown");
-
- applicationVersion = Optional.of(version);
- vespaVersion = Optional.of(deployment.get().version());
- applicationPackage = Optional.of(controller.applications().getApplicationPackage(applicationId,
- applicationVersion.get()));
+ // To avoid second guessing the orchestrated upgrades of system applications
+ // we don't allow to deploy these during an system upgrade (i.e when new vespa is being rolled out)
+ VersionStatus versionStatus = controller.readVersionStatus();
+ if (versionStatus.isUpgrading()) {
+ throw new IllegalArgumentException("Deployment of system applications during a system upgrade is not allowed");
}
-
- // TODO: get rid of the json object
- DeployOptions deployOptionsJsonClass = new DeployOptions(deployDirectly,
- vespaVersion,
- deployOptions.field("ignoreValidationErrors").asBool(),
- deployOptions.field("deployCurrentVersion").asBool());
-
- applicationPackage.ifPresent(aPackage -> controller.applications().verifyApplicationIdentityConfiguration(applicationId.tenant(),
- Optional.of(applicationId.instance()),
- Optional.of(zone),
- aPackage,
- Optional.of(requireUserPrincipal(request))));
-
- ActivateResult result = controller.applications().deploy(applicationId,
- zone,
- applicationPackage,
- applicationVersion,
- deployOptionsJsonClass);
-
+ Optional<VespaVersion> systemVersion = versionStatus.systemVersion();
+ if (systemVersion.isEmpty()) {
+ throw new IllegalArgumentException("Deployment of system applications is not permitted until system version is determined");
+ }
+ ActivateResult result = controller.applications()
+ .deploySystemApplicationPackage(systemApplication.get(), zone, systemVersion.get().versionNumber());
return new SlimeJsonResponse(toSlime(result));
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/OsVersionStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/OsVersionStatus.java
index ac9f32a2ae6..226852f1f3d 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/OsVersionStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/OsVersionStatus.java
@@ -63,7 +63,7 @@ public class OsVersionStatus {
for (var application : SystemApplication.all()) {
for (var zone : zonesToUpgrade(controller)) {
- if (!application.shouldUpgradeOsIn()) continue;
+ if (!application.shouldUpgradeOs()) continue;
var targetOsVersion = controller.serviceRegistry().configServer().nodeRepository()
.targetVersionsOf(zone.getId())
.osVersion(application.nodeType())
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 07c0a6ca122..362c980a906 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
@@ -13,12 +13,10 @@ import com.yahoo.config.provision.ClusterSpec;
import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.HostName;
import com.yahoo.config.provision.RegionName;
-import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.TenantName;
import com.yahoo.config.provision.zone.RoutingMethod;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.path.Path;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.EndpointStatus;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
import com.yahoo.vespa.hosted.controller.api.integration.certificates.EndpointCertificateMetadata;
@@ -32,7 +30,6 @@ import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.Deployment;
import com.yahoo.vespa.hosted.controller.application.DeploymentMetrics;
import com.yahoo.vespa.hosted.controller.application.Endpoint;
-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.DeploymentTester;
@@ -580,41 +577,6 @@ public class ControllerTest {
}
@Test
- public void testIntegrationTestDeployment() {
- Version six = Version.fromString("6.1");
- tester.controllerTester().zoneRegistry().setSystemName(SystemName.cd);
- tester.controllerTester().zoneRegistry().setZones(ZoneApiMock.fromId("prod.cd-us-central-1"));
- tester.configServer().bootstrap(List.of(ZoneId.from("prod.cd-us-central-1")), SystemApplication.all());
- tester.controllerTester().upgradeSystem(six);
- ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .majorVersion(6)
- .region("cd-us-central-1")
- .build();
-
- // Create application
- var context = tester.newDeploymentContext();
-
- // Direct deploy is allowed when deployDirectly is true
- ZoneId zone = ZoneId.from("prod", "cd-us-central-1");
- // Same options as used in our integration tests
- DeployOptions options = new DeployOptions(true, Optional.empty(), false,
- false);
- tester.controller().applications().deploy(context.instanceId(), zone, Optional.of(applicationPackage), options);
-
- assertTrue("Application deployed and activated",
- tester.configServer().application(context.instanceId(), zone).get().activated());
-
- assertTrue("No job status added",
- context.instanceJobs().isEmpty());
-
- Version seven = Version.fromString("7.2");
- tester.controllerTester().upgradeSystem(seven);
- tester.upgrader().maintain();
- tester.controller().applications().deploy(context.instanceId(), zone, Optional.of(applicationPackage), options);
- assertEquals(six, context.instance().deployments().get(zone).version());
- }
-
- @Test
public void testDevDeployment() {
ApplicationPackage applicationPackage = new ApplicationPackageBuilder().build();
@@ -625,7 +587,7 @@ public class ControllerTest {
.setRoutingMethod(ZoneApiMock.from(zone), RoutingMethod.shared, RoutingMethod.sharedLayer4);
// Deploy
- tester.controller().applications().deploy(context.instanceId(), zone, Optional.of(applicationPackage), DeployOptions.none());
+ context.runJob(zone, applicationPackage);
assertTrue("Application deployed and activated",
tester.configServer().application(context.instanceId(), zone).get().activated());
assertTrue("No job status added",
@@ -682,10 +644,10 @@ public class ControllerTest {
.region("us-west-1")
.build();
- ZoneId zone = ZoneId.from("prod", "us-west-1");
- tester.controller().applications().deploy(context.instanceId(), zone, Optional.of(applicationPackage), DeployOptions.none());
- tester.controller().applications().deactivate(context.instanceId(), ZoneId.from(Environment.prod, RegionName.from("us-west-1")));
- tester.controller().applications().deactivate(context.instanceId(), ZoneId.from(Environment.prod, RegionName.from("us-west-1")));
+ ZoneId zone = ZoneId.from(Environment.prod, RegionName.from("us-west-1"));
+ context.runJob(zone, applicationPackage);
+ tester.controller().applications().deactivate(context.instanceId(), zone);
+ tester.controller().applications().deactivate(context.instanceId(), zone);
}
@Test
@@ -748,7 +710,7 @@ public class ControllerTest {
var devZone = ZoneId.from("dev", "us-east-1");
// Deploy app2 in a zone with shared routing
- tester.controller().applications().deploy(context2.instanceId(), devZone, Optional.of(applicationPackage), DeployOptions.none());
+ context2.runJob(devZone, applicationPackage);
assertTrue("Application deployed and activated",
tester.configServer().application(context2.instanceId(), devZone).get().activated());
assertTrue("Provisions certificate also in zone with routing layer", certificate.apply(context2.instance()).isPresent());
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java
index f432a1f41ce..f8645139244 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java
@@ -16,7 +16,6 @@ import com.yahoo.vespa.athenz.api.AthenzUser;
import com.yahoo.vespa.athenz.api.OktaAccessToken;
import com.yahoo.vespa.athenz.api.OktaIdentityToken;
import com.yahoo.vespa.flags.InMemoryFlagSource;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.identifiers.Property;
import com.yahoo.vespa.hosted.controller.api.identifiers.PropertyId;
import com.yahoo.vespa.hosted.controller.api.integration.athenz.AthenzClientFactoryMock;
@@ -27,7 +26,6 @@ import com.yahoo.vespa.hosted.controller.api.integration.dns.RecordName;
import com.yahoo.vespa.hosted.controller.api.integration.organization.Contact;
import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockMavenRepository;
import com.yahoo.vespa.hosted.controller.api.role.SimplePrincipal;
-import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.SystemApplication;
import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId;
import com.yahoo.vespa.hosted.controller.athenz.impl.AthenzFacade;
@@ -347,21 +345,6 @@ public final class ControllerTester {
return application;
}
- public void deploy(ApplicationId id, ZoneId zone, ApplicationPackage applicationPackage, boolean deployCurrentVersion) {
- deploy(id, zone, Optional.of(applicationPackage), deployCurrentVersion);
- }
-
- public void deploy(ApplicationId id, ZoneId zone, Optional<ApplicationPackage> applicationPackage, boolean deployCurrentVersion) {
- deploy(id, zone, applicationPackage, deployCurrentVersion, Optional.empty());
- }
-
- public void deploy(ApplicationId id, ZoneId zone, Optional<ApplicationPackage> applicationPackage, boolean deployCurrentVersion, Optional<Version> version) {
- controller().applications().deploy(id,
- zone,
- applicationPackage,
- new DeployOptions(false, version, false, deployCurrentVersion));
- }
-
private static Controller createController(CuratorDb curator, RotationsConfig rotationsConfig,
AthenzDbMock athensDb,
ServiceRegistryMock serviceRegistry) {
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 289d48d3241..4a4159180b5 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
@@ -328,11 +328,16 @@ public class DeploymentContext {
return runJob(type);
}
- /** Runs a deployment of the given package to the given dev/perf job. */
+ /** Runs a deployment of the given package to the given manually deployable job. */
public DeploymentContext runJob(JobType type, ApplicationPackage applicationPackage) {
return runJob(type, applicationPackage, null);
}
+ /** Runs a deployment of the given package to the given manually deployable zone. */
+ public DeploymentContext runJob(ZoneId zone, ApplicationPackage applicationPackage) {
+ return runJob(JobType.from(tester.controller().system(), zone).get(), applicationPackage, null);
+ }
+
/** Pulls the ready job trigger, and then runs the whole of the given job, successfully. */
public DeploymentContext runJob(JobType type) {
var job = jobId(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 724ba61da2b..81c9f51278e 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
@@ -6,7 +6,6 @@ import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.zone.RoutingMethod;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.ControllerTester;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
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;
@@ -628,13 +627,8 @@ public class DeploymentTriggerTest {
assertEquals("First deployment gets system version", version1, app1.application().oldestDeployedPlatform().get());
assertEquals(version1, tester.configServer().lastPrepareVersion().get());
- // Unexpected deployment is ignored
- Version version2 = new Version(version1.getMajor(), version1.getMinor() + 1);
- tester.applications().deploy(app1.instanceId(), ZoneId.from("prod", "us-west-1"),
- Optional.empty(), new DeployOptions(false, Optional.of(version2), false, false));
- assertEquals(version1, app1.deployment(ZoneId.from("prod", "us-west-1")).version());
-
// Application change after a new system version, and a region added
+ Version version2 = new Version(version1.getMajor(), version1.getMinor() + 1);
tester.controllerTester().upgradeSystem(version2);
applicationPackage = new ApplicationPackageBuilder()
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java
index 62865ea9be2..51e63d5b4cf 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java
@@ -4,13 +4,12 @@ package com.yahoo.vespa.hosted.controller.restapi.application;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.ApplicationName;
import com.yahoo.config.provision.TenantName;
-import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.flags.Flags;
import com.yahoo.vespa.flags.InMemoryFlagSource;
import com.yahoo.vespa.flags.PermanentFlags;
import com.yahoo.vespa.hosted.controller.LockedTenant;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.integration.billing.PlanId;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
import com.yahoo.vespa.hosted.controller.api.integration.secrets.TenantSecretStore;
import com.yahoo.vespa.hosted.controller.api.role.Role;
import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId;
@@ -29,7 +28,10 @@ import java.util.Collections;
import java.util.Optional;
import java.util.Set;
-import static com.yahoo.application.container.handler.Request.Method.*;
+import static com.yahoo.application.container.handler.Request.Method.DELETE;
+import static com.yahoo.application.container.handler.Request.Method.GET;
+import static com.yahoo.application.container.handler.Request.Method.POST;
+import static com.yahoo.application.container.handler.Request.Method.PUT;
import static com.yahoo.vespa.hosted.controller.restapi.application.ApplicationApiTest.createApplicationSubmissionData;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@@ -153,7 +155,7 @@ public class ApplicationApiCloudTest extends ControllerContainerCloudTest {
public void validate_secret_store() {
deployApplication();
var secretStoreRequest =
- request("/application/v4/tenant/scoober/secret-store/secret-foo/validate?aws-region=us-west-1&parameter-name=foo&application-id=scoober.albums.default&zone=prod.us-central-1", GET)
+ request("/application/v4/tenant/scoober/secret-store/secret-foo/validate?aws-region=us-west-1&parameter-name=foo&application-id=scoober.albums.default&zone=prod.aws-us-east-1c", GET)
.roles(Set.of(Role.developer(tenantName)));
tester.assertResponse(secretStoreRequest, "{" +
"\"error-code\":\"NOT_FOUND\"," +
@@ -167,9 +169,9 @@ public class ApplicationApiCloudTest extends ControllerContainerCloudTest {
// ConfigServerMock returns message on format deployment.toString() + " - " + tenantSecretStore.toString()
secretStoreRequest =
- request("/application/v4/tenant/scoober/secret-store/secret-foo/validate?aws-region=us-west-1&parameter-name=foo&application-id=scoober.albums.default&zone=prod.us-central-1", GET)
+ request("/application/v4/tenant/scoober/secret-store/secret-foo/validate?aws-region=us-west-1&parameter-name=foo&application-id=scoober.albums.default&zone=prod.aws-us-east-1c", GET)
.roles(Set.of(Role.developer(tenantName)));
- tester.assertResponse(secretStoreRequest, "{\"target\":\"scoober.albums in prod.us-central-1\",\"result\":{\"settings\":{\"name\":\"foo\",\"role\":\"vespa-secretstore-access\",\"awsId\":\"892075328880\",\"externalId\":\"*****\",\"region\":\"us-east-1\"},\"status\":\"ok\"}}", 200);
+ tester.assertResponse(secretStoreRequest, "{\"target\":\"scoober.albums in prod.aws-us-east-1c\",\"result\":{\"settings\":{\"name\":\"foo\",\"role\":\"vespa-secretstore-access\",\"awsId\":\"892075328880\",\"externalId\":\"*****\",\"region\":\"us-east-1\"},\"status\":\"ok\"}}", 200);
}
@Test
@@ -219,12 +221,14 @@ public class ApplicationApiCloudTest extends ControllerContainerCloudTest {
var applicationPackage = new ApplicationPackageBuilder()
.instances("default")
.globalServiceId("foo")
- .region("us-central-1")
+ .region("aws-us-east-1c")
.build();
+ tester.controller().jobController().deploy(ApplicationId.from("scoober", "albums", "default"),
+ JobType.productionAwsUsEast1c,
+ Optional.empty(),
+ applicationPackage);
+
- tester.controller().applications().deploy(ApplicationId.from("scoober", "albums", "default"),
- ZoneId.from("prod", "us-central-1"),
- Optional.of(applicationPackage),
- new DeployOptions(true, Optional.empty(), false, false));
}
+
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
index 673563097f4..c69d2069650 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
@@ -28,7 +28,6 @@ import com.yahoo.vespa.hosted.controller.Instance;
import com.yahoo.vespa.hosted.controller.LockedTenant;
import com.yahoo.vespa.hosted.controller.RoutingController;
import com.yahoo.vespa.hosted.controller.api.application.v4.EnvironmentResource;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.application.v4.model.ProtonMetrics;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
import com.yahoo.vespa.hosted.controller.api.identifiers.Property;
@@ -54,7 +53,6 @@ import com.yahoo.vespa.hosted.controller.application.DeploymentMetrics;
import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId;
import com.yahoo.vespa.hosted.controller.athenz.HostedAthenzIdentities;
import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder;
-import com.yahoo.vespa.hosted.controller.deployment.DeploymentContext;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger;
import com.yahoo.vespa.hosted.controller.integration.ConfigServerMock;
@@ -303,32 +301,6 @@ public class ApplicationApiTest extends ControllerContainerTest {
app1.runJob(JobType.systemTest).runJob(JobType.stagingTest).runJob(JobType.productionUsCentral1);
- // POST an application deployment to a production zone - operator emergency deployment - fails since package is unknown
- entity = createApplicationDeployData(Optional.empty(),
- Optional.of(ApplicationVersion.from(DeploymentContext.defaultSourceRevision, 666)),
- true);
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/prod/region/us-central-1/instance/instance1/", POST)
- .data(entity)
- .userIdentity(HOSTED_VESPA_OPERATOR),
- "{\"error-code\":\"BAD_REQUEST\",\"message\":\"No application package found for tenant1.application1 with version 1.0.666-commit1\"}",
- 400);
-
- // POST an application deployment to a production zone - operator emergency deployment - works with known package
- entity = createApplicationDeployData(Optional.empty(),
- Optional.of(ApplicationVersion.from(DeploymentContext.defaultSourceRevision, 1)),
- true);
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/prod/region/us-central-1/instance/instance1/", POST)
- .data(entity)
- .userIdentity(HOSTED_VESPA_OPERATOR),
- new File("deploy-result.json"));
-
- // POST an application deployment to a production zone - operator emergency deployment - chooses latest package without arguments
- entity = createApplicationDeployData(Optional.empty(), true);
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/prod/region/us-central-1/instance/instance1/", POST)
- .data(entity)
- .userIdentity(HOSTED_VESPA_OPERATOR),
- new File("deploy-result.json"));
-
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
.instances("instance1")
.globalServiceId("foo")
@@ -702,14 +674,14 @@ public class ApplicationApiTest extends ControllerContainerTest {
"{\"message\":\"Deactivated tenant1.application1.instance1 in prod.us-central-1\"}");
// Setup for test config tests
- tester.controller().applications().deploy(ApplicationId.from("tenant1", "application1", "default"),
- ZoneId.from("prod", "us-central-1"),
- Optional.of(applicationPackageDefault),
- new DeployOptions(true, Optional.empty(), false, false));
- tester.controller().applications().deploy(ApplicationId.from("tenant1", "application1", "my-user"),
- ZoneId.from("dev", "us-east-1"),
- Optional.of(applicationPackageDefault),
- new DeployOptions(false, Optional.empty(), false, false));
+ tester.controller().jobController().deploy(ApplicationId.from("tenant1", "application1", "default"),
+ JobType.productionUsCentral1,
+ Optional.empty(),
+ applicationPackageDefault);
+ tester.controller().jobController().deploy(ApplicationId.from("tenant1", "application1", "my-user"),
+ JobType.devUsEast1,
+ Optional.empty(),
+ applicationPackageDefault);
// GET test-config for local tests against a dev deployment
tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/instance/my-user/job/dev-us-east-1/test-config", GET)
@@ -989,13 +961,6 @@ public class ApplicationApiTest extends ControllerContainerTest {
// Add build service to operator role
addUserToHostedOperatorRole(HostedAthenzIdentities.from(SCREWDRIVER_ID));
- // POST (deploy) an application to a prod zone - allowed when project ID is not specified
- MultiPartStreamer entity = createApplicationDeployData(applicationPackageInstance1, true);
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/prod/region/us-central-1/instance/instance1/deploy", POST)
- .data(entity)
- .screwdriverIdentity(SCREWDRIVER_ID),
- new File("deploy-result.json"));
-
// POST (deploy) a system application with an application package
MultiPartStreamer noAppEntity = createApplicationDeployData(Optional.empty(), true);
tester.assertResponse(request("/application/v4/tenant/hosted-vespa/application/routing/environment/prod/region/us-central-1/instance/default/deploy", POST)
@@ -1008,12 +973,6 @@ public class ApplicationApiTest extends ControllerContainerTest {
.data(noAppEntity)
.userIdentity(HOSTED_VESPA_OPERATOR),
new File("deploy-result.json"));
-
- // POST (deploy) a system application without an application package
- tester.assertResponse(request("/application/v4/tenant/hosted-vespa/application/proxy-host/environment/prod/region/us-central-1/instance/instance1/deploy", POST)
- .data(noAppEntity)
- .userIdentity(HOSTED_VESPA_OPERATOR),
- new File("deploy-no-deployment.json"), 400);
}
@Test
@@ -1178,33 +1137,12 @@ public class ApplicationApiTest extends ControllerContainerTest {
"{\"error-code\":\"BAD_REQUEST\",\"message\":\"Invalid build number: For input string: \\\"foobar\\\"\"}",
400);
- // POST (deploy) an application with an invalid application package
+ // POST (deploy) an application to legacy deploy path
MultiPartStreamer entity = createApplicationDeployData(applicationPackageInstance1, true);
tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/dev/region/us-east-1/instance/instance1/deploy", POST)
.data(entity)
.userIdentity(USER_ID),
- new File("deploy-failure.json"), 400);
-
- // POST (deploy) an application without available capacity
- configServer.throwOnNextPrepare(new ConfigServerException(new URI("server-url"), "Failed to prepare application", "Out of capacity", ConfigServerException.ErrorCode.OUT_OF_CAPACITY, null));
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/dev/region/us-east-1/instance/instance1/deploy", POST)
- .data(entity)
- .userIdentity(USER_ID),
- new File("deploy-out-of-capacity.json"), 400);
-
- // POST (deploy) an application where activation fails
- configServer.throwOnNextPrepare(new ConfigServerException(new URI("server-url"), "Failed to activate application", "Activation conflict", ConfigServerException.ErrorCode.ACTIVATION_CONFLICT, null));
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/dev/region/us-east-1/instance/instance1/deploy", POST)
- .data(entity)
- .userIdentity(USER_ID),
- new File("deploy-activation-conflict.json"), 409);
-
- // POST (deploy) an application where we get an internal server error
- configServer.throwOnNextPrepare(new ConfigServerException(new URI("server-url"), "Failed to deploy application", "Internal server error", ConfigServerException.ErrorCode.INTERNAL_SERVER_ERROR, null));
- tester.assertResponse(request("/application/v4/tenant/tenant1/application/application1/environment/dev/region/us-east-1/instance/instance1/deploy", POST)
- .data(entity)
- .userIdentity(USER_ID),
- new File("deploy-internal-server-error.json"), 500);
+ "{\"error-code\":\"BAD_REQUEST\",\"message\":\"Deployment of tenant1.application1.instance1 is not supported through this API\"}", 400);
// DELETE tenant which has an application
tester.assertResponse(request("/application/v4/tenant/tenant1", DELETE)
@@ -1248,7 +1186,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
"{\"error-code\":\"BAD_REQUEST\",\"message\":\"Tenant 'my-tenant' already exists\"}",
400);
}
-
+
@Test
public void testAuthorization() {
UserId authorizedUser = USER_ID;
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 c43abf276c5..f574d6bc3f1 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
@@ -5,7 +5,6 @@ import com.yahoo.component.Version;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.container.jdisc.HttpResponse;
import com.yahoo.test.json.JsonTestHelper;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
import com.yahoo.vespa.hosted.controller.api.integration.configserver.ConfigServerException;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationVersion;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
@@ -171,10 +170,7 @@ public class JobControllerApiHandlerHelperTest {
var region = "us-west-1";
var applicationPackage = new ApplicationPackageBuilder().region(region).build();
// Deploy directly to production zone, like integration tests.
- tester.controller().applications().deploy(tester.instance().id(), ZoneId.from("prod", region),
- Optional.of(applicationPackage),
- new DeployOptions(true, Optional.empty(),
- false, false));
+ tester.controller().jobController().deploy(tester.instance().id(), productionUsWest1, Optional.empty(), applicationPackage);
assertResponse(JobControllerApiHandlerHelper.jobTypeResponse(tester.controller(), app.instanceId(), URI.create("https://some.url:43/root/")),
"jobs-direct-deployment.json");
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/cost-report.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/cost-report.json
deleted file mode 100644
index 8f6dbf17d51..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/cost-report.json
+++ /dev/null
@@ -1,20 +0,0 @@
-{
- "month": "2019-09",
- "items": [
- {
- "applicationId":"tenant1:application1:instance1",
- "zoneId":"prod.us-south-1",
- "cpu": {"usage":7.0,"charge":35},
- "memory": {"usage":600.0,"charge":23},
- "disk": {"usage":1000.0,"charge":10}
- },
- {
- "applicationId":"tenant1:application1:instance1",
- "zoneId":"prod.us-north-1",
- "cpu": {"usage":2.0,"charge":10},
- "memory": {"usage":3.0,"charge":20},
- "disk": {"usage":4.0,"charge":30}
- }
- ]
-
-} \ No newline at end of file
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/create-user-response.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/create-user-response.json
deleted file mode 100644
index a6130122650..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/create-user-response.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
- "message":"Created user 'by-new-user'"
-}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-activation-conflict.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-activation-conflict.json
deleted file mode 100644
index 39d4faa53c9..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-activation-conflict.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
- "error-code":"ACTIVATION_CONFLICT",
- "message":"Failed to activate application: Activation conflict"
-}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-failure.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-failure.json
deleted file mode 100644
index c8802cce57b..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-failure.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
- "error-code":"INVALID_APPLICATION_PACKAGE",
- "message":"Failed to prepare application: Invalid application package"
-}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-internal-server-error.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-internal-server-error.json
deleted file mode 100644
index 9a845e2a7d6..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-internal-server-error.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
- "error-code":"INTERNAL_SERVER_ERROR",
- "message":"Failed to deploy application: Internal server error"
-}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-no-deployment.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-no-deployment.json
deleted file mode 100644
index f90420f28d7..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-no-deployment.json
+++ /dev/null
@@ -1 +0,0 @@
-{"error-code":"BAD_REQUEST","message":"Can't redeploy application, no deployment currently exist"} \ No newline at end of file
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-out-of-capacity.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-out-of-capacity.json
deleted file mode 100644
index 0bdf5a2653c..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deploy-out-of-capacity.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
- "error-code":"OUT_OF_CAPACITY",
- "message":"Failed to prepare application: Out of capacity"
-}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-overview.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-overview.json
index 961d36bd2f3..55be0881ec2 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-overview.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-overview.json
@@ -418,6 +418,7 @@
"targetPlatform": "6.1.0",
"targetApplication": {
"build": 1,
+ "compileVersion": "6.1.0",
"sourceUrl": "repository1/tree/commit1",
"commit": "commit1"
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list-with-user.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list-with-user.json
deleted file mode 100644
index 774b80f8b0c..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list-with-user.json
+++ /dev/null
@@ -1,18 +0,0 @@
-[
- {
- "tenant": "by-myuser",
- "metaData": {
- "type": "USER"
- },
- "url": "http://localhost:8080/application/v4/tenant/by-myuser"
- },
- {
- "tenant": "tenant1",
- "metaData": {
- "type": "ATHENS",
- "athensDomain": "domain1",
- "property": "property1"
- },
- "url": "http://localhost:8080/application/v4/tenant/tenant1"
- }
-]
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list.json
deleted file mode 100644
index d7ec9a738f2..00000000000
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/tenant-list.json
+++ /dev/null
@@ -1,11 +0,0 @@
-[
- {
- "tenant": "tenant1",
- "metaData": {
- "type": "ATHENS",
- "athensDomain": "domain1",
- "property": "property1"
- },
- "url": "http://localhost:8080/application/v4/tenant/tenant1"
- }
-]
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java
index fab61eeaec3..38f5a60cf8a 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java
@@ -19,7 +19,6 @@ import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.vespa.hosted.controller.Instance;
import com.yahoo.vespa.hosted.controller.RoutingController;
-import com.yahoo.vespa.hosted.controller.api.application.v4.model.DeployOptions;
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.deployment.JobType;
@@ -354,7 +353,7 @@ public class RoutingPoliciesTest {
.exclusiveRoutingIn(zoneApi);
// Deploy to dev
- tester.controllerTester().controller().applications().deploy(context.instanceId(), zone, Optional.of(emptyApplicationPackage), DeployOptions.none());
+ context.runJob(zone, emptyApplicationPackage);
assertEquals("DeploymentSpec is not persisted", DeploymentSpec.empty, context.application().deploymentSpec());
context.flushDnsUpdates();
@@ -378,13 +377,14 @@ public class RoutingPoliciesTest {
assertEquals(prodRecords, tester.recordNames());
// Deploy to dev under different instance
- var devInstance = context.application().id().instance("user");
- tester.controllerTester().controller().applications().deploy(devInstance, zone, Optional.of(applicationPackage), DeployOptions.none());
+ var devContext = tester.newDeploymentContext(context.application().id().instance("user"));
+ devContext.runJob(zone, applicationPackage);
+
assertEquals("DeploymentSpec is persisted", applicationPackage.deploymentSpec(), context.application().deploymentSpec());
context.flushDnsUpdates();
// Routing policy is created and DNS is updated
- assertEquals(1, tester.policiesOf(devInstance).size());
+ assertEquals(1, tester.policiesOf(devContext.instanceId()).size());
assertEquals(Sets.union(prodRecords, Set.of("user.app1.tenant1.us-east-1.dev.vespa.oath.cloud")), tester.recordNames());
}
@@ -732,6 +732,10 @@ public class RoutingPoliciesTest {
return tester.newDeploymentContext(tenant, application, instance);
}
+ public DeploymentContext newDeploymentContext(ApplicationId instance) {
+ return tester.newDeploymentContext(instance);
+ }
+
public ControllerTester controllerTester() {
return tester.controllerTester();
}