summaryrefslogtreecommitdiffstats
path: root/controller-server
diff options
context:
space:
mode:
authorHåkon Hallingstad <hakon@yahooinc.com>2023-07-28 20:36:56 +0200
committerHåkon Hallingstad <hakon@yahooinc.com>2023-07-28 20:36:56 +0200
commite3345ab2a84ccd238072d90edb6c59467322e07b (patch)
tree660bd7948c1d0b6f53c89b3d5412947fc8243504 /controller-server
parent2ee24c4628c75abb8f8495eac978b3eb75c66162 (diff)
parentd488a7482e93ae233be571d61946caa796aba588 (diff)
Merge branch 'master' into hakonhall/add-cloud-flag-dimension
Diffstat (limited to 'controller-server')
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/OsController.java35
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/RoutingController.java27
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/application/pkg/ApplicationPackageValidator.java40
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporter.java2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeScheduler.java50
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiHandler.java21
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicies.java56
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepository.java47
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/tls/ControllerSslContextFactoryProvider.java30
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/CertifiedOsVersion.java11
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/ControllerTest.java18
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/ApplicationPackageBuilder.java29
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTriggerTest.java18
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeSchedulerTest.java36
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsVersionStatusUpdaterTest.java6
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/ApplicationSerializerTest.java4
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/testdata/complete-application.json2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiCloudTest.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java10
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiTest.java15
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-all-upgraded.json2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-partially-upgraded.json2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java43
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepositoryTest.java17
25 files changed, 197 insertions, 330 deletions
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/OsController.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/OsController.java
index 1c77efe095c..c426c27418d 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/OsController.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/OsController.java
@@ -54,9 +54,7 @@ public record OsController(Controller controller) {
* @param pin Pin this version. This prevents automatic scheduling of upgrades until version is unpinned
*/
public void upgradeTo(Version version, CloudName cloud, boolean force, boolean pin) {
- if (version.isEmpty()) {
- throw new IllegalArgumentException("Invalid version '" + version.toFullString() + "'");
- }
+ requireNonEmpty(version);
requireCloud(cloud);
Instant scheduledAt = controller.clock().instant();
try (Mutex lock = curator().lockOsVersions()) {
@@ -124,10 +122,12 @@ public record OsController(Controller controller) {
/** Certify an OS version as compatible with given Vespa version */
public CertifiedOsVersion certify(Version version, CloudName cloud, Version vespaVersion) {
+ requireNonEmpty(version);
+ requireNonEmpty(vespaVersion);
requireCloud(cloud);
try (Mutex lock = curator().lockCertifiedOsVersions()) {
OsVersion osVersion = new OsVersion(version, cloud);
- Set<CertifiedOsVersion> certifiedVersions = curator().readCertifiedOsVersions();
+ Set<CertifiedOsVersion> certifiedVersions = readCertified();
Optional<CertifiedOsVersion> matching = certifiedVersions.stream()
.filter(cv -> cv.osVersion().equals(osVersion))
.findFirst();
@@ -145,12 +145,12 @@ public record OsController(Controller controller) {
public void uncertify(Version version, CloudName cloud) {
try (Mutex lock = curator().lockCertifiedOsVersions()) {
OsVersion osVersion = new OsVersion(version, cloud);
- Set<CertifiedOsVersion> certifiedVersions = curator().readCertifiedOsVersions();
+ Set<CertifiedOsVersion> certifiedVersions = readCertified();
Optional<CertifiedOsVersion> existing = certifiedVersions.stream()
.filter(cv -> cv.osVersion().equals(osVersion))
.findFirst();
if (existing.isEmpty()) {
- throw new IllegalArgumentException(version + " is not certified");
+ throw new IllegalArgumentException(osVersion + " is not certified");
}
certifiedVersions = new HashSet<>(certifiedVersions);
certifiedVersions.remove(existing.get());
@@ -162,7 +162,7 @@ public record OsController(Controller controller) {
public void removeStaleCertifications(OsVersionStatus currentStatus) {
try (Mutex lock = curator().lockCertifiedOsVersions()) {
Set<OsVersion> knownVersions = currentStatus.versions().keySet();
- Set<CertifiedOsVersion> certifiedVersions = new HashSet<>(curator().readCertifiedOsVersions());
+ Set<CertifiedOsVersion> certifiedVersions = new HashSet<>(readCertified());
if (certifiedVersions.removeIf(cv -> !knownVersions.contains(cv.osVersion()))) {
curator().writeCertifiedOsVersions(certifiedVersions);
}
@@ -174,11 +174,16 @@ public record OsController(Controller controller) {
if (controller.system().isCd()) return true; // Always certified (this is the system doing the certifying)
Version systemVersion = controller.readSystemVersion();
- return controller.curator().readCertifiedOsVersions().stream()
- .anyMatch(certifiedOsVersion -> certifiedOsVersion.osVersion().equals(osVersion) &&
- // A later system version is fine, as we don't guarantee that
- // an OS upgrade will always coincide with a Vespa release
- !certifiedOsVersion.vespaVersion().isAfter(systemVersion));
+ return readCertified().stream()
+ .anyMatch(certifiedOsVersion -> certifiedOsVersion.osVersion().equals(osVersion) &&
+ // A later system version is fine, as we don't guarantee that
+ // an OS upgrade will always coincide with a Vespa release
+ !certifiedOsVersion.vespaVersion().isAfter(systemVersion));
+ }
+
+ /** Returns all certified versions */
+ public Set<CertifiedOsVersion> readCertified() {
+ return controller.curator().readCertifiedOsVersions();
}
private void requireCloud(CloudName cloud) {
@@ -187,6 +192,12 @@ public record OsController(Controller controller) {
}
}
+ private void requireNonEmpty(Version version) {
+ if (version.isEmpty()) {
+ throw new IllegalArgumentException("Invalid version '" + version.toFullString() + "'");
+ }
+ }
+
private CuratorDb curator() {
return controller.curator();
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/RoutingController.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/RoutingController.java
index 3d550973f22..bceef3fd96f 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/RoutingController.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/RoutingController.java
@@ -4,7 +4,6 @@ package com.yahoo.vespa.hosted.controller;
import com.google.common.hash.HashCode;
import com.google.common.hash.Hashing;
import com.google.common.io.BaseEncoding;
-import com.yahoo.config.application.api.DeploymentInstanceSpec;
import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.ClusterSpec;
@@ -158,15 +157,6 @@ public class RoutingController {
DeploymentSpec deploymentSpec = application.deploymentSpec();
for (var spec : deploymentSpec.instances()) {
ApplicationId instance = application.id().instance(spec.name());
- // Add endpoint declared with legacy syntax
- spec.globalServiceId().ifPresent(clusterId -> {
- List<DeploymentId> deployments = spec.zones().stream()
- .filter(zone -> zone.concerns(Environment.prod))
- .map(zone -> new DeploymentId(instance, ZoneId.from(Environment.prod, zone.region().get())))
- .toList();
- RoutingId routingId = RoutingId.of(instance, EndpointId.defaultId());
- endpoints.addAll(computeGlobalEndpoints(routingId, ClusterSpec.Id.from(clusterId), deployments, generatedEndpoints));
- });
// Add endpoints declared with current syntax
spec.endpoints().forEach(declaredEndpoint -> {
RoutingId routingId = RoutingId.of(instance, EndpointId.of(declaredEndpoint.endpointId()));
@@ -275,7 +265,6 @@ public class RoutingController {
}
// Add endpoints backed by a rotation, and register them in DNS if necessary
- boolean registerLegacyNames = requiresLegacyNames(application.get().deploymentSpec(), instanceName);
Instance instance = application.get().require(instanceName);
Set<ContainerEndpoint> containerEndpoints = new HashSet<>();
DeploymentId deployment = new DeploymentId(instance.id(), zone);
@@ -285,16 +274,11 @@ public class RoutingController {
EndpointList rotationEndpoints = globalEndpoints.named(assignedRotation.endpointId(), Scope.global)
.requiresRotation();
- // Skip rotations which do not apply to this zone. Legacy names always point to all zones
- if (!registerLegacyNames && !assignedRotation.regions().contains(zone.region())) {
+ // Skip rotations which do not apply to this zone
+ if (!assignedRotation.regions().contains(zone.region())) {
continue;
}
- // Omit legacy DNS names when assigning rotations using <endpoints/> syntax
- if (!registerLegacyNames) {
- rotationEndpoints = rotationEndpoints.not().legacy();
- }
-
// Register names in DNS
Rotation rotation = rotationRepository.requireRotation(assignedRotation.rotationId());
for (var endpoint : rotationEndpoints) {
@@ -480,13 +464,6 @@ public class RoutingController {
return randomizedEndpoints.with(FetchVector.Dimension.APPLICATION_ID, instance.serializedForm()).value();
}
- /** Whether legacy global DNS names should be available for given application */
- private static boolean requiresLegacyNames(DeploymentSpec deploymentSpec, InstanceName instanceName) {
- return deploymentSpec.instance(instanceName)
- .flatMap(DeploymentInstanceSpec::globalServiceId)
- .isPresent();
- }
-
/** Create a common name based on a hash of given application. This must be less than 64 characters long. */
private static String commonNameHashOf(ApplicationId application, SystemName system) {
@SuppressWarnings("deprecation") // for Hashing.sha1()
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 186e6838a71..0c05d710763 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
@@ -21,8 +21,6 @@ import com.yahoo.config.provision.zone.ZoneApi;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.Controller;
-import com.yahoo.vespa.hosted.controller.application.EndpointId;
-import com.yahoo.vespa.hosted.controller.versions.VespaVersion;
import java.time.Instant;
import java.util.ArrayList;
@@ -31,7 +29,6 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
@@ -179,9 +176,10 @@ public class ApplicationPackageValidator {
if (applicationPackage.validationOverrides().allows(validationId, instant)) return;
var endpoints = application.deploymentSpec().instance(instanceName)
- .map(ApplicationPackageValidator::allEndpointsOf)
+ .map(deploymentInstanceSpec1 -> deploymentInstanceSpec1.endpoints())
.orElseGet(List::of);
- var newEndpoints = allEndpointsOf(applicationPackage.deploymentSpec().requireInstance(instanceName));
+ DeploymentInstanceSpec deploymentInstanceSpec = applicationPackage.deploymentSpec().requireInstance(instanceName);
+ var newEndpoints = new ArrayList<>(deploymentInstanceSpec.endpoints());
if (newEndpoints.containsAll(endpoints)) return; // Adding new endpoints is fine
if (containsAllDestinationsOf(endpoints, newEndpoints)) return; // Adding destinations is fine
@@ -255,26 +253,6 @@ public class ApplicationPackageValidator {
return containsAllRegions && hasSameCluster;
}
- /** Returns all configued endpoints of given deployment instance spec */
- private static List<Endpoint> allEndpointsOf(DeploymentInstanceSpec deploymentInstanceSpec) {
- var endpoints = new ArrayList<>(deploymentInstanceSpec.endpoints());
- legacyEndpoint(deploymentInstanceSpec).ifPresent(endpoints::add);
- return endpoints;
- }
-
- /** Returns global service ID as an endpoint, if any global service ID is set */
- private static Optional<Endpoint> legacyEndpoint(DeploymentInstanceSpec instance) {
- return instance.globalServiceId().map(globalServiceId -> {
- var targets = instance.zones().stream()
- .filter(zone -> zone.environment().isProduction())
- .flatMap(zone -> zone.region().stream())
- .distinct()
- .map(region -> new Endpoint.Target(region, instance.name(), 1))
- .toList();
- return new Endpoint(EndpointId.defaultId().id(), globalServiceId, Endpoint.Level.instance, targets);
- });
- }
-
/** Returns a list of the non-compactable IDs of given instance and endpoint */
private static List<String> nonCompactableIds(InstanceName instance, Endpoint endpoint) {
List<String> ids = new ArrayList<>(2);
@@ -287,16 +265,6 @@ public class ApplicationPackageValidator {
return ids;
}
- private static class InstanceEndpoint {
-
- private final InstanceName instance;
- private final String endpointId;
-
- public InstanceEndpoint(InstanceName instance, String endpointId) {
- this.instance = instance;
- this.endpointId = endpointId;
- }
-
- }
+ private record InstanceEndpoint(InstanceName instance, String endpointId) {}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporter.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporter.java
index 98de84216e0..71f9c37577a 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporter.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporter.java
@@ -340,7 +340,7 @@ public class MetricsReporter extends ControllerMaintainer {
}
private static Map<String, String> dimensions(ApplicationId application) {
- return Map.of("tenant", application.tenant().value(),
+ return Map.of("tenantName", application.tenant().value(),
"app", application.application().value() + "." + application.instance().value(),
"applicationId", application.toFullString());
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeScheduler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeScheduler.java
index 5701a495641..c751c0a130b 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeScheduler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeScheduler.java
@@ -43,9 +43,8 @@ public class OsUpgradeScheduler extends ControllerMaintainer {
int attempts = 0;
int failures = 0;
for (var cloud : controller().clouds()) {
- Optional<Change> change = changeIn(cloud, now);
+ Optional<Change> change = changeIn(cloud, now, false);
if (change.isEmpty()) continue;
- if (!change.get().scheduleAt(now)) continue;
try {
attempts++;
controller().os().upgradeTo(change.get().osVersion().version(), cloud, false, false);
@@ -58,15 +57,24 @@ public class OsUpgradeScheduler extends ControllerMaintainer {
return asSuccessFactorDeviation(attempts, failures);
}
- /** Returns the wanted change for cloud at given instant, if any */
- public Optional<Change> changeIn(CloudName cloud, Instant instant) {
+ /**
+ * Returns the next OS version change
+ *
+ * @param cloud The cloud where the change will be deployed
+ * @param now Current time
+ * @param future Whether to return a change that cannot be scheduled now
+ */
+ public Optional<Change> changeIn(CloudName cloud, Instant now, boolean future) {
Optional<OsVersionTarget> currentTarget = controller().os().target(cloud);
if (currentTarget.isEmpty()) return Optional.empty();
if (upgradingToNewMajor(cloud)) return Optional.empty(); // Skip further upgrades until major version upgrade is complete
- Release release = releaseIn(cloud);
- Optional<Change> change = release.change(currentTarget.get().version(), instant);
- return change.filter(this::certified);
+ Version currentVersion = currentTarget.get().version();
+ Change change = releaseIn(cloud).change(currentVersion, now);
+ if (!change.osVersion().version().isAfter(currentVersion)) return Optional.empty();
+ if (!future && !change.scheduleAt(now)) return Optional.empty();
+ if (!certified(change)) return Optional.empty();
+ return Optional.of(change);
}
private boolean certified(Change change) {
@@ -121,8 +129,8 @@ public class OsUpgradeScheduler extends ControllerMaintainer {
private interface Release {
- /** The pending change for this release at given instant, if any */
- Optional<Change> change(Version currentVersion, Instant instant);
+ /** The next available change of this release at given instant */
+ Change change(Version currentVersion, Instant instant);
}
@@ -151,12 +159,11 @@ public class OsUpgradeScheduler extends ControllerMaintainer {
}
@Override
- public Optional<Change> change(Version currentVersion, Instant instant) {
+ public Change change(Version currentVersion, Instant instant) {
OsRelease release = artifactRepository.osRelease(currentVersion.getMajor(), tag());
- if (!release.version().isAfter(currentVersion)) return Optional.empty();
Duration cooldown = remainingCooldownOf(cooldown(), release.age(instant));
Instant scheduleAt = schedulingInstant(instant.plus(cooldown), system);
- return Optional.of(new Change(new OsVersion(release.version(), cloud), scheduleAt));
+ return new Change(new OsVersion(release.version(), cloud), scheduleAt);
}
/** Returns the release tag tracked by this system */
@@ -185,25 +192,24 @@ public class OsUpgradeScheduler extends ControllerMaintainer {
/** The day of week new releases are published */
private static final DayOfWeek RELEASE_DAY = DayOfWeek.TUESDAY;
+ /** How far into release day we should wait before triggering. This is to give the new release some time to propagate */
+ private static final Duration COOLDOWN = Duration.ofHours(6);
+
public CalendarVersionedRelease {
Objects.requireNonNull(system);
}
@Override
- public Optional<Change> change(Version currentVersion, Instant instant) {
+ public Change change(Version currentVersion, Instant instant) {
CalendarVersion version = findVersion(instant, currentVersion);
- Instant predicatedInstant = instant;
+ Instant predicted = instant;
while (!version.version().isAfter(currentVersion)) {
- predicatedInstant = predicatedInstant.plus(Duration.ofDays(1));
- version = findVersion(predicatedInstant, currentVersion);
+ predicted = predicted.plus(Duration.ofDays(1));
+ version = findVersion(predicted, currentVersion);
}
- Duration cooldown = remainingCooldownOf(cooldown(), version.age(instant));
+ Duration cooldown = remainingCooldownOf(COOLDOWN, version.age(instant));
Instant schedulingInstant = schedulingInstant(instant.plus(cooldown), system);
- return Optional.of(new Change(new OsVersion(version.version(), cloud), schedulingInstant));
- }
-
- private Duration cooldown() {
- return Duration.ofDays(1); // Give new releases some time to propagate
+ return new Change(new OsVersion(version.version(), cloud), schedulingInstant);
}
/** Find the most recent version available according to the scheduling step, relative to now */
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiHandler.java
index fe1fb979abc..1639424b182 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiHandler.java
@@ -86,6 +86,7 @@ public class OsApiHandler extends AuditLoggingRequestHandler {
private HttpResponse get(HttpRequest request) {
Path path = new Path(request.getUri());
if (path.matches("/os/v1/")) return new SlimeJsonResponse(osVersions());
+ if (path.matches("/os/v1/certify")) return new SlimeJsonResponse(certifiedOsVersions());
return ErrorResponse.notFoundError("Nothing at " + path);
}
@@ -110,7 +111,11 @@ public class OsApiHandler extends AuditLoggingRequestHandler {
private HttpResponse certifyVersion(HttpRequest request, String versionString, String cloudName) {
Version version = Version.fromString(versionString);
CloudName cloud = CloudName.from(cloudName);
- Version vespaVersion = Version.fromString(asString(request.getData()));
+ String vespaVersionString = asString(request.getData());
+ if (vespaVersionString.isEmpty()) {
+ throw new IllegalArgumentException("Missing Vespa version in request body");
+ }
+ Version vespaVersion = Version.fromString(vespaVersionString);
CertifiedOsVersion certified = controller.os().certify(version, cloud, vespaVersion);
if (certified.vespaVersion().equals(vespaVersion)) {
return new MessageResponse("Certified " + version.toFullString() + " in cloud " + cloud +
@@ -177,6 +182,18 @@ public class OsApiHandler extends AuditLoggingRequestHandler {
target.toFullString() + (pin ? " (pinned)" : ""));
}
+ private Slime certifiedOsVersions() {
+ Slime slime = new Slime();
+ Cursor array = slime.setArray();
+ controller.os().readCertified().stream().sorted().forEach(cv -> {
+ Cursor object = array.addObject();
+ object.setString("version", cv.osVersion().version().toFullString());
+ object.setString("cloud", cv.osVersion().cloud().value());
+ object.setString("vespaVersion", cv.vespaVersion().toFullString());
+ });
+ return slime;
+ }
+
private Slime osVersions() {
Slime slime = new Slime();
Cursor root = slime.setObject();
@@ -193,7 +210,7 @@ public class OsApiHandler extends AuditLoggingRequestHandler {
currentVersionObject.setString("upgradeBudget", Duration.ZERO.toString());
currentVersionObject.setLong("scheduledAt", t.scheduledAt().toEpochMilli());
currentVersionObject.setBool("pinned", t.pinned());
- Optional<Change> nextChange = osUpgradeScheduler.changeIn(t.osVersion().cloud(), now);
+ Optional<Change> nextChange = osUpgradeScheduler.changeIn(t.osVersion().cloud(), now, true);
nextChange.ifPresent(c -> {
currentVersionObject.setString("nextVersion", c.osVersion().version().toFullString());
currentVersionObject.setLong("nextScheduledAt", c.scheduleAt().toEpochMilli());
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicies.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicies.java
index a309afcd039..c8c3d057ee3 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicies.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/RoutingPolicies.java
@@ -112,21 +112,20 @@ public class RoutingPolicies {
List<LoadBalancer> loadBalancers = controller.serviceRegistry().configServer()
.getLoadBalancers(instance, deployment.zoneId());
LoadBalancerAllocation allocation = new LoadBalancerAllocation(deployment, deploymentSpec, loadBalancers);
- Set<ZoneId> inactiveZones = inactiveZones(instance, deploymentSpec);
Optional<TenantAndApplicationId> owner = ownerOf(allocation);
try (var lock = db.lockRoutingPolicies()) {
RoutingPolicyList applicationPolicies = read(TenantAndApplicationId.from(instance));
RoutingPolicyList deploymentPolicies = applicationPolicies.deployment(allocation.deployment);
- removeGlobalDnsUnreferencedBy(allocation, deploymentPolicies, inactiveZones, lock);
+ removeGlobalDnsUnreferencedBy(allocation, deploymentPolicies, lock);
removeApplicationDnsUnreferencedBy(allocation, deploymentPolicies, lock);
RoutingPolicyList instancePolicies = storePoliciesOf(allocation, applicationPolicies, generatedEndpoints, lock);
instancePolicies = removePoliciesUnreferencedBy(allocation, instancePolicies, lock);
RoutingPolicyList updatedApplicationPolicies = applicationPolicies.replace(instance, instancePolicies);
- updateGlobalDnsOf(instancePolicies, Optional.of(deployment), inactiveZones, owner, lock);
- updateApplicationDnsOf(updatedApplicationPolicies, inactiveZones, deployment, owner, lock);
+ updateGlobalDnsOf(instancePolicies, Optional.of(deployment), owner, lock);
+ updateApplicationDnsOf(updatedApplicationPolicies, deployment, owner, lock);
}
}
@@ -137,7 +136,7 @@ public class RoutingPolicies {
controller.clock().instant())));
Map<ApplicationId, RoutingPolicyList> allPolicies = readAll().groupingBy(policy -> policy.id().owner());
allPolicies.forEach((instance, policies) -> {
- updateGlobalDnsOf(policies, Optional.empty(), Set.of(), Optional.of(TenantAndApplicationId.from(instance)), lock);
+ updateGlobalDnsOf(policies, Optional.empty(), Optional.of(TenantAndApplicationId.from(instance)), lock);
});
}
}
@@ -157,17 +156,16 @@ public class RoutingPolicies {
Map<ApplicationId, RoutingPolicyList> policiesByInstance = effectivePolicies.groupingBy(policy -> policy.id().owner());
policiesByInstance.forEach((ignored, instancePolicies) -> updateGlobalDnsOf(instancePolicies,
Optional.of(deployment),
- Set.of(),
ownerOf(deployment),
lock));
- updateApplicationDnsOf(effectivePolicies, Set.of(), deployment, ownerOf(deployment), lock);
+ updateApplicationDnsOf(effectivePolicies, deployment, ownerOf(deployment), lock);
policiesByInstance.forEach((owner, instancePolicies) -> db.writeRoutingPolicies(owner, instancePolicies.asList()));
}
}
/** Update global DNS records for given policies */
private void updateGlobalDnsOf(RoutingPolicyList instancePolicies, Optional<DeploymentId> deployment,
- Set<ZoneId> inactiveZones, Optional<TenantAndApplicationId> owner,
+ Optional<TenantAndApplicationId> owner,
@SuppressWarnings("unused") Mutex lock) {
Map<RoutingId, List<RoutingPolicy>> routingTable = instancePolicies.asInstanceRoutingTable();
for (Map.Entry<RoutingId, List<RoutingPolicy>> routeEntry : routingTable.entrySet()) {
@@ -175,17 +173,17 @@ public class RoutingPolicies {
controller.routing().readDeclaredEndpointsOf(routingId.instance())
.named(routingId.endpointId(), Endpoint.Scope.global)
.not().requiresRotation()
- .forEach(endpoint -> updateGlobalDnsOf(endpoint, inactiveZones, routeEntry.getValue(), deployment, owner));
+ .forEach(endpoint -> updateGlobalDnsOf(endpoint, routeEntry.getValue(), deployment, owner));
}
}
/** Update global DNS records for given global endpoint */
- private void updateGlobalDnsOf(Endpoint endpoint, Set<ZoneId> inactiveZones, List<RoutingPolicy> policies,
+ private void updateGlobalDnsOf(Endpoint endpoint, List<RoutingPolicy> policies,
Optional<DeploymentId> deployment, Optional<TenantAndApplicationId> owner) {
if (endpoint.scope() != Endpoint.Scope.global) throw new IllegalArgumentException("Endpoint " + endpoint + " is not global");
if (deployment.isPresent() && !endpoint.deployments().contains(deployment.get())) return;
- Collection<RegionEndpoint> regionEndpoints = computeRegionEndpoints(endpoint, policies, inactiveZones);
+ Collection<RegionEndpoint> regionEndpoints = computeRegionEndpoints(endpoint, policies);
Set<AliasTarget> latencyTargets = new LinkedHashSet<>();
Set<AliasTarget> inactiveLatencyTargets = new LinkedHashSet<>();
for (var regionEndpoint : regionEndpoints) {
@@ -237,7 +235,7 @@ public class RoutingPolicies {
}
/** Compute region endpoints and their targets from given policies */
- private Collection<RegionEndpoint> computeRegionEndpoints(Endpoint parent, List<RoutingPolicy> policies, Set<ZoneId> inactiveZones) {
+ private Collection<RegionEndpoint> computeRegionEndpoints(Endpoint parent, List<RoutingPolicy> policies) {
if (!parent.scope().multiDeployment()) {
throw new IllegalArgumentException(parent + " has unexpected scope");
}
@@ -248,7 +246,7 @@ public class RoutingPolicies {
Endpoint endpoint = policy.regionEndpointIn(controller.system(), RoutingMethod.exclusive, parent.generated());
var zonePolicy = db.readZoneRoutingPolicy(policy.id().zone());
long weight = 1;
- if (isConfiguredOut(zonePolicy, policy, inactiveZones)) {
+ if (isConfiguredOut(zonePolicy, policy)) {
weight = 0; // A record with 0 weight will not receive traffic. If all records within a group have 0
// weight, traffic is routed to all records with equal probability.
}
@@ -270,9 +268,8 @@ public class RoutingPolicies {
}
- private void updateApplicationDnsOf(RoutingPolicyList routingPolicies, Set<ZoneId> inactiveZones,
- DeploymentId deployment, Optional<TenantAndApplicationId> owner,
- @SuppressWarnings("unused") Mutex lock) {
+ private void updateApplicationDnsOf(RoutingPolicyList routingPolicies, DeploymentId deployment,
+ Optional<TenantAndApplicationId> owner, @SuppressWarnings("unused") Mutex lock) {
// In the context of single deployment (which this is) there is only one routing policy per routing ID. I.e.
// there is no scenario where more than one deployment within an instance can be a member the same
// application-level endpoint. However, to allow this in the future the routing table remains
@@ -297,7 +294,7 @@ public class RoutingPolicies {
Set<Target> activeTargets = targetsByEndpoint.computeIfAbsent(endpoint, (k) -> new LinkedHashSet<>());
Set<Target> inactiveTargets = inactiveTargetsByEndpoint.computeIfAbsent(endpoint, (k) -> new LinkedHashSet<>());
- if (isConfiguredOut(zonePolicy, policy, inactiveZones)) {
+ if (isConfiguredOut(zonePolicy, policy)) {
inactiveTargets.add(Target.weighted(policy, target));
} else {
activeTargets.add(Target.weighted(policy, target));
@@ -489,7 +486,7 @@ public class RoutingPolicies {
}
/** Remove unreferenced instance endpoints from DNS */
- private void removeGlobalDnsUnreferencedBy(LoadBalancerAllocation allocation, RoutingPolicyList deploymentPolicies, Set<ZoneId> inactiveZones, @SuppressWarnings("unused") Mutex lock) {
+ private void removeGlobalDnsUnreferencedBy(LoadBalancerAllocation allocation, RoutingPolicyList deploymentPolicies, @SuppressWarnings("unused") Mutex lock) {
Set<RoutingId> removalCandidates = new HashSet<>(deploymentPolicies.asInstanceRoutingTable().keySet());
Set<RoutingId> activeRoutingIds = instanceRoutingIds(allocation);
removalCandidates.removeAll(activeRoutingIds);
@@ -500,7 +497,7 @@ public class RoutingPolicies {
// This removes all ALIAS records having this DNS name. There is no attempt to delete only the entry for the
// affected zone. Instead, the correct set of records is (re)created by updateGlobalDnsOf
for (var endpoint : endpoints) {
- for (var regionEndpoint : computeRegionEndpoints(endpoint, deploymentPolicies.asList(), inactiveZones)) {
+ for (var regionEndpoint : computeRegionEndpoints(endpoint, deploymentPolicies.asList())) {
Record.Type type = regionEndpoint.zoneDirectTargets().isEmpty() ? Record.Type.ALIAS : Record.Type.DIRECT;
controller.nameServiceForwarder().removeRecords(type,
RecordName.from(regionEndpoint.target().name().value()),
@@ -571,14 +568,12 @@ public class RoutingPolicies {
}
/** Returns whether the endpoints of given policy are configured {@link RoutingStatus.Value#out} */
- private static boolean isConfiguredOut(ZoneRoutingPolicy zonePolicy, RoutingPolicy policy, Set<ZoneId> inactiveZones) {
+ private static boolean isConfiguredOut(ZoneRoutingPolicy zonePolicy, RoutingPolicy policy) {
// A deployment can be configured out from endpoints at any of the following levels:
// - zone level (ZoneRoutingPolicy)
// - deployment level (RoutingPolicy)
- // - application package level (deployment.xml)
return zonePolicy.routingStatus().value() == RoutingStatus.Value.out ||
- policy.routingStatus().value() == RoutingStatus.Value.out ||
- inactiveZones.contains(policy.id().zone());
+ policy.routingStatus().value() == RoutingStatus.Value.out;
}
/** Represents records for a region-wide endpoint */
@@ -658,10 +653,6 @@ public class RoutingPolicies {
if (instanceSpec.isEmpty()) {
return Set.of();
}
- if (instanceSpec.get().globalServiceId().filter(id -> id.equals(loadBalancer.cluster().value())).isPresent()) {
- // Legacy assignment always has the default endpoint ID
- return Set.of(EndpointId.defaultId());
- }
return instanceSpec.get().endpoints().stream()
.filter(endpoint -> endpoint.containerId().equals(loadBalancer.cluster().value()))
.filter(endpoint -> endpoint.regions().contains(deployment.zoneId().region()))
@@ -687,17 +678,6 @@ public class RoutingPolicies {
}
- /** Returns zones where global routing is declared inactive for instance through deploymentSpec */
- private static Set<ZoneId> inactiveZones(ApplicationId instance, DeploymentSpec deploymentSpec) {
- var instanceSpec = deploymentSpec.instance(instance.instance());
- if (instanceSpec.isEmpty()) return Set.of();
- return instanceSpec.get().zones().stream()
- .filter(zone -> zone.environment().isProduction())
- .filter(zone -> !zone.active())
- .map(zone -> ZoneId.from(zone.environment(), zone.region().get()))
- .collect(Collectors.toUnmodifiableSet());
- }
-
/** Returns the name updater to use for given endpoint */
private NameServiceForwarder nameServiceForwarder(Endpoint endpoint) {
return switch (endpoint.routingMethod()) {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepository.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepository.java
index 1e9f11e0349..d54bdead0bd 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepository.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/routing/rotation/RotationRepository.java
@@ -1,7 +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.routing.rotation;
-import com.yahoo.config.application.api.DeploymentInstanceSpec;
import com.yahoo.config.application.api.DeploymentSpec;
import com.yahoo.config.application.api.Endpoint;
import com.yahoo.config.provision.ApplicationId;
@@ -63,44 +62,10 @@ public class RotationRepository {
}
/**
- * Returns a single rotation for the given application. This is only used when a rotation is assigned through the
- * use of a global service ID.
- *
- * If a rotation is already assigned to the application, that rotation will be returned.
- * If no rotation is assigned, return an available rotation. The caller is responsible for assigning the rotation.
- *
- * @param instanceSpec the instance deployment spec
- * @param instance the instance requesting a rotation
- * @param lock lock which must be acquired by the caller
- */
- private AssignedRotation assignRotationTo(String globalServiceId, DeploymentInstanceSpec instanceSpec,
- Instance instance, RotationLock lock) {
- RotationId rotation;
- if (instance.rotations().isEmpty()) {
- rotation = findAvailableRotation(instance.id(), lock).id();
- } else {
- rotation = instance.rotations().get(0).rotationId();
- }
- var productionRegions = instanceSpec.zones().stream()
- .filter(zone -> zone.environment().isProduction())
- .flatMap(zone -> zone.region().stream())
- .collect(Collectors.toSet());
- if (productionRegions.size() < 2) {
- throw new IllegalArgumentException("global-service-id is set but less than 2 prod zones are defined " +
- "in instance '" + instance.name() + "'");
- }
- return new AssignedRotation(new ClusterSpec.Id(globalServiceId),
- EndpointId.defaultId(),
- rotation,
- productionRegions);
- }
-
- /**
* Returns rotation assignments for all endpoints in application.
*
* If rotations are already assigned, these will be returned.
* If rotations are not assigned, a new assignment will be created taking new rotations from the repository.
- * This method supports both global-service-id as well as the new endpoints tag.
*
* @param deploymentSpec The deployment spec of the application
* @param instance The application requesting rotations
@@ -112,19 +77,7 @@ public class RotationRepository {
if (allRotations.isEmpty()) {
return List.of();
}
-
- // Only allow one kind of configuration syntax
var instanceSpec = deploymentSpec.requireInstance(instance.name());
- if ( instanceSpec.globalServiceId().isPresent()
- && ! instanceSpec.endpoints().isEmpty()) {
- throw new IllegalArgumentException("Cannot provision rotations with both global-service-id and 'endpoints'");
- }
-
- // Support legacy global-service-id
- if (instanceSpec.globalServiceId().isPresent()) {
- return List.of(assignRotationTo(instanceSpec.globalServiceId().get(), instanceSpec, instance, lock));
- }
-
return assignRotationsTo(instanceSpec.endpoints(), instance, lock);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/tls/ControllerSslContextFactoryProvider.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/tls/ControllerSslContextFactoryProvider.java
index 9bf0f8dcde2..286d8c997fd 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/tls/ControllerSslContextFactoryProvider.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/tls/ControllerSslContextFactoryProvider.java
@@ -1,6 +1,7 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.controller.tls;
+import com.google.common.collect.Sets;
import com.yahoo.component.annotation.Inject;
import com.yahoo.container.jdisc.secretstore.SecretStore;
import com.yahoo.jdisc.http.ssl.impl.TlsContextBasedProvider;
@@ -19,6 +20,7 @@ import java.nio.file.Paths;
import java.security.KeyStore;
import java.security.PrivateKey;
import java.security.cert.X509Certificate;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@@ -45,9 +47,12 @@ public class ControllerSslContextFactoryProvider extends TlsContextBasedProvider
this.truststore = KeyStoreBuilder.withType(KeyStoreType.JKS)
.fromFile(Paths.get(config.caTrustStore()))
.build();
+
+ TlsCredentials tlsCredentials = latestValidCredentials(secretStore, config);
+
// Key store containing key pair from secret store
this.keystore = KeyStoreBuilder.withType(KeyStoreType.JKS)
- .withKeyEntry(getClass().getSimpleName(), privateKey(secretStore, config), certificates(secretStore, config))
+ .withKeyEntry(getClass().getSimpleName(), tlsCredentials.privateKey, tlsCredentials.certificates)
.build();
}
@@ -65,17 +70,32 @@ public class ControllerSslContextFactoryProvider extends TlsContextBasedProvider
port != 443 ? PeerAuthentication.WANT : PeerAuthentication.DISABLED);
}
+ record TlsCredentials(List<X509Certificate> certificates, PrivateKey privateKey){}
+
+ private static TlsCredentials latestValidCredentials(SecretStore secretStore, TlsConfig tlsConfig) {
+ int version = latestVersionInSecretStore(secretStore, tlsConfig);
+ return new TlsCredentials(certificates(secretStore, tlsConfig, version), privateKey(secretStore, tlsConfig, version));
+ }
+
+ private static int latestVersionInSecretStore(SecretStore secretStore, TlsConfig tlsConfig) {
+ var certVersions = new HashSet<>(secretStore.listSecretVersions(tlsConfig.certificateSecret()));
+ var keyVersions = new HashSet<>(secretStore.listSecretVersions(tlsConfig.privateKeySecret()));
+ return Sets.intersection(certVersions, keyVersions).stream().mapToInt(Integer::intValue).max().orElseThrow(
+ () -> new RuntimeException("No valid certificate versions found in secret store!")
+ );
+ }
+
/** Get private key from secret store **/
- private static PrivateKey privateKey(SecretStore secretStore, TlsConfig config) {
- return KeyUtils.fromPemEncodedPrivateKey(secretStore.getSecret(config.privateKeySecret()));
+ private static PrivateKey privateKey(SecretStore secretStore, TlsConfig config, int version) {
+ return KeyUtils.fromPemEncodedPrivateKey(secretStore.getSecret(config.privateKeySecret(), version));
}
/**
* Get certificate from secret store. If certificate secret contains multiple certificates, e.g. intermediate
* certificates, the entire chain will be read
*/
- private static List<X509Certificate> certificates(SecretStore secretStore, TlsConfig config) {
- return X509CertificateUtils.certificateListFromPem(secretStore.getSecret(config.certificateSecret()));
+ private static List<X509Certificate> certificates(SecretStore secretStore, TlsConfig config, int version) {
+ return X509CertificateUtils.certificateListFromPem(secretStore.getSecret(config.certificateSecret(), version));
}
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/CertifiedOsVersion.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/CertifiedOsVersion.java
index be99e170f03..9402165f112 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/CertifiedOsVersion.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/versions/CertifiedOsVersion.java
@@ -2,6 +2,7 @@ package com.yahoo.vespa.hosted.controller.versions;
import com.yahoo.component.Version;
+import java.util.Comparator;
import java.util.Objects;
/**
@@ -9,11 +10,19 @@ import java.util.Objects;
*
* @author mpolden
*/
-public record CertifiedOsVersion(OsVersion osVersion, Version vespaVersion) {
+public record CertifiedOsVersion(OsVersion osVersion, Version vespaVersion) implements Comparable<CertifiedOsVersion> {
+
+ private static final Comparator<CertifiedOsVersion> comparator = Comparator.comparing(CertifiedOsVersion::osVersion)
+ .thenComparing(CertifiedOsVersion::vespaVersion);
public CertifiedOsVersion {
Objects.requireNonNull(osVersion);
Objects.requireNonNull(vespaVersion);
}
+ @Override
+ public int compareTo(CertifiedOsVersion that) {
+ return comparator.compare(this, that);
+ }
+
}
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 3d2a66adc81..c46a28c4567 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
@@ -384,7 +384,7 @@ public class ControllerTest {
void testDnsUpdatesForGlobalEndpointLegacySyntax() {
var context = tester.newDeploymentContext("tenant1", "app1", "default");
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("us-west-1")
.region("us-central-1") // Two deployments should result in each DNS alias being registered once
.build();
@@ -1532,22 +1532,6 @@ public class ControllerTest {
}
@Test
- void testSubmitWithElementDeprecatedOnPreviousMajor() {
- DeploymentContext context = tester.newDeploymentContext();
- var applicationPackage = new ApplicationPackageBuilder()
- .compileVersion(Version.fromString("8.1"))
- .region("us-west-1")
- .globalServiceId("qrs")
- .build();
- try {
- context.submit(applicationPackage).deploy();
- fail("Expected exception");
- } catch (IllegalArgumentException e) {
- assertTrue(e.getMessage().contains("Element 'prod' contains attribute 'global-service-id' deprecated since major version 7"));
- }
- }
-
- @Test
void testDeactivateDeploymentUnknownByController() {
DeploymentContext context = tester.newDeploymentContext();
DeploymentId deployment = context.deploymentIdIn(ZoneId.from("prod", "us-west-1"));
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 965201ec6da..fb3026e1d80 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
@@ -65,7 +65,6 @@ public class ApplicationPackageBuilder {
private String revisionTarget = "latest";
private String revisionChange = "always";
private String upgradeRollout = null;
- private String globalServiceId = null;
private String athenzIdentityAttributes = "athenz-domain='domain' athenz-service='service'";
private String searchDefinition = "search test { }";
private Version compileVersion = Version.fromString("6.1");
@@ -101,11 +100,6 @@ public class ApplicationPackageBuilder {
return this;
}
- public ApplicationPackageBuilder globalServiceId(String globalServiceId) {
- this.globalServiceId = globalServiceId;
- return this;
- }
-
public ApplicationPackageBuilder endpoint(String id, String containerId, String... regions) {
endpointsBody.append(" <endpoint");
endpointsBody.append(" id='").append(id).append("'");
@@ -162,15 +156,8 @@ public class ApplicationPackageBuilder {
return this;
}
- public ApplicationPackageBuilder region(RegionName regionName) {
- return region(regionName, true);
- }
-
public ApplicationPackageBuilder region(String regionName) {
- prodBody.append(" <region>")
- .append(regionName)
- .append("</region>\n");
- return this;
+ return region(RegionName.from(regionName));
}
public ApplicationPackageBuilder region(String regionName, String cloudAccount) {
@@ -187,10 +174,8 @@ public class ApplicationPackageBuilder {
return this;
}
- public ApplicationPackageBuilder region(RegionName regionName, boolean active) {
- prodBody.append(" <region active='")
- .append(active)
- .append("'>")
+ public ApplicationPackageBuilder region(RegionName regionName) {
+ prodBody.append(" <region>")
.append(regionName.value())
.append("</region>\n");
return this;
@@ -335,13 +320,7 @@ public class ApplicationPackageBuilder {
xml.append(" />\n");
});
xml.append(blockChange);
- xml.append(" <prod");
- if (globalServiceId != null) {
- xml.append(" global-service-id='");
- xml.append(globalServiceId);
- xml.append("'");
- }
- xml.append(">\n");
+ xml.append(" <prod>\n");
xml.append(prodBody);
xml.append(" </prod>\n");
if (endpointsBody.length() > 0) {
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 e5a10bb5889..3fb4a040e0d 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
@@ -1312,19 +1312,19 @@ public class DeploymentTriggerTest {
<staging />
<prod>
<parallel>
- <region active='true'>us-west-1</region>
+ <region>us-west-1</region>
<steps>
- <region active='true'>us-east-3</region>
+ <region>us-east-3</region>
<delay hours='2' />
- <region active='true'>eu-west-1</region>
+ <region>eu-west-1</region>
<delay hours='2' />
</steps>
<steps>
<delay hours='3' />
- <region active='true'>us-central-1</region>
+ <region>us-central-1</region>
<parallel>
- <region active='true' athenz-service='no-service'>ap-northeast-1</region>
- <region active='true'>ap-northeast-2</region>
+ <region athenz-service='no-service'>ap-northeast-1</region>
+ <region>ap-northeast-2</region>
<test>us-central-1</test>
</parallel>
</steps>
@@ -1335,7 +1335,7 @@ public class DeploymentTriggerTest {
<test>ap-northeast-1</test>
</parallel>
<test>us-east-3</test>
- <region active='true'>ap-southeast-1</region>
+ <region>ap-southeast-1</region>
</prod>
<endpoints>
<endpoint id='foo' container-id='bar'>
@@ -1350,7 +1350,7 @@ public class DeploymentTriggerTest {
<test />
<block-change revision='true' version='false' days='sat' hours='0-23' time-zone='CET' />
<prod>
- <region active='true'>eu-west-1</region>
+ <region>eu-west-1</region>
<test>eu-west-1</test>
</prod>
<notifications when='failing'>
@@ -1363,7 +1363,7 @@ public class DeploymentTriggerTest {
<instance id='last'>
<upgrade policy='conservative' />
<prod>
- <region active='true'>eu-west-1</region>
+ <region>eu-west-1</region>
</prod>
</instance>
</deployment>
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java
index 53f2e85ad31..593d788fd7d 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/MetricsReporterTest.java
@@ -275,7 +275,7 @@ public class MetricsReporterTest {
void name_service_queue_size_metric() {
var tester = new DeploymentTester();
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .globalServiceId("default")
+ .endpoint("default", "foo")
.region("us-west-1")
.region("us-east-3")
.build();
@@ -660,7 +660,7 @@ public class MetricsReporterTest {
}
private Number getMetric(String name, ApplicationId id) {
- return metrics.getMetric((dimensions) -> id.tenant().value().equals(dimensions.get("tenant")) &&
+ return metrics.getMetric((dimensions) -> id.tenant().value().equals(dimensions.get("tenantName")) &&
appDimension(id).equals(dimensions.get("app")),
name)
.orElseThrow(() -> new RuntimeException("Expected metric to exist for " + id));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeSchedulerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeSchedulerTest.java
index 84227b4fd9f..178e8f18489 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeSchedulerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsUpgradeSchedulerTest.java
@@ -60,33 +60,33 @@ public class OsUpgradeSchedulerTest {
// New release becomes available, but is not triggered until cool-down period has passed, and we're inside a
// trigger period
Version version1 = Version.fromString("7.0.0.20220301");
- tester.clock().advance(Duration.ofDays(14));
- assertEquals("2022-03-01T09:05:00", formatInstant(tester.clock().instant()));
+ tester.clock().advance(Duration.ofDays(13).plusHours(15));
+ assertEquals("2022-03-01T00:05:00", formatInstant(tester.clock().instant()));
// Change does not become available until certification
- assertFalse(scheduler.changeIn(cloud, tester.clock().instant()).isPresent());
+ assertFalse(scheduler.changeIn(cloud, tester.clock().instant(), true).isPresent());
Version systemVersion = tester.controller().readSystemVersion();
Version olderThanSystemVersion = new Version(systemVersion.getMajor(), systemVersion.getMinor() - 1, systemVersion.getMicro());
tester.controller().os().certify(version1, cloud, olderThanSystemVersion);
// Change is now certified
- assertEquals(version1, scheduler.changeIn(cloud, tester.clock().instant()).get().osVersion().version());
+ assertEquals(version1, scheduler.changeIn(cloud, tester.clock().instant(), true).get().osVersion().version());
scheduler.maintain();
assertEquals(version0,
tester.controller().os().target(cloud).get().osVersion().version(),
"Target is unchanged because cooldown hasn't passed");
tester.clock().advance(Duration.ofDays(3).plusHours(18));
- assertEquals("2022-03-05T03:05:00", formatInstant(tester.clock().instant()));
+ assertEquals("2022-03-04T18:05:00", formatInstant(tester.clock().instant()));
scheduler.maintain();
assertEquals(version0,
tester.controller().os().target(cloud).get().osVersion().version(),
"Target is unchanged because we're outside trigger period");
- tester.clock().advance(Duration.ofDays(2).plusHours(5));
+ tester.clock().advance(Duration.ofDays(2).plusHours(14));
assertEquals("2022-03-07T08:05:00", formatInstant(tester.clock().instant()));
// Time constraints have now passed, but the current target has been pinned in the meantime
tester.controller().os().upgradeTo(version0, cloud, false, true);
- Optional<OsUpgradeScheduler.Change> change = scheduler.changeIn(cloud, tester.clock().instant());
+ Optional<OsUpgradeScheduler.Change> change = scheduler.changeIn(cloud, tester.clock().instant(), true);
assertTrue(change.isPresent());
assertEquals(-1, scheduler.maintain());
assertEquals(version0,
@@ -108,10 +108,10 @@ public class OsUpgradeSchedulerTest {
// Estimate next change
Version expected = Version.fromString("7.0.0.20220426");
tester.controller().os().certify(expected, cloud, systemVersion);
- Optional<OsUpgradeScheduler.Change> nextChange = scheduler.changeIn(cloud, tester.clock().instant());
+ Optional<OsUpgradeScheduler.Change> nextChange = scheduler.changeIn(cloud, tester.clock().instant(), true);
assertTrue(nextChange.isPresent());
assertEquals(expected, nextChange.get().osVersion().version());
- assertEquals("2022-04-27T07:00:00", formatInstant(nextChange.get().scheduleAt()));
+ assertEquals("2022-04-26T07:00:00", formatInstant(nextChange.get().scheduleAt()));
}
@Test
@@ -135,16 +135,16 @@ public class OsUpgradeSchedulerTest {
scheduler.maintain();
assertEquals(version0, tester.controller().os().target(cloud).get().osVersion().version());
// Cool-down passes
- tester.clock().advance(Duration.ofDays(1));
- assertEquals(version1, scheduler.changeIn(cloud, tester.clock().instant()).get().osVersion().version());
+ tester.clock().advance(Duration.ofHours(4));
+ assertEquals(version1, scheduler.changeIn(cloud, tester.clock().instant(), false).get().osVersion().version());
scheduler.maintain();
assertEquals(version1, tester.controller().os().target(cloud).get().osVersion().version());
// Estimate next change
- Optional<OsUpgradeScheduler.Change> nextChange = scheduler.changeIn(cloud, tester.clock().instant());
+ Optional<OsUpgradeScheduler.Change> nextChange = scheduler.changeIn(cloud, tester.clock().instant(), true);
assertTrue(nextChange.isPresent());
assertEquals("7.0.0.20220426", nextChange.get().osVersion().version().toFullString());
- assertEquals("2022-04-27T02:00:00", formatInstant(nextChange.get().scheduleAt()));
+ assertEquals("2022-04-26T06:00:00", formatInstant(nextChange.get().scheduleAt()));
}
@Test
@@ -166,13 +166,13 @@ public class OsUpgradeSchedulerTest {
scheduleUpgradeAfter(Duration.ZERO, version0, scheduler, tester);
// No change yet because it hasn't been certified
- Optional<OsUpgradeScheduler.Change> nextChange = scheduler.changeIn(cloud, tester.clock().instant());
+ Optional<OsUpgradeScheduler.Change> nextChange = scheduler.changeIn(cloud, tester.clock().instant(), true);
assertFalse(nextChange.isPresent(), "No change");
// Change is certified and upgrade is scheduled
Version systemVersion = tester.controller().readSystemVersion();
tester.controller().os().certify(version1, cloud, systemVersion);
- nextChange = scheduler.changeIn(cloud, tester.clock().instant());
+ nextChange = scheduler.changeIn(cloud, tester.clock().instant(), true);
assertTrue(nextChange.isPresent());
assertEquals(version1, nextChange.get().osVersion().version());
assertEquals("2021-06-22T07:00:00", formatInstant(nextChange.get().scheduleAt()));
@@ -184,7 +184,7 @@ public class OsUpgradeSchedulerTest {
// Nothing happens in next iteration as tagged release is older than manually triggered version
scheduleUpgradeAfter(Duration.ofDays(7), version3, scheduler, tester);
- assertTrue(scheduler.changeIn(cloud, tester.clock().instant()).isEmpty());
+ assertTrue(scheduler.changeIn(cloud, tester.clock().instant(), true).isEmpty());
}
@Test
@@ -203,8 +203,8 @@ public class OsUpgradeSchedulerTest {
Version version1 = Version.fromString("8.1");
tester.serviceRegistry().artifactRepository().addRelease(new OsRelease(version1, OsRelease.Tag.latest,
tester.clock().instant()));
- assertEquals(version1, scheduler.changeIn(cloud, tester.clock().instant()).get().osVersion().version());
- assertEquals("2021-06-22T07:05:00", formatInstant(scheduler.changeIn(cloud, tester.clock().instant()).get().scheduleAt()),
+ assertEquals(version1, scheduler.changeIn(cloud, tester.clock().instant(), true).get().osVersion().version());
+ assertEquals("2021-06-22T07:05:00", formatInstant(scheduler.changeIn(cloud, tester.clock().instant(), true).get().scheduleAt()),
"Not valid until cool-down period passes");
scheduleUpgradeAfter(Duration.ZERO, version0, scheduler, tester);
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsVersionStatusUpdaterTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsVersionStatusUpdaterTest.java
index 6644c3013ff..af535abce26 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsVersionStatusUpdaterTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/OsVersionStatusUpdaterTest.java
@@ -64,7 +64,9 @@ public class OsVersionStatusUpdaterTest {
assertTrue(osVersions.get(new OsVersion(version1, otherCloud)).isEmpty(), "No nodes on current target");
// Updating status cleans up stale certifications
- Set<OsVersion> knownVersions = osVersions.keySet();
+ Set<OsVersion> knownVersions = osVersions.keySet().stream()
+ .filter(osVersion -> !osVersion.version().isEmpty())
+ .collect(Collectors.toSet());
List<OsVersion> versionsToCertify = new ArrayList<>(knownVersions);
versionsToCertify.addAll(List.of(new OsVersion(Version.fromString("95.0.1"), cloud),
new OsVersion(Version.fromString("98.0.2"), cloud)));
@@ -77,7 +79,7 @@ public class OsVersionStatusUpdaterTest {
}
private static Set<OsVersion> certifiedOsVersions(ControllerTester tester) {
- return tester.controller().curator().readCertifiedOsVersions().stream()
+ return tester.controller().os().readCertified().stream()
.map(CertifiedOsVersion::osVersion)
.collect(Collectors.toSet());
}
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 69b473dce87..f287bc52604 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
@@ -78,8 +78,8 @@ public class ApplicationSerializerTest {
DeploymentSpec deploymentSpec = DeploymentSpec.fromXml("<deployment version='1.0'>\n" +
" <staging/>\n" +
" <instance id=\"i1\">\n" +
- " <prod global-service-id=\"default\">\n" +
- " <region active=\"true\">us-west-1</region>\n" +
+ " <prod>\n" +
+ " <region>us-west-1</region>\n" +
" </prod>\n" +
" </instance>\n" +
"</deployment>");
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/testdata/complete-application.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/testdata/complete-application.json
index ec36f52c23a..32f7e8e8f5a 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/testdata/complete-application.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/testdata/complete-application.json
@@ -2,7 +2,7 @@
"id": "tenant1:app1",
"internal": true,
"deploymentIssueId": "321",
- "deploymentSpecField": "<deployment version='1.0'>\n <test />\n <!--<staging />-->\n <prod global-service-id=\"foo\">\n <region active=\"true\">us-east-3</region>\n <region active=\"true\">us-west-1</region>\n </prod>\n</deployment>\n",
+ "deploymentSpecField": "<deployment version='1.0'>\n <test />\n <!--<staging />-->\n <prod>\n <region>us-east-3</region>\n <region>us-west-1</region>\n </prod>\n</deployment>\n",
"validationOverrides": "<validation-overrides>\n <allow until=\"2016-04-28\" comment=\"Renaming content cluster\">content-cluster-removal</allow>\n <allow until=\"2016-08-22\" comment=\"Migrating us-east-3 to C-2E\">cluster-size-reduction</allow>\n <allow until=\"2017-06-30\" comment=\"Test Vespa upgrade tests\">force-automatic-tenant-upgrade-test</allow>\n</validation-overrides>\n",
"projectId": 102889,
"deployingField": {
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 3cd9d586350..915466dac26 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
@@ -542,7 +542,7 @@ public class ApplicationApiCloudTest extends ControllerContainerCloudTest {
var applicationPackage = new ApplicationPackageBuilder()
.trustDefaultCertificate()
.instances("default")
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("aws-us-east-1c")
.build();
new ControllerTester(tester).upgradeSystem(new Version("6.1"));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
index 14c279c9ef8..98775ea214d 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiTest.java
@@ -130,7 +130,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
private static final ApplicationPackage applicationPackageDefault = new ApplicationPackageBuilder()
.withoutAthenzIdentity()
.instances("default")
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("us-central-1")
.region("us-east-3")
.region("us-west-1")
@@ -140,7 +140,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
private static final ApplicationPackage applicationPackageInstance1 = new ApplicationPackageBuilder()
.withoutAthenzIdentity()
.instances("instance1")
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("us-central-1")
.region("us-east-3")
.region("us-west-1")
@@ -343,7 +343,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
.withoutAthenzIdentity()
.instances("instance1")
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("us-west-1")
.region("us-east-3")
.allow(ValidationId.globalEndpointChange)
@@ -864,7 +864,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
// Third attempt has a service under the domain of the tenant, and also succeeds.
ApplicationPackage packageWithService = new ApplicationPackageBuilder()
.instances("instance1")
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.athenzIdentity(com.yahoo.config.provision.AthenzDomain.from(ATHENZ_TENANT_DOMAIN.getName()), AthenzService.from("service"))
.region("us-central-1")
.parallel("us-west-1", "us-east-3")
@@ -1043,7 +1043,7 @@ public class ApplicationApiTest extends ControllerContainerTest {
var eastZone = ZoneId.from("prod", "us-east-3");
var applicationPackage = new ApplicationPackageBuilder()
.instances("instance1")
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region(westZone.region())
.region(eastZone.region())
.build();
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiTest.java
index 15505dc3e95..f1a40307804 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/OsApiTest.java
@@ -115,8 +115,13 @@ public class OsApiTest extends ControllerContainerTest {
// Certify an OS and Vespa version pair
assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud1/7.5.2", "8.200.37", Request.Method.POST),
"{\"message\":\"Certified 7.5.2 in cloud cloud1 as compatible with Vespa version 8.200.37\"}", 200);
+ assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud2/7.5.2", "8.200.33", Request.Method.POST),
+ "{\"message\":\"Certified 7.5.2 in cloud cloud2 as compatible with Vespa version 8.200.33\"}", 200);
assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud1/7.5.2", "8.200.42", Request.Method.POST),
"{\"message\":\"7.5.2 is already certified in cloud cloud1 as compatible with Vespa version 8.200.37. Leaving certification unchanged\"}", 200);
+ assertResponse(new Request("http://localhost:8080/os/v1/certify", "", Request.Method.GET),
+ """
+[{"version":"7.5.2","cloud":"cloud1","vespaVersion":"8.200.37"},{"version":"7.5.2","cloud":"cloud2","vespaVersion":"8.200.33"}]""", 200);
assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud1/7.5.2", "", Request.Method.DELETE),
"{\"message\":\"Removed certification of 7.5.2 in cloud cloud1\"}", 200);
@@ -130,7 +135,7 @@ public class OsApiTest extends ControllerContainerTest {
assertResponse(new Request("http://localhost:8080/os/v1/", "{\"version\": \"0.0.0\", \"cloud\": \"cloud1\"}", Request.Method.PATCH),
"{\"error-code\":\"BAD_REQUEST\",\"message\":\"Invalid version '0.0.0'\"}", 400);
assertResponse(new Request("http://localhost:8080/os/v1/", "{\"version\": \"foo\", \"cloud\": \"cloud1\"}", Request.Method.PATCH),
- "{\"error-code\":\"BAD_REQUEST\",\"message\":\"Invalid version 'foo': For input string: \\\"foo\\\"\"}", 400);
+ "{\"error-code\":\"BAD_REQUEST\",\"message\":\"Invalid version 'foo': Invalid version component in 'foo': For input string: \\\"foo\\\"\"}", 400);
// Error: Invalid cloud
assertResponse(new Request("http://localhost:8080/os/v1/", "{\"version\": \"7.6\", \"cloud\": \"foo\"}", Request.Method.PATCH),
@@ -160,6 +165,14 @@ public class OsApiTest extends ControllerContainerTest {
assertResponse(new Request("http://localhost:8080/os/v1/firmware/dev/", "", Request.Method.DELETE),
"{\"error-code\":\"NOT_FOUND\",\"message\":\"No zones at path '/os/v1/firmware/dev/'\"}", 404);
+ // Error: Missing or invalid versions to certify
+ assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud1/7.5.2", "", Request.Method.POST),
+ "{\"error-code\":\"BAD_REQUEST\",\"message\":\"Missing Vespa version in request body\"}", 400);
+ assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud1/7.5.2", "foo", Request.Method.POST),
+ "{\"error-code\":\"BAD_REQUEST\",\"message\":\"Invalid version component in 'foo': For input string: \\\"foo\\\"\"}", 400);
+ assertResponse(new Request("http://localhost:8080/os/v1/certify/cloud1/bar", "1.2.3", Request.Method.POST),
+ "{\"error-code\":\"BAD_REQUEST\",\"message\":\"Invalid version component in 'bar': For input string: \\\"bar\\\"\"}", 400);
+
assertFalse(tester.controller().auditLogger().readLog().entries().isEmpty(), "Actions are logged to audit log");
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-all-upgraded.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-all-upgraded.json
index 0f2e05986b6..63a85bc5aad 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-all-upgraded.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-all-upgraded.json
@@ -107,7 +107,7 @@
"scheduledAt": 1234,
"pinned": false,
"nextVersion": "8.2.1.20211228",
- "nextScheduledAt": 1640743200000,
+ "nextScheduledAt": 1640671200000,
"cloud": "cloud2",
"nodes": [
{
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-partially-upgraded.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-partially-upgraded.json
index 20d7147a258..75e68b00676 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-partially-upgraded.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/os/responses/versions-partially-upgraded.json
@@ -166,7 +166,7 @@
"scheduledAt": 1234,
"pinned": false,
"nextVersion": "8.2.1.20211228",
- "nextScheduledAt": 1640743200000,
+ "nextScheduledAt": 1640671200000,
"cloud": "cloud2",
"nodes": [ ]
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java
index 02f030aa758..46ec42cab8f 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/routing/RoutingPoliciesTest.java
@@ -234,27 +234,6 @@ public class RoutingPoliciesTest {
}
@Test
- void global_routing_policies_legacy_global_service_id() {
- var tester = new RoutingPoliciesTester();
- var context = tester.newDeploymentContext("tenant1", "app1", "default");
- int clustersPerZone = 2;
- int numberOfDeployments = 2;
- var applicationPackage = applicationPackageBuilder()
- .region(zone1.region())
- .region(zone2.region())
- .globalServiceId("c0")
- .build();
- tester.provisionLoadBalancers(clustersPerZone, context.instanceId(), zone1, zone2);
-
- // Creates alias records
- context.submit(applicationPackage).deferLoadBalancerProvisioningIn(Environment.prod).deploy();
- tester.assertTargets(context.instanceId(), EndpointId.defaultId(), 0, zone1, zone2);
- assertEquals(numberOfDeployments * clustersPerZone,
- tester.policiesOf(context.instance().id()).size(),
- "Routing policy count is equal to cluster count");
- }
-
- @Test
void zone_routing_policies() {
zone_routing_policies(false);
zone_routing_policies(true);
@@ -663,28 +642,6 @@ public class RoutingPoliciesTest {
assertEquals(RoutingStatus.Value.in, policy1.routingStatus().value());
assertEquals(RoutingStatus.Agent.tenant, policy1.routingStatus().agent());
assertEquals(changedAt.truncatedTo(ChronoUnit.MILLIS), policy1.routingStatus().changedAt());
-
- // Deployment is set out through a new deployment.xml
- var applicationPackage2 = applicationPackageBuilder()
- .region(zone1.region())
- .region(zone2.region(), false)
- .endpoint("r0", "c0", zone1.region().value(), zone2.region().value())
- .endpoint("r1", "c0", zone1.region().value(), zone2.region().value())
- .build();
- context.submit(applicationPackage2).deferLoadBalancerProvisioningIn(Environment.prod).deploy();
- tester.assertTargets(context.instanceId(), EndpointId.of("r0"), 0, zone1);
- tester.assertTargets(context.instanceId(), EndpointId.of("r1"), 0, zone1);
-
- // ... back in
- var applicationPackage3 = applicationPackageBuilder()
- .region(zone1.region())
- .region(zone2.region())
- .endpoint("r0", "c0", zone1.region().value(), zone2.region().value())
- .endpoint("r1", "c0", zone1.region().value(), zone2.region().value())
- .build();
- context.submit(applicationPackage3).deferLoadBalancerProvisioningIn(Environment.prod).deploy();
- tester.assertTargets(context.instanceId(), EndpointId.of("r0"), 0, zone1, zone2);
- tester.assertTargets(context.instanceId(), EndpointId.of("r1"), 0, zone1, zone2);
}
@Test
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 168a1345c39..6190680d098 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
@@ -41,7 +41,7 @@ public class RotationRepositoryTest {
);
private static final ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("us-east-3")
.region("us-west-1")
.build();
@@ -74,7 +74,7 @@ public class RotationRepositoryTest {
// Adding region updates rotation
var applicationPackage = new ApplicationPackageBuilder()
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("us-east-3")
.region("us-west-1")
.region("us-central-1")
@@ -116,15 +116,6 @@ public class RotationRepositoryTest {
}
@Test
- void too_few_zones() {
- ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .globalServiceId("foo")
- .region("us-east-3")
- .build();
- application.submit(applicationPackage).runJobExpectingFailure(DeploymentContext.systemTest, "less than 2 prod zones are defined");
- }
-
- @Test
void no_rotation_assigned_for_application_without_service_id() {
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
.region("us-east-3")
@@ -137,7 +128,7 @@ public class RotationRepositoryTest {
@Test
void prefixes_system_when_not_main() {
ApplicationPackage applicationPackage = new ApplicationPackageBuilder()
- .globalServiceId("foo")
+ .endpoint("default", "foo")
.region("cd-us-east-1")
.region("cd-us-west-1")
.build();
@@ -164,7 +155,7 @@ public class RotationRepositoryTest {
.instances("instance1,instance2")
.region("us-central-1")
.parallel("us-west-1", "us-east-3")
- .globalServiceId("global")
+ .endpoint("default", "global")
.build();
var instance1 = tester.newDeploymentContext("tenant1", "application1", "instance1")
.submit(applicationPackage)