summaryrefslogtreecommitdiffstats
path: root/node-repository
diff options
context:
space:
mode:
authorValerij Fredriksen <valerijf@verizonmedia.com>2019-05-09 11:26:40 +0200
committerValerij Fredriksen <valerijf@verizonmedia.com>2019-05-09 11:29:00 +0200
commit68ce1d964984d11d50aed462751d0d9c7b80fb51 (patch)
tree966730757d274e42633714c9857d75d71267dc2f /node-repository
parentbd428eebc479f125cc1bbc286aa6c30ae246ad92 (diff)
Use InfraDeployer in InfrastructureProvisioner
Diffstat (limited to 'node-repository')
-rw-r--r--node-repository/src/main/config/node-repository.xml1
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisioner.java93
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/NodeRepositoryMaintenance.java16
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/ContainerConfig.java1
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/MockInfraDeployer.java20
-rw-r--r--node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisionerTest.java244
6 files changed, 39 insertions, 336 deletions
diff --git a/node-repository/src/main/config/node-repository.xml b/node-repository/src/main/config/node-repository.xml
index 2bc3ae5cfeb..274be6d572a 100644
--- a/node-repository/src/main/config/node-repository.xml
+++ b/node-repository/src/main/config/node-repository.xml
@@ -1,5 +1,6 @@
<!-- services.xml snippet for the node repository. Included in config server services.xml if the package is installed-->
<!-- Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root. -->
+<component id="com.yahoo.vespa.hosted.provision.provisioning.InfraDeployerImpl" bundle="node-repository"/>
<component id="com.yahoo.vespa.hosted.provision.provisioning.NodeRepositoryProvisioner" bundle="node-repository" />
<component id="NodeRepository" class="com.yahoo.vespa.hosted.provision.NodeRepository" bundle="node-repository"/>
<component id="com.yahoo.vespa.hosted.provision.maintenance.NodeRepositoryMaintenance" bundle="node-repository"/>
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisioner.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisioner.java
index 32a971b0d46..9ab816b8acc 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisioner.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisioner.java
@@ -1,31 +1,17 @@
// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.provision.maintenance;
-import com.yahoo.component.Version;
import com.yahoo.config.provision.ApplicationId;
-import com.yahoo.config.provision.HostName;
-import com.yahoo.config.provision.HostSpec;
-import com.yahoo.config.provision.NodeType;
-import com.yahoo.config.provision.Provisioner;
+import com.yahoo.config.provision.InfraDeployer;
import com.yahoo.log.LogLevel;
-import com.yahoo.transaction.Mutex;
-import com.yahoo.transaction.NestedTransaction;
-import com.yahoo.vespa.hosted.provision.Node;
import com.yahoo.vespa.hosted.provision.NodeRepository;
-import com.yahoo.vespa.service.duper.DuperModel;
-import com.yahoo.vespa.service.monitor.DuperModelInfraApi;
-import com.yahoo.vespa.service.monitor.InfraApplicationApi;
import java.time.Duration;
-import java.util.List;
-import java.util.Optional;
import java.util.logging.Logger;
-import java.util.stream.Collectors;
/**
- * This maintainer makes sure that 1. infrastructure nodes are allocated with correct wanted
- * version and 2. keeping {@link DuperModel} up to date. Source for the wanted version comes
- * from the target version set using /nodes/v2/upgrade/ endpoint.
+ * Periodically deploys infrastructure applications.
+ * TODO: Merge this with {@link PeriodicApplicationMaintainer}
*
* @author freva
*/
@@ -33,81 +19,22 @@ public class InfrastructureProvisioner extends Maintainer {
private static final Logger logger = Logger.getLogger(InfrastructureProvisioner.class.getName());
- private final Provisioner provisioner;
- private final DuperModelInfraApi duperModel;
+ private final InfraDeployer infraDeployer;
- InfrastructureProvisioner(Provisioner provisioner, NodeRepository nodeRepository,
- Duration interval, DuperModelInfraApi duperModel) {
+ InfrastructureProvisioner(NodeRepository nodeRepository, InfraDeployer infraDeployer, Duration interval) {
super(nodeRepository, interval);
- this.provisioner = provisioner;
- this.duperModel = duperModel;
+ this.infraDeployer = infraDeployer;
}
@Override
protected void maintain() {
- for (InfraApplicationApi application: duperModel.getSupportedInfraApplications()) {
- try (Mutex lock = nodeRepository().lock(application.getApplicationId())) {
- NodeType nodeType = application.getCapacity().type();
-
- Optional<Version> targetVersion = nodeRepository().infrastructureVersions().getTargetVersionFor(nodeType);
- if (!targetVersion.isPresent()) {
- logger.log(LogLevel.DEBUG, "No target version set for " + nodeType + ", removing application");
- removeApplication(application.getApplicationId());
- continue;
- }
-
- List<Node> candidateNodes = nodeRepository()
- .getNodes(nodeType, Node.State.ready, Node.State.reserved, Node.State.active, Node.State.inactive);
- if (candidateNodes.isEmpty()) {
- logger.log(LogLevel.DEBUG, "No nodes to provision for " + nodeType + ", removing application");
- removeApplication(application.getApplicationId());
- continue;
- }
-
- if (!candidateNodes.stream().allMatch(node ->
- node.state() == Node.State.active &&
- node.allocation()
- .map(allocation -> allocation.membership().cluster().vespaVersion().equals(targetVersion.get()))
- .orElse(false))) {
- List<HostSpec> hostSpecs = provisioner.prepare(
- application.getApplicationId(),
- application.getClusterSpecWithVersion(targetVersion.get()),
- application.getCapacity(),
- 1, // groups
- logger::log);
-
- // Sanity-check hostSpecs is the same list as candidateNodes?
-
- NestedTransaction nestedTransaction = new NestedTransaction();
- provisioner.activate(nestedTransaction, application.getApplicationId(), hostSpecs);
- nestedTransaction.commit();
- }
-
- duperModel.infraApplicationActivated(
- application.getApplicationId(),
- candidateNodes.stream().map(Node::hostname).map(HostName::from).collect(Collectors.toList()));
-
- String detail;
- if (candidateNodes.size() < 10) {
- detail = ": " + candidateNodes.stream().map(Node::hostname).collect(Collectors.joining(","));
- } else {
- detail = " with " + candidateNodes.size() + " hosts";
- }
- logger.log(LogLevel.DEBUG, "Infrastructure application " + application.getApplicationId() + " activated" + detail);
+ for (ApplicationId application : infraDeployer.getSupportedInfraApplications()) {
+ try {
+ infraDeployer.getDeployment(application).orElseThrow().activate();
} catch (RuntimeException e) {
- logger.log(LogLevel.INFO, "Failed to activate " + application.getApplicationId(), e);
+ logger.log(LogLevel.INFO, "Failed to activate " + application, e);
// loop around to activate the next application
}
}
}
-
- private void removeApplication(ApplicationId applicationId) {
- // Use the DuperModel as source-of-truth on whether it has also been activated (to avoid periodic removals)
- if (duperModel.infraApplicationIsActive(applicationId)) {
- NestedTransaction nestedTransaction = new NestedTransaction();
- provisioner.remove(nestedTransaction, applicationId);
- nestedTransaction.commit();
- duperModel.infraApplicationRemoved(applicationId);
- }
- }
}
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/NodeRepositoryMaintenance.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/NodeRepositoryMaintenance.java
index 4cfecb6c6aa..50a371458ce 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/NodeRepositoryMaintenance.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/NodeRepositoryMaintenance.java
@@ -7,7 +7,7 @@ import com.yahoo.component.AbstractComponent;
import com.yahoo.config.provision.Deployer;
import com.yahoo.config.provision.Environment;
import com.yahoo.config.provision.HostLivenessTracker;
-import com.yahoo.config.provision.Provisioner;
+import com.yahoo.config.provision.InfraDeployer;
import com.yahoo.config.provision.SystemName;
import com.yahoo.config.provision.Zone;
import com.yahoo.jdisc.Metric;
@@ -20,7 +20,6 @@ import com.yahoo.vespa.hosted.provision.provisioning.FlavorSpareChecker;
import com.yahoo.vespa.hosted.provision.provisioning.FlavorSpareCount;
import com.yahoo.vespa.hosted.provision.provisioning.ProvisionServiceProvider;
import com.yahoo.vespa.orchestrator.Orchestrator;
-import com.yahoo.vespa.service.monitor.DuperModelInfraApi;
import com.yahoo.vespa.service.monitor.ServiceMonitor;
import java.time.Clock;
@@ -57,21 +56,20 @@ public class NodeRepositoryMaintenance extends AbstractComponent {
private final Optional<HostDeprovisionMaintainer> hostDeprovisionMaintainer;
@Inject
- public NodeRepositoryMaintenance(NodeRepository nodeRepository, Deployer deployer, Provisioner provisioner,
+ public NodeRepositoryMaintenance(NodeRepository nodeRepository, Deployer deployer, InfraDeployer infraDeployer,
HostLivenessTracker hostLivenessTracker, ServiceMonitor serviceMonitor,
Zone zone, Orchestrator orchestrator, Metric metric,
ConfigserverConfig configserverConfig,
- DuperModelInfraApi duperModelInfraApi,
ProvisionServiceProvider provisionServiceProvider,
FlagSource flagSource) {
- this(nodeRepository, deployer, provisioner, hostLivenessTracker, serviceMonitor, zone, Clock.systemUTC(),
- orchestrator, metric, configserverConfig, duperModelInfraApi, provisionServiceProvider, flagSource);
+ this(nodeRepository, deployer, infraDeployer, hostLivenessTracker, serviceMonitor, zone, Clock.systemUTC(),
+ orchestrator, metric, configserverConfig, provisionServiceProvider, flagSource);
}
- public NodeRepositoryMaintenance(NodeRepository nodeRepository, Deployer deployer, Provisioner provisioner,
+ public NodeRepositoryMaintenance(NodeRepository nodeRepository, Deployer deployer, InfraDeployer infraDeployer,
HostLivenessTracker hostLivenessTracker, ServiceMonitor serviceMonitor,
Zone zone, Clock clock, Orchestrator orchestrator, Metric metric,
- ConfigserverConfig configserverConfig, DuperModelInfraApi duperModelInfraApi,
+ ConfigserverConfig configserverConfig,
ProvisionServiceProvider provisionServiceProvider, FlagSource flagSource) {
DefaultTimes defaults = new DefaultTimes(zone);
@@ -86,7 +84,7 @@ public class NodeRepositoryMaintenance extends AbstractComponent {
provisionedExpirer = new ProvisionedExpirer(nodeRepository, clock, durationFromEnv("provisioned_expiry").orElse(defaults.provisionedExpiry));
nodeRebooter = new NodeRebooter(nodeRepository, clock, durationFromEnv("reboot_interval").orElse(defaults.rebootInterval));
metricsReporter = new MetricsReporter(nodeRepository, metric, orchestrator, serviceMonitor, periodicApplicationMaintainer::pendingDeployments, durationFromEnv("metrics_interval").orElse(defaults.metricsInterval));
- infrastructureProvisioner = new InfrastructureProvisioner(provisioner, nodeRepository, durationFromEnv("infrastructure_provision_interval").orElse(defaults.infrastructureProvisionInterval), duperModelInfraApi);
+ infrastructureProvisioner = new InfrastructureProvisioner(nodeRepository, infraDeployer, durationFromEnv("infrastructure_provision_interval").orElse(defaults.infrastructureProvisionInterval));
loadBalancerExpirer = provisionServiceProvider.getLoadBalancerService().map(lbService ->
new LoadBalancerExpirer(nodeRepository, durationFromEnv("load_balancer_expiry").orElse(defaults.loadBalancerExpiry), lbService));
hostProvisionMaintainer = provisionServiceProvider.getHostProvisioner().map(hostProvisioner ->
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/ContainerConfig.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/ContainerConfig.java
index 2a07cadc6ad..71a9cfa4985 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/ContainerConfig.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/ContainerConfig.java
@@ -18,6 +18,7 @@ public class ContainerConfig {
" <component id='com.yahoo.vespa.curator.mock.MockCurator'/>\n" +
" <component id='com.yahoo.vespa.hosted.provision.testutils.OrchestratorMock'/>\n" +
" <component id='com.yahoo.vespa.hosted.provision.testutils.MockDeployer'/>\n" +
+ " <component id='com.yahoo.vespa.hosted.provision.testutils.MockInfraDeployer'/>\n" +
" <component id='com.yahoo.vespa.hosted.provision.testutils.MockProvisioner'/>\n" +
" <component id='com.yahoo.vespa.hosted.provision.testutils.TestHostLivenessTracker'/>\n" +
" <component id='com.yahoo.vespa.hosted.provision.testutils.ServiceMonitorStub'/>\n" +
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/MockInfraDeployer.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/MockInfraDeployer.java
new file mode 100644
index 00000000000..5a5f8431653
--- /dev/null
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/testutils/MockInfraDeployer.java
@@ -0,0 +1,20 @@
+package com.yahoo.vespa.hosted.provision.testutils;
+
+import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.config.provision.Deployment;
+import com.yahoo.config.provision.InfraDeployer;
+
+import java.util.List;
+import java.util.Optional;
+
+public class MockInfraDeployer implements InfraDeployer {
+ @Override
+ public Optional<Deployment> getDeployment(ApplicationId application) {
+ return Optional.empty();
+ }
+
+ @Override
+ public List<ApplicationId> getSupportedInfraApplications() {
+ return List.of();
+ }
+}
diff --git a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisionerTest.java b/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisionerTest.java
deleted file mode 100644
index 965a62770c6..00000000000
--- a/node-repository/src/test/java/com/yahoo/vespa/hosted/provision/maintenance/InfrastructureProvisionerTest.java
+++ /dev/null
@@ -1,244 +0,0 @@
-// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.provision.maintenance;
-
-import com.yahoo.component.Version;
-import com.yahoo.config.provision.ClusterMembership;
-import com.yahoo.config.provision.ClusterSpec;
-import com.yahoo.config.provision.HostName;
-import com.yahoo.config.provision.HostSpec;
-import com.yahoo.config.provision.NodeType;
-import com.yahoo.config.provision.Provisioner;
-import com.yahoo.vespa.hosted.provision.Node;
-import com.yahoo.vespa.hosted.provision.NodeRepository;
-import com.yahoo.vespa.hosted.provision.NodeRepositoryTester;
-import com.yahoo.vespa.hosted.provision.node.Agent;
-import com.yahoo.vespa.hosted.provision.node.Allocation;
-import com.yahoo.vespa.hosted.provision.node.Generation;
-import com.yahoo.vespa.service.duper.ConfigServerApplication;
-import com.yahoo.vespa.service.duper.ControllerApplication;
-import com.yahoo.vespa.service.monitor.DuperModelInfraApi;
-import com.yahoo.vespa.service.monitor.InfraApplicationApi;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Optional;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * @author freva
- */
-@RunWith(Parameterized.class)
-public class InfrastructureProvisionerTest {
-
- @Parameters(name = "application={0}")
- public static Iterable<Object[]> parameters() {
- return Arrays.asList(
- new InfraApplicationApi[]{new ConfigServerApplication()},
- new InfraApplicationApi[]{new ControllerApplication()}
- );
- }
-
- private final NodeRepositoryTester tester = new NodeRepositoryTester();
- private final Provisioner provisioner = mock(Provisioner.class);
- private final NodeRepository nodeRepository = tester.nodeRepository();
- private final InfrastructureVersions infrastructureVersions = nodeRepository.infrastructureVersions();
- private final DuperModelInfraApi duperModelInfraApi = mock(DuperModelInfraApi.class);
- private final InfrastructureProvisioner infrastructureProvisioner = new InfrastructureProvisioner(
- provisioner, nodeRepository, Duration.ofDays(99), duperModelInfraApi);
- private final HostName node1 = HostName.from("node-1");
- private final HostName node2 = HostName.from("node-2");
- private final HostName node3 = HostName.from("node-3");
- private final Version target = Version.fromString("6.123.456");
- private final Version oldVersion = Version.fromString("6.122.333");
-
- private final InfraApplicationApi application;
- private final NodeType nodeType;
-
- public InfrastructureProvisionerTest(InfraApplicationApi application) {
- this.application = application;
- this.nodeType = application.getCapacity().type();
- when(duperModelInfraApi.getSupportedInfraApplications()).thenReturn(Collections.singletonList(application));
- }
-
- @Test
- public void remove_application_if_without_target_version() {
- addNode(1, Node.State.active, Optional.of(target));
- when(duperModelInfraApi.infraApplicationIsActive(eq(application.getApplicationId()))).thenReturn(true);
- infrastructureProvisioner.maintain();
- verify(duperModelInfraApi).infraApplicationRemoved(application.getApplicationId());
- verifyRemoved(1);
- }
-
- @Test
- public void remove_application_if_without_nodes() {
- remove_application_without_nodes(true);
- }
-
- @Test
- public void skip_remove_unless_active() {
- remove_application_without_nodes(false);
- }
-
- private void remove_application_without_nodes(boolean applicationIsActive) {
- infrastructureVersions.setTargetVersion(nodeType, target, false);
- addNode(1, Node.State.failed, Optional.of(target));
- addNode(2, Node.State.parked, Optional.empty());
- when(duperModelInfraApi.infraApplicationIsActive(eq(application.getApplicationId()))).thenReturn(applicationIsActive);
- infrastructureProvisioner.maintain();
- if (applicationIsActive) {
- verify(duperModelInfraApi).infraApplicationRemoved(application.getApplicationId());
- verifyRemoved(1);
- } else {
- verifyRemoved(0);
- }
- }
-
- @Test
- public void activate_when_no_op() {
- infrastructureVersions.setTargetVersion(nodeType, target, false);
-
- addNode(1, Node.State.failed, Optional.of(oldVersion));
- addNode(2, Node.State.parked, Optional.of(target));
- addNode(3, Node.State.active, Optional.of(target));
- addNode(4, Node.State.inactive, Optional.of(target));
- addNode(5, Node.State.dirty, Optional.empty());
-
- when(duperModelInfraApi.infraApplicationIsActive(eq(application.getApplicationId()))).thenReturn(true);
-
- infrastructureProvisioner.maintain();
- verify(duperModelInfraApi, never()).infraApplicationRemoved(any());
- verify(duperModelInfraApi).infraApplicationActivated(any(), any());
- verify(provisioner).activate(any(), any(), any());
- }
-
- @Test
- public void activates_after_target_has_been_set_the_first_time() {
- infrastructureVersions.setTargetVersion(nodeType, target, false);
-
- addNode(1, Node.State.inactive, Optional.empty());
- addNode(2, Node.State.parked, Optional.empty());
- addNode(3, Node.State.active, Optional.empty());
- addNode(4, Node.State.failed, Optional.empty());
- addNode(5, Node.State.dirty, Optional.empty());
-
- when(provisioner.prepare(any(), any(), any(), anyInt(), any())).thenReturn(Arrays.asList(
- new HostSpec(node1.value(), Collections.emptyList()),
- new HostSpec(node3.value(), Collections.emptyList())));
-
- infrastructureProvisioner.maintain();
-
- verify(provisioner).prepare(eq(application.getApplicationId()), any(), any(), anyInt(), any());
- verify(provisioner).activate(any(), eq(application.getApplicationId()), any());
- verify(duperModelInfraApi).infraApplicationActivated(application.getApplicationId(), Arrays.asList(node3, node1));
- }
-
-
- @Test
- public void always_activates_for_dupermodel() {
- infrastructureVersions.setTargetVersion(nodeType, target, false);
-
- addNode(1, Node.State.active, Optional.of(target));
-
- when(duperModelInfraApi.infraApplicationIsActive(eq(application.getApplicationId()))).thenReturn(false);
- when(provisioner.prepare(any(), any(), any(), anyInt(), any())).thenReturn(Collections.singletonList(
- new HostSpec(node1.value(), Collections.emptyList())));
-
- infrastructureProvisioner.maintain();
-
- verify(provisioner, never()).prepare(any(), any(), any(), anyInt(), any());
- verify(provisioner, never()).activate(any(), any(), any());
- verify(duperModelInfraApi, times(1)).infraApplicationActivated(application.getApplicationId(), Arrays.asList(node1));
-
- infrastructureProvisioner.maintain();
-
- verify(provisioner, never()).prepare(any(), any(), any(), anyInt(), any());
- verify(provisioner, never()).activate(any(), any(), any());
- verify(duperModelInfraApi, times(2)).infraApplicationActivated(application.getApplicationId(), Arrays.asList(node1));
- }
-
- @Test
- public void provision_usable_nodes_on_old_version() {
- infrastructureVersions.setTargetVersion(nodeType, target, false);
-
- addNode(1, Node.State.failed, Optional.of(oldVersion));
- addNode(2, Node.State.inactive, Optional.of(target));
- addNode(3, Node.State.active, Optional.of(oldVersion));
-
- when(duperModelInfraApi.getSupportedInfraApplications()).thenReturn(Collections.singletonList(application));
- when(provisioner.prepare(any(), any(), any(), anyInt(), any())).thenReturn(Arrays.asList(
- new HostSpec(node2.value(), Collections.emptyList()),
- new HostSpec(node3.value(), Collections.emptyList())));
-
- infrastructureProvisioner.maintain();
-
- verify(provisioner).prepare(eq(application.getApplicationId()), any(), any(), anyInt(), any());
- verify(provisioner).activate(any(), eq(application.getApplicationId()), any());
- verify(duperModelInfraApi).infraApplicationActivated(application.getApplicationId(), Arrays.asList(node3, node2));
- }
-
- @Test
- public void provision_with_usable_node_without_version() {
- infrastructureVersions.setTargetVersion(nodeType, target, false);
-
- addNode(1, Node.State.failed, Optional.of(oldVersion));
- addNode(2, Node.State.ready, Optional.empty());
- addNode(3, Node.State.active, Optional.of(target));
-
- when(provisioner.prepare(any(), any(), any(), anyInt(), any()))
- .thenReturn(Arrays.asList(
- new HostSpec(node2.value(), Collections.emptyList()),
- new HostSpec(node3.value(), Collections.emptyList())));
-
- infrastructureProvisioner.maintain();
-
- verify(provisioner).prepare(eq(application.getApplicationId()), any(), any(), anyInt(), any());
- verify(provisioner).activate(any(), eq(application.getApplicationId()), any());
- verify(duperModelInfraApi).infraApplicationActivated(application.getApplicationId(), Arrays.asList(node2, node3));
- }
-
- @Test
- public void avoid_provisioning_if_no_usable_nodes() {
- when(duperModelInfraApi.infraApplicationIsActive(eq(application.getApplicationId()))).thenReturn(true);
- infrastructureVersions.setTargetVersion(nodeType, target, false);
-
- infrastructureProvisioner.maintain();
- verifyRemoved(1);
-
- // Add nodes in non-provisionable states
- addNode(1, Node.State.dirty, Optional.empty());
- addNode(2, Node.State.failed, Optional.empty());
-
- infrastructureProvisioner.maintain();
- verifyRemoved(2);
- }
-
- private void verifyRemoved(int removedCount) {
- verify(provisioner, times(removedCount)).remove(any(), any());
- verify(duperModelInfraApi, times(removedCount)).infraApplicationRemoved(any());
- }
-
- private Node addNode(int id, Node.State state, Optional<Version> wantedVespaVersion) {
- Node node = tester.addNode("id-" + id, "node-" + id, "default", nodeType);
- Optional<Node> nodeWithAllocation = wantedVespaVersion.map(version -> {
- ClusterSpec clusterSpec = ClusterSpec.from(ClusterSpec.Type.admin, new ClusterSpec.Id("clusterid"), ClusterSpec.Group.from(0), version, false, Collections.emptySet());
- ClusterMembership membership = ClusterMembership.from(clusterSpec, 1);
- Allocation allocation = new Allocation(application.getApplicationId(), membership, new Generation(0, 0), false);
- return node.with(allocation);
- });
- return nodeRepository.database().writeTo(state, nodeWithAllocation.orElse(node), Agent.system, Optional.empty());
- }
-
-}