summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorValerij Fredriksen <freva@users.noreply.github.com>2018-10-09 16:54:08 +0200
committerGitHub <noreply@github.com>2018-10-09 16:54:08 +0200
commitb9f12b1196a90ca19d6663da98eb088e4ba21c23 (patch)
treef757ec1d5220f8ad678f2aa33fa29cf11fac767d
parent183deed70c0ef01ba4615d2b588a2e7b3db13aae (diff)
parent2e3f4c8c3191ecd70685205fc1ae34f4e00e9efe (diff)
Merge pull request #7257 from vespa-engine/revert-7252-revert-7245-revert-7239-freva/new-file-helper
Revert "Revert "Revert "NodeAdmin: new file helper"""
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelper.java177
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainer.java248
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/coredump/CoredumpHandler.java58
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java5
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelper.java263
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerTester.java4
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/RunInContainerTest.java2
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/StorageMaintainerMock.java11
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelperTest.java324
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainerTest.java85
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java4
-rw-r--r--node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelperTest.java201
12 files changed, 855 insertions, 527 deletions
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelper.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelper.java
new file mode 100644
index 00000000000..cf010121c2a
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelper.java
@@ -0,0 +1,177 @@
+// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.maintenance;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.nio.file.LinkOption;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.attribute.FileTime;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.logging.Logger;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * @author freva
+ */
+public class FileHelper {
+ private static final Logger logger = Logger.getLogger(FileHelper.class.getSimpleName());
+
+ /**
+ * (Recursively) deletes files if they match all the criteria, also deletes empty directories.
+ *
+ * @param basePath Base path from where to start the search
+ * @param maxAge Delete files older (last modified date) than maxAge
+ * @param fileNameRegex Delete files where filename matches fileNameRegex
+ * @param recursive Delete files in sub-directories (with the same criteria)
+ */
+ public static void deleteFiles(Path basePath, Duration maxAge, Optional<String> fileNameRegex, boolean recursive) throws IOException {
+ Pattern fileNamePattern = fileNameRegex.map(Pattern::compile).orElse(null);
+
+ for (Path path : listContentsOfDirectory(basePath)) {
+ if (Files.isDirectory(path)) {
+ if (recursive) {
+ deleteFiles(path, maxAge, fileNameRegex, true);
+ if (listContentsOfDirectory(path).isEmpty() && !Files.deleteIfExists(path)) {
+ logger.warning("Could not delete directory: " + path.toAbsolutePath());
+ }
+ }
+ } else if (isPatternMatchingFilename(fileNamePattern, path) &&
+ isTimeSinceLastModifiedMoreThan(path, maxAge)) {
+ if (! Files.deleteIfExists(path)) {
+ logger.warning("Could not delete file: " + path.toAbsolutePath());
+ }
+ }
+ }
+ }
+
+ /**
+ * Deletes all files in target directory except the n most recent (by modified date)
+ *
+ * @param basePath Base path to delete from
+ * @param nMostRecentToKeep Number of most recent files to keep
+ */
+ static void deleteFilesExceptNMostRecent(Path basePath, int nMostRecentToKeep) throws IOException {
+ if (nMostRecentToKeep < 1) {
+ throw new IllegalArgumentException("Number of files to keep must be a positive number");
+ }
+
+ List<Path> pathsInDeleteDir = listContentsOfDirectory(basePath).stream()
+ .filter(Files::isRegularFile)
+ .sorted(Comparator.comparing(FileHelper::getLastModifiedTime))
+ .skip(nMostRecentToKeep)
+ .collect(Collectors.toList());
+
+ for (Path path : pathsInDeleteDir) {
+ if (!Files.deleteIfExists(path)) {
+ logger.warning("Could not delete file: " + path.toAbsolutePath());
+ }
+ }
+ }
+
+ static void deleteFilesLargerThan(Path basePath, long sizeInBytes) throws IOException {
+ for (Path path : listContentsOfDirectory(basePath)) {
+ if (Files.isDirectory(path)) {
+ deleteFilesLargerThan(path, sizeInBytes);
+ } else {
+ if (Files.size(path) > sizeInBytes && !Files.deleteIfExists(path)) {
+ logger.warning("Could not delete file: " + path.toAbsolutePath());
+ }
+ }
+ }
+ }
+
+ /**
+ * Deletes directories and their contents if they match all the criteria
+ *
+ * @param basePath Base path to delete the directories from
+ * @param maxAge Delete directories older (last modified date) than maxAge
+ * @param dirNameRegex Delete directories where directory name matches dirNameRegex
+ */
+ public static void deleteDirectories(Path basePath, Duration maxAge, Optional<String> dirNameRegex) throws IOException {
+ Pattern dirNamePattern = dirNameRegex.map(Pattern::compile).orElse(null);
+
+ for (Path path : listContentsOfDirectory(basePath)) {
+ if (Files.isDirectory(path) && isPatternMatchingFilename(dirNamePattern, path)) {
+ boolean mostRecentFileModifiedBeforeMaxAge = getMostRecentlyModifiedFileIn(path)
+ .map(mostRecentlyModified -> isTimeSinceLastModifiedMoreThan(mostRecentlyModified, maxAge))
+ .orElse(true);
+
+ if (mostRecentFileModifiedBeforeMaxAge) {
+ deleteFiles(path, Duration.ZERO, Optional.empty(), true);
+ if (listContentsOfDirectory(path).isEmpty() && !Files.deleteIfExists(path)) {
+ logger.warning("Could not delete directory: " + path.toAbsolutePath());
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Similar to rm -rf file:
+ * - It's not an error if file doesn't exist
+ * - If file is a directory, it and all content is removed
+ * - For symlinks: Only the symlink is removed, not what the symlink points to
+ */
+ public static void recursiveDelete(Path basePath) throws IOException {
+ if (Files.isDirectory(basePath)) {
+ for (Path path : listContentsOfDirectory(basePath)) {
+ recursiveDelete(path);
+ }
+ }
+
+ Files.deleteIfExists(basePath);
+ }
+
+ public static void moveIfExists(Path from, Path to) throws IOException {
+ if (Files.exists(from)) {
+ Files.move(from, to);
+ }
+ }
+
+ private static Optional<Path> getMostRecentlyModifiedFileIn(Path basePath) throws IOException {
+ return Files.walk(basePath).max(Comparator.comparing(FileHelper::getLastModifiedTime));
+ }
+
+ private static boolean isTimeSinceLastModifiedMoreThan(Path path, Duration duration) {
+ Instant nowMinusDuration = Instant.now().minus(duration);
+ Instant lastModified = getLastModifiedTime(path).toInstant();
+
+ // Return true also if they are equal for test stability
+ // (lastModified <= nowMinusDuration) is the same as !(lastModified > nowMinusDuration)
+ return !lastModified.isAfter(nowMinusDuration);
+ }
+
+ private static boolean isPatternMatchingFilename(Pattern pattern, Path path) {
+ return pattern == null || pattern.matcher(path.getFileName().toString()).find();
+ }
+
+ /**
+ * @return list all files in a directory, returns empty list if directory does not exist
+ */
+ public static List<Path> listContentsOfDirectory(Path basePath) {
+ try (Stream<Path> directoryStream = Files.list(basePath)) {
+ return directoryStream.collect(Collectors.toList());
+ } catch (NoSuchFileException ignored) {
+ return Collections.emptyList();
+ } catch (IOException e) {
+ throw new UncheckedIOException("Failed to list contents of directory " + basePath.toAbsolutePath(), e);
+ }
+ }
+
+ static FileTime getLastModifiedTime(Path path) {
+ try {
+ return Files.getLastModifiedTime(path, LinkOption.NOFOLLOW_LINKS);
+ } catch (IOException e) {
+ throw new UncheckedIOException("Failed to get last modified time of " + path.toAbsolutePath(), e);
+ }
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainer.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainer.java
index 1affe890eee..236415d1bcd 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainer.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainer.java
@@ -1,17 +1,23 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.maintenance;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
import com.yahoo.collections.Pair;
import com.yahoo.config.provision.NodeType;
import com.yahoo.io.IOUtils;
import com.yahoo.system.ProcessExecuter;
import com.yahoo.vespa.hosted.dockerapi.ContainerName;
+import com.yahoo.vespa.hosted.dockerapi.metrics.CounterWrapper;
+import com.yahoo.vespa.hosted.dockerapi.metrics.Dimensions;
+import com.yahoo.vespa.hosted.dockerapi.metrics.GaugeWrapper;
+import com.yahoo.vespa.hosted.dockerapi.metrics.MetricReceiverWrapper;
import com.yahoo.vespa.hosted.node.admin.configserver.noderepository.NodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.DockerNetworking;
import com.yahoo.vespa.hosted.node.admin.docker.DockerOperations;
import com.yahoo.vespa.hosted.node.admin.logging.FilebeatConfigProvider;
import com.yahoo.vespa.hosted.node.admin.component.Environment;
-import com.yahoo.vespa.hosted.node.admin.task.util.file.FileHelper;
import com.yahoo.vespa.hosted.node.admin.task.util.file.IOExceptionUtil;
import com.yahoo.vespa.hosted.node.admin.util.PrefixLogger;
import com.yahoo.vespa.hosted.node.admin.util.SecretAgentCheckConfig;
@@ -19,10 +25,13 @@ import com.yahoo.vespa.hosted.node.admin.maintenance.coredump.CoredumpHandler;
import java.io.IOException;
import java.io.InputStreamReader;
+import java.io.UncheckedIOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
+import java.time.Clock;
import java.time.Duration;
+import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -30,6 +39,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import java.util.stream.Stream;
@@ -40,18 +50,41 @@ import static com.yahoo.vespa.defaults.Defaults.getDefaults;
* @author freva
*/
public class StorageMaintainer {
+ private static final ContainerName NODE_ADMIN = new ContainerName("node-admin");
+ private static final ObjectMapper objectMapper = new ObjectMapper();
+ private final GaugeWrapper numberOfCoredumpsOnHost;
+ private final CounterWrapper numberOfNodeAdminMaintenanceFails;
private final DockerOperations dockerOperations;
private final ProcessExecuter processExecuter;
private final Environment environment;
- private final CoredumpHandler coredumpHandler;
+ private final Optional<CoredumpHandler> coredumpHandler;
+ private final Clock clock;
+
+ private final Map<ContainerName, MaintenanceThrottler> maintenanceThrottlerByContainerName = new ConcurrentHashMap<>();
+
+ public StorageMaintainer(DockerOperations dockerOperations, ProcessExecuter processExecuter,
+ MetricReceiverWrapper metricReceiver, Environment environment, Clock clock) {
+ this(dockerOperations, processExecuter, metricReceiver, environment, null, clock);
+ }
public StorageMaintainer(DockerOperations dockerOperations, ProcessExecuter processExecuter,
- Environment environment, CoredumpHandler coredumpHandler) {
+ MetricReceiverWrapper metricReceiver, Environment environment,
+ CoredumpHandler coredumpHandler, Clock clock) {
this.dockerOperations = dockerOperations;
this.processExecuter = processExecuter;
this.environment = environment;
- this.coredumpHandler = coredumpHandler;
+ this.coredumpHandler = Optional.ofNullable(coredumpHandler);
+ this.clock = clock;
+
+ Dimensions dimensions = new Dimensions.Builder()
+ .add("role", SecretAgentCheckConfig.nodeTypeToRole(environment.getNodeType()))
+ .build();
+ numberOfNodeAdminMaintenanceFails = metricReceiver.declareCounter(MetricReceiverWrapper.APPLICATION_DOCKER, dimensions, "nodes.maintenance.fails");
+ numberOfCoredumpsOnHost = metricReceiver.declareGauge(MetricReceiverWrapper.APPLICATION_DOCKER, dimensions, "nodes.coredumps");
+
+ metricReceiver.declareCounter(MetricReceiverWrapper.APPLICATION_DOCKER, dimensions, "nodes.running_on_host")
+ .add(environment.isRunningOnHost() ? 1 : 0);
}
public void writeMetricsConfig(ContainerName containerName, NodeSpec node) {
@@ -208,7 +241,17 @@ public class StorageMaintainer {
* Deletes old log files for vespa, nginx, logstash, etc.
*/
public void removeOldFilesFromNode(ContainerName containerName) {
- Path[] logPaths = {
+ if (! getMaintenanceThrottlerFor(containerName).shouldRemoveOldFilesNow()) return;
+
+ MaintainerExecutor maintainerExecutor = new MaintainerExecutor();
+ addRemoveOldFilesCommand(maintainerExecutor, containerName);
+
+ maintainerExecutor.execute();
+ getMaintenanceThrottlerFor(containerName).updateNextRemoveOldFilesTime();
+ }
+
+ private void addRemoveOldFilesCommand(MaintainerExecutor maintainerExecutor, ContainerName containerName) {
+ Path[] pathsToClean = {
environment.pathInNodeUnderVespaHome("logs/elasticsearch2"),
environment.pathInNodeUnderVespaHome("logs/logstash2"),
environment.pathInNodeUnderVespaHome("logs/daemontools_y"),
@@ -216,42 +259,79 @@ public class StorageMaintainer {
environment.pathInNodeUnderVespaHome("logs/vespa")
};
- for (Path pathToClean : logPaths) {
+ for (Path pathToClean : pathsToClean) {
Path path = environment.pathInNodeAdminFromPathInNode(containerName, pathToClean);
- FileHelper.streamFiles(path)
- .filterFile(FileHelper.olderThan(Duration.ofDays(3))
- .and(FileHelper.nameMatches(Pattern.compile(".*\\.log.+"))))
- .delete();
+ if (Files.exists(path)) {
+ maintainerExecutor.addJob("delete-files")
+ .withArgument("basePath", path)
+ .withArgument("maxAgeSeconds", Duration.ofDays(3).getSeconds())
+ .withArgument("fileNameRegex", ".*\\.log.+")
+ .withArgument("recursive", false);
+ }
}
Path qrsDir = environment.pathInNodeAdminFromPathInNode(
containerName, environment.pathInNodeUnderVespaHome("logs/vespa/qrs"));
- FileHelper.streamFiles(qrsDir)
- .filterFile(FileHelper.olderThan(Duration.ofDays(3)))
- .delete();
+ maintainerExecutor.addJob("delete-files")
+ .withArgument("basePath", qrsDir)
+ .withArgument("maxAgeSeconds", Duration.ofDays(3).getSeconds())
+ .withArgument("recursive", false);
Path logArchiveDir = environment.pathInNodeAdminFromPathInNode(
containerName, environment.pathInNodeUnderVespaHome("logs/vespa/logarchive"));
- FileHelper.streamFiles(logArchiveDir)
- .filterFile(FileHelper.olderThan(Duration.ofDays(31)))
- .delete();
+ maintainerExecutor.addJob("delete-files")
+ .withArgument("basePath", logArchiveDir)
+ .withArgument("maxAgeSeconds", Duration.ofDays(31).getSeconds())
+ .withArgument("recursive", false);
Path fileDistrDir = environment.pathInNodeAdminFromPathInNode(
containerName, environment.pathInNodeUnderVespaHome("var/db/vespa/filedistribution"));
- FileHelper.streamFiles(fileDistrDir)
- .filterFile(FileHelper.olderThan(Duration.ofDays(31)))
- .recursive(true)
- .delete();
+ maintainerExecutor.addJob("delete-files")
+ .withArgument("basePath", fileDistrDir)
+ .withArgument("maxAgeSeconds", Duration.ofDays(31).getSeconds())
+ .withArgument("recursive", true);
}
/**
* Checks if container has any new coredumps, reports and archives them if so
*/
public void handleCoreDumpsForContainer(ContainerName containerName, NodeSpec node) {
+ // Sample number of coredumps on the host
+ try (Stream<Path> files = Files.list(environment.pathInNodeAdminToDoneCoredumps())) {
+ numberOfCoredumpsOnHost.sample(files.count());
+ } catch (IOException e) {
+ // Ignore for now - this is either test or a misconfiguration
+ }
+
+ MaintainerExecutor maintainerExecutor = new MaintainerExecutor();
+ addHandleCoredumpsCommand(maintainerExecutor, containerName, node);
+ maintainerExecutor.execute();
+ }
+
+ /**
+ * Will either schedule coredump execution in the given maintainerExecutor or run coredump handling
+ * directly if {@link #coredumpHandler} is set.
+ */
+ private void addHandleCoredumpsCommand(MaintainerExecutor maintainerExecutor, ContainerName containerName, NodeSpec node) {
final Path coredumpsPath = environment.pathInNodeAdminFromPathInNode(
containerName, environment.pathInNodeUnderVespaHome("var/crash"));
final Map<String, Object> nodeAttributes = getCoredumpNodeAttributes(node);
- coredumpHandler.processAll(coredumpsPath, nodeAttributes);
+ if (coredumpHandler.isPresent()) {
+ try {
+ coredumpHandler.get().processAll(coredumpsPath, nodeAttributes);
+ } catch (IOException e) {
+ throw new UncheckedIOException("Failed to process coredumps", e);
+ }
+ } else {
+ // Core dump handling is disabled.
+ if (!environment.getCoredumpFeedEndpoint().isPresent()) return;
+
+ maintainerExecutor.addJob("handle-core-dumps")
+ .withArgument("coredumpsPath", coredumpsPath)
+ .withArgument("doneCoredumpsPath", environment.pathInNodeAdminToDoneCoredumps())
+ .withArgument("attributes", nodeAttributes)
+ .withArgument("feedEndpoint", environment.getCoredumpFeedEndpoint().get());
+ }
}
private Map<String, Object> getCoredumpNodeAttributes(NodeSpec node) {
@@ -274,22 +354,60 @@ public class StorageMaintainer {
}
/**
- * Prepares the container-storage for the next container by archiving container logs to a new directory
- * and deleting everything else owned by this container.
+ * Deletes old
+ * * archived app data
+ * * Vespa logs
+ * * Filedistribution files
+ */
+ public void cleanNodeAdmin() {
+ if (! getMaintenanceThrottlerFor(NODE_ADMIN).shouldRemoveOldFilesNow()) return;
+
+ MaintainerExecutor maintainerExecutor = new MaintainerExecutor();
+ maintainerExecutor.addJob("delete-directories")
+ .withArgument("basePath", environment.getPathResolver().getApplicationStoragePathForNodeAdmin())
+ .withArgument("maxAgeSeconds", Duration.ofDays(7).getSeconds())
+ .withArgument("dirNameRegex", "^" + Pattern.quote(Environment.APPLICATION_STORAGE_CLEANUP_PATH_PREFIX));
+
+ Path nodeAdminJDiskLogsPath = environment.pathInNodeAdminFromPathInNode(
+ NODE_ADMIN, environment.pathInNodeUnderVespaHome("logs/vespa/"));
+ maintainerExecutor.addJob("delete-files")
+ .withArgument("basePath", nodeAdminJDiskLogsPath)
+ .withArgument("maxAgeSeconds", Duration.ofDays(31).getSeconds())
+ .withArgument("recursive", false);
+
+ Path fileDistrDir = environment.pathInNodeAdminFromPathInNode(
+ NODE_ADMIN, environment.pathInNodeUnderVespaHome("var/db/vespa/filedistribution"));
+ maintainerExecutor.addJob("delete-files")
+ .withArgument("basePath", fileDistrDir)
+ .withArgument("maxAgeSeconds", Duration.ofDays(31).getSeconds())
+ .withArgument("recursive", true);
+
+ maintainerExecutor.execute();
+ getMaintenanceThrottlerFor(NODE_ADMIN).updateNextRemoveOldFilesTime();
+ }
+
+ /**
+ * Prepares the container-storage for the next container by deleting/archiving all the data of the current container.
+ * Removes old files, reports coredumps and archives container data, runs when container enters state "dirty"
*/
public void cleanupNodeStorage(ContainerName containerName, NodeSpec node) {
- removeOldFilesFromNode(containerName);
+ MaintainerExecutor maintainerExecutor = new MaintainerExecutor();
+ addRemoveOldFilesCommand(maintainerExecutor, containerName);
+ addHandleCoredumpsCommand(maintainerExecutor, containerName, node);
+ addArchiveNodeData(maintainerExecutor, containerName);
- Path logsDirInContainer = environment.pathInNodeUnderVespaHome("logs");
- Path containerLogsInArchiveDir = environment.pathInNodeAdminToNodeCleanup(containerName).resolve(logsDirInContainer);
- Path containerLogsPathOnHost = environment.pathInNodeAdminFromPathInNode(containerName, logsDirInContainer);
+ maintainerExecutor.execute();
+ getMaintenanceThrottlerFor(containerName).reset();
+ }
- FileHelper.createDirectories(containerLogsInArchiveDir.getParent());
- FileHelper.moveIfExists(containerLogsPathOnHost, containerLogsInArchiveDir);
+ private void addArchiveNodeData(MaintainerExecutor maintainerExecutor, ContainerName containerName) {
+ maintainerExecutor.addJob("recursive-delete")
+ .withArgument("path", environment.pathInNodeAdminFromPathInNode(
+ containerName, environment.pathInNodeUnderVespaHome("var")));
- FileHelper.streamContents(environment.pathInHostFromPathInNode(containerName, Paths.get("/")))
- .includeBase(true)
- .delete();
+ maintainerExecutor.addJob("move-files")
+ .withArgument("from", environment.pathInNodeAdminFromPathInNode(containerName, Paths.get("/")))
+ .withArgument("to", environment.pathInNodeAdminToNodeCleanup(containerName));
}
/**
@@ -334,6 +452,7 @@ public class StorageMaintainer {
Pair<Integer, String> result = processExecuter.exec(command);
if (result.getFirst() != 0) {
+ numberOfNodeAdminMaintenanceFails.add();
throw new RuntimeException(
String.format("Maintainer failed to execute command: %s, Exit code: %d, Stdout/stderr: %s",
Arrays.toString(command), result.getFirst(), result.getSecond()));
@@ -343,4 +462,69 @@ public class StorageMaintainer {
throw new RuntimeException("Failed to execute maintainer", e);
}
}
+
+ /**
+ * Wrapper for node-admin-maintenance, queues up maintenances jobs and sends a single request to maintenance JVM
+ */
+ private class MaintainerExecutor {
+ private final List<MaintainerExecutorJob> jobs = new ArrayList<>();
+
+ MaintainerExecutorJob addJob(String jobName) {
+ MaintainerExecutorJob job = new MaintainerExecutorJob(jobName);
+ jobs.add(job);
+ return job;
+ }
+
+ void execute() {
+ if (jobs.isEmpty()) return;
+
+ String args;
+ try {
+ args = objectMapper.writeValueAsString(jobs);
+ } catch (JsonProcessingException e) {
+ throw new RuntimeException("Failed transform list of maintenance jobs to JSON");
+ }
+
+ executeMaintainer("com.yahoo.vespa.hosted.node.maintainer.Maintainer", args);
+ }
+ }
+
+ private class MaintainerExecutorJob {
+ @JsonProperty(value="type")
+ private final String type;
+
+ @JsonProperty(value="arguments")
+ private final Map<String, Object> arguments = new HashMap<>();
+
+ MaintainerExecutorJob(String type) {
+ this.type = type;
+ }
+
+ MaintainerExecutorJob withArgument(String argument, Object value) {
+ // Transform Path to String, otherwise ObjectMapper wont encode/decode it properly on the other end
+ arguments.put(argument, (value instanceof Path) ? value.toString() : value);
+ return this;
+ }
+ }
+
+ private MaintenanceThrottler getMaintenanceThrottlerFor(ContainerName containerName) {
+ maintenanceThrottlerByContainerName.putIfAbsent(containerName, new MaintenanceThrottler());
+ return maintenanceThrottlerByContainerName.get(containerName);
+ }
+
+ private class MaintenanceThrottler {
+ private Instant nextRemoveOldFilesAt = Instant.EPOCH;
+
+ void updateNextRemoveOldFilesTime() {
+ nextRemoveOldFilesAt = clock.instant().plus(Duration.ofHours(1));
+ }
+
+ boolean shouldRemoveOldFilesNow() {
+ return !nextRemoveOldFilesAt.isAfter(clock.instant());
+ }
+
+ void reset() {
+ nextRemoveOldFilesAt = Instant.EPOCH;
+ }
+ }
}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/coredump/CoredumpHandler.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/coredump/CoredumpHandler.java
index e46b29cc078..eb48086eb0f 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/coredump/CoredumpHandler.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/maintenance/coredump/CoredumpHandler.java
@@ -3,18 +3,19 @@ package com.yahoo.vespa.hosted.node.admin.maintenance.coredump;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.yahoo.system.ProcessExecuter;
-import com.yahoo.vespa.hosted.node.admin.task.util.file.FileHelper;
+import com.yahoo.vespa.hosted.node.admin.maintenance.FileHelper;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
+import java.time.Duration;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
+import java.util.Optional;
import java.util.UUID;
import java.util.logging.Level;
import java.util.logging.Logger;
-import java.util.regex.Pattern;
/**
* Finds coredumps, collects metadata and reports them
@@ -23,7 +24,6 @@ import java.util.regex.Pattern;
*/
public class CoredumpHandler {
- private static final Pattern JAVA_COREDUMP_PATTERN = Pattern.compile("java_pid.*\\.hprof");
private static final String PROCESSING_DIRECTORY_NAME = "processing";
static final String METADATA_FILE_NAME = "metadata.json";
@@ -44,11 +44,23 @@ public class CoredumpHandler {
this.doneCoredumpsPath = doneCoredumpsPath;
}
- public void processAll(Path coredumpsPath, Map<String, Object> nodeAttributes) {
- FileHelper.streamFiles(coredumpsPath)
- .filterFile(FileHelper.nameMatches(JAVA_COREDUMP_PATTERN))
- .delete();
+ public void processAll(Path coredumpsPath, Map<String, Object> nodeAttributes) throws IOException {
+ removeJavaCoredumps(coredumpsPath);
+ handleNewCoredumps(coredumpsPath, nodeAttributes);
+ removeOldCoredumps();
+ }
+
+ private void removeJavaCoredumps(Path coredumpsPath) throws IOException {
+ if (! coredumpsPath.toFile().isDirectory()) return;
+ FileHelper.deleteFiles(coredumpsPath, Duration.ZERO, Optional.of("^java_pid.*\\.hprof$"), false);
+ }
+
+ private void removeOldCoredumps() throws IOException {
+ if (! doneCoredumpsPath.toFile().isDirectory()) return;
+ FileHelper.deleteDirectories(doneCoredumpsPath, Duration.ofDays(10), Optional.empty());
+ }
+ private void handleNewCoredumps(Path coredumpsPath, Map<String, Object> nodeAttributes) {
enqueueCoredumps(coredumpsPath);
processAndReportCoredumps(coredumpsPath, nodeAttributes);
}
@@ -60,14 +72,12 @@ public class CoredumpHandler {
*/
void enqueueCoredumps(Path coredumpsPath) {
Path processingCoredumpsPath = getProcessingCoredumpsPath(coredumpsPath);
- FileHelper.createDirectories(processingCoredumpsPath);
- if (!FileHelper.streamDirectories(processingCoredumpsPath).list().isEmpty()) return;
-
- FileHelper.streamFiles(coredumpsPath)
- .filterFile(FileHelper.nameStartsWith(".").negate())
- .stream()
- .min(Comparator.comparing(FileHelper.FileAttributes::lastModifiedTime))
- .map(FileHelper.FileAttributes::path)
+ processingCoredumpsPath.toFile().mkdirs();
+ if (!FileHelper.listContentsOfDirectory(processingCoredumpsPath).isEmpty()) return;
+
+ FileHelper.listContentsOfDirectory(coredumpsPath).stream()
+ .filter(path -> path.toFile().isFile() && ! path.getFileName().toString().startsWith("."))
+ .min((Comparator.comparingLong(o -> o.toFile().lastModified())))
.ifPresent(coredumpPath -> {
try {
enqueueCoredumpForProcessing(coredumpPath, processingCoredumpsPath);
@@ -79,10 +89,11 @@ public class CoredumpHandler {
void processAndReportCoredumps(Path coredumpsPath, Map<String, Object> nodeAttributes) {
Path processingCoredumpsPath = getProcessingCoredumpsPath(coredumpsPath);
- FileHelper.createDirectories(doneCoredumpsPath);
+ doneCoredumpsPath.toFile().mkdirs();
- FileHelper.streamDirectories(processingCoredumpsPath)
- .forEachPath(coredumpDirectory -> processAndReportSingleCoredump(coredumpDirectory, nodeAttributes));
+ FileHelper.listContentsOfDirectory(processingCoredumpsPath).stream()
+ .filter(path -> path.toFile().isDirectory())
+ .forEach(coredumpDirectory -> processAndReportSingleCoredump(coredumpDirectory, nodeAttributes));
}
private void processAndReportSingleCoredump(Path coredumpDirectory, Map<String, Object> nodeAttributes) {
@@ -98,20 +109,19 @@ public class CoredumpHandler {
}
private void enqueueCoredumpForProcessing(Path coredumpPath, Path processingCoredumpsPath) throws IOException {
+ // Make coredump readable
+ coredumpPath.toFile().setReadable(true, false);
+
// Create new directory for this coredump and move it into it
Path folder = processingCoredumpsPath.resolve(UUID.randomUUID().toString());
-
- FileHelper.createDirectories(folder);
+ folder.toFile().mkdirs();
Files.move(coredumpPath, folder.resolve(coredumpPath.getFileName()));
}
String collectMetadata(Path coredumpDirectory, Map<String, Object> nodeAttributes) throws IOException {
Path metadataPath = coredumpDirectory.resolve(METADATA_FILE_NAME);
if (!Files.exists(metadataPath)) {
- Path coredumpPath = FileHelper.streamFiles(coredumpDirectory)
- .stream()
- .map(FileHelper.FileAttributes::path)
- .findFirst()
+ Path coredumpPath = FileHelper.listContentsOfDirectory(coredumpDirectory).stream().findFirst()
.orElseThrow(() -> new RuntimeException("No coredump file found in processing directory " + coredumpDirectory));
Map<String, Object> metadata = coreCollector.collect(coredumpPath);
metadata.putAll(nodeAttributes);
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java
index 519d83bd7d4..2621156487d 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImpl.java
@@ -9,6 +9,7 @@ import com.yahoo.vespa.hosted.dockerapi.metrics.GaugeWrapper;
import com.yahoo.vespa.hosted.dockerapi.metrics.MetricReceiverWrapper;
import com.yahoo.vespa.hosted.node.admin.configserver.noderepository.NodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.DockerOperations;
+import com.yahoo.vespa.hosted.node.admin.maintenance.StorageMaintainer;
import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
import com.yahoo.vespa.hosted.node.admin.util.PrefixLogger;
@@ -41,6 +42,7 @@ public class NodeAdminImpl implements NodeAdmin {
private final DockerOperations dockerOperations;
private final Function<String, NodeAgent> nodeAgentFactory;
+ private final StorageMaintainer storageMaintainer;
private final Runnable aclMaintainer;
private final Clock clock;
@@ -55,11 +57,13 @@ public class NodeAdminImpl implements NodeAdmin {
public NodeAdminImpl(DockerOperations dockerOperations,
Function<String, NodeAgent> nodeAgentFactory,
+ StorageMaintainer storageMaintainer,
Runnable aclMaintainer,
MetricReceiverWrapper metricReceiver,
Clock clock) {
this.dockerOperations = dockerOperations;
this.nodeAgentFactory = nodeAgentFactory;
+ this.storageMaintainer = storageMaintainer;
this.aclMaintainer = aclMaintainer;
this.clock = clock;
@@ -78,6 +82,7 @@ public class NodeAdminImpl implements NodeAdmin {
.map(NodeSpec::getHostname)
.collect(Collectors.toSet());
+ storageMaintainer.cleanNodeAdmin();
synchronizeNodesToNodeAgents(hostnamesOfContainersToRun);
updateNodeAgentMetrics();
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelper.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelper.java
deleted file mode 100644
index a4b1a66c71b..00000000000
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelper.java
+++ /dev/null
@@ -1,263 +0,0 @@
-// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin.task.util.file;
-
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
-import java.nio.file.SimpleFileVisitor;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Stack;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-import java.util.regex.Pattern;
-import java.util.stream.Stream;
-
-import static com.yahoo.vespa.hosted.node.admin.task.util.file.IOExceptionUtil.uncheck;
-
-/**
- * @author freva
- */
-public class FileHelper {
-
- private final Path basePath;
- private Predicate<FileAttributes> fileFilter;
- private Predicate<FileAttributes> directoryFilter;
- private boolean includeBase = false;
- private boolean recursive = false;
-
- public FileHelper(Path basePath, boolean includeFiles, boolean includeDirectories) {
- this.basePath = basePath;
- this.fileFilter = path -> includeFiles;
- this.directoryFilter = path -> includeDirectories;
- }
-
- /**
- * Creates a {@link FileHelper} that will by default match all files and all directories
- * under the given basePath.
- */
- public static FileHelper streamContents(Path basePath) {
- return new FileHelper(basePath, true, true);
- }
-
- /**
- * Creates a {@link FileHelper} that will by default match all files and no directories
- * under the given basePath.
- */
- public static FileHelper streamFiles(Path basePath) {
- return new FileHelper(basePath, true, false);
- }
-
- /**
- * Creates a {@link FileHelper} that will by default match all directories and no files
- * under the given basePath.
- */
- public static FileHelper streamDirectories(Path basePath) {
- return new FileHelper(basePath, false, true);
- }
-
-
- /**
- * Filter that will be used to match files under the base path. Files include everything that
- * is not a directory (such as symbolic links)
- */
- public FileHelper filterFile(Predicate<FileAttributes> fileFilter) {
- this.fileFilter = fileFilter;
- return this;
- }
-
- /**
- * Filter that will be used to match directories under the base path.
- *
- * NOTE: When a directory is matched, all of its sub-directories and files are also matched
- */
- public FileHelper filterDirectory(Predicate<FileAttributes> directoryFilter) {
- this.directoryFilter = directoryFilter;
- return this;
- }
-
- /**
- * Whether the search should be recursive.
- *
- * WARNING: When using {@link #delete()} and matching directories, make sure that the directories
- * either are already empty or that recursive is set
- */
- public FileHelper recursive(boolean recursive) {
- this.recursive = recursive;
- return this;
- }
-
- /**
- * Whether the base path should also be considered (i.e. checked against the correspoding filter).
- * When using {@link #delete()} with directories, this is the difference between
- * `rm -rf basePath` (true) and `rm -rf basePath/*` (false)
- */
- public FileHelper includeBase(boolean includeBase) {
- this.includeBase = includeBase;
- return this;
- }
-
- public int delete() {
- int[] numDeletions = { 0 }; // :(
- forEach(attributes -> {
- if (deleteIfExists(attributes.path()))
- numDeletions[0]++;
- });
-
- return numDeletions[0];
- }
-
- public List<FileAttributes> list() {
- LinkedList<FileAttributes> list = new LinkedList<>();
- forEach(list::add);
- return list;
- }
-
- public Stream<FileAttributes> stream() {
- return list().stream();
- }
-
- public void forEachPath(Consumer<Path> action) {
- forEach(attributes -> action.accept(attributes.path()));
- }
-
- /** Applies a given consumer to all the matching {@link FileHelper.FileAttributes} */
- public void forEach(Consumer<FileAttributes> action) {
- applyForEachToMatching(basePath, fileFilter, directoryFilter, recursive, includeBase, action);
- }
-
-
- /**
- * <p> This method walks a file tree rooted at a given starting file. The file tree traversal is
- * <em>depth-first</em>: The filter function is applied in pre-order (NLR), but the given
- * {@link Consumer} will be called in post-order (LRN).
- */
- private void applyForEachToMatching(Path basePath, Predicate<FileAttributes> fileFilter, Predicate<FileAttributes> directoryFilter,
- boolean recursive, boolean includeBase, Consumer<FileAttributes> action) {
- try {
- Files.walkFileTree(basePath, Collections.emptySet(), recursive ? Integer.MAX_VALUE : 1, new SimpleFileVisitor<Path>() {
- private Stack<FileAttributes> matchingDirectoryStack = new Stack<>();
- private int currentLevel = -1;
-
- @Override
- public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) {
- currentLevel++;
-
- FileAttributes attributes = new FileAttributes(dir, attrs);
- if (!matchingDirectoryStack.empty() || directoryFilter.test(attributes))
- matchingDirectoryStack.push(attributes);
-
- return FileVisitResult.CONTINUE;
- }
-
- @Override
- public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) {
- // When we find a directory at the max depth given to Files.walkFileTree, the directory
- // will be passed to visitFile() rather than (pre|post)VisitDirectory
- if (attrs.isDirectory()) {
- preVisitDirectory(file, attrs);
- return postVisitDirectory(file, null);
- }
-
- FileAttributes attributes = new FileAttributes(file, attrs);
- if (!matchingDirectoryStack.isEmpty() || fileFilter.test(attributes))
- action.accept(attributes);
-
- return FileVisitResult.CONTINUE;
- }
-
- @Override
- public FileVisitResult postVisitDirectory(Path dir, IOException exc) {
- if (!matchingDirectoryStack.isEmpty()) {
- FileAttributes attributes = matchingDirectoryStack.pop();
- if (currentLevel != 0 || includeBase)
- action.accept(attributes);
- }
-
- currentLevel--;
- return FileVisitResult.CONTINUE;
- }
- });
- } catch (NoSuchFileException ignored) {
-
- } catch (IOException e) {
- throw new UncheckedIOException(e);
- }
- }
-
-
- // Ideally, we would reuse the FileAttributes in this package, but unfortunately we only get
- // BasicFileAttributes and not PosixFileAttributes from FileVisitor
- public static class FileAttributes {
- private final Path path;
- private final BasicFileAttributes attributes;
-
- FileAttributes(Path path, BasicFileAttributes attributes) {
- this.path = path;
- this.attributes = attributes;
- }
-
- public Path path() { return path; }
- public String filename() { return path.getFileName().toString(); }
- public Instant lastModifiedTime() { return attributes.lastModifiedTime().toInstant(); }
- public boolean isRegularFile() { return attributes.isRegularFile(); }
- public boolean isDirectory() { return attributes.isDirectory(); }
- public long size() { return attributes.size(); }
- }
-
-
- // Filters
- public static Predicate<FileAttributes> olderThan(Duration duration) {
- return attrs -> Duration.between(attrs.lastModifiedTime(), Instant.now()).compareTo(duration) > 0;
- }
-
- public static Predicate<FileAttributes> youngerThan(Duration duration) {
- return olderThan(duration).negate();
- }
-
- public static Predicate<FileAttributes> largerThan(long sizeInBytes) {
- return attrs -> attrs.size() > sizeInBytes;
- }
-
- public static Predicate<FileAttributes> smallerThan(long sizeInBytes) {
- return largerThan(sizeInBytes).negate();
- }
-
- public static Predicate<FileAttributes> nameMatches(Pattern pattern) {
- return attrs -> pattern.matcher(attrs.filename()).matches();
- }
-
- public static Predicate<FileAttributes> nameStartsWith(String string) {
- return attrs -> attrs.filename().startsWith(string);
- }
-
- public static Predicate<FileAttributes> nameEndsWith(String string) {
- return attrs -> attrs.filename().endsWith(string);
- }
-
-
- // Other helpful methods that no not throw checked exceptions
- public static boolean moveIfExists(Path from, Path to) {
- try {
- Files.move(from, to);
- return true;
- } catch (IOException e) {
- throw new UncheckedIOException(e);
- }
- }
-
- public static boolean deleteIfExists(Path path) {
- return uncheck(() -> Files.deleteIfExists(path));
- }
-
- public static Path createDirectories(Path path) {
- return uncheck(() -> Files.createDirectories(path));
- }
-}
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerTester.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerTester.java
index 49a03c454c1..15bb2825738 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerTester.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/DockerTester.java
@@ -87,7 +87,7 @@ public class DockerTester implements AutoCloseable {
Clock clock = Clock.systemUTC();
DockerOperations dockerOperations = new DockerOperationsImpl(dockerMock, environment, processExecuter);
- StorageMaintainerMock storageMaintainer = new StorageMaintainerMock(dockerOperations, null, environment, callOrderVerifier);
+ StorageMaintainerMock storageMaintainer = new StorageMaintainerMock(dockerOperations, null, environment, callOrderVerifier, clock);
AclMaintainer aclMaintainer = mock(AclMaintainer.class);
AthenzCredentialsMaintainer athenzCredentialsMaintainer = mock(AthenzCredentialsMaintainer.class);
@@ -95,7 +95,7 @@ public class DockerTester implements AutoCloseable {
MetricReceiverWrapper mr = new MetricReceiverWrapper(MetricReceiver.nullImplementation);
Function<String, NodeAgent> nodeAgentFactory = (hostName) -> new NodeAgentImpl(hostName, nodeRepositoryMock,
orchestratorMock, dockerOperations, storageMaintainer, aclMaintainer, environment, clock, NODE_AGENT_SCAN_INTERVAL, athenzCredentialsMaintainer);
- nodeAdmin = new NodeAdminImpl(dockerOperations, nodeAgentFactory, aclMaintainer, mr, Clock.systemUTC());
+ nodeAdmin = new NodeAdminImpl(dockerOperations, nodeAgentFactory, storageMaintainer, aclMaintainer, mr, Clock.systemUTC());
nodeAdminStateUpdater = new NodeAdminStateUpdaterImpl(nodeRepositoryMock, orchestratorMock, storageMaintainer,
nodeAdmin, DOCKER_HOST_HOSTNAME, clock, NODE_ADMIN_CONVERGE_STATE_INTERVAL,
Optional.of(new ClassLocking()));
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 2a3171762bc..a46defc991b 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
@@ -245,7 +245,7 @@ public class RunInContainerTest {
private final Function<String, NodeAgent> nodeAgentFactory =
(hostName) -> new NodeAgentImpl(hostName, nodeRepositoryMock, orchestratorMock, dockerOperationsMock,
storageMaintainer, aclMaintainer, environment, Clock.systemUTC(), NODE_AGENT_SCAN_INTERVAL, athenzCredentialsMaintainer);
- private final NodeAdmin nodeAdmin = new NodeAdminImpl(dockerOperationsMock, nodeAgentFactory, aclMaintainer, mr, Clock.systemUTC());
+ private final NodeAdmin nodeAdmin = new NodeAdminImpl(dockerOperationsMock, nodeAgentFactory, storageMaintainer, aclMaintainer, mr, Clock.systemUTC());
private final NodeAdminStateUpdaterImpl nodeAdminStateUpdater = new NodeAdminStateUpdaterImpl(nodeRepositoryMock,
orchestratorMock, storageMaintainer, nodeAdmin, "localhost.test.yahoo.com",
Clock.systemUTC(), NODE_ADMIN_CONVERGE_STATE_INTERVAL, Optional.of(new ClassLocking()));
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/StorageMaintainerMock.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/StorageMaintainerMock.java
index 62f1a59ecf2..6b7d545c286 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/StorageMaintainerMock.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/integrationTests/StorageMaintainerMock.java
@@ -1,13 +1,16 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.integrationTests;
+import com.yahoo.metrics.simple.MetricReceiver;
import com.yahoo.system.ProcessExecuter;
import com.yahoo.vespa.hosted.dockerapi.ContainerName;
+import com.yahoo.vespa.hosted.dockerapi.metrics.MetricReceiverWrapper;
import com.yahoo.vespa.hosted.node.admin.configserver.noderepository.NodeSpec;
import com.yahoo.vespa.hosted.node.admin.docker.DockerOperations;
import com.yahoo.vespa.hosted.node.admin.maintenance.StorageMaintainer;
import com.yahoo.vespa.hosted.node.admin.component.Environment;
+import java.time.Clock;
import java.util.Optional;
/**
@@ -16,8 +19,8 @@ import java.util.Optional;
public class StorageMaintainerMock extends StorageMaintainer {
private final CallOrderVerifier callOrderVerifier;
- public StorageMaintainerMock(DockerOperations dockerOperations, ProcessExecuter processExecuter, Environment environment, CallOrderVerifier callOrderVerifier) {
- super(dockerOperations, processExecuter, environment, null);
+ public StorageMaintainerMock(DockerOperations dockerOperations, ProcessExecuter processExecuter, Environment environment, CallOrderVerifier callOrderVerifier, Clock clock) {
+ super(dockerOperations, processExecuter, new MetricReceiverWrapper(MetricReceiver.nullImplementation), environment, clock);
this.callOrderVerifier = callOrderVerifier;
}
@@ -35,6 +38,10 @@ public class StorageMaintainerMock extends StorageMaintainer {
}
@Override
+ public void cleanNodeAdmin() {
+ }
+
+ @Override
public void cleanupNodeStorage(ContainerName containerName, NodeSpec node) {
callOrderVerifier.add("DeleteContainerStorage with " + containerName);
}
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelperTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelperTest.java
new file mode 100644
index 00000000000..6b53bc217c4
--- /dev/null
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/FileHelperTest.java
@@ -0,0 +1,324 @@
+// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.hosted.node.admin.maintenance;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Optional;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author freva
+ */
+public class FileHelperTest {
+ @Rule
+ public TemporaryFolder folder = new TemporaryFolder();
+
+ @Before
+ public void initFiles() throws IOException {
+ for (int i=0; i<10; i++) {
+ File temp = folder.newFile("test_" + i + ".json");
+ temp.setLastModified(System.currentTimeMillis() - i*Duration.ofSeconds(130).toMillis());
+ }
+
+ for (int i=0; i<7; i++) {
+ File temp = folder.newFile("test_" + i + "_file.test");
+ temp.setLastModified(System.currentTimeMillis() - i*Duration.ofSeconds(250).toMillis());
+ }
+
+ for (int i=0; i<5; i++) {
+ File temp = folder.newFile(i + "-abc" + ".json");
+ temp.setLastModified(System.currentTimeMillis() - i*Duration.ofSeconds(80).toMillis());
+ }
+
+ File temp = folder.newFile("week_old_file.json");
+ temp.setLastModified(System.currentTimeMillis() - Duration.ofDays(8).toMillis());
+ }
+
+ @Test
+ public void testDeleteAll() throws IOException {
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.empty(), false);
+
+ assertEquals(0, getContentsOfDirectory(folder.getRoot()).length);
+ }
+
+ @Test
+ public void testDeletePrefix() throws IOException {
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.of("^test_"), false);
+
+ assertEquals(6, getContentsOfDirectory(folder.getRoot()).length); // 5 abc files + 1 week_old_file
+ }
+
+ @Test
+ public void testDeleteSuffix() throws IOException {
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.of(".json$"), false);
+
+ assertEquals(7, getContentsOfDirectory(folder.getRoot()).length);
+ }
+
+ @Test
+ public void testDeletePrefixAndSuffix() throws IOException {
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.of("^test_.*\\.json$"), false);
+
+ assertEquals(13, getContentsOfDirectory(folder.getRoot()).length); // 5 abc files + 7 test_*_file.test files + week_old_file
+ }
+
+ @Test
+ public void testDeleteOld() throws IOException {
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ofSeconds(600), Optional.empty(), false);
+
+ assertEquals(13, getContentsOfDirectory(folder.getRoot()).length); // All 23 - 6 (from test_*_.json) - 3 (from test_*_file.test) - 1 week old file
+ }
+
+ @Test
+ public void testDeleteWithAllParameters() throws IOException {
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ofSeconds(200), Optional.of("^test_.*\\.json$"), false);
+
+ assertEquals(15, getContentsOfDirectory(folder.getRoot()).length); // All 23 - 8 (from test_*_.json)
+ }
+
+ @Test
+ public void testDeleteWithSubDirectoriesNoRecursive() throws IOException {
+ initSubDirectories();
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.of("^test_.*\\.json$"), false);
+
+ // 6 test_*.json from test_folder1/
+ // + 9 test_*.json and 4 abc_*.json from test_folder2/
+ // + 13 test_*.json from test_folder2/subSubFolder2/
+ // + 7 test_*_file.test and 5 *-abc.json and 1 week_old_file from root
+ // + test_folder1/ and test_folder2/ and test_folder2/subSubFolder2/ themselves
+ assertEquals(48, getNumberOfFilesAndDirectoriesIn(folder.getRoot()));
+ }
+
+ @Test
+ public void testDeleteWithSubDirectoriesRecursive() throws IOException {
+ initSubDirectories();
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.of("^test_.*\\.json$"), true);
+
+ // 4 abc_*.json from test_folder2/
+ // + 7 test_*_file.test and 5 *-abc.json and 1 week_old_file from root
+ // + test_folder2/ itself
+ assertEquals(18, getNumberOfFilesAndDirectoriesIn(folder.getRoot()));
+ }
+
+ @Test
+ public void testDeleteFilesWhereFilenameRegexAlsoMatchesDirectories() throws IOException {
+ initSubDirectories();
+
+ FileHelper.deleteFiles(folder.getRoot().toPath(), Duration.ZERO, Optional.of("^test_"), false);
+
+ assertEquals(8, getContentsOfDirectory(folder.getRoot()).length); // 5 abc files + 1 week_old_file + 2 directories
+ }
+
+ @Test
+ public void testGetContentsOfNonExistingDirectory() {
+ Path fakePath = Paths.get("/some/made/up/dir/");
+ assertEquals(Collections.emptyList(), FileHelper.listContentsOfDirectory(fakePath));
+ }
+
+ @Test(expected=IllegalArgumentException.class)
+ public void testDeleteFilesExceptNMostRecentWithNegativeN() throws IOException {
+ FileHelper.deleteFilesExceptNMostRecent(folder.getRoot().toPath(), -5);
+ }
+
+ @Test
+ public void testDeleteFilesExceptFiveMostRecent() throws IOException {
+ FileHelper.deleteFilesExceptNMostRecent(folder.getRoot().toPath(), 5);
+
+ assertEquals(5, getContentsOfDirectory(folder.getRoot()).length);
+
+ String[] oldestFiles = {"test_5_file.test", "test_6_file.test", "test_8.json", "test_9.json", "week_old_file.json"};
+ String[] remainingFiles = Arrays.stream(getContentsOfDirectory(folder.getRoot()))
+ .map(File::getName)
+ .sorted()
+ .toArray(String[]::new);
+
+ assertArrayEquals(oldestFiles, remainingFiles);
+ }
+
+ @Test
+ public void testDeleteFilesExceptNMostRecentWithLargeN() throws IOException {
+ String[] filesPreDelete = folder.getRoot().list();
+
+ FileHelper.deleteFilesExceptNMostRecent(folder.getRoot().toPath(), 50);
+
+ assertArrayEquals(filesPreDelete, folder.getRoot().list());
+ }
+
+ @Test
+ public void testDeleteFilesLargerThan10B() throws IOException {
+ initSubDirectories();
+
+ File temp1 = new File(folder.getRoot(), "small_file");
+ writeNBytesToFile(temp1, 50);
+
+ File temp2 = new File(folder.getRoot(), "some_file");
+ writeNBytesToFile(temp2, 20);
+
+ File temp3 = new File(folder.getRoot(), "test_folder1/some_other_file");
+ writeNBytesToFile(temp3, 75);
+
+ FileHelper.deleteFilesLargerThan(folder.getRoot().toPath(), 10);
+
+ assertEquals(58, getNumberOfFilesAndDirectoriesIn(folder.getRoot()));
+ assertFalse(temp1.exists() || temp2.exists() || temp3.exists());
+ }
+
+ @Test
+ public void testDeleteDirectories() throws IOException {
+ initSubDirectories();
+
+ FileHelper.deleteDirectories(folder.getRoot().toPath(), Duration.ZERO, Optional.of(".*folder2"));
+
+ //23 files in root
+ // + 6 in test_folder1 + test_folder1 itself
+ assertEquals(30, getNumberOfFilesAndDirectoriesIn(folder.getRoot()));
+ }
+
+ @Test
+ public void testDeleteDirectoriesBasedOnAge() throws IOException {
+ initSubDirectories();
+ // Create folder3 which is older than maxAge, inside have a single directory, subSubFolder3, inside it which is
+ // also older than maxAge inside the sub directory, create some files which are newer than maxAge.
+ // deleteDirectories() should NOT delete folder3
+ File subFolder3 = folder.newFolder("test_folder3");
+ File subSubFolder3 = folder.newFolder("test_folder3", "subSubFolder3");
+
+ for (int j=0; j<11; j++) {
+ File.createTempFile("test_", ".json", subSubFolder3);
+ }
+
+ subFolder3.setLastModified(System.currentTimeMillis() - Duration.ofHours(1).toMillis());
+ subSubFolder3.setLastModified(System.currentTimeMillis() - Duration.ofHours(3).toMillis());
+
+ FileHelper.deleteDirectories(folder.getRoot().toPath(), Duration.ofSeconds(50), Optional.of(".*folder.*"));
+
+ //23 files in root
+ // + 13 in test_folder2
+ // + 13 in subSubFolder2
+ // + 11 in subSubFolder3
+ // + test_folder2 + subSubFolder2 + folder3 + subSubFolder3 itself
+ assertEquals(64, getNumberOfFilesAndDirectoriesIn(folder.getRoot()));
+ }
+
+ @Test
+ public void testRecursivelyDeleteDirectory() throws IOException {
+ initSubDirectories();
+ FileHelper.recursiveDelete(folder.getRoot().toPath());
+ assertFalse(folder.getRoot().exists());
+ }
+
+ @Test
+ public void testRecursivelyDeleteRegularFile() throws IOException {
+ File file = folder.newFile();
+ assertTrue(file.exists());
+ assertTrue(file.isFile());
+ FileHelper.recursiveDelete(file.toPath());
+ assertFalse(file.exists());
+ }
+
+ @Test
+ public void testRecursivelyDeleteNonExistingFile() throws IOException {
+ File file = folder.getRoot().toPath().resolve("non-existing-file.json").toFile();
+ assertFalse(file.exists());
+ FileHelper.recursiveDelete(file.toPath());
+ assertFalse(file.exists());
+ }
+
+ @Test
+ public void testInitSubDirectories() throws IOException {
+ initSubDirectories();
+ assertTrue(folder.getRoot().exists());
+ assertTrue(folder.getRoot().isDirectory());
+
+ Path test_folder1 = folder.getRoot().toPath().resolve("test_folder1");
+ assertTrue(test_folder1.toFile().exists());
+ assertTrue(test_folder1.toFile().isDirectory());
+
+ Path test_folder2 = folder.getRoot().toPath().resolve("test_folder2");
+ assertTrue(test_folder2.toFile().exists());
+ assertTrue(test_folder2.toFile().isDirectory());
+
+ Path subSubFolder2 = test_folder2.resolve("subSubFolder2");
+ assertTrue(subSubFolder2.toFile().exists());
+ assertTrue(subSubFolder2.toFile().isDirectory());
+ }
+
+ @Test
+ public void testDoesNotFailOnLastModifiedOnSymLink() throws IOException {
+ Path symPath = folder.getRoot().toPath().resolve("symlink");
+ Path fakePath = Paths.get("/some/not/existant/file");
+
+ Files.createSymbolicLink(symPath, fakePath);
+ assertTrue(Files.isSymbolicLink(symPath));
+ assertFalse(Files.exists(fakePath));
+
+ // Not possible to set modified time on symlink in java, so just check that it doesn't crash
+ FileHelper.getLastModifiedTime(symPath).toInstant();
+ }
+
+ private void initSubDirectories() throws IOException {
+ File subFolder1 = folder.newFolder("test_folder1");
+ File subFolder2 = folder.newFolder("test_folder2");
+ File subSubFolder2 = folder.newFolder("test_folder2", "subSubFolder2");
+
+ for (int j=0; j<6; j++) {
+ File temp = File.createTempFile("test_", ".json", subFolder1);
+ temp.setLastModified(System.currentTimeMillis() - (j+1)*Duration.ofSeconds(60).toMillis());
+ }
+
+ for (int j=0; j<9; j++) {
+ File.createTempFile("test_", ".json", subFolder2);
+ }
+
+ for (int j=0; j<4; j++) {
+ File.createTempFile("abc_", ".txt", subFolder2);
+ }
+
+ for (int j=0; j<13; j++) {
+ File temp = File.createTempFile("test_", ".json", subSubFolder2);
+ temp.setLastModified(System.currentTimeMillis() - (j+1)*Duration.ofSeconds(40).toMillis());
+ }
+
+ //Must be after all the files have been created
+ subFolder1.setLastModified(System.currentTimeMillis() - Duration.ofHours(2).toMillis());
+ subFolder2.setLastModified(System.currentTimeMillis() - Duration.ofHours(1).toMillis());
+ subSubFolder2.setLastModified(System.currentTimeMillis() - Duration.ofHours(3).toMillis());
+ }
+
+ private static int getNumberOfFilesAndDirectoriesIn(File folder) {
+ int total = 0;
+ for (File file : getContentsOfDirectory(folder)) {
+ if (file.isDirectory()) {
+ total += getNumberOfFilesAndDirectoriesIn(file);
+ }
+ total++;
+ }
+
+ return total;
+ }
+
+ private static void writeNBytesToFile(File file, int nBytes) throws IOException {
+ Files.write(file.toPath(), new byte[nBytes]);
+ }
+
+ private static File[] getContentsOfDirectory(File directory) {
+ File[] directoryContents = directory.listFiles();
+
+ return directoryContents == null ? new File[0] : directoryContents;
+ }
+}
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainerTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainerTest.java
index 7722354a633..d9cce7f80a0 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainerTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/maintenance/StorageMaintainerTest.java
@@ -1,12 +1,20 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.node.admin.maintenance;
+import com.yahoo.collections.Pair;
+import com.yahoo.config.provision.NodeType;
+import com.yahoo.metrics.simple.MetricReceiver;
import com.yahoo.system.ProcessExecuter;
+import com.yahoo.test.ManualClock;
+import com.yahoo.vespa.hosted.dockerapi.ContainerName;
+import com.yahoo.vespa.hosted.dockerapi.metrics.MetricReceiverWrapper;
+import com.yahoo.vespa.hosted.node.admin.configserver.noderepository.NodeSpec;
import com.yahoo.vespa.hosted.node.admin.config.ConfigServerConfig;
import com.yahoo.vespa.hosted.node.admin.docker.DockerNetworking;
import com.yahoo.vespa.hosted.node.admin.docker.DockerOperations;
import com.yahoo.vespa.hosted.node.admin.component.Environment;
import com.yahoo.vespa.hosted.node.admin.component.PathResolver;
+import com.yahoo.vespa.hosted.provision.Node;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -14,15 +22,21 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
+import java.time.Duration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
/**
* @author dybis
*/
public class StorageMaintainerTest {
+ private final ManualClock clock = new ManualClock();
private final Environment environment = new Environment.Builder()
.configServerConfig(new ConfigServerConfig(new ConfigServerConfig.Builder()))
.region("us-east-1")
@@ -36,7 +50,7 @@ public class StorageMaintainerTest {
private final DockerOperations docker = mock(DockerOperations.class);
private final ProcessExecuter processExecuter = mock(ProcessExecuter.class);
private final StorageMaintainer storageMaintainer = new StorageMaintainer(docker, processExecuter,
- environment, null);
+ new MetricReceiverWrapper(MetricReceiver.nullImplementation), environment, clock);
@Rule
public TemporaryFolder folder = new TemporaryFolder();
@@ -57,7 +71,76 @@ public class StorageMaintainerTest {
assertEquals(0L, usedBytes);
}
+ @Test
+ public void testMaintenanceThrottlingAfterSuccessfulMaintenance() {
+ String hostname = "node-123.us-north-3.test.yahoo.com";
+ ContainerName containerName = ContainerName.fromHostname(hostname);
+ NodeSpec node = new NodeSpec.Builder()
+ .hostname(hostname)
+ .state(Node.State.ready)
+ .nodeType(NodeType.tenant)
+ .flavor("docker")
+ .minCpuCores(1)
+ .minMainMemoryAvailableGb(1)
+ .minDiskAvailableGb(1)
+ .build();
+
+ try {
+ when(processExecuter.exec(any(String[].class))).thenReturn(new Pair<>(0, ""));
+ } catch (IOException ignored) { }
+ storageMaintainer.removeOldFilesFromNode(containerName);
+ verifyProcessExecuterCalled(1);
+ // Will not actually run maintenance job until an hour passes
+ storageMaintainer.removeOldFilesFromNode(containerName);
+ verifyProcessExecuterCalled(1);
+
+ clock.advance(Duration.ofMinutes(61));
+ storageMaintainer.removeOldFilesFromNode(containerName);
+ verifyProcessExecuterCalled(2);
+
+ // Coredump handling is unthrottled
+ storageMaintainer.handleCoreDumpsForContainer(containerName, node);
+ verifyProcessExecuterCalled(3);
+
+ storageMaintainer.handleCoreDumpsForContainer(containerName, node);
+ verifyProcessExecuterCalled(4);
+
+ // cleanupNodeStorage is unthrottled and it should reset previous times
+ storageMaintainer.cleanupNodeStorage(containerName, node);
+ verifyProcessExecuterCalled(5);
+ storageMaintainer.cleanupNodeStorage(containerName, node);
+ verifyProcessExecuterCalled(6);
+ }
+
+ @Test
+ public void testMaintenanceThrottlingAfterFailedMaintenance() {
+ String hostname = "node-123.us-north-3.test.yahoo.com";
+ ContainerName containerName = ContainerName.fromHostname(hostname);
+
+ try {
+ when(processExecuter.exec(any(String[].class)))
+ .thenThrow(new RuntimeException("Something went wrong"))
+ .thenReturn(new Pair<>(0, ""));
+ } catch (IOException ignored) { }
+
+ try {
+ storageMaintainer.removeOldFilesFromNode(containerName);
+ fail("Maintenance job should've failed!");
+ } catch (RuntimeException ignored) { }
+ verifyProcessExecuterCalled(1);
+
+ // Maintenance job failed, we should be able to immediately re-run it
+ storageMaintainer.removeOldFilesFromNode(containerName);
+ verifyProcessExecuterCalled(2);
+ }
+
private static void writeNBytesToFile(File file, int nBytes) throws IOException {
Files.write(file.toPath(), new byte[nBytes]);
}
+
+ private void verifyProcessExecuterCalled(int times) {
+ try {
+ verify(processExecuter, times(times)).exec(any(String[].class));
+ } catch (IOException ignored) { }
+ }
}
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java
index 065b039c7fd..efd35cce00e 100644
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java
+++ b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/nodeadmin/NodeAdminImplTest.java
@@ -5,6 +5,7 @@ import com.yahoo.metrics.simple.MetricReceiver;
import com.yahoo.test.ManualClock;
import com.yahoo.vespa.hosted.dockerapi.metrics.MetricReceiverWrapper;
import com.yahoo.vespa.hosted.node.admin.docker.DockerOperations;
+import com.yahoo.vespa.hosted.node.admin.maintenance.StorageMaintainer;
import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgent;
import com.yahoo.vespa.hosted.node.admin.nodeagent.NodeAgentImpl;
import org.junit.Test;
@@ -39,10 +40,11 @@ public class NodeAdminImplTest {
private interface NodeAgentFactory extends Function<String, NodeAgent> {}
private final DockerOperations dockerOperations = mock(DockerOperations.class);
private final Function<String, NodeAgent> nodeAgentFactory = mock(NodeAgentFactory.class);
+ private final StorageMaintainer storageMaintainer = mock(StorageMaintainer.class);
private final Runnable aclMaintainer = mock(Runnable.class);
private final ManualClock clock = new ManualClock();
- private final NodeAdminImpl nodeAdmin = new NodeAdminImpl(dockerOperations, nodeAgentFactory, aclMaintainer,
+ private final NodeAdminImpl nodeAdmin = new NodeAdminImpl(dockerOperations, nodeAgentFactory, storageMaintainer, aclMaintainer,
new MetricReceiverWrapper(MetricReceiver.nullImplementation), clock);
@Test
diff --git a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelperTest.java b/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelperTest.java
deleted file mode 100644
index a3569853122..00000000000
--- a/node-admin/src/test/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileHelperTest.java
+++ /dev/null
@@ -1,201 +0,0 @@
-// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.vespa.hosted.node.admin.task.util.file;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.runners.Enclosed;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.nio.file.attribute.FileTime;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * @author freva
- */
-@RunWith(Enclosed.class)
-public class FileHelperTest {
-
- public static class GeneralLogicTests {
- @Rule
- public TemporaryFolder folder = new TemporaryFolder();
-
- @Test
- public void delete_all_files_non_recursive() {
- int numDeleted = FileHelper.streamFiles(testRoot())
- .delete();
-
- assertEquals(3, numDeleted);
- assertRecursiveContents("test", "test/file.txt", "test/data.json", "test/subdir-1", "test/subdir-1/file", "test/subdir-2");
- }
-
- @Test
- public void delete_all_files_recursive() {
- int numDeleted = FileHelper.streamFiles(testRoot())
- .recursive(true)
- .delete();
-
- assertEquals(6, numDeleted);
- assertRecursiveContents("test", "test/subdir-1", "test/subdir-2");
- }
-
- @Test
- public void delete_with_filter_recursive() {
- int numDeleted = FileHelper.streamFiles(testRoot())
- .filterFile(FileHelper.nameEndsWith(".json"))
- .recursive(true)
- .delete();
-
- assertEquals(3, numDeleted);
- assertRecursiveContents("test.txt", "test", "test/file.txt", "test/subdir-1", "test/subdir-1/file", "test/subdir-2");
- }
-
- @Test
- public void delete_directory_with_filter() {
- int numDeleted = FileHelper.streamDirectories(testRoot())
- .filterDirectory(FileHelper.nameStartsWith("subdir"))
- .recursive(true)
- .delete();
-
- assertEquals(3, numDeleted);
- assertRecursiveContents("file-1.json", "test.json", "test.txt", "test", "test/file.txt", "test/data.json");
- }
-
- @Test
- public void delete_all_contents() {
- int numDeleted = FileHelper.streamContents(testRoot())
- .recursive(true)
- .delete();
-
- assertEquals(9, numDeleted);
- assertTrue(Files.exists(testRoot()));
- assertRecursiveContents();
- }
-
- @Test
- public void delete_everything() {
- int numDeleted = FileHelper.streamContents(testRoot())
- .includeBase(true)
- .recursive(true)
- .delete();
-
- assertEquals(10, numDeleted);
- assertFalse(Files.exists(testRoot()));
- }
-
- @Before
- public void setup() throws IOException {
- Path root = testRoot();
-
- Files.createFile(root.resolve("file-1.json"));
- Files.createFile(root.resolve("test.json"));
- Files.createFile(root.resolve("test.txt"));
-
- Files.createDirectories(root.resolve("test"));
- Files.createFile(root.resolve("test/file.txt"));
- Files.createFile(root.resolve("test/data.json"));
-
- Files.createDirectories(root.resolve("test/subdir-1"));
- Files.createFile(root.resolve("test/subdir-1/file"));
-
- Files.createDirectories(root.resolve("test/subdir-2"));
- }
-
- private Path testRoot() {
- return folder.getRoot().toPath();
- }
-
- private void assertRecursiveContents(String... relativePaths) {
- Set<String> expectedPaths = new HashSet<>(Arrays.asList(relativePaths));
- Set<String> actualPaths = recursivelyListContents(testRoot()).stream()
- .map(testRoot()::relativize)
- .map(Path::toString)
- .collect(Collectors.toSet());
-
- assertEquals(expectedPaths, actualPaths);
- }
-
- private List<Path> recursivelyListContents(Path basePath) {
- try (Stream<Path> pathStream = Files.list(basePath)) {
- List<Path> paths = new LinkedList<>();
- pathStream.forEach(path -> {
- paths.add(path);
- if (Files.isDirectory(path))
- paths.addAll(recursivelyListContents(path));
- });
- return paths;
- } catch (IOException e) {
- throw new UncheckedIOException(e);
- }
- }
- }
-
- public static class FilterUnitTests {
-
- private final BasicFileAttributes attributes = mock(BasicFileAttributes.class);
-
- @Test
- public void age_filter_test() {
- Path path = Paths.get("/my/fake/path");
- when(attributes.lastModifiedTime()).thenReturn(FileTime.from(Instant.now().minus(Duration.ofHours(1))));
- FileHelper.FileAttributes fileAttributes = new FileHelper.FileAttributes(path, attributes);
-
- assertFalse(FileHelper.olderThan(Duration.ofMinutes(61)).test(fileAttributes));
- assertTrue(FileHelper.olderThan(Duration.ofMinutes(59)).test(fileAttributes));
-
- assertTrue(FileHelper.youngerThan(Duration.ofMinutes(61)).test(fileAttributes));
- assertFalse(FileHelper.youngerThan(Duration.ofMinutes(59)).test(fileAttributes));
- }
-
- @Test
- public void size_filters() {
- Path path = Paths.get("/my/fake/path");
- when(attributes.size()).thenReturn(100L);
- FileHelper.FileAttributes fileAttributes = new FileHelper.FileAttributes(path, attributes);
-
- assertFalse(FileHelper.largerThan(101).test(fileAttributes));
- assertTrue(FileHelper.largerThan(99).test(fileAttributes));
-
- assertTrue(FileHelper.smallerThan(101).test(fileAttributes));
- assertFalse(FileHelper.smallerThan(99).test(fileAttributes));
- }
-
- @Test
- public void filename_filters() {
- Path path = Paths.get("/my/fake/path/some-12352-file.json");
- FileHelper.FileAttributes fileAttributes = new FileHelper.FileAttributes(path, attributes);
-
- assertTrue(FileHelper.nameStartsWith("some-").test(fileAttributes));
- assertFalse(FileHelper.nameStartsWith("som-").test(fileAttributes));
-
- assertTrue(FileHelper.nameEndsWith(".json").test(fileAttributes));
- assertFalse(FileHelper.nameEndsWith("file").test(fileAttributes));
-
- assertTrue(FileHelper.nameMatches(Pattern.compile("some-[0-9]+-file.json")).test(fileAttributes));
- assertTrue(FileHelper.nameMatches(Pattern.compile("^some-[0-9]+-file.json$")).test(fileAttributes));
- assertFalse(FileHelper.nameMatches(Pattern.compile("some-[0-9]-file.json")).test(fileAttributes));
- }
- }
-}