YARN-7009. TestNMClient.testNMClientNoCleanupOnStop is flaky by design. (Miklos Szegedi via Haibo Chen)
This commit is contained in:
parent
b6942cbe9b
commit
c071aad5da
@ -4869,7 +4869,7 @@
|
||||
"hadoop.security.groups.cache.secs" : "300",
|
||||
"ipc.client.connect.max.retries" : "10",
|
||||
"dfs.namenode.delegation.key.update-interval" : "86400000",
|
||||
"yarn.nodemanager.process-kill-wait.ms" : "2000",
|
||||
"yarn.nodemanager.process-kill-wait.ms" : "5000",
|
||||
"yarn.application.classpath" : "$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$YARN_HOME/*,$YARN_HOME/lib/*",
|
||||
"yarn.app.mapreduce.client.max-retries" : "3",
|
||||
"dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction" : "0.75f",
|
||||
@ -9950,7 +9950,7 @@
|
||||
"hadoop.security.groups.cache.secs" : "300",
|
||||
"ipc.client.connect.max.retries" : "10",
|
||||
"dfs.namenode.delegation.key.update-interval" : "86400000",
|
||||
"yarn.nodemanager.process-kill-wait.ms" : "2000",
|
||||
"yarn.nodemanager.process-kill-wait.ms" : "5000",
|
||||
"yarn.application.classpath" : "$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$YARN_HOME/*,$YARN_HOME/lib/*",
|
||||
"yarn.app.mapreduce.client.max-retries" : "3",
|
||||
"dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction" : "0.75f",
|
||||
|
@ -1907,7 +1907,7 @@ public static boolean isAclEnabled(Configuration conf) {
|
||||
public static final String NM_PROCESS_KILL_WAIT_MS =
|
||||
NM_PREFIX + "process-kill-wait.ms";
|
||||
public static final long DEFAULT_NM_PROCESS_KILL_WAIT_MS =
|
||||
2000;
|
||||
5000;
|
||||
|
||||
/** Max time to wait to establish a connection to RM */
|
||||
public static final String RESOURCEMANAGER_CONNECT_MAX_WAIT_MS =
|
||||
|
@ -27,10 +27,15 @@
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
@ -67,6 +72,10 @@
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
@ -85,11 +94,78 @@ public class TestNMClient {
|
||||
ApplicationAttemptId attemptId = null;
|
||||
int nodeCount = 3;
|
||||
NMTokenCache nmTokenCache = null;
|
||||
|
||||
|
||||
/**
|
||||
* Container State transition listener to track the number of times
|
||||
* a container has transitioned into a state.
|
||||
*/
|
||||
public static class DebugSumContainerStateListener
|
||||
implements ContainerStateTransitionListener {
|
||||
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(DebugSumContainerStateListener.class);
|
||||
private static final Map<ContainerId,
|
||||
Map<org.apache.hadoop.yarn.server.nodemanager.containermanager
|
||||
.container.ContainerState, Long>>
|
||||
TRANSITION_COUNTER = new HashMap<>();
|
||||
|
||||
public void init(Context context) {
|
||||
}
|
||||
|
||||
public void preTransition(ContainerImpl op,
|
||||
org.apache.hadoop.yarn.server.nodemanager
|
||||
.containermanager.container.ContainerState
|
||||
beforeState,
|
||||
ContainerEvent eventToBeProcessed) {
|
||||
}
|
||||
|
||||
public void postTransition(
|
||||
ContainerImpl op,
|
||||
org.apache.hadoop.yarn.server.nodemanager.containermanager.container
|
||||
.ContainerState beforeState,
|
||||
org.apache.hadoop.yarn.server.nodemanager.containermanager.container
|
||||
.ContainerState afterState,
|
||||
ContainerEvent processedEvent) {
|
||||
synchronized (TRANSITION_COUNTER) {
|
||||
if (beforeState != afterState) {
|
||||
ContainerId id = op.getContainerId();
|
||||
TRANSITION_COUNTER
|
||||
.putIfAbsent(id, new HashMap<>());
|
||||
long sum = TRANSITION_COUNTER.get(id)
|
||||
.compute(afterState,
|
||||
(state, count) -> count == null ? 1 : count + 1);
|
||||
LOG.info("***** " + id +
|
||||
" Transition from " + beforeState +
|
||||
" to " + afterState +
|
||||
"sum:" + sum);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the current number of state transitions.
|
||||
* This is useful to check, if an event has occurred in unit tests.
|
||||
* @param id Container id to check
|
||||
* @param state Return the overall number of transitions to this state
|
||||
* @return Number of transitions to the state specified
|
||||
*/
|
||||
static long getTransitionCounter(ContainerId id,
|
||||
org.apache.hadoop.yarn.server.nodemanager
|
||||
.containermanager.container
|
||||
.ContainerState state) {
|
||||
Long ret = TRANSITION_COUNTER.getOrDefault(id, new HashMap<>())
|
||||
.get(state);
|
||||
return ret != null ? ret : 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws YarnException, IOException {
|
||||
// start minicluster
|
||||
conf = new YarnConfiguration();
|
||||
// Turn on state tracking
|
||||
conf.set(YarnConfiguration.NM_CONTAINER_STATE_TRANSITION_LISTENERS,
|
||||
DebugSumContainerStateListener.class.getName());
|
||||
yarnCluster =
|
||||
new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
|
||||
yarnCluster.init(conf);
|
||||
@ -290,7 +366,7 @@ private Set<Container> allocateContainers(
|
||||
return containers;
|
||||
}
|
||||
|
||||
private void testContainerManagement(NMClientImpl nmClient,
|
||||
private void testContainerManagement(NMClientImpl client,
|
||||
Set<Container> containers) throws YarnException, IOException {
|
||||
int size = containers.size();
|
||||
int i = 0;
|
||||
@ -298,7 +374,7 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
// getContainerStatus shouldn't be called before startContainer,
|
||||
// otherwise, NodeManager cannot find the container
|
||||
try {
|
||||
nmClient.getContainerStatus(container.getId(), container.getNodeId());
|
||||
client.getContainerStatus(container.getId(), container.getNodeId());
|
||||
fail("Exception is expected");
|
||||
} catch (YarnException e) {
|
||||
assertTrue("The thrown exception is not expected",
|
||||
@ -307,7 +383,7 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
// upadateContainerResource shouldn't be called before startContainer,
|
||||
// otherwise, NodeManager cannot find the container
|
||||
try {
|
||||
nmClient.updateContainerResource(container);
|
||||
client.updateContainerResource(container);
|
||||
fail("Exception is expected");
|
||||
} catch (YarnException e) {
|
||||
assertTrue("The thrown exception is not expected",
|
||||
@ -317,7 +393,7 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
// restart shouldn't be called before startContainer,
|
||||
// otherwise, NodeManager cannot find the container
|
||||
try {
|
||||
nmClient.restartContainer(container.getId());
|
||||
client.restartContainer(container.getId());
|
||||
fail("Exception is expected");
|
||||
} catch (YarnException e) {
|
||||
assertTrue("The thrown exception is not expected",
|
||||
@ -327,7 +403,7 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
// rollback shouldn't be called before startContainer,
|
||||
// otherwise, NodeManager cannot find the container
|
||||
try {
|
||||
nmClient.rollbackLastReInitialization(container.getId());
|
||||
client.rollbackLastReInitialization(container.getId());
|
||||
fail("Exception is expected");
|
||||
} catch (YarnException e) {
|
||||
assertTrue("The thrown exception is not expected",
|
||||
@ -337,7 +413,7 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
// commit shouldn't be called before startContainer,
|
||||
// otherwise, NodeManager cannot find the container
|
||||
try {
|
||||
nmClient.commitLastReInitialization(container.getId());
|
||||
client.commitLastReInitialization(container.getId());
|
||||
fail("Exception is expected");
|
||||
} catch (YarnException e) {
|
||||
assertTrue("The thrown exception is not expected",
|
||||
@ -347,14 +423,12 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
// stopContainer shouldn't be called before startContainer,
|
||||
// otherwise, an exception will be thrown
|
||||
try {
|
||||
nmClient.stopContainer(container.getId(), container.getNodeId());
|
||||
client.stopContainer(container.getId(), container.getNodeId());
|
||||
fail("Exception is expected");
|
||||
} catch (YarnException e) {
|
||||
if (!e.getMessage()
|
||||
.contains("is not handled by this NodeManager")) {
|
||||
throw (AssertionError)
|
||||
(new AssertionError("Exception is not expected: " + e).initCause(
|
||||
e));
|
||||
throw new AssertionError("Exception is not expected: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@ -367,80 +441,133 @@ private void testContainerManagement(NMClientImpl nmClient,
|
||||
Records.newRecord(ContainerLaunchContext.class);
|
||||
if (Shell.WINDOWS) {
|
||||
clc.setCommands(
|
||||
Arrays.asList("ping", "-n", "100", "127.0.0.1", ">nul"));
|
||||
Arrays.asList("ping", "-n", "10000000", "127.0.0.1", ">nul"));
|
||||
} else {
|
||||
clc.setCommands(Arrays.asList("sleep", "10"));
|
||||
clc.setCommands(Arrays.asList("sleep", "1000000"));
|
||||
}
|
||||
clc.setTokens(securityTokens);
|
||||
try {
|
||||
nmClient.startContainer(container, clc);
|
||||
client.startContainer(container, clc);
|
||||
} catch (YarnException e) {
|
||||
throw (AssertionError)
|
||||
(new AssertionError("Exception is not expected: " + e).initCause(e));
|
||||
throw new AssertionError("Exception is not expected ", e);
|
||||
}
|
||||
List<Integer> exitStatuses = Collections.singletonList(-1000);
|
||||
|
||||
// leave one container unclosed
|
||||
if (++i < size) {
|
||||
// NodeManager may still need some time to make the container started
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING, "",
|
||||
Arrays.asList(new Integer[] {-1000}));
|
||||
// Test increase container API and make sure requests can reach NM
|
||||
testIncreaseContainerResource(container);
|
||||
testContainer(client, i, container, clc, exitStatuses);
|
||||
|
||||
testRestartContainer(container.getId());
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Restarted", Arrays.asList(new Integer[] {-1000}));
|
||||
|
||||
if (i % 2 == 0) {
|
||||
testReInitializeContainer(container.getId(), clc, false);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Re-initialized", Arrays.asList(new Integer[] {-1000}));
|
||||
testRollbackContainer(container.getId(), false);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Rolled-back", Arrays.asList(new Integer[] {-1000}));
|
||||
testCommitContainer(container.getId(), true);
|
||||
testReInitializeContainer(container.getId(), clc, false);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Re-initialized", Arrays.asList(new Integer[] {-1000}));
|
||||
testCommitContainer(container.getId(), false);
|
||||
} else {
|
||||
testReInitializeContainer(container.getId(), clc, true);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Re-initialized", Arrays.asList(new Integer[] {-1000}));
|
||||
testRollbackContainer(container.getId(), true);
|
||||
testCommitContainer(container.getId(), true);
|
||||
}
|
||||
|
||||
try {
|
||||
nmClient.stopContainer(container.getId(), container.getNodeId());
|
||||
} catch (YarnException e) {
|
||||
throw (AssertionError)
|
||||
(new AssertionError("Exception is not expected: " + e)
|
||||
.initCause(e));
|
||||
}
|
||||
|
||||
// getContainerStatus can be called after stopContainer
|
||||
try {
|
||||
// O is possible if CLEANUP_CONTAINER is executed too late
|
||||
// -105 is possible if the container is not terminated but killed
|
||||
testGetContainerStatus(container, i, ContainerState.COMPLETE,
|
||||
"Container killed by the ApplicationMaster.", Arrays.asList(
|
||||
new Integer[] {ContainerExitStatus.KILLED_BY_APPMASTER,
|
||||
ContainerExitStatus.SUCCESS}));
|
||||
} catch (YarnException e) {
|
||||
// The exception is possible because, after the container is stopped,
|
||||
// it may be removed from NM's context.
|
||||
if (!e.getMessage()
|
||||
.contains("was recently stopped on node manager")) {
|
||||
throw (AssertionError)
|
||||
(new AssertionError("Exception is not expected: " + e).initCause(
|
||||
e));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void testContainer(NMClientImpl client, int i, Container container,
|
||||
ContainerLaunchContext clc, List<Integer> exitCode)
|
||||
throws YarnException, IOException {
|
||||
// NodeManager may still need some time to make the container started
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING, "",
|
||||
exitCode);
|
||||
waitForContainerTransitionCount(container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState.RUNNING, 1);
|
||||
// Test increase container API and make sure requests can reach NM
|
||||
testIncreaseContainerResource(container);
|
||||
|
||||
testRestartContainer(container.getId());
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Restarted", exitCode);
|
||||
waitForContainerTransitionCount(container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState.RUNNING, 2);
|
||||
|
||||
if (i % 2 == 0) {
|
||||
testReInitializeContainer(container.getId(), clc, false);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Re-initialized", exitCode);
|
||||
waitForContainerTransitionCount(container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState.RUNNING, 3);
|
||||
|
||||
testRollbackContainer(container.getId(), false);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Rolled-back", exitCode);
|
||||
waitForContainerTransitionCount(container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState.RUNNING, 4);
|
||||
|
||||
testCommitContainer(container.getId(), true);
|
||||
testReInitializeContainer(container.getId(), clc, false);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Re-initialized", exitCode);
|
||||
waitForContainerTransitionCount(container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState.RUNNING, 5);
|
||||
testCommitContainer(container.getId(), false);
|
||||
} else {
|
||||
testReInitializeContainer(container.getId(), clc, true);
|
||||
testGetContainerStatus(container, i, ContainerState.RUNNING,
|
||||
"will be Re-initialized", exitCode);
|
||||
waitForContainerTransitionCount(container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState.RUNNING, 3);
|
||||
testRollbackContainer(container.getId(), true);
|
||||
testCommitContainer(container.getId(), true);
|
||||
}
|
||||
|
||||
try {
|
||||
client.stopContainer(container.getId(), container.getNodeId());
|
||||
} catch (YarnException e) {
|
||||
throw (AssertionError)
|
||||
(new AssertionError("Exception is not expected: " + e, e));
|
||||
}
|
||||
|
||||
// getContainerStatus can be called after stopContainer
|
||||
try {
|
||||
// O is possible if CLEANUP_CONTAINER is executed too late
|
||||
// -105 is possible if the container is not terminated but killed
|
||||
testGetContainerStatus(container, i, ContainerState.COMPLETE,
|
||||
"Container killed by the ApplicationMaster.",
|
||||
Arrays.asList(
|
||||
ContainerExitStatus.KILLED_BY_APPMASTER,
|
||||
ContainerExitStatus.SUCCESS));
|
||||
} catch (YarnException e) {
|
||||
// The exception is possible because, after the container is stopped,
|
||||
// it may be removed from NM's context.
|
||||
if (!e.getMessage()
|
||||
.contains("was recently stopped on node manager")) {
|
||||
throw (AssertionError)
|
||||
(new AssertionError("Exception is not expected: ", e));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait until the container reaches a state N times.
|
||||
* @param container container to watch
|
||||
* @param state state to test
|
||||
* @param transitions the number N above
|
||||
* @throws YarnException This happens if the test times out while waiting
|
||||
*/
|
||||
private void waitForContainerTransitionCount(
|
||||
Container container,
|
||||
org.apache.hadoop.yarn.server.nodemanager.
|
||||
containermanager.container.ContainerState state, long transitions)
|
||||
throws YarnException {
|
||||
long transitionCount = -1;
|
||||
do {
|
||||
if (transitionCount != -1) {
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException e) {
|
||||
throw new YarnException(
|
||||
"Timeout at transition count:" + transitionCount, e);
|
||||
}
|
||||
}
|
||||
transitionCount = DebugSumContainerStateListener
|
||||
.getTransitionCounter(container.getId(), state);
|
||||
} while (transitionCount != transitions);
|
||||
}
|
||||
|
||||
private void sleep(int sleepTime) {
|
||||
try {
|
||||
Thread.sleep(sleepTime);
|
||||
|
@ -1747,7 +1747,7 @@
|
||||
<property>
|
||||
<description>Max time to wait for a process to come up when trying to cleanup a container</description>
|
||||
<name>yarn.nodemanager.process-kill-wait.ms</name>
|
||||
<value>2000</value>
|
||||
<value>5000</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
|
@ -275,7 +275,7 @@ public int launchContainer(ContainerStartContext ctx)
|
||||
sb.writeLocalWrapperScript(launchDst, pidFile);
|
||||
} else {
|
||||
LOG.info("Container " + containerIdStr
|
||||
+ " was marked as inactive. Returning terminated error");
|
||||
+ " pid file not set. Returning terminated error");
|
||||
return ExitCode.TERMINATED.getExitCode();
|
||||
}
|
||||
|
||||
|
@ -734,6 +734,26 @@ public void cleanupContainer() throws IOException {
|
||||
new DelayedProcessKiller(container, user,
|
||||
processId, sleepDelayBeforeSigKill, Signal.KILL, exec).start();
|
||||
}
|
||||
} else {
|
||||
// Normally this means that the process was notified about
|
||||
// deactivateContainer above and did not start.
|
||||
// Since we already set the state to RUNNING or REINITIALIZING
|
||||
// we have to send a killed event to continue.
|
||||
if (!completed.get()) {
|
||||
LOG.warn("Container clean up before pid file created "
|
||||
+ containerIdStr);
|
||||
dispatcher.getEventHandler().handle(
|
||||
new ContainerExitEvent(container.getContainerId(),
|
||||
ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
|
||||
Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
|
||||
ExitCode.TERMINATED.getExitCode(),
|
||||
"Container terminated before pid file created."));
|
||||
// There is a possibility that the launch grabbed the file name before
|
||||
// the deactivateContainer above but it was slow enough to avoid
|
||||
// getContainerPid.
|
||||
// Increasing YarnConfiguration.NM_PROCESS_KILL_WAIT_MS
|
||||
// reduces the likelihood of this race condition and process leak.
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
String message =
|
||||
|
Loading…
x
Reference in New Issue
Block a user