summaryrefslogtreecommitdiffstats
path: root/node-admin
diff options
context:
space:
mode:
authorHaakon Dybdahl <dybdahl@yahoo-inc.com>2016-06-29 08:49:35 +0200
committerHaakon Dybdahl <dybdahl@yahoo-inc.com>2016-06-29 11:17:01 +0200
commitcb87ddefda4f9c029092c5056e355e7bd261adbe (patch)
tree72720507d1760c933ca68e9c1816f97c060e0896 /node-admin
parent86df00e5eca27806b255a0a8fb064eaa88f610fd (diff)
Refactor agent to smaller methods and classes.
Diffstat (limited to 'node-admin')
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImpl.java442
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdmin.java (renamed from node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdmin.java)4
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java (renamed from node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdminImpl.java)12
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminStateUpdater.java (renamed from node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdminStateUpdater.java)9
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgent.java (renamed from node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgent.java)17
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImpl.java265
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAttributes.java48
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDocker.java236
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProvider.java2
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProviderImpl.java13
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/restapi/RestApiHandler.java2
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ComponentsProviderWithMocks.java14
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerFailTest.java15
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerMock.java1
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/MultiDockerTest.java15
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeRepoMock.java1
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeStateTest.java16
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/OrchestratorMock.java1
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ResumeTest.java14
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/RunInContainerTest.java1
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java (renamed from node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAdminImplTest.java)14
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminStateUpdaterTest.java (renamed from node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAdminStateUpdaterTest.java)5
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImplTest.java (renamed from node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImplTest.java)261
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDockerTest.java83
24 files changed, 862 insertions, 629 deletions
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImpl.java
deleted file mode 100644
index 62a375e6ae1..00000000000
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImpl.java
+++ /dev/null
@@ -1,442 +0,0 @@
-// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin;
-
-import com.yahoo.log.LogLevel;
-import com.yahoo.vespa.applicationmodel.HostName;
-import com.yahoo.vespa.defaults.Defaults;
-import com.yahoo.vespa.hosted.node.admin.docker.Container;
-import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
-import com.yahoo.vespa.hosted.node.admin.docker.Docker;
-import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
-import com.yahoo.vespa.hosted.node.admin.docker.ProcessResult;
-import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepository;
-import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
-import com.yahoo.vespa.hosted.node.admin.orchestrator.Orchestrator;
-import com.yahoo.vespa.hosted.node.admin.orchestrator.OrchestratorException;
-
-import javax.annotation.concurrent.GuardedBy;
-import java.util.Arrays;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Supplier;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * @author bakksjo
- */
-public class NodeAgentImpl implements NodeAgent {
- private static final Logger logger = Logger.getLogger(NodeAgentImpl.class.getName());
- static final String NODE_PROGRAM = Defaults.getDefaults().vespaHome() + "bin/vespa-nodectl";
- private static final String[] RESUME_NODE_COMMAND = new String[] {NODE_PROGRAM, "resume"};
- private static final String[] SUSPEND_NODE_COMMAND = new String[] {NODE_PROGRAM, "suspend"};
-
- private final String logPrefix;
- private final HostName hostname;
-
- private final Docker docker;
- private final NodeRepository nodeRepository;
- private final Orchestrator orchestrator;
-
- private final Thread thread;
-
- private final Object monitor = new Object();
- @GuardedBy("monitor")
- private State state = State.WAITING;
- @GuardedBy("monitor")
- private State wantedState = State.WAITING;
-
- // The attributes of the last successful noderepo attribute update for this node. Used to avoid redundant calls.
- // Only used internally by maintenance thread; no synchronization necessary.
- private NodeAttributes lastAttributesSet = null;
- // Whether we have successfully started the node using the node program. Used to avoid redundant start calls.
- private boolean nodeStarted = false;
-
-
-
-
- /**
- * @param hostName the hostname of the node managed by this agent
- * @param docker interface to docker daemon and docker-related tasks
- * @param nodeRepository interface to (remote) node repository
- * @param orchestrator interface to (remote) orchestrator
- */
- public NodeAgentImpl(
- final HostName hostName,
- final Docker docker,
- final NodeRepository nodeRepository,
- final Orchestrator orchestrator) {
- this.logPrefix = "NodeAgent(" + hostName + "): ";
- this.docker = docker;
- this.nodeRepository = nodeRepository;
- this.orchestrator = orchestrator;
- this.thread = new Thread(this::maintainWantedState, "Node Agent maintenance thread for node " + hostName);
- this.hostname = hostName;
- }
-
- @Override
- public void execute(Command command) {
- synchronized (monitor) {
- switch (command) {
- case UPDATE_FROM_NODE_REPO:
- wantedState = State.WORKING;
- break;
- case FREEZE:
- wantedState = State.FROZEN;
- break;
- case UNFREEZE:
- wantedState = State.WORKING;
- break;
- }
- monitor.notifyAll();
- }
- }
-
- @Override
- public State getState() {
- synchronized (monitor) {
- return state;
- }
- }
-
- @Override
- public void start() {
- logger.log(LogLevel.INFO, logPrefix + "Scheduling start of NodeAgent");
- synchronized (monitor) {
- if (state == State.TERMINATED) {
- throw new IllegalStateException("Cannot re-start a stopped (terminated) node agent");
- }
- }
- thread.start();
- }
-
- @Override
- public void stop() {
- logger.log(LogLevel.INFO, logPrefix + "Scheduling stop of NodeAgent");
- synchronized (monitor) {
- if (state == State.TERMINATED) {
- throw new IllegalStateException("Cannot stop an already stopped (terminated) node agent");
- }
- wantedState = State.TERMINATED;
- monitor.notifyAll();
- }
- try {
- thread.join();
- } catch (InterruptedException e) {
- logger.log(Level.WARNING, logPrefix + "Unexpected interrupt", e);
- }
- }
-
- void synchronizeLocalContainerState(
- final ContainerNodeSpec nodeSpec,
- Optional<Container> existingContainer) throws Exception {
- logger.log(Level.INFO, logPrefix + "Container " + nodeSpec.containerName + " state:" + nodeSpec.nodeState);
-
- if (nodeSpec.nodeState == NodeState.ACTIVE && !docker.imageIsDownloaded(nodeSpec.wantedDockerImage.get())) {
- logger.log(LogLevel.INFO, logPrefix + "Schedule async download of Docker image " + nodeSpec.wantedDockerImage.get());
- final CompletableFuture<DockerImage> asyncPullResult = docker.pullImageAsync(nodeSpec.wantedDockerImage.get());
- asyncPullResult.whenComplete((dockerImage, throwable) -> {
- if (throwable != null) {
- logger.log(
- Level.WARNING,
- logPrefix + "Failed to pull docker image " + nodeSpec.wantedDockerImage,
- throwable);
- return;
- }
- assert nodeSpec.wantedDockerImage.get().equals(dockerImage);
- scheduleWork();
- });
-
- return;
- }
-
- if (existingContainer.isPresent()) {
- Optional<String> removeReason = Optional.empty();
- if (nodeSpec.nodeState != NodeState.ACTIVE) {
- removeReason = Optional.of("Node no longer active");
- } else if (!nodeSpec.wantedDockerImage.get().equals(existingContainer.get().image)) {
- removeReason = Optional.of("The node is supposed to run a new Docker image: "
- + existingContainer.get() + " -> " + nodeSpec.wantedDockerImage.get());
- } else if (nodeSpec.currentRestartGeneration.get() < nodeSpec.wantedRestartGeneration.get()) {
- removeReason = Optional.of("Restart requested - wanted restart generation has been bumped: "
- + nodeSpec.currentRestartGeneration.get() + " -> " + nodeSpec.wantedRestartGeneration.get());
- } else if (!existingContainer.get().isRunning) {
- removeReason = Optional.of("Container no longer running");
- }
-
- if (removeReason.isPresent()) {
- logger.log(LogLevel.INFO, logPrefix + "Will remove container " + existingContainer.get() + ": "
- + removeReason.get());
- removeContainer(nodeSpec, existingContainer.get());
- existingContainer = Optional.empty(); // Make logic below easier
- }
- }
-
- switch (nodeSpec.nodeState) {
- case DIRTY: // intentional fall-through
- case PROVISIONED:
- logger.log(LogLevel.INFO, logPrefix + "State is " + nodeSpec.nodeState
- + ", will delete application storage and mark node as ready");
- docker.deleteApplicationStorage(nodeSpec.containerName);
- nodeRepository.markAsReady(nodeSpec.hostname);
- break;
- case ACTIVE:
- if (!existingContainer.isPresent()) {
- logger.log(Level.INFO, logPrefix + "Starting container " + nodeSpec.containerName);
- // TODO: Properly handle absent min* values
- docker.startContainer(
- nodeSpec.wantedDockerImage.get(),
- nodeSpec.hostname,
- nodeSpec.containerName,
- nodeSpec.minCpuCores.get(),
- nodeSpec.minDiskAvailableGb.get(),
- nodeSpec.minMainMemoryAvailableGb.get());
- nodeStarted = false;
- }
-
- if (!nodeStarted) {
- logger.log(Level.INFO, logPrefix + "Starting optional node program " + RESUME_NODE_COMMAND);
- Optional<ProcessResult> result = executeOptionalProgram(docker, nodeSpec.containerName, RESUME_NODE_COMMAND);
-
- if (result.isPresent() && !result.get().isSuccess()) {
- throw new RuntimeException("Container " + nodeSpec.containerName.asString()
- + ": Command " + Arrays.toString(RESUME_NODE_COMMAND) + " failed: " + result.get());
- }
-
- nodeStarted = true;
- }
-
- final String containerVespaVersion = nullOnException(() ->
- docker.getVespaVersion(nodeSpec.containerName));
-
- // Because it's more important to stop a bad release from rolling out in prod,
- // we put the resume call last. So if we fail after updating the node repo attributes
- // but before resume, the app may go through the tenant pipeline but will halt in prod.
- //
- // Note that this problem exists only because there are 2 different mechanisms
- // that should really be parts of a single mechanism:
- // - The content of node repo is used to determine whether a new Vespa+application
- // has been successfully rolled out.
- // - Slobrok and internal orchestrator state is used to determine whether
- // to allow upgrade (suspend).
-
- final NodeAttributes currentAttributes = new NodeAttributes(
- nodeSpec.wantedRestartGeneration.get(),
- nodeSpec.wantedDockerImage.get(),
- containerVespaVersion);
- if (!currentAttributes.equals(lastAttributesSet)) {
- logger.log(Level.INFO, logPrefix + "Publishing new set of attributes to node repo: "
- + lastAttributesSet + " -> " + currentAttributes);
- nodeRepository.updateNodeAttributes(
- nodeSpec.hostname,
- currentAttributes.restartGeneration,
- currentAttributes.dockerImage,
- currentAttributes.vespaVersion);
- lastAttributesSet = currentAttributes;
- }
-
- logger.log(Level.INFO, logPrefix + "Call resume against Orchestrator");
- orchestrator.resume(nodeSpec.hostname);
- break;
- default:
- // Nothing to do...
- }
- }
-
- private void removeContainer(final ContainerNodeSpec nodeSpec, final Container existingContainer)
- throws Exception {
- final ContainerName containerName = existingContainer.name;
- if (existingContainer.isRunning) {
- // If we're stopping the node only to upgrade or restart the node or similar, we need to suspend
- // the services.
- if (nodeSpec.nodeState == NodeState.ACTIVE) {
- // TODO: Also skip orchestration if we're downgrading in test/staging
- // How to implement:
- // - test/staging: We need to figure out whether we're in test/staging, by asking Chef!? Or,
- // let the Orchestrator handle it - it may know what zone we're in.
- // - downgrading: Impossible to know unless we look at the hosted version, which is
- // not available in the docker image (nor its name). Not sure how to solve this. Should
- // the node repo return the hosted version or a downgrade bit in addition to
- // wanted docker image etc?
- // Should the tenant pipeline instead use BCP tool to upgrade faster!?
- //
- // More generally, the node repo response should contain sufficient info on what the docker image is,
- // to allow the node admin to make decisions that depend on the docker image. Or, each docker image
- // needs to contain routines for drain and suspend. For many image, these can just be dummy routines.
-
- logger.log(Level.INFO, logPrefix + "Ask Orchestrator for permission to suspend node " + nodeSpec.hostname);
- final boolean suspendAllowed = orchestrator.suspend(nodeSpec.hostname);
- if (!suspendAllowed) {
- logger.log(Level.INFO, logPrefix + "Orchestrator rejected suspend of node");
- // TODO: change suspend() to throw an exception if suspend is denied
- throw new OrchestratorException("Failed to get permission to suspend " + nodeSpec.hostname);
- }
-
- trySuspendNode(containerName);
- }
-
- logger.log(Level.INFO, logPrefix + "Stopping container " + containerName);
- docker.stopContainer(containerName);
- }
-
- logger.log(Level.INFO, logPrefix + "Deleting container " + containerName);
- docker.deleteContainer(containerName);
- }
-
- static String[] programExistsCommand(String programPath) {
- return new String[]{ "/usr/bin/env", "test", "-x", programPath };
- }
-
- /**
- * Executes a program and returns its result, or if it doesn't exist, return a result
- * as-if the program executed with exit status 0 and no output.
- */
- static Optional<ProcessResult> executeOptionalProgram(Docker docker, ContainerName containerName, String... args) {
- assert args.length > 0;
- String[] nodeProgramExistsCommand = programExistsCommand(args[0]);
- if (!docker.executeInContainer(containerName, nodeProgramExistsCommand).isSuccess()) {
- return Optional.empty();
- }
-
- return Optional.of(docker.executeInContainer(containerName, args));
- }
-
- /**
- * Try to suspend node. Suspending a node means the node should be taken offline,
- * such that maintenance can be done of the node (upgrading, rebooting, etc),
- * and such that we will start serving again as soon as possible afterwards.
- *
- * Any failures are logged and ignored.
- */
- private void trySuspendNode(ContainerName containerName) {
- Optional<ProcessResult> result;
-
- try {
- // TODO: Change to waiting w/o timeout (need separate thread that we can stop).
- result = executeOptionalProgram(docker, containerName, SUSPEND_NODE_COMMAND);
- } catch (RuntimeException e) {
- // It's bad to continue as-if nothing happened, but on the other hand if we do not proceed to
- // remove container, we will not be able to upgrade to fix any problems in the suspend logic!
- logger.log(LogLevel.WARNING, logPrefix + "Failed trying to suspend node with "
- + Arrays.toString(SUSPEND_NODE_COMMAND), e);
- return;
- }
-
- if (result.isPresent() && !result.get().isSuccess()) {
- logger.log(LogLevel.WARNING, logPrefix + "The suspend program " + Arrays.toString(SUSPEND_NODE_COMMAND)
- + " failed: " + result.get().getOutput());
- }
- }
-
- private static <T> T nullOnException(Supplier<T> supplier) {
- try {
- return supplier.get();
- } catch (RuntimeException e) {
- logger.log(Level.WARNING, "Ignoring failure", e);
- return null;
- }
- }
-
- // It somewhat sucks that this class almost duplicates a binding class used by NodeRepositoryImpl,
- // but using the binding class here would be a layer violation, and would also tie this logic to
- // serialization-related dependencies it needs not have.
- private static class NodeAttributes {
- private final long restartGeneration;
- private final DockerImage dockerImage;
- private final String vespaVersion;
-
- private NodeAttributes(long restartGeneration, DockerImage dockerImage, String vespaVersion) {
- this.restartGeneration = restartGeneration;
- this.dockerImage = dockerImage;
- this.vespaVersion = vespaVersion;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(restartGeneration, dockerImage, vespaVersion);
- }
-
- @Override
- public boolean equals(final Object o) {
- if (!(o instanceof NodeAttributes)) {
- return false;
- }
- final NodeAttributes other = (NodeAttributes) o;
-
- return Objects.equals(restartGeneration, other.restartGeneration)
- && Objects.equals(dockerImage, other.dockerImage)
- && Objects.equals(vespaVersion, other.vespaVersion);
- }
-
- @Override
- public String toString() {
- return "NodeAttributes{" +
- "restartGeneration=" + restartGeneration +
- ", dockerImage=" + dockerImage +
- ", vespaVersion='" + vespaVersion + '\'' +
- '}';
- }
- }
-
- private void scheduleWork() {
- synchronized (monitor) {
- if (wantedState != State.FROZEN) {
- wantedState = State.WORKING;
- } else {
- logger.log(Level.FINE, "Not scheduling work since in freeze.");
- }
- monitor.notifyAll();
- }
- }
-
- private void blockUntilNotWaitingOrFrozen() {
- try {
- synchronized (monitor) {
- while (wantedState == State.WAITING || wantedState == State.FROZEN) {
- state = wantedState;
- monitor.wait();
- continue;
- }
- }
- } catch (InterruptedException e) {
- logger.severe("NodeAgent thread interrupted. Ignoring this: " + e.getMessage());
- }
- }
-
- private void maintainWantedState() {
- while (true) {
- blockUntilNotWaitingOrFrozen();
- synchronized (monitor) {
- switch (wantedState) {
- case WAITING:
- state = State.WAITING;
- continue;
- case WORKING:
- state = State.WORKING;
- wantedState = State.WAITING;
- break;
- case FROZEN:
- state = State.FROZEN;
- continue;
- case TERMINATED:
- return;
- }
- }
- // state is WORKING state.
- try {
- final ContainerNodeSpec nodeSpec = nodeRepository.getContainerNodeSpec(hostname)
- .orElseThrow(() ->
- new IllegalStateException(String.format("Node '%s' missing from node repository.", hostname)));
- final Optional<Container> existingContainer = docker.getContainer(hostname);
- synchronizeLocalContainerState(nodeSpec, existingContainer);
- } catch (RuntimeException e) {
- logger.log(LogLevel.ERROR, logPrefix + "Unhandled exception, ignoring.", e);
- }
- catch (Throwable t) {
- logger.log(LogLevel.ERROR, logPrefix + "Unhandled throwable, taking down system.", t);
- System.exit(234);
- }
- }
- }
-}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdmin.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdmin.java
index 863a8f47965..c14a4dd7286 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdmin.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdmin.java
@@ -1,6 +1,8 @@
-package com.yahoo.vespa.hosted.node.admin;
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.nodeadmin;
import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
import java.util.List;
import java.util.Set;
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdminImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java
index f9e131e3d6c..1eb491be9c2 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdminImpl.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java
@@ -1,11 +1,13 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin;
+package com.yahoo.vespa.hosted.node.admin.nodeadmin;
import com.yahoo.collections.Pair;
import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.Container;
import com.yahoo.vespa.hosted.node.admin.docker.Docker;
import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
import java.io.IOException;
import java.time.Duration;
@@ -58,7 +60,9 @@ public class NodeAdminImpl implements NodeAdmin {
public boolean freezeAndCheckIfAllFrozen() {
for (NodeAgent nodeAgent : nodeAgents.values()) {
- nodeAgent.execute(NodeAgent.Command.FREEZE);
+ // We could make this blocking, this could speed up the suspend call a bit, but not sure if it is
+ // worth it (it could block the rest call for some time and might have implications).
+ nodeAgent.execute(NodeAgent.Command.SET_FREEZE, false /* blocking*/);
}
for (NodeAgent nodeAgent : nodeAgents.values()) {
if (nodeAgent.getState() != NodeAgent.State.FROZEN) {
@@ -70,7 +74,7 @@ public class NodeAdminImpl implements NodeAdmin {
public void unfreeze() {
for (NodeAgent nodeAgent : nodeAgents.values()) {
- nodeAgent.execute(NodeAgent.Command.UNFREEZE);
+ nodeAgent.execute(NodeAgent.Command.UNFREEZE, false);
}
}
@@ -182,6 +186,6 @@ public class NodeAdminImpl implements NodeAdmin {
nodeAgents.put(nodeSpec.hostname, agent);
agent.start();
}
- agent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO);
+ agent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, false);
}
}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdminStateUpdater.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminStateUpdater.java
index deb1f37e00e..6c46dd893c7 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAdminStateUpdater.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminStateUpdater.java
@@ -1,7 +1,8 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin;
+package com.yahoo.vespa.hosted.node.admin.nodeadmin;
import com.yahoo.component.AbstractComponent;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepository;
import com.yahoo.vespa.hosted.node.admin.orchestrator.Orchestrator;
@@ -14,8 +15,8 @@ import java.util.concurrent.TimeUnit;
import java.util.logging.Level;
import java.util.logging.Logger;
-import static com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater.State.RESUMED;
-import static com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater.State.SUSPENDED;
+import static com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater.State.RESUMED;
+import static com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater.State.SUSPENDED;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
/**
@@ -31,7 +32,7 @@ public class NodeAdminStateUpdater extends AbstractComponent {
private final NodeAdmin nodeAdmin;
private boolean isRunningUpdates = true;
private final Object monitor = new Object();
- final Orchestrator orchestrator;
+ private final Orchestrator orchestrator;
private final String baseHostName;
public NodeAdminStateUpdater(
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgent.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgent.java
index 4d5e19e4bce..793d20b4f85 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/NodeAgent.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgent.java
@@ -1,5 +1,5 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin;
+package com.yahoo.vespa.hosted.node.admin.nodeagent;
/**
* Responsible for management of a single node over its lifecycle.
@@ -10,18 +10,21 @@ package com.yahoo.vespa.hosted.node.admin;
*/
public interface NodeAgent {
- enum Command {UPDATE_FROM_NODE_REPO, FREEZE, UNFREEZE}
+ enum Command {UPDATE_FROM_NODE_REPO, SET_FREEZE, UNFREEZE}
enum State {WAITING, WORKING, FROZEN, TERMINATED}
/**
* Signals to the agent that it should update the node specification and container state and maintain wanted state.
*
- * This method is to be assumed asynchronous by the caller; i.e. any actions the agent will take may execute after
- * this method call returns.
- *
* It is an error to call this method on an instance after stop() has been called.
*/
- void execute(Command wantedState);
+
+ /**
+ * Make the node agent execute a command soon.
+ * @param command task to be done
+ * @param blocking will wait for the node agent to execute the command.
+ */
+ void execute(Command command, boolean blocking);
/**
* Returns the state of the agent.
@@ -31,7 +34,7 @@ public interface NodeAgent {
/**
* Starts the agent. After this method is called, the agent will asynchronously maintain the node, continuously
* striving to make the current state equal to the wanted state. The current and wanted state update as part of
- * {@link #execute(Command)}.
+ * {@link #execute(Command, boolean)}.
*/
void start();
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImpl.java
new file mode 100644
index 00000000000..22522b6d346
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImpl.java
@@ -0,0 +1,265 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.nodeagent;
+
+import com.yahoo.log.LogLevel;
+import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
+import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepository;
+import com.yahoo.vespa.hosted.node.admin.orchestrator.Orchestrator;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl.ContainerState.CONTAINER_ABSENT;
+import static com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl.ContainerState.RUNNING_NODE;
+import static com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl.ContainerState.RUNNING_NODE_HOWEVER_RESUME_SCRIPT_NOT_RUN;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+/**
+ * @author dybis
+ * @author bakksjo
+ */
+public class NodeAgentImpl implements NodeAgent {
+
+ private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1);
+ private AtomicBoolean frozen = new AtomicBoolean(false);
+
+ private static final Logger logger = Logger.getLogger(NodeAgentImpl.class.getName());
+
+ private DockerImage imageBeingDownloaded = null;
+
+ private final String logPrefix;
+ private final HostName hostname;
+
+ private final NodeRepository nodeRepository;
+ private final Orchestrator orchestrator;
+ private final NodeDocker nodeDocker;
+
+ private final Object monitor = new Object();
+ @GuardedBy("monitor")
+ private State state = State.WAITING;
+
+
+ public enum ContainerState {
+ CONTAINER_ABSENT,
+ RUNNING_NODE_HOWEVER_RESUME_SCRIPT_NOT_RUN,
+ RUNNING_NODE
+ }
+ ContainerState containerState = CONTAINER_ABSENT;
+
+ // The attributes of the last successful noderepo attribute update for this node. Used to avoid redundant calls.
+ private NodeAttributes lastAttributesSet = null;
+
+ public NodeAgentImpl(
+ final HostName hostName,
+ final NodeRepository nodeRepository,
+ final Orchestrator orchestrator,
+ final NodeDocker nodeDocker) {
+ this.logPrefix = "NodeAgent(" + hostName + "): ";
+ this.nodeRepository = nodeRepository;
+ this.orchestrator = orchestrator;
+ this.hostname = hostName;
+ this.nodeDocker = nodeDocker;
+ }
+
+ @Override
+ public void execute(Command command, boolean blocking) {
+ switch (command) {
+ case UPDATE_FROM_NODE_REPO:
+ break;
+ case SET_FREEZE:
+ frozen.set(true);
+ break;
+ case UNFREEZE:
+ frozen.set(false);
+ break;
+ default:
+ throw new RuntimeException("Unknown command " + command.name());
+ }
+ if (blocking) {
+ nodeTick();
+ } else {
+ nodeTickInNewThread();
+ }
+ }
+
+ @Override
+ public State getState() {
+ synchronized (monitor) {
+ return state;
+ }
+ }
+
+ @Override
+ public void start() {
+ if (scheduler.isTerminated()) {
+ throw new RuntimeException("Can not restart a node agent.");
+ }
+ scheduler.scheduleWithFixedDelay(this::nodeTick, 1, 60, SECONDS);
+ }
+
+ @Override
+ public void stop() {
+ if (scheduler.isTerminated()) {
+ throw new RuntimeException("Can not re-stop a node agent.");
+ }
+ scheduler.shutdown();
+ synchronized (monitor) {
+ state = State.TERMINATED;
+ }
+ }
+
+ private boolean isFrozen() {
+ synchronized (monitor) {
+ if (state == State.TERMINATED) {
+ return true;
+ }
+ if (frozen.get()) {
+ state = State.FROZEN;
+ return true;
+ }
+ state = State.WORKING;
+ }
+ return false;
+ }
+
+ public void startNodeInContainerIfNeeded(final ContainerNodeSpec nodeSpec) {
+ if (containerState != RUNNING_NODE_HOWEVER_RESUME_SCRIPT_NOT_RUN) {
+ return;
+ }
+ logger.log(Level.INFO, logPrefix + "Starting optional node program resume command");
+ nodeDocker.executeResume(nodeSpec.containerName);//, RESUME_NODE_COMMAND);
+ containerState = RUNNING_NODE;
+ }
+
+ public void publishThatNodeIsRunningIfRequired(final ContainerNodeSpec nodeSpec) throws IOException {
+ final String containerVespaVersion = nodeDocker.getVespaVersionOrNull(nodeSpec.containerName);
+
+ final NodeAttributes currentAttributes = new NodeAttributes(
+ nodeSpec.wantedRestartGeneration.get(),
+ nodeSpec.wantedDockerImage.get(),
+ containerVespaVersion);
+ if (!currentAttributes.equals(lastAttributesSet)) {
+ logger.log(Level.INFO, logPrefix + "Publishing new set of attributes to node repo: "
+ + lastAttributesSet + " -> " + currentAttributes);
+ nodeRepository.updateNodeAttributes(
+ nodeSpec.hostname,
+ currentAttributes.restartGeneration,
+ currentAttributes.dockerImage,
+ currentAttributes.vespaVersion);
+ lastAttributesSet = currentAttributes;
+ }
+
+ logger.log(Level.INFO, logPrefix + "Call resume against Orchestrator");
+ orchestrator.resume(nodeSpec.hostname);
+ }
+
+ public void startContainerIfNeeded(final ContainerNodeSpec nodeSpec) {
+ if (nodeDocker.startContainerIfNeeded(nodeSpec)) {
+ containerState = RUNNING_NODE_HOWEVER_RESUME_SCRIPT_NOT_RUN;
+ } else {
+ // In case container was already running on startup, we found the container, but should call
+ if (containerState == CONTAINER_ABSENT) {
+ containerState = RUNNING_NODE_HOWEVER_RESUME_SCRIPT_NOT_RUN;
+ }
+ }
+ }
+
+ private void nodeTickInNewThread() {
+ new Thread(this::nodeTick).start();
+ }
+
+ private void removeContainerIfNeededUpdateContainerState(ContainerNodeSpec nodeSpec) throws Exception {
+ if (nodeDocker.removeContainerIfNeeded(nodeSpec, hostname, orchestrator)) {
+ containerState = CONTAINER_ABSENT;
+ }
+ }
+
+ private boolean scheduleDownLoadIfNeededIsImageReady(ContainerNodeSpec nodeSpec) {
+ if (nodeDocker.shouldScheduleDownloadOfImage(nodeSpec.wantedDockerImage.get())) {
+ if (imageBeingDownloaded == nodeSpec.wantedDockerImage.get()) {
+ // Downloading already scheduled, but not done.
+ return false;
+ }
+ imageBeingDownloaded = nodeSpec.wantedDockerImage.get();
+ // Create a tick when download is finished.
+ nodeDocker.scheduleDownloadOfImage(nodeSpec, this::nodeTickInNewThread);
+ return false;
+ } else {
+ imageBeingDownloaded = null;
+ return true;
+ }
+ }
+
+
+ private void nodeTick() {
+ if (isFrozen()) {
+ return;
+ }
+ synchronized (monitor) {
+ try {
+ final ContainerNodeSpec nodeSpec = nodeRepository.getContainerNodeSpec(hostname)
+ .orElseThrow(() ->
+ new IllegalStateException(String.format("Node '%s' missing from node repository.", hostname)));
+
+ switch (nodeSpec.nodeState) {
+ case PROVISIONED:
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+ logger.log(LogLevel.INFO, logPrefix + "State is provisioned, will delete application storage and mark node as ready");
+ nodeDocker.deleteContainerStorage(nodeSpec.containerName);
+ nodeRepository.markAsReady(nodeSpec.hostname);
+ break;
+ case READY:
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+ break;
+ case RESERVED:
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+ break;
+ case ACTIVE:
+ if (! scheduleDownLoadIfNeededIsImageReady(nodeSpec)) {
+ return;
+ }
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+
+ startContainerIfNeeded(nodeSpec);
+ startNodeInContainerIfNeeded(nodeSpec);
+ // Because it's more important to stop a bad release from rolling out in prod,
+ // we put the resume call last. So if we fail after updating the node repo attributes
+ // but before resume, the app may go through the tenant pipeline but will halt in prod.
+ //
+ // Note that this problem exists only because there are 2 different mechanisms
+ // that should really be parts of a single mechanism:
+ // - The content of node repo is used to determine whether a new Vespa+application
+ // has been successfully rolled out.
+ // - Slobrok and internal orchestrator state is used to determine whether
+ // to allow upgrade (suspend).
+ publishThatNodeIsRunningIfRequired(nodeSpec);
+ break;
+ case INACTIVE:
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+ break;
+ case DIRTY:
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+ logger.log(LogLevel.INFO, logPrefix + "State is dirty, will delete application storage and mark node as ready");
+ nodeDocker.deleteContainerStorage(nodeSpec.containerName);
+ nodeRepository.markAsReady(nodeSpec.hostname);
+ break;
+ case FAILED:
+ removeContainerIfNeededUpdateContainerState(nodeSpec);
+ break;
+ }
+ } catch (Exception e) {
+ logger.log(LogLevel.ERROR, logPrefix + "Unhandled exception, ignoring.", e);
+ } catch (Throwable t) {
+ logger.log(LogLevel.ERROR, logPrefix + "Unhandled throwable, taking down system.", t);
+ System.exit(234);
+ }
+ }
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAttributes.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAttributes.java
new file mode 100644
index 00000000000..b848b8c8e6a
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAttributes.java
@@ -0,0 +1,48 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.nodeagent;
+
+import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+
+import java.util.Objects;
+
+
+// It somewhat sucks that this class almost duplicates a binding class used by NodeRepositoryImpl,
+// but using the binding class here would be a layer violation, and would also tie this logic to
+// serialization-related dependencies it needs not have.
+class NodeAttributes {
+ public final long restartGeneration;
+ public final DockerImage dockerImage;
+ public final String vespaVersion;
+
+ NodeAttributes(long restartGeneration, DockerImage dockerImage, String vespaVersion) {
+ this.restartGeneration = restartGeneration;
+ this.dockerImage = dockerImage;
+ this.vespaVersion = vespaVersion;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(restartGeneration, dockerImage, vespaVersion);
+ }
+
+ @Override
+ public boolean equals(final Object o) {
+ if (!(o instanceof NodeAttributes)) {
+ return false;
+ }
+ final NodeAttributes other = (NodeAttributes) o;
+
+ return Objects.equals(restartGeneration, other.restartGeneration)
+ && Objects.equals(dockerImage, other.dockerImage)
+ && Objects.equals(vespaVersion, other.vespaVersion);
+ }
+
+ @Override
+ public String toString() {
+ return "NodeAttributes{" +
+ "restartGeneration=" + restartGeneration +
+ ", dockerImage=" + dockerImage +
+ ", vespaVersion='" + vespaVersion + '\'' +
+ '}';
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDocker.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDocker.java
new file mode 100644
index 00000000000..86217d5f704
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDocker.java
@@ -0,0 +1,236 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.nodeagent;
+
+import com.yahoo.log.LogLevel;
+import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.defaults.Defaults;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
+import com.yahoo.vespa.hosted.node.admin.docker.Container;
+import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
+import com.yahoo.vespa.hosted.node.admin.docker.Docker;
+import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.docker.ProcessResult;
+import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
+import com.yahoo.vespa.hosted.node.admin.orchestrator.Orchestrator;
+import com.yahoo.vespa.hosted.node.admin.orchestrator.OrchestratorException;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Class that wraps the Docker class and have some tools related to running programs in docker.
+ * @author dybis
+ */
+public class NodeDocker {
+ static final String NODE_PROGRAM = Defaults.getDefaults().vespaHome() + "bin/vespa-nodectl";
+
+ private static final String[] RESUME_NODE_COMMAND = new String[] {NODE_PROGRAM, "resume"};
+ private static final String[] SUSPEND_NODE_COMMAND = new String[] {NODE_PROGRAM, "suspend"};
+
+ private static final Logger logger = Logger.getLogger(NodeDocker.class.getName());
+ private final Docker docker;
+
+ public NodeDocker(Docker docker) {
+ this.docker = docker;
+ }
+
+ // Returns null on problems
+ public String getVespaVersionOrNull(ContainerName containerName) {
+ try {
+ return docker.getVespaVersion(containerName);
+ } catch (RuntimeException e) {
+ logger.log(Level.WARNING, "Ignoring failure", e);
+ return null;
+ }
+ }
+
+ // Returns true if started
+ public boolean startContainerIfNeeded(final ContainerNodeSpec nodeSpec) {
+ final Optional<Container> existingContainer = docker.getContainer(nodeSpec.hostname);
+ if (!existingContainer.isPresent()) {
+ startContainer(nodeSpec);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ // Returns true if scheduling download
+ public boolean shouldScheduleDownloadOfImage(final DockerImage dockerImage) {
+ return !docker.imageIsDownloaded(dockerImage);
+ }
+
+ public boolean removeContainerIfNeeded(ContainerNodeSpec nodeSpec, HostName hostname, Orchestrator orchestrator)
+ throws Exception {
+ Optional<Container> existingContainer = docker.getContainer(hostname);
+ if (! existingContainer.isPresent()) {
+ return true;
+ }
+ Optional<String> removeReason = shouldRemoveContainer(nodeSpec, existingContainer);
+ if (removeReason.isPresent()) {
+ logger.log(LogLevel.INFO, "NodeAgent(" + hostname + "): " + "Will remove container " + existingContainer.get() + ": "
+ + removeReason.get());
+ removeContainer(nodeSpec, existingContainer.get(), orchestrator);
+ return true;
+ }
+ return false;
+ }
+
+ public void deleteContainerStorage(ContainerName containerName) throws IOException {
+ docker.deleteApplicationStorage(containerName);
+ }
+
+ private Optional<String> shouldRemoveContainer(ContainerNodeSpec nodeSpec, Optional<Container> existingContainer) {
+ if (nodeSpec.nodeState != NodeState.ACTIVE) {
+ return Optional.of("Node no longer active");
+ }
+ if (!nodeSpec.wantedDockerImage.get().equals(existingContainer.get().image)) {
+ return Optional.of("The node is supposed to run a new Docker image: "
+ + existingContainer.get() + " -> " + nodeSpec.wantedDockerImage.get());
+ }
+ if (nodeSpec.currentRestartGeneration.get() < nodeSpec.wantedRestartGeneration.get()) {
+ return Optional.of("Restart requested - wanted restart generation has been bumped: "
+ + nodeSpec.currentRestartGeneration.get() + " -> " + nodeSpec.wantedRestartGeneration.get());
+ }
+ if (!existingContainer.get().isRunning) {
+ return Optional.of("Container no longer running");
+ }
+ return Optional.empty();
+ }
+
+ /**
+ * Executes a program and returns its result, or if it doesn't exist, return a result
+ * as-if the program executed with exit status 0 and no output.
+ */
+ Optional<ProcessResult> executeOptionalProgram(ContainerName containerName, String... args) {
+ assert args.length > 0;
+ String[] nodeProgramExistsCommand = programExistsCommand(args[0]);
+ if (!docker.executeInContainer(containerName, nodeProgramExistsCommand).isSuccess()) {
+ return Optional.empty();
+ }
+
+ return Optional.of(docker.executeInContainer(containerName, args));
+ }
+
+ String[] programExistsCommand(String programPath) {
+ return new String[]{ "/usr/bin/env", "test", "-x", programPath };
+ }
+
+ /**
+ * Try to suspend node. Suspending a node means the node should be taken offline,
+ * such that maintenance can be done of the node (upgrading, rebooting, etc),
+ * and such that we will start serving again as soon as possible afterwards.
+ *
+ * Any failures are logged and ignored.
+ */
+ private void trySuspendNode(ContainerName containerName) {
+
+ Optional<ProcessResult> result;
+
+ try {
+ // TODO: Change to waiting w/o timeout (need separate thread that we can stop).
+ result = executeOptionalProgram(containerName, SUSPEND_NODE_COMMAND);
+ } catch (RuntimeException e) {
+ // It's bad to continue as-if nothing happened, but on the other hand if we do not proceed to
+ // remove container, we will not be able to upgrade to fix any problems in the suspend logic!
+ logger.log(LogLevel.WARNING, "Failed trying to suspend container " + containerName.asString() + " with "
+ + Arrays.toString(SUSPEND_NODE_COMMAND), e);
+ return;
+ }
+
+ if (result.isPresent() && !result.get().isSuccess()) {
+
+ logger.log(LogLevel.WARNING, "The suspend program " + Arrays.toString(SUSPEND_NODE_COMMAND)
+ + " failed: " + result.get().getOutput() + " for container " + containerName.asString());
+ }
+ }
+
+ void startContainer(final ContainerNodeSpec nodeSpec) {
+ String logPrefix = "NodeAgent(" + nodeSpec.hostname+ "): ";
+
+ logger.log(Level.INFO, logPrefix + "Starting container " + nodeSpec.containerName);
+ // TODO: Properly handle absent min* values
+ docker.startContainer(
+ nodeSpec.wantedDockerImage.get(),
+ nodeSpec.hostname,
+ nodeSpec.containerName,
+ nodeSpec.minCpuCores.get(),
+ nodeSpec.minDiskAvailableGb.get(),
+ nodeSpec.minMainMemoryAvailableGb.get());
+ }
+
+ void scheduleDownloadOfImage(final ContainerNodeSpec nodeSpec, Runnable callback) {
+ String logPrefix = "NodeAgent(" + nodeSpec.hostname+ "): ";
+
+ logger.log(LogLevel.INFO, logPrefix + "Schedule async download of Docker image " + nodeSpec.wantedDockerImage.get());
+ final CompletableFuture<DockerImage> asyncPullResult = docker.pullImageAsync(nodeSpec.wantedDockerImage.get());
+ asyncPullResult.whenComplete((dockerImage, throwable) -> {
+ if (throwable != null) {
+ logger.log(
+ Level.WARNING,
+ logPrefix + "Failed to pull docker image " + nodeSpec.wantedDockerImage,
+ throwable);
+ return;
+ }
+ assert nodeSpec.wantedDockerImage.get().equals(dockerImage);
+ callback.run();
+ });
+
+ return;
+ }
+
+ private void removeContainer(final ContainerNodeSpec nodeSpec, final Container existingContainer, Orchestrator orchestrator)
+ throws Exception {
+ String logPrefix = "NodeAgent(" + nodeSpec.hostname+ "): ";
+ final ContainerName containerName = existingContainer.name;
+ if (existingContainer.isRunning) {
+ // If we're stopping the node only to upgrade or restart the node or similar, we need to suspend
+ // the services.
+ if (nodeSpec.nodeState == NodeState.ACTIVE) {
+ // TODO: Also skip orchestration if we're downgrading in test/staging
+ // How to implement:
+ // - test/staging: We need to figure out whether we're in test/staging, by asking Chef!? Or,
+ // let the Orchestrator handle it - it may know what zone we're in.
+ // - downgrading: Impossible to know unless we look at the hosted version, which is
+ // not available in the docker image (nor its name). Not sure how to solve this. Should
+ // the node repo return the hosted version or a downgrade bit in addition to
+ // wanted docker image etc?
+ // Should the tenant pipeline instead use BCP tool to upgrade faster!?
+ //
+ // More generally, the node repo response should contain sufficient info on what the docker image is,
+ // to allow the node admin to make decisions that depend on the docker image. Or, each docker image
+ // needs to contain routines for drain and suspend. For many image, these can just be dummy routines.
+
+ logger.log(Level.INFO, logPrefix + "Ask Orchestrator for permission to suspend node " + nodeSpec.hostname);
+ final boolean suspendAllowed = orchestrator.suspend(nodeSpec.hostname);
+ if (!suspendAllowed) {
+ logger.log(Level.INFO, logPrefix + "Orchestrator rejected suspend of node");
+ // TODO: change suspend() to throw an exception if suspend is denied
+ throw new OrchestratorException("Failed to get permission to suspend " + nodeSpec.hostname);
+ }
+
+ trySuspendNode(containerName);
+ }
+
+ logger.log(Level.INFO, logPrefix + "Stopping container " + containerName);
+ docker.stopContainer(containerName);
+ }
+
+ logger.log(Level.INFO, logPrefix + "Deleting container " + containerName);
+ docker.deleteContainer(containerName);
+ }
+
+
+ public void executeResume(ContainerName containerName) {
+ Optional<ProcessResult> result = executeOptionalProgram(containerName, RESUME_NODE_COMMAND);
+
+ if (result.isPresent() && !result.get().isSuccess()) {
+ throw new RuntimeException("Container " +containerName.asString()
+ + ": command " + RESUME_NODE_COMMAND + " failed: " + result.get());
+ }
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProvider.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProvider.java
index 343ac24b20f..4f2307882ad 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProvider.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProvider.java
@@ -1,6 +1,6 @@
package com.yahoo.vespa.hosted.node.admin.provider;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
/**
* Class for setting up instances of classes; enables testing.
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProviderImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProviderImpl.java
index fdc72d973cd..bf85d6652bc 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProviderImpl.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/provider/ComponentsProviderImpl.java
@@ -1,12 +1,13 @@
package com.yahoo.vespa.hosted.node.admin.provider;
import com.yahoo.vespa.applicationmodel.HostName;
-import com.yahoo.vespa.hosted.node.admin.NodeAdmin;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminImpl;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
-import com.yahoo.vespa.hosted.node.admin.NodeAgent;
-import com.yahoo.vespa.hosted.node.admin.NodeAgentImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdmin;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
import com.yahoo.vespa.hosted.node.admin.docker.Docker;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeDocker;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepository;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepositoryImpl;
import com.yahoo.vespa.hosted.node.admin.orchestrator.Orchestrator;
@@ -47,7 +48,7 @@ public class ComponentsProviderImpl implements ComponentsProvider {
Orchestrator orchestrator = OrchestratorImpl.createOrchestratorFromSettings();
final Function<HostName, NodeAgent> nodeAgentFactory = (hostName) ->
- new NodeAgentImpl(hostName, docker, nodeRepository, orchestrator);
+ new NodeAgentImpl(hostName, nodeRepository, orchestrator, new NodeDocker(docker));
final NodeAdmin nodeAdmin = new NodeAdminImpl(docker, nodeAgentFactory);
return new NodeAdminStateUpdater(
nodeRepository, nodeAdmin, INITIAL_SCHEDULER_DELAY_MILLIS, INTERVAL_SCHEDULER_IN_MILLIS, orchestrator, baseHostName);
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/restapi/RestApiHandler.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/restapi/RestApiHandler.java
index ace3fbfc764..dfb2f2bfd9b 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/restapi/RestApiHandler.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/restapi/RestApiHandler.java
@@ -6,7 +6,7 @@ import com.yahoo.container.jdisc.HttpRequest;
import com.yahoo.container.jdisc.HttpResponse;
import com.yahoo.container.jdisc.LoggingRequestHandler;
import com.yahoo.container.logging.AccessLog;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
import com.yahoo.vespa.hosted.node.admin.provider.ComponentsProvider;
import javax.ws.rs.core.MediaType;
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ComponentsProviderWithMocks.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ComponentsProviderWithMocks.java
index 66fcc9a4057..40ac6644e9e 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ComponentsProviderWithMocks.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ComponentsProviderWithMocks.java
@@ -1,12 +1,14 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
-import com.yahoo.vespa.hosted.node.admin.NodeAdmin;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminImpl;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
-import com.yahoo.vespa.hosted.node.admin.NodeAgent;
-import com.yahoo.vespa.hosted.node.admin.NodeAgentImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdmin;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
import com.yahoo.vespa.hosted.node.admin.docker.Docker;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeDocker;
import com.yahoo.vespa.hosted.node.admin.provider.ComponentsProvider;
import java.util.function.Function;
@@ -22,7 +24,7 @@ public class ComponentsProviderWithMocks implements ComponentsProvider {
private Docker dockerMock = new DockerMock();
private final Function<HostName, NodeAgent> nodeAgentFactory = (hostName) ->
- new NodeAgentImpl(hostName, dockerMock, nodeRepositoryMock, orchestratorMock);
+ new NodeAgentImpl(hostName, nodeRepositoryMock, orchestratorMock, new NodeDocker(dockerMock));
private NodeAdmin nodeAdmin = new NodeAdminImpl(dockerMock, nodeAgentFactory);
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerFailTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerFailTest.java
index 9f0227e7b38..878499d484f 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerFailTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerFailTest.java
@@ -1,15 +1,16 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
-
import com.yahoo.vespa.applicationmodel.HostName;
import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
-import com.yahoo.vespa.hosted.node.admin.NodeAdmin;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminImpl;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
-import com.yahoo.vespa.hosted.node.admin.NodeAgent;
-import com.yahoo.vespa.hosted.node.admin.NodeAgentImpl;
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdmin;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeDocker;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
import org.junit.After;
import org.junit.Before;
@@ -47,7 +48,7 @@ public class DockerFailTest {
dockerMock = new DockerMock();
Function<HostName, NodeAgent> nodeAgentFactory = (hostName) ->
- new NodeAgentImpl(hostName, dockerMock, nodeRepositoryMock, orchestratorMock);
+ new NodeAgentImpl(hostName, nodeRepositoryMock, orchestratorMock, new NodeDocker(dockerMock));
NodeAdmin nodeAdmin = new NodeAdminImpl(dockerMock, nodeAgentFactory);
HostName hostName = new HostName("hostName");
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerMock.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerMock.java
index b6165a8d545..61e29fd25e4 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerMock.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerMock.java
@@ -1,3 +1,4 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/MultiDockerTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/MultiDockerTest.java
index c61c690bb98..5a4057f3d1d 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/MultiDockerTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/MultiDockerTest.java
@@ -1,14 +1,17 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
-import com.yahoo.vespa.hosted.node.admin.NodeAdmin;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminImpl;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
-import com.yahoo.vespa.hosted.node.admin.NodeAgent;
-import com.yahoo.vespa.hosted.node.admin.NodeAgentImpl;
+
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdmin;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeDocker;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
import org.junit.After;
import org.junit.Before;
@@ -47,7 +50,7 @@ public class MultiDockerTest {
dockerMock = new DockerMock();
Function<HostName, NodeAgent> nodeAgentFactory = (hostName) ->
- new NodeAgentImpl(hostName, dockerMock, nodeRepositoryMock, orchestratorMock);
+ new NodeAgentImpl(hostName, nodeRepositoryMock, orchestratorMock, new NodeDocker(dockerMock));
nodeAdmin = new NodeAdminImpl(dockerMock, nodeAgentFactory);
updater = new NodeAdminStateUpdater(nodeRepositoryMock, nodeAdmin, 1, 1, orchestratorMock, "basehostname");
}
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeRepoMock.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeRepoMock.java
index bffd6ddd22a..ecdcf5e21e6 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeRepoMock.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeRepoMock.java
@@ -1,3 +1,4 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeStateTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeStateTest.java
index 76b6ca4533d..cfea96e113e 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeStateTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/NodeStateTest.java
@@ -1,14 +1,16 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
-import com.yahoo.vespa.hosted.node.admin.NodeAdmin;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminImpl;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
-import com.yahoo.vespa.hosted.node.admin.NodeAgent;
-import com.yahoo.vespa.hosted.node.admin.NodeAgentImpl;
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdmin;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeDocker;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
import org.junit.After;
import org.junit.Before;
@@ -20,7 +22,6 @@ import java.util.function.Function;
import static org.hamcrest.core.Is.is;
import static org.hamcrest.core.IsNot.not;
-import static org.hamcrest.core.StringEndsWith.endsWith;
import static org.junit.Assert.assertThat;
/**
@@ -53,7 +54,7 @@ public class NodeStateTest {
dockerMock = new DockerMock();
Function<HostName, NodeAgent> nodeAgentFactory = (hostName) ->
- new NodeAgentImpl(hostName, dockerMock, nodeRepositoryMock, orchestratorMock);
+ new NodeAgentImpl(hostName, nodeRepositoryMock, orchestratorMock, new NodeDocker(dockerMock));
NodeAdmin nodeAdmin = new NodeAdminImpl(dockerMock, nodeAgentFactory);
hostName = new HostName("hostName");
@@ -150,6 +151,7 @@ public class NodeStateTest {
while (!initialDockerRequests.equals(DockerMock.getRequests())) {
Thread.sleep(10);
}
+
assertThat(initialDockerRequests, is(DockerMock.getRequests()));
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/OrchestratorMock.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/OrchestratorMock.java
index c77c6fd64b6..7f01a1b3acc 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/OrchestratorMock.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/OrchestratorMock.java
@@ -1,3 +1,4 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ResumeTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ResumeTest.java
index 768f5964a60..3d6e5233df5 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ResumeTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/ResumeTest.java
@@ -1,14 +1,16 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.vespa.applicationmodel.HostName;
import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
-import com.yahoo.vespa.hosted.node.admin.NodeAdmin;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminImpl;
-import com.yahoo.vespa.hosted.node.admin.NodeAdminStateUpdater;
-import com.yahoo.vespa.hosted.node.admin.NodeAgent;
-import com.yahoo.vespa.hosted.node.admin.NodeAgentImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdmin;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminImpl;
+import com.yahoo.vespa.hosted.node.admin.nodeadmin.NodeAdminStateUpdater;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeDocker;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
import org.junit.After;
import org.junit.Before;
@@ -54,7 +56,7 @@ public class ResumeTest {
DockerMock dockerMock = new DockerMock();
Function<HostName, NodeAgent> nodeAgentFactory = (hostName) ->
- new NodeAgentImpl(hostName, dockerMock, nodeRepositoryMock, orchestratorMock);
+ new NodeAgentImpl(hostName, nodeRepositoryMock, orchestratorMock, new NodeDocker(dockerMock));
NodeAdmin nodeAdmin = new NodeAdminImpl(dockerMock, nodeAgentFactory);
NodeRepoMock.addContainerNodeSpec(new ContainerNodeSpec(
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/RunInContainerTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/RunInContainerTest.java
index e3f15e061ab..2bd720dba93 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/RunInContainerTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/RunInContainerTest.java
@@ -1,3 +1,4 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
import com.yahoo.application.Networking;
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAdminImplTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java
index 737dd25092a..b37a5abb786 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAdminImplTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java
@@ -1,12 +1,15 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin;
+package com.yahoo.vespa.hosted.node.admin.nodeadmin;
import com.yahoo.collections.Pair;
import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.Container;
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.docker.Docker;
import com.yahoo.vespa.hosted.node.admin.docker.DockerImage;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
+import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
import org.junit.Test;
import org.mockito.InOrder;
@@ -76,30 +79,27 @@ public class NodeAdminImplTest {
nodeAdmin.synchronizeLocalContainerState(asList(nodeSpec), asList(existingContainer));
inOrder.verify(nodeAgentFactory).apply(hostName);
inOrder.verify(nodeAgent1).start();
- inOrder.verify(nodeAgent1).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO);
+ inOrder.verify(nodeAgent1).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, false);
inOrder.verify(nodeAgent1, never()).stop();
nodeAdmin.synchronizeLocalContainerState(asList(nodeSpec), asList(existingContainer));
inOrder.verify(nodeAgentFactory, never()).apply(any(HostName.class));
inOrder.verify(nodeAgent1, never()).start();
- inOrder.verify(nodeAgent1).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO);
+ inOrder.verify(nodeAgent1).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, false);
inOrder.verify(nodeAgent1, never()).stop();
-
nodeAdmin.synchronizeLocalContainerState(Collections.emptyList(), asList(existingContainer));
inOrder.verify(nodeAgentFactory, never()).apply(any(HostName.class));
- inOrder.verify(nodeAgent1, never()).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO);
verify(nodeAgent1).stop();
nodeAdmin.synchronizeLocalContainerState(asList(nodeSpec), asList(existingContainer));
inOrder.verify(nodeAgentFactory).apply(hostName);
inOrder.verify(nodeAgent2).start();
- inOrder.verify(nodeAgent2).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO);
inOrder.verify(nodeAgent2, never()).stop();
nodeAdmin.synchronizeLocalContainerState(Collections.emptyList(), Collections.emptyList());
inOrder.verify(nodeAgentFactory, never()).apply(any(HostName.class));
inOrder.verify(nodeAgent2, never()).start();
- inOrder.verify(nodeAgent2, never()).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO);
+ inOrder.verify(nodeAgent2).execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, false);
inOrder.verify(nodeAgent2).stop();
verifyNoMoreInteractions(nodeAgent1);
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAdminStateUpdaterTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminStateUpdaterTest.java
index f2d1037062b..b56bd5d5ba2 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAdminStateUpdaterTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminStateUpdaterTest.java
@@ -1,7 +1,8 @@
-package com.yahoo.vespa.hosted.node.admin;
-
+package com.yahoo.vespa.hosted.node.admin.nodeadmin;
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
import com.yahoo.prelude.semantics.RuleBaseException;
import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.integrationTests.OrchestratorMock;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepository;
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImplTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImplTest.java
index f17c3919d81..219dc8282b2 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/NodeAgentImplTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeAgentImplTest.java
@@ -1,7 +1,8 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin;
+package com.yahoo.vespa.hosted.node.admin.nodeagent;
import com.yahoo.vespa.applicationmodel.HostName;
+import com.yahoo.vespa.hosted.node.admin.ContainerNodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.Container;
import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
import com.yahoo.vespa.hosted.node.admin.docker.Docker;
@@ -10,7 +11,6 @@ import com.yahoo.vespa.hosted.node.admin.docker.ProcessResult;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeRepository;
import com.yahoo.vespa.hosted.node.admin.noderepository.NodeState;
import com.yahoo.vespa.hosted.node.admin.orchestrator.Orchestrator;
-import com.yahoo.vespa.hosted.node.admin.orchestrator.OrchestratorException;
import org.junit.Test;
import org.mockito.InOrder;
@@ -42,7 +42,7 @@ public class NodeAgentImplTest {
private static final Optional<Double> MIN_CPU_CORES = Optional.of(1.0);
private static final Optional<Double> MIN_MAIN_MEMORY_AVAILABLE_GB = Optional.of(1.0);
private static final Optional<Double> MIN_DISK_AVAILABLE_GB = Optional.of(1.0);
-
+
private static final Optional<Container> NO_CONTAINER = Optional.empty();
private static final ProcessResult NODE_PROGRAM_DOESNT_EXIST = new ProcessResult(1, "");
@@ -52,7 +52,7 @@ public class NodeAgentImplTest {
private final NodeRepository nodeRepository = mock(NodeRepository.class);
private final Orchestrator orchestrator = mock(Orchestrator.class);
- private final NodeAgentImpl nodeAgent = new NodeAgentImpl(hostName, docker, nodeRepository, orchestrator);
+ private final NodeAgent nodeAgent = new NodeAgentImpl(hostName, nodeRepository, orchestrator, new NodeDocker(docker));
@Test
public void upToDateContainerIsUntouched() throws Exception {
@@ -72,12 +72,14 @@ public class NodeAgentImplTest {
final boolean isRunning = true;
final Container existingContainer = new Container(hostName, dockerImage, containerName, isRunning);
final String vespaVersion = "7.8.9";
-
+
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
when(docker.executeInContainer(eq(containerName), anyVararg())).thenReturn(NODE_PROGRAM_DOESNT_EXIST);
when(docker.getVespaVersion(containerName)).thenReturn(vespaVersion);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer));
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -114,13 +116,17 @@ public class NodeAgentImplTest {
final boolean isRunning = true;
final Container existingContainer = new Container(hostName, dockerImage, containerName, isRunning);
final String vespaVersion = "7.8.9";
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+ ;
+
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
when(docker.executeInContainer(eq(containerName), anyVararg())).thenReturn(NODE_PROGRAM_DOESNT_EXIST);
when(docker.getVespaVersion(containerName)).thenReturn(vespaVersion);
when(orchestrator.suspend(any(HostName.class))).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
final InOrder inOrder = inOrder(orchestrator, docker, nodeRepository);
inOrder.verify(orchestrator).suspend(hostName);
@@ -134,8 +140,10 @@ public class NodeAgentImplTest {
nodeSpec.minCpuCores.get(),
nodeSpec.minDiskAvailableGb.get(),
nodeSpec.minMainMemoryAvailableGb.get());
+
inOrder.verify(docker, times(1)).executeInContainer(any(), anyVararg());
inOrder.verify(nodeRepository).updateNodeAttributes(hostName, wantedRestartGeneration, dockerImage, vespaVersion);
+
inOrder.verify(orchestrator).resume(hostName);
}
@@ -164,7 +172,10 @@ public class NodeAgentImplTest {
when(docker.getVespaVersion(containerName)).thenReturn(vespaVersion);
when(orchestrator.suspend(any(HostName.class))).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
final InOrder inOrder = inOrder(orchestrator, docker, nodeRepository);
inOrder.verify(orchestrator).suspend(hostName);
@@ -200,11 +211,13 @@ public class NodeAgentImplTest {
MIN_MAIN_MEMORY_AVAILABLE_GB,
MIN_DISK_AVAILABLE_GB);
final Container existingContainer = new Container(hostName, oldDockerImage, containerName, true);
-
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer));
when(docker.imageIsDownloaded(newDockerImage)).thenReturn(false);
when(docker.pullImageAsync(newDockerImage)).thenReturn(new CompletableFuture<>());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
verify(docker, never()).stopContainer(containerName);
verify(docker).pullImageAsync(newDockerImage);
@@ -227,13 +240,19 @@ public class NodeAgentImplTest {
MIN_DISK_AVAILABLE_GB);
final boolean isRunning = false;
final Container existingContainer = new Container(hostName, dockerImage, containerName, isRunning);
+ final Container existingContainer2 = new Container(hostName, dockerImage, containerName, true);
+
final String vespaVersion = "7.8.9";
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty()).thenReturn(Optional.of(existingContainer2));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
when(docker.executeInContainer(eq(containerName), anyVararg())).thenReturn(NODE_PROGRAM_DOESNT_EXIST);
+
when(docker.getVespaVersion(containerName)).thenReturn(vespaVersion);
+ when(orchestrator.suspend(any(HostName.class))).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(docker, never()).stopContainer(any(ContainerName.class));
verify(orchestrator, never()).suspend(any(HostName.class));
@@ -270,9 +289,13 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
when(docker.executeInContainer(eq(containerName), anyVararg())).thenReturn(NODE_PROGRAM_DOESNT_EXIST);
+
when(docker.getVespaVersion(containerName)).thenReturn(vespaVersion);
+ when(orchestrator.suspend(any(HostName.class))).thenReturn(true);
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+ when(docker.getContainer(hostName)).thenReturn(Optional.empty());
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(docker, never()).stopContainer(any(ContainerName.class));
verify(docker, never()).deleteContainer(any(ContainerName.class));
@@ -310,14 +333,11 @@ public class NodeAgentImplTest {
final Container existingContainer = new Container(hostName, dockerImage, containerName, isRunning);
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- when(orchestrator.suspend(any(HostName.class))).thenReturn(false);
- try {
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
- fail("permission to suspend should fail so we should never get here");
- } catch (OrchestratorException e) {
- // expected
- }
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator).suspend(hostName);
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -354,7 +374,10 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
final InOrder inOrder = inOrder(orchestrator, docker);
@@ -393,7 +416,10 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -429,7 +455,10 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
+ when(docker.getContainer(hostName)).thenReturn(Optional.empty());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -467,7 +496,10 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
final InOrder inOrder = inOrder(orchestrator, docker);
@@ -506,7 +538,12 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
+
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -542,7 +579,13 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
+ when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
+
+ when(docker.getContainer(hostName)).thenReturn(Optional.empty());
+
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -580,7 +623,12 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
+
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
final InOrder inOrder = inOrder(orchestrator, docker, nodeRepository);
@@ -620,7 +668,12 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.getContainer(hostName)).thenReturn(Optional.empty());
+
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -658,7 +711,12 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
+ when(docker.getContainer(hostName)).thenReturn(Optional.empty());
+
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -696,7 +754,12 @@ public class NodeAgentImplTest {
when(docker.imageIsDownloaded(dockerImage)).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
+
+ when(docker.getContainer(hostName)).thenReturn(Optional.empty());
+
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
verify(orchestrator, never()).suspend(any(HostName.class));
verify(docker, never()).stopContainer(any(ContainerName.class));
@@ -754,33 +817,50 @@ public class NodeAgentImplTest {
final InOrder inOrder = inOrder(nodeRepository, docker);
- nodeAgent.synchronizeLocalContainerState(nodeSpec1, Optional.of(existingContainer1));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer1));
+
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec1));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
inOrder.verify(docker, times(1)).executeInContainer(any(), anyVararg());
// Should get exactly one invocation.
inOrder.verify(nodeRepository).updateNodeAttributes(hostName, restartGeneration, dockerImage1, vespaVersion);
verify(nodeRepository, times(1)).updateNodeAttributes(
any(HostName.class), anyLong(), any(DockerImage.class), anyString());
- nodeAgent.synchronizeLocalContainerState(nodeSpec1, Optional.of(existingContainer1));
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
inOrder.verify(docker, never()).executeInContainer(any(), anyVararg());
// No attributes have changed; no second invocation should take place.
verify(nodeRepository, times(1)).updateNodeAttributes(
any(HostName.class), anyLong(), any(DockerImage.class), anyString());
- nodeAgent.synchronizeLocalContainerState(nodeSpec2, Optional.of(existingContainer1));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer1));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec2));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
inOrder.verify(docker, times(2)).executeInContainer(any(), anyVararg());
// One attribute has changed, should cause new invocation.
inOrder.verify(nodeRepository).updateNodeAttributes(hostName, restartGeneration, dockerImage2, vespaVersion);
verify(nodeRepository, times(2)).updateNodeAttributes(
any(HostName.class), anyLong(), any(DockerImage.class), anyString());
- nodeAgent.synchronizeLocalContainerState(nodeSpec2, Optional.of(existingContainer2));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer2));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec2));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
inOrder.verify(docker, never()).executeInContainer(any(), anyVararg());
// No attributes have changed; no new invocation should take place.
verify(nodeRepository, times(2)).updateNodeAttributes(
any(HostName.class), anyLong(), any(DockerImage.class), anyString());
- nodeAgent.synchronizeLocalContainerState(nodeSpec1, Optional.of(existingContainer2));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer2));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec1));
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
inOrder.verify(docker, times(2)).executeInContainer(any(), anyVararg());
// Back to previous node spec should also count as new data and cause a new invocation.
inOrder.verify(nodeRepository).updateNodeAttributes(hostName, restartGeneration, dockerImage1, vespaVersion);
@@ -816,18 +896,19 @@ public class NodeAgentImplTest {
final InOrder inOrder = inOrder(nodeRepository);
- try {
- nodeAgent.synchronizeLocalContainerState(nodeSpec1, Optional.of(existingContainer));
- fail("Should throw exception");
- } catch (IOException e) {
- // As expected.
- }
+
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer));
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec1));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
// Should get exactly one invocation.
inOrder.verify(nodeRepository).updateNodeAttributes(hostName, restartGeneration, dockerImage1, vespaVersion);
verify(nodeRepository, times(1)).updateNodeAttributes(
any(HostName.class), anyLong(), any(DockerImage.class), anyString());
- nodeAgent.synchronizeLocalContainerState(nodeSpec1, Optional.of(existingContainer));
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
// First attribute update failed, so it should be retried now.
inOrder.verify(nodeRepository).updateNodeAttributes(hostName, restartGeneration, dockerImage1, vespaVersion);
verify(nodeRepository, times(2)).updateNodeAttributes(
@@ -869,12 +950,10 @@ public class NodeAgentImplTest {
final InOrder inOrder = inOrder(orchestrator, docker);
// 1st try
- try {
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
- fail("Should have been a failure");
- } catch (Exception e) {
- // expected
- }
+ when(docker.getContainer(hostName)).thenReturn(NO_CONTAINER);
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
inOrder.verify(docker).startContainer(
nodeSpec.wantedDockerImage.get(),
@@ -887,31 +966,27 @@ public class NodeAgentImplTest {
inOrder.verifyNoMoreInteractions();
// 2nd try
- try {
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
- fail("Should have been a failure");
- } catch (Exception e) {
- // expected
- }
-
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
inOrder.verify(docker, times(2)).executeInContainer(any(), anyVararg());
inOrder.verifyNoMoreInteractions();
// 3rd try success
- nodeAgent.synchronizeLocalContainerState(nodeSpec, NO_CONTAINER);
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
inOrder.verify(docker, times(2)).executeInContainer(any(), anyVararg());
inOrder.verify(orchestrator).resume(hostName);
inOrder.verifyNoMoreInteractions();
// 4th and 5th times, already started, no calls to executeInContainer
- nodeAgent.synchronizeLocalContainerState(nodeSpec, uptodateContainer);
+ when(docker.getContainer(hostName)).thenReturn(uptodateContainer);
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
inOrder.verify(docker, never()).executeInContainer(any(), anyVararg());
inOrder.verify(orchestrator).resume(hostName);
inOrder.verifyNoMoreInteractions();
- nodeAgent.synchronizeLocalContainerState(nodeSpec, uptodateContainer);
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
inOrder.verify(docker, never()).executeInContainer(any(), anyVararg());
inOrder.verify(orchestrator).resume(hostName);
@@ -920,7 +995,8 @@ public class NodeAgentImplTest {
// The suspend program can fail by returning non-zero exit code, or throw IOException.
private enum NodeProgramFailureScenario {
- EXCEPTION, NODE_PROGRAM_FAILURE}
+ EXCEPTION, NODE_PROGRAM_FAILURE
+ }
private void failSuspendProgram(NodeProgramFailureScenario scenario) throws Exception {
final long restartGeneration = 1;
@@ -960,7 +1036,11 @@ public class NodeAgentImplTest {
when(docker.getVespaVersion(containerName)).thenReturn(vespaVersion);
when(orchestrator.suspend(any(HostName.class))).thenReturn(true);
- nodeAgent.synchronizeLocalContainerState(nodeSpec, Optional.of(existingContainer));
+ when(docker.getContainer(hostName)).thenReturn(Optional.of(existingContainer)).thenReturn(Optional.empty());
+ when(nodeRepository.getContainerNodeSpec(hostName)).thenReturn(Optional.of(nodeSpec));
+
+ nodeAgent.execute(NodeAgent.Command.UPDATE_FROM_NODE_REPO, true);
+
final InOrder inOrder = inOrder(orchestrator, docker, nodeRepository);
inOrder.verify(orchestrator).suspend(hostName);
@@ -1007,67 +1087,4 @@ public class NodeAgentImplTest {
public void suspendFailureIsIgnored() throws Exception {
failSuspendProgram(NodeProgramFailureScenario.NODE_PROGRAM_FAILURE);
}
-
- @Test
- public void absenceOfNodeProgramIsSuccess() throws Exception {
- final ContainerName containerName = new ContainerName("container-name");
- final String programPath = "/bin/command";
-
- when(docker.executeInContainer(any(), anyVararg())).thenReturn(new ProcessResult(3, "output"));
-
- Optional<ProcessResult> result = NodeAgentImpl.executeOptionalProgram(
- docker,
- containerName,
- programPath,
- "arg1",
- "arg2");
-
- String[] nodeProgramExistsCommand = NodeAgentImpl.programExistsCommand(programPath);
- assertThat(nodeProgramExistsCommand.length, is(4));
-
- verify(docker, times(1)).executeInContainer(
- eq(containerName),
- // Mockito fails if we put the array here instead...
- eq(nodeProgramExistsCommand[0]),
- eq(nodeProgramExistsCommand[1]),
- eq(nodeProgramExistsCommand[2]),
- eq(nodeProgramExistsCommand[3]));
- assertThat(result.isPresent(), is(false));
- }
-
- @Test
- public void processResultFromNodeProgramWhenPresent() throws Exception {
- final ContainerName containerName = new ContainerName("container-name");
- final ProcessResult actualResult = new ProcessResult(3, "output");
- final String programPath = "/bin/command";
- final String[] command = new String[] {programPath, "arg"};
-
- when(docker.executeInContainer(any(), anyVararg()))
- .thenReturn(new ProcessResult(0, "")) // node program exists
- .thenReturn(actualResult); // output from node program
-
- Optional<ProcessResult> result = NodeAgentImpl.executeOptionalProgram(
- docker,
- containerName,
- command);
-
- String[] nodeProgramExistsCommand = NodeAgentImpl.programExistsCommand(programPath);
- assertThat(nodeProgramExistsCommand.length, is(4));
-
- final InOrder inOrder = inOrder(docker);
- inOrder.verify(docker, times(1)).executeInContainer(
- eq(containerName),
- // Mockito fails if we put the array here instead...
- eq(nodeProgramExistsCommand[0]),
- eq(nodeProgramExistsCommand[1]),
- eq(nodeProgramExistsCommand[2]),
- eq(nodeProgramExistsCommand[3]));
- inOrder.verify(docker, times(1)).executeInContainer(
- eq(containerName),
- eq(command[0]),
- eq(command[1]));
-
- assertThat(result.isPresent(), is(true));
- assertThat(result.get(), is(actualResult));
- }
}
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDockerTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDockerTest.java
new file mode 100644
index 00000000000..a2c81a83c80
--- /dev/null
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeagent/NodeDockerTest.java
@@ -0,0 +1,83 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.nodeagent;
+
+import com.yahoo.vespa.hosted.node.admin.docker.ContainerName;
+import com.yahoo.vespa.hosted.node.admin.docker.Docker;
+import com.yahoo.vespa.hosted.node.admin.docker.ProcessResult;
+import org.junit.Test;
+import org.mockito.InOrder;
+
+import java.util.Optional;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyVararg;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.*;
+
+public class NodeDockerTest {
+ private final Docker docker = mock(Docker.class);
+ private final NodeDocker nodeDocker = new NodeDocker(docker);
+
+ @Test
+ public void absenceOfNodeProgramIsSuccess() throws Exception {
+ final ContainerName containerName = new ContainerName("container-name");
+ final String programPath = "/bin/command";
+
+ when(docker.executeInContainer(any(), anyVararg())).thenReturn(new ProcessResult(3, "output"));
+
+ Optional<ProcessResult> result = nodeDocker.executeOptionalProgram(
+ containerName,
+ programPath,
+ "arg1",
+ "arg2");
+
+ String[] nodeProgramExistsCommand = nodeDocker.programExistsCommand(programPath);
+ assertThat(nodeProgramExistsCommand.length, is(4));
+
+ verify(docker, times(1)).executeInContainer(
+ eq(containerName),
+ // Mockito fails if we put the array here instead...
+ eq(nodeProgramExistsCommand[0]),
+ eq(nodeProgramExistsCommand[1]),
+ eq(nodeProgramExistsCommand[2]),
+ eq(nodeProgramExistsCommand[3]));
+ assertThat(result.isPresent(), is(false));
+ }
+
+ @Test
+ public void processResultFromNodeProgramWhenPresent() throws Exception {
+ final ContainerName containerName = new ContainerName("container-name");
+ final ProcessResult actualResult = new ProcessResult(3, "output");
+ final String programPath = "/bin/command";
+ final String[] command = new String[] {programPath, "arg"};
+
+ when(docker.executeInContainer(any(), anyVararg()))
+ .thenReturn(new ProcessResult(0, "")) // node program exists
+ .thenReturn(actualResult); // output from node program
+
+ Optional<ProcessResult> result = nodeDocker.executeOptionalProgram(
+ containerName,
+ command);
+
+ String[] nodeProgramExistsCommand = nodeDocker.programExistsCommand(programPath);
+ assertThat(nodeProgramExistsCommand.length, is(4));
+
+ final InOrder inOrder = inOrder(docker);
+ inOrder.verify(docker, times(1)).executeInContainer(
+ eq(containerName),
+ // Mockito fails if we put the array here instead...
+ eq(nodeProgramExistsCommand[0]),
+ eq(nodeProgramExistsCommand[1]),
+ eq(nodeProgramExistsCommand[2]),
+ eq(nodeProgramExistsCommand[3]));
+ inOrder.verify(docker, times(1)).executeInContainer(
+ eq(containerName),
+ eq(command[0]),
+ eq(command[1]));
+
+ assertThat(result.isPresent(), is(true));
+ assertThat(result.get(), is(actualResult));
+ }
+} \ No newline at end of file