summaryrefslogtreecommitdiffstats
path: root/node-admin/src/main
diff options
context:
space:
mode:
Diffstat (limited to 'node-admin/src/main')
-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
5 files changed, 432 insertions, 319 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));
- }
-}