mirror of https://github.com/apache/druid.git
ForkingTaskRunner: Use guava Closer to closer stuff
This commit is contained in:
parent
952b8ce06b
commit
5d44f0f15b
|
@ -32,6 +32,7 @@ import com.google.common.collect.Lists;
|
|||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.Closer;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -116,137 +117,135 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
|||
final ProcessHolder processHolder;
|
||||
|
||||
try {
|
||||
if (!attemptDir.mkdirs()) {
|
||||
throw new IOException(String.format("Could not create directories: %s", attemptDir));
|
||||
}
|
||||
|
||||
final File taskFile = new File(attemptDir, "task.json");
|
||||
final File statusFile = new File(attemptDir, "status.json");
|
||||
final File logFile = new File(attemptDir, "log");
|
||||
|
||||
// time to adjust process holders
|
||||
synchronized (tasks) {
|
||||
final TaskInfo taskInfo = tasks.get(task.getId());
|
||||
|
||||
if (taskInfo.shutdown) {
|
||||
throw new IllegalStateException("Task has been shut down!");
|
||||
final Closer closer = Closer.create();
|
||||
try {
|
||||
if (!attemptDir.mkdirs()) {
|
||||
throw new IOException(String.format("Could not create directories: %s", attemptDir));
|
||||
}
|
||||
|
||||
if (taskInfo == null) {
|
||||
throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId());
|
||||
}
|
||||
final File taskFile = new File(attemptDir, "task.json");
|
||||
final File statusFile = new File(attemptDir, "status.json");
|
||||
final File logFile = new File(attemptDir, "log");
|
||||
|
||||
if (taskInfo.processHolder != null) {
|
||||
throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId());
|
||||
}
|
||||
// time to adjust process holders
|
||||
synchronized (tasks) {
|
||||
final TaskInfo taskInfo = tasks.get(task.getId());
|
||||
|
||||
final List<String> command = Lists.newArrayList();
|
||||
final int childPort = findUnusedPort();
|
||||
final String childHost = String.format(config.getHostPattern(), childPort);
|
||||
if (taskInfo.shutdown) {
|
||||
throw new IllegalStateException("Task has been shut down!");
|
||||
}
|
||||
|
||||
command.add(config.getJavaCommand());
|
||||
command.add("-cp");
|
||||
command.add(config.getJavaClasspath());
|
||||
if (taskInfo == null) {
|
||||
throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId());
|
||||
}
|
||||
|
||||
Iterables.addAll(
|
||||
command,
|
||||
Splitter.on(CharMatcher.WHITESPACE)
|
||||
.omitEmptyStrings()
|
||||
.split(config.getJavaOptions())
|
||||
);
|
||||
if (taskInfo.processHolder != null) {
|
||||
throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId());
|
||||
}
|
||||
|
||||
for (String propName : props.stringPropertyNames()) {
|
||||
for (String allowedPrefix : config.getAllowedPrefixes()) {
|
||||
if (propName.startsWith(allowedPrefix)) {
|
||||
final List<String> command = Lists.newArrayList();
|
||||
final int childPort = findUnusedPort();
|
||||
final String childHost = String.format(config.getHostPattern(), childPort);
|
||||
|
||||
command.add(config.getJavaCommand());
|
||||
command.add("-cp");
|
||||
command.add(config.getJavaClasspath());
|
||||
|
||||
Iterables.addAll(
|
||||
command,
|
||||
Splitter.on(CharMatcher.WHITESPACE)
|
||||
.omitEmptyStrings()
|
||||
.split(config.getJavaOptions())
|
||||
);
|
||||
|
||||
for (String propName : props.stringPropertyNames()) {
|
||||
for (String allowedPrefix : config.getAllowedPrefixes()) {
|
||||
if (propName.startsWith(allowedPrefix)) {
|
||||
command.add(
|
||||
String.format(
|
||||
"-D%s=%s",
|
||||
propName,
|
||||
props.getProperty(propName)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Override child JVM specific properties
|
||||
for (String propName : props.stringPropertyNames()) {
|
||||
if (propName.startsWith(CHILD_PROPERTY_PREFIX)) {
|
||||
command.add(
|
||||
String.format(
|
||||
"-D%s=%s",
|
||||
propName,
|
||||
propName.substring(CHILD_PROPERTY_PREFIX.length()),
|
||||
props.getProperty(propName)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Override child JVM specific properties
|
||||
for (String propName : props.stringPropertyNames()) {
|
||||
if (propName.startsWith(CHILD_PROPERTY_PREFIX)) {
|
||||
command.add(
|
||||
String.format(
|
||||
"-D%s=%s",
|
||||
propName.substring(CHILD_PROPERTY_PREFIX.length()),
|
||||
props.getProperty(propName)
|
||||
)
|
||||
);
|
||||
String nodeType = task.getNodeType();
|
||||
if (nodeType != null) {
|
||||
command.add(String.format("-Ddruid.executor.nodeType=%s", nodeType));
|
||||
}
|
||||
|
||||
command.add(String.format("-Ddruid.host=%s", childHost));
|
||||
command.add(String.format("-Ddruid.port=%d", childPort));
|
||||
|
||||
command.add(config.getMainClass());
|
||||
command.add(taskFile.toString());
|
||||
command.add(statusFile.toString());
|
||||
|
||||
jsonMapper.writeValue(taskFile, task);
|
||||
|
||||
log.info("Running command: %s", Joiner.on(" ").join(command));
|
||||
taskInfo.processHolder = new ProcessHolder(
|
||||
new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(),
|
||||
logFile,
|
||||
childPort
|
||||
);
|
||||
|
||||
processHolder = taskInfo.processHolder;
|
||||
processHolder.registerWithCloser(closer);
|
||||
}
|
||||
|
||||
String nodeType = task.getNodeType();
|
||||
if (nodeType != null) {
|
||||
command.add(String.format("-Ddruid.executor.nodeType=%s", nodeType));
|
||||
}
|
||||
log.info("Logging task %s output to: %s", task.getId(), logFile);
|
||||
|
||||
command.add(String.format("-Ddruid.host=%s", childHost));
|
||||
command.add(String.format("-Ddruid.port=%d", childPort));
|
||||
|
||||
command.add(config.getMainClass());
|
||||
command.add(taskFile.toString());
|
||||
command.add(statusFile.toString());
|
||||
|
||||
jsonMapper.writeValue(taskFile, task);
|
||||
|
||||
log.info("Running command: %s", Joiner.on(" ").join(command));
|
||||
taskInfo.processHolder = new ProcessHolder(
|
||||
new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(),
|
||||
logFile,
|
||||
childPort
|
||||
final InputStream fromProc = processHolder.process.getInputStream();
|
||||
final OutputStream toLogfile = closer.register(
|
||||
Files.newOutputStreamSupplier(logFile).getOutput()
|
||||
);
|
||||
|
||||
processHolder = taskInfo.processHolder;
|
||||
}
|
||||
boolean runFailed = true;
|
||||
|
||||
log.info("Logging task %s output to: %s", task.getId(), logFile);
|
||||
|
||||
final InputStream fromProc = processHolder.process.getInputStream();
|
||||
final OutputStream toLogfile = Files.newOutputStreamSupplier(logFile).getOutput();
|
||||
|
||||
boolean runFailed = false;
|
||||
|
||||
try {
|
||||
ByteStreams.copy(fromProc, toLogfile);
|
||||
final int statusCode = processHolder.process.waitFor();
|
||||
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
|
||||
|
||||
if (statusCode != 0) {
|
||||
runFailed = true;
|
||||
if (statusCode == 0) {
|
||||
runFailed = false;
|
||||
}
|
||||
|
||||
toLogfile.close();
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Failed to log process output for task: %s", task.getId());
|
||||
runFailed = true;
|
||||
Closeables.close(toLogfile, true);
|
||||
}
|
||||
finally {
|
||||
Closeables.close(processHolder, true);
|
||||
}
|
||||
// Upload task logs
|
||||
|
||||
// Upload task logs
|
||||
// XXX: Consider uploading periodically for very long-lived tasks to prevent
|
||||
// XXX: bottlenecks at the end or the possibility of losing a lot of logs all
|
||||
// XXX: at once.
|
||||
|
||||
// XXX: Consider uploading periodically for very long-lived tasks to prevent
|
||||
// XXX: bottlenecks at the end or the possibility of losing a lot of logs all
|
||||
// XXX: at once.
|
||||
taskLogPusher.pushTaskLog(task.getId(), logFile);
|
||||
|
||||
taskLogPusher.pushTaskLog(task.getId(), logFile);
|
||||
|
||||
if (!runFailed) {
|
||||
// Process exited successfully
|
||||
return jsonMapper.readValue(statusFile, TaskStatus.class);
|
||||
} else {
|
||||
// Process exited unsuccessfully
|
||||
return TaskStatus.failure(task.getId());
|
||||
if (!runFailed) {
|
||||
// Process exited successfully
|
||||
return jsonMapper.readValue(statusFile, TaskStatus.class);
|
||||
} else {
|
||||
// Process exited unsuccessfully
|
||||
return TaskStatus.failure(task.getId());
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
throw closer.rethrow(t);
|
||||
} finally {
|
||||
closer.close();
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
@ -403,7 +402,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
|||
}
|
||||
}
|
||||
|
||||
private static class ProcessHolder implements Closeable
|
||||
private static class ProcessHolder
|
||||
{
|
||||
private final Process process;
|
||||
private final File logFile;
|
||||
|
@ -416,11 +415,10 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
|||
this.port = port;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
private void registerWithCloser(Closer closer)
|
||||
{
|
||||
process.getInputStream().close();
|
||||
process.getOutputStream().close();
|
||||
closer.register(process.getInputStream());
|
||||
closer.register(process.getOutputStream());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue