aboutsummaryrefslogtreecommitdiffstats
path: root/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment
diff options
context:
space:
mode:
Diffstat (limited to 'controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment')
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentStatus.java27
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentSteps.java19
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentTrigger.java14
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/InternalStepRunner.java28
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java53
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java2
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RetriggerEntrySerializer.java10
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializer.java2
8 files changed, 75 insertions, 80 deletions
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 e13175806bf..cc7031bab5a 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
@@ -12,7 +12,6 @@ import com.yahoo.config.application.api.DeploymentSpec.UpgradeRollout;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.InstanceName;
-import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.Application;
import com.yahoo.vespa.hosted.controller.Instance;
@@ -20,6 +19,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationV
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;
+import com.yahoo.vespa.hosted.controller.api.integration.zone.ZoneRegistry;
import com.yahoo.vespa.hosted.controller.application.Change;
import com.yahoo.vespa.hosted.controller.application.Deployment;
@@ -45,8 +45,6 @@ import static com.yahoo.config.application.api.DeploymentSpec.RevisionTarget.nex
import static com.yahoo.config.provision.Environment.prod;
import static com.yahoo.config.provision.Environment.staging;
import static com.yahoo.config.provision.Environment.test;
-import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.stagingTest;
-import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.systemTest;
import static java.util.Comparator.comparing;
import static java.util.Comparator.naturalOrder;
import static java.util.Objects.requireNonNull;
@@ -69,17 +67,19 @@ public class DeploymentStatus {
private final Application application;
private final JobList allJobs;
- private final SystemName system;
+ private final JobType systemTest;
+ private final JobType stagingTest;
private final Version systemVersion;
private final Function<InstanceName, VersionCompatibility> versionCompatibility;
private final Instant now;
private final Map<JobId, StepStatus> jobSteps;
private final List<StepStatus> allSteps;
- public DeploymentStatus(Application application, Function<JobId, JobStatus> allJobs, SystemName system,
+ public DeploymentStatus(Application application, Function<JobId, JobStatus> allJobs, ZoneRegistry zones,
Version systemVersion, Function<InstanceName, VersionCompatibility> versionCompatibility, Instant now) {
this.application = requireNonNull(application);
- this.system = requireNonNull(system);
+ this.systemTest = JobType.systemTest(zones);
+ this.stagingTest = JobType.stagingTest(zones);
this.systemVersion = requireNonNull(systemVersion);
this.versionCompatibility = versionCompatibility;
this.now = requireNonNull(now);
@@ -244,7 +244,7 @@ public class DeploymentStatus {
public Optional<Deployment> deploymentFor(JobId job) {
return Optional.ofNullable(application.require(job.application().instance())
- .deployments().get(job.type().zone(system)));
+ .deployments().get(job.type().zone()));
}
/**
@@ -388,7 +388,7 @@ public class DeploymentStatus {
// For a dual change, where both targets remain, we determine what to run by looking at when the two parts became ready:
// for deployments, we look at dependencies; for production tests, this may be overridden by what is already deployed.
- JobId deployment = new JobId(job.application(), JobType.from(system, job.type().zone(system)).get());
+ JobId deployment = new JobId(job.application(), JobType.deploymentTo(job.type().zone()));
UpgradeRollout rollout = application.deploymentSpec().requireInstance(job.application().instance()).upgradeRollout();
if (job.type().isTest()) {
Optional<Instant> platformDeployedAt = jobSteps.get(deployment).completedAt(change.withoutApplication(), Optional.of(deployment));
@@ -556,23 +556,20 @@ public class DeploymentStatus {
JobId jobId;
StepStatus stepStatus;
if (step.concerns(test) || step.concerns(staging)) {
- jobType = JobType.from(system, ((DeclaredZone) step).environment(), null)
- .orElseThrow(() -> new IllegalStateException(application + " specifies " + step + ", but this has no job in " + system));
+ jobType = step.concerns(test) ? systemTest : stagingTest;
jobId = new JobId(application.id().instance(instance), jobType);
stepStatus = JobStepStatus.ofTestDeployment((DeclaredZone) step, List.of(), this, jobs.apply(jobId), true);
previous = new ArrayList<>(previous);
previous.add(stepStatus);
}
else if (step.isTest()) {
- jobType = JobType.testFrom(system, ((DeclaredTest) step).region())
- .orElseThrow(() -> new IllegalStateException(application + " specifies " + step + ", but this has no job in " + system));
+ jobType = JobType.test(((DeclaredTest) step).region());
jobId = new JobId(application.id().instance(instance), jobType);
stepStatus = JobStepStatus.ofProductionTest((DeclaredTest) step, previous, this, jobs.apply(jobId));
previous = List.of(stepStatus);
}
else if (step.concerns(prod)) {
- jobType = JobType.from(system, ((DeclaredZone) step).environment(), ((DeclaredZone) step).region().get())
- .orElseThrow(() -> new IllegalStateException(application + " specifies " + step + ", but this has no job in " + system));
+ jobType = JobType.prod(((DeclaredZone) step).region().get());
jobId = new JobId(application.id().instance(instance), jobType);
stepStatus = JobStepStatus.ofProductionDeployment((DeclaredZone) step, previous, this, jobs.apply(jobId));
previous = List.of(stepStatus);
@@ -872,7 +869,7 @@ public class DeploymentStatus {
private static JobStepStatus ofProductionTest(DeclaredTest step, List<StepStatus> dependencies,
DeploymentStatus status, JobStatus job) {
- JobId prodId = new JobId(job.id().application(), JobType.from(status.system, job.id().type().zone(status.system)).get());
+ JobId prodId = new JobId(job.id().application(), JobType.deploymentTo(job.id().type().zone()));
return new JobStepStatus(StepType.test, step, dependencies, job, status) {
@Override
Optional<Instant> readyAt(Change change, Optional<JobId> dependent) {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentSteps.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentSteps.java
index 7ab895654f3..44079a90097 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentSteps.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/DeploymentSteps.java
@@ -7,6 +7,7 @@ import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
+import com.yahoo.vespa.hosted.controller.api.integration.zone.ZoneRegistry;
import com.yahoo.vespa.hosted.controller.application.Deployment;
import java.util.Collection;
@@ -29,16 +30,16 @@ import static java.util.stream.Collectors.collectingAndThen;
public class DeploymentSteps {
private final DeploymentInstanceSpec spec;
- private final Supplier<SystemName> system;
+ private final ZoneRegistry zones;
- public DeploymentSteps(DeploymentInstanceSpec spec, Supplier<SystemName> system) {
+ public DeploymentSteps(DeploymentInstanceSpec spec, ZoneRegistry zones) {
this.spec = Objects.requireNonNull(spec, "spec cannot be null");
- this.system = Objects.requireNonNull(system, "system cannot be null");
+ this.zones = Objects.requireNonNull(zones, "system cannot be null");
}
/** Returns jobs for this, in the order they should run */
public List<JobType> jobs() {
- return Stream.concat(production().isEmpty() ? Stream.of() : Stream.of(JobType.systemTest, JobType.stagingTest),
+ return Stream.concat(production().isEmpty() ? Stream.of() : Stream.of(JobType.systemTest(zones), JobType.stagingTest(zones)),
spec.steps().stream().flatMap(step -> toJobs(step).stream()))
.distinct()
.collect(Collectors.toUnmodifiableList());
@@ -67,7 +68,6 @@ public class DeploymentSteps {
public List<JobType> toJobs(DeploymentSpec.Step step) {
return step.zones().stream()
.map(this::toJob)
- .flatMap(Optional::stream)
.collect(Collectors.toUnmodifiableList());
}
@@ -93,8 +93,13 @@ public class DeploymentSteps {
}
/** Resolve job from deployment zone */
- private Optional<JobType> toJob(DeploymentSpec.DeclaredZone zone) {
- return JobType.from(system.get(), zone.environment(), zone.region().orElse(null));
+ private JobType toJob(DeploymentSpec.DeclaredZone zone) {
+ switch (zone.environment()) {
+ case prod: return JobType.prod(zone.region().get());
+ case test: return JobType.systemTest(zones);
+ case staging: return JobType.stagingTest(zones);
+ default: throw new IllegalArgumentException("region must be one with automated deployments, but got: " + zone.environment());
+ }
}
/** Resolve jobs from steps */
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 c98b3b76292..be07a2b0cb1 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
@@ -74,10 +74,7 @@ public class DeploymentTrigger {
}
public DeploymentSteps steps(DeploymentInstanceSpec spec) {
- return new DeploymentSteps(spec, controller::system);
- }
-
- public void notifyOfSubmission(TenantAndApplicationId id, ApplicationVersion version, long projectId) {
+ return new DeploymentSteps(spec, controller.zoneRegistry());
}
/**
@@ -268,8 +265,7 @@ public class DeploymentTrigger {
/** Retrigger job. If the job is already running, it will be canceled, and retrigger enqueued. */
public Optional<JobId> reTriggerOrAddToQueue(DeploymentId deployment, String reason) {
- JobType jobType = JobType.from(controller.system(), deployment.zoneId())
- .orElseThrow(() -> new IllegalArgumentException(Text.format("No job to trigger for (system/zone): %s/%s", controller.system().value(), deployment.zoneId().value())));
+ JobType jobType = JobType.deploymentTo(deployment.zoneId());
Optional<Run> existingRun = controller.jobController().active(deployment.applicationId()).stream()
.filter(run -> run.id().type().equals(jobType))
.findFirst();
@@ -389,7 +385,7 @@ public class DeploymentTrigger {
/** Returns whether the application is healthy in all other production zones. */
private boolean isUnhealthyInAnotherZone(Application application, JobId job) {
for (Deployment deployment : application.require(job.application().instance()).productionDeployments().values()) {
- if ( ! deployment.zone().equals(job.type().zone(controller.system()))
+ if ( ! deployment.zone().equals(job.type().zone())
&& ! controller.applications().isHealthy(new DeploymentId(job.application(), deployment.zone())))
return true;
}
@@ -418,9 +414,7 @@ public class DeploymentTrigger {
boolean blocked = status.jobs().get(job).get().isRunning();
if ( ! job.type().isTest()) {
- Optional<JobStatus> productionTest = JobType.testFrom(controller.system(), job.type().zone(controller.system()).region())
- .map(type -> new JobId(job.application(), type))
- .flatMap(status.jobs()::get);
+ Optional<JobStatus> productionTest = status.jobs().get(new JobId(job.application(), JobType.productionTestOf(job.type().zone())));
if (productionTest.isPresent()) {
abortIfOutdated(status, jobs, productionTest.get().id());
// Production deployments are also blocked by their declared tests, if the next versions to run
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 28d9439b457..9e551c7ce78 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
@@ -217,7 +217,7 @@ public class InternalStepRunner implements StepRunner {
logger.log("Deploying the tester container on platform " + platform + " ...");
return deploy(() -> controller.applications().deployTester(id.tester(),
testerPackage(id),
- id.type().zone(controller.system()),
+ id.type().zone(),
platform),
controller.jobController().run(id).get()
.stepInfo(deployTester).get()
@@ -314,19 +314,19 @@ public class InternalStepRunner implements StepRunner {
Version platform = setTheStage ? versions.sourcePlatform().orElse(versions.targetPlatform()) : versions.targetPlatform();
Run run = controller.jobController().run(id).get();
- Optional<ServiceConvergence> services = controller.serviceRegistry().configServer().serviceConvergence(new DeploymentId(id.application(), id.type().zone(controller.system())),
+ Optional<ServiceConvergence> services = controller.serviceRegistry().configServer().serviceConvergence(new DeploymentId(id.application(), id.type().zone()),
Optional.of(platform));
if (services.isEmpty()) {
logger.log("Config status not currently available -- will retry.");
return Optional.empty();
}
- List<Node> nodes = controller.serviceRegistry().configServer().nodeRepository().list(id.type().zone(controller.system()),
+ List<Node> nodes = controller.serviceRegistry().configServer().nodeRepository().list(id.type().zone(),
NodeFilter.all()
.applications(id.application())
.states(active));
Set<HostName> parentHostnames = nodes.stream().map(node -> node.parentHostname().get()).collect(toSet());
- List<Node> parents = controller.serviceRegistry().configServer().nodeRepository().list(id.type().zone(controller.system()),
+ List<Node> parents = controller.serviceRegistry().configServer().nodeRepository().list(id.type().zone(),
NodeFilter.all()
.hostnames(parentHostnames));
boolean firstTick = run.convergenceSummary().isEmpty();
@@ -357,8 +357,8 @@ public class InternalStepRunner implements StepRunner {
}
if (summary.converged()) {
controller.jobController().locked(id, lockedRun -> lockedRun.withSummary(null));
- if (endpointsAvailable(id.application(), id.type().zone(controller.system()), logger)) {
- if (containersAreUp(id.application(), id.type().zone(controller.system()), logger)) {
+ if (endpointsAvailable(id.application(), id.type().zone(), logger)) {
+ if (containersAreUp(id.application(), id.type().zone(), logger)) {
logger.log("Installation succeeded!");
return Optional.of(running);
}
@@ -440,7 +440,7 @@ public class InternalStepRunner implements StepRunner {
private Optional<RunStatus> installTester(RunId id, DualLogger logger) {
Run run = controller.jobController().run(id).get();
Version platform = testerPlatformVersion(id);
- ZoneId zone = id.type().zone(controller.system());
+ ZoneId zone = id.type().zone();
ApplicationId testerId = id.tester().id();
Optional<ServiceConvergence> services = controller.serviceRegistry().configServer().serviceConvergence(new DeploymentId(testerId, zone),
@@ -609,7 +609,7 @@ public class InternalStepRunner implements StepRunner {
.productionDeployments().keySet().stream()
.map(zone -> new DeploymentId(id.application(), zone))
.collect(Collectors.toSet());
- ZoneId zoneId = id.type().zone(controller.system());
+ ZoneId zoneId = id.type().zone();
deployments.add(new DeploymentId(id.application(), zoneId));
logger.log("Attempting to find endpoints ...");
@@ -722,8 +722,8 @@ public class InternalStepRunner implements StepRunner {
private Optional<RunStatus> deactivateReal(RunId id, DualLogger logger) {
try {
- logger.log("Deactivating deployment of " + id.application() + " in " + id.type().zone(controller.system()) + " ...");
- controller.applications().deactivate(id.application(), id.type().zone(controller.system()));
+ logger.log("Deactivating deployment of " + id.application() + " in " + id.type().zone() + " ...");
+ controller.applications().deactivate(id.application(), id.type().zone());
return Optional.of(running);
}
catch (RuntimeException e) {
@@ -737,7 +737,7 @@ public class InternalStepRunner implements StepRunner {
private Optional<RunStatus> deactivateTester(RunId id, DualLogger logger) {
try {
- logger.log("Deactivating tester of " + id.application() + " in " + id.type().zone(controller.system()) + " ...");
+ logger.log("Deactivating tester of " + id.application() + " in " + id.type().zone() + " ...");
controller.jobController().deactivateTester(id.tester(), id.type());
return Optional.of(running);
}
@@ -870,7 +870,7 @@ public class InternalStepRunner implements StepRunner {
/** Returns the deployment of the real application in the zone of the given job, if it exists. */
private Optional<Deployment> deployment(ApplicationId id, JobType type) {
- return Optional.ofNullable(application(id).deployments().get(type.zone(controller.system())));
+ return Optional.ofNullable(application(id).deployments().get(type.zone()));
}
/** Returns the real application with the given id. */
@@ -908,7 +908,7 @@ public class InternalStepRunner implements StepRunner {
RevisionId revision = controller.jobController().run(id).get().versions().targetRevision();
DeploymentSpec spec = controller.applications().requireApplication(TenantAndApplicationId.from(id.application())).deploymentSpec();
- ZoneId zone = id.type().zone(controller.system());
+ ZoneId zone = id.type().zone();
boolean useTesterCertificate = useTesterCertificate(id);
byte[] servicesXml = servicesXml( ! controller.system().isPublic(),
@@ -952,7 +952,7 @@ public class InternalStepRunner implements StepRunner {
}
private DeploymentId getTesterDeploymentId(RunId runId) {
- ZoneId zoneId = runId.type().zone(controller.system());
+ ZoneId zoneId = runId.type().zone();
return new DeploymentId(runId.tester().id(), zoneId);
}
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 6a6ed6e3b5d..69d9ba504a5 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
@@ -30,7 +30,6 @@ import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackage;
import com.yahoo.vespa.hosted.controller.application.pkg.ApplicationPackageDiff;
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 java.security.cert.X509Certificate;
@@ -51,6 +50,7 @@ import java.util.TreeMap;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
+import java.util.function.Predicate;
import java.util.function.UnaryOperator;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -169,7 +169,7 @@ public class JobController {
if ( ! run.hasStep(copyVespaLogs))
return run;
- ZoneId zone = id.type().zone(controller.system());
+ ZoneId zone = id.type().zone();
Optional<Deployment> deployment = Optional.ofNullable(controller.applications().requireInstance(id.application())
.deployments().get(zone));
if (deployment.isEmpty() || deployment.get().at().isBefore(run.start()))
@@ -197,7 +197,7 @@ public class JobController {
if (step.isEmpty())
return run;
- List<LogEntry> entries = cloud.getLog(new DeploymentId(id.tester().id(), id.type().zone(controller.system())),
+ List<LogEntry> entries = cloud.getLog(new DeploymentId(id.tester().id(), id.type().zone()),
run.lastTestLogEntry());
if (entries.isEmpty())
return run;
@@ -209,7 +209,7 @@ public class JobController {
public void updateTestReport(RunId id) {
locked(id, run -> {
- Optional<TestReport> report = cloud.getTestReport(new DeploymentId(id.tester().id(), id.type().zone(controller.system())));
+ Optional<TestReport> report = cloud.getTestReport(new DeploymentId(id.tester().id(), id.type().zone()));
if (report.isEmpty()) {
return run;
}
@@ -257,10 +257,9 @@ public class JobController {
/** Returns when given deployment last started deploying, falling back to time of deployment if it cannot be determined from job runs */
public Instant lastDeploymentStart(ApplicationId instanceId, Deployment deployment) {
- return jobStarts(new JobId(instanceId, JobType.from(controller.system(),
- deployment.zone()).get())).stream()
- .findFirst()
- .orElseGet(deployment::at);
+ return jobStarts(new JobId(instanceId, JobType.deploymentTo(deployment.zone()))).stream()
+ .findFirst()
+ .orElseGet(deployment::at);
}
/** Returns an immutable map of all known runs for the given application and job type. */
@@ -353,7 +352,7 @@ public class JobController {
private DeploymentStatus deploymentStatus(Application application, Version systemVersion) {
return new DeploymentStatus(application,
this::jobStatus,
- controller.system(),
+ controller.zoneRegistry(),
systemVersion,
instance -> controller.applications().versionCompatibility(application.id().instance(instance)),
controller.clock().instant());
@@ -526,7 +525,7 @@ public class JobController {
controller.applications().store(application.withRevisions(revisions -> revisions.withoutOlderThan(oldestRevision)));
}
else {
- controller.applications().applicationStore().pruneDevDiffs(new DeploymentId(run.id().application(), run.id().job().type().zone(controller.system())), oldestRevision.number());
+ controller.applications().applicationStore().pruneDevDiffs(new DeploymentId(run.id().application(), run.id().job().type().zone()), oldestRevision.number());
controller.applications().store(application.withRevisions(revisions -> revisions.withoutOlderThan(oldestRevision, run.id().job())));
}
});
@@ -566,6 +565,9 @@ 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, boolean dryRun) {
+ 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);
@@ -573,7 +575,7 @@ public class JobController {
controller.applications().store(application);
});
- DeploymentId deploymentId = new DeploymentId(id, type.zone(controller.system()));
+ DeploymentId deploymentId = new DeploymentId(id, type.zone());
Optional<Run> lastRun = last(id, type);
lastRun.filter(run -> ! run.hasEnded()).ifPresent(run -> abortAndWait(run.id()));
@@ -585,7 +587,7 @@ public class JobController {
controller.applications().lockApplicationOrThrow(TenantAndApplicationId.from(id), application -> {
controller.applications().applicationStore().putDev(deploymentId, version.id(), applicationPackage.zippedContent(), diff);
- Version targetPlatform = platform.orElseGet(() -> findTargetPlatform(applicationPackage, lastRun, id));
+ Version targetPlatform = platform.orElseGet(() -> findTargetPlatform(applicationPackage, deploymentId, application.get().get(id.instance())));
controller.applications().store(application.withRevisions(revisions -> revisions.with(version)));
start(id,
type,
@@ -615,24 +617,27 @@ public class JobController {
.orElseGet(() -> ApplicationPackageDiff.diffAgainstEmpty(applicationPackage));
}
- private Version findTargetPlatform(ApplicationPackage applicationPackage, Optional<Run> lastRun, ApplicationId id) {
+ private Version findTargetPlatform(ApplicationPackage applicationPackage, DeploymentId id, Optional<Instance> instance) {
Optional<Integer> major = applicationPackage.deploymentSpec().majorVersion();
if (major.isPresent())
return controller.applications().lastCompatibleVersion(major.get())
.orElseThrow(() -> new IllegalArgumentException("major " + major.get() + " specified in deployment.xml, " +
"but no version on this major was found"));
- // Prefer previous platform if possible.
- VersionStatus versionStatus = controller.readVersionStatus();
- VersionCompatibility compatibility = controller.applications().versionCompatibility(id);
- Optional<Version> target = lastRun.map(run -> run.versions().targetPlatform()).filter(versionStatus::isActive);
- if (target.isPresent() && compatibility.accept(target.get(), applicationPackage.compileVersion().orElse(target.get())))
- return target.get();
+ VersionCompatibility compatibility = controller.applications().versionCompatibility(id.applicationId());
+
+ // 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());
+ instance.map(Instance::deployments)
+ .map(deployments -> deployments.get(id.zoneId()))
+ .map(Deployment::version)
+ .ifPresent(versions::add);
- // Otherwise, use newest, compatible version.
- for (VespaVersion platform : reversed(versionStatus.deployableVersions()))
- if (compatibility.accept(platform.versionNumber(), applicationPackage.compileVersion().orElse(platform.versionNumber())))
- return platform.versionNumber();
+ for (Version target : reversed(versions))
+ if (applicationPackage.compileVersion().isEmpty() || compatibility.accept(target, applicationPackage.compileVersion().get()))
+ return target;
throw new IllegalArgumentException("no suitable platform version found" +
applicationPackage.compileVersion()
@@ -686,7 +691,7 @@ public class JobController {
}
public void deactivateTester(TesterId id, JobType type) {
- controller.serviceRegistry().configServer().deactivate(new DeploymentId(id.id(), type.zone(controller.system())));
+ controller.serviceRegistry().configServer().deactivate(new DeploymentId(id.id(), type.zone()));
}
/** Locks all runs and modifies the list of historic runs for the given application and job type. */
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java
index e95e515685f..14fce806152 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobMetrics.java
@@ -47,7 +47,7 @@ public class JobMetrics {
"tenantName", id.application().tenant().value(),
"app", id.application().application().value() + "." + id.application().instance().value(),
"test", Boolean.toString(id.type().isTest()),
- "zone", id.type().zone(system.get()).value());
+ "zone", id.type().zone().value());
}
static String valueOf(RunStatus status) {
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RetriggerEntrySerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RetriggerEntrySerializer.java
index 063167647d5..e0c1fef91b3 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RetriggerEntrySerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/RetriggerEntrySerializer.java
@@ -24,12 +24,6 @@ public class RetriggerEntrySerializer {
private static final String JOB_TYPE_KEY = "jobType";
private static final String MIN_REQUIRED_RUN_ID_KEY = "minimumRunId";
- private final SystemName system;
-
- public RetriggerEntrySerializer(SystemName system) {
- this.system = system;
- }
-
public List<RetriggerEntry> fromSlime(Slime slime) {
return SlimeUtils.entriesStream(slime.get().field("entries"))
.map(this::deserializeEntry)
@@ -48,14 +42,14 @@ public class RetriggerEntrySerializer {
Cursor root = array.addObject();
Cursor jobid = root.setObject(JOB_ID_KEY);
jobid.setString(APPLICATION_ID_KEY, entry.jobId().application().serializedForm());
- jobid.setString(JOB_TYPE_KEY, entry.jobId().type().serialized(system));
+ jobid.setString(JOB_TYPE_KEY, entry.jobId().type().serialized());
root.setLong(MIN_REQUIRED_RUN_ID_KEY, entry.requiredRun());
}
private RetriggerEntry deserializeEntry(Inspector inspector) {
Inspector jobid = inspector.field(JOB_ID_KEY);
ApplicationId applicationId = ApplicationId.fromSerializedForm(require(jobid, APPLICATION_ID_KEY).asString());
- JobType jobType = JobType.fromJobName(require(jobid, JOB_TYPE_KEY).asString());
+ JobType jobType = JobType.ofSerialized(require(jobid, JOB_TYPE_KEY).asString());
long minRequiredRunId = require(inspector, MIN_REQUIRED_RUN_ID_KEY).asLong();
return new RetriggerEntry(new JobId(applicationId, jobType), minRequiredRunId);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializer.java
index da42a00cd44..1680e064234 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/TestConfigSerializer.java
@@ -37,7 +37,7 @@ public class TestConfigSerializer {
Cursor root = slime.setObject();
root.setString("application", id.serializedForm());
- root.setString("zone", type.zone(system).value());
+ root.setString("zone", type.zone().value());
root.setString("system", system.value());
root.setBool("isCI", isCI);