Pārlūkot izejas kodu

YARN-3147. Clean up RM web proxy code. Contributed by Steve Loughran

(cherry picked from commit 83be450acc7fc9bb9f7bbd006e7b0804bf10279c)
Xuan 10 gadi atpakaļ
vecāks
revīzija
58d9ce2ea1
10 mainītis faili ar 324 papildinājumiem un 154 dzēšanām
  1. 2 0
      hadoop-yarn-project/CHANGES.txt
  2. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
  3. 127 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUtils.java
  4. 8 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
  5. 8 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
  6. 74 57
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
  7. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmFilterInitializer.java
  8. 23 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmIpFilter.java
  9. 38 46
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
  10. 37 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java

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

@@ -238,6 +238,8 @@ Release 2.7.0 - UNRELEASED
     YARN-3157. Refactor the exception handling in ConverterUtils#to*Id.
     (Bibin A Chundatt via ozawa)
 
+    YARN-3147. Clean up RM web proxy code. (Steve Loughran via xgong)
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java

@@ -18,11 +18,10 @@
 
 package org.apache.hadoop.yarn.server.webproxy;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.util.TrackingUriPlugin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.UnsupportedEncodingException;
 import java.net.URI;
@@ -34,8 +33,9 @@ import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
 
 public class ProxyUriUtils {
   @SuppressWarnings("unused")
-  private static final Log LOG = LogFactory.getLog(ProxyUriUtils.class);
-  
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ProxyUriUtils.class);
+
   /**Name of the servlet to use when registering the proxy servlet. */
   public static final String PROXY_SERVLET_NAME = "proxy";
   /**Base path where the proxy servlet will handle requests.*/
@@ -194,4 +194,5 @@ public class ProxyUriUtils {
       return "";
     }
   }
+
 }

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

@@ -0,0 +1,127 @@
+/*
+ * 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.yarn.webapp.MimeType;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.EnumSet;
+
+/**
+ * Class containing general purpose proxy utilities
+ */
+public class ProxyUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ProxyUtils.class);
+  public static final String E_HTTP_HTTPS_ONLY =
+      "This filter only works for HTTP/HTTPS";
+  public static final String LOCATION = "Location";
+
+  public static class _ implements Hamlet._ {
+    //Empty
+  }
+
+  public static class Page extends Hamlet {
+    Page(PrintWriter out) {
+      super(out, 0, false);
+    }
+
+    public HTML<ProxyUtils._> html() {
+      return new HTML<>("html", null, EnumSet.of(EOpt.ENDTAG));
+    }
+  }
+  
+  /**
+   * Handle redirects with a status code that can in future support verbs other
+   * than GET, thus supporting full REST functionality.
+   * <p>
+   * The target URL is included in the redirect text returned
+   * <p>
+   * At the end of this method, the output stream is closed.
+   * 
+   * @param request request (hence: the verb and any other information
+   * relevant to a redirect)
+   * @param response the response
+   * @param target the target URL -unencoded
+   *
+   */
+  public static void sendRedirect(HttpServletRequest request,
+      HttpServletResponse response,
+      String target)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Redirecting {} {} to {}",
+          request.getMethod(), 
+          request.getRequestURI(),
+          target);
+    }
+    String location = response.encodeRedirectURL(target);
+    response.setStatus(HttpServletResponse.SC_FOUND);
+    response.setHeader(LOCATION, location);
+    response.setContentType(MimeType.HTML);
+    PrintWriter writer = response.getWriter();
+    Page p = new Page(writer);
+    p.html()
+        .head().title("Moved")._()
+        .body()
+        .h1("Moved")
+        .div()
+          ._("Content has moved ")
+          .a(location, "here")._()
+        ._()._();
+    writer.close();
+  }
+
+
+  /**
+   * Output 404 with appropriate message.
+   * @param resp the http response.
+   * @param message the message to include on the page.
+   * @throws IOException on any error.
+   */
+  public static void notFound(HttpServletResponse resp, String message)
+      throws IOException {
+    resp.setStatus(HttpServletResponse.SC_NOT_FOUND);
+    resp.setContentType(MimeType.HTML);
+    Page p = new Page(resp.getWriter());
+    p.html().
+        h1(message).
+         _();
+  }
+
+  /**
+   * Reject any request that isn't from an HTTP servlet
+   * @param req request
+   * @throws ServletException if the request is of the wrong type
+   */
+  public static void rejectNonHttpRequests(ServletRequest req) throws
+      ServletException {
+    if (!(req instanceof HttpServletRequest)) {
+      throw new ServletException(E_HTTP_HTTPS_ONLY);
+    }
+  }
+}

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

@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.webproxy;
 import java.io.IOException;
 import java.net.URI;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authorize.AccessControlList;
@@ -34,12 +32,15 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 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";
-  private static final Log LOG = LogFactory.getLog(WebAppProxy.class);
+  private static final Logger LOG = LoggerFactory.getLogger(
+      WebAppProxy.class);
   
   private HttpServer2 proxyServer = null;
   private String bindAddress = null;
@@ -109,8 +110,8 @@ public class WebAppProxy extends AbstractService {
       proxyServer.setAttribute(PROXY_HOST_ATTRIBUTE, proxyHost);
       proxyServer.start();
     } catch (IOException e) {
-      LOG.fatal("Could not start proxy web server",e);
-      throw new YarnRuntimeException("Could not start proxy web server",e);
+      LOG.error("Could not start proxy web server",e);
+      throw e;
     }
     super.serviceStart();
   }
@@ -121,7 +122,7 @@ public class WebAppProxy extends AbstractService {
       try {
         proxyServer.stop();
       } catch (Exception e) {
-        LOG.fatal("Error stopping proxy web server", e);
+        LOG.error("Error stopping proxy web server", e);
         throw new YarnRuntimeException("Error stopping proxy web server",e);
       }
     }
@@ -136,6 +137,7 @@ public class WebAppProxy extends AbstractService {
       try {
         proxyServer.join();
       } catch (InterruptedException e) {
+        // ignored
       }
     }
   }

+ 8 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java

@@ -21,16 +21,16 @@ package org.apache.hadoop.yarn.server.webproxy;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * ProxyServer will sit in between the end user and AppMaster
@@ -43,8 +43,9 @@ public class WebAppProxyServer extends CompositeService {
    */
   public static final int SHUTDOWN_HOOK_PRIORITY = 30;
 
-  private static final Log LOG = LogFactory.getLog(WebAppProxyServer.class);
-  
+  private static final Logger LOG = LoggerFactory.getLogger(
+      WebAppProxyServer.class);
+
   private WebAppProxy proxy = null;
   
   public WebAppProxyServer() {
@@ -54,11 +55,7 @@ public class WebAppProxyServer extends CompositeService {
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
     Configuration config = new YarnConfiguration(conf);
-    try {
-      doSecureLogin(conf);      
-    } catch(IOException ie) {
-      throw new YarnRuntimeException("Proxy Server Failed to login", ie);
-    }
+    doSecureLogin(conf);      
     proxy = new WebAppProxy();
     addService(proxy);
     super.serviceInit(config);
@@ -95,8 +92,7 @@ public class WebAppProxyServer extends CompositeService {
       WebAppProxyServer proxyServer = startServer(configuration);
       proxyServer.proxy.join();
     } catch (Throwable t) {
-      LOG.fatal("Error starting Proxy server", t);
-      System.exit(-1);
+      ExitUtil.terminate(-1, t);
     }
   }
 

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

@@ -32,6 +32,7 @@ import java.util.EnumSet;
 import java.util.Enumeration;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import javax.servlet.http.Cookie;
 import javax.servlet.http.HttpServlet;
@@ -45,8 +46,6 @@ import org.apache.commons.httpclient.HttpMethod;
 import org.apache.commons.httpclient.cookie.CookiePolicy;
 import org.apache.commons.httpclient.methods.GetMethod;
 import org.apache.commons.httpclient.params.HttpClientParams;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -59,13 +58,20 @@ import org.apache.hadoop.yarn.util.TrackingUriPlugin;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class WebAppProxyServlet extends HttpServlet {
   private static final long serialVersionUID = 1L;
-  private static final Log LOG = LogFactory.getLog(WebAppProxyServlet.class);
-  private static final HashSet<String> passThroughHeaders = 
-    new HashSet<String>(Arrays.asList("User-Agent", "Accept", "Accept-Encoding",
-        "Accept-Language", "Accept-Charset"));
+  private static final Logger LOG = LoggerFactory.getLogger(
+      WebAppProxyServlet.class);
+  private static final Set<String> passThroughHeaders = 
+    new HashSet<>(Arrays.asList(
+        "User-Agent",
+        "Accept",
+        "Accept-Encoding",
+        "Accept-Language",
+        "Accept-Charset"));
   
   public static final String PROXY_USER_COOKIE_NAME = "proxy-user";
 
@@ -83,15 +89,14 @@ public class WebAppProxyServlet extends HttpServlet {
     }
   
     public HTML<WebAppProxyServlet._> html() {
-      return new HTML<WebAppProxyServlet._>("html", null, EnumSet.of(EOpt.ENDTAG));
+      return new HTML<>("html", null, EnumSet.of(EOpt.ENDTAG));
     }
   }
 
   /**
    * Default constructor
    */
-  public WebAppProxyServlet()
-  {
+  public WebAppProxyServlet() {
     super();
     conf = new YarnConfiguration();
     this.trackingUriPlugins =
@@ -109,12 +114,7 @@ public class WebAppProxyServlet extends HttpServlet {
    */
   private static void notFound(HttpServletResponse resp, String message) 
     throws IOException {
-    resp.setStatus(HttpServletResponse.SC_NOT_FOUND);
-    resp.setContentType(MimeType.HTML);
-    Page p = new Page(resp.getWriter());
-    p.html().
-      h1(message).
-    _();
+    ProxyUtils.notFound(resp, message);
   }
   
   /**
@@ -133,7 +133,8 @@ public class WebAppProxyServlet extends HttpServlet {
     resp.setContentType(MimeType.HTML);
     Page p = new Page(resp.getWriter());
     p.html().
-      h1("WARNING: The following page may not be safe!").h3().
+      h1("WARNING: The following page may not be safe!").
+      h3().
       _("click ").a(link, "here").
       _(" to continue to an Application Master web interface owned by ", user).
       _().
@@ -163,7 +164,7 @@ public class WebAppProxyServlet extends HttpServlet {
     HostConfiguration config = new HostConfiguration();
     InetAddress localAddress = InetAddress.getByName(proxyHost);
     if (LOG.isDebugEnabled()) {
-      LOG.debug("local InetAddress for proxy host: " + localAddress.toString());
+      LOG.debug("local InetAddress for proxy host: {}", localAddress);
     }
     config.setLocalAddress(localAddress);
     HttpMethod method = new GetMethod(uri.getEscapedURI());
@@ -174,15 +175,17 @@ public class WebAppProxyServlet extends HttpServlet {
       String name = names.nextElement();
       if(passThroughHeaders.contains(name)) {
         String value = req.getHeader(name);
-        LOG.debug("REQ HEADER: "+name+" : "+value);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("REQ HEADER: {} : {}", name, value);
+        }
         method.setRequestHeader(name, value);
       }
     }
 
     String user = req.getRemoteUser();
-    if(user != null && !user.isEmpty()) {
-      method.setRequestHeader("Cookie",PROXY_USER_COOKIE_NAME+"="+
-          URLEncoder.encode(user, "ASCII"));
+    if (user != null && !user.isEmpty()) {
+      method.setRequestHeader("Cookie",
+          PROXY_USER_COOKIE_NAME + "=" + URLEncoder.encode(user, "ASCII"));
     }
     OutputStream out = resp.getOutputStream();
     try {
@@ -190,11 +193,11 @@ public class WebAppProxyServlet extends HttpServlet {
       for(Header header : method.getResponseHeaders()) {
         resp.setHeader(header.getName(), header.getValue());
       }
-      if(c != null) {
+      if (c != null) {
         resp.addCookie(c);
       }
       InputStream in = method.getResponseBodyAsStream();
-      if(in != null) {
+      if (in != null) {
         IOUtils.copyBytes(in, out, 4096, true);
       }
     } finally {
@@ -216,8 +219,7 @@ public class WebAppProxyServlet extends HttpServlet {
   private boolean isSecurityEnabled() {
     Boolean b = (Boolean) getServletContext()
         .getAttribute(WebAppProxy.IS_SECURITY_ENABLED_ATTRIBUTE);
-    if(b != null) return b;
-    return false;
+    return b != null ? b : false;
   }
   
   private ApplicationReport getApplicationReport(ApplicationId id)
@@ -238,15 +240,14 @@ public class WebAppProxyServlet extends HttpServlet {
       String userApprovedParamS = 
         req.getParameter(ProxyUriUtils.PROXY_APPROVAL_PARAM);
       boolean userWasWarned = false;
-      boolean userApproved = 
-        (userApprovedParamS != null && Boolean.valueOf(userApprovedParamS));
+      boolean userApproved = Boolean.valueOf(userApprovedParamS);
       boolean securityEnabled = isSecurityEnabled();
       final String remoteUser = req.getRemoteUser();
       final String pathInfo = req.getPathInfo();
 
-      String parts[] = pathInfo.split("/", 3);
+      String[] parts = pathInfo.split("/", 3);
       if(parts.length < 2) {
-        LOG.warn(remoteUser+" Gave an invalid proxy path "+pathInfo);
+        LOG.warn("{} gave an invalid proxy path {}", remoteUser,  pathInfo);
         notFound(resp, "Your path appears to be formatted incorrectly.");
         return;
       }
@@ -255,9 +256,9 @@ public class WebAppProxyServlet extends HttpServlet {
       String rest = parts.length > 2 ? parts[2] : "";
       ApplicationId id = Apps.toAppID(appId);
       if(id == null) {
-        LOG.warn(req.getRemoteUser()+" Attempting to access "+appId+
-        " that is invalid");
-        notFound(resp, appId+" appears to be formatted incorrectly.");
+        LOG.warn("{} attempting to access {} that is invalid",
+            remoteUser, appId);
+        notFound(resp, appId + " appears to be formatted incorrectly.");
         return;
       }
       
@@ -277,35 +278,34 @@ public class WebAppProxyServlet extends HttpServlet {
       
       boolean checkUser = securityEnabled && (!userWasWarned || !userApproved);
 
-      ApplicationReport applicationReport = null;
+      ApplicationReport applicationReport;
       try {
         applicationReport = getApplicationReport(id);
       } catch (ApplicationNotFoundException e) {
         applicationReport = null;
       }
       if(applicationReport == null) {
-        LOG.warn(req.getRemoteUser()+" Attempting to access "+id+
-            " that was not found");
+        LOG.warn("{} attempting to access {} that was not found",
+            remoteUser, id);
 
         URI toFetch =
             ProxyUriUtils
                 .getUriFromTrackingPlugins(id, this.trackingUriPlugins);
-        if (toFetch != null)
-        {
-          resp.sendRedirect(resp.encodeRedirectURL(toFetch.toString()));
+        if (toFetch != null) {
+          ProxyUtils.sendRedirect(req, resp, toFetch.toString());
           return;
         }
 
-        notFound(resp, "Application "+appId+" could not be found, " +
-        		"please try the history server");
+        notFound(resp, "Application " + appId + " could not be found, " +
+                       "please try the history server");
         return;
       }
       String original = applicationReport.getOriginalTrackingUrl();
-      URI trackingUri = null;
+      URI trackingUri;
       // fallback to ResourceManager's app page if no tracking URI provided
       if(original == null || original.equals("N/A")) {
-        resp.sendRedirect(resp.encodeRedirectURL(
-            StringHelper.pjoin(rmAppPageUrlBase, id.toString())));
+        ProxyUtils.sendRedirect(req, resp, 
+            StringHelper.pjoin(rmAppPageUrlBase, id.toString()));
         return;
       } else {
         if (ProxyUriUtils.getSchemeFromUrl(original).isEmpty()) {
@@ -318,8 +318,9 @@ public class WebAppProxyServlet extends HttpServlet {
 
       String runningUser = applicationReport.getUser();
       if(checkUser && !runningUser.equals(remoteUser)) {
-        LOG.info("Asking "+remoteUser+" if they want to connect to the " +
-            "app master GUI of "+appId+" owned by "+runningUser);
+        LOG.info("Asking {} if they want to connect to the "
+            + "app master GUI of {} owned by {}",
+            remoteUser, appId, runningUser);
         warnUserPage(resp, ProxyUriUtils.getPathAndQuery(id, rest, 
             req.getQueryString(), true), runningUser, id);
         return;
@@ -329,29 +330,45 @@ public class WebAppProxyServlet extends HttpServlet {
           StringHelper.ujoin(trackingUri.getPath(), rest), req.getQueryString(),
           null);
       
-      LOG.info(req.getRemoteUser()+" is accessing unchecked "+toFetch+
-          " which is the app master GUI of "+appId+" owned by "+runningUser);
+      LOG.info("{} is accessing unchecked {}"
+          + " which is the app master GUI of {} owned by {}",
+          remoteUser, toFetch, appId, runningUser);
 
-      switch(applicationReport.getYarnApplicationState()) {
-      case KILLED:
-      case FINISHED:
-      case FAILED:
-        resp.sendRedirect(resp.encodeRedirectURL(toFetch.toString()));
-        return;
+      switch (applicationReport.getYarnApplicationState()) {
+        case KILLED:
+        case FINISHED:
+        case FAILED:
+          ProxyUtils.sendRedirect(req, resp, toFetch.toString());
+          return;
+        default:
+          // fall out of the switch
       }
       Cookie c = null;
-      if(userWasWarned && userApproved) {
+      if (userWasWarned && userApproved) {
         c = makeCheckCookie(id, true);
       }
       proxyLink(req, resp, toFetch, c, getProxyHost());
 
-    } catch(URISyntaxException e) {
+    } catch(URISyntaxException | YarnException e) {
       throw new IOException(e); 
-    } catch (YarnException e) {
-      throw new IOException(e);
     }
   }
 
+  /**
+   * This method is used by Java object deserialization, to fill in the
+   * transient {@link #trackingUriPlugins} field.
+   * See {@link ObjectInputStream#defaultReadObject()}
+   * <p>
+   *   <I>Do not remove</I>
+   * <p>
+   * Yarn isn't currently serializing this class, but findbugs
+   * complains in its absence.
+   * 
+   * 
+   * @param input source
+   * @throws IOException IO failure
+   * @throws ClassNotFoundException classloader fun
+   */
   private void readObject(ObjectInputStream input)
       throws IOException, ClassNotFoundException {
     input.defaultReadObject();

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

@@ -36,7 +36,7 @@ public class AmFilterInitializer extends FilterInitializer {
   
   @Override
   public void initFilter(FilterContainer container, Configuration conf) {
-    Map<String, String> params = new HashMap<String, String>();
+    Map<String, String> params = new HashMap<>();
     List<String> proxies = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
     StringBuilder sb = new StringBuilder();
     for (String proxy : proxies) {

+ 23 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmIpFilter.java

@@ -38,26 +38,27 @@ import javax.servlet.http.Cookie;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.webproxy.ProxyUtils;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet;
 import org.apache.hadoop.yarn.util.RMHAUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Public
 public class AmIpFilter implements Filter {
-  private static final Log LOG = LogFactory.getLog(AmIpFilter.class);
-
+  private static final Logger LOG = LoggerFactory.getLogger(AmIpFilter.class);
+  
   @Deprecated
   public static final String PROXY_HOST = "PROXY_HOST";
   @Deprecated
   public static final String PROXY_URI_BASE = "PROXY_URI_BASE";
-  static final String PROXY_HOSTS = "PROXY_HOSTS";
-  static final String PROXY_HOSTS_DELIMITER = ",";
-  static final String PROXY_URI_BASES = "PROXY_URI_BASES";
-  static final String PROXY_URI_BASES_DELIMITER = ",";
+  public static final String PROXY_HOSTS = "PROXY_HOSTS";
+  public static final String PROXY_HOSTS_DELIMITER = ",";
+  public static final String PROXY_URI_BASES = "PROXY_URI_BASES";
+  public static final String PROXY_URI_BASES_DELIMITER = ",";
   //update the proxy IP list about every 5 min
   private static final long updateInterval = 5 * 60 * 1000;
 
@@ -72,7 +73,7 @@ public class AmIpFilter implements Filter {
     if (conf.getInitParameter(PROXY_HOST) != null
         && conf.getInitParameter(PROXY_URI_BASE) != null) {
       proxyHosts = new String[]{conf.getInitParameter(PROXY_HOST)};
-      proxyUriBases = new HashMap<String, String>(1);
+      proxyUriBases = new HashMap<>(1);
       proxyUriBases.put("dummy", conf.getInitParameter(PROXY_URI_BASE));
     } else {
       proxyHosts = conf.getInitParameter(PROXY_HOSTS)
@@ -80,13 +81,13 @@ public class AmIpFilter implements Filter {
 
       String[] proxyUriBasesArr = conf.getInitParameter(PROXY_URI_BASES)
           .split(PROXY_URI_BASES_DELIMITER);
-      proxyUriBases = new HashMap<String, String>();
+      proxyUriBases = new HashMap<>(proxyUriBasesArr.length);
       for (String proxyUriBase : proxyUriBasesArr) {
         try {
           URL url = new URL(proxyUriBase);
           proxyUriBases.put(url.getHost() + ":" + url.getPort(), proxyUriBase);
         } catch(MalformedURLException e) {
-          LOG.warn(proxyUriBase + " does not appear to be a valid URL", e);
+          LOG.warn("{} does not appear to be a valid URL", proxyUriBase, e);
         }
       }
     }
@@ -96,18 +97,18 @@ public class AmIpFilter implements Filter {
     long now = System.currentTimeMillis();
     synchronized(this) {
       if(proxyAddresses == null || (lastUpdate + updateInterval) >= now) {
-        proxyAddresses = new HashSet<String>();
+        proxyAddresses = new HashSet<>();
         for (String proxyHost : proxyHosts) {
           try {
               for(InetAddress add : InetAddress.getAllByName(proxyHost)) {
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("proxy address is: " + add.getHostAddress());
+                  LOG.debug("proxy address is: {}", add.getHostAddress());
                 }
                 proxyAddresses.add(add.getHostAddress());
               }
               lastUpdate = now;
             } catch (UnknownHostException e) {
-              LOG.warn("Could not locate " + proxyHost + " - skipping", e);
+              LOG.warn("Could not locate {} - skipping", proxyHost, e);
             }
           }
         if (proxyAddresses.isEmpty()) {
@@ -126,20 +127,17 @@ public class AmIpFilter implements Filter {
   @Override
   public void doFilter(ServletRequest req, ServletResponse resp,
       FilterChain chain) throws IOException, ServletException {
-    if(!(req instanceof HttpServletRequest)) {
-      throw new ServletException("This filter only works for HTTP/HTTPS");
-    }
+    ProxyUtils.rejectNonHttpRequests(req);
 
     HttpServletRequest httpReq = (HttpServletRequest)req;
     HttpServletResponse httpResp = (HttpServletResponse)resp;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Remote address for request is: " + httpReq.getRemoteAddr());
+      LOG.debug("Remote address for request is: {}", httpReq.getRemoteAddr());
     }
-    if(!getProxyAddresses().contains(httpReq.getRemoteAddr())) {
+    if (!getProxyAddresses().contains(httpReq.getRemoteAddr())) {
       String redirectUrl = findRedirectUrl();
-      redirectUrl = httpResp.encodeRedirectURL(redirectUrl +
-          httpReq.getRequestURI());
-      httpResp.sendRedirect(redirectUrl);
+      String target = redirectUrl + httpReq.getRequestURI();
+      ProxyUtils.sendRedirect(httpReq,  httpResp,  target);
       return;
     }
 
@@ -153,9 +151,9 @@ public class AmIpFilter implements Filter {
         }
       }
     }
-    if(user == null) {
-      LOG.warn("Could not find "+WebAppProxyServlet.PROXY_USER_COOKIE_NAME
-          +" cookie, so user will not be set");
+    if (user == null) {
+      LOG.warn("Could not find " + WebAppProxyServlet.PROXY_USER_COOKIE_NAME
+               + " cookie, so user will not be set");
       chain.doFilter(req, resp);
     } else {
       final AmIpPrincipal principal = new AmIpPrincipal(user);

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

@@ -39,8 +39,6 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.http.HttpServer2;
@@ -54,7 +52,6 @@ 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.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -62,15 +59,16 @@ import org.junit.Test;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.ServletHolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Test the WebAppProxyServlet and WebAppProxy. For back end use simple web
  * server.
  */
 public class TestWebAppProxyServlet {
-
-  private static final Log LOG = LogFactory
-      .getLog(TestWebAppProxyServlet.class);
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestWebAppProxyServlet.class);
 
   private static Server server;
   private static int originalPort = 0;
@@ -273,11 +271,10 @@ public class TestWebAppProxyServlet {
     }
 
     @Override
-    public synchronized void init(Configuration conf) {
-      Configuration config = new YarnConfiguration(conf);
+    public synchronized void serviceInit(Configuration conf) throws Exception {
       proxy = new WebAppProxyForTest();
       addService(proxy);
-      super.init(config);
+      super.serviceInit(conf);
     }
 
   }
@@ -286,44 +283,39 @@ public class TestWebAppProxyServlet {
     
     HttpServer2 proxyServer;
     AppReportFetcherForTest appReportFetcher;
-    
+
     @Override
-    public void start() {
-      try {
-        Configuration conf = getConfig();
-        String bindAddress = conf.get(YarnConfiguration.PROXY_ADDRESS);
-        bindAddress = StringUtils.split(bindAddress, ':')[0];
-        AccessControlList acl = new AccessControlList(
-            conf.get(YarnConfiguration.YARN_ADMIN_ACL, 
-            YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
-        proxyServer = new HttpServer2.Builder()
-            .setName("proxy")
-            .addEndpoint(
-                URI.create(WebAppUtils.getHttpSchemePrefix(conf) + bindAddress
-                    + ":0")).setFindPort(true)
-            .setConf(conf)
-            .setACL(acl)
-            .build();
-        proxyServer.addServlet(ProxyUriUtils.PROXY_SERVLET_NAME,
-            ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
-
-        appReportFetcher = new AppReportFetcherForTest(conf);
-        proxyServer.setAttribute(FETCHER_ATTRIBUTE,
-            appReportFetcher );
-        proxyServer.setAttribute(IS_SECURITY_ENABLED_ATTRIBUTE, Boolean.TRUE);
-        
-        String proxy = WebAppUtils.getProxyHostAndPort(conf);
-        String[] proxyParts = proxy.split(":");
-        String proxyHost = proxyParts[0];
-        
-        proxyServer.setAttribute(PROXY_HOST_ATTRIBUTE, proxyHost);
-        proxyServer.start();
-        System.out.println("Proxy server is started at port " + 
-            proxyServer.getConnectorAddress(0).getPort());
-      } catch (Exception e) {
-        LOG.fatal("Could not start proxy web server", e);
-        throw new YarnRuntimeException("Could not start proxy web server", e);
-      }
+    protected void serviceStart() throws Exception {
+      Configuration conf = getConfig();
+      String bindAddress = conf.get(YarnConfiguration.PROXY_ADDRESS);
+      bindAddress = StringUtils.split(bindAddress, ':')[0];
+      AccessControlList acl = new AccessControlList(
+          conf.get(YarnConfiguration.YARN_ADMIN_ACL, 
+          YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+      proxyServer = new HttpServer2.Builder()
+          .setName("proxy")
+          .addEndpoint(
+              URI.create(WebAppUtils.getHttpSchemePrefix(conf) + bindAddress
+                  + ":0")).setFindPort(true)
+          .setConf(conf)
+          .setACL(acl)
+          .build();
+      proxyServer.addServlet(ProxyUriUtils.PROXY_SERVLET_NAME,
+          ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
+
+      appReportFetcher = new AppReportFetcherForTest(conf);
+      proxyServer.setAttribute(FETCHER_ATTRIBUTE,
+          appReportFetcher );
+      proxyServer.setAttribute(IS_SECURITY_ENABLED_ATTRIBUTE, Boolean.TRUE);
+      
+      String proxy = WebAppUtils.getProxyHostAndPort(conf);
+      String[] proxyParts = proxy.split(":");
+      String proxyHost = proxyParts[0];
+      
+      proxyServer.setAttribute(PROXY_HOST_ATTRIBUTE, proxyHost);
+      proxyServer.start();
+      LOG.info("Proxy server is started at port {}",
+          proxyServer.getConnectorAddress(0).getPort());
     }
 
   }

+ 37 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.webproxy.amfilter;
 
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -29,6 +31,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import static org.junit.Assert.*;
 
+import org.apache.hadoop.yarn.server.webproxy.ProxyUtils;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet;
 import org.glassfish.grizzly.servlet.HttpServletResponseImpl;
 import org.junit.Test;
@@ -151,7 +154,7 @@ public class TestAmFilter {
       testFilter.doFilter(failRequest, response, chain);
       fail();
     } catch (ServletException e) {
-      assertEquals("This filter only works for HTTP/HTTPS", e.getMessage());
+      assertEquals(ProxyUtils.E_HTTP_HTTPS_ONLY, e.getMessage());
     }
 
     // request with HttpServletRequest
@@ -160,7 +163,9 @@ public class TestAmFilter {
     Mockito.when(request.getRequestURI()).thenReturn("/redirect");
     testFilter.doFilter(request, response, chain);
     // address "redirect" is not in host list
-    assertEquals("http://bogus/redirect", response.getRedirect());
+    assertEquals(302, response.status);
+    String redirect = response.getHeader(ProxyUtils.LOCATION);
+    assertEquals("http://bogus/redirect", redirect);
     // "127.0.0.1" contains in host list. Without cookie
     Mockito.when(request.getRemoteAddr()).thenReturn("127.0.0.1");
     testFilter.doFilter(request, response, chain);
@@ -186,6 +191,11 @@ public class TestAmFilter {
 
   private class HttpServletResponseForTest extends HttpServletResponseImpl {
     String redirectLocation = "";
+    int status;
+    private String contentType;
+    private final Map<String, String> headers = new HashMap<>(1);
+    private StringWriter body;
+
 
     public String getRedirect() {
       return redirectLocation;
@@ -201,6 +211,31 @@ public class TestAmFilter {
       return url;
     }
 
+    @Override
+    public void setStatus(int status) {
+      this.status = status;
+    }
+
+    @Override
+    public void setContentType(String type) {
+      this.contentType = type;
+    }
+
+    @Override
+    public void setHeader(String name, String value) {
+      headers.put(name, value);
+    }
+
+    public String getHeader(String name) {
+      return headers.get(name);
+    }
+
+    @Override
+    public PrintWriter getWriter() throws IOException {
+      body = new StringWriter();
+      return new PrintWriter(body);
+    }
   }
 
+  
 }