summaryrefslogtreecommitdiffstats
path: root/controller-server
diff options
context:
space:
mode:
authorJon Marius Venstad <venstad@gmail.com>2021-07-05 16:18:09 +0200
committerJon Marius Venstad <venstad@gmail.com>2021-07-05 16:18:09 +0200
commit8e6228ad900623f7cc0c2c0efbcdfecdf6312ab8 (patch)
treeae49a3c3ebfb6c7f3bb040eeb4b756451627b459 /controller-server
parent1ecd90b29e762d8686dd637bc96925a3cd147684 (diff)
Upgrade dev at dep-local night
Diffstat (limited to 'controller-server')
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/JobController.java12
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Run.java31
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java1
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirer.java8
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java93
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializer.java5
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java2
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgraderTest.java101
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializerTest.java2
9 files changed, 235 insertions, 20 deletions
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 25bc21a0076..2281d98bb3d 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
@@ -440,18 +440,23 @@ public class JobController {
/** Orders a run of the given type, or throws an IllegalStateException if that job type is already running. */
public void start(ApplicationId id, JobType type, Versions versions) {
- start(id, type, versions, JobProfile.of(type));
+ start(id, type, versions, false);
}
/** Orders a run of the given type, or throws an IllegalStateException if that job type is already running. */
- public void start(ApplicationId id, JobType type, Versions versions, JobProfile profile) {
+ public void start(ApplicationId id, JobType type, Versions versions, boolean isRedeployment) {
+ start(id, type, versions, isRedeployment, JobProfile.of(type));
+ }
+
+ /** Orders a run of the given type, or throws an IllegalStateException if that job type is already running. */
+ public void start(ApplicationId id, JobType type, Versions versions, boolean isRedeployment, JobProfile profile) {
locked(id, type, __ -> {
Optional<Run> last = last(id, type);
if (last.flatMap(run -> active(run.id())).isPresent())
throw new IllegalStateException("Can not start " + type + " for " + id + "; it is already running!");
RunId newId = new RunId(id, type, last.map(run -> run.id().number()).orElse(0L) + 1);
- curator.writeLastRun(Run.initial(newId, versions, controller.clock().instant(), profile));
+ curator.writeLastRun(Run.initial(newId, versions, isRedeployment, controller.clock().instant(), profile));
metric.jobStarted(newId.job());
});
}
@@ -477,6 +482,7 @@ public class JobController {
ApplicationVersion.unknown,
Optional.empty(),
Optional.empty()),
+ false,
JobProfile.development);
});
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Run.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Run.java
index d2481cd97ad..d93a0133c97 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Run.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/deployment/Run.java
@@ -30,6 +30,7 @@ public class Run {
private final RunId id;
private final Map<Step, StepInfo> steps;
private final Versions versions;
+ private final boolean isRedeployment;
private final Instant start;
private final Optional<Instant> end;
private final RunStatus status;
@@ -40,12 +41,13 @@ public class Run {
private final Optional<X509Certificate> testerCertificate;
// For deserialisation only -- do not use!
- public Run(RunId id, Map<Step, StepInfo> steps, Versions versions, Instant start, Optional<Instant> end,
+ public Run(RunId id, Map<Step, StepInfo> steps, Versions versions, boolean isRedeployment, Instant start, Optional<Instant> end,
RunStatus status, long lastTestRecord, Instant lastVespaLogTimestamp, Optional<Instant> noNodesDownSince,
Optional<ConvergenceSummary> convergenceSummary, Optional<X509Certificate> testerCertificate) {
this.id = id;
this.steps = Collections.unmodifiableMap(new EnumMap<>(steps));
this.versions = versions;
+ this.isRedeployment = isRedeployment;
this.start = start;
this.end = end;
this.status = status;
@@ -56,10 +58,10 @@ public class Run {
this.testerCertificate = testerCertificate;
}
- public static Run initial(RunId id, Versions versions, Instant now, JobProfile profile) {
+ public static Run initial(RunId id, Versions versions, boolean isRedeployment, Instant now, JobProfile profile) {
EnumMap<Step, StepInfo> steps = new EnumMap<>(Step.class);
profile.steps().forEach(step -> steps.put(step, StepInfo.initial(step)));
- return new Run(id, steps, requireNonNull(versions), requireNonNull(now), Optional.empty(), running,
+ return new Run(id, steps, requireNonNull(versions), isRedeployment, requireNonNull(now), Optional.empty(), running,
-1, Instant.EPOCH, Optional.empty(), Optional.empty(), Optional.empty());
}
@@ -73,7 +75,7 @@ public class Run {
EnumMap<Step, StepInfo> steps = new EnumMap<>(this.steps);
steps.put(step.get(), stepInfo.with(Step.Status.of(status)));
- return new Run(id, steps, versions, start, end, this.status == running ? status : this.status,
+ return new Run(id, steps, versions, isRedeployment, start, end, this.status == running ? status : this.status,
lastTestRecord, lastVespaLogTimestamp, noNodesDownSince, convergenceSummary, testerCertificate);
}
@@ -88,49 +90,49 @@ public class Run {
EnumMap<Step, StepInfo> steps = new EnumMap<>(this.steps);
steps.put(step.get(), stepInfo.with(startTime));
- return new Run(id, steps, versions, start, end, status, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, status, lastTestRecord, lastVespaLogTimestamp,
noNodesDownSince, convergenceSummary, testerCertificate);
}
public Run finished(Instant now) {
requireActive();
- return new Run(id, steps, versions, start, Optional.of(now), status == running ? success : status,
+ return new Run(id, steps, versions, isRedeployment, start, Optional.of(now), status == running ? success : status,
lastTestRecord, lastVespaLogTimestamp, noNodesDownSince, convergenceSummary, Optional.empty());
}
public Run aborted() {
requireActive();
- return new Run(id, steps, versions, start, end, aborted, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, aborted, lastTestRecord, lastVespaLogTimestamp,
noNodesDownSince, convergenceSummary, testerCertificate);
}
public Run with(long lastTestRecord) {
requireActive();
- return new Run(id, steps, versions, start, end, status, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, status, lastTestRecord, lastVespaLogTimestamp,
noNodesDownSince, convergenceSummary, testerCertificate);
}
public Run with(Instant lastVespaLogTimestamp) {
requireActive();
- return new Run(id, steps, versions, start, end, status, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, status, lastTestRecord, lastVespaLogTimestamp,
noNodesDownSince, convergenceSummary, testerCertificate);
}
public Run noNodesDownSince(Instant noNodesDownSince) {
requireActive();
- return new Run(id, steps, versions, start, end, status, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, status, lastTestRecord, lastVespaLogTimestamp,
Optional.ofNullable(noNodesDownSince), convergenceSummary, testerCertificate);
}
public Run withSummary(ConvergenceSummary convergenceSummary) {
requireActive();
- return new Run(id, steps, versions, start, end, status, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, status, lastTestRecord, lastVespaLogTimestamp,
noNodesDownSince, Optional.ofNullable(convergenceSummary), testerCertificate);
}
public Run with(X509Certificate testerCertificate) {
requireActive();
- return new Run(id, steps, versions, start, end, status, lastTestRecord, lastVespaLogTimestamp,
+ return new Run(id, steps, versions, isRedeployment, start, end, status, lastTestRecord, lastVespaLogTimestamp,
noNodesDownSince, convergenceSummary, Optional.of(testerCertificate));
}
@@ -222,6 +224,11 @@ public class Run {
return testerCertificate;
}
+ /** Whether this is a automatic redeployment. */
+ public boolean isRedeployment() {
+ return isRedeployment;
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) return true;
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java
index 91dfed500e3..56bf870c7fc 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/ControllerMaintenance.java
@@ -42,6 +42,7 @@ public class ControllerMaintenance extends AbstractComponent {
maintainers.add(upgrader);
maintainers.addAll(osUpgraders(controller, intervals.osUpgrader));
maintainers.add(new DeploymentExpirer(controller, intervals.defaultInterval));
+ maintainers.add(new DeploymentUpgrader(controller, intervals.defaultInterval));
maintainers.add(new DeploymentIssueReporter(controller, controller.serviceRegistry().deploymentIssues(), intervals.defaultInterval));
maintainers.add(new MetricsReporter(controller, metric));
maintainers.add(new OutstandingChangeDeployer(controller, intervals.outstandingChangeDeployer));
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirer.java
index 9e3da506ca8..76fb63393e6 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirer.java
@@ -62,9 +62,11 @@ public class DeploymentExpirer extends ControllerMaintainer {
.map(type -> new JobId(instance, type));
if (jobId.isEmpty()) return false;
- return controller().jobController().last(jobId.get())
- .flatMap(Run::end)
- .map(end -> end.plus(ttl.get()).isBefore(controller().clock().instant()))
+ return controller().jobController().jobStatus(jobId.get())
+ .runs().descendingMap().values().stream()
+ .filter(run -> ! run.isRedeployment())
+ .findFirst()
+ .map(run -> run.start().plus(ttl.get()).isBefore(controller().clock().instant()))
.orElse(false);
}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java
new file mode 100644
index 00000000000..d40ad8e9f6c
--- /dev/null
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgrader.java
@@ -0,0 +1,93 @@
+// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.controller.maintenance;
+
+import com.yahoo.vespa.hosted.controller.Application;
+import com.yahoo.vespa.hosted.controller.Controller;
+import com.yahoo.vespa.hosted.controller.Instance;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.ApplicationVersion;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobId;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
+import com.yahoo.vespa.hosted.controller.application.Deployment;
+import com.yahoo.vespa.hosted.controller.deployment.Versions;
+import com.yahoo.yolean.Exceptions;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+
+/**
+ * Upgrades instances in manually deployed zones to the system version, at a convenient time.
+ *
+ * @author jonmv
+ */
+public class DeploymentUpgrader extends ControllerMaintainer {
+
+ public DeploymentUpgrader(Controller controller, Duration interval) {
+ super(controller, interval);
+ }
+
+ @Override
+ protected double maintain() {
+ AtomicInteger attempts = new AtomicInteger();
+ AtomicInteger failures = new AtomicInteger();
+ Versions target = new Versions(controller().readSystemVersion(), ApplicationVersion.unknown, Optional.empty(), Optional.empty());
+ for (Application application : controller().applications().readable())
+ for (Instance instance : application.instances().values())
+ for (Deployment deployment : instance.deployments().values())
+ try {
+ attempts.incrementAndGet();
+ JobId job = new JobId(instance.id(), JobType.from(controller().system(), deployment.zone()).get());
+ if ( ! deployment.zone().environment().isManuallyDeployed()) continue;
+ if ( ! deployment.version().isBefore(target.targetPlatform())) continue;
+ if ( controller().clock().instant().isBefore(controller().jobController().last(job).get().start().plus(Duration.ofDays(1)))) continue;
+ if ( ! isLikelyNightFor(job)) continue;
+
+ log.log(Level.FINE, "Upgrading deployment of " + instance.id() + " in " + deployment.zone());
+ controller().jobController().start(instance.id(), JobType.from(controller().system(), deployment.zone()).get(), target, true);
+ } catch (Exception e) {
+ failures.incrementAndGet();
+ log.log(Level.WARNING, "Failed upgrading " + deployment + " of " + instance +
+ ": " + Exceptions.toMessageString(e) + ". Retrying in " +
+ interval());
+ }
+ return asSuccessFactor(attempts.get(), failures.get());
+ }
+
+ private boolean isLikelyNightFor(JobId job) {
+ int hour = hourOf(controller().clock().instant());
+ int[] runStarts = controller().jobController().runs(job).descendingMap().values().stream()
+ .filter(run -> ! run.isRedeployment())
+ .mapToInt(run -> hourOf(run.start()))
+ .toArray();
+ int localNight = mostLikelyWeeHour(runStarts);
+ return Math.abs(hour - localNight) <= 1;
+ }
+
+ static int mostLikelyWeeHour(int[] starts) {
+ double weight = 1; // Weight more recent deployments higher.
+ double[] buckets = new double[24];
+ for (int start : starts)
+ buckets[start] += weight *= (Math.sqrt(5) - 1) * 0.5; // When in doubt, use the golden ratio.
+
+ int best = -1;
+ double min = Double.MAX_VALUE;
+ for (int i = 12; i < 36; i++) {
+ double sum = 0;
+ for (int j = -12; j < 12; j++)
+ sum += Math.abs(j) * buckets[(i + j) % 24];
+
+ if (sum < min) {
+ min = sum;
+ best = i;
+ }
+ }
+ return (best + 13) % 24; // rot13 of weighted average deployment start is likely in the middle of the night.
+ }
+
+ private static int hourOf(Instant instant) {
+ return (int) (instant.toEpochMilli() / 3_600_000 % 24);
+ }
+
+}
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializer.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializer.java
index 87527085237..8ffa4823ead 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializer.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializer.java
@@ -83,6 +83,7 @@ class RunSerializer {
private static final String endField = "end";
private static final String statusField = "status";
private static final String versionsField = "versions";
+ private static final String isRedeploymentField = "isRedeployment";
private static final String platformVersionField = "platform";
private static final String repositoryField = "repository";
private static final String branchField = "branch";
@@ -131,6 +132,7 @@ class RunSerializer {
runObject.field(numberField).asLong()),
steps,
versionsFromSlime(runObject.field(versionsField)),
+ runObject.field(isRedeploymentField).asBool(),
SlimeUtils.instant(runObject.field(startField)),
SlimeUtils.optionalInstant(runObject.field(endField)),
runStatusOf(runObject.field(statusField).asString()),
@@ -177,7 +179,7 @@ class RunSerializer {
compileVersion, buildTime, sourceUrl, commit);
}
- // Don't change this — introduce a separate array instead.
+ // Don't change this — introduce a separate array instead.
private Optional<ConvergenceSummary> convergenceSummaryFrom(Inspector summaryArray) {
if ( ! summaryArray.valid()) return Optional.empty();
@@ -215,6 +217,7 @@ class RunSerializer {
private void toSlime(Run run, Cursor runObject) {
runObject.setString(applicationField, run.id().application().serializedForm());
runObject.setString(jobTypeField, run.id().type().jobName());
+ runObject.setBool(isRedeploymentField, run.isRedeployment());
runObject.setLong(numberField, run.id().number());
runObject.setLong(startField, run.start().toEpochMilli());
run.end().ifPresent(end -> runObject.setLong(endField, end.toEpochMilli()));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java
index ffc82f90ad4..31f8aaf9e2d 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentExpirerTest.java
@@ -70,7 +70,9 @@ public class DeploymentExpirerTest {
assertEquals(1, permanentDeployments(prodApp.instance()));
// Dev application expires when enough time has passed since most recent attempt
+ // Redeployments done by DeploymentUpgrader do not affect this
tester.clock().advance(Duration.ofDays(12).plus(Duration.ofSeconds(1)));
+ tester.jobs().start(devApp.instanceId(), JobType.devUsEast1, lastRun.versions(), true);
expirer.maintain();
assertEquals(0, permanentDeployments(devApp.instance()));
assertEquals(1, permanentDeployments(prodApp.instance()));
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgraderTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgraderTest.java
new file mode 100644
index 00000000000..b4a78e55f62
--- /dev/null
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/DeploymentUpgraderTest.java
@@ -0,0 +1,101 @@
+// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.controller.maintenance;
+
+import com.yahoo.component.Version;
+import com.yahoo.config.provision.Environment;
+import com.yahoo.config.provision.RegionName;
+import com.yahoo.config.provision.zone.ZoneId;
+import com.yahoo.vespa.hosted.controller.Instance;
+import com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType;
+import com.yahoo.vespa.hosted.controller.application.ApplicationPackage;
+import com.yahoo.vespa.hosted.controller.application.Deployment;
+import com.yahoo.vespa.hosted.controller.deployment.ApplicationPackageBuilder;
+import com.yahoo.vespa.hosted.controller.deployment.DeploymentTester;
+import com.yahoo.vespa.hosted.controller.deployment.Run;
+import com.yahoo.vespa.hosted.controller.deployment.RunStatus;
+import org.junit.Test;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.TemporalUnit;
+import java.util.Optional;
+
+import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.devUsEast1;
+import static com.yahoo.vespa.hosted.controller.api.integration.deployment.JobType.productionUsWest1;
+import static com.yahoo.vespa.hosted.controller.maintenance.DeploymentUpgrader.mostLikelyWeeHour;
+import static java.time.temporal.ChronoUnit.MILLIS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author jonmv
+ */
+public class DeploymentUpgraderTest {
+
+ private final DeploymentTester tester = new DeploymentTester();
+
+ @Test
+ public void testDeploymentUpgrading() {
+ ZoneId devZone = ZoneId.from(Environment.dev, RegionName.from("us-east-1"));
+ DeploymentUpgrader upgrader = new DeploymentUpgrader(tester.controller(), Duration.ofDays(1));
+ var devApp = tester.newDeploymentContext("tenant1", "app1", "default");
+ var prodApp = tester.newDeploymentContext("tenant2", "app2", "default");
+
+ ApplicationPackage appPackage = new ApplicationPackageBuilder().region("us-west-1").build();
+ Version systemVersion = tester.controller().readSystemVersion();
+ Instant start = tester.clock().instant().truncatedTo(MILLIS);
+
+ devApp.runJob(devUsEast1, appPackage);
+ prodApp.submit(appPackage).deploy();
+ assertEquals(systemVersion, tester.jobs().last(devApp.instanceId(), devUsEast1).get().versions().targetPlatform());
+ assertEquals(systemVersion, tester.jobs().last(prodApp.instanceId(), productionUsWest1).get().versions().targetPlatform());
+
+ // Not upgraded initially
+ upgrader.maintain();
+ assertEquals(start, tester.jobs().last(devApp.instanceId(), devUsEast1).get().start());
+ assertEquals(start, tester.jobs().last(prodApp.instanceId(), productionUsWest1).get().start());
+
+ // Not upgraded immediately after system upgrades
+ tester.controllerTester().upgradeSystem(new Version(7, 8, 9));
+ upgrader.maintain();
+ assertEquals(start, tester.jobs().last(devApp.instanceId(), devUsEast1).get().start());
+ assertEquals(start, tester.jobs().last(prodApp.instanceId(), productionUsWest1).get().start());
+
+ // 13 hours pass, but not upgraded before a day has passed since last deployment
+ tester.clock().advance(Duration.ofHours(13));
+ upgrader.maintain();
+ assertEquals(start, tester.jobs().last(devApp.instanceId(), devUsEast1).get().start());
+ assertEquals(start, tester.jobs().last(prodApp.instanceId(), productionUsWest1).get().start());
+
+ // 34 hours pass, but not upgraded since it's not likely in the middle of the night
+ tester.clock().advance(Duration.ofHours(21));
+ upgrader.maintain();
+ assertEquals(start, tester.jobs().last(devApp.instanceId(), devUsEast1).get().start());
+ assertEquals(start, tester.jobs().last(prodApp.instanceId(), productionUsWest1).get().start());
+
+ // 37 hours pass, and the dev deployment, only, is upgraded
+ tester.clock().advance(Duration.ofHours(3));
+ upgrader.maintain();
+ assertEquals(tester.clock().instant().truncatedTo(MILLIS), tester.jobs().last(devApp.instanceId(), devUsEast1).get().start());
+ assertTrue(tester.jobs().last(devApp.instanceId(), devUsEast1).get().isRedeployment());
+ assertEquals(start, tester.jobs().last(prodApp.instanceId(), productionUsWest1).get().start());
+ devApp.runJob(devUsEast1);
+
+ // After the upgrade, the dev app is mostly (re)deployed to at night, but this doesn't affect what is likely the night.
+ tester.controllerTester().upgradeSystem(new Version(7, 9, 11));
+ tester.clock().advance(Duration.ofHours(48));
+ upgrader.maintain();
+ assertEquals(tester.clock().instant().truncatedTo(MILLIS), tester.jobs().last(devApp.instanceId(), devUsEast1).get().start());
+ }
+
+ @Test
+ public void testNight() {
+ assertEquals(14, mostLikelyWeeHour(new int[]{ 0, 1, 2, 3, 4, 5, 6 }));
+ assertEquals(12, mostLikelyWeeHour(new int[]{ 22, 23, 0, 1, 2, 3, 4 }));
+ assertEquals(18, mostLikelyWeeHour(new int[]{ 6, 5, 4, 3, 2, 1, 0 }));
+ assertEquals(13, mostLikelyWeeHour(new int[]{ 0, 12, 0, 12, 0, 12, 0, 12, 0, 12, 0, 12 }));
+ }
+
+}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializerTest.java
index 6e12373640d..03a050db74e 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/persistence/RunSerializerTest.java
@@ -149,7 +149,7 @@ public class RunSerializerTest {
assertEquals(run.versions(), phoenix.versions());
assertEquals(run.steps(), phoenix.steps());
- Run initial = Run.initial(id, run.versions(), run.start(), JobProfile.production);
+ Run initial = Run.initial(id, run.versions(), run.isRedeployment(), run.start(), JobProfile.production);
assertEquals(initial, serializer.runFromSlime(serializer.toSlime(initial)));
}