aboutsummaryrefslogtreecommitdiffstats
path: root/controller-server
diff options
context:
space:
mode:
Diffstat (limited to 'controller-server')
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/ApplicationController.java46
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/InstanceList.java24
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/pkg/ApplicationPackageValidator.java46
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java150
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java40
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java81
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RevisionHistory.java8
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Submission.java2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Versions.java8
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java13
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ResourceMeterMaintainer.java3
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainer.java16
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Upgrader.java6
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/VersionStatusUpdater.java17
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/Notification.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/NotificationFormatter.java47
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java7
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java92
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/MockCuratorDb.java15
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/NotificationsSerializer.java84
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java9
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiHandler.java3
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/horizon/TsdbQueryRewriter.java7
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiHandler.java57
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingId.java44
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicy.java38
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicyId.java44
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingStatus.java29
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/ZoneRoutingPolicy.java35
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/Rotation.java24
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationId.java23
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationStatus.java42
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VersionStatus.java19
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VespaVersion.java7
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java55
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTester.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/ApplicationPackageBuilder.java36
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentContext.java14
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java281
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainerTest.java3
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/UpgraderTest.java24
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializerTest.java3
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerTester.java5
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java3
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/JobControllerApiHandlerHelperTest.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/responses/root.json5
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiTest.java29
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepositoryTest.java5
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/versions/VersionStatusTest.java8
51 files changed, 964 insertions, 609 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 047c059e9a6..92c3198175a 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
@@ -59,7 +59,6 @@ import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackageValid
import com.yahoo.vespa.hosted.controller.athenz.impl.AthenzFacade;
import com.yahoo.vespa.hosted.controller.certificate.EndpointCertificates;
import com.yahoo.vespa.hosted.controller.concurrent.Once;
-import com.yahoo.vespa.hosted.controller.deployment.DeploymentStatus;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger;
import com.yahoo.vespa.hosted.controller.deployment.JobStatus;
import com.yahoo.vespa.hosted.controller.deployment.Run;
@@ -139,6 +138,7 @@ public class ApplicationController {
private final StringFlag dockerImageRepoFlag;
private final ListFlag<String> incompatibleVersions;
private final BillingController billingController;
+ private final ListFlag<String> cloudAccountsFlag;
ApplicationController(Controller controller, CuratorDb curator, AccessControl accessControl, Clock clock,
FlagSource flagSource, BillingController billingController) {
@@ -153,6 +153,7 @@ public class ApplicationController {
applicationStore = controller.serviceRegistry().applicationStore();
dockerImageRepoFlag = PermanentFlags.DOCKER_IMAGE_REPO.bindTo(flagSource);
incompatibleVersions = PermanentFlags.INCOMPATIBLE_VERSIONS.bindTo(flagSource);
+ cloudAccountsFlag = PermanentFlags.CLOUD_ACCOUNTS.bindTo(flagSource);
deploymentTrigger = new DeploymentTrigger(controller, clock);
applicationPackageValidator = new ApplicationPackageValidator(controller);
endpointCertificates = new EndpointCertificates(controller,
@@ -178,6 +179,11 @@ public class ApplicationController {
});
}
+ /** Validate the given application package */
+ public void validatePackage(ApplicationPackage applicationPackage, Application application) {
+ applicationPackageValidator.validate(application, applicationPackage, clock.instant());
+ }
+
/** Returns the application with the given id, or null if it is not present */
public Optional<Application> getApplication(TenantAndApplicationId id) {
return curator.readApplication(id);
@@ -344,10 +350,10 @@ public class ApplicationController {
// Target platforms are all versions not older than the oldest installed platform, unless forcing a major version change.
// Only platforms not older than the system version, and with appropriate confidence, are considered targets.
Predicate<Version> isTargetPlatform = wantedMajor.isEmpty() && oldestInstalledPlatform.isEmpty()
- ? __ -> true
- : wantedMajor.isEmpty() || wantedMajor.getAsInt() == oldestInstalledPlatform.get().getMajor()
- ? version -> ! version.isBefore(oldestInstalledPlatform.get())
- : version -> wantedMajor.getAsInt() == version.getMajor();
+ ? __ -> true // No preferences for version: any platform version is ok.
+ : wantedMajor.isEmpty() || (oldestInstalledPlatform.isPresent() && wantedMajor.getAsInt() == oldestInstalledPlatform.get().getMajor())
+ ? version -> ! version.isBefore(oldestInstalledPlatform.get()) // Major empty, or on same as oldest: ensure not a platform downgrade.
+ : version -> wantedMajor.getAsInt() == version.getMajor(); // Major specified, and not on same as oldest (possibly empty): any on that major.
Set<Version> platformVersions = versionStatus.deployableVersions().stream()
.filter(version -> version.confidence().equalOrHigherThan(targetConfidence))
.map(VespaVersion::versionNumber)
@@ -538,7 +544,7 @@ public class ApplicationController {
/** Stores the deployment spec and validation overrides from the application package, and runs cleanup. */
public void storeWithUpdatedConfig(LockedApplication application, ApplicationPackage applicationPackage) {
- applicationPackageValidator.validate(application.get(), applicationPackage, clock.instant());
+ validatePackage(applicationPackage, application.get());
application = application.with(applicationPackage.deploymentSpec());
application = application.with(applicationPackage.validationOverrides());
@@ -630,9 +636,7 @@ public class ApplicationController {
List<X509Certificate> operatorCertificates = controller.supportAccess().activeGrantsFor(deployment).stream()
.map(SupportAccessGrant::certificate)
.collect(toList());
- Optional<CloudAccount> cloudAccount = applicationPackage.deploymentSpec()
- .instance(application.instance())
- .flatMap(spec -> spec.cloudAccount(zone.environment(), zone.region()));
+ Optional<CloudAccount> cloudAccount = decideCloudAccountOf(deployment, applicationPackage.deploymentSpec());
ConfigServer.PreparedApplication preparedApplication =
configServer.deploy(new DeploymentData(application, zone, applicationPackage.zippedContent(), platform,
endpoints, endpointCertificateMetadata, dockerImageRepo, domain,
@@ -649,6 +653,30 @@ public class ApplicationController {
}
}
+ private Optional<CloudAccount> decideCloudAccountOf(DeploymentId deployment, DeploymentSpec spec) {
+ ZoneId zoneId = deployment.zoneId();
+ Optional<CloudAccount> requestedAccount = spec.instance(deployment.applicationId().instance())
+ .flatMap(instanceSpec -> instanceSpec.cloudAccount(zoneId.environment(),
+ Optional.of(zoneId.region())));
+ if (requestedAccount.isEmpty()) {
+ return Optional.empty();
+ }
+ TenantName tenant = deployment.applicationId().tenant();
+ Set<CloudAccount> tenantAccounts = cloudAccountsFlag.with(FetchVector.Dimension.TENANT_ID, tenant.value())
+ .value().stream()
+ .map(CloudAccount::new)
+ .collect(Collectors.toSet());
+ if (!tenantAccounts.contains(requestedAccount.get())) {
+ throw new IllegalArgumentException("Requested cloud account '" + requestedAccount.get().value() +
+ "' is not valid for tenant '" + tenant + "'");
+ }
+ if (!controller.zoneRegistry().hasZone(zoneId, requestedAccount.get())) {
+ throw new IllegalArgumentException("Zone " + zoneId + " is not configured in requested cloud account '" +
+ requestedAccount.get().value() + "'");
+ }
+ return requestedAccount;
+ }
+
private LockedApplication withoutDeletedDeployments(LockedApplication application, InstanceName instance) {
DeploymentSpec deploymentSpec = application.get().deploymentSpec();
List<ZoneId> deploymentsToRemove = application.get().require(instance).productionDeployments().values().stream()
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/InstanceList.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/InstanceList.java
index 626937dc6ac..2441da19b90 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/InstanceList.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/InstanceList.java
@@ -11,6 +11,9 @@ import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.Instance;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentStatus;
import com.yahoo.vespa.hosted.controller.deployment.DeploymentStatusList;
+import com.yahoo.vespa.hosted.controller.versions.VersionStatus;
+import com.yahoo.vespa.hosted.controller.versions.VespaVersion;
+import com.yahoo.vespa.hosted.controller.versions.VespaVersion.Confidence;
import java.time.Instant;
import java.util.Collection;
@@ -18,6 +21,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.OptionalInt;
import java.util.function.Function;
import static java.util.Comparator.comparing;
@@ -52,17 +56,25 @@ public class InstanceList extends AbstractFilteringList<ApplicationId, InstanceL
}
/**
- * Returns the subset of instances whose application have a deployment on the given major, or specify it in deployment spec.
+ * Returns the subset of instances whose application have a deployment on the given major,
+ * or specify it in deployment spec,
+ * or which are on a {@link VespaVersion.Confidence#legacy} platform, and do not specify that in deployment spec.
*
* @param targetMajorVersion the target major version which applications returned allows upgrading to
*/
- public InstanceList allowingMajorVersion(int targetMajorVersion) {
+ public InstanceList allowingMajorVersion(int targetMajorVersion, VersionStatus versions) {
return matching(id -> {
Application application = application(id);
- return application.deploymentSpec().majorVersion().map(allowed -> targetMajorVersion <= allowed)
- .orElseGet(() -> application.productionDeployments().values().stream()
- .flatMap(List::stream)
- .anyMatch(deployment -> targetMajorVersion <= deployment.version().getMajor()));
+ Optional<Integer> majorVersion = application.deploymentSpec().majorVersion();
+ if (majorVersion.isPresent())
+ return majorVersion.get() >= targetMajorVersion;
+
+ for (List<Deployment> deployments : application.productionDeployments().values())
+ for (Deployment deployment : deployments) {
+ if (deployment.version().getMajor() >= targetMajorVersion) return true;
+ if (versions.version(deployment.version()).confidence() == Confidence.legacy) return true;
+ }
+ return false;
});
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/pkg/ApplicationPackageValidator.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/pkg/ApplicationPackageValidator.java
index 5a131ba4a29..8e8a4e24970 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/pkg/ApplicationPackageValidator.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/pkg/ApplicationPackageValidator.java
@@ -7,17 +7,12 @@ import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.application.api.Endpoint;
import com.yahoo.config.application.api.ValidationId;
import com.yahoo.config.application.api.ValidationOverrides;
-import com.yahoo.config.provision.CloudAccount;
import com.yahoo.config.provision.CloudName;
import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.InstanceName;
import com.yahoo.config.provision.RegionName;
-import com.yahoo.config.provision.TenantName;
import com.yahoo.config.provision.zone.ZoneApi;
import com.yahoo.config.provision.zone.ZoneId;
-import com.yahoo.vespa.flags.FetchVector;
-import com.yahoo.vespa.flags.ListFlag;
-import com.yahoo.vespa.flags.PermanentFlags;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.application.EndpointId;
@@ -31,7 +26,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
-import java.util.Set;
import java.util.stream.Collectors;
/**
@@ -42,11 +36,9 @@ import java.util.stream.Collectors;
public class ApplicationPackageValidator {
private final Controller controller;
- private final ListFlag<String> cloudAccountsFlag;
public ApplicationPackageValidator(Controller controller) {
this.controller = Objects.requireNonNull(controller, "controller must be non-null");
- this.cloudAccountsFlag = PermanentFlags.CLOUD_ACCOUNTS.bindTo(controller.flagSource());
}
/**
@@ -55,7 +47,6 @@ public class ApplicationPackageValidator {
* @throws IllegalArgumentException if any validations fail
*/
public void validate(Application application, ApplicationPackage applicationPackage, Instant instant) {
- validateCloudAccounts(application, applicationPackage.deploymentSpec());
validateSteps(applicationPackage.deploymentSpec());
validateEndpointRegions(applicationPackage.deploymentSpec());
validateEndpointChange(application, applicationPackage, instant);
@@ -90,20 +81,10 @@ public class ApplicationPackageValidator {
for (var spec : deploymentSpec.instances()) {
for (var zone : spec.zones()) {
Environment environment = zone.environment();
- if (environment.isManuallyDeployed())
- throw new IllegalArgumentException("region must be one with automated deployments, but got: " + environment);
-
- if (environment == Environment.prod) {
- RegionName region = zone.region().orElseThrow();
- if (!controller.zoneRegistry().hasZone(ZoneId.from(environment, region))) {
- throw new IllegalArgumentException("Zone " + zone + " in deployment spec was not found in this system!");
- }
- Optional<CloudAccount> cloudAccount = spec.cloudAccount(environment, region);
- if (cloudAccount.isPresent() && !controller.zoneRegistry().hasZone(ZoneId.from(environment, region), cloudAccount.get())) {
- throw new IllegalArgumentException("Zone " + zone + " in deployment spec is not configured for " +
- "use in cloud account '" + cloudAccount.get().value() +
- "', in this system");
- }
+ if (zone.region().isEmpty()) continue;
+ ZoneId zoneId = ZoneId.from(environment, zone.region().get());
+ if (!controller.zoneRegistry().hasZone(zoneId)) {
+ throw new IllegalArgumentException("Zone " + zone + " in deployment spec was not found in this system!");
}
}
}
@@ -185,25 +166,6 @@ public class ApplicationPackageValidator {
". " + ValidationOverrides.toAllowMessage(validationId));
}
- /** Verify that declared cloud accounts are allowed to be used by the tenant */
- private void validateCloudAccounts(Application application, DeploymentSpec deploymentSpec) {
- TenantName tenant = application.id().tenant();
- Set<CloudAccount> validAccounts = cloudAccountsFlag.with(FetchVector.Dimension.TENANT_ID, tenant.value())
- .value().stream()
- .map(CloudAccount::new)
- .collect(Collectors.toSet());
- for (var spec : deploymentSpec.instances()) {
- for (var zone : spec.zones()) {
- if (!zone.environment().isProduction()) continue;
- Optional<CloudAccount> cloudAccount = spec.cloudAccount(zone.environment(), zone.region().get());
- if (cloudAccount.isEmpty()) continue;
- if (validAccounts.contains(cloudAccount.get())) continue;
- throw new IllegalArgumentException("Cloud account '" + cloudAccount.get().value() +
- "' is not valid for tenant '" + tenant + "'");
- }
- }
- }
-
/** Returns whether newEndpoints contains all destinations in endpoints */
private static boolean containsAllDestinationsOf(List<Endpoint> endpoints, List<Endpoint> newEndpoints) {
var containsAllRegions = true;
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java
index 50492077e20..3a6a8e67a75 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java
@@ -8,6 +8,7 @@ import com.yahoo.config.application.api.DeploymentInstanceSpec;
import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.application.api.DeploymentSpec.DeclaredTest;
import com.yahoo.config.application.api.DeploymentSpec.DeclaredZone;
+import com.yahoo.config.application.api.DeploymentSpec.UpgradePolicy;
import com.yahoo.config.application.api.DeploymentSpec.UpgradeRollout;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.CloudName;
@@ -27,12 +28,12 @@ import com.yahoo.vespa.hosted.controller.application.Change;
import com.yahoo.vespa.hosted.controller.application.Deployment;
import com.yahoo.vespa.hosted.controller.versions.VersionStatus;
import com.yahoo.vespa.hosted.controller.versions.VespaVersion;
+import com.yahoo.vespa.hosted.controller.versions.VespaVersion.Confidence;
import java.time.Duration;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -170,7 +171,40 @@ public class DeploymentStatus {
}
/** Returns change potentially with a compatibility platform added, if required for the change to roll out to the given instance. */
- public Change withCompatibilityPlatform(Change change, InstanceName instance) {
+ public Change withPermittedPlatform(Change change, InstanceName instance, boolean allowOudatedPlatform) {
+ Change augmented = withCompatibilityPlatform(change, instance);
+ if (allowOudatedPlatform)
+ return augmented;
+
+ boolean alreadyDeployedOnPlatform = augmented.platform().map(platform -> allJobs.production().asList().stream()
+ .anyMatch(job -> job.runs().values().stream()
+ .anyMatch(run -> run.versions().targetPlatform().getMajor() == platform.getMajor())))
+ .orElse(false);
+
+ // Verify target platform is either current, or was previously deployed for this app.
+ if (augmented.platform().isPresent() && ! versionStatus.isOnCurrentMajor(augmented.platform().get()) && ! alreadyDeployedOnPlatform)
+ throw new IllegalArgumentException("platform version " + augmented.platform().get() + " is not on a current major version in this system");
+
+ Version latestHighConfidencePlatform = null;
+ for (VespaVersion platform : versionStatus.deployableVersions())
+ if (platform.confidence().equalOrHigherThan(Confidence.high))
+ latestHighConfidencePlatform = platform.versionNumber();
+
+ // Verify package is compatible with the current major, or newer, or that there already are deployments on a compatible, outdated platform.
+ if (latestHighConfidencePlatform != null) {
+ Version target = latestHighConfidencePlatform;
+ augmented.revision().flatMap(revision -> application.revisions().get(revision).compileVersion())
+ .filter(target::isAfter)
+ .ifPresent(compiled -> {
+ if (versionCompatibility.apply(instance).refuse(target, compiled) && ! alreadyDeployedOnPlatform)
+ throw new IllegalArgumentException("compile version " + compiled + " is incompatible with the current major version of this system");
+ });
+ }
+
+ return augmented;
+ }
+
+ private Change withCompatibilityPlatform(Change change, InstanceName instance) {
if (change.revision().isEmpty())
return change;
@@ -178,18 +212,18 @@ public class DeploymentStatus {
.map(application.revisions()::get)
.flatMap(ApplicationVersion::compileVersion);
- // If the revision requires a certain platform for compatibility, add that here.
+ // If the revision requires a certain platform for compatibility, add that here, unless we're already deploying a compatible platform.
VersionCompatibility compatibility = versionCompatibility.apply(instance);
Predicate<Version> compatibleWithCompileVersion = version -> compileVersion.map(compiled -> compatibility.accept(version, compiled)).orElse(true);
- if ( application.productionDeployments().isEmpty() // TODO: replace with adding this for test jobs when needed
+ if (change.platform().map(compatibleWithCompileVersion::test).orElse(false))
+ return change;
+
+ if ( application.productionDeployments().isEmpty()
|| application.productionDeployments().getOrDefault(instance, List.of()).stream()
- .anyMatch(deployment -> ! compatibleWithCompileVersion.test(deployment.version()))) {
- return targetsForPolicy(versionStatus, systemVersion, application.deploymentSpec().requireInstance(instance).upgradePolicy())
- .stream() // Pick the latest platform with appropriate confidence, which is compatible with the compile version.
- .filter(compatibleWithCompileVersion)
- .findFirst()
- .map(platform -> change.withoutPin().with(platform))
- .orElse(change);
+ .anyMatch(deployment -> ! compatibleWithCompileVersion.test(deployment.version()))) {
+ for (Version platform : targetsForPolicy(versionStatus, systemVersion, application.deploymentSpec().requireInstance(instance).upgradePolicy()))
+ if (compatibleWithCompileVersion.test(platform))
+ return change.withoutPin().with(platform);
}
return change;
}
@@ -255,39 +289,82 @@ public class DeploymentStatus {
if (change == null || ! change.hasTargets())
return;
- Collection<Optional<JobId>> firstProductionJobsWithDeployment = jobSteps.keySet().stream()
- .filter(jobId -> jobId.type().isProduction() && jobId.type().isDeployment())
- .filter(jobId -> deploymentFor(jobId).isPresent())
- .collect(groupingBy(jobId -> findCloud(jobId.type()),
- Collectors.reducing((o, n) -> o))) // Take the first.
- .values();
- if (firstProductionJobsWithDeployment.isEmpty())
- firstProductionJobsWithDeployment = List.of(Optional.empty());
-
- for (Optional<JobId> firstProductionJobWithDeploymentInCloud : firstProductionJobsWithDeployment) {
+ Map<CloudName, Optional<JobId>> firstProductionJobsWithDeployment = firstDependentProductionJobsWithDeployment(job.application().instance());
+ firstProductionJobsWithDeployment.forEach((cloud, firstProductionJobWithDeploymentInCloud) -> {
Versions versions = Versions.from(change,
application,
firstProductionJobWithDeploymentInCloud.flatMap(this::deploymentFor),
fallbackPlatform(change, job));
if (step.completedAt(change, firstProductionJobWithDeploymentInCloud).isEmpty()) {
- CloudName cloud = firstProductionJobWithDeploymentInCloud.map(JobId::type).map(this::findCloud).orElse(zones.systemZone().getCloudName());
JobType typeWithZone = job.type().isSystemTest() ? JobType.systemTest(zones, cloud) : JobType.stagingTest(zones, cloud);
jobs.merge(job, List.of(new Job(typeWithZone, versions, step.readyAt(change), change)), DeploymentStatus::union);
}
- }
+ });
});
return Collections.unmodifiableMap(jobs);
}
+ /**
+ * Returns the clouds, and their first production deployments, that depend on this instance; or,
+ * if no such deployments exist, all clouds the application deploy to, and their first production deployments; or
+ * if no clouds are deployed to at all, the system default cloud.
+ */
+ Map<CloudName, Optional<JobId>> firstDependentProductionJobsWithDeployment(InstanceName testInstance) {
+ // Find instances' dependencies on each other: these are topologically ordered, so a simple traversal does it.
+ Map<InstanceName, Set<InstanceName>> dependencies = new HashMap<>();
+ instanceSteps().forEach((name, step) -> {
+ dependencies.put(name, new HashSet<>());
+ dependencies.get(name).add(name);
+ for (StepStatus dependency : step.dependencies()) {
+ dependencies.get(name).add(dependency.instance());
+ dependencies.get(name).addAll(dependencies.get(dependency.instance));
+ }
+ });
+
+ Map<CloudName, Optional<JobId>> independentJobsPerCloud = new HashMap<>();
+ Map<CloudName, Optional<JobId>> jobsPerCloud = new HashMap<>();
+ jobSteps.forEach((job, step) -> {
+ if ( ! job.type().isProduction() || ! job.type().isDeployment())
+ return;
+
+ (dependencies.get(step.instance()).contains(testInstance) ? jobsPerCloud
+ : independentJobsPerCloud)
+ .merge(findCloud(job.type()),
+ Optional.of(job),
+ (o, n) -> o.filter(v -> deploymentFor(v).isPresent()) // Keep first if its deployment is present.
+ .or(() -> n.filter(v -> deploymentFor(v).isPresent())) // Use next if only its deployment is present.
+ .or(() -> o)); // Keep first if none have deployments.
+ });
+
+ if (jobsPerCloud.isEmpty())
+ jobsPerCloud.putAll(independentJobsPerCloud);
+
+ if (jobsPerCloud.isEmpty())
+ jobsPerCloud.put(zones.systemZone().getCloudName(), Optional.empty());
+
+ return jobsPerCloud;
+ }
+
+
/** Fall back to the newest, deployable platform, which is compatible with what we want to deploy. */
public Version fallbackPlatform(Change change, JobId job) {
+ InstanceName instance = job.application().instance();
Optional<Version> compileVersion = change.revision().map(application.revisions()::get).flatMap(ApplicationVersion::compileVersion);
- if (compileVersion.isEmpty())
- return systemVersion;
-
- for (VespaVersion version : reversed(versionStatus.deployableVersions()))
- if (versionCompatibility.apply(job.application().instance()).accept(version.versionNumber(), compileVersion.get()))
- return version.versionNumber();
+ List<Version> targets = targetsForPolicy(versionStatus,
+ systemVersion,
+ application.deploymentSpec().instance(instance)
+ .map(DeploymentInstanceSpec::upgradePolicy)
+ .orElse(UpgradePolicy.defaultPolicy));
+
+ // Prefer fallback with proper confidence.
+ for (Version target : targets)
+ if (compileVersion.isEmpty() || versionCompatibility.apply(instance).accept(target, compileVersion.get()))
+ return target;
+
+ // Try fallback with any confidence.
+ for (VespaVersion target : reversed(versionStatus.deployableVersions()))
+ if (compileVersion.isEmpty() || versionCompatibility.apply(instance).accept(target.versionNumber(), compileVersion.get()))
+ return target.versionNumber();
throw new IllegalArgumentException("no legal platform version exists in this system for compile version " + compileVersion.get());
}
@@ -579,6 +656,7 @@ public class DeploymentStatus {
/** The test jobs that need to run prior to the given production deployment jobs. */
public Map<JobId, List<Job>> testJobs(Map<JobId, List<Job>> jobs) {
Map<JobId, List<Job>> testJobs = new LinkedHashMap<>();
+ // First, look for a declared test in the instance of each production job.
jobs.forEach((job, versionsList) -> {
for (JobType testType : List.of(systemTest(job.type()), stagingTest(job.type()))) {
if (job.type().isProduction() && job.type().isDeployment()) {
@@ -596,6 +674,7 @@ public class DeploymentStatus {
}
}
});
+ // If no declared test in the right instance was triggered, pick one from a different instance.
jobs.forEach((job, versionsList) -> {
for (JobType testType : List.of(systemTest(job.type()), stagingTest(job.type()))) {
for (Job productionJob : versionsList)
@@ -603,7 +682,8 @@ public class DeploymentStatus {
&& allJobs.successOn(testType, productionJob.versions()).asList().isEmpty()
&& testJobs.keySet().stream()
.noneMatch(test -> test.type().equals(testType) && test.type().zone().equals(testType.zone())
- && testJobs.get(test).stream().anyMatch(testJob -> testJob.versions().equals(productionJob.versions())))) {
+ && testJobs.get(test).stream().anyMatch(testJob -> test.type().isSystemTest() ? testJob.versions().targetsMatch(productionJob.versions())
+ : testJob.versions().equals(productionJob.versions())))) {
JobId testJob = firstDeclaredOrElseImplicitTest(testType);
testJobs.merge(testJob,
List.of(new Job(testJob.type(),
@@ -1014,10 +1094,18 @@ public class DeploymentStatus {
@Override
Optional<Instant> completedAt(Change change, Optional<JobId> dependent) {
Optional<Instant> deployedAt = status.jobSteps().get(prodId).completedAt(change, Optional.of(prodId));
+ Versions target = Versions.from(change, status.application(), status.deploymentFor(job.id()), status.fallbackPlatform(change, job.id()));
+ Change applied = Change.empty();
+ if (change.platform().isPresent())
+ applied = applied.with(target.targetPlatform());
+ if (change.revision().isPresent())
+ applied = applied.with(target.targetRevision());
+ Change relevant = applied;
+
return (dependent.equals(job()) ? job.lastTriggered().filter(run -> deployedAt.map(at -> ! run.start().isBefore(at)).orElse(false)).stream()
: job.runs().values().stream())
.filter(Run::hasSucceeded)
- .filter(run -> run.versions().targetsMatch(change))
+ .filter(run -> run.versions().targetsMatch(relevant))
.flatMap(run -> run.end().stream()).findFirst();
}
};
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java
index 7c0a4a83e4f..9e107a52b55 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java
@@ -1,8 +1,6 @@
// Copyright Yahoo. 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.component.VersionCompatibility;
import com.yahoo.config.application.api.DeploymentInstanceSpec;
import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.provision.ApplicationId;
@@ -14,7 +12,6 @@ import com.yahoo.vespa.hosted.controller.ApplicationController;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.Instance;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
-import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationVersion;
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.RevisionId;
@@ -22,8 +19,6 @@ import com.yahoo.vespa.hosted.controller.application.ApplicationList;
import com.yahoo.vespa.hosted.controller.application.Change;
import com.yahoo.vespa.hosted.controller.application.Deployment;
import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId;
-import com.yahoo.vespa.hosted.controller.versions.VersionStatus;
-import com.yahoo.vespa.hosted.controller.versions.VespaVersion;
import java.math.BigDecimal;
import java.time.Clock;
@@ -43,7 +38,6 @@ import java.util.logging.Logger;
import java.util.stream.Collectors;
import static java.util.Comparator.comparing;
-import static java.util.Comparator.reverseOrder;
import static java.util.stream.Collectors.groupingBy;
import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toMap;
@@ -91,11 +85,10 @@ public class DeploymentTrigger {
&& acceptNewRevision(status, instanceName, outstanding.revision().get());
application = application.with(instanceName,
instance -> withRemainingChange(instance,
- status.withCompatibilityPlatform((deployOutstanding ? outstanding
- : Change.empty())
- .onTopOf(instance.change()),
- instanceName),
- status));
+ deployOutstanding ? outstanding.onTopOf(instance.change())
+ : instance.change(),
+ status,
+ false));
}
applications().store(application);
});
@@ -114,7 +107,10 @@ public class DeploymentTrigger {
applications().lockApplicationOrThrow(TenantAndApplicationId.from(id), application -> {
if (application.get().deploymentSpec().instance(id.instance()).isPresent())
applications().store(application.with(id.instance(),
- instance -> withRemainingChange(instance, instance.change(), jobs.deploymentStatus(application.get()))));
+ instance -> withRemainingChange(instance,
+ instance.change(),
+ jobs.deploymentStatus(application.get()),
+ true)));
});
}
@@ -289,9 +285,17 @@ public class DeploymentTrigger {
/** Overrides the given instance's platform and application changes with any contained in the given change. */
public void forceChange(ApplicationId instanceId, Change change) {
+ forceChange(instanceId, change, true);
+ }
+
+ /** Overrides the given instance's platform and application changes with any contained in the given change. */
+ public void forceChange(ApplicationId instanceId, Change change, boolean allowOutdatedPlatform) {
applications().lockApplicationOrThrow(TenantAndApplicationId.from(instanceId), application -> {
applications().store(application.with(instanceId.instance(),
- instance -> withRemainingChange(instance, change.onTopOf(application.get().require(instanceId.instance()).change()), jobs.deploymentStatus(application.get()))));
+ instance -> withRemainingChange(instance,
+ change.onTopOf(instance.change()),
+ jobs.deploymentStatus(application.get()),
+ allowOutdatedPlatform)));
});
}
@@ -308,7 +312,10 @@ public class DeploymentTrigger {
default: throw new IllegalArgumentException("Unknown cancellation choice '" + cancellation + "'!");
}
applications().store(application.with(instanceId.instance(),
- instance -> withRemainingChange(instance, change, jobs.deploymentStatus(application.get()))));
+ instance -> withRemainingChange(instance,
+ change,
+ jobs.deploymentStatus(application.get()),
+ true)));
});
}
@@ -424,13 +431,14 @@ public class DeploymentTrigger {
}
}
- private Instance withRemainingChange(Instance instance, Change change, DeploymentStatus status) {
+ private Instance withRemainingChange(Instance instance, Change change, DeploymentStatus status, boolean allowOutdatedPlatform) {
Change remaining = change;
if (status.hasCompleted(instance.name(), change.withoutApplication()))
remaining = remaining.withoutPlatform();
if (status.hasCompleted(instance.name(), change.withoutPlatform()))
remaining = remaining.withoutApplication();
- return instance.withChange(remaining);
+
+ return instance.withChange(status.withPermittedPlatform(remaining, instance.name(), allowOutdatedPlatform));
}
// ---------- Version and job helpers ----------
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 ef3474e0c1e..965f1b09819 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
@@ -258,6 +258,10 @@ public class InternalStepRunner implements StepRunner {
throw e;
}
+ catch (IllegalArgumentException e) {
+ logger.log(WARNING, e.getMessage());
+ return Optional.of(deploymentFailed);
+ }
catch (EndpointCertificateException e) {
switch (e.type()) {
case CERT_NOT_AVAILABLE:
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 11a784ce899..6a4975c3458 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
@@ -5,7 +5,9 @@ import com.google.common.collect.ImmutableSortedMap;
import com.yahoo.component.Version;
import com.yahoo.component.VersionCompatibility;
import com.yahoo.concurrent.UncheckedTimeoutException;
+import com.yahoo.config.application.api.DeploymentSpec.UpgradePolicy;
import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.transaction.Mutex;
import com.yahoo.vespa.hosted.controller.Application;
@@ -29,7 +31,6 @@ import com.yahoo.vespa.hosted.controller.application.TenantAndApplicationId;
import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackageDiff;
import com.yahoo.vespa.hosted.controller.application.pkg.TestPackage;
-import com.yahoo.vespa.hosted.controller.application.pkg.TestPackage.TestSummary;
import com.yahoo.vespa.hosted.controller.notification.Notification;
import com.yahoo.vespa.hosted.controller.notification.Notification.Type;
import com.yahoo.vespa.hosted.controller.notification.NotificationSource;
@@ -37,6 +38,7 @@ import com.yahoo.vespa.hosted.controller.persistence.BufferedLogStore;
import com.yahoo.vespa.hosted.controller.persistence.CuratorDb;
import com.yahoo.vespa.hosted.controller.versions.VersionStatus;
import com.yahoo.vespa.hosted.controller.versions.VespaVersion;
+import com.yahoo.vespa.hosted.controller.versions.VespaVersion.Confidence;
import java.io.IOException;
import java.io.InputStream;
@@ -570,25 +572,7 @@ public class JobController {
application = application.withRevisions(revisions -> revisions.with(version.get()));
application = withPrunedPackages(application, version.get().id());
- TestSummary testSummary = TestPackage.validateTests(submission.applicationPackage().deploymentSpec(), submission.testPackage());
- if (testSummary.problems().isEmpty())
- controller.notificationsDb().removeNotification(NotificationSource.from(id), Type.testPackage);
- else
- controller.notificationsDb().setNotification(NotificationSource.from(id),
- Type.testPackage,
- Notification.Level.warning,
- testSummary.problems());
-
- submission.applicationPackage().parentVersion().ifPresent(parent -> {
- if (parent.getMajor() < controller.readSystemVersion().getMajor())
- controller.notificationsDb().setNotification(NotificationSource.from(id),
- Type.submission,
- Notification.Level.warning,
- "Parent version used to compile the application is on a " +
- "lower major version than the current Vespa Cloud version");
- else
- controller.notificationsDb().removeNotification(NotificationSource.from(id), Type.submission);
- });
+ validate(id, submission);
applications.storeWithUpdatedConfig(application, submission.applicationPackage());
if (application.get().projectId().isPresent())
@@ -597,6 +581,33 @@ public class JobController {
return version.get();
}
+ private void validate(TenantAndApplicationId id, Submission submission) {
+ controller.notificationsDb().removeNotification(NotificationSource.from(id), Type.testPackage);
+ controller.notificationsDb().removeNotification(NotificationSource.from(id), Type.submission);
+
+ validateTests(id, submission);
+ validateMajorVersion(id, submission);
+ }
+
+ private void validateTests(TenantAndApplicationId id, Submission submission) {
+ var testSummary = TestPackage.validateTests(submission.applicationPackage().deploymentSpec(), submission.testPackage());
+ if ( ! testSummary.problems().isEmpty())
+ controller.notificationsDb().setNotification(NotificationSource.from(id),
+ Type.testPackage,
+ Notification.Level.warning,
+ testSummary.problems());
+
+ }
+
+ private void validateMajorVersion(TenantAndApplicationId id, Submission submission) {
+ submission.applicationPackage().deploymentSpec().majorVersion().ifPresent(explicitMajor -> {
+ if ( ! controller.readVersionStatus().isOnCurrentMajor(new Version(explicitMajor)))
+ controller.notificationsDb().setNotification(NotificationSource.from(id), Type.submission, Notification.Level.warning,
+ "Vespa " + explicitMajor + " will soon be end of life, upgrade to Vespa " + (explicitMajor + 1) + " now: " +
+ "https://cloud.vespa.ai/en/vespa" + (explicitMajor + 1) + "-release-notes.html"); // ∠( ᐛ 」∠)_
+ });
+ }
+
private LockedApplication withPrunedPackages(LockedApplication application, RevisionId latest){
TenantAndApplicationId id = application.get().id();
Application wrapped = application.get();
@@ -665,18 +676,22 @@ public class JobController {
/** Stores the given package and starts a deployment of it, after aborting any such ongoing deployment. */
public void deploy(ApplicationId id, JobType type, Optional<Version> platform, ApplicationPackage applicationPackage) {
- deploy(id, type, platform, applicationPackage, false);
+ deploy(id, type, platform, applicationPackage, false, false);
}
/** Stores the given package and starts a deployment of it, after aborting any such ongoing deployment.*/
- public void deploy(ApplicationId id, JobType type, Optional<Version> platform, ApplicationPackage applicationPackage, boolean dryRun) {
+ public void deploy(ApplicationId id, JobType type, Optional<Version> platform, ApplicationPackage applicationPackage,
+ boolean dryRun, boolean allowOutdatedPlatform) {
if ( ! controller.zoneRegistry().hasZone(type.zone()))
throw new IllegalArgumentException(type.zone() + " is not present in this system");
controller.applications().lockApplicationOrThrow(TenantAndApplicationId.from(id), application -> {
if ( ! application.get().instances().containsKey(id.instance()))
application = controller.applications().withNewInstance(application, id);
-
+ // TODO(mpolden): Enable for public CD once all tests have been updated
+ if (controller.system() != SystemName.PublicCd) {
+ controller.applications().validatePackage(applicationPackage, application.get());
+ }
controller.applications().store(application);
});
@@ -686,13 +701,18 @@ public class JobController {
long build = 1 + lastRun.map(run -> run.versions().targetRevision().number()).orElse(0L);
RevisionId revisionId = RevisionId.forDevelopment(build, new JobId(id, type));
- ApplicationVersion version = ApplicationVersion.forDevelopment(revisionId, applicationPackage.compileVersion());
+ ApplicationVersion version = ApplicationVersion.forDevelopment(revisionId, applicationPackage.compileVersion(), applicationPackage.deploymentSpec().majorVersion());
byte[] diff = getDiff(applicationPackage, deploymentId, lastRun);
controller.applications().lockApplicationOrThrow(TenantAndApplicationId.from(id), application -> {
- controller.applications().applicationStore().putDev(deploymentId, version.id(), applicationPackage.zippedContent(), diff);
Version targetPlatform = platform.orElseGet(() -> findTargetPlatform(applicationPackage, deploymentId, application.get().get(id.instance())));
+ if ( ! allowOutdatedPlatform
+ && ! controller.readVersionStatus().isOnCurrentMajor(targetPlatform)
+ && runs(id, type).values().stream().noneMatch(run -> run.versions().targetPlatform().getMajor() == targetPlatform.getMajor()))
+ throw new IllegalArgumentException("platform version " + targetPlatform + " is not on a current major version in this system");
+
+ controller.applications().applicationStore().putDev(deploymentId, version.id(), applicationPackage.zippedContent(), diff);
controller.applications().store(application.withRevisions(revisions -> revisions.with(version)));
start(id,
type,
@@ -724,9 +744,14 @@ public class JobController {
private Version findTargetPlatform(ApplicationPackage applicationPackage, DeploymentId id, Optional<Instance> instance) {
// Prefer previous platform if possible. Candidates are all deployable, ascending, with existing version appended; then reversed.
- List<Version> versions = controller.readVersionStatus().deployableVersions().stream()
- .map(VespaVersion::versionNumber)
- .collect(toList());
+ VersionStatus versionStatus = controller.readVersionStatus();
+ Version systemVersion = controller.systemVersion(versionStatus);
+
+ List<Version> versions = new ArrayList<>(List.of(systemVersion));
+ for (VespaVersion version : versionStatus.deployableVersions())
+ if (version.confidence().equalOrHigherThan(Confidence.normal))
+ versions.add(version.versionNumber());
+
instance.map(Instance::deployments)
.map(deployments -> deployments.get(id.zoneId()))
.map(Deployment::version)
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RevisionHistory.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RevisionHistory.java
index 5bdc980f11a..ab8f7fe22d3 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RevisionHistory.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RevisionHistory.java
@@ -49,12 +49,6 @@ public class RevisionHistory {
for (ApplicationVersion revision : productionRevisions)
production.put(revision.id(), revision);
- // TODO jonmv: remove once it's run once on serialised data
- String hash = "";
- for (ApplicationVersion revision : List.copyOf(production.values()))
- if (hash.equals(hash = revision.bundleHash().orElse("")) && ! hash.isEmpty())
- production.put(revision.id(), revision.skipped());
-
NavigableMap<JobId, NavigableMap<RevisionId, ApplicationVersion>> development = new TreeMap<>(comparator);
developmentRevisions.forEach((job, jobRevisions) -> {
NavigableMap<RevisionId, ApplicationVersion> revisions = development.computeIfAbsent(job, __ -> new TreeMap<>());
@@ -100,7 +94,7 @@ public class RevisionHistory {
// Fallback for when an application version isn't known for the given key.
private static ApplicationVersion revisionOf(RevisionId id) {
- return new ApplicationVersion(id, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), false, false, Optional.empty(), 0);
+ return new ApplicationVersion(id, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), false, false, Optional.empty(), 0);
}
/** Returns the production {@link ApplicationVersion} with this revision ID. */
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Submission.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Submission.java
index e366920690b..6c9de2fd584 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Submission.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Submission.java
@@ -6,6 +6,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.deployment.SourceRevisi
import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackage;
import java.util.Optional;
+import java.util.OptionalInt;
import static com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackage.calculateHash;
@@ -42,6 +43,7 @@ public class Submission {
source,
authorEmail,
applicationPackage.compileVersion(),
+ applicationPackage.deploymentSpec().majorVersion(),
applicationPackage.buildTime(),
sourceUrl,
source.map(SourceRevision::commit),
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Versions.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Versions.java
index d683f1cb5c7..a172671fc8e 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Versions.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Versions.java
@@ -119,12 +119,8 @@ public class Versions {
}
/** Create versions using given change and application */
- public static Versions from(Change change, Application application, Optional<Deployment> deployment,
- Version defaultPlatformVersion) {
- return new Versions(targetPlatform(application, change, deployment.map(Deployment::version), defaultPlatformVersion),
- targetRevision(application, change, deployment.map(Deployment::revision)),
- deployment.map(Deployment::version),
- deployment.map(Deployment::revision));
+ public static Versions from(Change change, Application application, Optional<Deployment> deployment, Version defaultPlatformVersion) {
+ return from(change, application, deployment.map(Deployment::version), deployment.map(Deployment::revision), defaultPlatformVersion);
}
private static Version targetPlatform(Application application, Change change, Optional<Version> existing,
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java
index 93ef6d29450..fd177c469d2 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java
@@ -5,6 +5,7 @@ import com.yahoo.component.Version;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.Instance;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationVersion;
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.application.Deployment;
@@ -37,7 +38,7 @@ public class DeploymentUpgrader extends ControllerMaintainer {
Version targetPlatform = null; // Upgrade to the newest non-broken, deployable version.
for (VespaVersion platform : controller().readVersionStatus().deployableVersions())
- if (platform.confidence().equalOrHigherThan(VespaVersion.Confidence.low))
+ if (platform.confidence().equalOrHigherThan(VespaVersion.Confidence.normal))
targetPlatform = platform.versionNumber();
if (targetPlatform == null)
@@ -53,9 +54,13 @@ public class DeploymentUpgrader extends ControllerMaintainer {
Run last = controller().jobController().last(job).get();
Versions target = new Versions(targetPlatform, last.versions().targetRevision(), Optional.of(last.versions().targetPlatform()), Optional.of(last.versions().targetRevision()));
if ( ! last.hasEnded()) continue;
- if (application.revisions().get(last.versions().targetRevision()).compileVersion()
- .map(version -> controller().applications().versionCompatibility(instance.id()).refuse(version, target.targetPlatform()))
- .orElse(false)) continue;
+ ApplicationVersion devVersion = application.revisions().get(last.versions().targetRevision());
+ if (devVersion.compileVersion()
+ .map(version -> controller().applications().versionCompatibility(instance.id()).refuse(version, target.targetPlatform()))
+ .orElse(false)) continue;
+ if ( devVersion.allowedMajor().isPresent()
+ && devVersion.allowedMajor().get() < targetPlatform.getMajor()) continue;
+
if ( ! deployment.version().isBefore(target.targetPlatform())) continue;
if ( ! isLikelyNightFor(job)) continue;
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ResourceMeterMaintainer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ResourceMeterMaintainer.java
index 85588d2cf0f..d1f6ddc1644 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ResourceMeterMaintainer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ResourceMeterMaintainer.java
@@ -120,7 +120,8 @@ public class ResourceMeterMaintainer extends ControllerMaintainer {
Map<ZoneId, Double> deploymentCosts = snapshotsByInstance.getOrDefault(instanceName, List.of()).stream()
.collect(Collectors.toUnmodifiableMap(
ResourceSnapshot::getZoneId,
- snapshot -> cost(snapshot.allocation(), systemName)));
+ snapshot -> cost(snapshot.allocation(), systemName),
+ Double::sum));
locked = locked.with(instanceName, i -> i.withDeploymentCosts(deploymentCosts));
updateCostMetrics(tenantAndApplication.instance(instanceName), deploymentCosts);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainer.java
index 4c72f6747d5..d45818c2822 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainer.java
@@ -14,6 +14,12 @@ import java.util.logging.Level;
import java.util.logging.Logger;
import java.util.stream.Collectors;
+/**
+ * Trigger any jobs that are marked for re-triggering to effectuate some other change, e.g. a change in access to a
+ * deployment's nodes.
+ *
+ * @author tokle
+ */
public class RetriggerMaintainer extends ControllerMaintainer {
private static final Logger logger = Logger.getLogger(RetriggerMaintainer.class.getName());
@@ -32,7 +38,7 @@ public class RetriggerMaintainer extends ControllerMaintainer {
.filter(this::needsTrigger)
.filter(entry -> readyToTrigger(entry.jobId()))
.forEach(entry -> controller().applications().deploymentTrigger().reTrigger(entry.jobId().application(), entry.jobId().type(),
- "re-triggered by RetriggerMaintainer"));
+ "re-triggered by " + getClass().getSimpleName()));
// Remove all jobs that has succeeded with the required job run and persist the list
List<RetriggerEntry> remaining = retriggerEntries.stream()
@@ -46,9 +52,7 @@ public class RetriggerMaintainer extends ControllerMaintainer {
return 1.0;
}
- /*
- Returns true if a job is ready to run, i.e is currently not running
- */
+ /** Returns true if a job is ready to run, i.e. is currently not running */
private boolean readyToTrigger(JobId jobId) {
Optional<Run> existingRun = controller().jobController().active(jobId.application()).stream()
.filter(run -> run.id().type().equals(jobId.type()))
@@ -56,9 +60,7 @@ public class RetriggerMaintainer extends ControllerMaintainer {
return existingRun.isEmpty();
}
- /*
- Returns true of job needs triggering. I.e the job has not run since the queue item was last run.
- */
+ /** Returns true of job needs triggering. I.e. the job has not run since the queue item was last run */
private boolean needsTrigger(RetriggerEntry entry) {
return controller().jobController().lastCompleted(entry.jobId())
.filter(run -> run.id().number() < entry.requiredRun())
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Upgrader.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Upgrader.java
index f3c82e72ae3..037dacfcac9 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Upgrader.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/Upgrader.java
@@ -107,7 +107,7 @@ public class Upgrader extends ControllerMaintainer {
Map<ApplicationId, Version> targets = new LinkedHashMap<>();
for (Version version : DeploymentStatus.targetsForPolicy(versionStatus, controller().systemVersion(versionStatus), policy)) {
targetAndNewer.add(version);
- InstanceList eligible = eligibleForVersion(remaining, version);
+ InstanceList eligible = eligibleForVersion(remaining, version, versionStatus);
InstanceList outdated = cancellationCriterion.apply(eligible);
cancelUpgradesOf(outdated.upgrading(), "Upgrading to outdated versions");
@@ -134,10 +134,10 @@ public class Upgrader extends ControllerMaintainer {
}
}
- private InstanceList eligibleForVersion(InstanceList instances, Version version) {
+ private InstanceList eligibleForVersion(InstanceList instances, Version version, VersionStatus versionStatus) {
Change change = Change.of(version);
return instances.not().failingOn(version)
- .allowingMajorVersion(version.getMajor())
+ .allowingMajorVersion(version.getMajor(), versionStatus)
.compatibleWithPlatform(version, controller().applications()::versionCompatibility)
.not().hasCompleted(change) // Avoid rescheduling change for instances without production steps.
.onLowerVersionThan(version)
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/VersionStatusUpdater.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/VersionStatusUpdater.java
index 71b8f1cd9b7..154455c5198 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/VersionStatusUpdater.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/VersionStatusUpdater.java
@@ -13,6 +13,7 @@ import java.util.logging.Level;
import static com.yahoo.vespa.hosted.controller.api.integration.organization.SystemMonitor.Confidence.aborted;
import static com.yahoo.vespa.hosted.controller.api.integration.organization.SystemMonitor.Confidence.broken;
import static com.yahoo.vespa.hosted.controller.api.integration.organization.SystemMonitor.Confidence.high;
+import static com.yahoo.vespa.hosted.controller.api.integration.organization.SystemMonitor.Confidence.legacy;
import static com.yahoo.vespa.hosted.controller.api.integration.organization.SystemMonitor.Confidence.low;
import static com.yahoo.vespa.hosted.controller.api.integration.organization.SystemMonitor.Confidence.normal;
@@ -47,14 +48,14 @@ public class VersionStatusUpdater extends ControllerMaintainer {
}
static SystemMonitor.Confidence convert(VespaVersion.Confidence confidence) {
- switch (confidence) {
- case aborted: return aborted;
- case broken: return broken;
- case low: return low;
- case normal: return normal;
- case high: return high;
- default: throw new IllegalArgumentException("Unexpected confidence '" + confidence + "'");
- }
+ return switch (confidence) {
+ case aborted -> aborted;
+ case broken -> broken;
+ case low -> low;
+ case legacy -> legacy;
+ case normal -> normal;
+ case high -> high;
+ };
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/Notification.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/Notification.java
index b6e18881dab..da423995b7e 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/Notification.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/Notification.java
@@ -66,10 +66,11 @@ public class Notification {
}
public enum Type {
+
/** Related to contents of application package, e.g., usage of deprecated features/syntax */
applicationPackage,
- /** Related to contents of application package, e.g., old parent or compile version, or errors detectable on submission */
+ /** Related to contents of application package detectable by the controller on submission */
submission,
/** Related to contents of application test package, e.g., mismatch between deployment spec and provided tests */
@@ -83,6 +84,7 @@ public class Notification {
/** Application cluster is reindexing document(s) */
reindex
+
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/NotificationFormatter.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/NotificationFormatter.java
index d2b12ab6edc..f753f22608d 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/NotificationFormatter.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/notification/NotificationFormatter.java
@@ -28,21 +28,14 @@ public class NotificationFormatter {
}
public FormattedNotification format(Notification n) {
- switch (n.type()) {
- case applicationPackage:
- case submission:
- return applicationPackage(n);
- case deployment:
- return deployment(n);
- case testPackage:
- return testPackage(n);
- case reindex:
- return reindex(n);
- case feedBlock:
- return feedBlock(n);
- default:
- return new FormattedNotification(n, n.type().name(), "", zoneRegistry.dashboardUrl(n.source().tenant()));
- }
+ return switch (n.type()) {
+ case applicationPackage, submission -> applicationPackage(n);
+ case deployment -> deployment(n);
+ case testPackage -> testPackage(n);
+ case reindex -> reindex(n);
+ case feedBlock -> feedBlock(n);
+ default -> new FormattedNotification(n, n.type().name(), "", zoneRegistry.dashboardUrl(n.source().tenant()));
+ };
}
private FormattedNotification applicationPackage(Notification n) {
@@ -132,13 +125,10 @@ public class NotificationFormatter {
var applicationId = ApplicationId.from(source.tenant(), application, instance);
Function<Environment, URI> link = (Environment env) -> zoneRegistry.dashboardUrl(new RunId(applicationId, jobType, runNumber));
var environment = jobType.zone().environment();
- switch (environment) {
- case dev:
- case perf:
- return link.apply(environment);
- default:
- return link.apply(Environment.prod);
- }
+ return switch (environment) {
+ case dev, perf -> link.apply(environment);
+ default -> link.apply(Environment.prod);
+ };
}
private String jobText(NotificationSource source) {
@@ -162,14 +152,11 @@ public class NotificationFormatter {
}
private String levelText(Notification.Level level, int count) {
- switch (level) {
- case error:
- return "failed";
- case warning:
- return count > 1 ? Text.format("%d warnings", count) : "a warning";
- default:
- return count > 1 ? Text.format("%d messages", count) : "a message";
- }
+ return switch (level) {
+ case error -> "failed";
+ case warning -> count > 1 ? Text.format("%d warnings", count) : "a warning";
+ default -> count > 1 ? Text.format("%d messages", count) : "a message";
+ };
}
private String clusterInfo(NotificationSource source) {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java
index 48d8627f407..5aa847f648a 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializer.java
@@ -120,6 +120,7 @@ public class ApplicationSerializer {
private static final String hasPackageField = "hasPackage";
private static final String shouldSkipField = "shouldSkip";
private static final String compileVersionField = "compileVersion";
+ private static final String allowedMajorField = "allowedMajor";
private static final String buildTimeField = "buildTime";
private static final String sourceUrlField = "sourceUrl";
private static final String bundleHashField = "bundleHash";
@@ -257,6 +258,7 @@ public class ApplicationSerializer {
applicationVersion.source().ifPresent(source -> toSlime(source, object.setObject(sourceRevisionField)));
applicationVersion.authorEmail().ifPresent(email -> object.setString(authorEmailField, email));
applicationVersion.compileVersion().ifPresent(version -> object.setString(compileVersionField, version.toString()));
+ applicationVersion.allowedMajor().ifPresent(major -> object.setLong(allowedMajorField, major));
applicationVersion.buildTime().ifPresent(time -> object.setLong(buildTimeField, time.toEpochMilli()));
applicationVersion.sourceUrl().ifPresent(url -> object.setString(sourceUrlField, url));
applicationVersion.commit().ifPresent(commit -> object.setString(commitField, commit));
@@ -478,6 +480,7 @@ public class ApplicationSerializer {
Optional<SourceRevision> sourceRevision = sourceRevisionFromSlime(object.field(sourceRevisionField));
Optional<String> authorEmail = SlimeUtils.optionalString(object.field(authorEmailField));
Optional<Version> compileVersion = SlimeUtils.optionalString(object.field(compileVersionField)).map(Version::fromString);
+ Optional<Integer> allowedMajor = SlimeUtils.optionalInteger(object.field(allowedMajorField)).stream().boxed().findFirst();
Optional<Instant> buildTime = SlimeUtils.optionalInstant(object.field(buildTimeField));
Optional<String> sourceUrl = SlimeUtils.optionalString(object.field(sourceUrlField));
Optional<String> commit = SlimeUtils.optionalString(object.field(commitField));
@@ -487,8 +490,8 @@ public class ApplicationSerializer {
int risk = (int) object.field(riskField).asLong();
Optional<String> bundleHash = SlimeUtils.optionalString(object.field(bundleHashField));
- return new ApplicationVersion(id, sourceRevision, authorEmail, compileVersion, buildTime, sourceUrl,
- commit, bundleHash, hasPackage, shouldSkip, description, risk);
+ return new ApplicationVersion(id, sourceRevision, authorEmail, compileVersion, allowedMajor, buildTime,
+ sourceUrl, commit, bundleHash, hasPackage, shouldSkip, description, risk);
}
private Optional<SourceRevision> sourceRevisionFromSlime(Inspector object) {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java
index 38cea6f8cef..ff374c9bc8d 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/CuratorDb.java
@@ -7,7 +7,6 @@ import com.yahoo.component.annotation.Inject;
import com.yahoo.concurrent.UncheckedTimeoutException;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.HostName;
-import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.TenantName;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.path.Path;
@@ -15,10 +14,13 @@ import com.yahoo.slime.Slime;
import com.yahoo.slime.SlimeUtils;
import com.yahoo.transaction.Mutex;
import com.yahoo.vespa.curator.Curator;
+import com.yahoo.vespa.curator.MultiplePathsLock;
+import com.yahoo.vespa.flags.FlagSource;
+import com.yahoo.vespa.flags.Flags;
+import com.yahoo.vespa.flags.StringFlag;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.api.identifiers.ControllerVersion;
import com.yahoo.vespa.hosted.controller.api.identifiers.DeploymentId;
-import com.yahoo.vespa.hosted.controller.api.integration.ServiceRegistry;
import com.yahoo.vespa.hosted.controller.api.integration.archive.ArchiveBucket;
import com.yahoo.vespa.hosted.controller.api.integration.certificates.EndpointCertificateMetadata;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
@@ -51,10 +53,10 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
+import java.util.Locale;
import java.util.Map;
import java.util.NavigableMap;
import java.util.Optional;
-import java.util.OptionalInt;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeoutException;
@@ -66,7 +68,6 @@ import java.util.stream.Collectors;
import java.util.stream.LongStream;
import static java.util.stream.Collectors.collectingAndThen;
-import static java.util.stream.Collectors.toUnmodifiableList;
/**
* Curator backed database for storing the persistence state of controllers. This maps controller specific operations
@@ -84,7 +85,9 @@ public class CuratorDb {
private static final Duration defaultTryLockTimeout = Duration.ofSeconds(1);
private static final Path root = Path.fromString("/controller/v1");
+
private static final Path lockRoot = root.append("locks");
+
private static final Path tenantRoot = root.append("tenants");
private static final Path applicationRoot = root.append("applications");
private static final Path jobRoot = root.append("jobs");
@@ -116,6 +119,7 @@ public class CuratorDb {
private final Curator curator;
private final Duration tryLockTimeout;
+ private final StringFlag lockScheme;
// For each application id (path), store the ZK node version and its deserialised data - update when version changes.
// This will grow to keep all applications in memory, but this should be OK
@@ -125,13 +129,14 @@ public class CuratorDb {
private final Map<Path, Pair<Integer, NavigableMap<RunId, Run>>> cachedHistoricRuns = new ConcurrentHashMap<>();
@Inject
- public CuratorDb(Curator curator, ServiceRegistry services) {
- this(curator, defaultTryLockTimeout, services.zoneRegistry().system());
+ public CuratorDb(Curator curator, FlagSource flagSource) {
+ this(curator, defaultTryLockTimeout, flagSource);
}
- CuratorDb(Curator curator, Duration tryLockTimeout, SystemName system) {
+ CuratorDb(Curator curator, Duration tryLockTimeout, FlagSource flagSource) {
this.curator = curator;
this.tryLockTimeout = tryLockTimeout;
+ this.lockScheme = Flags.CONTROLLER_LOCK_SCHEME.bindTo(flagSource);
}
/** Returns all hostnames configured to be part of this ZooKeeper cluster */
@@ -139,29 +144,52 @@ public class CuratorDb {
return Arrays.stream(curator.zooKeeperEnsembleConnectionSpec().split(","))
.filter(hostAndPort -> !hostAndPort.isEmpty())
.map(hostAndPort -> hostAndPort.split(":")[0])
- .collect(Collectors.toUnmodifiableList());
+ .toList();
}
// -------------- Locks ---------------------------------------------------
+ private enum LockScheme {
+
+ OLD, BOTH, NEW;
+
+ boolean isLegacy() { return this == OLD; }
+
+ }
+
+ /** Acquire the appropriate lock according to the lock scheme set by feature flag */
+ private Mutex lock(Function<LockScheme, Path> pathFactory, Duration timeout) {
+ LockScheme scheme = LockScheme.valueOf(lockScheme.value().toUpperCase(Locale.ENGLISH));
+ return switch (scheme) {
+ case OLD, NEW -> curator.lock(pathFactory.apply(scheme), timeout);
+ case BOTH -> new MultiplePathsLock(curator.lock(pathFactory.apply(LockScheme.OLD), timeout),
+ curator.lock(pathFactory.apply(LockScheme.NEW), timeout));
+ };
+ }
+
public Mutex lock(TenantName name) {
- return curator.lock(lockPath(name), defaultLockTimeout.multipliedBy(2));
+ return lock(scheme -> lockPath(name, scheme.isLegacy()), defaultLockTimeout.multipliedBy(2));
}
public Mutex lock(TenantAndApplicationId id) {
- return curator.lock(lockPath(id), defaultLockTimeout.multipliedBy(2));
+ return lock(scheme -> lockPath(id, scheme.isLegacy()), defaultLockTimeout.multipliedBy(2));
}
public Mutex lockForDeployment(ApplicationId id, ZoneId zone) {
- return curator.lock(lockPath(id, zone), deployLockTimeout);
+ return lock(scheme -> lockPath(id, zone, scheme.isLegacy()), deployLockTimeout);
}
public Mutex lock(ApplicationId id, JobType type) {
- return curator.lock(lockPath(id, type), defaultLockTimeout);
+ return lock(scheme -> lockPath(id, type, scheme.isLegacy()), defaultLockTimeout);
}
public Mutex lock(ApplicationId id, JobType type, Step step) throws TimeoutException {
- return tryLock(lockPath(id, type, step));
+ try {
+ // TODO(mpolden): Revert to tryLock once lock scheme is removed
+ return lock(scheme -> lockPath(id, type, step, scheme.isLegacy()), tryLockTimeout);
+ } catch (UncheckedTimeoutException e) {
+ throw new TimeoutException(e.getMessage());
+ }
}
public Mutex lockRotations() {
@@ -387,7 +415,7 @@ public class CuratorDb {
return curator.getChildren(applicationRoot).stream()
.map(TenantAndApplicationId::fromSerialized)
.sorted()
- .collect(toUnmodifiableList());
+ .toList();
}
public void removeApplication(TenantAndApplicationId id) {
@@ -620,8 +648,8 @@ public class CuratorDb {
public List<TenantName> listTenantsWithNotifications() {
return curator.getChildren(notificationsRoot).stream()
- .map(TenantName::from)
- .collect(Collectors.toUnmodifiableList());
+ .map(TenantName::from)
+ .toList();
}
public void writeNotifications(TenantName tenantName, List<Notification> notifications) {
@@ -638,7 +666,6 @@ public class CuratorDb {
return readSlime(supportAccessPath(deploymentId)).map(SupportAccessSerializer::fromSlime).orElse(SupportAccess.DISALLOWED_NO_HISTORY);
}
- /** Take lock before reading before writing */
public void writeSupportAccess(DeploymentId deploymentId, SupportAccess supportAccess) {
curator.set(supportAccessPath(deploymentId), asJson(SupportAccessSerializer.toSlime(supportAccess)));
}
@@ -655,31 +682,34 @@ public class CuratorDb {
// -------------- Paths ---------------------------------------------------
- private Path lockPath(TenantName tenant) {
- return lockRoot
- .append(tenant.value());
+ private Path lockPath(TenantName tenant, boolean legacy) {
+ Path currentRoot = legacy ? lockRoot : lockRoot.append("tenants");
+ return currentRoot.append(tenant.value());
}
- private Path lockPath(TenantAndApplicationId application) {
- return lockRoot.append(application.tenant().value() + ":" + application.application().value());
+ private Path lockPath(TenantAndApplicationId application, boolean legacy) {
+ Path currentRoot = legacy ? lockRoot : lockRoot.append("applications");
+ return currentRoot.append(application.tenant().value() + ":" + application.application().value());
}
- private Path lockPath(ApplicationId instance, ZoneId zone) {
- return lockRoot.append(instance.serializedForm() + ":" + zone.environment().value() + ":" + zone.region().value());
+ private Path lockPath(ApplicationId instance, ZoneId zone, boolean legacy) {
+ Path currentRoot = legacy ? lockRoot : lockRoot.append("instances");
+ return currentRoot.append(instance.serializedForm() + ":" + zone.environment().value() + ":" + zone.region().value());
}
- private Path lockPath(ApplicationId instance, JobType type) {
- return lockRoot.append(instance.serializedForm() + ":" + type.jobName());
+ private Path lockPath(ApplicationId instance, JobType type, boolean legacy) {
+ Path currentRoot = legacy ? lockRoot : lockRoot.append("jobs");
+ return currentRoot.append(instance.serializedForm() + ":" + type.jobName());
}
- private Path lockPath(ApplicationId instance, JobType type, Step step) {
- return lockRoot.append(instance.serializedForm() + ":" + type.jobName() + ":" + step.name());
+ private Path lockPath(ApplicationId instance, JobType type, Step step, boolean legacy) {
+ Path currentRoot = legacy ? lockRoot : lockRoot.append("steps");
+ return currentRoot.append(instance.serializedForm() + ":" + type.jobName() + ":" + step.name());
}
private Path lockPath(String provisionId) {
- return lockRoot
- .append(provisionStatePath())
- .append(provisionId);
+ return lockRoot.append(provisionStatePath())
+ .append(provisionId);
}
private static Path upgradesPerMinutePath() {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/MockCuratorDb.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/MockCuratorDb.java
index 8f36daf9756..85feb97f39b 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/MockCuratorDb.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/MockCuratorDb.java
@@ -5,6 +5,8 @@ import com.yahoo.component.annotation.Inject;
import com.yahoo.cloud.config.ConfigserverConfig;
import com.yahoo.config.provision.SystemName;
import com.yahoo.vespa.curator.mock.MockCurator;
+import com.yahoo.vespa.flags.InMemoryFlagSource;
+
import java.time.Duration;
/**
@@ -19,20 +21,19 @@ public class MockCuratorDb extends CuratorDb {
@Inject
public MockCuratorDb(ConfigserverConfig config) {
- this("test-controller:2222", SystemName.from(config.system()));
+ this("test-controller:2222");
}
public MockCuratorDb(SystemName system) {
- this("test-controller:2222", system);
+ this("test-controller:2222");
}
- public MockCuratorDb(String zooKeeperEnsembleConnectionSpec, SystemName system) {
- this(new MockCurator() { @Override public String zooKeeperEnsembleConnectionSpec() { return zooKeeperEnsembleConnectionSpec; } },
- system);
+ public MockCuratorDb(String zooKeeperEnsembleConnectionSpec) {
+ this(new MockCurator() { @Override public String zooKeeperEnsembleConnectionSpec() { return zooKeeperEnsembleConnectionSpec; } });
}
- public MockCuratorDb(MockCurator curator, SystemName system) {
- super(curator, Duration.ofMillis(100), system);
+ public MockCuratorDb(MockCurator curator) {
+ super(curator, Duration.ofMillis(100), new InMemoryFlagSource());
this.curator = curator;
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/NotificationsSerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/NotificationsSerializer.java
index 99687a9893c..beda8942fc2 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/NotificationsSerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/NotificationsSerializer.java
@@ -69,18 +69,16 @@ public class NotificationsSerializer {
public List<Notification> fromSlime(TenantName tenantName, Slime slime) {
return SlimeUtils.entriesStream(slime.get().field(notificationsFieldName))
- .filter(inspector -> { // TODO: remove in summer.
- if ( ! inspector.field(jobTypeField).valid()) return true;
- try {
- JobType.ofSerialized(inspector.field(jobTypeField).asString());
- return true;
- }
- catch (RuntimeException e) {
- return false;
- }
- })
- .map(inspector -> fromInspector(tenantName, inspector))
- .collect(Collectors.toUnmodifiableList());
+ .filter(inspector -> { // TODO: remove in summer.
+ if (!inspector.field(jobTypeField).valid()) return true;
+ try {
+ JobType.ofSerialized(inspector.field(jobTypeField).asString());
+ return true;
+ } catch (RuntimeException e) {
+ return false;
+ }
+ })
+ .map(inspector -> fromInspector(tenantName, inspector)).toList();
}
private Notification fromInspector(TenantName tenantName, Inspector inspector) {
@@ -96,49 +94,49 @@ public class NotificationsSerializer {
SlimeUtils.optionalString(inspector.field(clusterIdField)).map(ClusterSpec.Id::from),
SlimeUtils.optionalString(inspector.field(jobTypeField)).map(jobName -> JobType.ofSerialized(jobName)),
SlimeUtils.optionalLong(inspector.field(runNumberField))),
- SlimeUtils.entriesStream(inspector.field(messagesField)).map(Inspector::asString).collect(Collectors.toUnmodifiableList()));
+ SlimeUtils.entriesStream(inspector.field(messagesField)).map(Inspector::asString).toList());
}
private static String asString(Notification.Type type) {
- switch (type) {
- case applicationPackage: return "applicationPackage";
- case submission: return "submission";
- case testPackage: return "testPackage";
- case deployment: return "deployment";
- case feedBlock: return "feedBlock";
- case reindex: return "reindex";
- default: throw new IllegalArgumentException("No serialization defined for notification type " + type);
- }
+ return switch (type) {
+ case applicationPackage -> "applicationPackage";
+ case submission -> "submission";
+ case testPackage -> "testPackage";
+ case deployment -> "deployment";
+ case feedBlock -> "feedBlock";
+ case reindex -> "reindex";
+ default -> throw new IllegalArgumentException("No serialization defined for notification type " + type);
+ };
}
private static Notification.Type typeFrom(Inspector field) {
- switch (field.asString()) {
- case "applicationPackage": return Notification.Type.applicationPackage;
- case "submission": return Notification.Type.submission;
- case "testPackage": return Notification.Type.testPackage;
- case "deployment": return Notification.Type.deployment;
- case "feedBlock": return Notification.Type.feedBlock;
- case "reindex": return Notification.Type.reindex;
- default: throw new IllegalArgumentException("Unknown serialized notification type value '" + field.asString() + "'");
- }
+ return switch (field.asString()) {
+ case "applicationPackage" -> Notification.Type.applicationPackage;
+ case "submission" -> Notification.Type.submission;
+ case "testPackage" -> Notification.Type.testPackage;
+ case "deployment" -> Notification.Type.deployment;
+ case "feedBlock" -> Notification.Type.feedBlock;
+ case "reindex" -> Notification.Type.reindex;
+ default -> throw new IllegalArgumentException("Unknown serialized notification type value '" + field.asString() + "'");
+ };
}
private static String asString(Notification.Level level) {
- switch (level) {
- case info: return "info";
- case warning: return "warning";
- case error: return "error";
- default: throw new IllegalArgumentException("No serialization defined for notification level " + level);
- }
+ return switch (level) {
+ case info -> "info";
+ case warning -> "warning";
+ case error -> "error";
+ default -> throw new IllegalArgumentException("No serialization defined for notification level " + level);
+ };
}
private static Notification.Level levelFrom(Inspector field) {
- switch (field.asString()) {
- case "info": return Notification.Level.info;
- case "warning": return Notification.Level.warning;
- case "error": return Notification.Level.error;
- default: throw new IllegalArgumentException("Unknown serialized notification level value '" + field.asString() + "'");
- }
+ return switch (field.asString()) {
+ case "info" -> Notification.Level.info;
+ case "warning" -> Notification.Level.warning;
+ case "error" -> Notification.Level.error;
+ default -> throw new IllegalArgumentException("Unknown serialized notification level value '" + field.asString() + "'");
+ };
}
}
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 6a50f4c52b4..341cba60519 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
@@ -2073,7 +2073,7 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
if (pin)
change = change.withPin();
- controller.applications().deploymentTrigger().forceChange(id, change);
+ controller.applications().deploymentTrigger().forceChange(id, change, isOperator(request));
response.append("Triggered ").append(change).append(" for ").append(id);
});
return new MessageResponse(response.toString());
@@ -2090,7 +2090,7 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
RevisionId revision = build == -1 ? application.get().revisions().last().get().id()
: getRevision(application.get(), build);
Change change = Change.of(revision);
- controller.applications().deploymentTrigger().forceChange(id, change);
+ controller.applications().deploymentTrigger().forceChange(id, change, isOperator(request));
response.append("Triggered ").append(change).append(" for ").append(id);
});
return new MessageResponse(response.toString());
@@ -2112,6 +2112,9 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
RevisionId revision = RevisionId.forProduction(Long.parseLong(build));
controller.applications().lockApplicationOrThrow(id, application -> {
controller.applications().store(application.withRevisions(revisions -> revisions.with(revisions.get(revision).skipped())));
+ for (Instance instance : application.get().instances().values())
+ if (instance.change().revision().equals(Optional.of(revision)))
+ controller.applications().deploymentTrigger().cancelChange(instance.id(), ChangesToCancel.APPLICATION);
});
return new MessageResponse("Marked build '" + build + "' as non-deployable");
}
@@ -2278,7 +2281,7 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
.map(Boolean::valueOf)
.orElse(false);
- controller.jobController().deploy(id, type, version, applicationPackage, dryRun);
+ controller.jobController().deploy(id, type, version, applicationPackage, dryRun, isOperator(request));
RunId runId = controller.jobController().last(id, type).get().id();
Slime slime = new Slime();
Cursor rootObject = slime.setObject();
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiHandler.java
index 1bf2f78f866..76925241c1e 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/deployment/DeploymentApiHandler.java
@@ -14,6 +14,7 @@ import com.yahoo.restapi.UriBuilder;
import com.yahoo.slime.Cursor;
import com.yahoo.slime.Slime;
import com.yahoo.vespa.hosted.controller.Controller;
+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.application.ApplicationList;
import com.yahoo.vespa.hosted.controller.application.Change;
@@ -174,6 +175,8 @@ public class DeploymentApiHandler extends ThreadedHttpRequestHandler {
instanceObject.setString("upgradePolicy", toString(status.application().deploymentSpec().instance(instance.instance())
.map(DeploymentInstanceSpec::upgradePolicy)
.orElse(DeploymentSpec.UpgradePolicy.defaultPolicy)));
+ status.application().revisions().last().flatMap(ApplicationVersion::compileVersion)
+ .ifPresent(compiled -> instanceObject.setString("compileVersion", compiled.toFullString()));
Cursor jobsArray = instanceObject.setArray("jobs");
status.jobSteps().forEach((job, jobStatus) -> {
if ( ! job.application().equals(instance)) return;
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/horizon/TsdbQueryRewriter.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/horizon/TsdbQueryRewriter.java
index 2211e83c51a..5953c51782a 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/horizon/TsdbQueryRewriter.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/horizon/TsdbQueryRewriter.java
@@ -40,7 +40,12 @@ public class TsdbQueryRewriter {
JsonNode execution = executionGraph.get(i);
// Will be handled by rewriteFilters()
- if (execution.has("filterId") && filterExists(root, execution.get("filterId").asText())) continue;
+ if (execution.has("filterId")) {
+ if (filterExists(root, execution.get("filterId").asText()))
+ continue;
+ else
+ throw new IllegalArgumentException("Invalid filterId: " + execution.get("filterId").asText());
+ }
rewriteFilter((ObjectNode) execution, tenantNames, operator, systemName);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiHandler.java
index 9cced2b8159..893befb57a2 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiHandler.java
@@ -101,6 +101,7 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
private HttpResponse handleGET(Path path, HttpRequest request) {
if (path.matches("/user/v1/user")) return userMetadata(request);
+ if (path.matches("/user/v1/find")) return findUser(request);
if (path.matches("/user/v1/tenant/{tenant}")) return listTenantRoleMembers(path.get("tenant"));
if (path.matches("/user/v1/tenant/{tenant}/application/{application}")) return listApplicationRoleMembers(path.get("tenant"), path.get("application"));
@@ -133,6 +134,45 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
RoleDefinition.hostedSupporter,
RoleDefinition.hostedAccountant);
+ private HttpResponse findUser(HttpRequest request) {
+ var email = request.getProperty("email");
+ var query = request.getProperty("query");
+ if (email != null) return userMetadataFromUserId(email);
+ if (query != null) return userMetadataQuery(query);
+ return ErrorResponse.badRequest("Need 'email' or 'query' parameter");
+ }
+
+ private HttpResponse userMetadataFromUserId(String email) {
+ var maybeUser = users.findUser(email);
+
+ var slime = new Slime();
+ var root = slime.setObject();
+ var usersRoot = root.setArray("users");
+
+ if (maybeUser.isPresent()) {
+ var user = maybeUser.get();
+ var roles = users.listRoles(new UserId(user.email()));
+ renderUserMetaData(usersRoot.addObject(), user, Set.copyOf(roles));
+ }
+
+ return new SlimeJsonResponse(slime);
+ }
+
+ private HttpResponse userMetadataQuery(String query) {
+ var userList = users.findUsers(query);
+
+ var slime = new Slime();
+ var root = slime.setObject();
+ var userSlime = root.setArray("users");
+
+ for (var user : userList) {
+ var roles = users.listRoles(new UserId((user.email())));
+ renderUserMetaData(userSlime.addObject(), user, Set.copyOf(roles));
+ }
+
+ return new SlimeJsonResponse(slime);
+ }
+
private HttpResponse userMetadata(HttpRequest request) {
User user;
if (request.getJDiscRequest().context().get(User.ATTRIBUTE_NAME) instanceof User) {
@@ -146,6 +186,12 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
Set<Role> roles = getAttribute(request, SecurityContext.ATTRIBUTE_NAME, SecurityContext.class).roles();
+ var slime = new Slime();
+ renderUserMetaData(slime.setObject(), user, roles);
+ return new SlimeJsonResponse(slime);
+ }
+
+ private void renderUserMetaData(Cursor root, User user, Set<Role> roles) {
Map<TenantName, List<TenantRole>> tenantRolesByTenantName = roles.stream()
.flatMap(role -> filterTenantRoles(role).stream())
.distinct()
@@ -156,10 +202,7 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
List<Role> operatorRoles = roles.stream()
.filter(role -> hostedOperators.contains(role.definition()))
.sorted(Comparator.comparing(Role::definition))
- .collect(Collectors.toList());
-
- Slime slime = new Slime();
- Cursor root = slime.setObject();
+ .toList();
root.setBool("isPublic", controller.system().isPublic());
root.setBool("isCd", controller.system().isCd());
@@ -185,8 +228,6 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
}
UserFlagsSerializer.toSlime(root, flagsDb.getAllFlagData(), tenantRolesByTenantName.keySet(), !operatorRoles.isEmpty(), user.email());
-
- return new SlimeJsonResponse(slime);
}
private HttpResponse listTenantRoleMembers(String tenantName) {
@@ -256,7 +297,7 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
var user = new UserId(require("user", Inspector::asString, requestObject));
var roles = SlimeStream.fromArray(requestObject.field("roles"), Inspector::asString)
.map(roleName -> Roles.toRole(tenant, roleName))
- .collect(Collectors.toUnmodifiableList());
+ .toList();
users.addToRoles(user, roles);
return new MessageResponse(user + " is now a member of " + roles.stream().map(Role::toString).collect(Collectors.joining(", ")));
@@ -268,7 +309,7 @@ public class UserApiHandler extends ThreadedHttpRequestHandler {
var user = new UserId(require("user", Inspector::asString, requestObject));
var roles = SlimeStream.fromArray(requestObject.field("roles"), Inspector::asString)
.map(roleName -> Roles.toRole(tenant, roleName))
- .collect(Collectors.toUnmodifiableList());
+ .toList();
enforceLastAdminOfTenant(tenant, user, roles);
removeDeveloperKey(tenant, user, roles);
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingId.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingId.java
index 67eafe6235d..21c8b5aeb87 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingId.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingId.java
@@ -12,42 +12,14 @@ import java.util.Objects;
*
* @author mpolden
*/
-public class RoutingId {
+public record RoutingId(ApplicationId instance,
+ EndpointId endpointId,
+ TenantAndApplicationId application) {
- private final TenantAndApplicationId application;
- private final ApplicationId instance;
- private final EndpointId endpointId;
-
- private RoutingId(ApplicationId instance, EndpointId endpointId) {
- this.instance = Objects.requireNonNull(instance, "application must be non-null");
- this.endpointId = Objects.requireNonNull(endpointId, "endpointId must be non-null");
-
- application = TenantAndApplicationId.from(instance);
- }
-
- public TenantAndApplicationId application() {
- return application;
- }
-
- public ApplicationId instance() {
- return instance;
- }
-
- public EndpointId endpointId() {
- return endpointId;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- RoutingId routingId = (RoutingId) o;
- return application.equals(routingId.application) && instance.equals(routingId.instance) && endpointId.equals(routingId.endpointId);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(application, instance, endpointId);
+ public RoutingId {
+ Objects.requireNonNull(instance, "application must be non-null");
+ Objects.requireNonNull(endpointId, "endpointId must be non-null");
+ Objects.requireNonNull(application, "application must be non-null");
}
@Override
@@ -56,7 +28,7 @@ public class RoutingId {
}
public static RoutingId of(ApplicationId instance, EndpointId endpoint) {
- return new RoutingId(instance, endpoint);
+ return new RoutingId(instance, endpoint, TenantAndApplicationId.from(instance));
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicy.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicy.java
index fa6741ec8b8..585cda65e66 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicy.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicy.java
@@ -23,14 +23,12 @@ import java.util.Set;
* @author mortent
* @author mpolden
*/
-public class RoutingPolicy {
-
- private final RoutingPolicyId id;
- private final DomainName canonicalName;
- private final Optional<String> dnsZone;
- private final Set<EndpointId> instanceEndpoints;
- private final Set<EndpointId> applicationEndpoints;
- private final Status status;
+public record RoutingPolicy(RoutingPolicyId id,
+ DomainName canonicalName,
+ Optional<String> dnsZone,
+ Set<EndpointId> instanceEndpoints,
+ Set<EndpointId> applicationEndpoints,
+ Status status) {
/** DO NOT USE. Public for serialization purposes */
public RoutingPolicy(RoutingPolicyId id, DomainName canonicalName, Optional<String> dnsZone,
@@ -123,15 +121,11 @@ public class RoutingPolicy {
}
/** The status of a routing policy */
- public static class Status {
-
- private final boolean active;
- private final RoutingStatus routingStatus;
+ public record Status(boolean active, RoutingStatus routingStatus) {
/** DO NOT USE. Public for serialization purposes */
- public Status(boolean active, RoutingStatus routingStatus) {
- this.active = active;
- this.routingStatus = Objects.requireNonNull(routingStatus, "globalRouting must be non-null");
+ public Status {
+ Objects.requireNonNull(routingStatus, "routingStatus must be non-null");
}
/** Returns whether this is considered active according to the load balancer status */
@@ -149,20 +143,6 @@ public class RoutingPolicy {
return new Status(active, routingStatus);
}
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- Status status = (Status) o;
- return active == status.active &&
- routingStatus.equals(status.routingStatus);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(active, routingStatus);
- }
-
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicyId.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicyId.java
index d64241b1239..1c0b41155fd 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicyId.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicyId.java
@@ -13,26 +13,12 @@ import java.util.Objects;
*
* @author mpolden
*/
-public class RoutingPolicyId {
+public record RoutingPolicyId(ApplicationId owner, ClusterSpec.Id cluster, ZoneId zone) {
- private final ApplicationId owner;
- private final ClusterSpec.Id cluster;
- private final ZoneId zone;
-
- public RoutingPolicyId(ApplicationId owner, ClusterSpec.Id cluster, ZoneId zone) {
- this.owner = Objects.requireNonNull(owner, "owner must be non-null");
- this.cluster = Objects.requireNonNull(cluster, "cluster must be non-null");
- this.zone = Objects.requireNonNull(zone, "zone must be non-null");
- }
-
- /** The application owning this */
- public ApplicationId owner() {
- return owner;
- }
-
- /** The zone this applies to */
- public ZoneId zone() {
- return zone;
+ public RoutingPolicyId {
+ Objects.requireNonNull(owner, "owner must be non-null");
+ Objects.requireNonNull(cluster, "cluster must be non-null");
+ Objects.requireNonNull(zone, "zone must be non-null");
}
/** The deployment this applies to */
@@ -40,26 +26,6 @@ public class RoutingPolicyId {
return new DeploymentId(owner, zone);
}
- /** The cluster this applies to */
- public ClusterSpec.Id cluster() {
- return cluster;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- RoutingPolicyId that = (RoutingPolicyId) o;
- return owner.equals(that.owner) &&
- cluster.equals(that.cluster) &&
- zone.equals(that.zone);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(owner, cluster, zone);
- }
-
@Override
public String toString() {
return "routing policy for " + cluster + ", in " + zone + ", owned by " + owner;
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingStatus.java
index 58f0005d488..de16089e735 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingStatus.java
@@ -13,19 +13,15 @@ import java.util.Objects;
*
* @author mpolden
*/
-public class RoutingStatus {
+public record RoutingStatus(Value value, Agent agent, Instant changedAt) {
public static final RoutingStatus DEFAULT = new RoutingStatus(Value.in, Agent.system, Instant.EPOCH);
- private final Value value;
- private final Agent agent;
- private final Instant changedAt;
-
/** DO NOT USE. Public for serialization purposes */
- public RoutingStatus(Value value, Agent agent, Instant changedAt) {
- this.value = Objects.requireNonNull(value, "value must be non-null");
- this.agent = Objects.requireNonNull(agent, "agent must be non-null");
- this.changedAt = Objects.requireNonNull(changedAt, "changedAt must be non-null");
+ public RoutingStatus {
+ Objects.requireNonNull(value, "value must be non-null");
+ Objects.requireNonNull(agent, "agent must be non-null");
+ Objects.requireNonNull(changedAt, "changedAt must be non-null");
}
/**
@@ -47,21 +43,6 @@ public class RoutingStatus {
}
@Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- RoutingStatus that = (RoutingStatus) o;
- return value == that.value &&
- agent == that.agent &&
- changedAt.equals(that.changedAt);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(value, agent, changedAt);
- }
-
- @Override
public String toString() {
return "status " + value + ", changed by " + agent + " @ " + changedAt;
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/ZoneRoutingPolicy.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/ZoneRoutingPolicy.java
index 60605df1002..a404be76507 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/ZoneRoutingPolicy.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/ZoneRoutingPolicy.java
@@ -13,38 +13,11 @@ import java.util.Objects;
*
* @author mpolden
*/
-public class ZoneRoutingPolicy {
+public record ZoneRoutingPolicy(ZoneId zone, RoutingStatus routingStatus) {
- private final ZoneId zone;
- private final RoutingStatus routingStatus;
-
- public ZoneRoutingPolicy(ZoneId zone, RoutingStatus routingStatus) {
- this.zone = Objects.requireNonNull(zone, "zone must be non-null");
- this.routingStatus = Objects.requireNonNull(routingStatus, "globalRouting must be non-null");
- }
-
- /** The zone this applies to */
- public ZoneId zone() {
- return zone;
- }
-
- /** Routing status of this policy */
- public RoutingStatus routingStatus() {
- return routingStatus;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- ZoneRoutingPolicy that = (ZoneRoutingPolicy) o;
- return zone.equals(that.zone) &&
- routingStatus.equals(that.routingStatus);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(zone, routingStatus);
+ public ZoneRoutingPolicy {
+ Objects.requireNonNull(zone, "zone must be non-null");
+ Objects.requireNonNull(routingStatus, "globalRouting must be non-null");
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/Rotation.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/Rotation.java
index 8eeab8c20e3..ea97b1da4de 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/Rotation.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/Rotation.java
@@ -10,14 +10,11 @@ import java.util.Objects;
*
* @author mpolden
*/
-public class Rotation {
+public record Rotation(RotationId id, String name) {
- private final RotationId id;
- private final String name;
-
- public Rotation(RotationId id, String name) {
- this.id = Objects.requireNonNull(id);
- this.name = Objects.requireNonNull(name);
+ public Rotation {
+ Objects.requireNonNull(id);
+ Objects.requireNonNull(name);
}
/** The ID of the allocated rotation. This value is generated by global routing system */
@@ -31,19 +28,6 @@ public class Rotation {
}
@Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof Rotation)) return false;
- final Rotation rotation = (Rotation) o;
- return id().equals(rotation.id()) && name().equals(rotation.name());
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(id(), name());
- }
-
- @Override
public String toString() {
return Text.format("rotation %s -> %s", id().asString(), name());
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationId.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationId.java
index 4d97962a40a..95cebf7ea78 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationId.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationId.java
@@ -1,20 +1,12 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.controller.routing.rotation;
-import java.util.Objects;
-
/**
* ID of a global rotation.
*
* @author mpolden
*/
-public class RotationId {
-
- private final String id;
-
- public RotationId(String id) {
- this.id = id;
- }
+public record RotationId(String id) {
/** Rotation ID, e.g. rotation-42.vespa.global.routing */
public String asString() {
@@ -22,19 +14,6 @@ public class RotationId {
}
@Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- RotationId that = (RotationId) o;
- return Objects.equals(id, that.id);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(id);
- }
-
- @Override
public String toString() {
return "rotation ID " + id;
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationStatus.java
index 6d95ad9a230..89247ca2a31 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationStatus.java
@@ -13,13 +13,11 @@ import java.util.Objects;
*
* @author mpolden
*/
-public class RotationStatus {
+public record RotationStatus(Map<RotationId, Targets> status) {
public static final RotationStatus EMPTY = new RotationStatus(Map.of());
- private final Map<RotationId, Targets> status;
-
- private RotationStatus(Map<RotationId, Targets> status) {
+ public RotationStatus(Map<RotationId, Targets> status) {
this.status = Map.copyOf(Objects.requireNonNull(status));
}
@@ -46,31 +44,15 @@ public class RotationStatus {
return "rotation status " + status;
}
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- RotationStatus that = (RotationStatus) o;
- return status.equals(that.status);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(status);
- }
-
public static RotationStatus from(Map<RotationId, Targets> targets) {
return targets.isEmpty() ? EMPTY : new RotationStatus(targets);
}
/** Targets of a rotation */
- public static class Targets {
+ public record Targets(Map<ZoneId, RotationState> targets, Instant lastUpdated) {
public static final Targets NONE = new Targets(Map.of(), Instant.EPOCH);
- private final Map<ZoneId, RotationState> targets;
- private final Instant lastUpdated;
-
public Targets(Map<ZoneId, RotationState> targets, Instant lastUpdated) {
this.targets = Map.copyOf(Objects.requireNonNull(targets, "states must be non-null"));
this.lastUpdated = Objects.requireNonNull(lastUpdated, "lastUpdated must be non-null");
@@ -80,24 +62,6 @@ public class RotationStatus {
return targets;
}
- public Instant lastUpdated() {
- return lastUpdated;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- Targets targets1 = (Targets) o;
- return targets.equals(targets1.targets) &&
- lastUpdated.equals(targets1.lastUpdated);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(targets, lastUpdated);
- }
-
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VersionStatus.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VersionStatus.java
index add95c2f7b2..b0a37474af7 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VersionStatus.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VersionStatus.java
@@ -1,6 +1,8 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.controller.versions;
+import ai.vespa.validation.Validation;
+import com.yahoo.collections.Iterables;
import com.yahoo.component.Version;
import com.yahoo.config.provision.HostName;
import com.yahoo.vespa.hosted.controller.Controller;
@@ -9,6 +11,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.configserver.NodeFilter
import com.yahoo.vespa.hosted.controller.application.ApplicationList;
import com.yahoo.vespa.hosted.controller.application.SystemApplication;
import com.yahoo.vespa.hosted.controller.maintenance.SystemUpgrader;
+import com.yahoo.vespa.hosted.controller.versions.VespaVersion.Confidence;
import java.util.ArrayList;
import java.util.Collections;
@@ -225,14 +228,14 @@ public record VersionStatus(List<VespaVersion> versions) {
if (!confidenceIsOverridden) {
// Always compute confidence for system and controller
if (isSystemVersion || isControllerVersion) {
- confidence = VespaVersion.confidenceFrom(statistics, controller);
+ confidence = VespaVersion.confidenceFrom(statistics, controller, versionStatus);
} else {
- // This is an older version so we preserve the existing confidence, if any
+ // This is an older version, so we preserve the existing confidence, if any
confidence = versionStatus.versions().stream()
.filter(v -> statistics.version().equals(v.versionNumber()))
.map(VespaVersion::confidence)
.findFirst()
- .orElseGet(() -> VespaVersion.confidenceFrom(statistics, controller));
+ .orElseGet(() -> VespaVersion.confidenceFrom(statistics, controller, versionStatus));
}
}
@@ -262,4 +265,14 @@ public record VersionStatus(List<VespaVersion> versions) {
confidence);
}
+ /** Whether no version on a newer major, with high confidence, can be deployed. */
+ public boolean isOnCurrentMajor(Version version) {
+ for (VespaVersion available : deployableVersions())
+ if ( available.confidence().equalOrHigherThan(Confidence.high)
+ && available.versionNumber().getMajor() > version.getMajor())
+ return false;
+
+ return true;
+ }
+
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VespaVersion.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VespaVersion.java
index 4cb10aa6ba9..0814e9ea6ec 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VespaVersion.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/VespaVersion.java
@@ -29,11 +29,11 @@ public record VespaVersion(Version version,
List<NodeVersion> nodeVersions,
Confidence confidence) implements Comparable<VespaVersion> {
- public static Confidence confidenceFrom(DeploymentStatistics statistics, Controller controller) {
+ public static Confidence confidenceFrom(DeploymentStatistics statistics, Controller controller, VersionStatus versionStatus) {
int thisMajorVersion = statistics.version().getMajor();
InstanceList all = InstanceList.from(controller.jobController().deploymentStatuses(ApplicationList.from(controller.applications().asList())
.withProductionDeployment()))
- .allowingMajorVersion(thisMajorVersion);
+ .allowingMajorVersion(thisMajorVersion, versionStatus);
// 'production on this': All production deployment jobs upgrading to this version have completed without failure
InstanceList productionOnThis = all.matching(instance -> statistics.productionSuccesses().stream().anyMatch(run -> run.id().application().equals(instance)))
.not().failingUpgrade()
@@ -123,6 +123,9 @@ public record VespaVersion(Version version,
/** We don't have sufficient evidence that this version is working */
low,
+
+ /** This version works, but we want users to stop using it */
+ legacy,
/** We have sufficient evidence that this version is working */
normal,
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 e09e42c9900..dede0c8dec8 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
@@ -67,6 +67,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import static com.yahoo.config.provision.SystemName.main;
+import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.devUsEast1;
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.productionUsEast3;
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.productionUsWest1;
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.stagingTest;
@@ -92,6 +93,7 @@ public class ControllerTest {
void testDeployment() {
// Setup system
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
+ .explicitEnvironment(Environment.dev, Environment.perf)
.region("us-west-1")
.region("us-east-3")
.build();
@@ -694,7 +696,8 @@ public class ControllerTest {
@Test
void testDevDeployment() {
- ApplicationPackage applicationPackage = new ApplicationPackageBuilder().build();
+ // A package without deployment.xml is considered valid
+ ApplicationPackage applicationPackage = new ApplicationPackage(new byte[0]);
// Create application
var context = tester.newDeploymentContext();
@@ -1166,7 +1169,12 @@ public class ControllerTest {
tester.controllerTester().upgradeSystem(version0);
tester.upgrader().overrideConfidence(version0, Confidence.normal);
tester.controllerTester().computeVersionStatus();
+ assertEquals(version0, tester.applications().compileVersion(application, OptionalInt.of(7)));
assertEquals(version0, tester.applications().compileVersion(application, OptionalInt.empty()));
+ assertEquals("this system has no available versions on specified major: 8",
+ assertThrows(IllegalArgumentException.class,
+ () -> tester.applications().compileVersion(application, OptionalInt.of(8)))
+ .getMessage());
context.submit(applicationPackage).deploy();
// System is upgraded
@@ -1266,33 +1274,38 @@ public class ControllerTest {
@Test
void testCloudAccount() {
DeploymentContext context = tester.newDeploymentContext();
- ZoneId zone = ZoneId.from("prod", "us-west-1");
+ ZoneId devZone = devUsEast1.zone();
+ ZoneId prodZone = productionUsWest1.zone();
String cloudAccount = "012345678912";
var applicationPackage = new ApplicationPackageBuilder()
.cloudAccount(cloudAccount)
- .region(zone.region())
+ .region(prodZone.region())
.build();
- // Deployment fails because cloud account is not declared for this tenant
- try {
- context.submit(applicationPackage).deploy();
- fail("Expected exception");
- } catch (IllegalArgumentException e) {
- assertEquals("Cloud account '012345678912' is not valid for tenant 'tenant'", e.getMessage());
- }
+ // Prod and dev deployments fail because cloud account is not declared for this tenant
+ context.submit(applicationPackage).runJobExpectingFailure(systemTest, "Requested cloud account '012345678912' is not valid for tenant 'tenant'");
- // Deployment fails because requested region is not configured in cloud account
+ // Deployment fails because zone is not configured in requested cloud account
tester.controllerTester().flagSource().withListFlag(PermanentFlags.CLOUD_ACCOUNTS.id(), List.of(cloudAccount), String.class);
- try {
- context.submit(applicationPackage).deploy();
- fail("Expected exception");
- } catch (IllegalArgumentException e) {
- assertEquals("Zone prod.us-west-1 in deployment spec is not configured for use in cloud account '012345678912', in this system", e.getMessage());
- }
-
- // Deployment succeeds
- tester.controllerTester().zoneRegistry().setCloudAccountZones(new CloudAccount(cloudAccount), zone);
+ context.runJobExpectingFailure(systemTest, "Zone test.us-east-1 is not configured in requested cloud account '012345678912'")
+ .abortJob(stagingTest);
+
+ // Deployment to prod succeeds once all zones are configured in requested account
+ tester.controllerTester().zoneRegistry().configureCloudAccount(new CloudAccount(cloudAccount),
+ systemTest.zone(),
+ stagingTest.zone(),
+ prodZone);
context.submit(applicationPackage).deploy();
- assertEquals(cloudAccount, tester.controllerTester().configServer().cloudAccount(context.deploymentIdIn(zone)).get().value());
+
+ // Dev zone is added as a configured zone and deployment succeeds
+ tester.controllerTester().zoneRegistry().configureCloudAccount(new CloudAccount(cloudAccount), devZone);
+ context.runJob(devZone, applicationPackage);
+
+ // All deployments use the custom account
+ for (var zoneId : List.of(systemTest.zone(), stagingTest.zone(), devZone, prodZone)) {
+ assertEquals(cloudAccount, tester.controllerTester().configServer()
+ .cloudAccount(context.deploymentIdIn(zoneId))
+ .get().value());
+ }
}
@Test
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 10fd57ce032..5380cf4ee27 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
@@ -344,7 +344,7 @@ public final class ControllerTester {
private TenantName createCloudTenant(String tenantName) {
TenantName tenant = TenantName.from(tenantName);
TenantSpec spec = new CloudTenantSpec(tenant, "token");
- controller().tenants().create(spec, new Auth0Credentials(new SimplePrincipal("dev"), Set.of(Role.administrator(tenant))));
+ controller().tenants().create(spec, new Auth0Credentials(new SimplePrincipal("dev-" + tenantName), Set.of(Role.administrator(tenant))));
return tenant;
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/ApplicationPackageBuilder.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/ApplicationPackageBuilder.java
index 146135491a3..8ddd0ef2be3 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/ApplicationPackageBuilder.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/ApplicationPackageBuilder.java
@@ -5,6 +5,7 @@ import com.yahoo.component.Version;
import com.yahoo.config.application.api.ValidationId;
import com.yahoo.config.provision.AthenzDomain;
import com.yahoo.config.provision.AthenzService;
+import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.InstanceName;
import com.yahoo.config.provision.RegionName;
import com.yahoo.security.SignatureAlgorithm;
@@ -26,6 +27,7 @@ import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.OptionalInt;
@@ -55,6 +57,7 @@ public class ApplicationPackageBuilder {
private final StringBuilder endpointsBody = new StringBuilder();
private final StringBuilder applicationEndpointsBody = new StringBuilder();
private final List<X509Certificate> trustedCertificates = new ArrayList<>();
+ private final Map<Environment, Map<String, String>> nonProductionEnvironments = new LinkedHashMap<>();
private OptionalInt majorVersion = OptionalInt.empty();
private String instances = "default";
@@ -65,8 +68,6 @@ public class ApplicationPackageBuilder {
private String globalServiceId = null;
private String athenzIdentityAttributes = "athenz-domain='domain' athenz-service='service'";
private String searchDefinition = "search test { }";
- private boolean explicitSystemTest = false;
- private boolean explicitStagingTest = false;
private Version compileVersion = Version.fromString("6.1");
private String cloudAccount = null;
@@ -135,12 +136,22 @@ public class ApplicationPackageBuilder {
}
public ApplicationPackageBuilder systemTest() {
- explicitSystemTest = true;
- return this;
+ return explicitEnvironment(Environment.test);
}
public ApplicationPackageBuilder stagingTest() {
- explicitStagingTest = true;
+ return explicitEnvironment(Environment.staging);
+ }
+
+ public ApplicationPackageBuilder explicitEnvironment(Environment environment, Environment... rest) {
+ Stream.concat(Stream.of(environment), Arrays.stream(rest))
+ .forEach(env -> nonProductionEnvironment(env, Map.of()));
+ return this;
+ }
+
+ private ApplicationPackageBuilder nonProductionEnvironment(Environment environment, Map<String, String> attributes) {
+ if (environment.isProduction()) throw new IllegalArgumentException("Expected non-production environment, got " + environment);
+ nonProductionEnvironments.put(environment, attributes);
return this;
}
@@ -267,6 +278,10 @@ public class ApplicationPackageBuilder {
return this;
}
+ public ApplicationPackageBuilder cloudAccount(Environment environment, String cloudAccount) {
+ return nonProductionEnvironment(environment, Map.of("cloud-account", cloudAccount));
+ }
+
private byte[] deploymentSpec() {
StringBuilder xml = new StringBuilder();
xml.append("<deployment version='1.0' ");
@@ -291,10 +306,13 @@ public class ApplicationPackageBuilder {
xml.append("/>\n");
}
xml.append(notifications);
- if (explicitSystemTest)
- xml.append(" <test />\n");
- if (explicitStagingTest)
- xml.append(" <staging />\n");
+ nonProductionEnvironments.forEach((environment, attributes) -> {
+ xml.append(" <").append(environment.value());
+ attributes.forEach((attribute, value) -> {
+ xml.append(" ").append(attribute).append("='").append(value).append("'");
+ });
+ xml.append(" />\n");
+ });
xml.append(blockChange);
xml.append(" <prod");
if (globalServiceId != null) {
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 934c6b07c29..f0402eb764b 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
@@ -338,18 +338,18 @@ public class DeploymentContext {
/** Fail current deployment in given job */
public DeploymentContext failDeployment(JobType type) {
- return failDeployment(type, new IllegalArgumentException("Exception from test code"));
+ return failDeployment(type, new RuntimeException("Exception from test code"));
}
/** Fail current deployment in given job */
private DeploymentContext failDeployment(JobType type, RuntimeException exception) {
configServer().throwOnNextPrepare(exception);
- runJobExpectingFailure(type, Optional.empty());
+ runJobExpectingFailure(type, null);
return this;
}
/** Run given job and expect it to fail with given message, if any */
- public DeploymentContext runJobExpectingFailure(JobType type, Optional<String> messagePart) {
+ public DeploymentContext runJobExpectingFailure(JobType type, String messagePart) {
triggerJobs();
var job = jobId(type);
RunId id = currentRun(job).id();
@@ -357,7 +357,7 @@ public class DeploymentContext {
Run run = jobs.run(id);
assertTrue(run.hasFailed());
assertTrue(run.hasEnded());
- if (messagePart.isPresent()) {
+ if (messagePart != null) {
Optional<Step> firstFailing = run.stepStatuses().entrySet().stream()
.filter(kv -> kv.getValue() == failed)
.map(Entry::getKey)
@@ -366,8 +366,8 @@ public class DeploymentContext {
Optional<RunLog> details = jobs.details(id);
assertTrue(details.isPresent(), "Found log entries for run " + id);
assertTrue(details.get().get(firstFailing.get()).stream()
- .anyMatch(entry -> entry.message().contains(messagePart.get())),
- "Found log message containing '" + messagePart.get() + "'");
+ .anyMatch(entry -> entry.message().contains(messagePart)),
+ "Found log message containing '" + messagePart + "'");
}
return this;
}
@@ -406,7 +406,7 @@ public class DeploymentContext {
/** Runs a deployment of the given package to the given dev/perf job, on the given version. */
public DeploymentContext runJob(JobType type, ApplicationPackage applicationPackage, Version vespaVersion) {
- jobs.deploy(instanceId, type, Optional.ofNullable(vespaVersion), applicationPackage, false);
+ jobs.deploy(instanceId, type, Optional.ofNullable(vespaVersion), applicationPackage, false, true);
return runJob(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 3b8d0374a45..89b6f6ca606 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
@@ -2,6 +2,7 @@
package com.yahoo.vespa.hosted.controller.deployment;
import com.yahoo.component.Version;
+import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.CloudName;
import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.InstanceName;
@@ -23,6 +24,7 @@ import com.yahoo.vespa.hosted.controller.application.SystemApplication;
import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.integration.ZoneApiMock;
import com.yahoo.vespa.hosted.controller.integration.ZoneRegistryMock;
+import com.yahoo.vespa.hosted.controller.maintenance.DeploymentUpgrader;
import com.yahoo.vespa.hosted.controller.versions.VespaVersion;
import com.yahoo.vespa.hosted.controller.versions.VespaVersion.Confidence;
import org.junit.jupiter.api.Test;
@@ -41,6 +43,7 @@ import java.util.stream.Collectors;
import static ai.vespa.validation.Validation.require;
import static com.yahoo.config.provision.SystemName.cd;
+import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.applicationPackage;
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.productionApNortheast1;
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.productionApNortheast2;
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentContext.productionApSoutheast1;
@@ -64,8 +67,10 @@ import static java.util.Collections.emptyList;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
/**
* Tests a wide variety of deployment scenarios and configurations
@@ -1189,6 +1194,7 @@ public class DeploymentTriggerTest {
.region("us-central-1")
.test("us-central-1")
.test("us-west-1")
+ .region("eu-west-1")
.build();
var app = tester.newDeploymentContext().submit(applicationPackage);
@@ -1197,15 +1203,18 @@ public class DeploymentTriggerTest {
tester.clock().advance(Duration.ofMinutes(1));
app.runJob(testUsEast3)
- .runJob(productionUsWest1).runJob(productionUsCentral1)
- .runJob(testUsCentral1).runJob(testUsWest1);
+ .runJob(productionUsWest1).runJob(productionUsCentral1)
+ .runJob(testUsCentral1).runJob(testUsWest1)
+ .runJob(productionEuWest1);
assertEquals(Change.empty(), app.instance().change());
- // Application starts upgrade, but is confidence is broken cancelled after first zone. Tests won't run.
+ // Application starts upgrade, but confidence is broken after first zone. Tests won't run.
Version version0 = app.application().oldestDeployedPlatform().get();
Version version1 = Version.fromString("6.7");
+ Version version2 = Version.fromString("6.8");
tester.controllerTester().upgradeSystem(version1);
tester.upgrader().maintain();
+ tester.newDeploymentContext("keep", "version1", "alive").submit().deploy();
app.runJob(systemTest).runJob(stagingTest).runJob(productionUsEast3);
tester.clock().advance(Duration.ofMinutes(1));
@@ -1229,12 +1238,17 @@ public class DeploymentTriggerTest {
app.runJob(testUsEast3);
assertEquals(Change.empty().withPin(), app.instance().change());
- // Same upgrade is attempted, and production tests wait for redeployment.
+ // A new upgrade is attempted, and production tests wait for redeployment.
+ tester.controllerTester().upgradeSystem(version2);
tester.deploymentTrigger().cancelChange(app.instanceId(), ALL);
+
tester.upgrader().overrideConfidence(version1, VespaVersion.Confidence.high);
tester.controllerTester().computeVersionStatus();
- tester.upgrader().maintain();
+ tester.upgrader().maintain(); // App should target version2.
+ assertEquals(Change.of(version2), app.instance().change());
+ // App partially upgrades to version2.
+ app.runJob(systemTest).runJob(stagingTest);
app.triggerJobs();
app.assertRunning(productionUsEast3);
app.assertNotRunning(testUsEast3);
@@ -1245,6 +1259,20 @@ public class DeploymentTriggerTest {
tester.runner().run();
app.triggerJobs();
app.assertNotRunning(testUsCentral1);
+ app.assertNotRunning(testUsWest1);
+
+ // Version2 gets broken, but Version1 has high confidence now, and is the new target.
+ // Since us-east-3 is already on Version2, both deployment and tests to it should be skipped.
+ tester.upgrader().overrideConfidence(version2, VespaVersion.Confidence.broken);
+ tester.controllerTester().computeVersionStatus();
+ tester.upgrader().maintain(); // App should target version2.
+ assertEquals(Change.of(version1), app.instance().change());
+ app.triggerJobs();
+
+ // Deployment to 6.8 already happened, so a downgrade to 6.7 won't, but production tests will still run.
+ app.timeOutConvergence(productionUsCentral1);
+ app.runJob(testUsCentral1).runJob(testUsWest1).runJob(productionEuWest1);
+ assertEquals(version1, app.instance().deployments().get(ZoneId.from("prod.eu-west-1")).version());
}
@Test
@@ -2027,16 +2055,16 @@ public class DeploymentTriggerTest {
var conservative = tester.newDeploymentContext("t", "a", "default");
canary.runJob(systemTest)
- .runJob(stagingTest);
+ .runJob(stagingTest);
conservative.runJob(productionEuWest1)
- .runJob(testEuWest1);
+ .runJob(testEuWest1);
canary.submit(applicationPackage)
- .runJob(systemTest)
- .runJob(stagingTest);
+ .runJob(systemTest)
+ .runJob(stagingTest);
tester.outstandingChangeDeployer().run();
conservative.runJob(productionEuWest1)
- .runJob(testEuWest1);
+ .runJob(testEuWest1);
tester.controllerTester().upgradeSystem(new Version("6.7.7"));
tester.upgrader().maintain();
@@ -2045,7 +2073,7 @@ public class DeploymentTriggerTest {
.runJob(stagingTest);
tester.upgrader().maintain();
conservative.runJob(productionEuWest1)
- .runJob(testEuWest1);
+ .runJob(testEuWest1);
}
@@ -2178,6 +2206,7 @@ public class DeploymentTriggerTest {
app.submit(new ApplicationPackageBuilder().region("us-east-3")
.compileVersion(version2)
.build());
+ tester.upgrader().overrideConfidence(version2, Confidence.normal);
tester.upgrader().overrideConfidence(version3, Confidence.broken);
tester.controllerTester().computeVersionStatus();
tester.upgrader().run();
@@ -2222,6 +2251,143 @@ public class DeploymentTriggerTest {
newApp.deploy();
assertEquals(version2, tester.jobs().last(newApp.instanceId(), productionUsEast3).get().versions().targetPlatform());
assertEquals(version2, newApp.application().revisions().get(tester.jobs().last(newApp.instanceId(), productionUsEast3).get().versions().targetRevision()).compileVersion().get());
+
+ // New app compiles against old major, and downgrades when a pin is also applied.
+ newApp.submit(new ApplicationPackageBuilder().compileVersion(version1)
+ .systemTest()
+ .region("us-east-3")
+ .build());
+ newRevision = newApp.lastSubmission().get();
+
+ assertEquals(Change.of(newRevision).with(version1), newApp.instance().change());
+ tester.triggerJobs();
+ newApp.assertNotRunning(systemTest); // Without a pin, the platform won't downgrade, and 8 is incompatible with compiled 7.
+
+ tester.outstandingChangeDeployer().run();
+ assertEquals(Change.of(newRevision).with(version1), newApp.instance().change());
+ tester.upgrader().run();
+ assertEquals(Change.of(newRevision).with(version1), newApp.instance().change());
+
+ tester.deploymentTrigger().forceChange(newApp.instanceId(), newApp.instance().change().withPin());
+ tester.outstandingChangeDeployer().run();
+ assertEquals(Change.of(newRevision).with(version1).withPin(), newApp.instance().change());
+ tester.upgrader().run();
+ assertEquals(Change.of(newRevision).with(version1).withPin(), newApp.instance().change());
+
+ newApp.deploy();
+ assertEquals(version1, tester.jobs().last(newApp.instanceId(), productionUsEast3).get().versions().targetPlatform());
+ assertEquals(version1, newApp.application().revisions().get(tester.jobs().last(newApp.instanceId(), productionUsEast3).get().versions().targetRevision()).compileVersion().get());
+ }
+
+ @Test
+ void testOutdatedMajorIsIllegal() {
+ Version version0 = new Version("6.2");
+ Version version1 = new Version("7.1");
+ tester.controllerTester().upgradeSystem(version0);
+ DeploymentContext old = tester.newDeploymentContext("t", "a", "default").submit()
+ .runJob(systemTest).runJob(stagingTest).runJob(productionUsCentral1);
+ old.runJob(JobType.dev("us-east-1"), applicationPackage());
+
+ tester.controllerTester().upgradeSystem(version1);
+ tester.upgrader().overrideConfidence(version1, Confidence.high);
+ tester.controllerTester().computeVersionStatus();
+
+ // New app can't deploy to 6.2
+ DeploymentContext app = tester.newDeploymentContext("t", "b", "default");
+ assertEquals("platform version 6.2 is not on a current major version in this system",
+ assertThrows(IllegalArgumentException.class,
+ () -> tester.jobs().deploy(app.instanceId(),
+ JobType.dev("us-east-1"),
+ Optional.of(version0),
+ DeploymentContext.applicationPackage()))
+ .getMessage());
+
+ // App which already deployed to 6.2 can still do so.
+ tester.jobs().deploy(old.instanceId(),
+ JobType.dev("us-east-1"),
+ Optional.of(version0),
+ DeploymentContext.applicationPackage());
+
+ app.submit();
+ assertEquals("platform version 6.2 is not on a current major version in this system",
+ assertThrows(IllegalArgumentException.class,
+ () -> tester.deploymentTrigger().forceChange(app.instanceId(), Change.of(version0), false))
+ .getMessage());
+
+ tester.deploymentTrigger().forceChange(old.instanceId(), Change.of(version0), false);
+
+ // Not even version incompatibility tricks the system.
+ tester.controllerTester().flagSource().withListFlag(PermanentFlags.INCOMPATIBLE_VERSIONS.id(), List.of("7"), String.class);
+ assertEquals("compile version 6.2 is incompatible with the current major version of this system",
+ assertThrows(IllegalArgumentException.class,
+ () ->
+ app.submit(new ApplicationPackageBuilder().region("us-central-1").region("us-east-3").region("us-west-1")
+ .compileVersion(version0)
+ .build()))
+ .getMessage());
+
+ // Upgrade.
+ old.submit(new ApplicationPackageBuilder().region("us-central-1").region("us-east-3").region("us-west-1")
+ .compileVersion(version1)
+ .build())
+ .deploy();
+
+ // And downgrade again.
+ old.submit(new ApplicationPackageBuilder().region("us-central-1").region("us-east-3").region("us-west-1")
+ .compileVersion(version0)
+ .build());
+
+ assertEquals(Change.of(version0).with(old.lastSubmission().get()), old.instance().change());
+
+ // An operator can still trigger roll-out of the otherwise illegal submission.
+ tester.deploymentTrigger().forceChange(app.instanceId(), Change.of(app.lastSubmission().get()));
+ assertEquals(Change.of(app.lastSubmission().get()), app.instance().change());
+ }
+
+ @Test
+ void testInitialDeploymentPlatform() {
+ Version version0 = tester.controllerTester().controller().readSystemVersion();
+ Version version1 = new Version("6.2");
+ Version version2 = new Version("6.3");
+ assertEquals(version0, tester.newDeploymentContext("t", "a1", "default").submit().deploy().application().oldestDeployedPlatform().get());
+
+ // A new version, with normal confidence, is the default for a new app.
+ tester.controllerTester().upgradeSystem(version1);
+ tester.upgrader().overrideConfidence(version1, Confidence.normal);
+ tester.controllerTester().computeVersionStatus();
+ assertEquals(version1, tester.newDeploymentContext("t", "a2", "default").submit().deploy().application().oldestDeployedPlatform().get());
+
+ // A newer version has broken confidence, leaving the previous version as the default.
+ tester.controllerTester().upgradeSystem(version2);
+ tester.upgrader().overrideConfidence(version2, Confidence.broken);
+ tester.controllerTester().computeVersionStatus();
+ assertEquals(version1, tester.newDeploymentContext("t", "a3", "default").submit().deploy().application().oldestDeployedPlatform().get());
+
+ DeploymentContext dev1 = tester.newDeploymentContext("t", "d1", "default");
+ DeploymentContext dev2 = tester.newDeploymentContext("t", "d2", "default");
+ assertEquals(version1, dev1.runJob(JobType.dev("us-east-1"), DeploymentContext.applicationPackage()).deployment(ZoneId.from("dev", "us-east-1")).version());
+
+ DeploymentUpgrader devUpgrader = new DeploymentUpgrader(tester.controller(), Duration.ofHours(1));
+ for (int i = 0; i < 24; i++) {
+ tester.clock().advance(Duration.ofHours(1));
+ devUpgrader.run();
+ }
+ dev1.assertNotRunning(JobType.dev("us-east-1"));
+
+ // Normal confidence lets the newest version be the default again.
+ tester.upgrader().overrideConfidence(version2, Confidence.normal);
+ tester.controllerTester().computeVersionStatus();
+ assertEquals(version2, tester.newDeploymentContext("t", "a4", "default").submit().deploy().application().oldestDeployedPlatform().get());
+ assertEquals(version1, dev1.runJob(JobType.dev("us-east-1"), DeploymentContext.applicationPackage()).deployment(ZoneId.from("dev", "us-east-1")).version());
+ assertEquals(version2, dev2.runJob(JobType.dev("us-east-1"), DeploymentContext.applicationPackage()).deployment(ZoneId.from("dev", "us-east-1")).version());
+
+ for (int i = 0; i < 24; i++) {
+ tester.clock().advance(Duration.ofHours(1));
+ devUpgrader.run();
+ }
+ dev1.assertRunning(JobType.dev("us-east-1"));
+ dev1.runJob(JobType.dev("us-east-1"));
+ assertEquals(version2, dev1.deployment(ZoneId.from("dev", "us-east-1")).version());
}
@Test
@@ -2300,7 +2466,7 @@ public class DeploymentTriggerTest {
Version version3 = new Version("6.4");
tester.controllerTester().upgradeSystem(version3);
tests.runJob(systemTest) // Success in default cloud.
- .failDeployment(systemTest); // Failure in centauri cloud.
+ .failDeployment(systemTest); // Failure in centauri cloud.
tester.upgrader().run();
assertEquals(Change.of(version3), tests.instance().change());
@@ -2397,6 +2563,97 @@ public class DeploymentTriggerTest {
}
@Test
+ void testInstancesWithMultipleClouds() {
+ String spec = """
+ <deployment>
+ <parallel>
+ <instance id='separate'>
+ <test />
+ <staging />
+ <prod>
+ <region>alpha-centauri</region>
+ </prod>
+ </instance>
+ <instance id='independent'>
+ <test />
+ </instance>
+ <steps>
+ <parallel>
+ <instance id='alpha'>
+ <test />
+ <prod>
+ <region>us-east-3</region>
+ </prod>
+ </instance>
+ <instance id='beta'>
+ <test />
+ <prod>
+ <region>alpha-centauri</region>
+ </prod>
+ </instance>
+ <instance id='gamma'>
+ <test />
+ </instance>
+ </parallel>
+ <instance id='nu'>
+ <staging />
+ </instance>
+ <instance id='omega'>
+ <prod>
+ <region>alpha-centauri</region>
+ </prod>
+ </instance>
+ </steps>
+ <instance id='dependent'>
+ <prod>
+ <region>us-east-3</region>
+ </prod>
+ </instance>
+ </parallel>
+ </deployment>
+ """;
+
+ RegionName alphaCentauri = RegionName.from("alpha-centauri");
+ ZoneApiMock.Builder builder = ZoneApiMock.newBuilder().withCloud("centauri").withSystem(tester.controller().system());
+ ZoneApi testAlphaCentauri = builder.with(ZoneId.from(Environment.test, alphaCentauri)).build();
+ ZoneApi stagingAlphaCentauri = builder.with(ZoneId.from(Environment.staging, alphaCentauri)).build();
+ ZoneApi prodAlphaCentauri = builder.with(ZoneId.from(Environment.prod, alphaCentauri)).build();
+
+ tester.controllerTester().zoneRegistry().addZones(testAlphaCentauri, stagingAlphaCentauri, prodAlphaCentauri);
+ tester.controllerTester().setRoutingMethod(tester.controllerTester().zoneRegistry().zones().all().ids(), RoutingMethod.sharedLayer4);
+ tester.configServer().bootstrap(tester.controllerTester().zoneRegistry().zones().all().ids(), SystemApplication.notController());
+
+ ApplicationPackage appPackage = ApplicationPackageBuilder.fromDeploymentXml(spec);
+ DeploymentContext app = tester.newDeploymentContext("tenant", "application", "alpha").submit(appPackage).deploy();
+ app.submit(appPackage);
+ Map<JobId, List<DeploymentStatus.Job>> jobs = app.deploymentStatus().jobsToRun();
+
+ JobType centauriTest = JobType.systemTest(tester.controller().zoneRegistry(), CloudName.from("centauri"));
+ JobType centauriStaging = JobType.stagingTest(tester.controller().zoneRegistry(), CloudName.from("centauri"));
+ assertQueued("separate", jobs, centauriTest);
+ assertQueued("separate", jobs, stagingTest, centauriStaging);
+ assertQueued("independent", jobs, systemTest, centauriTest);
+ assertQueued("alpha", jobs, systemTest);
+ assertQueued("beta", jobs, centauriTest);
+ assertQueued("gamma", jobs, centauriTest);
+
+ // Once alpha runs its default system test, it also runs the centauri system test, as omega depends on it.
+ app.runJob(systemTest);
+ assertQueued("alpha", app.deploymentStatus().jobsToRun(), centauriTest);
+ }
+
+ private static void assertQueued(String instance, Map<JobId, List<DeploymentStatus.Job>> jobs, JobType... expected) {
+ List<DeploymentStatus.Job> queued = jobs.get(new JobId(ApplicationId.from("tenant", "application", instance), expected[0]));
+ Set<ZoneId> remaining = new HashSet<>();
+ for (JobType ex : expected) remaining.add(ex.zone());
+ for (DeploymentStatus.Job q : queued)
+ if ( ! remaining.remove(q.type().zone()))
+ fail("unexpected queued job for " + instance + ": " + q.type());
+ if ( ! remaining.isEmpty())
+ fail("expected tests for " + instance + " were not queued in : " + remaining);
+ }
+
+ @Test
void testNoTests() {
DeploymentContext app = tester.newDeploymentContext();
app.submit(new ApplicationPackageBuilder().systemTest().region("us-east-3").build());
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java
index 680e69998b6..0211a052f76 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java
@@ -148,8 +148,8 @@ public class ZoneRegistryMock extends AbstractComponent implements ZoneRegistry
return this;
}
- public ZoneRegistryMock setCloudAccountZones(CloudAccount cloudAccount, ZoneId... zones) {
- this.cloudAccountZones.put(cloudAccount, Set.of(zones));
+ public ZoneRegistryMock configureCloudAccount(CloudAccount cloudAccount, ZoneId... zones) {
+ this.cloudAccountZones.computeIfAbsent(cloudAccount, (k) -> new HashSet<>()).addAll(Set.of(zones));
return this;
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainerTest.java
index 69301ea91ef..174cf93286c 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/RetriggerMaintainerTest.java
@@ -11,7 +11,6 @@ import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester;
import com.yahoo.vespa.hosted.controller.deployment.RetriggerEntry;
import org.junit.jupiter.api.Test;
-import java.io.IOException;
import java.time.Duration;
import java.util.List;
@@ -25,7 +24,7 @@ public class RetriggerMaintainerTest {
private final DeploymentTester tester = new DeploymentTester();
@Test
- void processes_queue() throws IOException {
+ void processes_queue() {
RetriggerMaintainer maintainer = new RetriggerMaintainer(tester.controller(), Duration.ofDays(1));
ApplicationId applicationId = ApplicationId.from("tenant", "app", "default");
var devApp = tester.newDeploymentContext(applicationId);
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 76f0cacb7c5..35ccc86d465 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
@@ -5,6 +5,7 @@ import com.yahoo.component.Version;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.test.ManualClock;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.RevisionId;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.RunId;
import com.yahoo.vespa.hosted.controller.application.Change;
@@ -41,6 +42,7 @@ import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger.Cha
import static com.yahoo.vespa.hosted.controller.deployment.DeploymentTrigger.ChangesToCancel.PLATFORM;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
@@ -661,8 +663,8 @@ public class UpgraderTest {
@Test
void testPinningMajorVersionInDeploymentXml() {
- Version version = Version.fromString("6.2");
- tester.controllerTester().upgradeSystem(version);
+ Version version0 = Version.fromString("6.2");
+ tester.controllerTester().upgradeSystem(version0);
ApplicationPackageBuilder builder = new ApplicationPackageBuilder().region("us-west-1").majorVersion(7);
ApplicationPackage defaultPackage = new ApplicationPackageBuilder().region("us-west-1").build();
@@ -674,7 +676,7 @@ public class UpgraderTest {
var lazyApp = tester.newDeploymentContext().submit(defaultPackage).deploy();
// New major version is released; more apps upgrade with increasing confidence.
- version = Version.fromString("7.0");
+ Version version = Version.fromString("7.0");
tester.controllerTester().upgradeSystem(version);
tester.upgrader().overrideConfidence(version, Confidence.broken);
tester.controllerTester().computeVersionStatus();
@@ -921,8 +923,20 @@ public class UpgraderTest {
instance -> instance.withChange(instance.change().withoutPin()))));
tester.upgrader().maintain();
assertEquals(version1,
- app1.instance().change().platform().orElseThrow(),
- "Application upgrades to latest allowed major");
+ app1.instance().change().platform().orElseThrow(),
+ "Application upgrades to latest allowed major");
+
+ // Version on old major becomes legacy, so app upgrades once it does not specify the old major in deployment spec.
+ app1.runJob(systemTest).runJob(stagingTest).runJob(productionUsWest1).runJob(productionUsEast3);
+ app1.submit(new ApplicationPackageBuilder().majorVersion(6).region("us-east-3").region("us-west-1").build()).deploy();
+ tester.upgrader().maintain();
+ assertEquals(Change.empty(), app1.instance().change());
+
+ app1.submit(new ApplicationPackageBuilder().region("us-east-3").region("us-west-1").build()).deploy();
+ tester.upgrader().overrideConfidence(version1, Confidence.legacy);
+ tester.controllerTester().computeVersionStatus();
+ tester.upgrader().maintain();
+ assertEquals(Change.of(version2), app1.instance().change());
}
@Test
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializerTest.java
index 88da698b3f2..849261d5ae4 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializerTest.java
@@ -92,6 +92,7 @@ public class ApplicationSerializerTest {
Optional.of(new SourceRevision("git@github:org/repo.git", "branch1", "commit1")),
Optional.of("william@shakespeare"),
Optional.of(Version.fromString("1.2.3")),
+ Optional.of(123),
Optional.of(Instant.ofEpochMilli(666)),
Optional.empty(),
Optional.of("best commit"),
@@ -161,6 +162,8 @@ public class ApplicationSerializerTest {
assertEquals(applicationVersion1, serialized.revisions().last().get());
assertEquals(applicationVersion1, serialized.revisions().get(serialized.instances().get(id1.instance()).deployments().get(zone1).revision()));
assertEquals(original.revisions().last(), serialized.revisions().last());
+ assertEquals(original.revisions().last().get().compileVersion(), serialized.revisions().last().get().compileVersion());
+ assertEquals(original.revisions().last().get().allowedMajor(), serialized.revisions().last().get().allowedMajor());
assertEquals(original.revisions().last().get().authorEmail(), serialized.revisions().last().get().authorEmail());
assertEquals(original.revisions().last().get().buildTime(), serialized.revisions().last().get().buildTime());
assertEquals(original.revisions().last().get().sourceUrl(), serialized.revisions().last().get().sourceUrl());
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerTester.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerTester.java
index 539d6cff06d..4194131e7fb 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerTester.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/ContainerTester.java
@@ -16,6 +16,7 @@ import com.yahoo.vespa.flags.InMemoryFlagSource;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.api.integration.athenz.ApplicationAction;
import com.yahoo.vespa.hosted.controller.api.integration.athenz.AthenzClientFactoryMock;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockUserManagement;
import com.yahoo.vespa.hosted.controller.integration.ServiceRegistryMock;
import java.io.File;
@@ -64,6 +65,10 @@ public class ContainerTester {
return (ServiceRegistryMock) container.components().getComponent(ServiceRegistryMock.class.getName());
}
+ public MockUserManagement userManagement() {
+ return (MockUserManagement) container.components().getComponent(MockUserManagement.class.getName());
+ }
+
public void authorize(AthenzDomain tenantDomain, AthenzIdentity identity, ApplicationAction action, ApplicationName application) {
athenzClientFactory().getSetup()
.domains.get(tenantDomain)
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 3b90be40a32..af0a85f1a90 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
@@ -422,7 +422,7 @@ public class ApplicationApiCloudTest extends ControllerContainerCloudTest {
@Test
void create_application_on_deploy() {
var application = ApplicationName.from("unique");
- var applicationPackage = new ApplicationPackageBuilder().withoutAthenzIdentity().build();
+ var applicationPackage = new ApplicationPackageBuilder().trustDefaultCertificate().withoutAthenzIdentity().build();
new ControllerTester(tester).upgradeSystem(new Version("6.1"));
assertTrue(tester.controller().applications().getApplication(TenantAndApplicationId.from(tenantName, application)).isEmpty());
@@ -482,6 +482,7 @@ public class ApplicationApiCloudTest extends ControllerContainerCloudTest {
private void deployApplication() {
var applicationPackage = new ApplicationPackageBuilder()
+ .trustDefaultCertificate()
.instances("default")
.globalServiceId("foo")
.region("aws-us-east-1c")
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 168b9b374f3..71e3607983c 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
@@ -192,7 +192,7 @@ public class JobControllerApiHandlerHelperTest {
var region = "us-west-1";
var applicationPackage = new ApplicationPackageBuilder().region(region).build();
// Deploy directly to production zone, like integration tests, with dryRun.
- tester.controller().jobController().deploy(tester.instance().id(), productionUsWest1, Optional.empty(), applicationPackage, true);
+ tester.controller().jobController().deploy(tester.instance().id(), productionUsWest1, Optional.empty(), applicationPackage, true, true);
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/deployment/responses/root.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/responses/root.json
index 13a6391d5da..df682cf8e1b 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/responses/root.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/deployment/responses/root.json
@@ -37,6 +37,7 @@
"instance": "default",
"upgrading": false,
"upgradePolicy": "default",
+ "compileVersion": "6.1.0",
"jobs": [
{
"name": "system-test",
@@ -77,6 +78,7 @@
"instance": "i2",
"upgrading": false,
"upgradePolicy": "default",
+ "compileVersion": "6.1.0",
"jobs": [
{
"name": "production-us-west-1"
@@ -184,6 +186,7 @@
"instance": "default",
"upgrading": true,
"upgradePolicy": "default",
+ "compileVersion": "6.1.0",
"jobs": [
{
"name": "system-test",
@@ -263,6 +266,7 @@
"instance": "i1",
"upgrading": false,
"upgradePolicy": "default",
+ "compileVersion": "6.1.0",
"jobs": [
{
"name": "system-test",
@@ -322,6 +326,7 @@
"instance": "i2",
"upgrading": true,
"upgradePolicy": "default",
+ "compileVersion": "6.1.0",
"jobs": [
{
"name": "production-us-west-1",
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiTest.java
index f34dd3fe629..b573940d150 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/user/UserApiTest.java
@@ -10,6 +10,8 @@ import com.yahoo.vespa.flags.PermanentFlags;
import com.yahoo.vespa.hosted.controller.ControllerTester;
import com.yahoo.vespa.hosted.controller.api.integration.billing.PlanId;
import com.yahoo.jdisc.http.filter.security.misc.User;
+import com.yahoo.vespa.hosted.controller.api.integration.stubs.MockUserManagement;
+import com.yahoo.vespa.hosted.controller.api.integration.user.UserId;
import com.yahoo.vespa.hosted.controller.api.role.Role;
import com.yahoo.vespa.hosted.controller.restapi.ContainerTester;
import com.yahoo.vespa.hosted.controller.restapi.ControllerContainerCloudTest;
@@ -253,6 +255,33 @@ public class UserApiTest extends ControllerContainerCloudTest {
}
@Test
+ void findUser() {
+ try (Flags.Replacer ignored = Flags.clearFlagsForTesting(PermanentFlags.MAX_TRIAL_TENANTS.id(), PermanentFlags.ENABLE_PUBLIC_SIGNUP_FLOW.id())) {
+ ContainerTester tester = new ContainerTester(container, responseFiles);
+ ((InMemoryFlagSource) tester.controller().flagSource())
+ .withBooleanFlag(PermanentFlags.ENABLE_PUBLIC_SIGNUP_FLOW.id(), true);
+ Set<Role> operator = Set.of(Role.hostedOperator(), Role.hostedSupporter(), Role.hostedAccountant());
+ User user = new User("dev@domail", "Joe Developer", "dev", null);
+
+ Role developer = Role.developer(TenantName.from("scoober"));
+ tester.userManagement().createRole(developer);
+ tester.userManagement().addToRoles(new UserId("dev@domail"), Set.of(developer));
+
+ tester.assertResponse(request("/user/v1/find?email=dev@domail")
+ .roles(operator)
+ .user(user),
+ """
+ {"users":[{"isPublic":true,"isCd":false,"hasTrialCapacity":true,"user":{"name":"dev@domail","email":"dev@domail","verified":false},"tenants":{"scoober":{"supported":false,"roles":["developer"]}},"flags":[{"id":"enable-public-signup-flow","rules":[{"value":false}]}]}]}""");
+
+ tester.assertResponse(request("/user/v1/find?query=email:dev@domail")
+ .roles(operator)
+ .user(user),
+ """
+ {"users":[]}""");
+ }
+ }
+
+ @Test
void maxTrialTenants() {
try (Flags.Replacer ignored = Flags.clearFlagsForTesting(PermanentFlags.MAX_TRIAL_TENANTS.id(), PermanentFlags.ENABLE_PUBLIC_SIGNUP_FLOW.id())) {
ContainerTester tester = new ContainerTester(container, responseFiles);
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepositoryTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepositoryTest.java
index 7477b94e3f4..cd8a9e72051 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepositoryTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepositoryTest.java
@@ -17,7 +17,6 @@ import org.junit.jupiter.api.Test;
import java.net.URI;
import java.util.List;
-import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
@@ -114,7 +113,7 @@ public class RotationRepositoryTest {
// We're now out of rotations and next deployment fails
var application3 = tester.newDeploymentContext("tenant3", "app3", "default");
application3.submit(applicationPackage)
- .runJobExpectingFailure(DeploymentContext.systemTest, Optional.of("out of rotations"));
+ .runJobExpectingFailure(DeploymentContext.systemTest, "out of rotations");
}
@Test
@@ -123,7 +122,7 @@ public class RotationRepositoryTest {
.globalServiceId("foo")
.region("us-east-3")
.build();
- application.submit(applicationPackage).runJobExpectingFailure(DeploymentContext.systemTest, Optional.of("less than 2 prod zones are defined"));
+ application.submit(applicationPackage).runJobExpectingFailure(DeploymentContext.systemTest, "less than 2 prod zones are defined");
}
@Test
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 6cb7c059b1d..a0e3c63e58b 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
@@ -4,7 +4,6 @@ package com.yahoo.vespa.hosted.controller.versions;
import com.yahoo.component.Version;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.HostName;
-import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.zone.ZoneApi;
import com.yahoo.vespa.hosted.controller.Controller;
import com.yahoo.vespa.hosted.controller.ControllerTester;
@@ -28,7 +27,6 @@ import java.time.Duration;
import java.time.Instant;
import java.util.List;
import java.util.Map;
-import java.util.OptionalInt;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -91,8 +89,7 @@ public class VersionStatusTest {
HostName controller3 = HostName.of("controller-3");
MockCuratorDb db = new MockCuratorDb(Stream.of(controller1, controller2, controller3)
.map(hostName -> hostName.value() + ":2222")
- .collect(Collectors.joining(",")),
- SystemName.main);
+ .collect(Collectors.joining(",")));
ControllerTester tester = new ControllerTester(db);
writeControllerVersion(controller1, Version.fromString("6.2"), db);
@@ -525,8 +522,7 @@ public class VersionStatusTest {
HostName controller3 = HostName.of("controller-3");
MockCuratorDb db = new MockCuratorDb(Stream.of(controller1, controller2, controller3)
.map(hostName -> hostName.value() + ":2222")
- .collect(Collectors.joining(",")),
- SystemName.main);
+ .collect(Collectors.joining(",")));
DeploymentTester tester = new DeploymentTester(new ControllerTester(db));
// Commit details are set for initial version