Browse Source

YARN-1972. Added a secure container-executor for Windows. Contributed by Remus Rusanu.

Vinod Kumar Vavilapalli 10 năm trước cách đây
mục cha
commit
ba7f31c2ee
12 tập tin đã thay đổi với 392 bổ sung46 xóa
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  3. 16 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
  4. 41 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
  5. 1 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  6. 171 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
  7. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  8. 25 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
  9. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerExecutor.java
  10. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
  11. 118 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm
  12. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -118,6 +118,9 @@ Release 2.6.0 - UNRELEASED
     YARN-1063. Augmented Hadoop common winutils to have the ability to create
     containers as domain users. (Remus Rusanu via vinodkv)
 
+    YARN-1972. Added a secure container-executor for Windows. (Remus Rusanu via
+    vinodkv)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -928,6 +928,12 @@ public class YarnConfiguration extends Configuration {
 
   public static final long DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT =
       1000;
+  
+  /** 
+  /* The Windows group that the windows-secure-container-executor should run as.
+  */
+  public static final String NM_WINDOWS_SECURE_CONTAINER_GROUP =
+      NM_PREFIX + "windows-secure-container-executor.group";
 
   /** T-file compression types used to compress aggregated logs.*/
   public static final String NM_LOG_AGG_COMPRESSION_TYPE = 

+ 16 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -78,6 +78,20 @@ public abstract class ContainerExecutor implements Configurable {
    */
   public abstract void init() throws IOException;
 
+  /**
+   * On Windows the ContainerLaunch creates a temporary empty jar to workaround the CLASSPATH length
+   * In a  secure cluster this jar must be localized so that the container has access to it
+   * This function localizes on-demand the jar.
+   * 
+   * @param classPathJar
+   * @param owner
+   * @throws IOException
+   */
+  public void localizeClasspathJar(Path classPathJar, String owner) throws IOException {
+    // For the default container this is a no-op
+    // The WindowsSecureContainerExecutor overrides this
+  }
+
   /**
    * Prepare the environment for containers in this application to execute.
    * For $x in local.dirs
@@ -264,8 +278,8 @@ public abstract class ContainerExecutor implements Configurable {
    *  and associate the given groupId in a process group. On
    *  non-Windows, groupId is ignored. 
    */
-  protected static String[] getRunCommand(String command, String groupId,
-                                          Configuration conf) {
+  protected String[] getRunCommand(String command, String groupId,
+      String userName, Path pidFile, Configuration conf) {
     boolean containerSchedPriorityIsSet = false;
     int containerSchedPriorityAdjustment = 
         YarnConfiguration.DEFAULT_NM_CONTAINER_EXECUTOR_SCHED_PRIORITY;
@@ -396,5 +410,4 @@ public abstract class ContainerExecutor implements Configurable {
       }
     }
   }
-
 }

+ 41 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java

@@ -61,7 +61,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
 
   private static final int WIN_MAX_PATH = 260;
 
-  private final FileContext lfs;
+  protected final FileContext lfs;
 
   public DefaultContainerExecutor() {
     try {
@@ -75,11 +75,19 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     this.lfs = lfs;
   }
 
+  protected void copyFile(Path src, Path dst, String owner) throws IOException {
+    lfs.util().copy(src, dst);
+  }
+  
+  protected void setScriptExecutable(Path script, String owner) throws IOException {
+    lfs.setPermission(script, ContainerExecutor.TASK_LAUNCH_SCRIPT_PERMISSION);
+  }
+
   @Override
   public void init() throws IOException {
     // nothing to do or verify here
   }
-  
+
   @Override
   public synchronized void startLocalizer(Path nmPrivateContainerTokensPath,
       InetSocketAddress nmAddr, String user, String appId, String locId,
@@ -93,14 +101,14 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     createUserLocalDirs(localDirs, user);
     createUserCacheDirs(localDirs, user);
     createAppDirs(localDirs, user, appId);
-    createAppLogDirs(appId, logDirs);
+    createAppLogDirs(appId, logDirs, user);
 
     // TODO: Why pick first app dir. The same in LCE why not random?
     Path appStorageDir = getFirstApplicationDir(localDirs, user, appId);
 
     String tokenFn = String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
     Path tokenDst = new Path(appStorageDir, tokenFn);
-    lfs.util().copy(nmPrivateContainerTokensPath, tokenDst);
+    copyFile(nmPrivateContainerTokensPath, tokenDst, user);
     LOG.info("Copying from " + nmPrivateContainerTokensPath + " to " + tokenDst);
     lfs.setWorkingDirectory(appStorageDir);
     LOG.info("CWD set to " + appStorageDir + " = " + lfs.getWorkingDirectory());
@@ -129,30 +137,29 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       Path appCacheDir = new Path(userdir, ContainerLocalizer.APPCACHE);
       Path appDir = new Path(appCacheDir, appIdStr);
       Path containerDir = new Path(appDir, containerIdStr);
-      createDir(containerDir, dirPerm, true);
+      createDir(containerDir, dirPerm, true, userName);
     }
 
     // Create the container log-dirs on all disks
-    createContainerLogDirs(appIdStr, containerIdStr, logDirs);
+    createContainerLogDirs(appIdStr, containerIdStr, logDirs, userName);
 
     Path tmpDir = new Path(containerWorkDir,
         YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
-    createDir(tmpDir, dirPerm, false);
+    createDir(tmpDir, dirPerm, false, userName);
 
     // copy launch script to work dir
     Path launchDst =
         new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
-    lfs.util().copy(nmPrivateContainerScriptPath, launchDst);
+    copyFile(nmPrivateContainerScriptPath, launchDst, userName);
 
     // copy container tokens to work dir
     Path tokenDst =
       new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
-    lfs.util().copy(nmPrivateTokensPath, tokenDst);
+    copyFile(nmPrivateTokensPath, tokenDst, userName);
 
     // Create new local launch wrapper script
-    LocalWrapperScriptBuilder sb = Shell.WINDOWS ?
-      new WindowsLocalWrapperScriptBuilder(containerIdStr, containerWorkDir) :
-      new UnixLocalWrapperScriptBuilder(containerWorkDir);
+    LocalWrapperScriptBuilder sb = getLocalWrapperScriptBuilder(
+        containerIdStr, containerWorkDir); 
 
     // Fail fast if attempting to launch the wrapper script would fail due to
     // Windows path length limitation.
@@ -178,14 +185,12 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     // fork script
     ShellCommandExecutor shExec = null;
     try {
-      lfs.setPermission(launchDst,
-          ContainerExecutor.TASK_LAUNCH_SCRIPT_PERMISSION);
-      lfs.setPermission(sb.getWrapperScriptPath(),
-          ContainerExecutor.TASK_LAUNCH_SCRIPT_PERMISSION);
+      setScriptExecutable(launchDst, userName);
+      setScriptExecutable(sb.getWrapperScriptPath(), userName);
 
       // Setup command to run
       String[] command = getRunCommand(sb.getWrapperScriptPath().toString(),
-        containerIdStr, this.getConf());
+        containerIdStr, userName, pidFile, this.getConf());
 
       LOG.info("launchContainer: " + Arrays.toString(command));
       shExec = new ShellCommandExecutor(
@@ -241,7 +246,14 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     return 0;
   }
 
-  private abstract class LocalWrapperScriptBuilder {
+  protected LocalWrapperScriptBuilder getLocalWrapperScriptBuilder(
+      String containerIdStr, Path containerWorkDir) {
+   return  Shell.WINDOWS ?
+       new WindowsLocalWrapperScriptBuilder(containerIdStr, containerWorkDir) :
+       new UnixLocalWrapperScriptBuilder(containerWorkDir);
+  }
+
+  protected abstract class LocalWrapperScriptBuilder {
 
     private final Path wrapperScriptPath;
 
@@ -449,7 +461,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * $logdir/$user/$appId */
   static final short LOGDIR_PERM = (short)0710;
 
-  private Path getFirstApplicationDir(List<String> localDirs, String user,
+  protected Path getFirstApplicationDir(List<String> localDirs, String user,
       String appId) {
     return getApplicationDir(new Path(localDirs.get(0)), user, appId);
   }
@@ -472,8 +484,8 @@ public class DefaultContainerExecutor extends ContainerExecutor {
         ContainerLocalizer.FILECACHE);
   }
 
-  private void createDir(Path dirPath, FsPermission perms,
-      boolean createParent) throws IOException {
+  protected void createDir(Path dirPath, FsPermission perms,
+      boolean createParent, String user) throws IOException {
     lfs.mkdir(dirPath, perms, createParent);
     if (!perms.equals(perms.applyUMask(lfs.getUMask()))) {
       lfs.setPermission(dirPath, perms);
@@ -493,7 +505,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     for (String localDir : localDirs) {
       // create $local.dir/usercache/$user and its immediate parent
       try {
-        createDir(getUserCacheDir(new Path(localDir), user), userperms, true);
+        createDir(getUserCacheDir(new Path(localDir), user), userperms, true, user);
       } catch (IOException e) {
         LOG.warn("Unable to create the user directory : " + localDir, e);
         continue;
@@ -529,7 +541,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       Path localDirPath = new Path(localDir);
       final Path appDir = getAppcacheDir(localDirPath, user);
       try {
-        createDir(appDir, appCachePerms, true);
+        createDir(appDir, appCachePerms, true, user);
         appcacheDirStatus = true;
       } catch (IOException e) {
         LOG.warn("Unable to create app cache directory : " + appDir, e);
@@ -537,7 +549,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       // create $local.dir/usercache/$user/filecache
       final Path distDir = getFileCacheDir(localDirPath, user);
       try {
-        createDir(distDir, fileperms, true);
+        createDir(distDir, fileperms, true, user);
         distributedCacheDirStatus = true;
       } catch (IOException e) {
         LOG.warn("Unable to create file cache directory : " + distDir, e);
@@ -570,7 +582,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       Path fullAppDir = getApplicationDir(new Path(localDir), user, appId);
       // create $local.dir/usercache/$user/appcache/$appId
       try {
-        createDir(fullAppDir, appperms, true);
+        createDir(fullAppDir, appperms, true, user);
         initAppDirStatus = true;
       } catch (IOException e) {
         LOG.warn("Unable to create app directory " + fullAppDir.toString(), e);
@@ -586,7 +598,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   /**
    * Create application log directories on all disks.
    */
-  void createAppLogDirs(String appId, List<String> logDirs)
+  void createAppLogDirs(String appId, List<String> logDirs, String user)
       throws IOException {
 
     boolean appLogDirStatus = false;
@@ -595,7 +607,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       // create $log.dir/$appid
       Path appLogDir = new Path(rootLogDir, appId);
       try {
-        createDir(appLogDir, appLogDirPerms, true);
+        createDir(appLogDir, appLogDirPerms, true, user);
       } catch (IOException e) {
         LOG.warn("Unable to create the app-log directory : " + appLogDir, e);
         continue;
@@ -612,7 +624,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * Create application log directories on all disks.
    */
   void createContainerLogDirs(String appId, String containerId,
-      List<String> logDirs) throws IOException {
+      List<String> logDirs, String user) throws IOException {
 
     boolean containerLogDirStatus = false;
     FsPermission containerLogDirPerms = new FsPermission(LOGDIR_PERM);
@@ -621,7 +633,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       Path appLogDir = new Path(rootLogDir, appId);
       Path containerLogDir = new Path(appLogDir, containerId);
       try {
-        createDir(containerLogDir, containerLogDirPerms, true);
+        createDir(containerLogDir, containerLogDirPerms, true, user);
       } catch (IOException e) {
         LOG.warn("Unable to create the container-log directory : "
             + appLogDir, e);

+ 1 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -220,15 +220,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     if (javaLibPath != null) {
       command.add("-Djava.library.path=" + javaLibPath);
     }
-    command.add(ContainerLocalizer.class.getName());
-    command.add(user);
-    command.add(appId);
-    command.add(locId);
-    command.add(nmAddr.getHostName());
-    command.add(Integer.toString(nmAddr.getPort()));
-    for (String dir : localDirs) {
-      command.add(dir);
-    }
+    ContainerLocalizer.buildMainArgs(command, user, appId, locId, nmAddr, localDirs);
     String[] commandArray = command.toArray(new String[command.size()]);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
     if (LOG.isDebugEnabled()) {

+ 171 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java

@@ -0,0 +1,171 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.LocalWrapperScriptBuilder;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+
+/**
+ * Windows secure container executor. Uses winutils task createAsUser.
+ *
+ */
+public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
+
+  private static final Log LOG = LogFactory
+      .getLog(WindowsSecureContainerExecutor.class);
+
+  private class WindowsSecureWrapperScriptBuilder 
+    extends LocalWrapperScriptBuilder {
+
+    public WindowsSecureWrapperScriptBuilder(Path containerWorkDir) {
+      super(containerWorkDir);
+    }
+
+    @Override
+    protected void writeLocalWrapperScript(Path launchDst, Path pidFile, PrintStream pout) {
+      pout.format("@call \"%s\"", launchDst);
+    }
+  }
+
+  private String nodeManagerGroup;
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    nodeManagerGroup = conf.get(YarnConfiguration.NM_WINDOWS_SECURE_CONTAINER_GROUP);
+  }
+
+  @Override
+  protected String[] getRunCommand(String command, String groupId,
+      String userName, Path pidFile, Configuration conf) {
+    return new String[] { Shell.WINUTILS, "task", "createAsUser", groupId, userName,
+        pidFile.toString(), "cmd /c " + command };
+  }
+
+  @Override
+  protected LocalWrapperScriptBuilder getLocalWrapperScriptBuilder(
+      String containerIdStr, Path containerWorkDir) {
+   return  new WindowsSecureWrapperScriptBuilder(containerWorkDir);
+  }
+  
+  @Override
+  protected void copyFile(Path src, Path dst, String owner) throws IOException {
+    super.copyFile(src, dst, owner);
+    lfs.setOwner(dst,  owner, nodeManagerGroup);
+  }
+
+  @Override
+  protected void createDir(Path dirPath, FsPermission perms,
+      boolean createParent, String owner) throws IOException {
+    super.createDir(dirPath, perms, createParent, owner);
+    lfs.setOwner(dirPath, owner, nodeManagerGroup);
+  }
+
+  @Override
+  protected void setScriptExecutable(Path script, String owner) throws IOException {
+    super.setScriptExecutable(script, null);
+    lfs.setOwner(script, owner, nodeManagerGroup);
+  }
+
+  @Override
+  public void localizeClasspathJar(Path classpathJar, String owner) throws IOException {
+    lfs.setOwner(classpathJar, owner, nodeManagerGroup);
+  }
+
+ @Override
+ public void startLocalizer(Path nmPrivateContainerTokens,
+     InetSocketAddress nmAddr, String user, String appId, String locId,
+     List<String> localDirs, List<String> logDirs) throws IOException,
+     InterruptedException {
+
+     createUserLocalDirs(localDirs, user);
+     createUserCacheDirs(localDirs, user);
+     createAppDirs(localDirs, user, appId);
+     createAppLogDirs(appId, logDirs, user);
+     
+     // TODO: Why pick first app dir. The same in LCE why not random?
+     Path appStorageDir = getFirstApplicationDir(localDirs, user, appId);
+
+     String tokenFn = String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
+     Path tokenDst = new Path(appStorageDir, tokenFn);
+     LOG.info("Copying from " + nmPrivateContainerTokens + " to " + tokenDst);
+     copyFile(nmPrivateContainerTokens, tokenDst, user);
+
+     List<String> command ;
+     String[] commandArray;
+     ShellCommandExecutor shExec;
+
+     File cwdApp = new File(appStorageDir.toString());
+     LOG.info(String.format("cwdApp: %s", cwdApp));
+
+     command = new ArrayList<String>();
+
+     command.add(Shell.WINUTILS);
+     command.add("task");
+     command.add("createAsUser");
+     command.add("START_LOCALIZER_" + locId);
+     command.add(user);
+     command.add("nul:"); // PID file    
+   
+   //use same jvm as parent
+     File jvm = new File(new File(System.getProperty("java.home"), "bin"), "java.exe");
+     command.add(jvm.toString());
+     
+     
+     // Build a temp classpath jar. See ContainerLaunch.sanitizeEnv().
+     // Passing CLASSPATH explicitly is *way* too long for command line.
+     String classPath = System.getProperty("java.class.path");
+     Map<String, String> env = new HashMap<String, String>(System.getenv());
+     String classPathJar = FileUtil.createJarWithClassPath(classPath, appStorageDir, env);
+     localizeClasspathJar(new Path(classPathJar), user);
+     command.add("-classpath");
+     command.add(classPathJar);
+     
+     String javaLibPath = System.getProperty("java.library.path");
+     if (javaLibPath != null) {
+       command.add("-Djava.library.path=" + javaLibPath);
+     }
+     
+     ContainerLocalizer.buildMainArgs(command, user, appId, locId, nmAddr, localDirs);
+     commandArray = command.toArray(new String[command.size()]);
+
+     shExec = new ShellCommandExecutor(
+         commandArray, cwdApp);
+
+     shExec.execute();
+   }
+}
+

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -766,6 +766,8 @@ public class ContainerLaunch implements Callable<Integer> {
 
         String classPathJar = FileUtil.createJarWithClassPath(
           newClassPath.toString(), pwd, mergedEnv);
+        // In a secure cluster the classpath jar must be localized to grant access
+        this.exec.localizeClasspathJar(new Path(classPathJar), container.getUser());
         environment.put(Environment.CLASSPATH.name(), classPathJar);
       }
     }

+ 25 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java

@@ -312,6 +312,31 @@ public class ContainerLocalizer {
     status.addAllResources(currentResources);
     return status;
   }
+  
+  /**
+   * Adds the ContainerLocalizer arguments for a @{link ShellCommandExecutor},
+   * as expected by ContainerLocalizer.main
+   * @param command the current ShellCommandExecutor command line
+   * @param user localization user
+   * @param appId localized app id
+   * @param locId localizer id
+   * @param nmAddr nodemanager address
+   * @param localDirs list of local dirs
+   */
+  public static void buildMainArgs(List<String> command,
+      String user, String appId, String locId,
+      InetSocketAddress nmAddr, List<String> localDirs) {
+    
+    command.add(ContainerLocalizer.class.getName());
+    command.add(user);
+    command.add(appId);
+    command.add(locId);
+    command.add(nmAddr.getHostName());
+    command.add(Integer.toString(nmAddr.getPort()));
+    for(String dir : localDirs) {
+      command.add(dir);
+    }
+  }
 
   public static void main(String[] argv) throws Throwable {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerExecutor.java

@@ -27,11 +27,13 @@ import org.junit.Test;
 import static org.junit.Assert.*;
 
 public class TestContainerExecutor {
+  
+  private ContainerExecutor containerExecutor = new DefaultContainerExecutor();
 
   @Test (timeout = 5000)
   public void testRunCommandNoPriority() throws Exception {
     Configuration conf = new Configuration();
-    String[] command = ContainerExecutor.getRunCommand("echo", "group1", conf);
+    String[] command = containerExecutor.getRunCommand("echo", "group1", "user", null, conf);
     assertTrue("first command should be the run command for the platform", 
                command[0].equals(Shell.WINUTILS) || command[0].equals("bash"));  
   }
@@ -40,7 +42,7 @@ public class TestContainerExecutor {
   public void testRunCommandwithPriority() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(YarnConfiguration.NM_CONTAINER_EXECUTOR_SCHED_PRIORITY, 2);
-    String[] command = ContainerExecutor.getRunCommand("echo", "group1", conf);
+    String[] command = containerExecutor.getRunCommand("echo", "group1", "user", null, conf);
     if (Shell.WINDOWS) {
       // windows doesn't currently support
       assertEquals("first command should be the run command for the platform", 
@@ -54,7 +56,7 @@ public class TestContainerExecutor {
 
     // test with negative number
     conf.setInt(YarnConfiguration.NM_CONTAINER_EXECUTOR_SCHED_PRIORITY, -5);
-    command = ContainerExecutor.getRunCommand("echo", "group1", conf);
+    command = containerExecutor.getRunCommand("echo", "group1", "user", null, conf);
     if (Shell.WINDOWS) {
       // windows doesn't currently support
       assertEquals("first command should be the run command for the platform", 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java

@@ -199,7 +199,7 @@ public class TestDefaultContainerExecutor {
         Assert.assertEquals(appDirPerm, stats.getPermission());
       }
 
-      executor.createAppLogDirs(appId, logDirs);
+      executor.createAppLogDirs(appId, logDirs, user);
 
       for (String dir : logDirs) {
         FileStatus stats = lfs.getFileStatus(new Path(dir, appId));
@@ -277,7 +277,7 @@ public class TestDefaultContainerExecutor {
       mockExec.createUserLocalDirs(localDirs, appSubmitter);
       mockExec.createUserCacheDirs(localDirs, appSubmitter);
       mockExec.createAppDirs(localDirs, appSubmitter, appId);
-      mockExec.createAppLogDirs(appId, logDirs);
+      mockExec.createAppLogDirs(appId, logDirs, appSubmitter);
 
       Path scriptPath = new Path("file:///bin/echo");
       Path tokensPath = new Path("file:///dev/null");

+ 118 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm

@@ -0,0 +1,118 @@
+~~ Licensed 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. See accompanying LICENSE file.
+
+  ---
+  YARN Secure Containers
+  ---
+  ---
+  ${maven.build.timestamp}
+
+YARN Secure Containers
+
+%{toc|section=1|fromDepth=0|toDepth=3}
+
+* {Overview}
+
+  YARN containers in a secure cluster use the operating system facilities to offer
+  execution isolation for containers. Secure containers execute under the credentials
+  of the job user. The operating system enforces access restriction for the container.
+  The container must run as the use that submitted the application.
+  
+  Secure Containers work only in the context of secured YARN clusters.
+  
+  ** Container isolation requirements
+  
+    The container executor must access the local files and directories needed by the 
+    container such as jars, configuration files, log files, shared objects etc. Although
+    it is launched by the NodeManager, the container should not have access to the 
+    NodeManager private files and configuration. Container running applications 
+    submitted by different users should be isolated and unable to access each other
+    files and directories. Similar requirements apply to other system non-file securable 
+    objects like named pipes, critical sections, LPC queues, shared memory etc.
+    
+    
+  ** Linux Secure Container Executor
+
+    On Linux environment the secure container executor is the <<<LinuxContainerExecutor>>>.
+    It uses an external program called the <<container-executor>>> to launch the container.
+    This program has the <<<setuid>>> access right flag set which allows it to launch 
+    the container with the permissions of the YARN application user.
+    
+  *** Configuration
+
+      The configured directories for <<<yarn.nodemanager.local-dirs>>> and 
+      <<<yarn.nodemanager.log-dirs>>> must be owned by the configured NodeManager user
+      (<<<yarn>>>) and group (<<<hadoop>>>). The permission set on these directories must
+      be <<<drwxr-xr-x>>>.
+      
+      The <<<container-executor>>> program must be owned by <<<root>>> and have the
+      permission set <<<---sr-s--->>>.
+
+      To configure the <<<NodeManager>>> to use the <<<LinuxContainerExecutor>>> set the following 
+      in the <<conf/yarn-site.xml>>:
+
++---+
+<property>
+  <name>yarn.nodemanager.container-executor.class</name>
+  <value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.linux-container-executor.group</name>
+  <value>hadoop</value>
+</property>
++---+
+
+      Additionally the LCE requires the <<<container-executor.cfg>>> file, which is read by the
+      <<<container-executor>>> program. 
+
++---+
+yarn.nodemanager.linux-container-executor.group=#configured value of yarn.nodemanager.linux-container-executor.group
+banned.users=#comma separated list of users who can not run applications
+allowed.system.users=#comma separated list of allowed system users
+min.user.id=1000#Prevent other super-users
++---+
+
+   
+  ** Windows Secure Container Executor
+  
+    The Windows environment secure container executor is the <<<WindowsSecureContainerExecutor>>>.
+    It uses the Windows S4U infrastructure to launch the container as the 
+    YARN application user.
+    
+  *** Configuration
+  
+      To configure the <<<NodeManager>>> to use the <<<WindowsSecureContainerExecutor>>> 
+      set the following in the <<conf/yarn-site.xml>>:
+
++---+
+<property>
+  <name>yarn.nodemanager.container-executor.class</name>
+  <value>org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.windows-secure-container-executor.group</name>
+  <value>hadoop</value>
+</property>
++---+
+
+      The NodeManager must run as a member of the local <<<Administrators>>> group or as 
+      <<<LocalSystem>>>. It is not enough for the NodeManager to simply impersonate such an user.
+      
+  *** Useful Links
+    
+    * {{{http://msdn.microsoft.com/en-us/magazine/cc188757.aspx}Exploring S4U Kerberos Extensions in Windows Server 2003}}
+    
+    * {{{https://issues.apache.org/jira/browse/YARN-1063}Winutils needs ability to create task as domain user}}
+    
+    * {{{https://issues.apache.org/jira/browse/YARN-1972}Implement secure Windows Container Executor}}

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm

@@ -52,6 +52,8 @@ MapReduce NextGen aka YARN aka MRv2
   * {{{./WebApplicationProxy.html}Web Application Proxy}}
 
   * {{{./TimelineServer.html}YARN Timeline Server}}
+  
+  * {{{./SecureContainer.html}YARN Secure Containers}}
 
   * {{{../../hadoop-project-dist/hadoop-common/CLIMiniCluster.html}CLI MiniCluster}}