Browse Source

YARN-5076. YARN web interfaces lack XFS protection. Contributed by Jonathan Maron.
(cherry picked from commit 2703ec68712279494d67b0d76b7ac10e7a1628be)

Junping Du 9 years ago
parent
commit
22fcd819f0
10 changed files with 265 additions and 15 deletions
  1. 8 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
  2. 8 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  3. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
  4. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  5. 40 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
  6. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  7. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  8. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java
  9. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  10. 158 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java

+ 8 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java

@@ -238,6 +238,14 @@ public class JHAdminConfig {
   public static final String MR_HISTORY_METHODS_TO_IGNORE =
       MR_HISTORY_CSRF_PREFIX + "methods-to-ignore";
 
+  /**
+   * XFS settings.
+   */
+  public static final String MR_HISTORY_XFS_PREFIX = MR_HISTORY_PREFIX +
+      "webapp.xfs-filter.";
+  public static final String MR_HISTORY_XFS_OPTIONS = MR_HISTORY_XFS_PREFIX +
+      "xframe-options";
+
   /**
    * Settings for .jhist file format.
    */

+ 8 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -1888,4 +1888,12 @@
   <value>GET,OPTIONS,HEAD</value>
 </property>
 
+<property>
+  <description>
+    Value of the xframe-options
+  </description>
+  <name>mapreduce.jobhistory.webapp.xfs-filter.xframe-options</name>
+  <value>SAMEORIGIN</value>
+</property>
+
 </configuration>

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java

@@ -161,6 +161,7 @@ public class HistoryClientService extends AbstractService {
         .withHttpSpnegoPrincipalKey(
             JHAdminConfig.MR_WEBAPP_SPNEGO_USER_NAME_KEY)
         .withCSRFProtection(JHAdminConfig.MR_HISTORY_CSRF_PREFIX)
+        .withXFSProtection(JHAdminConfig.MR_HISTORY_XFS_PREFIX)
         .at(NetUtils.getHostPortString(bindAddress)).start(webApp);
     
     String connectHost = MRWebAppUtil.getJHSWebappURLWithoutScheme(conf).split(":")[0];

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

@@ -2565,6 +2565,19 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_CSRF_METHODS_TO_IGNORE =
       TIMELINE_CSRF_PREFIX + "methods-to-ignore";
 
+  // RM and NM XFS props
+  public static final String XFS = "webapp.xfs-filter.";
+  public static final String YARN_XFS_ENABLED = "yarn." + XFS + "enabled";
+  public static final String RM_XFS_PREFIX = RM_PREFIX + XFS;
+  public static final String NM_XFS_PREFIX = NM_PREFIX + XFS;
+  public static final String TIMELINE_XFS_PREFIX = TIMELINE_SERVICE_PREFIX +
+      XFS;
+  public static final String RM_XFS_OPTIONS = RM_XFS_PREFIX +
+      "xframe-options";
+  public static final String NM_XFS_OPTIONS = NM_XFS_PREFIX +
+      "xframe-options";
+  public static final String TIMELINE_XFS_OPTIONS =
+      TIMELINE_XFS_PREFIX + "xframe-options";
 
   public YarnConfiguration() {
     super();

+ 40 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.http.RestCsrfPreventionFilter;
+import org.apache.hadoop.security.http.XFrameOptionsFilter;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.slf4j.Logger;
@@ -93,7 +94,8 @@ public class WebApps {
     boolean devMode = false;
     private String spnegoPrincipalKey;
     private String spnegoKeytabKey;
-    private String configPrefix;
+    private String csrfConfigPrefix;
+    private String xfsConfigPrefix;
     private final HashSet<ServletStruct> servlets = new HashSet<ServletStruct>();
     private final HashMap<String, Object> attributes = new HashMap<String, Object>();
 
@@ -166,13 +168,25 @@ public class WebApps {
 
     /**
      * Enable the CSRF filter.
-     * @param csrfConfigPrefix The config prefix that identifies the
+     * @param prefix The config prefix that identifies the
      *                         CSRF parameters applicable for this filter
      *                         instance.
      * @return the Builder instance
      */
-    public Builder<T> withCSRFProtection(String csrfConfigPrefix) {
-      this.configPrefix = csrfConfigPrefix;
+    public Builder<T> withCSRFProtection(String prefix) {
+      this.csrfConfigPrefix = prefix;
+      return this;
+    }
+
+    /**
+     * Enable the XFS filter.
+     * @param prefix The config prefix that identifies the
+     *                         XFS parameters applicable for this filter
+     *                         instance.
+     * @return the Builder instance
+     */
+    public Builder<T> withXFSProtection(String prefix) {
+      this.xfsConfigPrefix = prefix;
       return this;
     }
 
@@ -281,7 +295,7 @@ public class WebApps {
         for(Map.Entry<String, Object> entry : attributes.entrySet()) {
           server.setAttribute(entry.getKey(), entry.getValue());
         }
-        Map<String, String> params = getCsrfConfigParameters();
+        Map<String, String> params = getConfigParameters(csrfConfigPrefix);
 
         if (hasCSRFEnabled(params)) {
           LOG.info("CSRF Protection has been enabled for the {} application. "
@@ -294,6 +308,15 @@ public class WebApps {
                                    new String[] {"/*"});
         }
 
+        params = getConfigParameters(xfsConfigPrefix);
+
+        if (hasXFSEnabled()) {
+          String xfsClassName = XFrameOptionsFilter.class.getName();
+          HttpServer2.defineFilter(server.getWebAppContext(), xfsClassName,
+              xfsClassName, params,
+              new String[] {"/*"});
+        }
+
         HttpServer2.defineFilter(server.getWebAppContext(), "guice",
           GuiceFilter.class.getName(), null, new String[] { "/*" });
 
@@ -327,14 +350,18 @@ public class WebApps {
       return params != null && Boolean.valueOf(params.get("enabled"));
     }
 
-    private Map<String, String> getCsrfConfigParameters() {
-      Map<String, String> params = null;
-      if (configPrefix != null) {
-        // need to obtain parameters for CSRF filter
-        params =
-            RestCsrfPreventionFilter.getFilterParams(conf, configPrefix);
-      }
-      return params;
+    /**
+     * XFS filter is enabled by default.  If the enabled flag is not explicitly
+     * specified and set to "false", this method returns true.
+     * @return true if XFS is enabled, false otherwise.
+     */
+    private boolean hasXFSEnabled() {
+      return conf.getBoolean(YarnConfiguration.YARN_XFS_ENABLED, true);
+    }
+
+    private Map<String, String> getConfigParameters(String configPrefix) {
+      return configPrefix != null ? conf.getPropsWithPrefix(configPrefix) :
+          null;
     }
 
     public WebApp start() {

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

@@ -2753,6 +2753,38 @@
     <value>GET,OPTIONS,HEAD</value>
   </property>
 
+  <property>
+    <description>
+      Enable the XFS filter for YARN
+    </description>
+    <name>yarn.webapp.xfs-filter.enabled</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>
+      Property specifying the xframe options value.
+    </description>
+    <name>yarn.resourcemanager.webapp.xfs-filter.xframe-options</name>
+    <value>SAMEORIGIN</value>
+  </property>
+
+  <property>
+    <description>
+      Property specifying the xframe options value.
+    </description>
+    <name>yarn.nodemanager.webapp.xfs-filter.xframe-options</name>
+    <value>SAMEORIGIN</value>
+  </property>
+
+  <property>
+    <description>
+      Property specifying the xframe options value.
+    </description>
+    <name>yarn.timeline-service.webapp.xfs-filter.xframe-options</name>
+    <value>SAMEORIGIN</value>
+  </property>
+
   <property>
     <description>
     The least amount of time(msec.) an inactive (decommissioned or shutdown) node can

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java

@@ -308,6 +308,7 @@ public class ApplicationHistoryServer extends CompositeService {
               .withAttribute(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
                  conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS))
               .withCSRFProtection(YarnConfiguration.TIMELINE_CSRF_PREFIX)
+              .withXFSProtection(YarnConfiguration.TIMELINE_XFS_PREFIX)
               .at(bindAddress).build(ahsWebApp);
        HttpServer2 httpServer = webApp.httpServer();
 

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java

@@ -78,8 +78,9 @@ public class WebServer extends AbstractService {
             .withHttpSpnegoPrincipalKey(
               YarnConfiguration.NM_WEBAPP_SPNEGO_USER_NAME_KEY)
             .withHttpSpnegoKeytabKey(
-              YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
-            .withCSRFProtection(YarnConfiguration.NM_CSRF_PREFIX)
+                YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
+              .withCSRFProtection(YarnConfiguration.NM_CSRF_PREFIX)
+              .withXFSProtection(YarnConfiguration.NM_XFS_PREFIX)
             .start(this.nmWebApp);
       this.port = this.webApp.httpServer().getConnectorAddress(0).getPort();
     } catch (Exception e) {

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

@@ -968,6 +968,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
             .withHttpSpnegoKeytabKey(
                 YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
             .withCSRFProtection(YarnConfiguration.RM_CSRF_PREFIX)
+            .withXFSProtection(YarnConfiguration.RM_XFS_PREFIX)
             .at(webAppAddress);
     String proxyHostAndPort = WebAppUtils.getProxyHostAndPort(conf);
     if(WebAppUtils.getResolvedRMWebAppURLWithoutScheme(conf).

+ 158 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java

@@ -0,0 +1,158 @@
+/**
+ * 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.webapp;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.servlet.GuiceServletContextListener;
+import com.google.inject.servlet.ServletModule;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.test.framework.WebAppDescriptor;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.http.XFrameOptionsFilter;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Used TestRMWebServices as an example of web invocations of RM and added
+ * test for XFS Filter.
+ */
+public class TestRMWithXFSFilter extends JerseyTestBase {
+
+  private static MockRM rm;
+
+  private Injector injector;
+
+  /**
+   *
+   */
+  public class GuiceServletConfig extends GuiceServletContextListener {
+
+    @Override
+    protected Injector getInjector() {
+      return injector;
+    }
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  public TestRMWithXFSFilter() {
+    super(new WebAppDescriptor.Builder(
+        "org.apache.hadoop.yarn.server.resourcemanager.webapp")
+        .contextListenerClass(GuiceServletConfig.class)
+        .filterClass(com.google.inject.servlet.GuiceFilter.class)
+        .contextPath("jersey-guice-filter").servletPath("/").build());
+  }
+
+  @Test
+  public void testDefaultBehavior() throws Exception {
+    createInjector();
+
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info").accept("application/xml")
+        .get(ClientResponse.class);
+    assertTrue("Should have received DENY x-frame options header",
+        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS).get(0)
+            .equals("DENY"));
+  }
+
+  protected void createInjector(String headerValue) {
+    createInjector(headerValue, false);
+  }
+
+
+  protected void createInjector() {
+    createInjector(null, false);
+  }
+
+  protected void createInjector(final String headerValue,
+      final boolean explicitlyDisabled) {
+    injector = Guice.createInjector(new ServletModule() {
+      @Override
+      protected void configureServlets() {
+        bind(JAXBContextResolver.class);
+        bind(RMWebServices.class);
+        bind(GenericExceptionHandler.class);
+        Configuration conf = new Configuration();
+        conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
+            ResourceScheduler.class);
+        rm = new MockRM(conf);
+        bind(ResourceManager.class).toInstance(rm);
+        serve("/*").with(GuiceContainer.class);
+        XFrameOptionsFilter xfsFilter = new XFrameOptionsFilter();
+        Map<String, String> initParams = new HashMap<>();
+        if (headerValue != null) {
+          initParams.put(XFrameOptionsFilter.CUSTOM_HEADER_PARAM, headerValue);
+        }
+        if (explicitlyDisabled) {
+          initParams.put(
+              "xframe-options-enabled", "false");
+        }
+
+        filter("/*").through(xfsFilter, initParams);
+      }
+    });
+  }
+
+  @Test
+  public void testSameOrigin() throws Exception {
+    createInjector("SAMEORIGIN");
+
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info").accept("application/xml")
+        .get(ClientResponse.class);
+    assertTrue("Should have received SAMEORIGIN x-frame options header",
+        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS).get(0)
+            .equals("SAMEORIGIN"));
+  }
+
+  @Test
+  public void testExplicitlyDisabled() throws Exception {
+    createInjector(null, true);
+
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info").accept("application/xml")
+        .get(ClientResponse.class);
+    assertFalse("Should have not received x-frame options header",
+        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS) == null);
+  }
+
+}