summaryrefslogtreecommitdiffstats
path: root/node-admin/src/main/java
diff options
context:
space:
mode:
Diffstat (limited to 'node-admin/src/main/java')
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/component/TestTaskContext.java29
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileAttributes.java1
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/InputStreamUtil.java40
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2.java16
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2Impl.java138
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessException.java66
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessFailureException.java15
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandLine.java265
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandResult.java60
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/LargeOutputChildProcessException.java15
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2.java17
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2Impl.java36
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApiImpl.java1
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactory.java10
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactoryImpl.java93
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarter.java10
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarterImpl.java16
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/Terminal.java14
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TerminalImpl.java19
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestChildProcess2.java52
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestProcessFactory.java95
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestTerminal.java67
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TimeoutChildProcessException.java18
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnexpectedOutputException2.java16
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnkillableChildProcessException.java21
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/time/TestTimer.java29
-rw-r--r--node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/yum/Yum.java15
27 files changed, 1173 insertions, 1 deletions
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/component/TestTaskContext.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/component/TestTaskContext.java
new file mode 100644
index 00000000000..6806e5096c5
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/component/TestTaskContext.java
@@ -0,0 +1,29 @@
+// 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.component;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+public class TestTaskContext implements TaskContext {
+ private final List<String> logs = new ArrayList<>();
+
+ @Override
+ public void recordSystemModification(Logger logger, String description) {
+ logs.add(description);
+ }
+
+ public List<String> getSystemModificationLog() {
+ return logs;
+ }
+
+ public void clearSystemModificationLog() {
+ logs.clear();
+ }
+
+ @Override
+ public boolean executeSubtask(IdempotentTask task) {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileAttributes.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileAttributes.java
index 3910398a040..611e2c32bcd 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileAttributes.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/FileAttributes.java
@@ -24,4 +24,5 @@ public class FileAttributes {
public String permissions() { return PosixFilePermissions.toString(attributes.permissions()); }
public boolean isRegularFile() { return attributes.isRegularFile(); }
public boolean isDirectory() { return attributes.isDirectory(); }
+ public long size() { return attributes.size(); }
}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/InputStreamUtil.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/InputStreamUtil.java
new file mode 100644
index 00000000000..780102e9c9e
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/file/InputStreamUtil.java
@@ -0,0 +1,40 @@
+// 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.ByteArrayOutputStream;
+import java.io.InputStream;
+
+import static com.yahoo.vespa.hosted.node.admin.task.util.file.IOExceptionUtil.uncheck;
+
+/**
+ * @author hakonhall
+ */
+public class InputStreamUtil {
+ private final InputStream inputStream;
+
+ public InputStreamUtil(InputStream inputStream) {
+ this.inputStream = inputStream;
+ }
+
+ public InputStream getInputStream() {
+ return inputStream;
+ }
+
+ /**
+ * TODO: Replace usages with Java 9's InputStream::readAllBytes
+ */
+ byte[] readAllBytes() {
+ // According to https://stackoverflow.com/questions/309424/read-convert-an-inputstream-to-a-string
+ // all other implementations are much inferior to this in performance.
+
+ ByteArrayOutputStream result = new ByteArrayOutputStream();
+ byte[] buffer = new byte[1024];
+ int length;
+ while ((length = uncheck(() -> inputStream.read(buffer))) != -1) {
+ result.write(buffer, 0, length);
+ }
+
+ return result.toByteArray();
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2.java
new file mode 100644
index 00000000000..172203a281a
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2.java
@@ -0,0 +1,16 @@
+// 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.process;
+
+/**
+ * @author hakonhall
+ */
+interface ChildProcess2 extends AutoCloseable {
+ void waitForTermination();
+ int exitCode();
+ String getOutput();
+
+ /** Close/cleanup any resources held. Must not throw an exception. */
+ @Override
+ void close();
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2Impl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2Impl.java
new file mode 100644
index 00000000000..67020270a99
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcess2Impl.java
@@ -0,0 +1,138 @@
+// 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.process;
+
+import com.yahoo.jdisc.Timer;
+import com.yahoo.log.LogLevel;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.concurrent.TimeUnit;
+import java.util.logging.Logger;
+
+import static com.yahoo.vespa.hosted.node.admin.task.util.file.IOExceptionUtil.uncheck;
+
+/**
+ * @author hakonhall
+ */
+public class ChildProcess2Impl implements ChildProcess2 {
+ private static final Logger logger = Logger.getLogger(ChildProcess2Impl.class.getName());
+
+ private final CommandLine commandLine;
+ private final ProcessApi2 process;
+ private final Path outputPath;
+ private final Timer timer;
+
+ public ChildProcess2Impl(CommandLine commandLine,
+ ProcessApi2 process,
+ Path outputPath,
+ Timer timer) {
+ this.commandLine = commandLine;
+ this.process = process;
+ this.outputPath = outputPath;
+ this.timer = timer;
+ }
+
+ @Override
+ public void waitForTermination() {
+ Duration timeoutDuration = commandLine.getTimeout();
+ Instant timeout = timer.currentTime().plus(timeoutDuration);
+ long maxOutputBytes = commandLine.getMaxOutputBytes();
+
+ // How frequently do we want to wake up and check the output file size?
+ final Duration pollInterval = Duration.ofSeconds(10);
+
+ boolean hasTerminated = false;
+ while (!hasTerminated) {
+ Instant now = timer.currentTime();
+ long sleepPeriodMillis = pollInterval.toMillis();
+ if (now.plusMillis(sleepPeriodMillis).isAfter(timeout)) {
+ sleepPeriodMillis = Duration.between(now, timeout).toMillis();
+
+ if (sleepPeriodMillis <= 0) {
+ gracefullyKill();
+ throw new TimeoutChildProcessException(
+ timeoutDuration, commandLine.toString(), getOutput());
+ }
+ }
+
+ try {
+ hasTerminated = process.waitFor(sleepPeriodMillis, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ // Ignore, just loop around.
+ continue;
+ }
+
+ // Always check output file size to ensure we don't load too much into memory.
+ long sizeInBytes = uncheck(() -> Files.size(outputPath));
+ if (sizeInBytes > maxOutputBytes) {
+ gracefullyKill();
+ throw new LargeOutputChildProcessException(
+ sizeInBytes, commandLine.toString(), getOutput());
+ }
+ }
+ }
+
+ @Override
+ public int exitCode() {
+ return process.exitValue();
+ }
+
+ @Override
+ public String getOutput() {
+ byte[] bytes = uncheck(() -> Files.readAllBytes(outputPath));
+ return new String(bytes, commandLine.getOutputEncoding());
+ }
+
+ @Override
+ public void close() {
+ try {
+ Files.delete(outputPath);
+ } catch (Throwable t) {
+ logger.log(LogLevel.WARNING, "Failed to delete " + outputPath, t);
+ }
+ }
+
+ Path getOutputPath() {
+ return outputPath;
+ }
+
+ private void gracefullyKill() {
+ process.destroy();
+
+ Duration maxWaitAfterSigTerm = commandLine.getSigTermGracePeriod();
+ Instant timeout = timer.currentTime().plus(maxWaitAfterSigTerm);
+ if (!waitForTermination(timeout)) {
+ process.destroyForcibly();
+
+ // If waiting for the process now takes a long time, it's probably a kernel issue
+ // or huge core is getting dumped.
+ Duration maxWaitAfterSigKill = commandLine.getSigKillGracePeriod();
+ if (!waitForTermination(timer.currentTime().plus(maxWaitAfterSigKill))) {
+ throw new UnkillableChildProcessException(
+ maxWaitAfterSigTerm,
+ maxWaitAfterSigKill,
+ commandLine.toString(),
+ getOutput());
+ }
+ }
+ }
+
+ /** @return true if process terminated, false on timeout. */
+ private boolean waitForTermination(Instant timeout) {
+ while (true) {
+ long waitDurationMillis = Duration.between(timer.currentTime(), timeout).toMillis();
+ if (waitDurationMillis <= 0) {
+ return false;
+ }
+
+ try {
+ return process.waitFor(waitDurationMillis, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ // ignore
+ }
+ }
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessException.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessException.java
new file mode 100644
index 00000000000..b84bd2d8fef
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessException.java
@@ -0,0 +1,66 @@
+// 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.process;
+
+/**
+ * Base class for child process related exceptions, with a util to build an error message
+ * that includes a large part of the output.
+ *
+ * @author hakonhall
+ */
+@SuppressWarnings("serial")
+public abstract class ChildProcessException extends RuntimeException {
+ private static final int MAX_OUTPUT_PREFIX = 200;
+ private static final int MAX_OUTPUT_SUFFIX = 200;
+ // Omitting a number of chars less than 10 or less than 10% would be ridiculous.
+ private static final int MAX_OUTPUT_SLACK = Math.max(10, (10 * (MAX_OUTPUT_PREFIX + MAX_OUTPUT_SUFFIX)) / 100);
+
+ /**
+ * An exception with a message of the following format:
+ * Command 'COMMANDLINE' PROBLEM: stdout/stderr: 'OUTPUT'
+ *
+ * If the output of the terminated command is too large it will be sampled.
+ *
+ * @param problem E.g. "terminated with exit code 1"
+ * @param commandLine The command that failed in a concise (e.g. shell-like) format
+ * @param possiblyHugeOutput The output of the command
+ */
+ protected ChildProcessException(String problem, String commandLine, String possiblyHugeOutput) {
+ super(makeSnippet(
+ problem,
+ commandLine,
+ possiblyHugeOutput,
+ MAX_OUTPUT_PREFIX,
+ MAX_OUTPUT_SUFFIX,
+ MAX_OUTPUT_SLACK));
+ }
+
+ // Package-private instead of private for testing.
+ static String makeSnippet(String problem,
+ String commandLine,
+ String possiblyHugeOutput,
+ int maxOutputPrefix,
+ int maxOutputSuffix,
+ int maxOutputSlack) {
+ StringBuilder stringBuilder = new StringBuilder()
+ .append("Command '")
+ .append(commandLine)
+ .append("' ")
+ .append(problem)
+ .append(": stdout/stderr: '");
+
+ if (possiblyHugeOutput.length() <= maxOutputPrefix + maxOutputSuffix + maxOutputSlack) {
+ stringBuilder.append(possiblyHugeOutput);
+ } else {
+ stringBuilder.append(possiblyHugeOutput.substring(0, maxOutputPrefix))
+ .append("... [")
+ .append(possiblyHugeOutput.length() - maxOutputPrefix - maxOutputSuffix)
+ .append(" chars omitted] ...")
+ .append(possiblyHugeOutput.substring(possiblyHugeOutput.length() - maxOutputSuffix));
+ }
+
+ stringBuilder.append("'");
+
+ return stringBuilder.toString();
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessFailureException.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessFailureException.java
new file mode 100644
index 00000000000..5c6785a646c
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ChildProcessFailureException.java
@@ -0,0 +1,15 @@
+// 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.process;
+
+/**
+ * The child process terminated with a non-zero exit code.
+ *
+ * @author hakonhall
+ */
+@SuppressWarnings("serial")
+public class ChildProcessFailureException extends ChildProcessException {
+ ChildProcessFailureException(int exitCode, String commandLine, String possiblyHugeOutput) {
+ super("terminated with exit code " + exitCode, commandLine, possiblyHugeOutput);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandLine.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandLine.java
new file mode 100644
index 00000000000..6c4de7ac1e3
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandLine.java
@@ -0,0 +1,265 @@
+// 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.process;
+
+import com.yahoo.vespa.hosted.node.admin.component.TaskContext;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.logging.Logger;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * A CommandLine is used to specify and execute a shell-like program in a child process,
+ * and capture its output.
+ *
+ * @author hakonhall
+ */
+public class CommandLine {
+ private static Logger logger = Logger.getLogger(CommandLine.class.getName());
+ private static Pattern UNESCAPED_ARGUMENT_PATTERN = Pattern.compile("^[a-zA-Z0-9=@%/+:.,_-]+$");
+
+ /** The default timeout. See setTimeout() for details. */
+ public static final Duration DEFAULT_TIMEOUT = Duration.ofMinutes(10);
+
+ /** The default maximum number of output bytes. See setMaxOutputBytes() for details. */
+ public static final long DEFAULT_MAX_OUTPUT_BYTES = 1024 * 1024 * 1024; // 1 Gb
+
+ /**
+ * The default grace period after SIGTERM has been sent during a graceful kill.
+ * See setSigTermGracePeriod for details.
+ */
+ public static final Duration DEFAULT_SIGTERM_GRACE_PERIOD = Duration.ofMinutes(1);
+
+ /**
+ * The default grace period after SIGKILL has been sent during a graceful kill.
+ * See setSigKillGracePeriod for details.
+ */
+ public static final Duration DEFAULT_SIGKILL_GRACE_PERIOD = Duration.ofMinutes(30);
+
+ private final List<String> arguments = new ArrayList<>();
+ private final TaskContext taskContext;
+ private final ProcessFactory processFactory;
+
+ private boolean redirectStderrToStdoutInsteadOfDiscard = true;
+ private boolean executeSilentlyCalled = false;
+ private Charset outputEncoding = StandardCharsets.UTF_8;
+ private Duration timeout = DEFAULT_TIMEOUT;
+ private long maxOutputBytes = DEFAULT_MAX_OUTPUT_BYTES;
+ private Duration sigTermGracePeriod = DEFAULT_SIGTERM_GRACE_PERIOD;
+ private Duration sigKillGracePeriod = DEFAULT_SIGKILL_GRACE_PERIOD;
+ private Predicate<Integer> successfulExitCodePredicate = code -> code == 0;
+
+ public CommandLine(TaskContext taskContext, ProcessFactory processFactory) {
+ this.taskContext = taskContext;
+ this.processFactory = processFactory;
+ }
+
+ /** Add arguments to the command. The first argument in the first call to add() is the program. */
+ public CommandLine add(String... arguments) { return add(Arrays.asList(arguments)); }
+
+ /** Add arguments to the command. The first argument in the first call to add() is the program. */
+ public CommandLine add(List<String> arguments) {
+ this.arguments.addAll(arguments);
+ return this;
+ }
+
+ /**
+ * Execute a shell-like program in a child process:
+ * - the program is recorded and logged as modifying the system, but see executeSilently().
+ * - the program's stderr is redirected to stdout, but see discardStderr().
+ * - the program's output is assumed to be UTF-8, but see setOutputEncoding().
+ * - the program must terminate with exit code 0, but see ignoreExitCode().
+ * - the output of the program will be accessible in the returned CommandResult.
+ *
+ * Footnote 1: As a safety measure the size of the output is capped, and the program is
+ * only allowed to execute up to a timeout. The defaults are set high so you typically do
+ * not have to worry about reaching these limits, but otherwise see setMaxOutputBytes()
+ * and setTimeout(), respectively.
+ *
+ * Footnote 2: If the child process is forced to be killed due to footnote 1, then
+ * setSigTermGracePeriod() and setSigKillGracePeriod() can be used to tweak how much time
+ * is given to the program to shut down. Again, the defaults should be reasonable.
+ */
+ public CommandResult execute() {
+ taskContext.recordSystemModification(logger, "Executing command: " + toString());
+ return doExecute();
+ }
+
+ /**
+ * Same as execute(), except it will not record the program as modifying the system.
+ *
+ * If the program is later found to have modified the system, or otherwise worthy of
+ * a record, call recordSilentExecutionAsSystemModification().
+ */
+ public CommandResult executeSilently() {
+ executeSilentlyCalled = true;
+ return doExecute();
+ }
+
+ /**
+ * Record an already executed executeSilently() as having modified the system.
+ * For instance with YUM it is not known until after a 'yum install' whether it
+ * modified the system.
+ */
+ public void recordSilentExecutionAsSystemModification() {
+ if (!executeSilentlyCalled) {
+ throw new IllegalStateException("executeSilently has not been called");
+ }
+ // Disallow multiple consecutive calls to this method without an intervening call
+ // to executeSilently().
+ executeSilentlyCalled = false;
+
+ taskContext.recordSystemModification(logger, "Executed command: " + toString());
+ }
+
+ /**
+ * The first argument of the command specifies the program and is either the program's
+ * filename (in case the environment variable PATH will be used to search for the program
+ * file) or a path with the last component being the program's filename.
+ *
+ * @return The filename of the program.
+ */
+ public String programName() {
+ if (arguments.isEmpty()) {
+ throw new IllegalStateException(
+ "The program name cannot be determined yet as no arguments have been given");
+ }
+ String path = arguments.get(0);
+ int lastIndex = path.lastIndexOf('/');
+ if (lastIndex == -1) {
+ return path;
+ } else {
+ return path.substring(lastIndex + 1);
+ }
+ }
+
+ /** Returns a shell-like representation of the command. */
+ @Override
+ public String toString() {
+ String command = arguments.stream()
+ .map(CommandLine::maybeEscapeArgument)
+ .collect(Collectors.joining(" "));
+
+ // Note: Both of these cannot be confused with an argument since they would
+ // require escaping.
+ command += redirectStderrToStdoutInsteadOfDiscard ? " 2>&1" : " 2>/dev/null";
+
+ return command;
+ }
+
+
+ /**
+ * By default, stderr is redirected to stderr. This method will instead discard stderr.
+ */
+ public CommandLine discardStderr() {
+ this.redirectStderrToStdoutInsteadOfDiscard = false;
+ return this;
+ }
+
+ /**
+ * By default, a non-zero exit code will cause the command execution to fail. This method
+ * will instead ignore the exit code.
+ */
+ public CommandLine ignoreExitCode() {
+ this.successfulExitCodePredicate = code -> true;
+ return this;
+ }
+
+ /**
+ * By default, the output of the command is parsed as UTF-8. This method will set a
+ * different encoding.
+ */
+ public CommandLine setOutputEncoding(Charset outputEncoding) {
+ this.outputEncoding = outputEncoding;
+ return this;
+ }
+
+ /**
+ * By default, the command will be gracefully killed after DEFAULT_TIMEOUT. This method
+ * overrides that default.
+ */
+ public CommandLine setTimeout(Duration timeout) {
+ this.timeout = timeout;
+ return this;
+ }
+
+ /**
+ * By default, the command will be gracefully killed if it ever outputs more than
+ * DEFAULT_MAX_OUTPUT_BYTES. This method overrides that default.
+ */
+ public CommandLine setMaxOutputBytes(long maxOutputBytes) {
+ this.maxOutputBytes = maxOutputBytes;
+ return this;
+ }
+
+ /**
+ * By default, if the program needs to be gracefully killed it will wait up to
+ * DEFAULT_SIGTERM_GRACE_PERIOD for the program to exit after it has been killed with
+ * the SIGTERM signal.
+ */
+ public CommandLine setSigTermGracePeriod(Duration period) {
+ this.sigTermGracePeriod = period;
+ return this;
+ }
+
+ public CommandLine setSigKillGracePeriod(Duration period) {
+ this.sigTermGracePeriod = period;
+ return this;
+ }
+ // Accessor fields necessary for classes in this package. Could be public if necessary.
+ List<String> getArguments() { return Collections.unmodifiableList(arguments); }
+ boolean getRedirectStderrToStdoutInsteadOfDiscard() { return redirectStderrToStdoutInsteadOfDiscard; }
+ Predicate<Integer> getSuccessfulExitCodePredicate() { return successfulExitCodePredicate; }
+ Charset getOutputEncoding() { return outputEncoding; }
+ Duration getTimeout() { return timeout; }
+ long getMaxOutputBytes() { return maxOutputBytes; }
+ Duration getSigTermGracePeriod() { return sigTermGracePeriod; }
+ Duration getSigKillGracePeriod() { return sigKillGracePeriod; }
+
+ private CommandResult doExecute() {
+ try (ChildProcess2 child = processFactory.spawn(this)) {
+ child.waitForTermination();
+ int exitCode = child.exitCode();
+ if (!successfulExitCodePredicate.test(exitCode)) {
+ throw new ChildProcessFailureException(exitCode, toString(), child.getOutput());
+ }
+
+ String output = child.getOutput();
+ return new CommandResult(this, exitCode, output);
+ }
+ }
+
+ private static String maybeEscapeArgument(String argument) {
+ if (UNESCAPED_ARGUMENT_PATTERN.matcher(argument).matches()) {
+ return argument;
+ } else {
+ return escapeArgument(argument);
+ }
+ }
+
+ private static String escapeArgument(String argument) {
+ StringBuilder doubleQuoteEscaped = new StringBuilder(argument.length() + 10);
+
+ for (int i = 0; i < argument.length(); ++i) {
+ char c = argument.charAt(i);
+ switch (c) {
+ case '"':
+ case '\\':
+ doubleQuoteEscaped.append("\\").append(c);
+ break;
+ default:
+ doubleQuoteEscaped.append(c);
+ }
+ }
+
+ return "\"" + doubleQuoteEscaped + "\"";
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandResult.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandResult.java
new file mode 100644
index 00000000000..12f0d546b36
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/CommandResult.java
@@ -0,0 +1,60 @@
+// 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.process;
+
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A CommandResult is the result of the execution of a CommandLine.
+ *
+ * @author hakonhall
+ */
+public class CommandResult {
+ private static final Pattern NEWLINE = Pattern.compile("\\n");
+
+ private final CommandLine commandLine;
+ private final int exitCode;
+ private final String output;
+
+ CommandResult(CommandLine commandLine, int exitCode, String output) {
+ this.commandLine = commandLine;
+ this.exitCode = exitCode;
+ this.output = output;
+ }
+
+ public int getExitCode() {
+ return exitCode;
+ }
+
+ /** Returns the output with leading and trailing white-space removed. */
+ public String getOutput() { return output.trim(); }
+
+ public String getUntrimmedOutput() { return output; }
+
+ /** Returns the output lines of the command, omitting trailing empty lines. */
+ public List<String> getOutputLines() {
+ return getOutputLinesStream().collect(Collectors.toList());
+ }
+
+ public Stream<String> getOutputLinesStream() {
+ if (output.isEmpty()) {
+ // For some reason an empty string => one-element list.
+ return Stream.empty();
+ }
+
+ // For some reason this removes trailing empty elements, but that's OK with us.
+ return NEWLINE.splitAsStream(output);
+ }
+
+ /**
+ * Convenience method for getting the CommandLine, whose execution resulted in
+ * this CommandResult instance.
+ *
+ * Warning: the CommandLine is mutable and may be changed by the caller of the execution
+ * through other references! This is just a convenience method for getting that instance.
+ */
+ public CommandLine getCommandLine() { return commandLine; }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/LargeOutputChildProcessException.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/LargeOutputChildProcessException.java
new file mode 100644
index 00000000000..5c764757e84
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/LargeOutputChildProcessException.java
@@ -0,0 +1,15 @@
+// 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.process;
+
+/**
+ * Exception thrown if the output of the child process is larger than the maximum limit.
+ *
+ * @author hakonhall
+ */
+@SuppressWarnings("serial")
+public class LargeOutputChildProcessException extends ChildProcessException {
+ LargeOutputChildProcessException(long maxFileSize, String commandLine, String possiblyHugeOutput) {
+ super("output more than " + maxFileSize + " bytes", commandLine, possiblyHugeOutput);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2.java
new file mode 100644
index 00000000000..124f319e932
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2.java
@@ -0,0 +1,17 @@
+// Copyright 2018 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+package com.yahoo.vespa.hosted.node.admin.task.util.process;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Process abstraction.
+ *
+ * @author hakonhall
+ */
+public interface ProcessApi2 {
+ boolean waitFor(long timeout, TimeUnit unit) throws InterruptedException;
+ int exitValue();
+ void destroy();
+ void destroyForcibly();
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2Impl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2Impl.java
new file mode 100644
index 00000000000..853558c38e6
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApi2Impl.java
@@ -0,0 +1,36 @@
+// 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.process;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * @author hakonhall
+ */
+public class ProcessApi2Impl implements ProcessApi2 {
+ private final Process process;
+
+ ProcessApi2Impl(Process process) {
+ this.process = process;
+ }
+
+ @Override
+ public boolean waitFor(long timeout, TimeUnit unit) throws InterruptedException {
+ return process.waitFor(timeout, unit);
+ }
+
+ @Override
+ public int exitValue() {
+ return process.exitValue();
+ }
+
+ @Override
+ public void destroy() {
+ process.destroy();
+ }
+
+ @Override
+ public void destroyForcibly() {
+ process.destroyForcibly();
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApiImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApiImpl.java
index e664a68aeff..3620ec9089e 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApiImpl.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessApiImpl.java
@@ -41,7 +41,6 @@ public class ProcessApiImpl implements ProcessApi {
@Override
public void close() {
- // TODO: Should kill process if still alive?
processOutputPath.toFile().delete();
}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactory.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactory.java
new file mode 100644
index 00000000000..3351563faf5
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactory.java
@@ -0,0 +1,10 @@
+// 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.process;
+
+/**
+ * @author hakonhall
+ */
+public interface ProcessFactory {
+ ChildProcess2 spawn(CommandLine commandLine);
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactoryImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactoryImpl.java
new file mode 100644
index 00000000000..705aea1aafc
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessFactoryImpl.java
@@ -0,0 +1,93 @@
+// 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.process;
+
+import com.yahoo.jdisc.Timer;
+import com.yahoo.log.LogLevel;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.FileAttribute;
+import java.nio.file.attribute.PosixFilePermission;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Logger;
+
+import static com.yahoo.vespa.hosted.node.admin.task.util.file.IOExceptionUtil.uncheck;
+
+/**
+ * @author hakonhall
+ */
+public class ProcessFactoryImpl implements ProcessFactory {
+ private static final Logger logger = Logger.getLogger(ProcessFactoryImpl.class.getName());
+ private static final File DEV_NULL = new File("/dev/null");
+
+ private final ProcessStarter processStarter;
+ private final Timer timer;
+
+ ProcessFactoryImpl(ProcessStarter processStarter, Timer timer) {
+ this.processStarter = processStarter;
+ this.timer = timer;
+ }
+
+ @Override
+ public ChildProcess2Impl spawn(CommandLine commandLine) {
+ List<String> arguments = commandLine.getArguments();
+ if (arguments.isEmpty()) {
+ throw new IllegalArgumentException("No arguments specified - missing program to spawn");
+ }
+
+ ProcessBuilder processBuilder = new ProcessBuilder(arguments);
+
+ if (commandLine.getRedirectStderrToStdoutInsteadOfDiscard()) {
+ processBuilder.redirectErrorStream(true);
+ } else {
+ processBuilder.redirectError(ProcessBuilder.Redirect.to(DEV_NULL));
+ }
+
+ // The output is redirected to a temporary file because:
+ // - We could read continuously from process.getInputStream, but that may block
+ // indefinitely with a faulty program.
+ // - If we don't read continuously from process.getInputStream, then because
+ // the underlying channel may be a pipe, the child may be stopped because the pipe
+ // is full.
+ // - To honor the timeout, no API can be used that may end up blocking indefinitely.
+ //
+ // Therefore, we redirect the output to a file and use waitFor w/timeout. This also
+ // has the benefit of allowing for inspection of the file during execution, and
+ // allowing the inspection of the file if it e.g. gets too large to hold in-memory.
+
+ String temporaryFilePrefix =
+ ProcessFactoryImpl.class.getSimpleName() + "-" + commandLine.programName() + "-";
+
+ FileAttribute<Set<PosixFilePermission>> fileAttribute = PosixFilePermissions.asFileAttribute(
+ PosixFilePermissions.fromString("rw-------"));
+
+ Path temporaryFile = uncheck(() -> Files.createTempFile(
+ temporaryFilePrefix,
+ ".out",
+ fileAttribute));
+
+ try {
+ processBuilder.redirectOutput(temporaryFile.toFile());
+ ProcessApi2 process = processStarter.start(processBuilder);
+ ChildProcess2Impl child = new ChildProcess2Impl(commandLine, process, temporaryFile, timer);
+ // child is now responsible for deleting temporaryFile (it's AutoClosable)
+ temporaryFile = null;
+ return child;
+ } finally {
+ if (temporaryFile != null) {
+ try {
+ Files.delete(temporaryFile);
+ } catch (IOException e) {
+ logger.log(LogLevel.WARNING, "Failed to delete temporary file at " +
+ temporaryFile, e);
+ }
+ }
+ }
+
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarter.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarter.java
new file mode 100644
index 00000000000..0afd4c6ee37
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarter.java
@@ -0,0 +1,10 @@
+// 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.process;
+
+/**
+ * @author hakonhall
+ */
+public interface ProcessStarter {
+ ProcessApi2 start(ProcessBuilder processBuilder);
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarterImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarterImpl.java
new file mode 100644
index 00000000000..2694a2929c4
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/ProcessStarterImpl.java
@@ -0,0 +1,16 @@
+// 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.process;
+
+import static com.yahoo.vespa.hosted.node.admin.task.util.file.IOExceptionUtil.uncheck;
+
+/**
+ * @author hakonhall
+ */
+public class ProcessStarterImpl implements ProcessStarter {
+ @Override
+ public ProcessApi2 start(ProcessBuilder processBuilder) {
+ Process process = uncheck(() -> processBuilder.start());
+ return new ProcessApi2Impl(process);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/Terminal.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/Terminal.java
new file mode 100644
index 00000000000..849099ab5ca
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/Terminal.java
@@ -0,0 +1,14 @@
+// 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.process;
+
+import com.yahoo.vespa.hosted.node.admin.component.TaskContext;
+
+/**
+ * A Terminal is a light-weight terminal-like interface for executing shell-like programs.
+ *
+ * @author hakonhall
+ */
+public interface Terminal {
+ CommandLine newCommandLine(TaskContext taskContext);
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TerminalImpl.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TerminalImpl.java
new file mode 100644
index 00000000000..93c1055fc58
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TerminalImpl.java
@@ -0,0 +1,19 @@
+// 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.process;
+
+import com.yahoo.vespa.hosted.node.admin.component.TaskContext;
+
+/**
+ * @author hakonhall
+ */
+public class TerminalImpl {
+ private final ProcessFactory processFactory;
+
+ public TerminalImpl(ProcessFactory processFactory) {
+ this.processFactory = processFactory;
+ }
+
+ public CommandLine newCommandLine(TaskContext taskContext) {
+ return new CommandLine(taskContext, processFactory);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestChildProcess2.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestChildProcess2.java
new file mode 100644
index 00000000000..4e678522168
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestChildProcess2.java
@@ -0,0 +1,52 @@
+// 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.process;
+
+import java.util.Optional;
+
+/**
+ * @author hakonhall
+ */
+public class TestChildProcess2 implements ChildProcess2 {
+ private final int exitCode;
+ private final String output;
+ private Optional<RuntimeException> exceptionToThrowInWaitForTermination = Optional.empty();
+ private boolean closeCalled = false;
+
+ public TestChildProcess2(int exitCode, String output) {
+ this.exitCode = exitCode;
+ this.output = output;
+ }
+
+ public void throwInWaitForTermination(RuntimeException e) {
+ this.exceptionToThrowInWaitForTermination = Optional.of(e);
+ }
+
+ @Override
+ public void waitForTermination() {
+ if (exceptionToThrowInWaitForTermination.isPresent()) {
+ throw exceptionToThrowInWaitForTermination.get();
+ }
+ }
+
+ @Override
+ public int exitCode() {
+ return exitCode;
+ }
+
+ @Override
+ public String getOutput() {
+ return output;
+ }
+
+ @Override
+ public void close() {
+ if (closeCalled) {
+ throw new IllegalStateException("close already called");
+ }
+ closeCalled = true;
+ }
+
+ public boolean closeCalled() {
+ return closeCalled;
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestProcessFactory.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestProcessFactory.java
new file mode 100644
index 00000000000..0586797d259
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestProcessFactory.java
@@ -0,0 +1,95 @@
+// 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.process;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+
+/**
+ * @author hakonhall
+ */
+public class TestProcessFactory implements ProcessFactory {
+ private static class SpawnCall {
+ private final String commandDescription;
+ private final Function<CommandLine, ChildProcess2> callback;
+
+ private SpawnCall(String commandDescription,
+ Function<CommandLine, ChildProcess2> callback) {
+ this.commandDescription = commandDescription;
+ this.callback = callback;
+ }
+ }
+ private final List<SpawnCall> expectedSpawnCalls = new ArrayList<>();
+ private final List<CommandLine> spawnCommandLines = new ArrayList<>();
+
+ /** Forward call to spawn() to callback. */
+ public TestProcessFactory interceptSpawn(String commandDescription,
+ Function<CommandLine, ChildProcess2> callback) {
+ expectedSpawnCalls.add(new SpawnCall(commandDescription, callback));
+ return this;
+ }
+
+ // Convenience method for the caller to avoid having to create a TestChildProcess2 instance.
+ public TestProcessFactory expectSpawn(String commandLineString, TestChildProcess2 toReturn) {
+ return interceptSpawn(
+ commandLineString,
+ commandLine -> defaultSpawn(commandLine, commandLineString, toReturn));
+ }
+
+ // Convenience method for the caller to avoid having to create a TestChildProcess2 instance.
+ public TestProcessFactory expectSpawn(String commandLine, int exitCode, String output) {
+ return expectSpawn(commandLine, new TestChildProcess2(exitCode, output));
+ }
+
+ /** Ignore the CommandLine passed to spawn(), just return successfully with the given output. */
+ public TestProcessFactory ignoreSpawn(String output) {
+ return interceptSpawn(
+ "[call index " + expectedSpawnCalls.size() + "]",
+ commandLine -> new TestChildProcess2(0, output));
+ }
+
+ public TestProcessFactory ignoreSpawn() {
+ return ignoreSpawn("");
+ }
+
+ public void verifyAllCommandsExecuted() {
+ if (spawnCommandLines.size() < expectedSpawnCalls.size()) {
+ int missingCommandIndex = spawnCommandLines.size();
+ throw new IllegalStateException("Command #" + missingCommandIndex +
+ " never executed: " +
+ expectedSpawnCalls.get(missingCommandIndex).commandDescription);
+ }
+ }
+
+ /**
+ * WARNING: CommandLine is mutable, and e.g. reusing a CommandLine for the next call
+ * would make the CommandLine in this list no longer reflect the original CommandLine.
+ */
+ public List<CommandLine> getMutableCommandLines() {
+ return spawnCommandLines;
+ }
+
+ @Override
+ public ChildProcess2 spawn(CommandLine commandLine) {
+ String commandLineString = commandLine.toString();
+ if (spawnCommandLines.size() + 1 > expectedSpawnCalls.size()) {
+ throw new IllegalStateException("Too many invocations: " + commandLineString);
+ }
+ spawnCommandLines.add(commandLine);
+
+ return expectedSpawnCalls.get(spawnCommandLines.size() - 1).callback.apply(commandLine);
+ }
+
+ private static ChildProcess2 defaultSpawn(CommandLine commandLine,
+ String expectedCommandLineString,
+ ChildProcess2 toReturn) {
+ String actualCommandLineString = commandLine.toString();
+ if (!Objects.equals(actualCommandLineString, expectedCommandLineString)) {
+ throw new IllegalArgumentException("Expected command line '" +
+ expectedCommandLineString + "' but got '" + actualCommandLineString + "'");
+ }
+
+ return toReturn;
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestTerminal.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestTerminal.java
new file mode 100644
index 00000000000..57aeeb04532
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TestTerminal.java
@@ -0,0 +1,67 @@
+// 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.process;
+
+import com.yahoo.vespa.hosted.node.admin.component.TaskContext;
+
+import java.util.function.Function;
+
+/**
+ * @author hakonhall
+ */
+public class TestTerminal implements Terminal {
+ private final TerminalImpl realTerminal;
+ private final TestProcessFactory testProcessFactory = new TestProcessFactory();
+
+ public TestTerminal() {
+ this.realTerminal = new TerminalImpl(testProcessFactory);
+ }
+
+ /** Get the TestProcessFactory the terminal was started with. */
+ public TestProcessFactory getTestProcessFactory() { return testProcessFactory; }
+
+ /** Forward call to spawn() to callback. */
+ public TestTerminal interceptCommand(String commandDescription,
+ Function<CommandLine, ChildProcess2> callback) {
+ testProcessFactory.interceptSpawn(commandDescription, callback);
+ return this;
+ }
+
+ /** Wraps expectSpawn in TestProcessFactory, provided here as convenience. */
+ public TestTerminal expectCommand(String commandLine, TestChildProcess2 toReturn) {
+ testProcessFactory.expectSpawn(commandLine, toReturn);
+ return this;
+ }
+
+ /** Wraps expectSpawn in TestProcessFactory, provided here as convenience. */
+ public TestTerminal expectCommand(String commandLine, int exitCode, String output) {
+ testProcessFactory.expectSpawn(commandLine, new TestChildProcess2(exitCode, output));
+ return this;
+ }
+
+ /** Verifies command line matches commandLine, and returns successfully with output "". */
+ public TestTerminal expectCommand(String commandLine) {
+ expectCommand(commandLine, 0, "");
+ return this;
+ }
+
+ /** Wraps expectSpawn in TestProcessFactory, provided here as convenience. */
+ public TestTerminal ignoreCommand(String output) {
+ testProcessFactory.ignoreSpawn(output);
+ return this;
+ }
+
+ /** Wraps expectSpawn in TestProcessFactory, provided here as convenience. */
+ public TestTerminal ignoreCommand() {
+ testProcessFactory.ignoreSpawn();
+ return this;
+ }
+
+ public void verifyAllCommandsExecuted() {
+ testProcessFactory.verifyAllCommandsExecuted();
+ }
+
+ @Override
+ public CommandLine newCommandLine(TaskContext taskContext) {
+ return realTerminal.newCommandLine(taskContext);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TimeoutChildProcessException.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TimeoutChildProcessException.java
new file mode 100644
index 00000000000..df9e2dc3471
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/TimeoutChildProcessException.java
@@ -0,0 +1,18 @@
+// 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.process;
+
+import java.time.Duration;
+
+/**
+ * Exception thrown when a child process has taken too long to terminate, in case it has been
+ * forcibly killed.
+ *
+ * @author hakonhall
+ */
+@SuppressWarnings("serial")
+public class TimeoutChildProcessException extends ChildProcessException {
+ TimeoutChildProcessException(Duration timeout, String commandLine, String possiblyHugeOutput) {
+ super("timed out after " + timeout, commandLine, possiblyHugeOutput);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnexpectedOutputException2.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnexpectedOutputException2.java
new file mode 100644
index 00000000000..e786452c0ef
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnexpectedOutputException2.java
@@ -0,0 +1,16 @@
+// 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.process;
+
+/**
+ * @author hakonhall
+ */
+@SuppressWarnings("serial")
+public class UnexpectedOutputException2 extends ChildProcessException {
+ /**
+ * @param problem Problem description, e.g. "Output is not of the form ^NAME=VALUE$"
+ */
+ public UnexpectedOutputException2(String problem, String commandLine, String possiblyHugeOutput) {
+ super("output was not of the expected format: " + problem, commandLine, possiblyHugeOutput);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnkillableChildProcessException.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnkillableChildProcessException.java
new file mode 100644
index 00000000000..1da27dd853e
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/process/UnkillableChildProcessException.java
@@ -0,0 +1,21 @@
+// 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.process;
+
+import java.time.Duration;
+
+/**
+ * @author hakonhall
+ */
+@SuppressWarnings("serial")
+public class UnkillableChildProcessException extends ChildProcessException {
+ public UnkillableChildProcessException(Duration waitForSigTerm,
+ Duration waitForSigKill,
+ String commandLine,
+ String possiblyHugeOutput) {
+ super("did not terminate even after SIGTERM, +" + waitForSigTerm +
+ ", SIGKILL, and +" + waitForSigKill,
+ commandLine,
+ possiblyHugeOutput);
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/time/TestTimer.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/time/TestTimer.java
new file mode 100644
index 00000000000..beadeeed4a3
--- /dev/null
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/time/TestTimer.java
@@ -0,0 +1,29 @@
+// 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.time;
+
+import com.yahoo.jdisc.Timer;
+
+import java.time.Duration;
+
+/**
+ * @author hakonhall
+ */
+public class TestTimer implements Timer {
+ private Duration durationSinceEpoch = Duration.ZERO;
+
+ public void setMillis(long millisSinceEpoch) {
+ durationSinceEpoch = Duration.ofMillis(millisSinceEpoch);
+ }
+
+ public void advanceMillis(long millis) { advance(Duration.ofMillis(millis)); }
+ public void advanceSeconds(long seconds) { advance(Duration.ofSeconds(seconds)); }
+ public void advanceMinutes(long minutes) { advance(Duration.ofMinutes(minutes)); }
+ public void advance(Duration duration) {
+ durationSinceEpoch = durationSinceEpoch.plus(duration);
+ }
+
+ @Override
+ public long currentTimeMillis() {
+ return durationSinceEpoch.toMillis();
+ }
+}
diff --git a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/yum/Yum.java b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/yum/Yum.java
index dbb4c909a4b..188de5a2f17 100644
--- a/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/yum/Yum.java
+++ b/node-admin/src/main/java/com/yahoo/vespa/hosted/node/admin/task/util/yum/Yum.java
@@ -88,6 +88,21 @@ public class Yum {
}
public boolean converge() {
+// Terminal terminal;
+// CommandLine commandLine = terminal.newCommandLine(taskContext);
+// commandLine.add("yum", yumCommand, "--assumeyes");
+// enabledRepo.ifPresent(repo -> commandLine.add("--enablerepo=" + repo));
+// commandLine.add(packages);
+// CommandResult result = commandLine.executeSilently();
+//
+// String output = result.getUntrimmedOutput();
+// if (commandOutputNoopPattern.matcher(output).find()) {
+// return false;
+// } else {
+// commandLine.recordSilentExecutionAsSystemModification();
+// return true;
+// }
+
Command command = commandSupplier.get();
command.add("yum", yumCommand, "--assumeyes");
enabledRepo.ifPresent(repo -> command.add("--enablerepo=" + repo));