Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1202940 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2011-11-16 23:46:19 +00:00
commit 61f77396fa
14 changed files with 238 additions and 42 deletions

View File

@ -113,6 +113,9 @@ Release 0.23.1 - Unreleased
OPTIMIZATIONS
BUG FIXES
HADOOP-7811. TestUserGroupInformation#testGetServerSideGroups test fails in chroot.
(Jonathan Eagles via mahadev)
Release 0.23.0 - 2011-11-01

View File

@ -99,7 +99,7 @@ public class TestUserGroupInformation {
(new InputStreamReader(pp.getInputStream()));
String userName = br.readLine().trim();
// get the groups
pp = Runtime.getRuntime().exec("id -Gn");
pp = Runtime.getRuntime().exec("id -Gn " + userName);
br = new BufferedReader(new InputStreamReader(pp.getInputStream()));
String line = br.readLine();
System.out.println(userName + ":" + line);

View File

@ -91,6 +91,9 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3331. Improvement to single node cluster setup documentation for
0.23 (Anupam Seth via mahadev)
MAPREDUCE-3102. Changed NodeManager to fail fast when LinuxContainerExecutor
has wrong configuration or permissions. (Hitesh Shah via vinodkv)
OPTIMIZATIONS
BUG FIXES
@ -132,6 +135,12 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3324. Not All HttpServer tools links (stacks,logs,config,metrics) are
accessible through all UI servers (Jonathan Eagles via mahadev)
MAPREDUCE-3355. Fixed MR AM's ContainerLauncher to handle node-command
timeouts correctly. (vinodkv)
MAPREDUCE-3407. Fixed pom files to refer to the correct MR app-jar needed
by the integration tests. (Hitesh Shah via vinodkv)
Release 0.23.0 - 2011-11-01
INCOMPATIBLE CHANGES

View File

@ -82,6 +82,7 @@ public class ContainerLauncherImpl extends AbstractService implements
private Thread eventHandlingThread;
private BlockingQueue<ContainerLauncherEvent> eventQueue =
new LinkedBlockingQueue<ContainerLauncherEvent>();
final Timer commandTimer = new Timer(true);
YarnRPC rpc;
// To track numNodes.
@ -201,14 +202,14 @@ public class ContainerLauncherImpl extends AbstractService implements
return proxy;
}
private static class CommandTimer extends TimerTask {
private static class CommandTimerTask extends TimerTask {
private final Thread commandThread;
protected final ContainerLauncherEvent event;
protected final String message;
private boolean cancelled = false;
public CommandTimer(Thread thread, ContainerLauncherEvent event) {
public CommandTimerTask(Thread thread, ContainerLauncherEvent event) {
super();
this.commandThread = thread;
this.event = event;
this.message = "Couldn't complete " + event.getType() + " on "
+ event.getContainerID() + "/" + event.getTaskAttemptID()
+ ". Interrupting and returning";
@ -216,8 +217,27 @@ public class ContainerLauncherImpl extends AbstractService implements
@Override
public void run() {
LOG.warn(this.message);
this.commandThread.interrupt();
synchronized (this) {
if (this.cancelled) {
return;
}
LOG.warn(this.message);
StackTraceElement[] trace = this.commandThread.getStackTrace();
StringBuilder logMsg = new StringBuilder();
for (int i = 0; i < trace.length; i++) {
logMsg.append("\n\tat " + trace[i]);
}
LOG.info("Stack trace of the command-thread: \n" + logMsg.toString());
this.commandThread.interrupt();
}
}
@Override
public boolean cancel() {
synchronized (this) {
this.cancelled = true;
return super.cancel();
}
}
}
@ -243,10 +263,11 @@ public class ContainerLauncherImpl extends AbstractService implements
ContainerToken containerToken = event.getContainerToken();
TaskAttemptId taskAttemptID = event.getTaskAttemptID();
Timer timer = new Timer(true);
ContainerManager proxy = null;
CommandTimerTask timerTask = new CommandTimerTask(Thread
.currentThread(), event);
switch(event.getType()) {
case CONTAINER_REMOTE_LAUNCH:
@ -254,16 +275,16 @@ public class ContainerLauncherImpl extends AbstractService implements
= (ContainerRemoteLaunchEvent) event;
try {
timer.schedule(new CommandTimer(Thread.currentThread(), event),
nmTimeOut);
commandTimer.schedule(timerTask, nmTimeOut);
proxy = getCMProxy(containerID, containerManagerBindAddr,
containerToken);
// Interruped during getProxy, but that didn't throw exception
if (Thread.currentThread().isInterrupted()) {
if (Thread.interrupted()) {
// The timer cancelled the command in the mean while.
String message = "Start-container for " + event.getContainerID()
String message = "Container launch failed for " + containerID
+ " : Start-container for " + event.getContainerID()
+ " got interrupted. Returning.";
sendContainerLaunchFailedMsg(taskAttemptID, message);
return;
@ -280,11 +301,12 @@ public class ContainerLauncherImpl extends AbstractService implements
StartContainerResponse response = proxy.startContainer(startRequest);
// container started properly. Stop the timer
timer.cancel();
if (Thread.currentThread().isInterrupted()) {
timerTask.cancel();
if (Thread.interrupted()) {
// The timer cancelled the command in the mean while, but
// startContainer didn't throw exception
String message = "Start-container for " + event.getContainerID()
String message = "Container launch failed for " + containerID
+ " : Start-container for " + event.getContainerID()
+ " got interrupted. Returning.";
sendContainerLaunchFailedMsg(taskAttemptID, message);
return;
@ -309,12 +331,19 @@ public class ContainerLauncherImpl extends AbstractService implements
context.getEventHandler().handle(
new TaskAttemptContainerLaunchedEvent(taskAttemptID, port));
} catch (Throwable t) {
if (Thread.interrupted()) {
// The timer cancelled the command in the mean while.
LOG.info("Start-container for " + event.getContainerID()
+ " got interrupted.");
}
String message = "Container launch failed for " + containerID
+ " : " + StringUtils.stringifyException(t);
sendContainerLaunchFailedMsg(taskAttemptID, message);
} finally {
timer.cancel();
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
timerTask.cancel();
if (proxy != null) {
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
}
}
break;
@ -331,13 +360,12 @@ public class ContainerLauncherImpl extends AbstractService implements
} else {
try {
timer.schedule(new CommandTimer(Thread.currentThread(), event),
nmTimeOut);
commandTimer.schedule(timerTask, nmTimeOut);
proxy = getCMProxy(containerID, containerManagerBindAddr,
containerToken);
if (Thread.currentThread().isInterrupted()) {
if (Thread.interrupted()) {
// The timer cancelled the command in the mean while. No need to
// return, send cleanedup event anyways.
LOG.info("Stop-container for " + event.getContainerID()
@ -353,6 +381,14 @@ public class ContainerLauncherImpl extends AbstractService implements
proxy.stopContainer(stopRequest);
}
} catch (Throwable t) {
if (Thread.interrupted()) {
// The timer cancelled the command in the mean while, clear the
// interrupt flag
LOG.info("Stop-container for " + event.getContainerID()
+ " got interrupted.");
}
// ignore the cleanup failure
String message = "cleanup failed for container "
+ event.getContainerID() + " : "
@ -363,8 +399,18 @@ public class ContainerLauncherImpl extends AbstractService implements
message));
LOG.warn(message);
} finally {
timer.cancel();
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
timerTask.cancel();
if (Thread.interrupted()) {
LOG.info("Stop-container for " + event.getContainerID()
+ " got interrupted.");
// ignore the cleanup failure
context.getEventHandler()
.handle(new TaskAttemptDiagnosticsUpdateEvent(taskAttemptID,
"cleanup failed for container " + event.getContainerID()));
}
if (proxy != null) {
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
}
}
// after killing, send killed event to taskattempt

View File

@ -88,11 +88,19 @@ public class TestContainerLauncher {
app.waitForState(job, JobState.FAILED);
LOG.info("attempt.getDiagnostics: " + attempt.getDiagnostics());
Assert.assertTrue(attempt.getDiagnostics().toString().contains(
"Container launch failed for container_0_0000_01_000000 : "));
Assert.assertTrue(attempt.getDiagnostics().toString().contains(
": java.lang.InterruptedException"));
String diagnostics = attempt.getDiagnostics().toString();
LOG.info("attempt.getDiagnostics: " + diagnostics);
if (swallowInterrupts) {
Assert.assertEquals("[Container launch failed for "
+ "container_0_0000_01_000000 : Start-container for "
+ "container_0_0000_01_000000 got interrupted. Returning.]",
diagnostics);
} else {
Assert.assertTrue(diagnostics.contains("Container launch failed for "
+ "container_0_0000_01_000000 : "));
Assert.assertTrue(diagnostics
.contains(": java.lang.InterruptedException"));
}
app.stop();
}
@ -119,11 +127,10 @@ public class TestContainerLauncher {
}
} catch (InterruptedException e) {
LOG.info(e);
if (!swallowInterrupts) {
if (!MRAppWithSlowNM.this.swallowInterrupts) {
throw new IOException(e);
} else {
Thread.currentThread().interrupt();
}
Thread.currentThread().interrupt();
}
return null;
}

View File

@ -102,8 +102,11 @@
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<yarn.mr.jar>${project.build.directory}/${project.artifactId}-${project.version}.jar</yarn.mr.jar>
<yarn.mr.jar>${project.parent.basedir}/hadoop-mapreduce-client-app/target/hadoop-mapreduce-client-app-${project.version}.jar</yarn.mr.jar>
</systemPropertyVariables>
<environmentVariables>
<JAVA_HOME>${java.home}</JAVA_HOME>
</environmentVariables>
<additionalClasspathElements>
<!-- workaround for JobConf#setJarByClass -->
<additionalClasspathElement>${project.build.directory}/${project.artifactId}-${project.version}-tests.jar</additionalClasspathElement>

View File

@ -112,7 +112,10 @@
<systemPropertyVariables>
<yarn.ds.jar>${project.build.directory}/${project.artifactId}-${project.version}.jar</yarn.ds.jar>
</systemPropertyVariables>
</configuration>
<environmentVariables>
<JAVA_HOME>${java.home}</JAVA_HOME>
</environmentVariables>
</configuration>
</plugin>
</plugins>
</build>

View File

@ -62,6 +62,13 @@ public abstract class ContainerExecutor implements Configurable {
return conf;
}
/**
* Run the executor initialization steps.
* Verify that the necessary configs, permissions are in place.
* @throws IOException
*/
public abstract void init() throws IOException;
/**
* Prepare the environment for containers in this application to execute.
* For $x in local.dirs

View File

@ -69,6 +69,11 @@ public class DefaultContainerExecutor extends ContainerExecutor {
this.lfs = lfs;
}
@Override
public void init() throws IOException {
// nothing to do or verify here
}
@Override
public void startLocalizer(Path nmPrivateContainerTokensPath,
InetSocketAddress nmAddr, String user, String appId, String locId,

View File

@ -100,6 +100,29 @@ public class LinuxContainerExecutor extends ContainerExecutor {
: conf.get(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, defaultPath);
}
@Override
public void init() throws IOException {
// Send command to executor which will just start up,
// verify configuration/permissions and exit
List<String> command = new ArrayList<String>(
Arrays.asList(containerExecutorExe,
"--checksetup"));
String[] commandArray = command.toArray(new String[command.size()]);
ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
if (LOG.isDebugEnabled()) {
LOG.debug("checkLinuxExecutorSetup: " + Arrays.toString(commandArray));
}
try {
shExec.execute();
} catch (ExitCodeException e) {
int exitCode = shExec.getExitCode();
LOG.warn("Exit code from container is : " + exitCode);
logOutput(shExec.getOutput());
throw new IOException("Linux container executor not configured properly"
+ " (error=" + exitCode + ")", e);
}
}
@Override
public void startLocalizer(Path nmPrivateContainerTokensPath,
InetSocketAddress nmAddr, String user, String appId, String locId,

View File

@ -110,6 +110,11 @@ public class NodeManager extends CompositeService implements
ContainerExecutor exec = ReflectionUtils.newInstance(
conf.getClass(YarnConfiguration.NM_CONTAINER_EXECUTOR,
DefaultContainerExecutor.class, ContainerExecutor.class), conf);
try {
exec.init();
} catch (IOException e) {
throw new YarnException("Failed to initialize container executor", e);
}
DeletionService del = new DeletionService(exec);
addService(del);

View File

@ -302,10 +302,12 @@ public class ContainerLaunch implements Callable<Integer> {
// by this time, it will never be launched
exec.deactivateContainer(containerId);
LOG.debug("Getting pid for container " + containerIdStr + " to kill"
+ " from pid file "
+ (pidFilePath != null ? pidFilePath.toString() : "null"));
if (LOG.isDebugEnabled()) {
LOG.debug("Getting pid for container " + containerIdStr + " to kill"
+ " from pid file "
+ (pidFilePath != null ? pidFilePath.toString() : "null"));
}
// however the container process may have already started
try {

View File

@ -38,12 +38,15 @@
#endif
void display_usage(FILE *stream) {
fprintf(stream,
"Usage: container-executor --checksetup\n");
fprintf(stream,
"Usage: container-executor user command command-args\n");
fprintf(stream, "Commands:\n");
fprintf(stream, " initialize container: %2d appid tokens cmd app...\n",
INITIALIZE_CONTAINER);
fprintf(stream, " launch container: %2d appid containerid workdir container-script tokens\n",
fprintf(stream,
" launch container: %2d appid containerid workdir container-script tokens pidfile\n",
LAUNCH_CONTAINER);
fprintf(stream, " signal container: %2d container-pid signal\n",
SIGNAL_CONTAINER);
@ -52,14 +55,31 @@ void display_usage(FILE *stream) {
}
int main(int argc, char **argv) {
//Minimum number of arguments required to run the container-executor
int invalid_args = 0;
int do_check_setup = 0;
LOGFILE = stdout;
ERRORFILE = stderr;
// Minimum number of arguments required to run
// the std. container-executor commands is 4
// 4 args not needed for checksetup option
if (argc < 4) {
invalid_args = 1;
if (argc == 2) {
const char *arg1 = argv[1];
if (strcmp("--checksetup", arg1) == 0) {
invalid_args = 0;
do_check_setup = 1;
}
}
}
if (invalid_args != 0) {
display_usage(stdout);
return INVALID_ARGUMENT_NUMBER;
}
LOGFILE = stdout;
ERRORFILE = stderr;
int command;
const char * app_id = NULL;
const char * container_id = NULL;
@ -111,11 +131,19 @@ int main(int argc, char **argv) {
return INVALID_CONTAINER_EXEC_PERMISSIONS;
}
if (do_check_setup != 0) {
// basic setup checks done
// verified configs available and valid
// verified executor permissions
return 0;
}
//checks done for user name
if (argv[optind] == NULL) {
fprintf(ERRORFILE, "Invalid user name.\n");
return INVALID_USER_NAME;
}
int ret = set_user(argv[optind]);
if (ret != 0) {
return ret;
@ -143,7 +171,7 @@ int main(int argc, char **argv) {
break;
case LAUNCH_CONTAINER:
if (argc < 9) {
fprintf(ERRORFILE, "Too few arguments (%d vs 8) for launch container\n",
fprintf(ERRORFILE, "Too few arguments (%d vs 9) for launch container\n",
argc);
fflush(ERRORFILE);
return INVALID_ARGUMENT_NUMBER;

View File

@ -0,0 +1,55 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.nodemanager;
import static org.junit.Assert.fail;
import java.io.IOException;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.junit.Test;
public class TestNodeManager {
public static final class InvalidContainerExecutor extends
DefaultContainerExecutor {
@Override
public void init() throws IOException {
throw new IOException("dummy executor init called");
}
}
@Test
public void testContainerExecutorInitCall() {
NodeManager nm = new NodeManager();
YarnConfiguration conf = new YarnConfiguration();
conf.setClass(YarnConfiguration.NM_CONTAINER_EXECUTOR,
InvalidContainerExecutor.class,
ContainerExecutor.class);
try {
nm.init(conf);
fail("Init should fail");
} catch (YarnException e) {
//PASS
assert(e.getCause().getMessage().contains("dummy executor init called"));
}
}
}