HADOOP-15537. Clean up ContainerLaunch and ContainerExecutor pre-HADOOP-15528. Contributed by Giovanni Matteo Fumarola.
This commit is contained in:
parent
3e37a9a70b
commit
b272b71095
|
@ -24,6 +24,7 @@ import java.io.OutputStream;
|
|||
import java.io.PrintStream;
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -309,7 +310,8 @@ public abstract class ContainerExecutor implements Configurable {
|
|||
}
|
||||
|
||||
try {
|
||||
return Integer.parseInt(FileUtils.readFileToString(file).trim());
|
||||
return Integer.parseInt(
|
||||
FileUtils.readFileToString(file, Charset.defaultCharset()).trim());
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IOException("Error parsing exit code from pid " + pid, e);
|
||||
}
|
||||
|
|
|
@ -22,8 +22,6 @@ import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|||
import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
|
||||
|
||||
import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
|
@ -189,7 +187,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked") // dispatcher not typed
|
||||
public Integer call() {
|
||||
if (!validateContainerState()) {
|
||||
return 0;
|
||||
|
@ -374,7 +371,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
.build());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected boolean validateContainerState() {
|
||||
// CONTAINER_KILLED_ON_REQUEST should not be missed if the container
|
||||
// is already at KILLING
|
||||
|
@ -486,7 +482,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
return localResources;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected int launchContainer(ContainerStartContext ctx)
|
||||
throws IOException, ConfigurationException {
|
||||
int launchPrep = prepareForLaunch(ctx);
|
||||
|
@ -496,7 +491,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
return launchPrep;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected int relaunchContainer(ContainerStartContext ctx)
|
||||
throws IOException, ConfigurationException {
|
||||
int launchPrep = prepareForLaunch(ctx);
|
||||
|
@ -546,7 +540,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected void handleContainerExitCode(int exitCode, Path containerLogDir) {
|
||||
ContainerId containerId = container.getContainerId();
|
||||
|
||||
|
@ -592,7 +585,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
* @param containerLogDir
|
||||
* @param diagnosticInfo
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
protected void handleContainerExitWithFailure(ContainerId containerID,
|
||||
int ret, Path containerLogDir, StringBuilder diagnosticInfo) {
|
||||
LOG.warn("Container launch failed : " + diagnosticInfo.toString());
|
||||
|
@ -727,7 +719,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
* the process id is available.
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("unchecked") // dispatcher not typed
|
||||
public void cleanupContainer() throws IOException {
|
||||
ContainerId containerId = container.getContainerId();
|
||||
String containerIdStr = containerId.toString();
|
||||
|
@ -855,7 +846,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("unchecked") // dispatcher not typed
|
||||
public void signalContainer(SignalContainerCommand command)
|
||||
throws IOException {
|
||||
ContainerId containerId =
|
||||
|
@ -994,7 +984,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
* executor to pause the container.
|
||||
* @throws IOException in case of errors.
|
||||
*/
|
||||
@SuppressWarnings("unchecked") // dispatcher not typed
|
||||
public void pauseContainer() throws IOException {
|
||||
ContainerId containerId = container.getContainerId();
|
||||
String containerIdStr = containerId.toString();
|
||||
|
@ -1044,7 +1033,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
* executor to pause the container.
|
||||
* @throws IOException in case of error.
|
||||
*/
|
||||
@SuppressWarnings("unchecked") // dispatcher not typed
|
||||
public void resumeContainer() throws IOException {
|
||||
ContainerId containerId = container.getContainerId();
|
||||
String containerIdStr = containerId.toString();
|
||||
|
@ -1357,6 +1345,7 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
}
|
||||
|
||||
private static final class UnixShellScriptBuilder extends ShellScriptBuilder {
|
||||
@SuppressWarnings("unused")
|
||||
private void errorCheck() {
|
||||
line("hadoop_shell_errorcode=$?");
|
||||
line("if [[ \"$hadoop_shell_errorcode\" -ne 0 ]]");
|
||||
|
@ -1651,20 +1640,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|||
}
|
||||
}
|
||||
|
||||
private static void putEnvIfNotNull(
|
||||
Map<String, String> environment, String variable, String value) {
|
||||
if (value != null) {
|
||||
environment.put(variable, value);
|
||||
}
|
||||
}
|
||||
|
||||
private static void putEnvIfAbsent(
|
||||
Map<String, String> environment, String variable) {
|
||||
if (environment.get(variable) == null) {
|
||||
putEnvIfNotNull(environment, variable, System.getenv(variable));
|
||||
}
|
||||
}
|
||||
|
||||
private static void addToEnvMap(
|
||||
Map<String, String> envMap, Set<String> envSet,
|
||||
String envName, String envValue) {
|
||||
|
|
|
@ -58,7 +58,6 @@ public class ContainerRelaunch extends ContainerLaunch {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Integer call() {
|
||||
if (!validateContainerState()) {
|
||||
return 0;
|
||||
|
|
|
@ -776,8 +776,6 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|||
String testKey3 = "MOUNT_LIST";
|
||||
String testVal3 = "/home/a/b/c,/home/d/e/f,/home/g/e/h";
|
||||
conf.set("yarn.nodemanager.admin-env." + testKey3, testVal3);
|
||||
Map<String, String> environment = new HashMap<>();
|
||||
LinkedHashSet<String> nmVars = new LinkedHashSet<>();
|
||||
ContainerLaunch launch = new ContainerLaunch(distContext, conf,
|
||||
dispatcher, exec, null, container, dirsHandler, containerManager);
|
||||
String testDir = System.getProperty("test.build.data",
|
||||
|
|
Loading…
Reference in New Issue