Skip to content

Commit 6683866

Browse files
authored
TEZ-4646: Periodic jstack collection for tez (tez.thread.dump.interval) only collects jstacks once. (#425). (Ayush Saxena, reviewed by Laszlo Bodor)
1 parent 6e357a0 commit 6683866

File tree

3 files changed

+38
-12
lines changed

3 files changed

+38
-12
lines changed

tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2309,6 +2309,16 @@ static Set<String> getPropertySet() {
23092309
public static final String TEZ_THREAD_DUMP_INTERVAL = "tez.thread.dump.interval";
23102310
public static final String TEZ_THREAD_DUMP_INTERVAL_DEFAULT = "100ms";
23112311

2312+
/**
2313+
* Time after which the first thread dump should be captured. Supports TimeUnits. This is effective only
2314+
* when org.apache.tez.dag.app.ThreadDumpDAGHook is configured to tez.am.hooks or
2315+
* org.apache.tez.runtime.task.ThreadDumpTaskAttemptHook is configured to tez.task.attempt.hooks.
2316+
*/
2317+
@ConfigurationScope(Scope.DAG)
2318+
@ConfigurationProperty
2319+
public static final String TEZ_THREAD_DUMP_INITIAL_DELAY = "tez.thread.dump.initial.delay";
2320+
public static final String TEZ_THREAD_DUMP_INITIAL_DELAY_DEFAULT = "0ms";
2321+
23122322
/**
23132323
* Limits the amount of data that can be written to LocalFileSystem by a Task.
23142324
*/

tez-runtime-internals/src/main/java/org/apache/tez/runtime/TezThreadDumpHelper.java

Lines changed: 22 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@
2020

2121
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR;
2222
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_REMOTE_APP_LOG_DIR;
23+
import static org.apache.tez.dag.api.TezConfiguration.TEZ_THREAD_DUMP_INITIAL_DELAY;
24+
import static org.apache.tez.dag.api.TezConfiguration.TEZ_THREAD_DUMP_INITIAL_DELAY_DEFAULT;
2325
import static org.apache.tez.dag.api.TezConfiguration.TEZ_THREAD_DUMP_INTERVAL;
2426
import static org.apache.tez.dag.api.TezConfiguration.TEZ_THREAD_DUMP_INTERVAL_DEFAULT;
2527

@@ -28,6 +30,7 @@
2830
import java.lang.management.ManagementFactory;
2931
import java.lang.management.ThreadInfo;
3032
import java.lang.management.ThreadMXBean;
33+
import java.nio.charset.StandardCharsets;
3134
import java.util.concurrent.Executors;
3235
import java.util.concurrent.ScheduledExecutorService;
3336
import java.util.concurrent.TimeUnit;
@@ -49,7 +52,8 @@
4952

5053
public class TezThreadDumpHelper {
5154

52-
private final long duration;
55+
private final long threadDumpFrequency;
56+
private final long threadDumpInitialDelay;
5357
private final Path basePath;
5458
private final FileSystem fs;
5559

@@ -58,8 +62,10 @@ public class TezThreadDumpHelper {
5862

5963
private ScheduledExecutorService periodicThreadDumpServiceExecutor;
6064

61-
private TezThreadDumpHelper(long duration, Configuration conf) throws IOException {
62-
this.duration = duration;
65+
private TezThreadDumpHelper(long threadDumpFrequency, long threadDumpInitialDelay, Configuration conf)
66+
throws IOException {
67+
this.threadDumpFrequency = threadDumpFrequency;
68+
this.threadDumpInitialDelay = threadDumpInitialDelay;
6369
Appender appender = org.apache.log4j.Logger.getRootLogger().getAppender(TezConstants.TEZ_CONTAINER_LOGGER_NAME);
6470
if (appender instanceof TezContainerLogAppender) {
6571
this.basePath = new Path(((TezContainerLogAppender) appender).getContainerLogDir());
@@ -69,18 +75,21 @@ private TezThreadDumpHelper(long duration, Configuration conf) throws IOExceptio
6975
this.basePath = new Path(conf.get(NM_REMOTE_APP_LOG_DIR, DEFAULT_NM_REMOTE_APP_LOG_DIR));
7076
this.fs = this.basePath.getFileSystem(conf);
7177
}
72-
LOG.info("Periodic Thread Dump Capture Service Configured to capture Thread Dumps at {} ms frequency and at " +
73-
"path: {}", duration, basePath);
78+
LOG.info("Periodic Thread Dump Capture Service Configured to capture Thread Dumps at {} ms frequency and at "
79+
+ "path: {} with an initial delay of {}", threadDumpFrequency, basePath, threadDumpInitialDelay);
7480
}
7581

7682
public static TezThreadDumpHelper getInstance(Configuration conf) {
77-
long periodicThreadDumpFrequency = conf.getTimeDuration(TEZ_THREAD_DUMP_INTERVAL,
78-
TEZ_THREAD_DUMP_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
79-
Preconditions.checkArgument(periodicThreadDumpFrequency > 0, "%s must be positive duration",
80-
TEZ_THREAD_DUMP_INTERVAL);
83+
long threadDumpFrequency =
84+
conf.getTimeDuration(TEZ_THREAD_DUMP_INTERVAL, TEZ_THREAD_DUMP_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
85+
Preconditions.checkArgument(threadDumpFrequency > 0, "%s must be positive duration", TEZ_THREAD_DUMP_INTERVAL);
86+
long threadDumpInitialDelay =
87+
conf.getTimeDuration(TEZ_THREAD_DUMP_INITIAL_DELAY, TEZ_THREAD_DUMP_INITIAL_DELAY_DEFAULT,
88+
TimeUnit.MILLISECONDS);
89+
Preconditions.checkArgument(threadDumpInitialDelay >= 0, "%s can not be negative", TEZ_THREAD_DUMP_INITIAL_DELAY);
8190

8291
try {
83-
return new TezThreadDumpHelper(periodicThreadDumpFrequency, conf);
92+
return new TezThreadDumpHelper(threadDumpFrequency, threadDumpInitialDelay, conf);
8493
} catch (IOException e) {
8594
throw new TezUncheckedException("Can not initialize periodic thread dump service", e);
8695
}
@@ -91,7 +100,8 @@ public TezThreadDumpHelper start(String name) {
91100
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("PeriodicThreadDumpService{" + name + "} #%d")
92101
.build());
93102
Runnable threadDumpCollector = new ThreadDumpCollector(basePath, name, fs);
94-
periodicThreadDumpServiceExecutor.schedule(threadDumpCollector, duration, TimeUnit.MILLISECONDS);
103+
periodicThreadDumpServiceExecutor.scheduleWithFixedDelay(threadDumpCollector, threadDumpInitialDelay,
104+
threadDumpFrequency, TimeUnit.MILLISECONDS);
95105
return this;
96106
}
97107

@@ -128,7 +138,7 @@ public void run() {
128138
if (!Thread.interrupted()) {
129139
try (FSDataOutputStream fsStream = fs.create(
130140
new Path(path, name + "_" + System.currentTimeMillis() + ".jstack"));
131-
PrintStream printStream = new PrintStream(fsStream, false, "UTF8")) {
141+
PrintStream printStream = new PrintStream(fsStream, false, StandardCharsets.UTF_8)) {
132142
printThreadInfo(printStream, name);
133143
} catch (IOException e) {
134144
throw new RuntimeException(e);

tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -628,18 +628,24 @@ private static void validateThreadDumpCaptured(Path jstackPath) throws IOExcepti
628628
RemoteIterator<LocatedFileStatus> files = localFs.listFiles(jstackPath, true);
629629
boolean appMasterDumpFound = false;
630630
boolean tezChildDumpFound = false;
631+
int numAppMasterDumps = 0;
632+
int numTezChildDumps = 0;
631633
while (files.hasNext()) {
632634
LocatedFileStatus file = files.next();
633635
if (file.getPath().getName().endsWith(".jstack")) {
634636
if (file.getPath().getName().contains("attempt")) {
635637
tezChildDumpFound = true;
638+
numTezChildDumps++;
636639
} else {
637640
appMasterDumpFound = true;
641+
numAppMasterDumps++;
638642
}
639643
}
640644
}
641645
assertTrue(tezChildDumpFound);
642646
assertTrue(appMasterDumpFound);
647+
assertTrue(numAppMasterDumps >= 2);
648+
assertTrue(numTezChildDumps >= 2);
643649
}
644650

645651
/**

0 commit comments

Comments
 (0)