|
@@ -1,20 +1,20 @@
|
|
|
/**
|
|
|
-* 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.
|
|
|
-*/
|
|
|
+ * 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;
|
|
|
|
|
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
@@ -32,13 +33,13 @@ import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.PrintWriter;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Set;
|
|
|
|
|
|
-import org.junit.Assert;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -61,57 +62,88 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Cont
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.util.LCEResourcesHandler;
|
|
|
import org.junit.After;
|
|
|
+import org.junit.Assert;
|
|
|
+import org.junit.Assume;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
/**
|
|
|
- * This is intended to test the LinuxContainerExecutor code, but because of
|
|
|
- * some security restrictions this can only be done with some special setup
|
|
|
- * first.
|
|
|
- * <br><ol>
|
|
|
+ * This is intended to test the LinuxContainerExecutor code, but because of some
|
|
|
+ * security restrictions this can only be done with some special setup first. <br>
|
|
|
+ * <ol>
|
|
|
* <li>Compile the code with container-executor.conf.dir set to the location you
|
|
|
- * want for testing.
|
|
|
- * <br><pre><code>
|
|
|
+ * want for testing. <br>
|
|
|
+ *
|
|
|
+ * <pre>
|
|
|
+ * <code>
|
|
|
* > mvn clean install -Pnative -Dcontainer-executor.conf.dir=/etc/hadoop
|
|
|
* -DskipTests
|
|
|
- * </code></pre>
|
|
|
+ * </code>
|
|
|
+ * </pre>
|
|
|
*
|
|
|
* <li>Set up <code>${container-executor.conf.dir}/container-executor.cfg</code>
|
|
|
* container-executor.cfg needs to be owned by root and have in it the proper
|
|
|
- * config values.
|
|
|
- * <br><pre><code>
|
|
|
+ * config values. <br>
|
|
|
+ *
|
|
|
+ * <pre>
|
|
|
+ * <code>
|
|
|
* > cat /etc/hadoop/container-executor.cfg
|
|
|
* yarn.nodemanager.linux-container-executor.group=mapred
|
|
|
* #depending on the user id of the application.submitter option
|
|
|
* min.user.id=1
|
|
|
* > sudo chown root:root /etc/hadoop/container-executor.cfg
|
|
|
* > sudo chmod 444 /etc/hadoop/container-executor.cfg
|
|
|
- * </code></pre>
|
|
|
+ * </code>
|
|
|
+ * </pre>
|
|
|
*
|
|
|
- * <li>Move the binary and set proper permissions on it. It needs to be owned
|
|
|
- * by root, the group needs to be the group configured in container-executor.cfg,
|
|
|
+ * <li>Move the binary and set proper permissions on it. It needs to be owned by
|
|
|
+ * root, the group needs to be the group configured in container-executor.cfg,
|
|
|
* and it needs the setuid bit set. (The build will also overwrite it so you
|
|
|
- * need to move it to a place that you can support it.
|
|
|
- * <br><pre><code>
|
|
|
+ * need to move it to a place that you can support it. <br>
|
|
|
+ *
|
|
|
+ * <pre>
|
|
|
+ * <code>
|
|
|
* > cp ./hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/c/container-executor/container-executor /tmp/
|
|
|
* > sudo chown root:mapred /tmp/container-executor
|
|
|
- * > sudo chmod 4550 /tmp/container-executor
|
|
|
- * </code></pre>
|
|
|
+ * > sudo chmod 4050 /tmp/container-executor
|
|
|
+ * </code>
|
|
|
+ * </pre>
|
|
|
*
|
|
|
* <li>Run the tests with the execution enabled (The user you run the tests as
|
|
|
- * needs to be part of the group from the config.
|
|
|
- * <br><pre><code>
|
|
|
+ * needs to be part of the group from the config. <br>
|
|
|
+ *
|
|
|
+ * <pre>
|
|
|
+ * <code>
|
|
|
* mvn test -Dtest=TestLinuxContainerExecutor -Dapplication.submitter=nobody -Dcontainer-executor.path=/tmp/container-executor
|
|
|
- * </code></pre>
|
|
|
+ * </code>
|
|
|
+ * </pre>
|
|
|
+ *
|
|
|
+ * <li>The test suite also contains tests to test mounting of CGroups. By
|
|
|
+ * default, these tests are not run. To run them, add -Dcgroups.mount=<mount-point>
|
|
|
+ * Please note that the test does not unmount the CGroups at the end of the test,
|
|
|
+ * since that requires root permissions. <br>
|
|
|
+ *
|
|
|
+ * <li>The tests that are run are sensitive to directory permissions. All parent
|
|
|
+ * directories must be searchable by the user that the tasks are run as. If you
|
|
|
+ * wish to run the tests in a different directory, please set it using
|
|
|
+ * -Dworkspace.dir
|
|
|
+ *
|
|
|
* </ol>
|
|
|
*/
|
|
|
public class TestLinuxContainerExecutor {
|
|
|
private static final Log LOG = LogFactory
|
|
|
- .getLog(TestLinuxContainerExecutor.class);
|
|
|
-
|
|
|
- private static File workSpace = new File("target",
|
|
|
- TestLinuxContainerExecutor.class.getName() + "-workSpace");
|
|
|
-
|
|
|
+ .getLog(TestLinuxContainerExecutor.class);
|
|
|
+
|
|
|
+ private static File workSpace;
|
|
|
+ static {
|
|
|
+ String basedir = System.getProperty("workspace.dir");
|
|
|
+ if(basedir == null || basedir.isEmpty()) {
|
|
|
+ basedir = "target";
|
|
|
+ }
|
|
|
+ workSpace = new File(basedir,
|
|
|
+ TestLinuxContainerExecutor.class.getName() + "-workSpace");
|
|
|
+ }
|
|
|
+
|
|
|
private LinuxContainerExecutor exec = null;
|
|
|
private String appSubmitter = null;
|
|
|
private LocalDirsHandlerService dirsHandler;
|
|
@@ -125,20 +157,26 @@ public class TestLinuxContainerExecutor {
|
|
|
files.mkdir(workSpacePath, null, true);
|
|
|
FileUtil.chmod(workSpace.getAbsolutePath(), "777");
|
|
|
File localDir = new File(workSpace.getAbsoluteFile(), "localDir");
|
|
|
- files.mkdir(new Path(localDir.getAbsolutePath()),
|
|
|
- new FsPermission("777"), false);
|
|
|
+ files.mkdir(new Path(localDir.getAbsolutePath()), new FsPermission("777"),
|
|
|
+ false);
|
|
|
File logDir = new File(workSpace.getAbsoluteFile(), "logDir");
|
|
|
- files.mkdir(new Path(logDir.getAbsolutePath()),
|
|
|
- new FsPermission("777"), false);
|
|
|
+ files.mkdir(new Path(logDir.getAbsolutePath()), new FsPermission("777"),
|
|
|
+ false);
|
|
|
String exec_path = System.getProperty("container-executor.path");
|
|
|
- if(exec_path != null && !exec_path.isEmpty()) {
|
|
|
+ if (exec_path != null && !exec_path.isEmpty()) {
|
|
|
conf = new Configuration(false);
|
|
|
conf.setClass("fs.AbstractFileSystem.file.impl",
|
|
|
org.apache.hadoop.fs.local.LocalFs.class,
|
|
|
org.apache.hadoop.fs.AbstractFileSystem.class);
|
|
|
- conf.set(YarnConfiguration.NM_NONSECURE_MODE_LOCAL_USER_KEY, "xuan");
|
|
|
- LOG.info("Setting "+YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH
|
|
|
- +"="+exec_path);
|
|
|
+
|
|
|
+ appSubmitter = System.getProperty("application.submitter");
|
|
|
+ if (appSubmitter == null || appSubmitter.isEmpty()) {
|
|
|
+ appSubmitter = "nobody";
|
|
|
+ }
|
|
|
+
|
|
|
+ conf.set(YarnConfiguration.NM_NONSECURE_MODE_LOCAL_USER_KEY, appSubmitter);
|
|
|
+ LOG.info("Setting " + YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH
|
|
|
+ + "=" + exec_path);
|
|
|
conf.set(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, exec_path);
|
|
|
exec = new LinuxContainerExecutor();
|
|
|
exec.setConf(conf);
|
|
@@ -146,34 +184,86 @@ public class TestLinuxContainerExecutor {
|
|
|
conf.set(YarnConfiguration.NM_LOG_DIRS, logDir.getAbsolutePath());
|
|
|
dirsHandler = new LocalDirsHandlerService();
|
|
|
dirsHandler.init(conf);
|
|
|
+ List<String> localDirs = dirsHandler.getLocalDirs();
|
|
|
+ for (String dir : localDirs) {
|
|
|
+ Path userDir = new Path(dir, ContainerLocalizer.USERCACHE);
|
|
|
+ files.mkdir(userDir, new FsPermission("777"), false);
|
|
|
+ // $local/filecache
|
|
|
+ Path fileDir = new Path(dir, ContainerLocalizer.FILECACHE);
|
|
|
+ files.mkdir(fileDir, new FsPermission("777"), false);
|
|
|
+ }
|
|
|
}
|
|
|
- appSubmitter = System.getProperty("application.submitter");
|
|
|
- if(appSubmitter == null || appSubmitter.isEmpty()) {
|
|
|
- appSubmitter = "nobody";
|
|
|
- }
|
|
|
+
|
|
|
}
|
|
|
|
|
|
@After
|
|
|
public void tearDown() throws Exception {
|
|
|
FileContext.getLocalFSFileContext().delete(
|
|
|
- new Path(workSpace.getAbsolutePath()), true);
|
|
|
+ new Path(workSpace.getAbsolutePath()), true);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cleanupUserAppCache(String user) throws Exception {
|
|
|
+ List<String> localDirs = dirsHandler.getLocalDirs();
|
|
|
+ for (String dir : localDirs) {
|
|
|
+ Path usercachedir = new Path(dir, ContainerLocalizer.USERCACHE);
|
|
|
+ Path userdir = new Path(usercachedir, user);
|
|
|
+ Path appcachedir = new Path(userdir, ContainerLocalizer.APPCACHE);
|
|
|
+ exec.deleteAsUser(user, appcachedir);
|
|
|
+ FileContext.getLocalFSFileContext().delete(usercachedir, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cleanupUserFileCache(String user) {
|
|
|
+ List<String> localDirs = dirsHandler.getLocalDirs();
|
|
|
+ for (String dir : localDirs) {
|
|
|
+ Path filecache = new Path(dir, ContainerLocalizer.FILECACHE);
|
|
|
+ Path filedir = new Path(filecache, user);
|
|
|
+ exec.deleteAsUser(user, filedir);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cleanupLogDirs(String user) {
|
|
|
+ List<String> logDirs = dirsHandler.getLogDirs();
|
|
|
+ for (String dir : logDirs) {
|
|
|
+ String appId = "APP_" + id;
|
|
|
+ String containerId = "CONTAINER_" + (id - 1);
|
|
|
+ Path appdir = new Path(dir, appId);
|
|
|
+ Path containerdir = new Path(appdir, containerId);
|
|
|
+ exec.deleteAsUser(user, containerdir);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cleanupAppFiles(String user) throws Exception {
|
|
|
+ cleanupUserAppCache(user);
|
|
|
+ cleanupUserFileCache(user);
|
|
|
+ cleanupLogDirs(user);
|
|
|
+
|
|
|
+ String[] files =
|
|
|
+ { "launch_container.sh", "container_tokens", "touch-file" };
|
|
|
+ Path ws = new Path(workSpace.toURI());
|
|
|
+ for (String file : files) {
|
|
|
+ File f = new File(workSpace, file);
|
|
|
+ if (f.exists()) {
|
|
|
+ exec.deleteAsUser(user, new Path(file), ws);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private boolean shouldRun() {
|
|
|
- if(exec == null) {
|
|
|
+ if (exec == null) {
|
|
|
LOG.warn("Not running test because container-executor.path is not set");
|
|
|
return false;
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
|
-
|
|
|
- private String writeScriptFile(String ... cmd) throws IOException {
|
|
|
+
|
|
|
+ private String writeScriptFile(String... cmd) throws IOException {
|
|
|
File f = File.createTempFile("TestLinuxContainerExecutor", ".sh");
|
|
|
f.deleteOnExit();
|
|
|
PrintWriter p = new PrintWriter(new FileOutputStream(f));
|
|
|
p.println("#!/bin/sh");
|
|
|
p.print("exec");
|
|
|
- for(String part: cmd) {
|
|
|
+ for (String part : cmd) {
|
|
|
p.print(" '");
|
|
|
p.print(part.replace("\\", "\\\\").replace("'", "\\'"));
|
|
|
p.print("'");
|
|
@@ -182,36 +272,36 @@ public class TestLinuxContainerExecutor {
|
|
|
p.close();
|
|
|
return f.getAbsolutePath();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private int id = 0;
|
|
|
+
|
|
|
private synchronized int getNextId() {
|
|
|
id += 1;
|
|
|
return id;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private ContainerId getNextContainerId() {
|
|
|
ContainerId cId = mock(ContainerId.class);
|
|
|
- String id = "CONTAINER_"+getNextId();
|
|
|
+ String id = "CONTAINER_" + getNextId();
|
|
|
when(cId.toString()).thenReturn(id);
|
|
|
return cId;
|
|
|
}
|
|
|
-
|
|
|
|
|
|
- private int runAndBlock(String ... cmd) throws IOException {
|
|
|
+ private int runAndBlock(String... cmd) throws IOException {
|
|
|
return runAndBlock(getNextContainerId(), cmd);
|
|
|
}
|
|
|
-
|
|
|
- private int runAndBlock(ContainerId cId, String ... cmd) throws IOException {
|
|
|
- String appId = "APP_"+getNextId();
|
|
|
+
|
|
|
+ private int runAndBlock(ContainerId cId, String... cmd) throws IOException {
|
|
|
+ String appId = "APP_" + getNextId();
|
|
|
Container container = mock(Container.class);
|
|
|
ContainerLaunchContext context = mock(ContainerLaunchContext.class);
|
|
|
- HashMap<String, String> env = new HashMap<String,String>();
|
|
|
+ HashMap<String, String> env = new HashMap<String, String>();
|
|
|
|
|
|
when(container.getContainerId()).thenReturn(cId);
|
|
|
when(container.getLaunchContext()).thenReturn(context);
|
|
|
|
|
|
when(context.getEnvironment()).thenReturn(env);
|
|
|
-
|
|
|
+
|
|
|
String script = writeScriptFile(cmd);
|
|
|
|
|
|
Path scriptPath = new Path(script);
|
|
@@ -221,46 +311,36 @@ public class TestLinuxContainerExecutor {
|
|
|
|
|
|
exec.activateContainer(cId, pidFile);
|
|
|
return exec.launchContainer(container, scriptPath, tokensPath,
|
|
|
- appSubmitter, appId, workDir, dirsHandler.getLocalDirs(),
|
|
|
- dirsHandler.getLogDirs());
|
|
|
+ appSubmitter, appId, workDir, dirsHandler.getLocalDirs(),
|
|
|
+ dirsHandler.getLogDirs());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testContainerLocalizer() throws Exception {
|
|
|
- if (!shouldRun()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- List<String> localDirs = dirsHandler.getLocalDirs();
|
|
|
- List<String> logDirs = dirsHandler.getLogDirs();
|
|
|
- for (String localDir : localDirs) {
|
|
|
- Path userDir =
|
|
|
- new Path(localDir, ContainerLocalizer.USERCACHE);
|
|
|
- files.mkdir(userDir, new FsPermission("777"), false);
|
|
|
- // $local/filecache
|
|
|
- Path fileDir =
|
|
|
- new Path(localDir, ContainerLocalizer.FILECACHE);
|
|
|
- files.mkdir(fileDir, new FsPermission("777"), false);
|
|
|
- }
|
|
|
+
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
+
|
|
|
String locId = "container_01_01";
|
|
|
Path nmPrivateContainerTokensPath =
|
|
|
- dirsHandler.getLocalPathForWrite(
|
|
|
- ResourceLocalizationService.NM_PRIVATE_DIR + Path.SEPARATOR
|
|
|
- + String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT,
|
|
|
- locId));
|
|
|
+ dirsHandler
|
|
|
+ .getLocalPathForWrite(ResourceLocalizationService.NM_PRIVATE_DIR
|
|
|
+ + Path.SEPARATOR
|
|
|
+ + String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId));
|
|
|
files.create(nmPrivateContainerTokensPath, EnumSet.of(CREATE, OVERWRITE));
|
|
|
Configuration config = new YarnConfiguration(conf);
|
|
|
- InetSocketAddress nmAddr = config.getSocketAddr(
|
|
|
- YarnConfiguration.NM_BIND_HOST,
|
|
|
- YarnConfiguration.NM_LOCALIZER_ADDRESS,
|
|
|
- YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS,
|
|
|
- YarnConfiguration.DEFAULT_NM_LOCALIZER_PORT);
|
|
|
+ InetSocketAddress nmAddr =
|
|
|
+ config.getSocketAddr(YarnConfiguration.NM_BIND_HOST,
|
|
|
+ YarnConfiguration.NM_LOCALIZER_ADDRESS,
|
|
|
+ YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS,
|
|
|
+ YarnConfiguration.DEFAULT_NM_LOCALIZER_PORT);
|
|
|
String appId = "application_01_01";
|
|
|
exec = new LinuxContainerExecutor() {
|
|
|
@Override
|
|
|
- public void buildMainArgs(List<String> command, String user, String appId,
|
|
|
- String locId, InetSocketAddress nmAddr, List<String> localDirs) {
|
|
|
- MockContainerLocalizer.buildMainArgs(command, user, appId, locId, nmAddr,
|
|
|
- localDirs);
|
|
|
+ public void buildMainArgs(List<String> command, String user,
|
|
|
+ String appId, String locId, InetSocketAddress nmAddr,
|
|
|
+ List<String> localDirs) {
|
|
|
+ MockContainerLocalizer.buildMainArgs(command, user, appId, locId,
|
|
|
+ nmAddr, localDirs);
|
|
|
}
|
|
|
};
|
|
|
exec.setConf(conf);
|
|
@@ -277,44 +357,68 @@ public class TestLinuxContainerExecutor {
|
|
|
files.create(nmPrivateContainerTokensPath2, EnumSet.of(CREATE, OVERWRITE));
|
|
|
exec.startLocalizer(nmPrivateContainerTokensPath2, nmAddr, appSubmitter,
|
|
|
appId, locId2, dirsHandler);
|
|
|
+ cleanupUserAppCache(appSubmitter);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
- public void testContainerLaunch() throws IOException {
|
|
|
- if (!shouldRun()) {
|
|
|
- return;
|
|
|
- }
|
|
|
+ public void testContainerLaunch() throws Exception {
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
+ String expectedRunAsUser =
|
|
|
+ conf.get(YarnConfiguration.NM_NONSECURE_MODE_LOCAL_USER_KEY,
|
|
|
+ YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER);
|
|
|
+
|
|
|
+ File touchFile = new File(workSpace, "touch-file");
|
|
|
+ int ret = runAndBlock("touch", touchFile.getAbsolutePath());
|
|
|
+
|
|
|
+ assertEquals(0, ret);
|
|
|
+ FileStatus fileStatus =
|
|
|
+ FileContext.getLocalFSFileContext().getFileStatus(
|
|
|
+ new Path(touchFile.getAbsolutePath()));
|
|
|
+ assertEquals(expectedRunAsUser, fileStatus.getOwner());
|
|
|
+ cleanupAppFiles(expectedRunAsUser);
|
|
|
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testNonSecureRunAsSubmitter() throws Exception {
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
+ Assume.assumeFalse(UserGroupInformation.isSecurityEnabled());
|
|
|
+ String expectedRunAsUser = appSubmitter;
|
|
|
+ conf.set(YarnConfiguration.NM_NONSECURE_MODE_LIMIT_USERS, "false");
|
|
|
+ exec.setConf(conf);
|
|
|
File touchFile = new File(workSpace, "touch-file");
|
|
|
int ret = runAndBlock("touch", touchFile.getAbsolutePath());
|
|
|
-
|
|
|
+
|
|
|
assertEquals(0, ret);
|
|
|
- FileStatus fileStatus = FileContext.getLocalFSFileContext().getFileStatus(
|
|
|
+ FileStatus fileStatus =
|
|
|
+ FileContext.getLocalFSFileContext().getFileStatus(
|
|
|
new Path(touchFile.getAbsolutePath()));
|
|
|
- assertEquals(appSubmitter, fileStatus.getOwner());
|
|
|
+ assertEquals(expectedRunAsUser, fileStatus.getOwner());
|
|
|
+ cleanupAppFiles(expectedRunAsUser);
|
|
|
+ // reset conf
|
|
|
+ conf.unset(YarnConfiguration.NM_NONSECURE_MODE_LIMIT_USERS);
|
|
|
+ exec.setConf(conf);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testContainerKill() throws Exception {
|
|
|
- if (!shouldRun()) {
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- final ContainerId sleepId = getNextContainerId();
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
+
|
|
|
+ final ContainerId sleepId = getNextContainerId();
|
|
|
Thread t = new Thread() {
|
|
|
public void run() {
|
|
|
try {
|
|
|
runAndBlock(sleepId, "sleep", "100");
|
|
|
} catch (IOException e) {
|
|
|
- LOG.warn("Caught exception while running sleep",e);
|
|
|
+ LOG.warn("Caught exception while running sleep", e);
|
|
|
}
|
|
|
};
|
|
|
};
|
|
|
- t.setDaemon(true); //If it does not exit we shouldn't block the test.
|
|
|
+ t.setDaemon(true); // If it does not exit we shouldn't block the test.
|
|
|
t.start();
|
|
|
|
|
|
assertTrue(t.isAlive());
|
|
|
-
|
|
|
+
|
|
|
String pid = null;
|
|
|
int count = 10;
|
|
|
while ((pid = exec.getProcessId(sleepId)) == null && count > 0) {
|
|
@@ -328,40 +432,77 @@ public class TestLinuxContainerExecutor {
|
|
|
exec.signalContainer(appSubmitter, pid, Signal.TERM);
|
|
|
LOG.info("sleeping for 100ms to let the sleep be killed");
|
|
|
Thread.sleep(100);
|
|
|
-
|
|
|
+
|
|
|
assertFalse(t.isAlive());
|
|
|
+ cleanupAppFiles(appSubmitter);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCGroups() throws Exception {
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
+ String cgroupsMount = System.getProperty("cgroups.mount");
|
|
|
+ Assume.assumeTrue((cgroupsMount != null) && !cgroupsMount.isEmpty());
|
|
|
+
|
|
|
+ assertTrue("Cgroups mount point does not exist", new File(
|
|
|
+ cgroupsMount).exists());
|
|
|
+ List<String> cgroupKVs = new ArrayList<>();
|
|
|
+
|
|
|
+ String hierarchy = "hadoop-yarn";
|
|
|
+ String[] controllers = { "cpu", "net_cls" };
|
|
|
+ for (String controller : controllers) {
|
|
|
+ cgroupKVs.add(controller + "=" + cgroupsMount + "/" + controller);
|
|
|
+ assertTrue(new File(cgroupsMount, controller).exists());
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ exec.mountCgroups(cgroupKVs, hierarchy);
|
|
|
+ for (String controller : controllers) {
|
|
|
+ assertTrue(controller + " cgroup not mounted", new File(
|
|
|
+ cgroupsMount + "/" + controller + "/tasks").exists());
|
|
|
+ assertTrue(controller + " cgroup hierarchy not created",
|
|
|
+ new File(cgroupsMount + "/" + controller + "/" + hierarchy).exists());
|
|
|
+ assertTrue(controller + " cgroup hierarchy created incorrectly",
|
|
|
+ new File(cgroupsMount + "/" + controller + "/" + hierarchy
|
|
|
+ + "/tasks").exists());
|
|
|
+ }
|
|
|
+ } catch (IOException ie) {
|
|
|
+ fail("Couldn't mount cgroups " + ie.toString());
|
|
|
+ throw ie;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testLocalUser() throws Exception {
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
try {
|
|
|
- //nonsecure default
|
|
|
+ // nonsecure default
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
- "simple");
|
|
|
+ "simple");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
LinuxContainerExecutor lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
|
- Assert.assertEquals(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
|
|
|
+ Assert.assertEquals(
|
|
|
+ YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
|
|
|
lce.getRunAsUser("foo"));
|
|
|
|
|
|
- //nonsecure custom setting
|
|
|
+ // nonsecure custom setting
|
|
|
conf.set(YarnConfiguration.NM_NONSECURE_MODE_LOCAL_USER_KEY, "bar");
|
|
|
lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
|
Assert.assertEquals("bar", lce.getRunAsUser("foo"));
|
|
|
|
|
|
- //nonsecure without limits
|
|
|
+ // nonsecure without limits
|
|
|
conf.set(YarnConfiguration.NM_NONSECURE_MODE_LOCAL_USER_KEY, "bar");
|
|
|
conf.setBoolean(YarnConfiguration.NM_NONSECURE_MODE_LIMIT_USERS, false);
|
|
|
lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
|
Assert.assertEquals("foo", lce.getRunAsUser("foo"));
|
|
|
|
|
|
- //secure
|
|
|
+ // secure
|
|
|
conf = new YarnConfiguration();
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
- "kerberos");
|
|
|
+ "kerberos");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
@@ -369,49 +510,50 @@ public class TestLinuxContainerExecutor {
|
|
|
} finally {
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
- "simple");
|
|
|
+ "simple");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testNonsecureUsernamePattern() throws Exception {
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
try {
|
|
|
- //nonsecure default
|
|
|
+ // nonsecure default
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
- "simple");
|
|
|
+ "simple");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
LinuxContainerExecutor lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
|
lce.verifyUsernamePattern("foo");
|
|
|
try {
|
|
|
lce.verifyUsernamePattern("foo/x");
|
|
|
- Assert.fail();
|
|
|
+ fail();
|
|
|
} catch (IllegalArgumentException ex) {
|
|
|
- //NOP
|
|
|
+ // NOP
|
|
|
} catch (Throwable ex) {
|
|
|
- Assert.fail(ex.toString());
|
|
|
+ fail(ex.toString());
|
|
|
}
|
|
|
-
|
|
|
- //nonsecure custom setting
|
|
|
+
|
|
|
+ // nonsecure custom setting
|
|
|
conf.set(YarnConfiguration.NM_NONSECURE_MODE_USER_PATTERN_KEY, "foo");
|
|
|
lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
|
lce.verifyUsernamePattern("foo");
|
|
|
try {
|
|
|
lce.verifyUsernamePattern("bar");
|
|
|
- Assert.fail();
|
|
|
+ fail();
|
|
|
} catch (IllegalArgumentException ex) {
|
|
|
- //NOP
|
|
|
+ // NOP
|
|
|
} catch (Throwable ex) {
|
|
|
- Assert.fail(ex.toString());
|
|
|
+ fail(ex.toString());
|
|
|
}
|
|
|
|
|
|
- //secure, pattern matching does not kick in.
|
|
|
+ // secure, pattern matching does not kick in.
|
|
|
conf = new YarnConfiguration();
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
- "kerberos");
|
|
|
+ "kerberos");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
@@ -420,13 +562,14 @@ public class TestLinuxContainerExecutor {
|
|
|
} finally {
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
- "simple");
|
|
|
+ "simple");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=10000)
|
|
|
+ @Test(timeout = 10000)
|
|
|
public void testPostExecuteAfterReacquisition() throws Exception {
|
|
|
+ Assume.assumeTrue(shouldRun());
|
|
|
// make up some bogus container ID
|
|
|
ApplicationId appId = ApplicationId.newInstance(12345, 67890);
|
|
|
ApplicationAttemptId attemptId =
|
|
@@ -435,7 +578,7 @@ public class TestLinuxContainerExecutor {
|
|
|
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.setClass(YarnConfiguration.NM_LINUX_CONTAINER_RESOURCES_HANDLER,
|
|
|
- TestResourceHandler.class, LCEResourcesHandler.class);
|
|
|
+ TestResourceHandler.class, LCEResourcesHandler.class);
|
|
|
LinuxContainerExecutor lce = new LinuxContainerExecutor();
|
|
|
lce.setConf(conf);
|
|
|
try {
|
|
@@ -444,7 +587,7 @@ public class TestLinuxContainerExecutor {
|
|
|
// expected if LCE isn't setup right, but not necessary for this test
|
|
|
}
|
|
|
lce.reacquireContainer("foouser", cid);
|
|
|
- Assert.assertTrue("postExec not called after reacquisition",
|
|
|
+ assertTrue("postExec not called after reacquisition",
|
|
|
TestResourceHandler.postExecContainers.contains(cid));
|
|
|
}
|
|
|
|