Ver Fonte

YARN-8448. AM HTTPS Support for AM communication with RMWeb proxy. (Contributed by Robert Kanter)

Haibo Chen há 6 anos atrás
pai
commit
c2288ac45b
37 ficheiros alterados com 2406 adições e 323 exclusões
  1. 1 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  2. 19 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
  3. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
  4. 20 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  5. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  6. 45 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/AMSecretKeys.java
  7. 14 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
  8. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  9. 50 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
  10. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
  11. 12 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java
  12. 23 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
  13. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/LinuxContainerRuntimeConstants.java
  14. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/executor/ContainerStartContext.java
  15. 105 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
  16. 16 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
  17. 29 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
  18. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
  19. 145 83
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
  20. 157 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
  21. 77 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java
  22. 149 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  23. 31 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerRelaunch.java
  24. 91 134
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
  25. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
  26. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
  27. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
  28. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  29. 29 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  30. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/ProxyCAManager.java
  31. 80 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
  32. 51 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestProxyCAManager.java
  33. 408 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyCA.java
  34. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
  35. 74 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
  36. 518 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestProxyCA.java
  37. 54 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -59,6 +59,7 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class Credentials implements Writable {
+
   public enum SerializedFormat {
     WRITABLE((byte) 0x00),
     PROTOBUF((byte) 0x01);

+ 19 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import org.apache.hadoop.test.GenericTestUtils;
 
+import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
@@ -50,6 +51,7 @@ import java.security.NoSuchProviderException;
 import java.security.SignatureException;
 import java.security.cert.CertificateEncodingException;
 import javax.security.auth.x500.X500Principal;
+
 import org.bouncycastle.x509.X509V1CertificateGenerator;
 
 public class KeyStoreTestUtil {
@@ -127,9 +129,16 @@ public class KeyStoreTestUtil {
       String password, String alias,
       Key privateKey, Certificate cert)
       throws GeneralSecurityException, IOException {
-    KeyStore ks = createEmptyKeyStore();
-    ks.setKeyEntry(alias, privateKey, password.toCharArray(),
+    createKeyStore(filename, password, alias, privateKey,
         new Certificate[]{cert});
+  }
+
+  public static void createKeyStore(String filename,
+      String password, String alias,
+      Key privateKey, Certificate[] certs)
+      throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    ks.setKeyEntry(alias, privateKey, password.toCharArray(), certs);
     saveKeyStore(ks, filename, password);
   }
 
@@ -174,6 +183,14 @@ public class KeyStoreTestUtil {
     saveKeyStore(ks, filename, password);
   }
 
+  public static KeyStore bytesToKeyStore(byte[] bytes, String password)
+      throws GeneralSecurityException, IOException {
+    KeyStore keyStore = createEmptyKeyStore();
+    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+    keyStore.load(bais, password.toCharArray());
+    return keyStore;
+  }
+
   public static void cleanupSSLConfig(String keystoresDir, String sslConfDir)
       throws Exception {
     File f = new File(keystoresDir + "/clientKS.jks");

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java

@@ -47,6 +47,28 @@ public interface ApplicationConstants {
   public static final String CONTAINER_TOKEN_FILE_ENV_NAME =
       UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION;
 
+  /**
+   * The file into which the keystore containing the AM's certificate is
+   * written.
+   */
+  String KEYSTORE_FILE_LOCATION_ENV_NAME = "KEYSTORE_FILE_LOCATION";
+
+  /**
+   * The password for the AM's keystore.
+   */
+  String KEYSTORE_PASSWORD_ENV_NAME = "KEYSTORE_PASSWORD";
+
+  /**
+   * The file into which the truststore containing the AM's certificate is
+   * written.
+   */
+  String TRUSTSTORE_FILE_LOCATION_ENV_NAME = "TRUSTSTORE_FILE_LOCATION";
+
+  /**
+   * The password for the AM's truststore.
+   */
+  String TRUSTSTORE_PASSWORD_ENV_NAME = "TRUSTSTORE_PASSWORD";
+
   /**
    * The environmental variable for APPLICATION_WEB_PROXY_BASE. Set in
    * ApplicationMaster's environment only. This states that for all non-relative

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

@@ -2131,6 +2131,26 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_APPLICATION_MONITOR_INTERVAL_MS =
       3000;
 
+  /**
+   * Specifies what the RM does regarding HTTPS enforcement for communication
+   * with AM Web Servers, as well as generating and providing certificates.
+   * Possible values are:
+   * <ul>
+   *   <li>NONE - the RM will do nothing special.</li>
+   *   <li>LENIENT - the RM will generate and provide a keystore and truststore
+   *   to the AM, which it is free to use for HTTPS in its tracking URL web
+   *   server.  The RM proxy will still allow HTTP connections to AMs that opt
+   *   not to use HTTPS.</li>
+   *   <li>STRICT - this is the same as LENIENT, except that the RM proxy will
+   *   only allow HTTPS connections to AMs; HTTP connections will be blocked
+   *   and result in a warning page to the user.</li>
+   * </ul>
+   */
+  public static final String RM_APPLICATION_HTTPS_POLICY =
+      RM_PREFIX + "application-https.policy";
+
+  public static final String DEFAULT_RM_APPLICATION_HTTPS_POLICY = "NONE";
+
   /**
    * Interval of time the linux container executor should try cleaning up
    * cgroups entry when cleaning up a container. This is required due to what 

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -3503,6 +3503,24 @@
     <value>3000</value>
   </property>
 
+  <property>
+    <description>
+      Specifies what the RM does regarding HTTPS enforcement for communication
+      with AM Web Servers, as well as generating and providing certificates.
+      Possible values are:
+      - NONE - the RM will do nothing special.
+      - LENIENT - the RM will generate and provide a keystore and truststore
+      to the AM, which it is free to use for HTTPS in its tracking URL web
+      server.  The RM proxy will still allow HTTP connections to AMs that opt
+      not to use HTTPS.
+      - STRICT - this is the same as LENIENT, except that the RM proxy will
+      only allow HTTPS connections to AMs; HTTP connections will be blocked
+      and result in a warning page to the user.
+    </description>
+    <name>yarn.resourcemanager.application-https.policy</name>
+    <value>NONE</value>
+  </property>
+
   <property>
     <description>
       Defines the limit of the diagnostics message of an application

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/AMSecretKeys.java

@@ -0,0 +1,45 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Constants for AM Secret Keys.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class AMSecretKeys {
+
+  public final static Text YARN_APPLICATION_AM_KEYSTORE =
+      new Text("yarn.application.am.keystore");
+  public final static Text YARN_APPLICATION_AM_KEYSTORE_PASSWORD =
+      new Text("yarn.application.am.keystore.password");
+  public final static Text YARN_APPLICATION_AM_TRUSTSTORE =
+      new Text("yarn.application.am.truststore");
+  public final static Text YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD =
+      new Text("yarn.application.am.truststore.password");
+
+  private AMSecretKeys() {
+    // not used
+  }
+}
+

+ 14 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java

@@ -218,6 +218,8 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     Container container = ctx.getContainer();
     Path nmPrivateContainerScriptPath = ctx.getNmPrivateContainerScriptPath();
     Path nmPrivateTokensPath = ctx.getNmPrivateTokensPath();
+    Path nmPrivateKeystorePath = ctx.getNmPrivateKeystorePath();
+    Path nmPrivateTruststorePath = ctx.getNmPrivateTruststorePath();
     String user = ctx.getUser();
     Path containerWorkDir = ctx.getContainerWorkDir();
     List<String> localDirs = ctx.getLocalDirs();
@@ -253,6 +255,18 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
     copyFile(nmPrivateTokensPath, tokenDst, user);
 
+    if (nmPrivateKeystorePath != null) {
+      Path keystoreDst =
+          new Path(containerWorkDir, ContainerLaunch.KEYSTORE_FILE);
+      copyFile(nmPrivateKeystorePath, keystoreDst, user);
+    }
+
+    if (nmPrivateTruststorePath != null) {
+      Path truststoreDst =
+          new Path(containerWorkDir, ContainerLaunch.TRUSTSTORE_FILE);
+      copyFile(nmPrivateTruststorePath, truststoreDst, user);
+    }
+
     // copy launch script to work dir
     Path launchDst =
         new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -658,6 +658,10 @@ public class LinuxContainerExecutor extends ContainerExecutor {
         ctx.getNmPrivateContainerScriptPath())
       .setExecutionAttribute(NM_PRIVATE_TOKENS_PATH,
         ctx.getNmPrivateTokensPath())
+      .setExecutionAttribute(NM_PRIVATE_KEYSTORE_PATH,
+        ctx.getNmPrivateKeystorePath())
+      .setExecutionAttribute(NM_PRIVATE_TRUSTSTORE_PATH,
+        ctx.getNmPrivateTruststorePath())
       .setExecutionAttribute(PID_FILE_PATH, pidFilePath)
       .setExecutionAttribute(LOCAL_DIRS, ctx.getLocalDirs())
       .setExecutionAttribute(LOG_DIRS, ctx.getLogDirs())

+ 50 - 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

@@ -91,6 +91,7 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerReapContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.util.ProcessIdFileReader;
+import org.apache.hadoop.yarn.server.security.AMSecretKeys;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 
@@ -112,6 +113,9 @@ public class ContainerLaunch implements Callable<Integer> {
 
   public static final String FINAL_CONTAINER_TOKENS_FILE = "container_tokens";
 
+  public static final String KEYSTORE_FILE = "yarn_provided.keystore";
+  public static final String TRUSTSTORE_FILE = "yarn_provided.truststore";
+
   private static final String PID_FILE_NAME_FMT = "%s.pid";
   static final String EXIT_CODE_FILE_SUFFIX = ".exitcode";
 
@@ -232,6 +236,12 @@ public class ContainerLaunch implements Callable<Integer> {
           getContainerPrivateDir(appIdStr, containerIdStr) + Path.SEPARATOR
               + String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT,
               containerIdStr));
+      Path nmPrivateKeystorePath = dirsHandler.getLocalPathForWrite(
+          getContainerPrivateDir(appIdStr, containerIdStr) + Path.SEPARATOR
+              + KEYSTORE_FILE);
+      Path nmPrivateTruststorePath = dirsHandler.getLocalPathForWrite(
+          getContainerPrivateDir(appIdStr, containerIdStr) + Path.SEPARATOR
+              + TRUSTSTORE_FILE);
       Path nmPrivateClasspathJarDir = dirsHandler.getLocalPathForWrite(
           getContainerPrivateDir(appIdStr, containerIdStr));
 
@@ -267,6 +277,44 @@ public class ContainerLaunch implements Callable<Integer> {
         appDirs.add(new Path(appsdir, appIdStr));
       }
 
+      byte[] keystore = container.getCredentials().getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE);
+      if (keystore != null) {
+        try (DataOutputStream keystoreOutStream =
+                 lfs.create(nmPrivateKeystorePath,
+                     EnumSet.of(CREATE, OVERWRITE))) {
+          keystoreOutStream.write(keystore);
+          environment.put(ApplicationConstants.KEYSTORE_FILE_LOCATION_ENV_NAME,
+              new Path(containerWorkDir,
+                  ContainerLaunch.KEYSTORE_FILE).toUri().getPath());
+          environment.put(ApplicationConstants.KEYSTORE_PASSWORD_ENV_NAME,
+              new String(container.getCredentials().getSecretKey(
+                  AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE_PASSWORD),
+                  StandardCharsets.UTF_8));
+        }
+      } else {
+        nmPrivateKeystorePath = null;
+      }
+      byte[] truststore = container.getCredentials().getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE);
+      if (truststore != null) {
+        try (DataOutputStream truststoreOutStream =
+                 lfs.create(nmPrivateTruststorePath,
+                     EnumSet.of(CREATE, OVERWRITE))) {
+          truststoreOutStream.write(truststore);
+          environment.put(
+              ApplicationConstants.TRUSTSTORE_FILE_LOCATION_ENV_NAME,
+              new Path(containerWorkDir,
+                  ContainerLaunch.TRUSTSTORE_FILE).toUri().getPath());
+          environment.put(ApplicationConstants.TRUSTSTORE_PASSWORD_ENV_NAME,
+              new String(container.getCredentials().getSecretKey(
+                  AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD),
+                  StandardCharsets.UTF_8));
+        }
+      } else {
+        nmPrivateTruststorePath = null;
+      }
+
       // Set the token location too.
       addToEnvMap(environment, nmEnvVars,
           ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME,
@@ -304,6 +352,8 @@ public class ContainerLaunch implements Callable<Integer> {
           .setLocalizedResources(localResources)
           .setNmPrivateContainerScriptPath(nmPrivateContainerScriptPath)
           .setNmPrivateTokensPath(nmPrivateTokensPath)
+          .setNmPrivateKeystorePath(nmPrivateKeystorePath)
+          .setNmPrivateTruststorePath(nmPrivateTruststorePath)
           .setUser(user)
           .setAppId(appIdStr)
           .setContainerWorkDir(containerWorkDir)

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

@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
+import org.apache.hadoop.yarn.server.security.AMSecretKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -81,6 +82,12 @@ public class ContainerRelaunch extends ContainerLaunch {
           getNmPrivateContainerScriptPath(appIdStr, containerIdStr);
       Path nmPrivateTokensPath =
           getNmPrivateTokensPath(appIdStr, containerIdStr);
+      Path nmPrivateKeystorePath = (container.getCredentials().getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE) == null) ? null :
+          getNmPrivateKeystorePath(appIdStr, containerIdStr);
+      Path nmPrivateTruststorePath = (container.getCredentials().getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE) == null) ? null :
+          getNmPrivateTruststorePath(appIdStr, containerIdStr);
       pidFilePath = getPidFilePath(appIdStr, containerIdStr);
 
       LOG.info("Relaunch container with "
@@ -112,6 +119,8 @@ public class ContainerRelaunch extends ContainerLaunch {
           .setLocalizedResources(localResources)
           .setNmPrivateContainerScriptPath(nmPrivateContainerScriptPath)
           .setNmPrivateTokensPath(nmPrivateTokensPath)
+          .setNmPrivateKeystorePath(nmPrivateKeystorePath)
+          .setNmPrivateTruststorePath(nmPrivateTruststorePath)
           .setUser(container.getUser())
           .setAppId(appIdStr)
           .setContainerWorkDir(containerWorkDir)
@@ -173,6 +182,20 @@ public class ContainerRelaunch extends ContainerLaunch {
             containerIdStr));
   }
 
+  private Path getNmPrivateKeystorePath(String appIdStr,
+      String containerIdStr) throws IOException {
+    return dirsHandler.getLocalPathForRead(
+        getContainerPrivateDir(appIdStr, containerIdStr) + Path.SEPARATOR
+            + ContainerLaunch.KEYSTORE_FILE);
+  }
+
+  private Path getNmPrivateTruststorePath(String appIdStr,
+      String containerIdStr) throws IOException {
+    return dirsHandler.getLocalPathForRead(
+        getContainerPrivateDir(appIdStr, containerIdStr) + Path.SEPARATOR
+            + ContainerLaunch.TRUSTSTORE_FILE);
+  }
+
   private Path getPidFilePath(String appIdStr,
       String containerIdStr) throws IOException {
     return dirsHandler.getLocalPathForRead(

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java

@@ -23,6 +23,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
@@ -106,8 +107,17 @@ public class DefaultLinuxContainerRuntime implements LinuxContainerRuntime {
         ctx.getExecutionAttribute(CONTAINER_WORK_DIR).toString(),
         ctx.getExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH).toUri()
             .getPath(),
-        ctx.getExecutionAttribute(NM_PRIVATE_TOKENS_PATH).toUri().getPath(),
-        ctx.getExecutionAttribute(PID_FILE_PATH).toString(),
+        ctx.getExecutionAttribute(NM_PRIVATE_TOKENS_PATH).toUri().getPath());
+    Path keystorePath = ctx.getExecutionAttribute(NM_PRIVATE_KEYSTORE_PATH);
+    Path truststorePath = ctx.getExecutionAttribute(NM_PRIVATE_TRUSTSTORE_PATH);
+    if (keystorePath != null && truststorePath != null) {
+      launchOp.appendArgs("--https",
+          keystorePath.toUri().getPath(),
+          truststorePath.toUri().getPath());
+    } else {
+      launchOp.appendArgs("--http");
+    }
+    launchOp.appendArgs(ctx.getExecutionAttribute(PID_FILE_PATH).toString(),
         StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
             ctx.getExecutionAttribute(LOCAL_DIRS)),
         StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,

+ 23 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java

@@ -1205,19 +1205,29 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
             PrivilegedOperation.OperationType.LAUNCH_DOCKER_CONTAINER);
 
     launchOp.appendArgs(runAsUser, ctx.getExecutionAttribute(USER),
-            Integer.toString(PrivilegedOperation
-                    .RunAsUserCommand.LAUNCH_DOCKER_CONTAINER.getValue()),
-            ctx.getExecutionAttribute(APPID),
-            containerIdStr,
-            containerWorkDir.toString(),
-            nmPrivateContainerScriptPath.toUri().getPath(),
-            ctx.getExecutionAttribute(NM_PRIVATE_TOKENS_PATH).toUri().getPath(),
-            ctx.getExecutionAttribute(PID_FILE_PATH).toString(),
-            StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
-                    localDirs),
-            StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
-                    logDirs),
-            commandFile);
+        Integer.toString(PrivilegedOperation
+            .RunAsUserCommand.LAUNCH_DOCKER_CONTAINER.getValue()),
+        ctx.getExecutionAttribute(APPID),
+        containerIdStr,
+        containerWorkDir.toString(),
+        nmPrivateContainerScriptPath.toUri().getPath(),
+        ctx.getExecutionAttribute(NM_PRIVATE_TOKENS_PATH).toUri().getPath());
+    Path keystorePath = ctx.getExecutionAttribute(NM_PRIVATE_KEYSTORE_PATH);
+    Path truststorePath = ctx.getExecutionAttribute(NM_PRIVATE_TRUSTSTORE_PATH);
+    if (keystorePath != null && truststorePath != null) {
+      launchOp.appendArgs("--https",
+          keystorePath.toUri().getPath(),
+          truststorePath.toUri().getPath());
+    } else {
+      launchOp.appendArgs("--http");
+    }
+    launchOp.appendArgs(
+        ctx.getExecutionAttribute(PID_FILE_PATH).toString(),
+        StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
+            localDirs),
+        StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
+            logDirs),
+        commandFile);
 
     String tcCommandFile = ctx.getExecutionAttribute(TC_COMMAND_FILE);
 

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/LinuxContainerRuntimeConstants.java

@@ -58,6 +58,10 @@ public final class LinuxContainerRuntimeConstants {
       Attribute.attribute(Path.class, "nm_private_container_script_path");
   public static final Attribute<Path> NM_PRIVATE_TOKENS_PATH = Attribute
       .attribute(Path.class, "nm_private_tokens_path");
+  public static final Attribute<Path> NM_PRIVATE_KEYSTORE_PATH = Attribute
+      .attribute(Path.class, "nm_private_keystore_path");
+  public static final Attribute<Path> NM_PRIVATE_TRUSTSTORE_PATH = Attribute
+      .attribute(Path.class, "nm_private_truststore_path");
   public static final Attribute<Path> PID_FILE_PATH = Attribute.attribute(
       Path.class, "pid_file_path");
   public static final Attribute<List> LOCAL_DIRS = Attribute.attribute(

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/executor/ContainerStartContext.java

@@ -40,6 +40,8 @@ public final class ContainerStartContext {
   private final Map<Path, List<String>> localizedResources;
   private final Path nmPrivateContainerScriptPath;
   private final Path nmPrivateTokensPath;
+  private final Path nmPrivateKeystorePath;
+  private final Path nmPrivateTruststorePath;
   private final String user;
   private final String appId;
   private final Path containerWorkDir;
@@ -57,6 +59,8 @@ public final class ContainerStartContext {
     private Map<Path, List<String>> localizedResources;
     private Path nmPrivateContainerScriptPath;
     private Path nmPrivateTokensPath;
+    private Path nmPrivateKeystorePath;
+    private Path nmPrivateTruststorePath;
     private String user;
     private String appId;
     private Path containerWorkDir;
@@ -94,6 +98,16 @@ public final class ContainerStartContext {
       return this;
     }
 
+    public Builder setNmPrivateKeystorePath(Path nmPrivateKeystorePath) {
+      this.nmPrivateKeystorePath = nmPrivateKeystorePath;
+      return this;
+    }
+
+    public Builder setNmPrivateTruststorePath(Path nmPrivateTruststorePath) {
+      this.nmPrivateTruststorePath = nmPrivateTruststorePath;
+      return this;
+    }
+
     public Builder setUser(String user) {
       this.user = user;
       return this;
@@ -161,6 +175,8 @@ public final class ContainerStartContext {
     this.localizedResources = builder.localizedResources;
     this.nmPrivateContainerScriptPath = builder.nmPrivateContainerScriptPath;
     this.nmPrivateTokensPath = builder.nmPrivateTokensPath;
+    this.nmPrivateKeystorePath = builder.nmPrivateKeystorePath;
+    this.nmPrivateTruststorePath = builder.nmPrivateTruststorePath;
     this.user = builder.user;
     this.appId = builder.appId;
     this.containerWorkDir = builder.containerWorkDir;
@@ -194,6 +210,14 @@ public final class ContainerStartContext {
     return this.nmPrivateTokensPath;
   }
 
+  public Path getNmPrivateKeystorePath() {
+    return this.nmPrivateKeystorePath;
+  }
+
+  public Path getNmPrivateTruststorePath() {
+    return this.nmPrivateTruststorePath;
+  }
+
   public String getUser() {
     return this.user;
   }

+ 105 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c

@@ -613,6 +613,16 @@ char *get_container_credentials_file(const char* work_dir) {
       CREDENTIALS_FILENAME);
 }
 
+char *get_container_keystore_file(const char* work_dir) {
+  return concatenate("%s/%s", "am container keystore", 2, work_dir,
+      KEYSTORE_FILENAME);
+}
+
+char *get_container_truststore_file(const char* work_dir) {
+  return concatenate("%s/%s", "am container truststore", 2, work_dir,
+      TRUSTSTORE_FILENAME);
+}
+
 /**
  * Get the app log directory under the given log_root
  */
@@ -1565,9 +1575,11 @@ int exec_docker_command(char *docker_command, char **argv, int argc) {
 }
 
 int create_script_paths(const char *work_dir,
-  const char *script_name, const char *cred_file,
-  char** script_file_dest, char** cred_file_dest,
-  int* container_file_source, int* cred_file_source ) {
+  const char *script_name, const char *cred_file, const int https,
+  const char *keystore_file, const char *truststore_file,
+  char** script_file_dest, char** cred_file_dest, char** keystore_file_dest,
+  char** truststore_file_dest, int* container_file_source,
+  int* cred_file_source, int* keystore_file_source, int* truststore_file_source) {
   int exit_code = -1;
 
   *script_file_dest = get_container_launcher_file(work_dir);
@@ -1585,6 +1597,24 @@ int create_script_paths(const char *work_dir,
     fflush(ERRORFILE);
     return exit_code;
   }
+
+  if (https == 1) {
+    *keystore_file_dest = get_container_keystore_file(work_dir);
+    if (NULL == keystore_file_dest) {
+      exit_code = OUT_OF_MEMORY;
+      fprintf(ERRORFILE, "Could not create keystore_file_dest");
+      fflush(ERRORFILE);
+      return exit_code;
+    }
+    *truststore_file_dest = get_container_truststore_file(work_dir);
+    if (NULL == truststore_file_dest) {
+      exit_code = OUT_OF_MEMORY;
+      fprintf(ERRORFILE, "Could not create truststore_file_dest");
+      fflush(ERRORFILE);
+      return exit_code;
+    }
+  }
+
   // open launch script
   *container_file_source = open_file_as_nm(script_name);
   if (*container_file_source == -1) {
@@ -1596,12 +1626,31 @@ int create_script_paths(const char *work_dir,
   // open credentials
   *cred_file_source = open_file_as_nm(cred_file);
   if (*cred_file_source == -1) {
-    exit_code = INVALID_ARGUMENT_NUMBER;
+    exit_code = INVALID_NM_ROOT_DIRS;
     fprintf(ERRORFILE, "Could not open cred file");
     fflush(ERRORFILE);
     return exit_code;
   }
 
+  if (https == 1) {
+    // open keystore
+    *keystore_file_source = open_file_as_nm(keystore_file);
+    if (*keystore_file_source == -1) {
+      exit_code = INVALID_NM_ROOT_DIRS;
+      fprintf(ERRORFILE, "Could not open keystore file");
+      fflush(ERRORFILE);
+      return exit_code;
+    }
+    // open truststore
+    *truststore_file_source = open_file_as_nm(truststore_file);
+    if (*truststore_file_source == -1) {
+      exit_code = INVALID_NM_ROOT_DIRS;
+      fprintf(ERRORFILE, "Could not open truststore file");
+      fflush(ERRORFILE);
+      return exit_code;
+    }
+  }
+
   exit_code = 0;
   return exit_code;
 }
@@ -1609,10 +1658,14 @@ int create_script_paths(const char *work_dir,
 int create_local_dirs(const char * user, const char *app_id,
                        const char *container_id, const char *work_dir,
                        const char *script_name, const char *cred_file,
+                       const int https,
+                       const char *keystore_file, const char *truststore_file,
                        char* const* local_dirs,
                        char* const* log_dirs, int effective_user,
                        char* script_file_dest, char* cred_file_dest,
-                       int container_file_source, int cred_file_source) {
+                       char* keystore_file_dest, char* truststore_file_dest,
+                       int container_file_source, int cred_file_source,
+                       int keystore_file_source, int truststore_file_source) {
   int exit_code = -1;
   // create the user directory on all disks
   int result = initialize_user(user, local_dirs);
@@ -1665,12 +1718,32 @@ int create_local_dirs(const char * user, const char *app_id,
   // Copy credential file to permissions 600
   if (copy_file(cred_file_source, cred_file, cred_file_dest,
         S_IRUSR | S_IWUSR) != 0) {
-    exit_code = COULD_NOT_CREATE_CREDENTIALS_FILE;
+    exit_code = COULD_NOT_CREATE_CREDENTIALS_COPY;
     fprintf(ERRORFILE, "Could not copy file");
     fflush(ERRORFILE);
     goto cleanup;
   }
 
+  if (https == 1) {
+    // Copy keystore file to permissions 600
+    if (copy_file(keystore_file_source, keystore_file, keystore_file_dest,
+          S_IRUSR | S_IWUSR) != 0) {
+      exit_code = COULD_NOT_CREATE_KEYSTORE_COPY;
+      fprintf(ERRORFILE, "Could not copy file");
+      fflush(ERRORFILE);
+      goto cleanup;
+    }
+
+    // Copy truststore file to permissions 600
+    if (copy_file(truststore_file_source, truststore_file, truststore_file_dest,
+          S_IRUSR | S_IWUSR) != 0) {
+      exit_code = COULD_NOT_CREATE_TRUSTSTORE_COPY;
+      fprintf(ERRORFILE, "Could not copy file");
+      fflush(ERRORFILE);
+      goto cleanup;
+    }
+  }
+
   if (chdir(work_dir) != 0) {
     fprintf(ERRORFILE, "Can't change directory to %s -%s\n", work_dir,
       strerror(errno));
@@ -1708,17 +1781,23 @@ int create_user_filecache_dirs(const char * user, char* const* local_dirs) {
 int launch_docker_container_as_user(const char * user, const char *app_id,
                               const char *container_id, const char *work_dir,
                               const char *script_name, const char *cred_file,
+                              const int https,
+                              const char *keystore_file, const char *truststore_file,
                               const char *pid_file, char* const* local_dirs,
                               char* const* log_dirs, const char *command_file) {
   int exit_code = -1;
   char *script_file_dest = NULL;
   char *cred_file_dest = NULL;
+  char *keystore_file_dest = NULL;
+  char *truststore_file_dest = NULL;
   char *exit_code_file = NULL;
   char *docker_command_with_binary = NULL;
   char *docker_inspect_command = NULL;
   char *docker_inspect_exitcode_command = NULL;
   int container_file_source =-1;
   int cred_file_source = -1;
+  int keystore_file_source = -1;
+  int truststore_file_source = -1;
   int use_entry_point = 0;
 
   gid_t user_gid = getegid();
@@ -1729,8 +1808,8 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
 
   fprintf(LOGFILE, "Creating script paths...\n");
   exit_code = create_script_paths(
-    work_dir, script_name, cred_file, &script_file_dest, &cred_file_dest,
-    &container_file_source, &cred_file_source);
+    work_dir, script_name, cred_file, https, keystore_file, truststore_file, &script_file_dest, &cred_file_dest,
+    &keystore_file_dest, &truststore_file_dest, &container_file_source, &cred_file_source, &keystore_file_source, &truststore_file_source);
   if (exit_code != 0) {
     fprintf(ERRORFILE, "Could not create script path\n");
     fflush(ERRORFILE);
@@ -1739,9 +1818,9 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
 
   fprintf(LOGFILE, "Creating local dirs...\n");
   exit_code = create_local_dirs(user, app_id, container_id,
-    work_dir, script_name, cred_file, local_dirs, log_dirs,
-    1, script_file_dest, cred_file_dest,
-    container_file_source, cred_file_source);
+    work_dir, script_name, cred_file, https, keystore_file, truststore_file, local_dirs, log_dirs,
+    1, script_file_dest, cred_file_dest, keystore_file_dest, truststore_file_dest,
+    container_file_source, cred_file_source, keystore_file_source, truststore_file_source);
   if (exit_code != 0) {
     fprintf(ERRORFILE, "Could not create local files and directories %d %d\n", container_file_source, cred_file_source);
     fflush(ERRORFILE);
@@ -1973,6 +2052,8 @@ cleanup:
   free(exit_code_file);
   free(script_file_dest);
   free(cred_file_dest);
+  free(keystore_file_dest);
+  free(truststore_file_dest);
   free(docker_command_with_binary);
   free(docker_inspect_command);
   free_values(docker_command);
@@ -1983,12 +2064,16 @@ cleanup:
 int launch_container_as_user(const char *user, const char *app_id,
                    const char *container_id, const char *work_dir,
                    const char *script_name, const char *cred_file,
+                   const int https,
+                   const char *keystore_file, const char *truststore_file,
                    const char* pid_file, char* const* local_dirs,
                    char* const* log_dirs, const char *resources_key,
                    char* const* resources_values) {
   int exit_code = -1;
   char *script_file_dest = NULL;
   char *cred_file_dest = NULL;
+  char *keystore_file_dest = NULL;
+  char *truststore_file_dest = NULL;
   char *exit_code_file = NULL;
 
   fprintf(LOGFILE, "Getting exit code file...\n");
@@ -2000,11 +2085,13 @@ int launch_container_as_user(const char *user, const char *app_id,
 
   int container_file_source =-1;
   int cred_file_source = -1;
+  int keystore_file_source = -1;
+  int truststore_file_source = -1;
 
   fprintf(LOGFILE, "Creating script paths...\n");
   exit_code = create_script_paths(
-    work_dir, script_name, cred_file, &script_file_dest, &cred_file_dest,
-    &container_file_source, &cred_file_source);
+    work_dir, script_name, cred_file, https, keystore_file, truststore_file, &script_file_dest, &cred_file_dest,
+    &keystore_file_dest, &truststore_file_dest, &container_file_source, &cred_file_source, &keystore_file_source, &truststore_file_source);
   if (exit_code != 0) {
     fprintf(ERRORFILE, "Could not create local files and directories");
     fflush(ERRORFILE);
@@ -2052,9 +2139,9 @@ int launch_container_as_user(const char *user, const char *app_id,
 
   fprintf(LOGFILE, "Creating local dirs...\n");
   exit_code = create_local_dirs(user, app_id, container_id,
-    work_dir, script_name, cred_file, local_dirs, log_dirs,
-    0, script_file_dest, cred_file_dest,
-    container_file_source, cred_file_source);
+    work_dir, script_name, cred_file, https, keystore_file, truststore_file, local_dirs, log_dirs,
+    0, script_file_dest, cred_file_dest, keystore_file_dest, truststore_file_dest,
+    container_file_source, cred_file_source, keystore_file_source, truststore_file_source);
   if (exit_code != 0) {
     fprintf(ERRORFILE, "Could not create local files and directories");
     fflush(ERRORFILE);
@@ -2087,6 +2174,8 @@ int launch_container_as_user(const char *user, const char *app_id,
     free(exit_code_file);
     free(script_file_dest);
     free(cred_file_dest);
+    free(keystore_file_dest);
+    free(truststore_file_dest);
     return exit_code;
 }
 

+ 16 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h

@@ -59,6 +59,8 @@ enum operations {
 #define CONTAINER_DIR_PATTERN NM_APP_DIR_PATTERN "/%s"
 #define CONTAINER_SCRIPT "launch_container.sh"
 #define CREDENTIALS_FILENAME "container_tokens"
+#define KEYSTORE_FILENAME "yarn_provided.keystore"
+#define TRUSTSTORE_FILENAME "yarn_provided.truststore"
 #define MIN_USERID_KEY "min.user.id"
 #define BANNED_USERS_KEY "banned.users"
 #define ALLOWED_SYSTEM_USERS_KEY "allowed.system.users"
@@ -102,6 +104,8 @@ int initialize_app(const char *user, const char *app_id,
 int launch_docker_container_as_user(const char * user, const char *app_id,
                               const char *container_id, const char *work_dir,
                               const char *script_name, const char *cred_file,
+                              const int https,
+                              const char *keystore_file, const char *truststore_file,
                               const char *pid_file, char* const* local_dirs,
                               char* const* log_dirs,
                               const char *command_file);
@@ -118,8 +122,13 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
  * @param container_id the container id
  * @param work_dir the working directory for the container.
  * @param script_name the name of the script to be run to launch the container.
- * @param cred_file the credentials file that needs to be compied to the
+ * @param cred_file the credentials file that needs to be copied to the
  * working directory.
+ * @param https 1 if a keystore and truststore will be provided, 0 if not
+ * @param keystore_file the keystore file that needs to be copied to the
+ * working directory.
+ * @param truststore_file the truststore file that needs to be copied to the
+ * working directory
  * @param pid_file file where pid of process should be written to
  * @param local_dirs nodemanager-local-directories to be used
  * @param log_dirs nodemanager-log-directories to be used
@@ -130,6 +139,8 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
 int launch_container_as_user(const char * user, const char *app_id,
                      const char *container_id, const char *work_dir,
                      const char *script_name, const char *cred_file,
+                     const int https,
+                     const char *keystore_file, const char *truststore_file,
                      const char *pid_file, char* const* local_dirs,
                      char* const* log_dirs, const char *resources_key,
                      char* const* resources_value);
@@ -194,6 +205,10 @@ char *get_container_launcher_file(const char* work_dir);
 
 char *get_container_credentials_file(const char* work_dir);
 
+char *get_container_keystore_file(const char* work_dir);
+
+char *get_container_truststore_file(const char* work_dir);
+
 /**
  * Get the app log directory under log_root
  */

+ 29 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c

@@ -227,6 +227,9 @@ static struct {
   char **resources_values;
   const char *app_id;
   const char *container_id;
+  int https;
+  const char *keystore_file;
+  const char *truststore_file;
   const char *cred_file;
   const char *script_file;
   const char *current_dir;
@@ -432,8 +435,8 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
  case LAUNCH_DOCKER_CONTAINER:
    if(is_docker_support_enabled()) {
       //kill me now.
-      if (!(argc == 13 || argc == 14)) {
-        fprintf(ERRORFILE, "Wrong number of arguments (%d vs 13 or 14) for"
+      if (!(argc >= 14 && argc <= 17)) {
+        fprintf(ERRORFILE, "Wrong number of arguments (%d vs 14 - 17) for"
           " launch docker container\n", argc);
         fflush(ERRORFILE);
         return INVALID_ARGUMENT_NUMBER;
@@ -444,6 +447,13 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
       cmd_input.current_dir = argv[optind++];
       cmd_input.script_file = argv[optind++];
       cmd_input.cred_file = argv[optind++];
+      if (strcmp("--https", argv[optind++]) == 0) {
+        cmd_input.https = 1;
+        cmd_input.keystore_file = argv[optind++];
+        cmd_input.truststore_file = argv[optind++];
+      } else {
+        cmd_input.https = 0;
+      }
       cmd_input.pid_file = argv[optind++];
       // good local dirs as a comma separated list
       cmd_input.local_dirs = argv[optind++];
@@ -451,7 +461,7 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
       cmd_input.log_dirs = argv[optind++];
       cmd_input.docker_command_file = argv[optind++];
       //network isolation through tc
-      if (argc == 14) {
+      if ((argc == 15 && !cmd_input.https) || (argc == 17 && cmd_input.https)) {
         if(is_tc_support_enabled()) {
           cmd_input.traffic_control_command_file = argv[optind++];
         } else {
@@ -469,8 +479,8 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
 
   case LAUNCH_CONTAINER:
     //kill me now.
-    if (!(argc == 13 || argc == 14)) {
-      fprintf(ERRORFILE, "Wrong number of arguments (%d vs 13 or 14)"
+    if (!(argc >= 14 && argc <= 17)) {
+      fprintf(ERRORFILE, "Wrong number of arguments (%d vs 14 - 17)"
         " for launch container\n", argc);
       fflush(ERRORFILE);
       return INVALID_ARGUMENT_NUMBER;
@@ -481,6 +491,13 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
     cmd_input.current_dir = argv[optind++];
     cmd_input.script_file = argv[optind++];
     cmd_input.cred_file = argv[optind++];
+    if (strcmp("--https", argv[optind++]) == 0) {
+      cmd_input.https = 1;
+      cmd_input.keystore_file = argv[optind++];
+      cmd_input.truststore_file = argv[optind++];
+    } else {
+      cmd_input.https = 0;
+    }
     cmd_input.pid_file = argv[optind++];
     cmd_input.local_dirs = argv[optind++];// good local dirs as a comma separated list
     cmd_input.log_dirs = argv[optind++];// good log dirs as a comma separated list
@@ -499,7 +516,7 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
     }
 
     //network isolation through tc
-    if (argc == 14) {
+    if ((argc == 15 && !cmd_input.https) || (argc == 17 && cmd_input.https)) {
       if(is_tc_support_enabled()) {
         cmd_input.traffic_control_command_file = argv[optind++];
       } else {
@@ -636,6 +653,9 @@ int main(int argc, char **argv) {
                       cmd_input.current_dir,
                       cmd_input.script_file,
                       cmd_input.cred_file,
+                      cmd_input.https,
+                      cmd_input.keystore_file,
+                      cmd_input.truststore_file,
                       cmd_input.pid_file,
                       split(cmd_input.local_dirs),
                       split(cmd_input.log_dirs),
@@ -662,6 +682,9 @@ int main(int argc, char **argv) {
                     cmd_input.current_dir,
                     cmd_input.script_file,
                     cmd_input.cred_file,
+                    cmd_input.https,
+                    cmd_input.keystore_file,
+                    cmd_input.truststore_file,
                     cmd_input.pid_file,
                     split(cmd_input.local_dirs),
                     split(cmd_input.log_dirs),

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h

@@ -59,7 +59,7 @@ enum errorcodes {
   ERROR_READING_DOCKER_FILE = 31,
   FEATURE_DISABLED = 32,
   COULD_NOT_CREATE_SCRIPT_COPY = 33,
-  COULD_NOT_CREATE_CREDENTIALS_FILE = 34,
+  COULD_NOT_CREATE_CREDENTIALS_COPY = 34,
   COULD_NOT_CREATE_WORK_DIRECTORIES = 35,
   COULD_NOT_CREATE_APP_LOG_DIRECTORIES = 36,
   COULD_NOT_CREATE_TMP_DIRECTORIES = 37,
@@ -69,7 +69,9 @@ enum errorcodes {
   // DOCKER_CONTAINER_NAME_INVALID = 41, (NOT USED)
   ERROR_COMPILING_REGEX = 42,
   INVALID_CONTAINER_ID = 43,
-  DOCKER_EXEC_FAILED = 44
+  DOCKER_EXEC_FAILED = 44,
+  COULD_NOT_CREATE_KEYSTORE_COPY = 45,
+  COULD_NOT_CREATE_TRUSTSTORE_COPY = 46
 };
 
 /* Macros for min/max. */

+ 145 - 83
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c

@@ -168,13 +168,13 @@ void test_get_user_directory() {
 void test_check_nm_local_dir() {
   // check filesystem is same as running user.
   int expected = 0;
-  char *local_path = TEST_ROOT "target";
+  char *local_path = TEST_ROOT "/target";
   char *root_path = "/";
   if (mkdirs(local_path, 0700) != 0) {
     printf("FAIL: unble to create node manager local directory: %s\n", local_path);
     exit(1);
   }
-  int actual = check_nm_local_dir(nm_uid, local_path);
+  int actual = check_nm_local_dir(user_detail->pw_uid, local_path);
   if (expected != actual) {
     printf("test_nm_local_dir expected %d got %d\n", expected, actual);
     exit(1);
@@ -199,31 +199,76 @@ void test_get_app_directory() {
   free(app_dir);
 }
 
-void test_get_container_directory() {
-  char *container_dir = get_container_work_directory(TEST_ROOT, "owen", "app_1",
+void test_get_container_work_directory() {
+  char *expected_file = TEST_ROOT "/usercache/user/appcache/app_1/container_1";
+  char *work_dir = get_container_work_directory(TEST_ROOT, "user", "app_1",
 						 "container_1");
-  char *expected = TEST_ROOT "/usercache/owen/appcache/app_1/container_1";
-  if (strcmp(container_dir, expected) != 0) {
-    printf("Fail get_container_work_directory got %s expected %s\n",
-	   container_dir, expected);
+  if (strcmp(work_dir, expected_file) != 0) {
+    printf("Fail get_container_work_directory expected %s got %s\n",
+         expected_file, work_dir);
     exit(1);
   }
-  free(container_dir);
+  free(work_dir);
 }
 
 void test_get_container_launcher_file() {
-  char *expected_file = (TEST_ROOT "/usercache/user/appcache/app_200906101234_0001"
-			 "/launch_container.sh");
-  char *app_dir = get_app_directory(TEST_ROOT, "user",
-                                    "app_200906101234_0001");
-  char *container_file =  get_container_launcher_file(app_dir);
-  if (strcmp(container_file, expected_file) != 0) {
-    printf("failure to match expected container file %s vs %s\n", container_file,
-           expected_file);
+  char *expected_file = (TEST_ROOT "/usercache/user/appcache/"
+            "app_200906101234_0001/container_1/launch_container.sh");
+  char *work_dir = get_container_work_directory(TEST_ROOT, "user",
+            "app_200906101234_0001", "container_1");
+  char *launcher_file =  get_container_launcher_file(work_dir);
+  if (strcmp(launcher_file, expected_file) != 0) {
+    printf("failure to match expected launcher file %s got %s\n",
+      expected_file, launcher_file);
+    exit(1);
+  }
+  free(work_dir);
+  free(launcher_file);
+}
+
+void test_get_container_credentials_file() {
+  char *expected_file = (TEST_ROOT "/usercache/user/appcache/"
+            "app_200906101234_0001/container_1/container_tokens");
+  char *work_dir = get_container_work_directory(TEST_ROOT, "user",
+            "app_200906101234_0001", "container_1");
+  char *credentials_file =  get_container_credentials_file(work_dir);
+  if (strcmp(credentials_file, expected_file) != 0) {
+    printf("failure to match expected credentials file %s got %s\n",
+      expected_file, credentials_file);
     exit(1);
   }
-  free(app_dir);
-  free(container_file);
+  free(work_dir);
+  free(credentials_file);
+}
+
+void test_get_container_keystore_file() {
+  char *expected_file = (TEST_ROOT "/usercache/user/appcache/"
+            "app_200906101234_0001/container_1/yarn_provided.keystore");
+  char *work_dir = get_container_work_directory(TEST_ROOT, "user",
+            "app_200906101234_0001", "container_1");
+  char *keystore_file =  get_container_keystore_file(work_dir);
+  if (strcmp(keystore_file, expected_file) != 0) {
+    printf("failure to match expected keystore file %s got %s\n",
+      expected_file, keystore_file);
+    exit(1);
+  }
+  free(work_dir);
+  free(keystore_file);
+}
+
+void test_get_container_truststore_file() {
+  char *expected_file = (TEST_ROOT "/usercache/user/appcache/"
+            "app_200906101234_0001/container_1/yarn_provided.truststore");
+  char *work_dir = get_container_work_directory(TEST_ROOT, "user",
+            "app_200906101234_0001", "container_1");
+  char *truststore_file =  get_container_truststore_file(work_dir);
+  if (strcmp(truststore_file, expected_file) != 0) {
+    printf("failure to match expected truststore file %s got %s\n",
+      expected_file, truststore_file);
+    exit(1);
+  }
+  free(work_dir);
+  free(truststore_file);
 }
 
 void test_get_app_log_dir() {
@@ -762,38 +807,31 @@ void test_signal_container_group() {
   }
 }
 
-void test_init_app() {
-  printf("\nTesting init app\n");
-  if (seteuid(0) != 0) {
-    printf("FAIL: seteuid to root failed - %s\n", strerror(errno));
-    exit(1);
-  }
-  FILE* creds = fopen(TEST_ROOT "/creds.txt", "w");
+void create_text_file(const char* filename, const char* contents) {
+  FILE* creds = fopen(filename, "w");
   if (creds == NULL) {
-    printf("FAIL: failed to create credentials file - %s\n", strerror(errno));
+    printf("FAIL: failed to create %s file - %s\n", filename, strerror(errno));
     exit(1);
   }
-  if (fprintf(creds, "secret key\n") < 0) {
-    printf("FAIL: fprintf failed - %s\n", strerror(errno));
+  if (fwrite(contents, sizeof(char), sizeof(contents), creds)
+        < sizeof(contents)) {
+    printf("FAIL: fwrite failed on file %s- %s\n", filename, strerror(errno));
     exit(1);
   }
   if (fclose(creds) != 0) {
-    printf("FAIL: fclose failed - %s\n", strerror(errno));
-    exit(1);
-  }
-  FILE* job_xml = fopen(TEST_ROOT "/job.xml", "w");
-  if (job_xml == NULL) {
-    printf("FAIL: failed to create job file - %s\n", strerror(errno));
-    exit(1);
-  }
-  if (fprintf(job_xml, "<jobconf/>\n") < 0) {
-    printf("FAIL: fprintf failed - %s\n", strerror(errno));
+    printf("FAIL: fclose failed on file %s - %s\n", filename, strerror(errno));
     exit(1);
   }
-  if (fclose(job_xml) != 0) {
-    printf("FAIL: fclose failed - %s\n", strerror(errno));
+}
+
+void test_init_app() {
+  printf("\nTesting init app\n");
+  if (seteuid(0) != 0) {
+    printf("FAIL: seteuid to root failed - %s\n", strerror(errno));
     exit(1);
   }
+  create_text_file(TEST_ROOT "/creds.txt", "secret key");
+  create_text_file(TEST_ROOT "/job.xml", "<jobconf/>\n");
   if (seteuid(user_detail->pw_uid) != 0) {
     printf("FAIL: failed to seteuid back to user - %s\n", strerror(errno));
     exit(1);
@@ -807,14 +845,9 @@ void test_init_app() {
     exit(1);
   } else if (child == 0) {
     char *final_pgm[] = {"touch", "my-touch-file", 0};
-    if (initialize_app(yarn_username, "app_4", "container_1",
+    exit(initialize_app(yarn_username, "app_4", "container_1",
                        TEST_ROOT "/creds.txt",
-                       local_dirs, log_dirs, final_pgm) != 0) {
-      printf("FAIL: failed in child\n");
-      exit(42);
-    }
-    // should never return
-    exit(1);
+                       local_dirs, log_dirs, final_pgm));
   }
   int status = 0;
   if (waitpid(child, &status, 0) <= 0) {
@@ -822,6 +855,11 @@ void test_init_app() {
 	   strerror(errno));
     exit(1);
   }
+  if (WEXITSTATUS(status) != 0) {
+    printf("FAIL: child %" PRId64 " exited with bad status %d\n",
+           (int64_t)child, WEXITSTATUS(status));
+    exit(1);
+  }
   if (access(TEST_ROOT "/logs/userlogs/app_4", R_OK) != 0) {
     printf("FAIL: failed to create app log directory\n");
     exit(1);
@@ -859,24 +897,24 @@ void test_init_app() {
   free(container_dir);
 }
 
-void test_run_container() {
-  printf("\nTesting run container\n");
+void test_launch_container(const char* app, int https) {
+  if (https == 1) {
+    printf("\nTesting launch container with HTTPS\n");
+  } else {
+    printf("\nTesting launch container without HTTPS\n");
+  }
   if (seteuid(0) != 0) {
     printf("FAIL: seteuid to root failed - %s\n", strerror(errno));
     exit(1);
   }
-  FILE* creds = fopen(TEST_ROOT "/creds.txt", "w");
-  if (creds == NULL) {
-    printf("FAIL: failed to create credentials file - %s\n", strerror(errno));
-    exit(1);
-  }
-  if (fprintf(creds, "secret key\n") < 0) {
-    printf("FAIL: fprintf failed - %s\n", strerror(errno));
-    exit(1);
-  }
-  if (fclose(creds) != 0) {
-    printf("FAIL: fclose failed - %s\n", strerror(errno));
-    exit(1);
+  create_text_file(TEST_ROOT "/creds.txt", "secret key");
+  char* keystore_file = NULL;
+  char* truststore_file = NULL;
+  if (https == 1) {
+    keystore_file = TEST_ROOT "/yarn_provided.keystore";
+    truststore_file = TEST_ROOT "/yarn_provided.truststore";
+    create_text_file(keystore_file, "keystore");
+    create_text_file(truststore_file, "truststore");
   }
 
   char * cgroups_pids[] = { TEST_ROOT "/cgroups-pid1.txt", TEST_ROOT "/cgroups-pid2.txt", 0 };
@@ -906,32 +944,36 @@ void test_run_container() {
   fflush(stdout);
   fflush(stderr);
   char* container_dir = get_container_work_directory(TEST_ROOT "/local-1",
-					      yarn_username, "app_4", "container_1");
+         yarn_username, app, "container_1");
   const char * pid_file = TEST_ROOT "/pid.txt";
 
   pid_t child = fork();
   if (child == -1) {
     printf("FAIL: failed to fork process for init_app - %s\n",
-	   strerror(errno));
+         strerror(errno));
     exit(1);
   } else if (child == 0) {
-    if (launch_container_as_user(yarn_username, "app_4", "container_1",
-          container_dir, script_name, TEST_ROOT "/creds.txt", pid_file,
-          local_dirs, log_dirs,
-          "cgroups", cgroups_pids) != 0) {
-      printf("FAIL: failed in child\n");
-      exit(42);
-    }
-    // should never return
-    exit(1);
+    exit(launch_container_as_user(yarn_username, app, "container_1",
+                        container_dir, script_name, TEST_ROOT "/creds.txt",
+                        https, keystore_file, truststore_file,
+                        pid_file, local_dirs, log_dirs,
+                        "cgroups", cgroups_pids));
   }
   int status = 0;
   if (waitpid(child, &status, 0) <= 0) {
     printf("FAIL: failed waiting for process %" PRId64 " - %s\n", (int64_t)child,
-	   strerror(errno));
+         strerror(errno));
+    exit(1);
+  }
+  if (WEXITSTATUS(status) != 0) {
+    printf("FAIL: child %" PRId64 " exited with bad status %d\n",
+           (int64_t)child, WEXITSTATUS(status));
     exit(1);
   }
-  if (access(TEST_ROOT "/logs/userlogs/app_4/container_1", R_OK) != 0) {
+  char container_log_path[100000];
+  snprintf(container_log_path, sizeof container_log_path, "%s%s%s%s", TEST_ROOT,
+            "/logs/userlogs/", app, "/container_1");
+  if (access(container_log_path, R_OK) != 0) {
     printf("FAIL: failed to create container log directory\n");
     exit(1);
   }
@@ -939,14 +981,17 @@ void test_run_container() {
     printf("FAIL: failed to create container directory %s\n", container_dir);
     exit(1);
   }
-  char buffer[100000];
-  sprintf(buffer, "%s/foobar", container_dir);
-  if (access(buffer, R_OK) != 0) {
-    printf("FAIL: failed to create touch file %s\n", buffer);
+  char touchfile[100000];
+  sprintf(touchfile, "%s/foobar", container_dir);
+  if (access(touchfile, R_OK) != 0) {
+    printf("FAIL: failed to create touch file %s\n", touchfile);
     exit(1);
   }
   free(container_dir);
-  container_dir = get_app_log_directory(TEST_ROOT "/logs/userlogs", "app_4/container_1");
+  char app_log_path[100000];
+  snprintf(app_log_path, sizeof app_log_path, "%s%s%s", TEST_ROOT,
+            "/logs/userlogs/", app);
+  container_dir = get_app_log_directory(app_log_path, "container_1");
   if (access(container_dir, R_OK) != 0) {
     printf("FAIL: failed to create app log directory %s\n", container_dir);
     exit(1);
@@ -1395,6 +1440,13 @@ int main(int argc, char **argv) {
     exit(1);
   }
 
+  if (mkdirs(TEST_ROOT, 0777) != 0) {
+    exit(1);
+  }
+  if (chmod(TEST_ROOT, 0777) != 0) {    // in case of umask
+    exit(1);
+  }
+
   if (mkdirs(TEST_ROOT "/logs/userlogs", 0755) != 0) {
     exit(1);
   }
@@ -1448,12 +1500,21 @@ int main(int argc, char **argv) {
   printf("\nTesting get_app_directory()\n");
   test_get_app_directory();
 
-  printf("\nTesting get_container_directory()\n");
-  test_get_container_directory();
+  printf("\nTesting get_container_work_directory()\n");
+  test_get_container_work_directory();
 
   printf("\nTesting get_container_launcher_file()\n");
   test_get_container_launcher_file();
 
+  printf("\nTesting get_container_credentials_file()\n");
+  test_get_container_credentials_file();
+
+  printf("\nTesting get_container_keystore_file()\n");
+  test_get_container_keystore_file();
+
+  printf("\nTesting get_container_truststore_file()\n");
+  test_get_container_truststore_file();
+
   printf("\nTesting get_app_log_dir()\n");
   test_get_app_log_dir();
 
@@ -1510,7 +1571,8 @@ int main(int argc, char **argv) {
     // these tests do internal forks so that the change_owner and execs
     // don't mess up our process.
     test_init_app();
-    test_run_container();
+    test_launch_container("app_4", 0);
+    test_launch_container("app_5", 1);
   }
 
   /*

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

@@ -47,6 +47,7 @@ import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
@@ -69,6 +70,7 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAc
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.MockLocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
@@ -225,6 +227,157 @@ public class TestDefaultContainerExecutor {
     }
   }
 
+  private void writeStringToRelativePath(FileContext fc, Path p, String str)
+      throws IOException {
+    p = p.makeQualified(fc.getDefaultFileSystem().getUri(),
+        new Path(new File(".").getAbsolutePath()));
+    try (FSDataOutputStream os = fc.create(p).build()) {
+      os.writeUTF(str);
+    }
+  }
+
+  private String readStringFromPath(FileContext fc, Path p) throws IOException {
+    try (FSDataInputStream is = fc.open(p)) {
+      return is.readUTF();
+    }
+  }
+
+  @Test
+  public void testLaunchContainerCopyFilesWithoutHTTPS() throws Exception {
+    testLaunchContainerCopyFiles(false);
+  }
+
+  @Test
+  public void testLaunchContainerCopyFilesWithHTTPS() throws Exception {
+    testLaunchContainerCopyFiles(true);
+  }
+
+  private void testLaunchContainerCopyFiles(boolean https) throws Exception {
+    if (Shell.WINDOWS) {
+      BASE_TMP_PATH =
+          new Path(new File("target").getAbsolutePath(),
+              TestDefaultContainerExecutor.class.getSimpleName());
+    }
+
+    Path localDir = new Path(BASE_TMP_PATH, "localDir");
+    List<String> localDirs = new ArrayList<String>();
+    localDirs.add(localDir.toString());
+    List<String> logDirs = new ArrayList<String>();
+    Path logDir = new Path(BASE_TMP_PATH, "logDir");
+    logDirs.add(logDir.toString());
+
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
+    conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.toString());
+    conf.set(YarnConfiguration.NM_LOG_DIRS, logDir.toString());
+
+    FileContext lfs = FileContext.getLocalFSFileContext(conf);
+    deleteTmpFiles();
+    lfs.mkdir(BASE_TMP_PATH, FsPermission.getDefault(), true);
+    DefaultContainerExecutor dce = new DefaultContainerExecutor(lfs);
+    dce.setConf(conf);
+
+    Container container = mock(Container.class);
+    ContainerId cId = mock(ContainerId.class);
+    ContainerLaunchContext context = mock(ContainerLaunchContext.class);
+    HashMap<String, String> env = new HashMap<String, String>();
+    env.put("LANG", "C");
+
+    String appSubmitter = "nobody";
+    String appId = "APP_ID";
+    String containerId = "CONTAINER_ID";
+
+    when(container.getContainerId()).thenReturn(cId);
+    when(container.getLaunchContext()).thenReturn(context);
+    when(cId.toString()).thenReturn(containerId);
+    when(cId.getApplicationAttemptId()).thenReturn(
+        ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 0));
+    when(context.getEnvironment()).thenReturn(env);
+
+    Path scriptPath = new Path(BASE_TMP_PATH, "script");
+    Path tokensPath = new Path(BASE_TMP_PATH, "tokens");
+    Path keystorePath = new Path(BASE_TMP_PATH, "keystore");
+    Path truststorePath = new Path(BASE_TMP_PATH, "truststore");
+    writeStringToRelativePath(lfs, scriptPath, "script");
+    writeStringToRelativePath(lfs, tokensPath, "tokens");
+    if (https) {
+      writeStringToRelativePath(lfs, keystorePath, "keystore");
+      writeStringToRelativePath(lfs, truststorePath, "truststore");
+    }
+
+    Path workDir = localDir;
+    Path pidFile = new Path(workDir, "pid.txt");
+
+    dce.init(null);
+    dce.activateContainer(cId, pidFile);
+    ContainerStartContext.Builder ctxBuilder =
+        new ContainerStartContext.Builder()
+            .setContainer(container)
+            .setNmPrivateContainerScriptPath(scriptPath)
+            .setNmPrivateTokensPath(tokensPath)
+            .setUser(appSubmitter)
+            .setAppId(appId)
+            .setContainerWorkDir(workDir)
+            .setLocalDirs(localDirs)
+            .setLogDirs(logDirs);
+    if (https) {
+      ctxBuilder.setNmPrivateTruststorePath(truststorePath)
+          .setNmPrivateKeystorePath(keystorePath);
+    }
+    ContainerStartContext ctx = ctxBuilder.build();
+
+    // #launchContainer will copy a number of files to this directory.
+    // Ensure that it doesn't exist first
+    lfs.delete(workDir, true);
+    try {
+      lfs.getFileStatus(workDir);
+      Assert.fail("Expected FileNotFoundException on " + workDir);
+    } catch (FileNotFoundException e) {
+      // expected
+    }
+
+    dce.launchContainer(ctx);
+
+    Path finalScriptPath = new Path(workDir,
+        ContainerLaunch.CONTAINER_SCRIPT);
+    Path finalTokensPath = new Path(workDir,
+        ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
+    Path finalKeystorePath = new Path(workDir,
+        ContainerLaunch.KEYSTORE_FILE);
+    Path finalTrustorePath = new Path(workDir,
+        ContainerLaunch.TRUSTSTORE_FILE);
+
+    Assert.assertTrue(lfs.getFileStatus(workDir).isDirectory());
+    Assert.assertTrue(lfs.getFileStatus(finalScriptPath).isFile());
+    Assert.assertTrue(lfs.getFileStatus(finalTokensPath).isFile());
+    if (https) {
+      Assert.assertTrue(lfs.getFileStatus(finalKeystorePath).isFile());
+      Assert.assertTrue(lfs.getFileStatus(finalTrustorePath).isFile());
+    } else {
+      try {
+        lfs.getFileStatus(finalKeystorePath);
+        Assert.fail("Expected FileNotFoundException on " + finalKeystorePath);
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+      try {
+        lfs.getFileStatus(finalTrustorePath);
+        Assert.fail("Expected FileNotFoundException on " + finalKeystorePath);
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+
+    Assert.assertEquals("script", readStringFromPath(lfs, finalScriptPath));
+    Assert.assertEquals("tokens", readStringFromPath(lfs, finalTokensPath));
+    if (https) {
+      Assert.assertEquals("keystore", readStringFromPath(lfs,
+          finalKeystorePath));
+      Assert.assertEquals("truststore", readStringFromPath(lfs,
+          finalTrustorePath));
+    }
+  }
+
   @Test
   public void testContainerLaunchError()
       throws IOException, InterruptedException, ConfigurationException {
@@ -303,6 +456,8 @@ public class TestDefaultContainerExecutor {
 
       Path scriptPath = new Path("file:///bin/echo");
       Path tokensPath = new Path("file:///dev/null");
+      Path keystorePath = new Path("file:///dev/null");
+      Path truststorePath = new Path("file:///dev/null");
       if (Shell.WINDOWS) {
         File tmp = new File(BASE_TMP_PATH.toString(), "test_echo.cmd");
         BufferedWriter output = new BufferedWriter(new FileWriter(tmp));
@@ -323,6 +478,8 @@ public class TestDefaultContainerExecutor {
           .setContainer(container)
           .setNmPrivateContainerScriptPath(scriptPath)
           .setNmPrivateTokensPath(tokensPath)
+          .setNmPrivateKeystorePath(keystorePath)
+          .setNmPrivateTruststorePath(truststorePath)
           .setUser(appSubmitter)
           .setAppId(appId)
           .setContainerWorkDir(workDir)

+ 77 - 38
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java

@@ -37,6 +37,8 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.LineNumberReader;
 import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.ArrayList;
@@ -88,12 +90,11 @@ public class TestLinuxContainerExecutorWithMocks {
   private static final Logger LOG =
        LoggerFactory.getLogger(TestLinuxContainerExecutorWithMocks.class);
 
-  private static final String MOCK_EXECUTOR =
-      "./src/test/resources/mock-container-executor";
+  private static final String MOCK_EXECUTOR = "mock-container-executor";
   private static final String MOCK_EXECUTOR_WITH_ERROR =
-      "./src/test/resources/mock-container-executer-with-error";
+      "mock-container-executer-with-error";
   private static final String MOCK_EXECUTOR_WITH_CONFIG_ERROR =
-      "./src/test/resources/mock-container-executer-with-configuration-error";
+      "mock-container-executer-with-configuration-error";
 
   private String tmpMockExecutor;
   private LinuxContainerExecutor mockExec = null;
@@ -121,11 +122,13 @@ public class TestLinuxContainerExecutorWithMocks {
     return ret;
   }
 
-  private void setupMockExecutor(String executorPath, Configuration conf)
-      throws IOException {
+  private void setupMockExecutor(String executorName, Configuration conf)
+      throws IOException, URISyntaxException {
     //we'll always use the tmpMockExecutor - since
     // PrivilegedOperationExecutor can only be initialized once.
 
+    URI executorPath = getClass().getClassLoader().getResource(executorName)
+        .toURI();
     Files.copy(Paths.get(executorPath), Paths.get(tmpMockExecutor),
         REPLACE_EXISTING);
 
@@ -140,7 +143,8 @@ public class TestLinuxContainerExecutorWithMocks {
   }
 
   @Before
-  public void setup() throws IOException, ContainerExecutionException {
+  public void setup() throws IOException, ContainerExecutionException,
+      URISyntaxException {
     assumeNotWindows();
 
     tmpMockExecutor = System.getProperty("test.build.data") +
@@ -172,7 +176,18 @@ public class TestLinuxContainerExecutorWithMocks {
   }
 
   @Test
-  public void testContainerLaunch()
+  public void testContainerLaunchWithoutHTTPS()
+      throws IOException, ConfigurationException {
+    testContainerLaunch(false);
+  }
+
+  @Test
+  public void testContainerLaunchWithHTTPS()
+      throws IOException, ConfigurationException {
+    testContainerLaunch(true);
+  }
+
+  private void testContainerLaunch(boolean https)
       throws IOException, ConfigurationException {
     String appSubmitter = "nobody";
     String cmd = String.valueOf(
@@ -193,41 +208,64 @@ public class TestLinuxContainerExecutorWithMocks {
     
     Path scriptPath = new Path("file:///bin/echo");
     Path tokensPath = new Path("file:///dev/null");
+    Path keystorePath = new Path("file:///dev/null");
+    Path truststorePath = new Path("file:///dev/null");
     Path workDir = new Path("/tmp");
     Path pidFile = new Path(workDir, "pid.txt");
 
     mockExec.activateContainer(cId, pidFile);
-    int ret = mockExec.launchContainer(new ContainerStartContext.Builder()
-        .setContainer(container)
-        .setNmPrivateContainerScriptPath(scriptPath)
-        .setNmPrivateTokensPath(tokensPath)
-        .setUser(appSubmitter)
-        .setAppId(appId)
-        .setContainerWorkDir(workDir)
-        .setLocalDirs(dirsHandler.getLocalDirs())
-        .setLogDirs(dirsHandler.getLogDirs())
-        .setFilecacheDirs(new ArrayList<>())
-        .setUserLocalDirs(new ArrayList<>())
-        .setContainerLocalDirs(new ArrayList<>())
-        .setContainerLogDirs(new ArrayList<>())
-        .setUserFilecacheDirs(new ArrayList<>())
-        .setApplicationLocalDirs(new ArrayList<>())
-        .build());
+    ContainerStartContext.Builder ctxBuilder =
+        new ContainerStartContext.Builder()
+            .setContainer(container)
+            .setNmPrivateContainerScriptPath(scriptPath)
+            .setNmPrivateTokensPath(tokensPath)
+            .setUser(appSubmitter)
+            .setAppId(appId)
+            .setContainerWorkDir(workDir)
+            .setLocalDirs(dirsHandler.getLocalDirs())
+            .setLogDirs(dirsHandler.getLogDirs())
+            .setFilecacheDirs(new ArrayList<>())
+            .setUserLocalDirs(new ArrayList<>())
+            .setContainerLocalDirs(new ArrayList<>())
+            .setContainerLogDirs(new ArrayList<>())
+            .setUserFilecacheDirs(new ArrayList<>())
+            .setApplicationLocalDirs(new ArrayList<>());
+    if (https) {
+      ctxBuilder.setNmPrivateKeystorePath(keystorePath);
+      ctxBuilder.setNmPrivateTruststorePath(truststorePath);
+    }
+    int ret = mockExec.launchContainer(ctxBuilder.build());
     assertEquals(0, ret);
-    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
-        appSubmitter, cmd, appId, containerId,
-        workDir.toString(), "/bin/echo", "/dev/null", pidFile.toString(),
-        StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
-            dirsHandler.getLocalDirs()),
-        StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
-            dirsHandler.getLogDirs()), "cgroups=none"),
-        readMockParams());
-    
+    if (https) {
+      assertEquals(Arrays.asList(
+          YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+          appSubmitter, cmd, appId, containerId,
+          workDir.toString(), scriptPath.toUri().getPath(),
+          tokensPath.toUri().getPath(), "--https",
+          keystorePath.toUri().getPath(), truststorePath.toUri().getPath(),
+          pidFile.toString(),
+          StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
+              dirsHandler.getLocalDirs()),
+          StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
+              dirsHandler.getLogDirs()), "cgroups=none"),
+          readMockParams());
+    } else {
+      assertEquals(Arrays.asList(
+          YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+          appSubmitter, cmd, appId, containerId,
+          workDir.toString(), scriptPath.toUri().getPath(),
+          tokensPath.toUri().getPath(), "--http", pidFile.toString(),
+          StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
+              dirsHandler.getLocalDirs()),
+          StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
+              dirsHandler.getLogDirs()), "cgroups=none"),
+          readMockParams());
+    }
   }
 
   @Test (timeout = 5000)
   public void testContainerLaunchWithPriority()
-      throws IOException, ConfigurationException {
+      throws IOException, ConfigurationException, URISyntaxException {
 
     // set the scheduler priority to make sure still works with nice -n prio
     Configuration conf = new Configuration();
@@ -242,7 +280,7 @@ public class TestLinuxContainerExecutorWithMocks {
     assertEquals("third should be the priority", Integer.toString(2),
                  command.get(2));
 
-    testContainerLaunch();
+    testContainerLaunchWithoutHTTPS();
   }
 
   @Test (timeout = 5000)
@@ -306,7 +344,7 @@ public class TestLinuxContainerExecutorWithMocks {
   
   @Test
   public void testContainerLaunchError()
-      throws IOException, ContainerExecutionException {
+      throws IOException, ContainerExecutionException, URISyntaxException {
 
     final String[] expecetedMessage = {"badcommand", "Exit code: 24"};
     final String[] executor = {
@@ -410,7 +448,8 @@ public class TestLinuxContainerExecutorWithMocks {
         assertEquals(Arrays.asList(YarnConfiguration.
                 DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
             appSubmitter, cmd, appId, containerId,
-            workDir.toString(), "/bin/echo", "/dev/null", pidFile.toString(),
+            workDir.toString(), "/bin/echo", "/dev/null", "--http",
+            pidFile.toString(),
             StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
                 dirsHandler.getLocalDirs()),
             StringUtils.join(PrivilegedOperation.LINUX_FILE_PATH_SEPARATOR,
@@ -462,7 +501,7 @@ public class TestLinuxContainerExecutorWithMocks {
   }
   
   @Test
-  public void testDeleteAsUser() throws IOException {
+  public void testDeleteAsUser() throws IOException, URISyntaxException {
     String appSubmitter = "nobody";
     String cmd = String.valueOf(
         PrivilegedOperation.RunAsUserCommand.DELETE_AS_USER.getValue());

+ 149 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -25,6 +25,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.*;
 
 import java.io.BufferedReader;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileReader;
@@ -55,9 +56,12 @@ import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -108,11 +112,13 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.security.AMSecretKeys;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
@@ -124,6 +130,8 @@ import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestContainerLaunch extends BaseContainerManagerTest {
 
@@ -2443,4 +2451,145 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         launch.getUserFilecacheDirs(localDirsForRead)),
         StringUtils.join(",", ctx.getUserFilecacheDirs()));
   }
+
+  @Test(timeout = 20000)
+  public void testFilesAndEnvWithoutHTTPS() throws Exception {
+    testFilesAndEnv(false);
+  }
+
+  @Test(timeout = 20000)
+  public void testFilesAndEnvWithHTTPS() throws Exception {
+    testFilesAndEnv(true);
+  }
+
+  private void testFilesAndEnv(boolean https) throws Exception {
+    // setup mocks
+    Dispatcher dispatcher = mock(Dispatcher.class);
+    EventHandler handler = mock(EventHandler.class);
+    when(dispatcher.getEventHandler()).thenReturn(handler);
+    ContainerExecutor containerExecutor = mock(ContainerExecutor.class);
+    doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        Object[] args = invocation.getArguments();
+        DataOutputStream dos = (DataOutputStream) args[0];
+        dos.writeBytes("script");
+        return null;
+      }
+    }).when(containerExecutor).writeLaunchEnv(
+        any(), any(), any(), any(), any(), any(), any());
+    Application app = mock(Application.class);
+    ApplicationId appId = mock(ApplicationId.class);
+    when(appId.toString()).thenReturn("1");
+    when(app.getAppId()).thenReturn(appId);
+    Container container = mock(Container.class);
+    ContainerId id = mock(ContainerId.class);
+    when(id.toString()).thenReturn("1");
+    when(container.getContainerId()).thenReturn(id);
+    when(container.getUser()).thenReturn("user");
+    ContainerLaunchContext clc = mock(ContainerLaunchContext.class);
+    when(clc.getCommands()).thenReturn(Lists.newArrayList());
+    when(container.getLaunchContext()).thenReturn(clc);
+    Credentials credentials = mock(Credentials.class);
+    when(container.getCredentials()).thenReturn(credentials);
+    doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        Object[] args = invocation.getArguments();
+        DataOutputStream dos = (DataOutputStream) args[0];
+        dos.writeBytes("credentials");
+        return null;
+      }
+    }).when(credentials).writeTokenStorageToStream(any(DataOutputStream.class));
+    if (https) {
+      when(credentials.getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE))
+          .thenReturn("keystore".getBytes());
+      when(credentials.getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE_PASSWORD))
+          .thenReturn("keystore_password".getBytes());
+      when(credentials.getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE))
+          .thenReturn("truststore".getBytes());
+      when(credentials.getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD))
+          .thenReturn("truststore_password".getBytes());
+    }
+
+    // call containerLaunch
+    ContainerLaunch containerLaunch = new ContainerLaunch(
+        distContext, conf, dispatcher,
+        containerExecutor, app, container, dirsHandler, containerManager);
+    containerLaunch.call();
+
+    // verify the nmPrivate paths and files
+    ArgumentCaptor<ContainerStartContext> cscArgument =
+        ArgumentCaptor.forClass(ContainerStartContext.class);
+    verify(containerExecutor, times(1)).launchContainer(cscArgument.capture());
+    ContainerStartContext csc = cscArgument.getValue();
+    Path nmPrivate = dirsHandler.getLocalPathForWrite(
+        ResourceLocalizationService.NM_PRIVATE_DIR + Path.SEPARATOR +
+            appId.toString() + Path.SEPARATOR + id.toString());
+    Assert.assertEquals(new Path(nmPrivate, ContainerLaunch.CONTAINER_SCRIPT),
+        csc.getNmPrivateContainerScriptPath());
+    Assert.assertEquals(new Path(nmPrivate,
+        String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT,
+            id.toString())), csc.getNmPrivateTokensPath());
+    Assert.assertEquals("script",
+        readStringFromPath(csc.getNmPrivateContainerScriptPath()));
+    Assert.assertEquals("credentials",
+        readStringFromPath(csc.getNmPrivateTokensPath()));
+    if (https) {
+      Assert.assertEquals(new Path(nmPrivate, ContainerLaunch.KEYSTORE_FILE),
+          csc.getNmPrivateKeystorePath());
+      Assert.assertEquals(new Path(nmPrivate, ContainerLaunch.TRUSTSTORE_FILE),
+          csc.getNmPrivateTruststorePath());
+      Assert.assertEquals("keystore",
+          readStringFromPath(csc.getNmPrivateKeystorePath()));
+      Assert.assertEquals("truststore",
+          readStringFromPath(csc.getNmPrivateTruststorePath()));
+    } else {
+      Assert.assertNull(csc.getNmPrivateKeystorePath());
+      Assert.assertNull(csc.getNmPrivateTruststorePath());
+    }
+
+    // verify env
+    ArgumentCaptor<Map> envArgument = ArgumentCaptor.forClass(Map.class);
+    verify(containerExecutor, times(1)).writeLaunchEnv(any(),
+        envArgument.capture(), any(), any(), any(), any(), any());
+    Map env = envArgument.getValue();
+    Path workDir = dirsHandler.getLocalPathForWrite(
+        ContainerLocalizer.USERCACHE + Path.SEPARATOR + container.getUser() +
+            Path.SEPARATOR + ContainerLocalizer.APPCACHE + Path.SEPARATOR +
+            app.getAppId().toString() + Path.SEPARATOR +
+            container.getContainerId().toString());
+    Assert.assertEquals(new Path(workDir,
+            ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE).toUri().getPath(),
+        env.get(ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME));
+    if (https) {
+      Assert.assertEquals(new Path(workDir,
+              ContainerLaunch.KEYSTORE_FILE).toUri().getPath(),
+          env.get(ApplicationConstants.KEYSTORE_FILE_LOCATION_ENV_NAME));
+      Assert.assertEquals("keystore_password",
+          env.get(ApplicationConstants.KEYSTORE_PASSWORD_ENV_NAME));
+      Assert.assertEquals(new Path(workDir,
+              ContainerLaunch.TRUSTSTORE_FILE).toUri().getPath(),
+          env.get(ApplicationConstants.TRUSTSTORE_FILE_LOCATION_ENV_NAME));
+      Assert.assertEquals("truststore_password",
+          env.get(ApplicationConstants.TRUSTSTORE_PASSWORD_ENV_NAME));
+    } else {
+      Assert.assertNull(env.get("KEYSTORE_FILE_LOCATION"));
+      Assert.assertNull(env.get("KEYSTORE_PASSWORD"));
+      Assert.assertNull(env.get("TRUSTSTORE_FILE_LOCATION"));
+      Assert.assertNull(env.get("TRUSTSTORE_PASSWORD"));
+    }
+  }
+
+  private String readStringFromPath(Path p) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    try (FSDataInputStream is = fs.open(p)) {
+      byte[] bytes = IOUtils.readFullyToByteArray(is);
+      return new String(bytes);
+    }
+  }
 }

+ 31 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerRelaunch.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -32,22 +33,34 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
+import org.apache.hadoop.yarn.server.security.AMSecretKeys;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /** Unit tests for relaunching containers. */
 public class TestContainerRelaunch {
 
   @Test
-  public void testRelaunchContext() throws Exception {
+  public void testRelaunchContextWithoutHTTPS() throws Exception {
+    testRelaunchContext(false);
+  }
+
+  @Test
+  public void testRelaunchContextWithHTTPS() throws Exception {
+    testRelaunchContext(true);
+  }
+
+  private void testRelaunchContext(boolean https) throws Exception {
     Configuration conf = new Configuration();
 
     Context mockContext = mock(Context.class);
@@ -63,6 +76,16 @@ public class TestContainerRelaunch {
     doReturn(cid).when(mockContainer).getContainerId();
     doReturn("/foo").when(mockContainer).getWorkDir();
     doReturn("/bar").when(mockContainer).getLogDir();
+    Credentials mockCredentials = mock(Credentials.class);
+    when(mockContainer.getCredentials()).thenReturn(mockCredentials);
+    if (https) {
+      when(mockCredentials.getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE))
+          .thenReturn("keystore".getBytes());
+      when(mockCredentials.getSecretKey(
+          AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE))
+          .thenReturn("truststore".getBytes());
+    }
     LocalDirsHandlerService mockDirsHandler =
         mock(LocalDirsHandlerService.class);
     doReturn(true).when(mockDirsHandler).isGoodLocalDir(any(String.class));
@@ -91,6 +114,13 @@ public class TestContainerRelaunch {
     assertNotNull("log dirs null", csc.getLogDirs());
     assertNotNull("script path null", csc.getNmPrivateContainerScriptPath());
     assertNotNull("tokens path null", csc.getNmPrivateTokensPath());
+    if (https) {
+      assertNotNull("keystore path null", csc.getNmPrivateKeystorePath());
+      assertNotNull("truststore path null", csc.getNmPrivateTruststorePath());
+    } else {
+      assertNull("keystore path not null", csc.getNmPrivateKeystorePath());
+      assertNull("truststore path not null", csc.getNmPrivateTruststorePath());
+    }
     assertNotNull("user null", csc.getUser());
     assertNotNull("user local dirs null", csc.getUserLocalDirs());
     assertNotNull("user filecache dirs null", csc.getUserFilecacheDirs());

+ 91 - 134
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java

@@ -57,6 +57,8 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
@@ -75,6 +77,8 @@ import java.nio.file.attribute.FileAttribute;
 import java.nio.file.attribute.PosixFilePermission;
 import java.nio.file.attribute.PosixFilePermissions;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -98,7 +102,9 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.LOCAL_DIRS;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.LOG_DIRS;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.NM_PRIVATE_CONTAINER_SCRIPT_PATH;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.NM_PRIVATE_KEYSTORE_PATH;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.NM_PRIVATE_TOKENS_PATH;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.NM_PRIVATE_TRUSTSTORE_PATH;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.PID;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.PID_FILE_PATH;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.PROCFS;
@@ -118,6 +124,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@RunWith(Parameterized.class)
 public class TestDockerContainerRuntime {
   private static final Logger LOG =
        LoggerFactory.getLogger(TestDockerContainerRuntime.class);
@@ -141,6 +148,8 @@ public class TestDockerContainerRuntime {
   private Path containerWorkDir;
   private Path nmPrivateContainerScriptPath;
   private Path nmPrivateTokensPath;
+  private Path nmPrivateKeystorePath;
+  private Path nmPrivateTruststorePath;
   private Path pidFilePath;
   private List<String> localDirs;
   private List<String> logDirs;
@@ -159,6 +168,16 @@ public class TestDockerContainerRuntime {
   @Rule
   public TemporaryFolder tempDir = new TemporaryFolder();
 
+  @Parameterized.Parameters(name = "https={0}")
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+        {true}, {false}
+    });
+  }
+
+  @Parameterized.Parameter
+  public boolean https;
+
   @Before
   public void setup() {
     String tmpPath = new StringBuffer(System.getProperty("test.build.data"))
@@ -227,6 +246,13 @@ public class TestDockerContainerRuntime {
     containerWorkDir = new Path("/test_container_work_dir");
     nmPrivateContainerScriptPath = new Path("/test_script_path");
     nmPrivateTokensPath = new Path("/test_private_tokens_path");
+    if (https) {
+      nmPrivateKeystorePath = new Path("/test_private_keystore_path");
+      nmPrivateTruststorePath = new Path("/test_private_truststore_path");
+    } else {
+      nmPrivateKeystorePath = null;
+      nmPrivateTruststorePath = null;
+    }
     pidFilePath = new Path("/test_pid_file_path");
     localDirs = new ArrayList<>();
     logDirs = new ArrayList<>();
@@ -261,6 +287,9 @@ public class TestDockerContainerRuntime {
         .setExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH,
             nmPrivateContainerScriptPath)
         .setExecutionAttribute(NM_PRIVATE_TOKENS_PATH, nmPrivateTokensPath)
+        .setExecutionAttribute(NM_PRIVATE_KEYSTORE_PATH, nmPrivateKeystorePath)
+        .setExecutionAttribute(NM_PRIVATE_TRUSTSTORE_PATH,
+            nmPrivateTruststorePath)
         .setExecutionAttribute(PID_FILE_PATH, pidFilePath)
         .setExecutionAttribute(LOCAL_DIRS, localDirs)
         .setExecutionAttribute(LOG_DIRS, logDirs)
@@ -385,9 +414,9 @@ public class TestDockerContainerRuntime {
 
     List<String> args = op.getArguments();
 
-    //This invocation of container-executor should use 12 arguments in a
+    //This invocation of container-executor should use 15 or 13 arguments in a
     // specific order
-    int expected = 12;
+    int expected = (https) ? 15 : 13;
     int counter = 1;
     Assert.assertEquals(expected, args.size());
     Assert.assertEquals(user, args.get(counter++));
@@ -396,10 +425,19 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals(appId, args.get(counter++));
     Assert.assertEquals(containerId, args.get(counter++));
     Assert.assertEquals(containerWorkDir.toString(), args.get(counter++));
-    Assert.assertEquals(nmPrivateContainerScriptPath.toUri()
-        .toString(), args.get(counter++));
+    Assert.assertEquals(nmPrivateContainerScriptPath.toUri().toString(),
+        args.get(counter++));
     Assert.assertEquals(nmPrivateTokensPath.toUri().getPath(),
         args.get(counter++));
+    if (https) {
+      Assert.assertEquals("--https", args.get(counter++));
+      Assert.assertEquals(nmPrivateKeystorePath.toUri().toString(),
+          args.get(counter++));
+      Assert.assertEquals(nmPrivateTruststorePath.toUri().toString(),
+          args.get(counter++));
+    } else {
+      Assert.assertEquals("--http", args.get(counter++));
+    }
     Assert.assertEquals(pidFilePath.toString(), args.get(counter++));
     Assert.assertEquals(localDirs.get(0), args.get(counter++));
     Assert.assertEquals(logDirs.get(0), args.get(counter++));
@@ -415,13 +453,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
     runtime.launchContainer(builder.build());
-
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get
-            (dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -467,12 +499,7 @@ public class TestDockerContainerRuntime {
     runtime.initialize(conf, nmContext);
     runtime.launchContainer(builder.build());
 
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get(
-        dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -516,13 +543,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
     runtime.launchContainer(builder.build());
-
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     Assert.assertEquals(13, dockerCommands.size());
     int counter = 0;
@@ -631,13 +652,9 @@ public class TestDockerContainerRuntime {
     //this should cause no failures.
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
+    List<String> dockerCommands = readDockerCommands();
 
     //This is the expected docker invocation for this case
-    List<String> dockerCommands = Files
-        .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
     int expected = 14;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
@@ -690,13 +707,9 @@ public class TestDockerContainerRuntime {
         expectedHostname);
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
+    List<String> dockerCommands = readDockerCommands();
 
     //This is the expected docker invocation for this case
-    List<String> dockerCommands = Files
-        .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
     int expected = 14;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
@@ -757,15 +770,10 @@ public class TestDockerContainerRuntime {
     //this should cause no failures.
     runtime.initialize(conf, nmContext);
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
+    List<String> dockerCommands = readDockerCommands();
 
     //This is the expected docker invocation for this case. customNetwork1
     // ("sdn1") is the expected network to be used in this case
-    List<String> dockerCommands = Files
-        .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
-
     int expected = 14;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
@@ -807,15 +815,10 @@ public class TestDockerContainerRuntime {
     env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         customNetwork2);
     runtime.launchContainer(builder.build());
-
-    op = capturePrivilegedOperationAndVerifyArgs();
-    args = op.getArguments();
-    dockerCommandFile = args.get(11);
+    dockerCommands = readDockerCommands();
 
     //This is the expected docker invocation for this case. customNetwork2
     // ("sdn2") is the expected network to be used in this case
-    dockerCommands = Files
-        .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
     counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -874,13 +877,7 @@ public class TestDockerContainerRuntime {
     env.put(DockerLinuxContainerRuntime
         .ENV_DOCKER_CONTAINER_PID_NAMESPACE, "invalid-value");
     runtime.launchContainer(builder.build());
-
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get
-        (dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     Assert.assertEquals(expected, dockerCommands.size());
@@ -926,12 +923,7 @@ public class TestDockerContainerRuntime {
         .ENV_DOCKER_CONTAINER_PID_NAMESPACE, "host");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 14;
     int counter = 0;
@@ -977,13 +969,7 @@ public class TestDockerContainerRuntime {
     env.put(DockerLinuxContainerRuntime
             .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "invalid-value");
     runtime.launchContainer(builder.build());
-
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     Assert.assertEquals(expected, dockerCommands.size());
@@ -1087,12 +1073,7 @@ public class TestDockerContainerRuntime {
             .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get
-        (dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -1203,12 +1184,7 @@ public class TestDockerContainerRuntime {
         "test_dir/test_resource_file:test_mount:ro");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get
-        (dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -1257,12 +1233,7 @@ public class TestDockerContainerRuntime {
             "test_dir/test_resource_file:test_mount2:ro");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get
-        (dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -1312,12 +1283,7 @@ public class TestDockerContainerRuntime {
             "/a:/a:shared,/b:/b:ro+shared,/c:/c:rw+rshared,/d:/d:private");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -1421,12 +1387,7 @@ public class TestDockerContainerRuntime {
         "/run");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     Assert.assertTrue(dockerCommands.contains("  tmpfs=/run"));
   }
@@ -1444,12 +1405,7 @@ public class TestDockerContainerRuntime {
         "/run,/tmp");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     Assert.assertTrue(dockerCommands.contains("  tmpfs=/run,/tmp"));
   }
@@ -1468,12 +1424,7 @@ public class TestDockerContainerRuntime {
         "/tmpfs");
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     Assert.assertTrue(dockerCommands.contains("  tmpfs=/tmpfs,/run,/var/run"));
   }
@@ -1567,12 +1518,7 @@ public class TestDockerContainerRuntime {
     runtime.initialize(conf, nmContext);
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -1634,12 +1580,7 @@ public class TestDockerContainerRuntime {
     runtime.initialize(conf, nmContext);
 
     runtime.launchContainer(builder.build());
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 13;
     int counter = 0;
@@ -2224,12 +2165,7 @@ public class TestDockerContainerRuntime {
     checkVolumeCreateCommand();
 
     runtime.launchContainer(containerRuntimeContext);
-    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(Paths.get
-        (dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands();
 
     int expected = 14;
     int counter = 0;
@@ -2348,7 +2284,7 @@ public class TestDockerContainerRuntime {
 
     List<String> args = op.getArguments();
 
-    int expectedArgs = 12;
+    int expectedArgs = (https) ? 15 : 13;
     int argsCounter = 0;
     Assert.assertEquals(expectedArgs, args.size());
     Assert.assertEquals(runAsUser, args.get(argsCounter++));
@@ -2362,6 +2298,15 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals(outDir.toUri().getPath(), args.get(argsCounter++));
     Assert.assertEquals(nmPrivateTokensPath.toUri().getPath(),
         args.get(argsCounter++));
+    if (https) {
+      Assert.assertEquals("--https", args.get(argsCounter++));
+      Assert.assertEquals(nmPrivateKeystorePath.toUri().toString(),
+          args.get(argsCounter++));
+      Assert.assertEquals(nmPrivateTruststorePath.toUri().toString(),
+          args.get(argsCounter++));
+    } else {
+      Assert.assertEquals("--http", args.get(argsCounter++));
+    }
     Assert.assertEquals(pidFilePath.toString(), args.get(argsCounter++));
     Assert.assertEquals(localDirs.get(0), args.get(argsCounter++));
     Assert.assertEquals(logDirs.get(0), args.get(argsCounter++));
@@ -2416,13 +2361,7 @@ public class TestDockerContainerRuntime {
         DockerCommandExecutor.DockerContainerStatus.STOPPED.getName());
     runtime.initialize(conf, nmContext);
     runtime.relaunchContainer(builder.build());
-
-    PrivilegedOperation op = capturePrivilegedOperation(2);
-    List<String> args = op.getArguments();
-    String dockerCommandFile = args.get(11);
-
-    List<String> dockerCommands = Files.readAllLines(
-        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    List<String> dockerCommands = readDockerCommands(2);
 
     int expected = 3;
     int counter = 0;
@@ -2445,4 +2384,22 @@ public class TestDockerContainerRuntime {
         dockerCommands.get(2));
     Assert.assertEquals("  signal=" + signal, dockerCommands.get(3));
   }
+
+  private List<String> readDockerCommands() throws IOException,
+      PrivilegedOperationException {
+    return readDockerCommands(1);
+  }
+
+  private List<String> readDockerCommands(int invocations) throws IOException,
+      PrivilegedOperationException {
+    PrivilegedOperation op = (invocations == 1)
+        ? capturePrivilegedOperationAndVerifyArgs()
+        : capturePrivilegedOperation(invocations);
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get((https) ? 14 : 12);
+
+    List<String> dockerCommands = Files.readAllLines(
+        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+    return dockerCommands;
+  }
 }

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.ProxyCAManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
@@ -119,6 +120,8 @@ public class RMActiveServiceContext {
   private ResourceProfilesManager resourceProfilesManager;
   private MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager;
 
+  private ProxyCAManager proxyCAManager;
+
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
   }
@@ -554,4 +557,16 @@ public class RMActiveServiceContext {
       ResourceProfilesManager resourceProfilesManager) {
     this.resourceProfilesManager = resourceProfilesManager;
   }
+
+  @Private
+  @Unstable
+  public ProxyCAManager getProxyCAManager() {
+    return proxyCAManager;
+  }
+
+  @Private
+  @Unstable
+  public void setProxyCAManager(ProxyCAManager proxyCAManager) {
+    this.proxyCAManager = proxyCAManager;
+  }
 }

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.ProxyCAManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
@@ -188,4 +189,8 @@ public interface RMContext extends ApplicationMasterServiceContext {
 
   void setMultiNodeSortingManager(
       MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager);
+
+  ProxyCAManager getProxyCAManager();
+
+  void setProxyCAManager(ProxyCAManager proxyCAManager);
 }

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.ProxyCAManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
@@ -637,6 +638,16 @@ public class RMContextImpl implements RMContext {
   public void setResourceProfilesManager(ResourceProfilesManager mgr) {
     this.activeServiceContext.setResourceProfilesManager(mgr);
   }
+
+  @Override
+  public ProxyCAManager getProxyCAManager() {
+    return this.activeServiceContext.getProxyCAManager();
+  }
+
+  @Override
+  public void setProxyCAManager(ProxyCAManager proxyCAManager) {
+    this.activeServiceContext.setProxyCAManager(proxyCAManager);
+  }
   // Note: Read java doc before adding any services over here.
 
   @Override

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -106,6 +106,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
+import org.apache.hadoop.yarn.server.resourcemanager.security.ProxyCAManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp;
@@ -113,6 +114,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.service.SystemServiceManager;
 import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher;
+import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxy;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet;
@@ -197,6 +199,7 @@ public class ResourceManager extends CompositeService
   protected ApplicationACLsManager applicationACLsManager;
   protected QueueACLsManager queueACLsManager;
   private FederationStateStoreService federationStateStoreService;
+  private ProxyCAManager proxyCAManager;
   private WebApp webApp;
   private AppReportFetcher fetcher = null;
   protected ResourceTrackerService resourceTracker;
@@ -830,6 +833,10 @@ public class ResourceManager extends CompositeService
         LOG.info("Initialized Federation membership.");
       }
 
+      proxyCAManager = new ProxyCAManager(new ProxyCA(), rmContext);
+      addService(proxyCAManager);
+      rmContext.setProxyCAManager(proxyCAManager);
+
       new RMNMInfo(rmContext, scheduler);
 
       if (conf.getBoolean(YarnConfiguration.YARN_API_SERVICES_ENABLE,
@@ -1180,6 +1187,8 @@ public class ResourceManager extends CompositeService
       }
       builder.withServlet(ProxyUriUtils.PROXY_SERVLET_NAME,
           ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
+      builder.withAttribute(WebAppProxy.PROXY_CA,
+          rmContext.getProxyCAManager().getProxyCA());
       builder.withAttribute(WebAppProxy.FETCHER_ATTRIBUTE, fetcher);
       String[] proxyParts = proxyHostAndPort.split(":");
       builder.withAttribute(WebAppProxy.PROXY_HOST_ATTRIBUTE, proxyParts[0]);

+ 29 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.amlauncher;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -62,6 +63,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
+import org.apache.hadoop.yarn.server.security.AMSecretKeys;
+import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
@@ -233,6 +236,32 @@ public class AMLauncher implements Runnable {
     if (amrmToken != null) {
       credentials.addToken(amrmToken.getService(), amrmToken);
     }
+
+    // Setup Keystore and Truststore
+    String httpsPolicy = conf.get(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY,
+        YarnConfiguration.DEFAULT_RM_APPLICATION_HTTPS_POLICY);
+    if (httpsPolicy.equals("LENIENT") || httpsPolicy.equals("STRICT")) {
+      ProxyCA proxyCA = rmContext.getProxyCAManager().getProxyCA();
+      try {
+        String kPass = proxyCA.generateKeyStorePassword();
+        byte[] keyStore = proxyCA.createChildKeyStore(applicationId, kPass);
+        credentials.addSecretKey(
+            AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE, keyStore);
+        credentials.addSecretKey(
+            AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE_PASSWORD,
+            kPass.getBytes(StandardCharsets.UTF_8));
+        String tPass = proxyCA.generateKeyStorePassword();
+        byte[] trustStore = proxyCA.getChildTrustStore(tPass);
+        credentials.addSecretKey(
+            AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE, trustStore);
+        credentials.addSecretKey(
+            AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD,
+            tPass.getBytes(StandardCharsets.UTF_8));
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+
     DataOutputBuffer dob = new DataOutputBuffer();
     credentials.writeTokenStorageToStream(dob);
     container.setTokens(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/ProxyCAManager.java

@@ -0,0 +1,68 @@
+/**
+ * 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.resourcemanager.security;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
+import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manager for {@link ProxyCA}, which contains the Certificate Authority for
+ * AMs to have certificates for HTTPS communication with the RM Proxy.
+ */
+@Private
+@InterfaceStability.Unstable
+public class ProxyCAManager extends AbstractService implements Recoverable {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ProxyCAManager.class);
+
+  private ProxyCA proxyCA;
+  private RMContext rmContext;
+
+  public ProxyCAManager(ProxyCA proxyCA, RMContext rmContext) {
+    super(ProxyCAManager.class.getName());
+    this.proxyCA = proxyCA;
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    proxyCA.init();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    super.serviceStop();
+  }
+
+  public ProxyCA getProxyCA() {
+    return proxyCA;
+  }
+
+  public void recover(RMState state) {
+    // TODO: RM HA YARN-8449
+  }
+}

+ 80 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java

@@ -29,8 +29,11 @@ import java.util.concurrent.TimeoutException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
@@ -79,7 +82,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.server.resourcemanager.security.ProxyCAManager;
+import org.apache.hadoop.yarn.server.security.AMSecretKeys;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -91,6 +97,7 @@ import com.google.common.base.Supplier;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
 public class TestApplicationMasterLauncher {
@@ -424,16 +431,47 @@ public class TestApplicationMasterLauncher {
   }
 
   @Test
-  public void testSetupTokens() throws Exception {
-    MockRM rm = new MockRM();
+  public void testSetupTokensWithoutHTTPS() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    // default conf
+    testSetupTokens(false, conf);
+    conf.set(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY, "NONE");
+    testSetupTokens(false, conf);
+  }
+
+  @Test
+  public void testSetupTokensWithHTTPS() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY, "LENIENT");
+    testSetupTokens(true, conf);
+    conf.set(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY, "STRICT");
+    testSetupTokens(true, conf);
+  }
+
+  private void testSetupTokens(boolean https, YarnConfiguration conf)
+      throws Exception {
+    MockRM rm = new MockRM(conf);
     rm.start();
     MockNM nm1 = rm.registerNode("h1:1234", 5000);
     RMApp app = rm.submitApp(2000);
     /// kick the scheduling
     nm1.nodeHeartbeat(true);
     RMAppAttempt attempt = app.getCurrentAppAttempt();
-    MyAMLauncher launcher = new MyAMLauncher(rm.getRMContext(),
-        attempt, AMLauncherEventType.LAUNCH, rm.getConfig());
+    AMRMTokenIdentifier tokenIdentifier =
+        new AMRMTokenIdentifier(attempt.getAppAttemptId(), 1);
+    ProxyCA proxyCA = mock(ProxyCA.class);
+    when(proxyCA.generateKeyStorePassword())
+        .thenReturn("kPassword").thenReturn("tPassword");
+    when(proxyCA.createChildKeyStore(any(), any()))
+        .thenReturn("keystore".getBytes());
+    when(proxyCA.getChildTrustStore(any()))
+        .thenReturn("truststore".getBytes());
+    RMContext rmContext = spy(rm.getRMContext());
+    ProxyCAManager proxyCAManager = mock(ProxyCAManager.class);
+    when(proxyCAManager.getProxyCA()).thenReturn(proxyCA);
+    when(rmContext.getProxyCAManager()).thenReturn(proxyCAManager);
+    MyAMLauncher launcher = new MyAMLauncher(rmContext,
+        attempt, AMLauncherEventType.LAUNCH, rm.getConfig(), tokenIdentifier);
     DataOutputBuffer dob = new DataOutputBuffer();
     Credentials ts = new Credentials();
     ts.writeTokenStorageToStream(dob);
@@ -455,14 +493,48 @@ public class TestApplicationMasterLauncher {
     } catch (java.io.EOFException e) {
       Assert.fail("EOFException should not happen.");
     }
+
+    // verify token
+    DataInputByteBuffer dibb = new DataInputByteBuffer();
+    dibb.reset(amContainer.getTokens());
+    Credentials credentials = new Credentials();
+    credentials.readTokenStorageStream(dibb);
+    Assert.assertEquals(1, credentials.numberOfTokens());
+    org.apache.hadoop.security.token.Token<? extends TokenIdentifier> token =
+        credentials.getAllTokens().iterator().next();
+    Assert.assertEquals(tokenIdentifier.getKind(), token.getKind());
+    Assert.assertArrayEquals(tokenIdentifier.getBytes(), token.getIdentifier());
+    Assert.assertArrayEquals("password".getBytes(), token.getPassword());
+
+    // verify keystore and truststore
+    if (https) {
+      Assert.assertEquals(4, credentials.numberOfSecretKeys());
+      Assert.assertArrayEquals("keystore".getBytes(),
+          credentials.getSecretKey(
+              AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE));
+      Assert.assertArrayEquals("kPassword".getBytes(),
+          credentials.getSecretKey(
+              AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE_PASSWORD));
+      Assert.assertArrayEquals("truststore".getBytes(),
+          credentials.getSecretKey(
+              AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE));
+      Assert.assertArrayEquals("tPassword".getBytes(),
+          credentials.getSecretKey(
+              AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD));
+    } else {
+      Assert.assertEquals(0, credentials.numberOfSecretKeys());
+    }
   }
 
   static class MyAMLauncher extends AMLauncher {
     int count;
+    AMRMTokenIdentifier tokenIdentifier;
     public MyAMLauncher(RMContext rmContext, RMAppAttempt application,
-        AMLauncherEventType eventType, Configuration conf) {
+        AMLauncherEventType eventType, Configuration conf,
+        AMRMTokenIdentifier tokenIdentifier) {
       super(rmContext, application, eventType, conf);
       count = 0;
+      this.tokenIdentifier = tokenIdentifier;
     }
 
     protected org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>
@@ -471,7 +543,9 @@ public class TestApplicationMasterLauncher {
       if (count == 1) {
         throw new RuntimeException("createAndSetAMRMToken failure");
       }
-      return null;
+      return new org.apache.hadoop.security.token.Token<>(
+          tokenIdentifier.getBytes(), "password".getBytes(),
+          tokenIdentifier.getKind(), new Text());
     }
 
     protected void setupTokens(ContainerLaunchContext container,

+ 51 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestProxyCAManager.java

@@ -0,0 +1,51 @@
+/**
+ * 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.resourcemanager.security;
+
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class TestProxyCAManager {
+
+  @Test
+  public void testBasics() throws Exception {
+    ProxyCA proxyCA = spy(new ProxyCA());
+    RMContext rmContext = mock(RMContext.class);
+    ProxyCAManager proxyCAManager = new ProxyCAManager(proxyCA, rmContext);
+    proxyCAManager.init(new YarnConfiguration());
+    Assert.assertEquals(proxyCA, proxyCAManager.getProxyCA());
+    verify(proxyCA, times(0)).init();
+    Assert.assertNull(proxyCA.getCaCert());
+    Assert.assertNull(proxyCA.getCaKeyPair());
+
+    proxyCAManager.start();
+    verify(proxyCA, times(1)).init();
+    Assert.assertNotNull(proxyCA.getCaCert());
+    Assert.assertNotNull(proxyCA.getCaKeyPair());
+  }
+}

+ 408 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyCA.java

@@ -0,0 +1,408 @@
+/*
+ * 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.webproxy;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.http.conn.ssl.DefaultHostnameVerifier;
+import org.apache.http.conn.util.PublicSuffixMatcherLoader;
+import org.bouncycastle.asn1.x500.X500Name;
+import org.bouncycastle.asn1.x509.AlgorithmIdentifier;
+import org.bouncycastle.asn1.x509.BasicConstraints;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.cert.jcajce.JcaX509ExtensionUtils;
+import org.bouncycastle.crypto.util.PrivateKeyFactory;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.DefaultDigestAlgorithmIdentifierFinder;
+import org.bouncycastle.operator.DefaultSignatureAlgorithmIdentifierFinder;
+import org.bouncycastle.operator.OperatorCreationException;
+import org.bouncycastle.operator.bc.BcRSAContentSignerBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509KeyManager;
+import javax.net.ssl.X509TrustManager;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.net.Socket;
+import java.security.GeneralSecurityException;
+import java.security.InvalidKeyException;
+import java.security.Key;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.Principal;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.SecureRandom;
+import java.security.Security;
+import java.security.SignatureException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.Random;
+import java.util.UUID;
+
+/**
+ * Allows for the generation and acceptance of specialized HTTPS Certificates to
+ * be used for HTTPS communication between the AMs and the RM Proxy.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ProxyCA {
+  private static final Logger LOG = LoggerFactory.getLogger(ProxyCA.class);
+
+  private X509Certificate caCert;
+  private KeyPair caKeyPair;
+  private KeyStore childTrustStore;
+  private final Random srand;
+  private X509TrustManager defaultTrustManager;
+  private X509KeyManager x509KeyManager;
+  private HostnameVerifier hostnameVerifier;
+  private static final AlgorithmIdentifier SIG_ALG_ID =
+      new DefaultSignatureAlgorithmIdentifierFinder().find("SHA512WITHRSA");
+
+  public ProxyCA() {
+    srand = new SecureRandom();
+
+    // This only has to be done once
+    Security.addProvider(new BouncyCastleProvider());
+  }
+
+  public void init() throws GeneralSecurityException, IOException {
+    createCACertAndKeyPair();
+
+    defaultTrustManager = null;
+    TrustManagerFactory factory = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+    factory.init((KeyStore) null);
+    for (TrustManager manager : factory.getTrustManagers()) {
+      if (manager instanceof X509TrustManager) {
+        defaultTrustManager = (X509TrustManager) manager;
+        break;
+      }
+    }
+    if (defaultTrustManager == null) {
+      throw new YarnRuntimeException(
+          "Could not find default X509 Trust Manager");
+    }
+
+    this.x509KeyManager = createKeyManager();
+    this.hostnameVerifier = createHostnameVerifier();
+    this.childTrustStore = createTrustStore("client", caCert);
+  }
+
+  private X509Certificate createCert(boolean isCa, String issuerStr,
+      String subjectStr, Date from, Date to, PublicKey publicKey,
+      PrivateKey privateKey) throws GeneralSecurityException, IOException {
+    X500Name issuer = new X500Name(issuerStr);
+    X500Name subject = new X500Name(subjectStr);
+    SubjectPublicKeyInfo subPubKeyInfo =
+        SubjectPublicKeyInfo.getInstance(publicKey.getEncoded());
+    X509v3CertificateBuilder certBuilder = new X509v3CertificateBuilder(
+        issuer, new BigInteger(64, srand), from, to, subject, subPubKeyInfo);
+    AlgorithmIdentifier digAlgId =
+        new DefaultDigestAlgorithmIdentifierFinder().find(SIG_ALG_ID);
+    ContentSigner contentSigner;
+    try {
+      contentSigner = new BcRSAContentSignerBuilder(SIG_ALG_ID, digAlgId)
+          .build(PrivateKeyFactory.createKey(privateKey.getEncoded()));
+    } catch (OperatorCreationException oce) {
+      throw new GeneralSecurityException(oce);
+    }
+    if (isCa) {
+      // BasicConstraints(0) indicates a CA and a path length of 0.  This is
+      // important to indicate that child certificates can't issue additional
+      // grandchild certificates
+      certBuilder.addExtension(Extension.basicConstraints, true,
+          new BasicConstraints(0));
+    } else {
+      // BasicConstraints(false) indicates this is not a CA
+      certBuilder.addExtension(Extension.basicConstraints, true,
+          new BasicConstraints(false));
+      certBuilder.addExtension(Extension.authorityKeyIdentifier, false,
+          new JcaX509ExtensionUtils().createAuthorityKeyIdentifier(caCert));
+    }
+    X509CertificateHolder certHolder = certBuilder.build(contentSigner);
+    X509Certificate cert = new JcaX509CertificateConverter().setProvider("BC")
+        .getCertificate(certHolder);
+    LOG.info("Created Certificate for {}", subject);
+    return cert;
+  }
+
+  private void createCACertAndKeyPair()
+      throws GeneralSecurityException, IOException {
+    Date from = new Date();
+    Date to = new GregorianCalendar(2037, Calendar.DECEMBER, 31).getTime();
+    KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+    keyGen.initialize(2048);
+    caKeyPair = keyGen.genKeyPair();
+    String subject = "OU=YARN-" + UUID.randomUUID();
+    caCert = createCert(true, subject, subject, from, to,
+        caKeyPair.getPublic(), caKeyPair.getPrivate());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CA Certificate: \n{}", caCert);
+    }
+  }
+
+  public byte[] createChildKeyStore(ApplicationId appId, String ksPassword)
+      throws Exception {
+    // We don't check the expiration date, and this will provide further reason
+    // for outside users to not accept these certificates
+    Date from = new Date();
+    Date to = from;
+    KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+    keyGen.initialize(2048);
+    KeyPair keyPair = keyGen.genKeyPair();
+    String issuer = caCert.getSubjectX500Principal().getName();
+    String subject = "CN=" + appId;
+    X509Certificate cert = createCert(false, issuer, subject, from, to,
+        keyPair.getPublic(), caKeyPair.getPrivate());
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Certificate for {}: \n{}", appId, cert);
+    }
+
+    KeyStore keyStore = createChildKeyStore(ksPassword, "server",
+        keyPair.getPrivate(), cert);
+    return keyStoreToBytes(keyStore, ksPassword);
+  }
+
+  public byte[] getChildTrustStore(String password)
+      throws GeneralSecurityException, IOException {
+    return keyStoreToBytes(childTrustStore, password);
+  }
+
+  private KeyStore createEmptyKeyStore()
+      throws GeneralSecurityException, IOException {
+    KeyStore ks = KeyStore.getInstance("JKS");
+    ks.load(null, null); // initialize
+    return ks;
+  }
+
+  private KeyStore createChildKeyStore(String password, String alias,
+      Key privateKey, Certificate cert)
+      throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    ks.setKeyEntry(alias, privateKey, password.toCharArray(),
+        new Certificate[]{cert, caCert});
+    return ks;
+  }
+
+  public String generateKeyStorePassword() {
+    return RandomStringUtils.random(16, 0, 0, true, true, null, srand);
+  }
+
+  private byte[] keyStoreToBytes(KeyStore ks, String password)
+      throws GeneralSecurityException, IOException {
+    try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      ks.store(out, password.toCharArray());
+      return out.toByteArray();
+    }
+  }
+
+  private KeyStore createTrustStore(String alias, Certificate cert)
+      throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    ks.setCertificateEntry(alias, cert);
+    return ks;
+  }
+
+  public SSLContext createSSLContext(ApplicationId appId)
+      throws GeneralSecurityException {
+    // We need the normal TrustManager, plus our custom one.  While the
+    // SSLContext accepts an array of TrustManagers, the docs indicate that only
+    // the first instance of any particular implementation type is used
+    // (e.g. X509KeyManager) - this means that simply putting both TrustManagers
+    // in won't work.  We need to have ours do both.
+    TrustManager[] trustManagers = new TrustManager[] {
+        createTrustManager(appId)};
+    KeyManager[] keyManagers = new KeyManager[]{x509KeyManager};
+
+    SSLContext sc = SSLContext.getInstance("SSL");
+    sc.init(keyManagers, trustManagers, new SecureRandom());
+    return sc;
+  }
+
+  @VisibleForTesting
+  X509TrustManager createTrustManager(ApplicationId appId) {
+    return new X509TrustManager() {
+      @Override
+      public java.security.cert.X509Certificate[] getAcceptedIssuers() {
+        return defaultTrustManager.getAcceptedIssuers();
+      }
+
+      @Override
+      public void checkClientTrusted(
+          java.security.cert.X509Certificate[] certs, String authType) {
+        // not used
+      }
+
+      @Override
+      public void checkServerTrusted(
+          java.security.cert.X509Certificate[] certs, String authType)
+          throws CertificateException {
+        // Our certs will always have 2 in the chain, with 0 being the app's
+        // cert and 1 being the RM's cert
+        boolean issuedByRM = false;
+        if (certs.length == 2) {
+          try {
+            // We can verify both certs using the CA cert's public key - the
+            // child cert's info is not needed
+            certs[0].verify(caKeyPair.getPublic());
+            certs[1].verify(caKeyPair.getPublic());
+            issuedByRM = true;
+          } catch (CertificateException | NoSuchAlgorithmException
+              | InvalidKeyException | NoSuchProviderException
+              | SignatureException e) {
+            // Fall back to the default trust manager
+            LOG.debug("Could not verify certificate with RM CA, falling " +
+                "back to default", e);
+            defaultTrustManager.checkServerTrusted(certs, authType);
+          }
+        } else {
+          LOG.debug("Certificate not issued by RM CA, falling back to " +
+              "default");
+          defaultTrustManager.checkServerTrusted(certs, authType);
+        }
+        if (issuedByRM) {
+          // Check that it has the correct App ID
+          if (!certs[0].getSubjectX500Principal().getName()
+              .equals("CN=" + appId)) {
+            throw new CertificateException(
+                "Expected to find Subject X500 Principal with CN="
+                    + appId + " but found "
+                    + certs[0].getSubjectX500Principal().getName());
+          }
+          LOG.debug("Verified certificate signed by RM CA");
+        }
+      }
+    };
+  }
+
+  @VisibleForTesting
+  X509KeyManager getX509KeyManager() {
+    return x509KeyManager;
+  }
+
+  private X509KeyManager createKeyManager() {
+    return new X509KeyManager() {
+      @Override
+      public String[] getClientAliases(String s, Principal[] principals) {
+        return new String[]{"client"};
+      }
+
+      @Override
+      public String chooseClientAlias(String[] strings,
+          Principal[] principals, Socket socket) {
+        return "client";
+      }
+
+      @Override
+      public String[] getServerAliases(String s, Principal[] principals) {
+        return null;
+      }
+
+      @Override
+      public String chooseServerAlias(String s, Principal[] principals,
+          Socket socket) {
+        return null;
+      }
+
+      @Override
+      public X509Certificate[] getCertificateChain(String s) {
+        return new X509Certificate[]{caCert};
+      }
+
+      @Override
+      public PrivateKey getPrivateKey(String s) {
+        return caKeyPair.getPrivate();
+      }
+    };
+  }
+
+  public HostnameVerifier getHostnameVerifier() {
+    return hostnameVerifier;
+  }
+
+  private HostnameVerifier createHostnameVerifier() {
+    HostnameVerifier defaultHostnameVerifier =
+        new DefaultHostnameVerifier(PublicSuffixMatcherLoader.getDefault());
+    return new HostnameVerifier() {
+      @Override
+      public boolean verify(String host, SSLSession sslSession) {
+        try {
+          Certificate[] certs = sslSession.getPeerCertificates();
+          if (certs.length == 2) {
+            // Make sure this is one of our certs.  More thorough checking would
+            // have already been done by the SSLContext
+            certs[0].verify(caKeyPair.getPublic());
+            LOG.debug("Verified certificate signed by RM CA, " +
+                "skipping hostname verification");
+            return true;
+          }
+        } catch (SSLPeerUnverifiedException e) {
+          // No certificate
+          return false;
+        } catch (CertificateException | NoSuchAlgorithmException
+            | InvalidKeyException | SignatureException
+            | NoSuchProviderException e) {
+          // fall back to normal verifier below
+          LOG.debug("Could not verify certificate with RM CA, " +
+              "falling back to default hostname verification", e);
+        }
+        return defaultHostnameVerifier.verify(host, sslSession);
+      }
+    };
+  }
+
+  @VisibleForTesting
+  void setDefaultTrustManager(X509TrustManager trustManager) {
+    this.defaultTrustManager = trustManager;
+  }
+
+  @VisibleForTesting
+  public X509Certificate getCaCert() {
+    return caCert;
+  }
+
+  @VisibleForTesting
+  public KeyPair getCaKeyPair() {
+    return caKeyPair;
+  }
+}

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java

@@ -39,6 +39,7 @@ public class WebAppProxy extends AbstractService {
   public static final String FETCHER_ATTRIBUTE= "AppUrlFetcher";
   public static final String IS_SECURITY_ENABLED_ATTRIBUTE = "IsSecurityEnabled";
   public static final String PROXY_HOST_ATTRIBUTE = "proxyHost";
+  public static final String PROXY_CA = "ProxyCA";
   private static final Logger LOG = LoggerFactory.getLogger(
       WebAppProxy.class);
   

+ 74 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java

@@ -45,6 +45,7 @@ import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.core.UriBuilder;
 import javax.ws.rs.core.UriBuilderException;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -63,15 +64,14 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.http.Header;
 import org.apache.http.HttpResponse;
 import org.apache.http.NameValuePair;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.config.RequestConfig;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPut;
 import org.apache.http.client.methods.HttpRequestBase;
-import org.apache.http.client.params.ClientPNames;
-import org.apache.http.client.params.CookiePolicy;
 import org.apache.http.client.utils.URLEncodedUtils;
-import org.apache.http.conn.params.ConnRoutePNames;
 import org.apache.http.entity.StringEntity;
-import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -177,6 +177,39 @@ public class WebAppProxyServlet extends HttpServlet {
         __().
         __();
   }
+
+  /**
+   * Show the user a page that says that HTTPS must be used but was not.
+   * @param resp the http response
+   * @param link the link to point to
+   * @return true if HTTPS must be used but was not, false otherwise
+   * @throws IOException on any error.
+   */
+  @VisibleForTesting
+  static boolean checkHttpsStrictAndNotProvided(
+      HttpServletResponse resp, URI link, YarnConfiguration conf)
+      throws IOException {
+    String httpsPolicy = conf.get(
+        YarnConfiguration.RM_APPLICATION_HTTPS_POLICY,
+        YarnConfiguration.DEFAULT_RM_APPLICATION_HTTPS_POLICY);
+    boolean required = httpsPolicy.equals("STRICT");
+    boolean provided = link.getScheme().equals("https");
+    if (required && !provided) {
+      resp.setContentType(MimeType.HTML);
+      Page p = new Page(resp.getWriter());
+      p.html().
+          h1("HTTPS must be used").
+          h3().
+          __(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY,
+              "is set to STRICT, which means that the tracking URL ",
+              "must be an HTTPS URL, but it is not.").
+          __("The tracking URL is: ", link).
+          __().
+          __();
+      return true;
+    }
+    return false;
+  }
   
   /**
    * Download link and have it be the response.
@@ -186,17 +219,31 @@ public class WebAppProxyServlet extends HttpServlet {
    * @param c the cookie to set if any
    * @param proxyHost the proxy host
    * @param method the http method
+   * @param appId the ApplicationID
    * @throws IOException on any error.
    */
-  private static void proxyLink(final HttpServletRequest req,
+  private void proxyLink(final HttpServletRequest req,
       final HttpServletResponse resp, final URI link, final Cookie c,
-      final String proxyHost, final HTTP method) throws IOException {
-    DefaultHttpClient client = new DefaultHttpClient();
-    client
-        .getParams()
-        .setParameter(ClientPNames.COOKIE_POLICY,
-            CookiePolicy.BROWSER_COMPATIBILITY)
-        .setBooleanParameter(ClientPNames.ALLOW_CIRCULAR_REDIRECTS, true);
+      final String proxyHost, final HTTP method, final ApplicationId appId)
+      throws IOException {
+    HttpClientBuilder httpClientBuilder = HttpClientBuilder.create();
+
+    String httpsPolicy = conf.get(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY,
+        YarnConfiguration.DEFAULT_RM_APPLICATION_HTTPS_POLICY);
+    if (httpsPolicy.equals("LENIENT") || httpsPolicy.equals("STRICT")) {
+      ProxyCA proxyCA = getProxyCA();
+      // ProxyCA could be null when the Proxy is run outside the RM
+      if (proxyCA != null) {
+        try {
+          httpClientBuilder.setSSLContext(proxyCA.createSSLContext(appId));
+          httpClientBuilder.setSSLHostnameVerifier(
+              proxyCA.getHostnameVerifier());
+        } catch (Exception e) {
+          throw new IOException(e);
+        }
+      }
+    }
+
     // Make sure we send the request from the proxy address in the config
     // since that is what the AM filter checks against. IP aliasing or
     // similar could cause issues otherwise.
@@ -204,8 +251,11 @@ public class WebAppProxyServlet extends HttpServlet {
     if (LOG.isDebugEnabled()) {
       LOG.debug("local InetAddress for proxy host: {}", localAddress);
     }
-    client.getParams()
-        .setParameter(ConnRoutePNames.LOCAL_ADDRESS, localAddress);
+    httpClientBuilder.setDefaultRequestConfig(
+        RequestConfig.custom()
+        .setCircularRedirectsAllowed(true)
+        .setLocalAddress(localAddress)
+        .build());
 
     HttpRequestBase base = null;
     if (method.equals(HTTP.GET)) {
@@ -247,6 +297,7 @@ public class WebAppProxyServlet extends HttpServlet {
           PROXY_USER_COOKIE_NAME + "=" + URLEncoder.encode(user, "ASCII"));
     }
     OutputStream out = resp.getOutputStream();
+    HttpClient client = httpClientBuilder.build();
     try {
       HttpResponse httpResp = client.execute(base);
       resp.setStatus(httpResp.getStatusLine().getStatusCode());
@@ -287,6 +338,10 @@ public class WebAppProxyServlet extends HttpServlet {
     return ((AppReportFetcher) getServletContext()
         .getAttribute(WebAppProxy.FETCHER_ATTRIBUTE)).getApplicationReport(id);
   }
+
+  private ProxyCA getProxyCA() {
+    return ((ProxyCA) getServletContext().getAttribute(WebAppProxy.PROXY_CA));
+  }
   
   private String getProxyHost() throws IOException {
     return ((String) getServletContext()
@@ -420,6 +475,10 @@ public class WebAppProxyServlet extends HttpServlet {
         return;
       }
 
+      if (checkHttpsStrictAndNotProvided(resp, trackingUri, conf)) {
+        return;
+      }
+
       String runningUser = applicationReport.getUser();
 
       if (checkUser && !runningUser.equals(remoteUser)) {
@@ -453,7 +512,7 @@ public class WebAppProxyServlet extends HttpServlet {
       if (userWasWarned && userApproved) {
         c = makeCheckCookie(id, true);
       }
-      proxyLink(req, resp, toFetch, c, getProxyHost(), method);
+      proxyLink(req, resp, toFetch, c, getProxyHost(), method, id);
 
     } catch(URISyntaxException | YarnException e) {
       throw new IOException(e); 

+ 518 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestProxyCA.java

@@ -0,0 +1,518 @@
+/**
+* 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.webproxy;
+
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.net.ssl.X509KeyManager;
+import javax.net.ssl.X509TrustManager;
+import javax.security.auth.x500.X500Principal;
+import java.security.InvalidKeyException;
+import java.security.Key;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.Signature;
+import java.security.SignatureException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+public class TestProxyCA {
+
+  @Test
+  public void testInit() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    Assert.assertNull(proxyCA.getCaCert());
+    Assert.assertNull(proxyCA.getCaKeyPair());
+    Assert.assertNull(proxyCA.getX509KeyManager());
+    Assert.assertNull(proxyCA.getHostnameVerifier());
+
+    proxyCA.init();
+    Assert.assertNotNull(proxyCA.getCaCert());
+    Assert.assertNotNull(proxyCA.getCaKeyPair());
+    Assert.assertNotNull(proxyCA.getX509KeyManager());
+    Assert.assertNotNull(proxyCA.getHostnameVerifier());
+  }
+
+  @Test
+  public void testCreateChildKeyStore() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    byte[] keystoreBytes = proxyCA.createChildKeyStore(appId,
+        "password");
+    KeyStore keyStore = KeyStoreTestUtil.bytesToKeyStore(keystoreBytes,
+        "password");
+    Assert.assertEquals(1, keyStore.size());
+    Certificate[] certChain = keyStore.getCertificateChain("server");
+    Assert.assertEquals(2, certChain.length);
+    X509Certificate caCert = (X509Certificate) certChain[1];
+    X509Certificate cert = (X509Certificate) certChain[0];
+
+    // check child cert
+    Assert.assertEquals(caCert.getSubjectX500Principal().toString(),
+        cert.getIssuerDN().toString());
+    Assert.assertEquals(new X500Principal("CN=" + appId),
+        cert.getSubjectX500Principal());
+    Assert.assertFalse("Found multiple fields in X500 Principal, when there " +
+            "should have only been one: " + cert.getSubjectX500Principal(),
+        cert.getSubjectX500Principal().toString().contains(","));
+    Assert.assertEquals("SHA512withRSA", cert.getSigAlgName());
+    Assert.assertEquals(cert.getNotBefore(), cert.getNotAfter());
+    Assert.assertTrue("Expected certificate to be expired but was not: "
+        + cert.getNotAfter(), cert.getNotAfter().before(new Date()));
+    Assert.assertEquals(new X500Principal("CN=" + appId).toString(),
+        cert.getSubjectDN().toString());
+    Key privateKey = keyStore.getKey("server", "password".toCharArray());
+    Assert.assertEquals("RSA", privateKey.getAlgorithm());
+    Assert.assertEquals(-1, cert.getBasicConstraints());
+
+    // verify signature on child cert
+    PublicKey caPublicKey = caCert.getPublicKey();
+    cert.verify(caPublicKey);
+
+    // check CA cert
+    checkCACert(caCert);
+    Assert.assertEquals(proxyCA.getCaCert(), caCert);
+
+    // verify signature on CA cert
+    caCert.verify(caPublicKey);
+
+    // verify CA public key matches private key
+    PrivateKey caPrivateKey =
+        proxyCA.getX509KeyManager().getPrivateKey(null);
+    checkPrivatePublicKeys(caPrivateKey, caPublicKey);
+    Assert.assertEquals(proxyCA.getCaKeyPair().getPublic(), caPublicKey);
+    Assert.assertEquals(proxyCA.getCaKeyPair().getPrivate(), caPrivateKey);
+  }
+
+  @Test
+  public void testGetChildTrustStore() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    byte[] truststoreBytes = proxyCA.getChildTrustStore("password");
+    KeyStore truststore = KeyStoreTestUtil.bytesToKeyStore(truststoreBytes,
+        "password");
+    Assert.assertEquals(1, truststore.size());
+    X509Certificate caCert =
+        (X509Certificate) truststore.getCertificate("client");
+
+    // check CA cert
+    checkCACert(caCert);
+    Assert.assertEquals(proxyCA.getCaCert(), caCert);
+
+    // verify signature on CA cert
+    PublicKey caPublicKey = caCert.getPublicKey();
+    caCert.verify(caPublicKey);
+
+    // verify CA public key matches private key
+    PrivateKey caPrivateKey =
+        proxyCA.getX509KeyManager().getPrivateKey(null);
+    checkPrivatePublicKeys(caPrivateKey, caPublicKey);
+    Assert.assertEquals(proxyCA.getCaKeyPair().getPublic(), caPublicKey);
+    Assert.assertEquals(proxyCA.getCaKeyPair().getPrivate(), caPrivateKey);
+  }
+
+  @Test
+  public void testGenerateKeyStorePassword() throws Exception {
+    // We can't possibly test every possible string, but we can at least verify
+    // a few things about a few of the generated strings as a sanity check
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    Set<String> passwords = new HashSet<>();
+
+    for (int i = 0; i < 5; i++) {
+      String password = proxyCA.generateKeyStorePassword();
+      Assert.assertEquals(16, password.length());
+      for (char c : password.toCharArray()) {
+        Assert.assertFalse("Found character '" + c + "' in password '"
+            + password + "' which is outside of the expected range", c < ' ');
+        Assert.assertFalse("Found character '" + c + "' in password '"
+            + password + "' which is outside of the expected range", c > 'z');
+      }
+      Assert.assertFalse("Password " + password
+          + " was generated twice, which is _extremely_ unlikely"
+          + " and shouldn't practically happen: " + passwords,
+          passwords.contains(password));
+      passwords.add(password);
+    }
+  }
+
+  @Test
+  public void testCreateTrustManagerDefaultTrustManager() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509TrustManager defaultTrustManager = Mockito.mock(X509TrustManager.class);
+    proxyCA.setDefaultTrustManager(defaultTrustManager);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    X509TrustManager trustManager = proxyCA.createTrustManager(appId);
+    Mockito.when(defaultTrustManager.getAcceptedIssuers()).thenReturn(
+        new X509Certificate[]{KeyStoreTestUtil.generateCertificate(
+            "CN=foo", KeyStoreTestUtil.generateKeyPair("RSA"), 30,
+            "SHA1withRSA")});
+
+    Assert.assertArrayEquals(defaultTrustManager.getAcceptedIssuers(),
+        trustManager.getAcceptedIssuers());
+    trustManager.checkClientTrusted(null, null);
+  }
+
+  @Test
+  public void testCreateTrustManagerYarnCert() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509TrustManager defaultTrustManager = Mockito.mock(X509TrustManager.class);
+    proxyCA.setDefaultTrustManager(defaultTrustManager);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    X509TrustManager trustManager = proxyCA.createTrustManager(appId);
+
+    X509Certificate[] certChain = castCertificateArrayToX509CertificateArray(
+        KeyStoreTestUtil.bytesToKeyStore(
+            proxyCA.createChildKeyStore(appId, "password"), "password")
+            .getCertificateChain("server"));
+    trustManager.checkServerTrusted(certChain, "RSA");
+    Mockito.verify(defaultTrustManager, Mockito.times(0))
+        .checkServerTrusted(certChain, "RSA");
+  }
+
+  @Test
+  public void testCreateTrustManagerWrongApp() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509TrustManager defaultTrustManager = Mockito.mock(X509TrustManager.class);
+    proxyCA.setDefaultTrustManager(defaultTrustManager);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationId appId2 =
+        ApplicationId.newInstance(System.currentTimeMillis(), 2);
+    X509TrustManager trustManager = proxyCA.createTrustManager(appId);
+
+    X509Certificate[] certChain = castCertificateArrayToX509CertificateArray(
+        KeyStoreTestUtil.bytesToKeyStore(
+            proxyCA.createChildKeyStore(appId2, "password"), "password")
+            .getCertificateChain("server"));
+    try {
+      trustManager.checkServerTrusted(certChain, "RSA");
+      Assert.fail("Should have thrown a CertificateException, but did not");
+    } catch (CertificateException ce) {
+      Assert.assertEquals("Expected to find Subject X500 Principal with CN=" +
+          appId + " but found CN=" + appId2, ce.getMessage());
+    }
+  }
+
+  @Test
+  public void testCreateTrustManagerWrongRM() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509TrustManager defaultTrustManager = Mockito.mock(X509TrustManager.class);
+    proxyCA.setDefaultTrustManager(defaultTrustManager);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    X509TrustManager trustManager = proxyCA.createTrustManager(appId);
+
+    ProxyCA proxyCA2 = new ProxyCA(); // Simulates another RM
+    proxyCA2.init();
+    X509Certificate[] certChain = castCertificateArrayToX509CertificateArray(
+        KeyStoreTestUtil.bytesToKeyStore(
+            proxyCA2.createChildKeyStore(appId, "password"), "password")
+            .getCertificateChain("server"));
+    Mockito.verify(defaultTrustManager, Mockito.times(0))
+        .checkServerTrusted(certChain, "RSA");
+    trustManager.checkServerTrusted(certChain, "RSA");
+    Mockito.verify(defaultTrustManager, Mockito.times(1))
+        .checkServerTrusted(certChain, "RSA");
+  }
+
+  @Test
+  public void testCreateTrustManagerRealCert() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509TrustManager defaultTrustManager = Mockito.mock(X509TrustManager.class);
+    proxyCA.setDefaultTrustManager(defaultTrustManager);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    X509TrustManager trustManager = proxyCA.createTrustManager(appId);
+
+    // "real" cert
+    X509Certificate[]
+        certChain = new X509Certificate[]{
+        KeyStoreTestUtil.generateCertificate("CN=foo.com",
+            KeyStoreTestUtil.generateKeyPair("RSA"), 30, "SHA1withRSA")};
+    Mockito.verify(defaultTrustManager, Mockito.times(0))
+        .checkServerTrusted(certChain, "RSA");
+    trustManager.checkServerTrusted(certChain, "RSA");
+    Mockito.verify(defaultTrustManager, Mockito.times(1))
+        .checkServerTrusted(certChain, "RSA");
+
+    // "real" cert x2
+    certChain = new X509Certificate[]{
+        KeyStoreTestUtil.generateCertificate("CN=foo.com",
+            KeyStoreTestUtil.generateKeyPair("RSA"), 30, "SHA1withRSA"),
+        KeyStoreTestUtil.generateCertificate("CN=foo.com",
+            KeyStoreTestUtil.generateKeyPair("RSA"), 30, "SHA1withRSA")};
+    Mockito.verify(defaultTrustManager, Mockito.times(0))
+        .checkServerTrusted(certChain, "RSA");
+    trustManager.checkServerTrusted(certChain, "RSA");
+    Mockito.verify(defaultTrustManager, Mockito.times(1))
+        .checkServerTrusted(certChain, "RSA");
+  }
+
+  @Test
+  public void testCreateTrustManagerExceptions() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509TrustManager defaultTrustManager = Mockito.mock(X509TrustManager.class);
+    proxyCA.setDefaultTrustManager(defaultTrustManager);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    X509TrustManager trustManager = proxyCA.createTrustManager(appId);
+
+    for (Exception e : new Exception[]{
+        new CertificateException(), new NoSuchAlgorithmException(),
+        new InvalidKeyException(), new SignatureException(),
+        new NoSuchProviderException()}) {
+      X509Certificate[] certChain = castCertificateArrayToX509CertificateArray(
+          KeyStoreTestUtil.bytesToKeyStore(
+              proxyCA.createChildKeyStore(appId, "password"), "password")
+              .getCertificateChain("server"));
+      X509Certificate cert = Mockito.spy(certChain[0]);
+      certChain[0] = cert;
+      // Throw e to simulate problems with verifying
+      Mockito.doThrow(e).when(certChain[0]).verify(Mockito.any());
+      Mockito.verify(defaultTrustManager, Mockito.times(0))
+          .checkServerTrusted(certChain, "RSA");
+      trustManager.checkServerTrusted(certChain, "RSA");
+      Mockito.verify(defaultTrustManager, Mockito.times(1))
+          .checkServerTrusted(certChain, "RSA");
+    }
+  }
+
+  @Test
+  public void testCreateKeyManager() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    X509KeyManager keyManager = proxyCA.getX509KeyManager();
+
+    Assert.assertArrayEquals(new String[]{"client"},
+        keyManager.getClientAliases(null, null));
+    Assert.assertEquals("client",
+        keyManager.chooseClientAlias(null, null, null));
+    Assert.assertNull(keyManager.getServerAliases(null, null));
+    Assert.assertNull(keyManager.chooseServerAlias(null, null, null));
+
+    byte[] truststoreBytes = proxyCA.getChildTrustStore("password");
+    KeyStore truststore = KeyStoreTestUtil.bytesToKeyStore(truststoreBytes,
+        "password");
+    Assert.assertEquals(1, truststore.size());
+    X509Certificate caCert =
+        (X509Certificate) truststore.getCertificate("client");
+    Assert.assertArrayEquals(new X509Certificate[]{caCert},
+        keyManager.getCertificateChain(null));
+    Assert.assertEquals(proxyCA.getCaCert(), caCert);
+
+    PrivateKey caPrivateKey = keyManager.getPrivateKey(null);
+    PublicKey caPublicKey = caCert.getPublicKey();
+    checkPrivatePublicKeys(caPrivateKey, caPublicKey);
+    Assert.assertEquals(proxyCA.getCaKeyPair().getPublic(), caPublicKey);
+    Assert.assertEquals(proxyCA.getCaKeyPair().getPrivate(), caPrivateKey);
+  }
+
+  @Test
+  public void testCreateHostnameVerifier() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    HostnameVerifier verifier = proxyCA.getHostnameVerifier();
+
+    SSLSession sslSession = Mockito.mock(SSLSession.class);
+    Mockito.when(sslSession.getPeerCertificates()).thenReturn(
+        KeyStoreTestUtil.bytesToKeyStore(
+            proxyCA.createChildKeyStore(
+                ApplicationId.newInstance(System.currentTimeMillis(), 1),
+                "password"), "password").getCertificateChain("server"));
+    Assert.assertTrue(verifier.verify("foo", sslSession));
+  }
+
+  @Test
+  public void testCreateHostnameVerifierSSLPeerUnverifiedException()
+      throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    HostnameVerifier verifier = proxyCA.getHostnameVerifier();
+
+    SSLSession sslSession = Mockito.mock(SSLSession.class);
+    Mockito.when(sslSession.getPeerCertificates()).thenThrow(
+        new SSLPeerUnverifiedException(""));
+    Assert.assertFalse(verifier.verify("foo", sslSession));
+  }
+
+  @Test
+  public void testCreateHostnameVerifierWrongRM() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    HostnameVerifier verifier = proxyCA.getHostnameVerifier();
+
+    SSLSession sslSession = Mockito.mock(SSLSession.class);
+    ProxyCA proxyCA2 = new ProxyCA(); // Simulate another RM
+    proxyCA2.init();
+    Mockito.when(sslSession.getPeerCertificates()).thenReturn(
+        KeyStoreTestUtil.bytesToKeyStore(
+            proxyCA2.createChildKeyStore(
+                ApplicationId.newInstance(System.currentTimeMillis(), 1),
+                "password"), "password").getCertificateChain("server"));
+    Assert.assertFalse(verifier.verify("foo", sslSession));
+  }
+
+  @Test
+  public void testCreateHostnameVerifierExceptions() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    HostnameVerifier verifier = proxyCA.getHostnameVerifier();
+
+    for (Exception e : new Exception[]{
+        new CertificateException(), new NoSuchAlgorithmException(),
+        new InvalidKeyException(), new SignatureException(),
+        new NoSuchProviderException()}) {
+      SSLSession sslSession = Mockito.mock(SSLSession.class);
+      Mockito.when(sslSession.getPeerCertificates()).thenAnswer(
+          new Answer<Certificate[]>() {
+            @Override
+            public Certificate[] answer(InvocationOnMock invocation)
+                throws Throwable {
+              Certificate[] certChain = KeyStoreTestUtil.bytesToKeyStore(
+                  proxyCA.createChildKeyStore(
+                      ApplicationId.newInstance(System.currentTimeMillis(), 1),
+                      "password"), "password").getCertificateChain("server");
+              Certificate cert = Mockito.spy(certChain[0]);
+              certChain[0] = cert;
+              // Throw e to simulate problems with verifying
+              Mockito.doThrow(e).when(cert).verify(Mockito.any());
+              return certChain;
+            }
+          });
+      Assert.assertFalse(verifier.verify("foo", sslSession));
+    }
+  }
+
+  @Test
+  public void testCreateHostnameVerifierRealCert() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    HostnameVerifier verifier = proxyCA.getHostnameVerifier();
+
+    SSLSession sslSession = Mockito.mock(SSLSession.class);
+    Mockito.when(sslSession.getPeerCertificates()).thenAnswer(
+        new Answer<Certificate[]>() {
+          @Override
+          public Certificate[] answer(InvocationOnMock invocation)
+              throws Throwable {
+            // "real" cert
+            Certificate[] certChain = new Certificate[]{
+                KeyStoreTestUtil.generateCertificate("CN=foo.com",
+                    KeyStoreTestUtil.generateKeyPair("RSA"), 30, "SHA1withRSA")
+            };
+            return certChain;
+          }
+        });
+    Assert.assertTrue(verifier.verify("foo.com", sslSession));
+  }
+
+  @Test
+  public void testCreateHostnameVerifierRealCertBad() throws Exception {
+    ProxyCA proxyCA = new ProxyCA();
+    proxyCA.init();
+    HostnameVerifier verifier = proxyCA.getHostnameVerifier();
+
+    SSLSession sslSession = Mockito.mock(SSLSession.class);
+    Mockito.when(sslSession.getPeerCertificates()).thenAnswer(
+        new Answer<Certificate[]>() {
+          @Override
+          public Certificate[] answer(InvocationOnMock invocation)
+              throws Throwable {
+            // "real" cert
+            Certificate[] certChain = new Certificate[]{
+                KeyStoreTestUtil.generateCertificate("CN=foo.com",
+                    KeyStoreTestUtil.generateKeyPair("RSA"), 30, "SHA1withRSA")
+            };
+            return certChain;
+          }
+        });
+    Assert.assertFalse(verifier.verify("bar.com", sslSession));
+  }
+
+  private void checkCACert(X509Certificate caCert) {
+    Assert.assertEquals(caCert.getSubjectX500Principal().toString(),
+        caCert.getIssuerDN().toString());
+    Assert.assertEquals(caCert.getSubjectX500Principal().toString(),
+        caCert.getSubjectDN().toString());
+    Assert.assertTrue("Expected CA certificate X500 Principal to start with" +
+            " 'OU=YARN-', but did not: " + caCert.getSubjectX500Principal(),
+        caCert.getSubjectX500Principal().toString().startsWith("OU=YARN-"));
+    Assert.assertFalse("Found multiple fields in X500 Principal, when there " +
+            "should have only been one: " + caCert.getSubjectX500Principal(),
+        caCert.getSubjectX500Principal().toString().contains(","));
+    Assert.assertEquals("SHA512withRSA", caCert.getSigAlgName());
+    Assert.assertEquals(
+        new GregorianCalendar(2037, Calendar.DECEMBER, 31).getTime(),
+        caCert.getNotAfter());
+    Assert.assertTrue("Expected certificate to have started but was not: "
+        + caCert.getNotBefore(), caCert.getNotBefore().before(new Date()));
+    Assert.assertEquals(0, caCert.getBasicConstraints());
+  }
+
+  private void checkPrivatePublicKeys(PrivateKey privateKey,
+      PublicKey publicKey) throws NoSuchAlgorithmException, InvalidKeyException,
+      SignatureException {
+    byte[] data = new byte[2000];
+    new Random().nextBytes(data);
+    Signature signer = Signature.getInstance("SHA512withRSA");
+    signer.initSign(privateKey);
+    signer.update(data);
+    byte[] sig = signer.sign();
+    signer = Signature.getInstance("SHA512withRSA");
+    signer.initVerify(publicKey);
+    signer.update(data);
+    Assert.assertTrue(signer.verify(sig));
+  }
+
+  private X509Certificate[] castCertificateArrayToX509CertificateArray(
+      Certificate[] certs) {
+    return Arrays.copyOf(certs, certs.length, X509Certificate[].class);
+  }
+}

+ 54 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java

@@ -27,6 +27,8 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.net.ConnectException;
 import java.net.HttpCookie;
 import java.net.HttpURLConnection;
@@ -54,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.servlet.ServletContextHandler;
@@ -63,6 +66,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.servlet.ServletHolder;
+import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -333,12 +337,13 @@ public class TestWebAppProxyServlet {
       assertEquals(proxyConn.getRequestProperties().size(), 4);
       proxyConn.connect();
       assertEquals(HttpURLConnection.HTTP_OK, proxyConn.getResponseCode());
-      // Verify if number of headers received by end server is 8.
-      // Eight headers include Accept, Host, Connection, User-Agent, Cookie,
-      // Origin, Access-Control-Request-Method and
+      // Verify if number of headers received by end server is 9.
+      // This should match WebAppProxyServlet#PASS_THROUGH_HEADERS.
+      // Nine headers include Accept, Host, Connection, User-Agent, Cookie,
+      // Origin, Access-Control-Request-Method, Accept-Encoding, and
       // Access-Control-Request-Headers. Pls note that Unknown-Header is dropped
       // by proxy as it is not in the list of allowed headers.
-      assertEquals(numberOfHeaders, 8);
+      assertEquals(numberOfHeaders, 9);
       assertFalse(hasUnknownHeader);
     } finally {
       proxy.close();
@@ -383,6 +388,51 @@ public class TestWebAppProxyServlet {
     }
   }
 
+  @Test(timeout=5000)
+  public void testCheckHttpsStrictAndNotProvided() throws Exception {
+    HttpServletResponse resp = Mockito.mock(HttpServletResponse.class);
+    StringWriter sw = new StringWriter();
+    Mockito.when(resp.getWriter()).thenReturn(new PrintWriter(sw));
+    YarnConfiguration conf = new YarnConfiguration();
+    final URI httpLink = new URI("http://foo.com");
+    final URI httpsLink = new URI("https://foo.com");
+
+    // NONE policy
+    conf.set(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY, "NONE");
+    assertFalse(WebAppProxyServlet.checkHttpsStrictAndNotProvided(
+        resp, httpsLink, conf));
+    assertEquals("", sw.toString());
+    Mockito.verify(resp, Mockito.times(0)).setContentType(Mockito.any());
+    assertFalse(WebAppProxyServlet.checkHttpsStrictAndNotProvided(
+        resp, httpLink, conf));
+    assertEquals("", sw.toString());
+    Mockito.verify(resp, Mockito.times(0)).setContentType(Mockito.any());
+
+    // LENIENT policy
+    conf.set(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY, "LENIENT");
+    assertFalse(WebAppProxyServlet.checkHttpsStrictAndNotProvided(
+        resp, httpsLink, conf));
+    assertEquals("", sw.toString());
+    Mockito.verify(resp, Mockito.times(0)).setContentType(Mockito.any());
+    assertFalse(WebAppProxyServlet.checkHttpsStrictAndNotProvided(
+        resp, httpLink, conf));
+    assertEquals("", sw.toString());
+    Mockito.verify(resp, Mockito.times(0)).setContentType(Mockito.any());
+
+    // STRICT policy
+    conf.set(YarnConfiguration.RM_APPLICATION_HTTPS_POLICY, "STRICT");
+    assertFalse(WebAppProxyServlet.checkHttpsStrictAndNotProvided(
+        resp, httpsLink, conf));
+    assertEquals("", sw.toString());
+    Mockito.verify(resp, Mockito.times(0)).setContentType(Mockito.any());
+    assertTrue(WebAppProxyServlet.checkHttpsStrictAndNotProvided(
+        resp, httpLink, conf));
+    String s = sw.toString();
+    assertTrue("Was expecting an HTML page explaining that an HTTPS tracking" +
+        " url must be used but found " + s, s.contains("HTTPS must be used"));
+    Mockito.verify(resp, Mockito.times(1)).setContentType(MimeType.HTML);
+  }
+
   private String readInputStream(InputStream input) throws Exception {
     ByteArrayOutputStream data = new ByteArrayOutputStream();
     byte[] buffer = new byte[512];