You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2017/11/18 02:37:06 UTC

nifi git commit: NIFI-4501 - Changed request header handling logic. - Removed unnecessary Maven dependency. - This closes #2279

Repository: nifi
Updated Branches:
  refs/heads/master c89d79336 -> 5d643edfa


NIFI-4501
- Changed request header handling logic.
- Removed unnecessary Maven dependency.
- This closes #2279


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/5d643edf
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/5d643edf
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/5d643edf

Branch: refs/heads/master
Commit: 5d643edfaba4f5369c94ee1b4eaa5c59e3a9f37a
Parents: c89d793
Author: Andy LoPresto <al...@apache.org>
Authored: Tue Oct 31 17:00:03 2017 -0700
Committer: Matt Gilman <ma...@gmail.com>
Committed: Fri Nov 17 20:36:55 2017 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/util/NiFiProperties.java    |  45 ++-
 .../java/org/apache/nifi/web/util/WebUtils.java | 156 +++++++++--
 .../apache/nifi/web/util/WebUtilsTest.groovy    | 275 +++++++++++++++++++
 .../StandardNiFiPropertiesGroovyTest.groovy     | 111 ++++++++
 .../nifi-framework/nifi-resources/pom.xml       |   1 +
 .../src/main/resources/conf/nifi.properties     |   1 +
 .../nifi/web/server/HostHeaderHandler.java      | 114 ++++++++
 .../HostHeaderSanitizationCustomizer.java       |  80 ++++++
 .../org/apache/nifi/web/server/JettyServer.java | 126 ++++++---
 .../nifi/web/api/ApplicationResource.java       | 114 ++++----
 .../nifi/web/api/ApplicationResourceTest.groovy | 181 ++++++++++++
 .../src/test/resources/logback-test.xml         |   1 +
 .../nifi-web/nifi-web-error/pom.xml             |   5 +
 .../apache/nifi/web/filter/CatchAllFilter.java  |  36 +++
 .../nifi-web-error/src/main/webapp/index.jsp    |  55 ++--
 .../nifi/processors/standard/ListenHTTP.java    |  31 +--
 .../websocket/jetty/JettyWebSocketServer.java   |  13 +-
 17 files changed, 1162 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index b211b71..55cae14 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -177,6 +177,7 @@ public abstract class NiFiProperties {
     public static final String WEB_WORKING_DIR = "nifi.web.jetty.working.directory";
     public static final String WEB_THREADS = "nifi.web.jetty.threads";
     public static final String WEB_MAX_HEADER_SIZE = "nifi.web.max.header.size";
+    public static final String WEB_PROXY_CONTEXT_PATH = "nifi.web.proxy.context.path";
 
     // ui properties
     public static final String UI_BANNER_TEXT = "nifi.ui.banner.text";
@@ -256,8 +257,8 @@ public abstract class NiFiProperties {
     public static final String DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = "3 secs";
     public static final String DEFAULT_ZOOKEEPER_ROOT_NODE = "/nifi";
     public static final String DEFAULT_ZOOKEEPER_AUTH_TYPE = "default";
-    public static final String DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_HOST_FROM_PRINCIPAL  = "true";
-    public static final String DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_REALM_FROM_PRINCIPAL  = "true";
+    public static final String DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_HOST_FROM_PRINCIPAL = "true";
+    public static final String DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_REALM_FROM_PRINCIPAL = "true";
     public static final String DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL = "30 secs";
     public static final String DEFAULT_FLOW_CONFIGURATION_ARCHIVE_ENABLED = "true";
     public static final String DEFAULT_FLOW_CONFIGURATION_ARCHIVE_MAX_TIME = "30 days";
@@ -1084,7 +1085,7 @@ public abstract class NiFiProperties {
      * Returns the number of claims to keep open for writing. Ideally, this will be at
      * least as large as the number of threads that will be updating the repository simultaneously but we don't want
      * to get too large because it will hold open up to this many FileOutputStreams.
-     *
+     * <p>
      * Default is {@link #DEFAULT_MAX_FLOWFILES_PER_CLAIM}
      *
      * @return the maximum number of flow files per claim
@@ -1100,7 +1101,7 @@ public abstract class NiFiProperties {
     /**
      * Returns the maximum size, in bytes, that claims should grow before writing a new file. This means that we won't continually write to one
      * file that keeps growing but gives us a chance to bunch together many small files.
-     *
+     * <p>
      * Default is {@link #DEFAULT_MAX_APPENDABLE_CLAIM_SIZE}
      *
      * @return the maximum appendable claim size
@@ -1285,6 +1286,42 @@ public abstract class NiFiProperties {
         return keys;
     }
 
+    /**
+     * Returns the whitelisted proxy context paths as a comma-delimited string. The paths have been normalized to the form {@code /some/context/path}.
+     *
+     * Note: Calling {@code NiFiProperties.getProperty(NiFiProperties.WEB_PROXY_CONTEXT_PATH)} will not normalize the paths.
+     *
+     * @return the path(s)
+     */
+    public String getWhitelistedContextPaths() {
+        return StringUtils.join(getWhitelistedContextPathsAsList(), ",");
+    }
+
+    /**
+     * Returns the whitelisted proxy context paths as a list of paths. The paths have been normalized to the form {@code /some/context/path}.
+     *
+     * @return the path(s)
+     */
+    public List<String> getWhitelistedContextPathsAsList() {
+        String rawProperty = getProperty(WEB_PROXY_CONTEXT_PATH, "");
+        List<String> contextPaths = Arrays.asList(rawProperty.split(","));
+        return contextPaths.stream()
+                .map(this::normalizeContextPath).collect(Collectors.toList());
+    }
+
+    private String normalizeContextPath(String cp) {
+        if (cp == null || cp.equalsIgnoreCase("")) {
+            return "";
+        } else {
+            String trimmedCP = cp.trim();
+            // Ensure it starts with a leading slash and does not end in a trailing slash
+            // There's a potential for the path to be something like bad/path/// but this is semi-trusted data from an admin-accessible file and there are way worse possibilities here
+            trimmedCP = trimmedCP.startsWith("/") ? trimmedCP : "/" + trimmedCP;
+            trimmedCP = trimmedCP.endsWith("/") ? trimmedCP.substring(0, trimmedCP.length() - 1) : trimmedCP;
+            return trimmedCP;
+        }
+    }
+
     private List<String> getProvenanceRepositoryEncryptionKeyProperties() {
         // Filter all the property keys that define a key
         return getPropertyKeys().stream().filter(k ->

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java b/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
index 358cbea..351d7f9 100644
--- a/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
+++ b/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
@@ -16,28 +16,31 @@
  */
 package org.apache.nifi.web.util;
 
-import org.apache.nifi.security.util.CertificateUtils;
-import org.glassfish.jersey.client.ClientConfig;
-import org.glassfish.jersey.jackson.internal.jackson.jaxrs.json.JacksonJaxbJsonProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLPeerUnverifiedException;
-import javax.net.ssl.SSLSession;
-import javax.ws.rs.client.Client;
-import javax.ws.rs.client.ClientBuilder;
+import java.net.URI;
 import java.security.cert.Certificate;
 import java.security.cert.CertificateParsingException;
 import java.security.cert.X509Certificate;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.core.UriBuilderException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.glassfish.jersey.client.ClientConfig;
+import org.glassfish.jersey.jackson.internal.jackson.jaxrs.json.JacksonJaxbJsonProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Common utilities related to web development.
- *
  */
 public final class WebUtils {
 
@@ -45,6 +48,9 @@ public final class WebUtils {
 
     final static ReadWriteLock lock = new ReentrantReadWriteLock();
 
+    private static final String PROXY_CONTEXT_PATH_HTTP_HEADER = "X-ProxyContextPath";
+    private static final String FORWARDED_CONTEXT_HTTP_HEADER = "X-Forwarded-Context";
+
     private WebUtils() {
     }
 
@@ -54,7 +60,6 @@ public final class WebUtils {
      * automatically configured for JSON serialization/deserialization.
      *
      * @param config client configuration
-     *
      * @return a Client instance
      */
     public static Client createClient(final ClientConfig config) {
@@ -67,8 +72,7 @@ public final class WebUtils {
      * will be automatically configured for JSON serialization/deserialization.
      *
      * @param config client configuration
-     * @param ctx security context
-     *
+     * @param ctx    security context
      * @return a Client instance
      */
     public static Client createClient(final ClientConfig config, final SSLContext ctx) {
@@ -81,9 +85,8 @@ public final class WebUtils {
      * will be automatically configured for JSON serialization/deserialization.
      *
      * @param config client configuration
-     * @param ctx security context, which may be null for non-secure client
-     * creation
-     *
+     * @param ctx    security context, which may be null for non-secure client
+     *               creation
      * @return a Client instance
      */
     private static Client createClientHelper(final ClientConfig config, final SSLContext ctx) {
@@ -128,4 +131,119 @@ public final class WebUtils {
 
     }
 
+    /**
+     * This method will check the provided context path headers against a whitelist (provided in nifi.properties) and throw an exception if the requested context path is not registered.
+     *
+     * @param uri                     the request URI
+     * @param request                 the HTTP request
+     * @param whitelistedContextPaths comma-separated list of valid context paths
+     * @return the resource path
+     * @throws UriBuilderException if the requested context path is not registered (header poisoning)
+     */
+    public static String getResourcePath(URI uri, HttpServletRequest request, String whitelistedContextPaths) throws UriBuilderException {
+        String resourcePath = uri.getPath();
+
+        // Determine and normalize the context path
+        String determinedContextPath = determineContextPath(request);
+        determinedContextPath = normalizeContextPath(determinedContextPath);
+
+        // If present, check it and prepend to the resource path
+        if (StringUtils.isNotBlank(determinedContextPath)) {
+            verifyContextPath(whitelistedContextPaths, determinedContextPath);
+
+            // Determine the complete resource path
+            resourcePath = determinedContextPath + resourcePath;
+        }
+
+        return resourcePath;
+    }
+
+    /**
+     * Throws an exception if the provided context path is not in the whitelisted context paths list.
+     *
+     * @param whitelistedContextPaths a comma-delimited list of valid context paths
+     * @param determinedContextPath   the normalized context path from a header
+     * @throws UriBuilderException if the context path is not safe
+     */
+    public static void verifyContextPath(String whitelistedContextPaths, String determinedContextPath) throws UriBuilderException {
+        // If blank, ignore
+        if (StringUtils.isBlank(determinedContextPath)) {
+            return;
+        }
+
+        // Check it against the whitelist
+        List<String> individualContextPaths = Arrays.asList(StringUtils.split(whitelistedContextPaths, ","));
+        if (!individualContextPaths.contains(determinedContextPath)) {
+            final String msg = "The provided context path [" + determinedContextPath + "] was not whitelisted [" + whitelistedContextPaths + "]";
+            logger.error(msg);
+            throw new UriBuilderException(msg);
+        }
+    }
+
+    /**
+     * Returns a normalized context path (leading /, no trailing /). If the parameter is blank, an empty string will be returned.
+     *
+     * @param determinedContextPath the raw context path
+     * @return the normalized context path
+     */
+    public static String normalizeContextPath(String determinedContextPath) {
+        if (StringUtils.isNotBlank(determinedContextPath)) {
+            // normalize context path
+            if (!determinedContextPath.startsWith("/")) {
+                determinedContextPath = "/" + determinedContextPath;
+            }
+
+            if (determinedContextPath.endsWith("/")) {
+                determinedContextPath = determinedContextPath.substring(0, determinedContextPath.length() - 1);
+            }
+
+            return determinedContextPath;
+        } else {
+            return "";
+        }
+    }
+
+    /**
+     * Determines the context path if populated in {@code X-ProxyContextPath} or {@code X-ForwardContext} headers. If not populated, returns an empty string.
+     *
+     * @param request the HTTP request
+     * @return the provided context path or an empty string
+     */
+    public static String determineContextPath(HttpServletRequest request) {
+        String contextPath = request.getContextPath();
+        String proxyContextPath = request.getHeader(PROXY_CONTEXT_PATH_HTTP_HEADER);
+        String forwardedContext = request.getHeader(FORWARDED_CONTEXT_HTTP_HEADER);
+
+        logger.debug("Context path: " + contextPath);
+        String determinedContextPath = "";
+
+        // If either header is set, log both
+        if (anyNotBlank(proxyContextPath, forwardedContext)) {
+            logger.debug(String.format("On the request, the following context paths were parsed" +
+                            " from headers:\n\t X-ProxyContextPath: %s\n\tX-Forwarded-Context: %s",
+                    proxyContextPath, forwardedContext));
+
+            // Implementing preferred order here: PCP, FCP
+            determinedContextPath = StringUtils.isNotBlank(proxyContextPath) ? proxyContextPath : forwardedContext;
+        }
+
+        logger.debug("Determined context path: " + determinedContextPath);
+        return determinedContextPath;
+    }
+
+    /**
+     * Returns true if any of the provided arguments are not blank.
+     *
+     * @param strings a variable number of strings
+     * @return true if any string has content (not empty or all whitespace)
+     */
+    private static boolean anyNotBlank(String... strings) {
+        for (String s : strings) {
+            if (StringUtils.isNotBlank(s)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-commons/nifi-web-utils/src/test/groovy/org/apache/nifi/web/util/WebUtilsTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-web-utils/src/test/groovy/org/apache/nifi/web/util/WebUtilsTest.groovy b/nifi-commons/nifi-web-utils/src/test/groovy/org/apache/nifi/web/util/WebUtilsTest.groovy
new file mode 100644
index 0000000..62755a1
--- /dev/null
+++ b/nifi-commons/nifi-web-utils/src/test/groovy/org/apache/nifi/web/util/WebUtilsTest.groovy
@@ -0,0 +1,275 @@
+/*
+ * 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.nifi.web.util
+
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.servlet.http.HttpServletRequest
+import javax.ws.rs.core.UriBuilderException
+
+@RunWith(JUnit4.class)
+class WebUtilsTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(WebUtilsTest.class)
+
+    static final String PCP_HEADER = "X-ProxyContextPath"
+    static final String FC_HEADER = "X-Forwarded-Context"
+
+    static final String WHITELISTED_PATH = "/some/context/path"
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() throws Exception {
+    }
+
+    @After
+    void tearDown() throws Exception {
+    }
+
+    HttpServletRequest mockRequest(Map keys) {
+        HttpServletRequest mockRequest = [
+                getContextPath: { ->
+                    logger.mock("Request.getContextPath() -> default/path")
+                    "default/path"
+                },
+                getHeader     : { String k ->
+                    logger.mock("Request.getHeader($k) -> ${keys}")
+                    switch (k) {
+                        case PCP_HEADER:
+                            return keys["proxy"]
+                            break
+                        case FC_HEADER:
+                            return keys["forward"]
+                            break
+                        default:
+                            return ""
+                    }
+                }] as HttpServletRequest
+        mockRequest
+    }
+
+    @Test
+    void testShouldDetermineCorrectContextPathWhenPresent() throws Exception {
+        // Arrange
+        final String CORRECT_CONTEXT_PATH = WHITELISTED_PATH
+        final String WRONG_CONTEXT_PATH = "this/is/a/bad/path"
+
+        // Variety of requests with different ordering of context paths (the correct one is always "some/context/path"
+        HttpServletRequest proxyRequest = mockRequest([proxy: CORRECT_CONTEXT_PATH])
+        HttpServletRequest forwardedRequest = mockRequest([forward: CORRECT_CONTEXT_PATH])
+        HttpServletRequest proxyBeforeForwardedRequest = mockRequest([proxy: CORRECT_CONTEXT_PATH, forward: WRONG_CONTEXT_PATH])
+        List<HttpServletRequest> requests = [proxyRequest, forwardedRequest, proxyBeforeForwardedRequest]
+
+        // Act
+        requests.each { HttpServletRequest request ->
+            String determinedContextPath = WebUtils.determineContextPath(request)
+            logger.info("Determined context path: ${determinedContextPath}")
+
+            // Assert
+            assert determinedContextPath == CORRECT_CONTEXT_PATH
+        }
+    }
+
+    @Test
+    void testShouldDetermineCorrectContextPathWhenAbsent() throws Exception {
+        // Arrange
+        final String CORRECT_CONTEXT_PATH = ""
+
+        // Variety of requests with different ordering of non-existent context paths (the correct one is always ""
+        HttpServletRequest proxyRequest = mockRequest([proxy: ""])
+        HttpServletRequest proxySpacesRequest = mockRequest([proxy: "   "])
+        HttpServletRequest forwardedRequest = mockRequest([forward: ""])
+        HttpServletRequest forwardedSpacesRequest = mockRequest([forward: "   "])
+        HttpServletRequest proxyBeforeForwardedRequest = mockRequest([proxy: "", forward: ""])
+        List<HttpServletRequest> requests = [proxyRequest, proxySpacesRequest, forwardedRequest, forwardedSpacesRequest, proxyBeforeForwardedRequest]
+
+        // Act
+        requests.each { HttpServletRequest request ->
+            String determinedContextPath = WebUtils.determineContextPath(request)
+            logger.info("Determined context path: ${determinedContextPath}")
+
+            // Assert
+            assert determinedContextPath == CORRECT_CONTEXT_PATH
+        }
+    }
+
+    @Test
+    void testShouldNormalizeContextPath() throws Exception {
+        // Arrange
+        final String CORRECT_CONTEXT_PATH = WHITELISTED_PATH
+        final String TRIMMED_PATH = WHITELISTED_PATH[1..-1] // Trims leading /
+
+        // Variety of different context paths (the correct one is always "/some/context/path")
+        List<String> contextPaths = ["/$TRIMMED_PATH", "/" + TRIMMED_PATH, TRIMMED_PATH, TRIMMED_PATH + "/"]
+
+        // Act
+        contextPaths.each { String contextPath ->
+            String normalizedContextPath = WebUtils.normalizeContextPath(contextPath)
+            logger.info("Normalized context path: ${normalizedContextPath} <- ${contextPath}")
+
+            // Assert
+            assert normalizedContextPath == CORRECT_CONTEXT_PATH
+        }
+    }
+
+    @Test
+    void testGetResourcePathShouldBlockContextPathHeaderIfNotInWhitelist() throws Exception {
+        // Arrange
+        logger.info("Whitelisted path(s): ")
+
+        HttpServletRequest requestWithProxyHeader = mockRequest([proxy: "any/context/path"])
+        HttpServletRequest requestWithProxyAndForwardHeader = mockRequest([proxy: "any/context/path", forward: "any/other/context/path"])
+        List<HttpServletRequest> requests = [requestWithProxyHeader, requestWithProxyAndForwardHeader]
+
+        // Act
+        requests.each { HttpServletRequest request ->
+            def msg = shouldFail(UriBuilderException) {
+                String generatedResourcePath = WebUtils.getResourcePath(new URI('https://nifi.apache.org/actualResource'), request, "")
+                logger.unexpected("Generated Resource Path: ${generatedResourcePath}")
+            }
+
+            // Assert
+            logger.expected(msg)
+            assert msg =~ "The provided context path \\[.*\\] was not whitelisted \\[\\]"
+        }
+    }
+
+    @Test
+    void testGetResourcePathShouldAllowContextPathHeaderIfInWhitelist() throws Exception {
+        // Arrange
+        logger.info("Whitelisted path(s): ${WHITELISTED_PATH}")
+
+        HttpServletRequest requestWithProxyHeader = mockRequest([proxy: "some/context/path"])
+        HttpServletRequest requestWithForwardHeader = mockRequest([forward: "some/context/path"])
+        HttpServletRequest requestWithProxyAndForwardHeader = mockRequest([proxy: "some/context/path", forward: "any/other/context/path"])
+        List<HttpServletRequest> requests = [requestWithProxyHeader, requestWithForwardHeader, requestWithProxyAndForwardHeader]
+
+        // Act
+        requests.each { HttpServletRequest request ->
+            String generatedResourcePath = WebUtils.getResourcePath(new URI('https://nifi.apache.org/actualResource'), request, WHITELISTED_PATH)
+            logger.info("Generated Resource Path: ${generatedResourcePath}")
+
+            // Assert
+            assert generatedResourcePath == "${WHITELISTED_PATH}/actualResource"
+        }
+    }
+
+    @Test
+    void testGetResourcePathShouldAllowContextPathHeaderIfElementInMultipleWhitelist() throws Exception {
+        // Arrange
+        String multipleWhitelistedPaths = [WHITELISTED_PATH, "/another/path", "/a/third/path"].join(",")
+        logger.info("Whitelisted path(s): ${multipleWhitelistedPaths}")
+
+        final List<String> VALID_RESOURCE_PATHS = multipleWhitelistedPaths.split(",").collect { "$it/actualResource" }
+
+        HttpServletRequest requestWithProxyHeader = mockRequest([proxy: "some/context/path"])
+        HttpServletRequest requestWithForwardHeader = mockRequest([forward: "another/path"])
+        HttpServletRequest requestWithProxyAndForwardHeader = mockRequest([proxy: "a/third/path", forward: "any/other/context/path"])
+        List<HttpServletRequest> requests = [requestWithProxyHeader, requestWithForwardHeader, requestWithProxyAndForwardHeader]
+
+        // Act
+        requests.each { HttpServletRequest request ->
+            String generatedResourcePath = WebUtils.getResourcePath(new URI('https://nifi.apache.org/actualResource'), request, multipleWhitelistedPaths)
+            logger.info("Generated Resource Path: ${generatedResourcePath}")
+
+            // Assert
+            assert VALID_RESOURCE_PATHS.any { it == generatedResourcePath }
+        }
+    }
+
+    @Test
+    void testVerifyContextPathShouldAllowContextPathHeaderIfInWhitelist() throws Exception {
+        // Arrange
+        logger.info("Whitelisted path(s): ${WHITELISTED_PATH}")
+        String contextPath = WHITELISTED_PATH
+
+        // Act
+        logger.info("Testing [${contextPath}] against ${WHITELISTED_PATH}")
+        WebUtils.verifyContextPath(WHITELISTED_PATH, contextPath)
+        logger.info("Verified [${contextPath}]")
+
+        // Assert
+        // Would throw exception if invalid
+    }
+
+    @Test
+    void testVerifyContextPathShouldAllowContextPathHeaderIfInMultipleWhitelist() throws Exception {
+        // Arrange
+        String multipleWhitelist = [WHITELISTED_PATH, WebUtils.normalizeContextPath(WHITELISTED_PATH.reverse())].join(",")
+        logger.info("Whitelisted path(s): ${multipleWhitelist}")
+        String contextPath = WHITELISTED_PATH
+
+        // Act
+        logger.info("Testing [${contextPath}] against ${multipleWhitelist}")
+        WebUtils.verifyContextPath(multipleWhitelist, contextPath)
+        logger.info("Verified [${contextPath}]")
+
+        // Assert
+        // Would throw exception if invalid
+    }
+
+    @Test
+    void testVerifyContextPathShouldAllowContextPathHeaderIfBlank() throws Exception {
+        // Arrange
+        logger.info("Whitelisted path(s): ${WHITELISTED_PATH}")
+
+        def emptyContextPaths = ["", "  ", "\t", null]
+
+        // Act
+        emptyContextPaths.each { String contextPath ->
+            logger.info("Testing [${contextPath}] against ${WHITELISTED_PATH}")
+            WebUtils.verifyContextPath(WHITELISTED_PATH, contextPath)
+            logger.info("Verified [${contextPath}]")
+
+            // Assert
+            // Would throw exception if invalid
+        }
+    }
+
+    @Test
+    void testVerifyContextPathShouldBlockContextPathHeaderIfNotAllowed() throws Exception {
+        // Arrange
+        logger.info("Whitelisted path(s): ${WHITELISTED_PATH}")
+
+        def invalidContextPaths = ["/other/path", "somesite.com", "/../trying/to/escape"]
+
+        // Act
+        invalidContextPaths.each { String contextPath ->
+            logger.info("Testing [${contextPath}] against ${WHITELISTED_PATH}")
+            def msg = shouldFail(UriBuilderException) {
+                WebUtils.verifyContextPath(WHITELISTED_PATH, contextPath)
+                logger.info("Verified [${contextPath}]")
+            }
+
+            // Assert
+            logger.expected(msg)
+            assert msg =~ " was not whitelisted "
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
index ae43a3d..64c4f0f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
@@ -322,4 +322,115 @@ class StandardNiFiPropertiesGroovyTest extends GroovyTestCase {
         assert key == KEY_HEX
         assert keys == [(KEY_ID): KEY_HEX, (KEY_ID_2): KEY_HEX_2, (KEY_ID_3): KEY_HEX_3]
     }
+
+
+    @Test
+    void testShouldNormalizeContextPathProperty() {
+        // Arrange
+        String noLeadingSlash = "some/context/path"
+        Properties rawProps = new Properties(["nifi.web.proxy.context.path": noLeadingSlash])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw context path property [${noLeadingSlash}]")
+
+        // Act
+        String normalizedContextPath = props.getWhitelistedContextPaths()
+        logger.info("Read from NiFiProperties instance: ${normalizedContextPath}")
+
+        // Assert
+        assert normalizedContextPath == "/" + noLeadingSlash
+    }
+
+    @Test
+    void testShouldHandleNormalizedContextPathProperty() {
+        // Arrange
+        String leadingSlash = "/some/context/path"
+        Properties rawProps = new Properties(["nifi.web.proxy.context.path": leadingSlash])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw context path property [${leadingSlash}]")
+
+        // Act
+        String normalizedContextPath = props.getWhitelistedContextPaths()
+        logger.info("Read from NiFiProperties instance: ${normalizedContextPath}")
+
+        // Assert
+        assert normalizedContextPath == leadingSlash
+    }
+
+    @Test
+    void testShouldNormalizeMultipleContextPathsInProperty() {
+        // Arrange
+        String noLeadingSlash = "some/context/path"
+        String leadingSlash = "some/other/path"
+        String leadingAndTrailingSlash = "/a/third/path/"
+        List<String> paths = [noLeadingSlash, leadingSlash, leadingAndTrailingSlash]
+        String combinedPaths = paths.join(",")
+        Properties rawProps = new Properties(["nifi.web.proxy.context.path": combinedPaths])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw context path property [${noLeadingSlash}]")
+
+        // Act
+        String normalizedContextPath = props.getWhitelistedContextPaths()
+        logger.info("Read from NiFiProperties instance: ${normalizedContextPath}")
+
+        // Assert
+        def splitPaths = normalizedContextPath.split(",")
+        splitPaths.every {
+            assert it.startsWith("/")
+            assert !it.endsWith("/")
+        }
+    }
+
+    @Test
+    void testShouldHandleNormalizedContextPathPropertyAsList() {
+        // Arrange
+        String leadingSlash = "/some/context/path"
+        Properties rawProps = new Properties(["nifi.web.proxy.context.path": leadingSlash])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw context path property [${leadingSlash}]")
+
+        // Act
+        def normalizedContextPaths = props.getWhitelistedContextPathsAsList()
+        logger.info("Read from NiFiProperties instance: ${normalizedContextPaths}")
+
+        // Assert
+        assert normalizedContextPaths.size() == 1
+        assert normalizedContextPaths.contains(leadingSlash)
+    }
+
+    @Test
+    void testShouldNormalizeMultipleContextPathsInPropertyAsList() {
+        // Arrange
+        String noLeadingSlash = "some/context/path"
+        String leadingSlash = "/some/other/path"
+        String leadingAndTrailingSlash = "/a/third/path/"
+        List<String> paths = [noLeadingSlash, leadingSlash, leadingAndTrailingSlash]
+        String combinedPaths = paths.join(",")
+        Properties rawProps = new Properties(["nifi.web.proxy.context.path": combinedPaths])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw context path property [${noLeadingSlash}]")
+
+        // Act
+        def normalizedContextPaths = props.getWhitelistedContextPathsAsList()
+        logger.info("Read from NiFiProperties instance: ${normalizedContextPaths}")
+
+        // Assert
+        assert normalizedContextPaths.size() == 3
+        assert normalizedContextPaths.containsAll([leadingSlash, "/" + noLeadingSlash, leadingAndTrailingSlash[0..-2]])
+    }
+
+    @Test
+    void testShouldHandleNormalizingEmptyContextPathProperty() {
+        // Arrange
+        String empty = ""
+        Properties rawProps = new Properties(["nifi.web.proxy.context.path": empty])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw context path property [${empty}]")
+
+        // Act
+        String normalizedContextPath = props.getWhitelistedContextPaths()
+        logger.info("Read from NiFiProperties instance: ${normalizedContextPath}")
+
+        // Assert
+        assert normalizedContextPath == empty
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
index 3e6a65d..35ec7dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
@@ -132,6 +132,7 @@
         <nifi.jetty.work.dir>./work/jetty</nifi.jetty.work.dir>
         <nifi.web.jetty.threads>200</nifi.web.jetty.threads>
         <nifi.web.max.header.size>16 KB</nifi.web.max.header.size>
+        <nifi.web.proxy.context.path />
 
         <!-- nifi.properties: security properties -->
         <nifi.security.keystore />

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index 49f08ad..19358f6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -137,6 +137,7 @@ nifi.web.https.network.interface.default=${nifi.web.https.network.interface.defa
 nifi.web.jetty.working.directory=${nifi.jetty.work.dir}
 nifi.web.jetty.threads=${nifi.web.jetty.threads}
 nifi.web.max.header.size=${nifi.web.max.header.size}
+nifi.web.proxy.context.path=${nifi.web.proxy.context.path}
 
 # security properties #
 nifi.sensitive.props.key=

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderHandler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderHandler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderHandler.java
new file mode 100644
index 0000000..989d8d7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderHandler.java
@@ -0,0 +1,114 @@
+/*
+ * 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.nifi.web.server;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.handler.ScopedHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HostHeaderHandler extends ScopedHandler {
+    private static final Logger logger = LoggerFactory.getLogger(HostHeaderHandler.class);
+
+    private final String serverName;
+    private final int serverPort;
+    private final List<String> validHosts;
+
+    /**
+     * @param serverName the {@code serverName} to set on the request (the {@code serverPort} will not be set)
+     */
+    public HostHeaderHandler(String serverName) {
+        this(serverName, 0);
+    }
+
+    /**
+     * @param serverName the {@code serverName} to set on the request
+     * @param serverPort the {@code serverPort} to set on the request
+     */
+    public HostHeaderHandler(String serverName, int serverPort) {
+        this.serverName = Objects.requireNonNull(serverName);
+        this.serverPort = serverPort;
+
+        validHosts = new ArrayList<>();
+        validHosts.add(serverName.toLowerCase());
+        validHosts.add(serverName.toLowerCase() + ":" + serverPort);
+        // Sometimes the hostname is left empty but the port is always populated
+        validHosts.add("localhost");
+        validHosts.add("localhost:" + serverPort);
+        // Different from customizer -- empty is ok here
+        validHosts.add("");
+
+        logger.info("Created " + this.toString());
+    }
+
+    @Override
+    public void doScope(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
+            throws IOException, ServletException {
+        logger.debug("HostHeaderHandler#doScope on " + request.getRequestURI());
+        nextScope(target, baseRequest, request, response);
+    }
+
+    private boolean hostHeaderIsValid(String hostHeader) {
+        return validHosts.contains(hostHeader.toLowerCase().trim());
+    }
+
+    @Override
+    public String toString() {
+        return "HostHeaderHandler for " + serverName + ":" + serverPort;
+    }
+
+    /**
+     * Returns an error message to the response and marks the request as handled if the host header is not valid.
+     * Otherwise passes the request on to the next scoped handler.
+     *
+     * @param target the target (not relevant here)
+     * @param baseRequest the original request object
+     * @param request the request as an HttpServletRequest
+     * @param response the current response
+     */
+    @Override
+    public void doHandle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+        final String hostHeader = request.getHeader("Host");
+        logger.debug("Received request [" + request.getRequestURI() + "] with host header: " + hostHeader);
+        if (!hostHeaderIsValid(hostHeader)) {
+            logger.warn("Request host header [" + hostHeader + "] different from web hostname [" +
+                    serverName + "(:" + serverPort + ")]. Overriding to [" + serverName + ":" +
+                    serverPort + request.getRequestURI() + "]");
+
+            response.setContentType("text/html; charset=utf-8");
+            response.setStatus(HttpServletResponse.SC_OK);
+
+            PrintWriter out = response.getWriter();
+
+            out.println("<h1>System Error</h1>");
+            out.println("<h2>The request contained an invalid host header [" + StringEscapeUtils.escapeHtml4(hostHeader) +
+                    "] in the request [" + StringEscapeUtils.escapeHtml4(request.getRequestURI()) +
+                    "]. Check for request manipulation or third-party intercept. </h2>");
+
+            baseRequest.setHandled(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderSanitizationCustomizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderSanitizationCustomizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderSanitizationCustomizer.java
new file mode 100644
index 0000000..9d55d71
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderSanitizationCustomizer.java
@@ -0,0 +1,80 @@
+/*
+ * 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.nifi.web.server;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.Request;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HostHeaderSanitizationCustomizer implements HttpConfiguration.Customizer {
+    private static final Logger logger = LoggerFactory.getLogger(HostHeaderSanitizationCustomizer.class);
+
+    private final String serverName;
+    private final int serverPort;
+    private final List<String> validHosts;
+
+    /**
+     * @param serverName the {@code serverName} to set on the request (the {@code serverPort} will not be set)
+     */
+    public HostHeaderSanitizationCustomizer(String serverName) {
+        this(serverName, 0);
+    }
+
+    /**
+     * @param serverName the {@code serverName} to set on the request
+     * @param serverPort the {@code serverPort} to set on the request
+     */
+    public HostHeaderSanitizationCustomizer(String serverName, int serverPort) {
+        this.serverName = Objects.requireNonNull(serverName);
+        this.serverPort = serverPort;
+
+        validHosts = new ArrayList<>();
+        validHosts.add(serverName.toLowerCase());
+        validHosts.add(serverName.toLowerCase() + ":" + serverPort);
+        // Sometimes the hostname is left empty but the port is always populated
+        validHosts.add("localhost");
+        validHosts.add("localhost:" + serverPort);
+
+        logger.info("Created " + this.toString());
+    }
+
+    @Override
+    public void customize(Connector connector, HttpConfiguration channelConfig, Request request) {
+        final String hostHeader = request.getHeader("Host");
+        logger.debug("Received request [" + request.getRequestURI() + "] with host header: " + hostHeader);
+        if (!hostHeaderIsValid(hostHeader)) {
+            logger.warn("Request host header [" + hostHeader + "] different from web hostname [" +
+                    serverName + "(:" + serverPort + ")]. Overriding to [" + serverName + ":" +
+                    serverPort + request.getRequestURI() + "]");
+            request.setAuthority(serverName, serverPort);
+        }
+    }
+
+    private boolean hostHeaderIsValid(String hostHeader) {
+        return validHosts.contains(hostHeader.toLowerCase().trim());
+    }
+
+    @Override
+    public String toString() {
+        return "HostHeaderSanitizationCustomizer for " + serverName + ":" + serverPort;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
index bd72c81..744c127 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
@@ -18,6 +18,39 @@ package org.apache.nifi.web.server;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.stream.Collectors;
+import javax.servlet.DispatcherType;
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.NiFiServer;
@@ -51,6 +84,7 @@ import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.server.handler.ContextHandler;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerCollection;
+import org.eclipse.jetty.server.handler.HandlerList;
 import org.eclipse.jetty.server.handler.ResourceHandler;
 import org.eclipse.jetty.server.handler.gzip.GzipHandler;
 import org.eclipse.jetty.servlet.FilterHolder;
@@ -69,40 +103,6 @@ import org.springframework.context.ApplicationContext;
 import org.springframework.web.context.WebApplicationContext;
 import org.springframework.web.context.support.WebApplicationContextUtils;
 
-import javax.servlet.DispatcherType;
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-import java.util.stream.Collectors;
-
 /**
  * Encapsulates the Jetty instance.
  */
@@ -153,10 +153,33 @@ public class JettyServer implements NiFiServer {
         configureConnectors(server);
 
         // load wars from the bundle
-        loadWars(bundles);
+        Handler warHandlers = loadWars(bundles);
+
+        // Create a handler for the host header and add it to the server
+        String serverName = determineServerHostname();
+        int serverPort = determineServerPort();
+        HostHeaderHandler hostHeaderHandler = new HostHeaderHandler(serverName, serverPort);
+        logger.info("Created HostHeaderHandler [" + hostHeaderHandler.toString() + "]");
+
+        HandlerList allHandlers = new HandlerList();
+        allHandlers.addHandler(hostHeaderHandler);
+        allHandlers.addHandler(warHandlers);
+        server.setHandler(allHandlers);
+    }
+
+    private int determineServerPort() {
+        return props.getSslPort() != null ? props.getSslPort() : props.getPort();
+    }
+
+    private String determineServerHostname() {
+        if (props.getSslPort() != null) {
+            return props.getProperty(NiFiProperties.WEB_HTTPS_HOST, "localhost");
+        } else {
+            return props.getProperty(NiFiProperties.WEB_HTTP_HOST, "localhost");
+        }
     }
 
-    private void loadWars(final Set<Bundle> bundles) {
+    private Handler loadWars(final Set<Bundle> bundles) {
 
         // load WARs
         final Map<File, Bundle> warToBundleLookup = findWars(bundles);
@@ -316,17 +339,19 @@ public class JettyServer implements NiFiServer {
         handlers.addHandler(documentationHandlers);
 
         // load the web error app
-        handlers.addHandler(loadWar(webErrorWar, "/", frameworkClassLoader));
+        final WebAppContext webErrorContext = loadWar(webErrorWar, "/", frameworkClassLoader);
+        webErrorContext.getInitParams().put("whitelistedContextPaths", props.getWhitelistedContextPaths());
+        handlers.addHandler(webErrorContext);
 
         // deploy the web apps
-        server.setHandler(gzip(handlers));
+        return gzip(handlers);
     }
 
     /**
      * Returns whether or not the specified ui extensions already contains an extension of the specified type.
      *
-     * @param componentUiExtensionsForType  ui extensions for the type
-     * @param extensionType type of ui extension
+     * @param componentUiExtensionsForType ui extensions for the type
+     * @param extensionType                type of ui extension
      * @return whether or not the specified ui extensions already contains an extension of the specified type
      */
     private boolean containsUiExtensionType(final List<UiExtension> componentUiExtensionsForType, final UiExtensionType extensionType) {
@@ -409,7 +434,7 @@ public class JettyServer implements NiFiServer {
      * Identifies all known UI extensions and stores them in the specified map.
      *
      * @param uiExtensions extensions
-     * @param warFile war
+     * @param warFile      war
      */
     private void identifyUiExtensionsForComponents(final Map<UiExtensionType, List<String>> uiExtensions, final File warFile) {
         try (final JarFile jarFile = new JarFile(warFile)) {
@@ -445,7 +470,7 @@ public class JettyServer implements NiFiServer {
         webappContext.setDisplayName(contextPath);
 
         // instruction jetty to examine these jars for tlds, web-fragments, etc
-        webappContext.setAttribute("org.eclipse.jetty.server.webapp.ContainerIncludeJarPattern", ".*/[^/]*servlet-api-[^/]*\\.jar$|.*/javax.servlet.jsp.jstl-.*\\\\.jar$|.*/[^/]*taglibs.*\\.jar$" );
+        webappContext.setAttribute("org.eclipse.jetty.server.webapp.ContainerIncludeJarPattern", ".*/[^/]*servlet-api-[^/]*\\.jar$|.*/javax.servlet.jsp.jstl-.*\\\\.jar$|.*/[^/]*taglibs.*\\.jar$");
 
         // remove slf4j server class to allow WAR files to have slf4j dependencies in WEB-INF/lib
         List<String> serverClasses = new ArrayList<>(Arrays.asList(webappContext.getServerClasses()));
@@ -522,7 +547,7 @@ public class JettyServer implements NiFiServer {
 
     /**
      * Returns a File object for the directory containing NIFI documentation.
-     *
+     * <p>
      * Formerly, if the docsDirectory did not exist NIFI would fail to start
      * with an IllegalStateException and a rather unhelpful log message.
      * NIFI-2184 updates the process such that if the docsDirectory does not
@@ -582,6 +607,8 @@ public class JettyServer implements NiFiServer {
         httpConfiguration.setRequestHeaderSize(headerSize);
         httpConfiguration.setResponseHeaderSize(headerSize);
 
+        addHostHeaderSanitizationCustomizer(httpConfiguration);
+
         if (props.getPort() != null) {
             final Integer port = props.getPort();
             if (port < 0 || (int) Math.pow(2, 16) <= port) {
@@ -683,6 +710,7 @@ public class JettyServer implements NiFiServer {
         httpsConfiguration.setSecureScheme("https");
         httpsConfiguration.setSecurePort(props.getSslPort());
         httpsConfiguration.addCustomizer(new SecureRequestCustomizer());
+        addHostHeaderSanitizationCustomizer(httpsConfiguration);
 
         // build the connector
         return new ServerConnector(server,
@@ -690,6 +718,18 @@ public class JettyServer implements NiFiServer {
                 new HttpConnectionFactory(httpsConfiguration));
     }
 
+    private void addHostHeaderSanitizationCustomizer(HttpConfiguration httpConfiguration) {
+        // Add the HostHeaderCustomizer to the configuration
+        HttpConfiguration.Customizer hostHeaderCustomizer;
+        if (props.getSslPort() != null) {
+            hostHeaderCustomizer = new HostHeaderSanitizationCustomizer(props.getProperty(NiFiProperties.WEB_HTTPS_HOST), props.getSslPort());
+        } else {
+            hostHeaderCustomizer = new HostHeaderSanitizationCustomizer(props.getProperty(NiFiProperties.WEB_HTTP_HOST), props.getPort());
+        }
+        httpConfiguration.addCustomizer(hostHeaderCustomizer);
+        logger.info("Added HostHeaderSanitizationCustomizer to HttpConfiguration: " + hostHeaderCustomizer);
+    }
+
     private SslContextFactory createSslContextFactory() {
         final SslContextFactory contextFactory = new SslContextFactory();
         configureSslContextFactory(contextFactory, props);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index c1f2226..7118e01 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -16,8 +16,40 @@
  */
 package org.apache.nifi.web.api;
 
+import static javax.ws.rs.core.Response.Status.NOT_FOUND;
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
+
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.CacheControl;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.ResponseBuilder;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriBuilderException;
+import javax.ws.rs.core.UriInfo;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.AuthorizeAccess;
@@ -54,42 +86,10 @@ import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.TransactionResultEntity;
 import org.apache.nifi.web.security.ProxiedEntitiesUtils;
 import org.apache.nifi.web.security.util.CacheKey;
+import org.apache.nifi.web.util.WebUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.core.CacheControl;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedHashMap;
-import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.Response.ResponseBuilder;
-import javax.ws.rs.core.UriBuilder;
-import javax.ws.rs.core.UriBuilderException;
-import javax.ws.rs.core.UriInfo;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import static javax.ws.rs.core.Response.Status.NOT_FOUND;
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
-
 /**
  * Base class for controllers.
  */
@@ -115,10 +115,10 @@ public abstract class ApplicationResource {
     public static final String NODEWISE = "false";
 
     @Context
-    private HttpServletRequest httpServletRequest;
+    protected HttpServletRequest httpServletRequest;
 
     @Context
-    private UriInfo uriInfo;
+    protected UriInfo uriInfo;
 
     protected NiFiProperties properties;
     private RequestReplicator requestReplicator;
@@ -141,26 +141,14 @@ public abstract class ApplicationResource {
         try {
 
             // check for proxy settings
+
             final String scheme = getFirstHeaderValue(PROXY_SCHEME_HTTP_HEADER, FORWARDED_PROTO_HTTP_HEADER);
             final String host = getFirstHeaderValue(PROXY_HOST_HTTP_HEADER, FORWARDED_HOST_HTTP_HEADER);
             final String port = getFirstHeaderValue(PROXY_PORT_HTTP_HEADER, FORWARDED_PORT_HTTP_HEADER);
-            String baseContextPath = getFirstHeaderValue(PROXY_CONTEXT_PATH_HTTP_HEADER, FORWARDED_CONTEXT_HTTP_HEADER);
-
-            // if necessary, prepend the context path
-            String resourcePath = uri.getPath();
-            if (baseContextPath != null) {
-                // normalize context path
-                if (!baseContextPath.startsWith("/")) {
-                    baseContextPath = "/" + baseContextPath;
-                }
 
-                if (baseContextPath.endsWith("/")) {
-                    baseContextPath = StringUtils.substringBeforeLast(baseContextPath, "/");
-                }
-
-                // determine the complete resource path
-                resourcePath = baseContextPath + resourcePath;
-            }
+            // Catch header poisoning
+            String whitelistedContextPaths = properties.getWhitelistedContextPaths();
+            String resourcePath = WebUtils.getResourcePath(uri, httpServletRequest, whitelistedContextPaths);
 
             // determine the port uri
             int uriPort = uri.getPort();
@@ -462,13 +450,13 @@ public abstract class ApplicationResource {
     /**
      * Authorizes the specified process group.
      *
-     * @param processGroupAuthorizable      process group
-     * @param authorizer                    authorizer
-     * @param lookup                        lookup
-     * @param action                        action
-     * @param authorizeReferencedServices   whether to authorize referenced services
-     * @param authorizeTemplates            whether to authorize templates
-     * @param authorizeControllerServices   whether to authorize controller services
+     * @param processGroupAuthorizable    process group
+     * @param authorizer                  authorizer
+     * @param lookup                      lookup
+     * @param action                      action
+     * @param authorizeReferencedServices whether to authorize referenced services
+     * @param authorizeTemplates          whether to authorize templates
+     * @param authorizeControllerServices whether to authorize controller services
      */
     protected void authorizeProcessGroup(final ProcessGroupAuthorizable processGroupAuthorizable, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action,
                                          final boolean authorizeReferencedServices, final boolean authorizeTemplates,
@@ -857,8 +845,8 @@ public abstract class ApplicationResource {
     /**
      * Replicates the request to the given node
      *
-     * @param method the HTTP method
-     * @param entity the Entity to replicate
+     * @param method   the HTTP method
+     * @param entity   the Entity to replicate
      * @param nodeUuid the UUID of the node to replicate the request to
      * @return the response from the node
      * @throws UnknownNodeException if the nodeUuid given does not map to any node in the cluster
@@ -963,7 +951,7 @@ public abstract class ApplicationResource {
      * @throws InterruptedException if interrupted while replicating the request
      */
     protected NodeResponse replicateNodeResponse(final String method) throws InterruptedException {
-        return replicateNodeResponse(method, getRequestParameters(), (Map<String, String>) null);
+        return replicateNodeResponse(method, getRequestParameters(), null);
     }
 
     /**
@@ -1084,8 +1072,8 @@ public abstract class ApplicationResource {
         return properties;
     }
 
-    public static enum ReplicationTarget {
-        CLUSTER_NODES, CLUSTER_COORDINATOR;
+    public enum ReplicationTarget {
+        CLUSTER_NODES, CLUSTER_COORDINATOR
     }
 
     // -----------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/api/ApplicationResourceTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/api/ApplicationResourceTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/api/ApplicationResourceTest.groovy
new file mode 100644
index 0000000..7e773c1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/api/ApplicationResourceTest.groovy
@@ -0,0 +1,181 @@
+/*
+ * 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.nifi.web.api
+
+import org.apache.nifi.properties.StandardNiFiProperties
+import org.apache.nifi.util.NiFiProperties
+import org.glassfish.jersey.uri.internal.JerseyUriBuilder
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.servlet.http.HttpServletRequest
+import javax.ws.rs.core.UriBuilderException
+import javax.ws.rs.core.UriInfo
+
+@RunWith(JUnit4.class)
+class ApplicationResourceTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationResourceTest.class)
+
+    static final String PROXY_SCHEME_HTTP_HEADER = "X-ProxyScheme"
+    static final String PROXY_PORT_HTTP_HEADER = "X-ProxyPort"
+    static final String PROXY_CONTEXT_PATH_HTTP_HEADER = "X-ProxyContextPath"
+
+    static final String FORWARDED_PROTO_HTTP_HEADER = "X-Forwarded-Proto"
+    static final String FORWARDED_PORT_HTTP_HEADER = "X-Forwarded-Port"
+    static final String FORWARDED_CONTEXT_HTTP_HEADER = "X-Forwarded-Context"
+
+    static final String PROXY_CONTEXT_PATH_PROP = NiFiProperties.WEB_PROXY_CONTEXT_PATH
+    static final String WHITELISTED_PATH = "/some/context/path"
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() throws Exception {
+    }
+
+    @After
+    void tearDown() throws Exception {
+    }
+
+    class MockApplicationResource extends ApplicationResource {
+        void setHttpServletRequest(HttpServletRequest request) {
+            super.httpServletRequest = request
+        }
+
+        void setUriInfo(UriInfo uriInfo) {
+            super.uriInfo = uriInfo
+        }
+    }
+
+    private ApplicationResource buildApplicationResource() {
+        ApplicationResource resource = new MockApplicationResource()
+        HttpServletRequest mockRequest = [getHeader: { String k ->
+            logger.mock("Request.getHeader($k)")
+            if ([FORWARDED_CONTEXT_HTTP_HEADER, PROXY_CONTEXT_PATH_HTTP_HEADER].contains(k)) {
+                WHITELISTED_PATH
+            } else if ([FORWARDED_PORT_HTTP_HEADER, PROXY_PORT_HTTP_HEADER].contains(k)) {
+                "8081"
+            } else if ([FORWARDED_PROTO_HTTP_HEADER, PROXY_SCHEME_HTTP_HEADER].contains(k)) {
+                "https"
+            } else {
+                "nifi.apache.org"
+            }
+        }, getContextPath: { ->
+            logger.mock("Request.getContextPath()")
+            ""
+        }] as HttpServletRequest
+
+        UriInfo mockUriInfo = [getBaseUriBuilder: { ->
+            logger.mock("Returning mock UriBuilder")
+            new JerseyUriBuilder().uri(new URI('https://nifi.apache.org/'))
+        }] as UriInfo
+
+        resource.setHttpServletRequest(mockRequest)
+        resource.setUriInfo(mockUriInfo)
+        resource.properties = new StandardNiFiProperties()
+
+        resource
+    }
+
+    @Test
+    void testGenerateUriShouldBlockProxyContextPathHeaderIfNotInWhitelist() throws Exception {
+        // Arrange
+        ApplicationResource resource = buildApplicationResource()
+        logger.info("Whitelisted path(s): ")
+
+        // Act
+        def msg = shouldFail(UriBuilderException) {
+            String generatedUri = resource.generateResourceUri('actualResource')
+            logger.unexpected("Generated URI: ${generatedUri}")
+        }
+
+        // Assert
+        logger.expected(msg)
+        assert msg =~ "The provided context path \\[.*\\] was not whitelisted \\[\\]"
+    }
+
+    @Test
+    void testGenerateUriShouldAllowProxyContextPathHeaderIfInWhitelist() throws Exception {
+        // Arrange
+        ApplicationResource resource = buildApplicationResource()
+        logger.info("Whitelisted path(s): ${WHITELISTED_PATH}")
+        NiFiProperties niFiProperties = new StandardNiFiProperties([(PROXY_CONTEXT_PATH_PROP): WHITELISTED_PATH] as Properties)
+        resource.properties = niFiProperties
+
+        // Act
+        String generatedUri = resource.generateResourceUri('actualResource')
+        logger.info("Generated URI: ${generatedUri}")
+
+        // Assert
+        assert generatedUri == "https://nifi.apache.org:8081${WHITELISTED_PATH}/actualResource"
+    }
+
+    @Test
+    void testGenerateUriShouldAllowProxyContextPathHeaderIfElementInMultipleWhitelist() throws Exception {
+        // Arrange
+        ApplicationResource resource = buildApplicationResource()
+        String multipleWhitelistedPaths = [WHITELISTED_PATH, "another/path", "a/third/path"].join(",")
+        logger.info("Whitelisted path(s): ${multipleWhitelistedPaths}")
+        NiFiProperties niFiProperties = new StandardNiFiProperties([(PROXY_CONTEXT_PATH_PROP): multipleWhitelistedPaths] as Properties)
+        resource.properties = niFiProperties
+
+        // Act
+        String generatedUri = resource.generateResourceUri('actualResource')
+        logger.info("Generated URI: ${generatedUri}")
+
+        // Assert
+        assert generatedUri == "https://nifi.apache.org:8081${WHITELISTED_PATH}/actualResource"
+    }
+
+    @Test
+    void testGenerateUriShouldBlockForwardedContextHeaderIfNotInWhitelist() throws Exception {
+        // Arrange
+
+        // Act
+
+        // Assert
+    }
+
+    @Test
+    void testGenerateUriShouldAllowForwardedContextHeaderIfInWhitelist() throws Exception {
+        // Arrange
+
+        // Act
+
+        // Assert
+    }
+
+    @Test
+    void testGenerateUriShouldAllowForwardedContextHeaderIfElementInMultipleWhitelist() throws Exception {
+        // Arrange
+
+        // Act
+
+        // Assert
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/logback-test.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/logback-test.xml
index 6bbe800..3f875e4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/logback-test.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/logback-test.xml
@@ -30,6 +30,7 @@
     
     
     <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.web.api" level="DEBUG"/>
     <root level="INFO">
         <appender-ref ref="CONSOLE"/>
     </root>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
index 7b102a8..9437a3a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
@@ -31,5 +31,10 @@
             <groupId>javax.servlet</groupId>
             <artifactId>javax.servlet-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/java/org/apache/nifi/web/filter/CatchAllFilter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/java/org/apache/nifi/web/filter/CatchAllFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/java/org/apache/nifi/web/filter/CatchAllFilter.java
index eeb3c1f..71d1a2b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/java/org/apache/nifi/web/filter/CatchAllFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/java/org/apache/nifi/web/filter/CatchAllFilter.java
@@ -23,18 +23,35 @@ import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.UriBuilderException;
+import org.apache.nifi.web.util.WebUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Filter for forward all requests to index.jsp.
  */
 public class CatchAllFilter implements Filter {
+    private static final Logger logger = LoggerFactory.getLogger(CatchAllFilter.class);
+
+    private static String whitelistedContextPaths = "";
 
     @Override
     public void init(FilterConfig filterConfig) throws ServletException {
+        String providedWhitelist = filterConfig.getServletContext().getInitParameter("whitelistedContextPaths");
+        logger.debug("CatchAllFilter received provided whitelisted context paths from NiFi properties: " + providedWhitelist);
+        if (providedWhitelist != null) {
+            whitelistedContextPaths = providedWhitelist;
+        }
     }
 
     @Override
     public void doFilter(ServletRequest request, ServletResponse response, FilterChain filterChain) throws IOException, ServletException {
+        // Capture the provided context path headers and sanitize them before using in the response
+        String contextPath = getSanitizedContextPath(request);
+        request.setAttribute("contextPath", contextPath);
+
         // for all requests to index.jsp
         request.getRequestDispatcher("/index.jsp").forward(request, response);
     }
@@ -42,4 +59,23 @@ public class CatchAllFilter implements Filter {
     @Override
     public void destroy() {
     }
+
+    /**
+     * Returns a "safe" context path value from the request headers to use in a proxy environment.
+     * This is used on the JSP to build the resource paths for the external resources (CSS, JS, etc.).
+     * If no headers are present specifying this value, it is an empty string.
+     *
+     * @param request the HTTP request
+     * @return the context path safe to be printed to the page
+     */
+    private String getSanitizedContextPath(ServletRequest request) {
+        String contextPath = WebUtils.normalizeContextPath(WebUtils.determineContextPath((HttpServletRequest) request));
+        try {
+            WebUtils.verifyContextPath(whitelistedContextPaths, contextPath);
+            return contextPath;
+        } catch (UriBuilderException e) {
+            logger.error("Error determining context path on index.jsp: " + e.getMessage());
+            return "";
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/webapp/index.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/webapp/index.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/webapp/index.jsp
index e029a80..37ea642 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/webapp/index.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/src/main/webapp/index.jsp
@@ -17,36 +17,29 @@
 <%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
 <!DOCTYPE html>
 <html>
-    <%
-        String contextPath = request.getHeader("X-ProxyContextPath");
-        if (contextPath == null) {
-            contextPath = request.getHeader("X-Forwarded-Context");
-        }
-        if (contextPath == null) {
-            contextPath = "";
-        }
-        if (contextPath.endsWith("/")) {
-            contextPath = contextPath.substring(0, contextPath.length() - 1);
-        }
-    %>
-    <head>
-        <meta http-equiv="Content-Type" content="text/html; charset=utf-8" />
-        <link rel="shortcut icon" href="<%= contextPath %>/nifi/images/nifi16.ico"/>
-        <title>NiFi</title>
-        <link rel="stylesheet" href="<%= contextPath %>/nifi/assets/reset.css/reset.css" type="text/css" />
-        <link rel="stylesheet" href="<%= contextPath %>/nifi/css/common-ui.css" type="text/css" />
-        <link rel="stylesheet" href="<%= contextPath %>/nifi/fonts/flowfont/flowfont.css" type="text/css" />
-        <link rel="stylesheet" href="<%= contextPath %>/nifi/assets/font-awesome/css/font-awesome.min.css" type="text/css" />
-        <link rel="stylesheet" href="<%= contextPath %>/nifi/css/message-pane.css" type="text/css" />
-        <link rel="stylesheet" href="<%= contextPath %>/nifi/css/message-page.css" type="text/css" />
-    </head>
+<%
+    // Sanitize the contextPath to ensure it is on this server
+    // rather than getting it from the header directly
+    String contextPath = request.getAttribute("contextPath").toString();
+%>
+<head>
+    <meta http-equiv="Content-Type" content="text/html; charset=utf-8"/>
+    <link rel="shortcut icon" href="<%= contextPath %>/nifi/images/nifi16.ico"/>
+    <title>NiFi</title>
+    <link rel="stylesheet" href="<%= contextPath %>/nifi/assets/reset.css/reset.css" type="text/css"/>
+    <link rel="stylesheet" href="<%= contextPath %>/nifi/css/common-ui.css" type="text/css"/>
+    <link rel="stylesheet" href="<%= contextPath %>/nifi/fonts/flowfont/flowfont.css" type="text/css"/>
+    <link rel="stylesheet" href="<%= contextPath %>/nifi/assets/font-awesome/css/font-awesome.min.css" type="text/css"/>
+    <link rel="stylesheet" href="<%= contextPath %>/nifi/css/message-pane.css" type="text/css"/>
+    <link rel="stylesheet" href="<%= contextPath %>/nifi/css/message-page.css" type="text/css"/>
+</head>
 
-    <body class="message-pane">
-        <div class="message-pane-message-box">
-            <p class="message-pane-title">
-                Did you mean: <a href="<%= contextPath %>/nifi/">/nifi</a>
-            </p>
-            <p class="message-pane-content">You may have mistyped...</p>
-        </div>
-    </body>
+<body class="message-pane">
+<div class="message-pane-message-box">
+    <p class="message-pane-title">
+        Did you mean: <a href="<%= contextPath %>/nifi/">/nifi</a>
+    </p>
+    <p class="message-pane-content">You may have mistyped...</p>
+</div>
+</body>
 </html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index 5d23370..9ef4b74 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@ -16,6 +16,20 @@
  */
 package org.apache.nifi.processors.standard;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import javax.servlet.Servlet;
+import javax.ws.rs.Path;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -49,21 +63,6 @@ import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.eclipse.jetty.util.thread.QueuedThreadPool;
 
-import javax.servlet.Servlet;
-import javax.ws.rs.Path;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-
 @InputRequirement(Requirement.INPUT_FORBIDDEN)
 @Tags({"ingest", "http", "https", "rest", "listen"})
 @CapabilityDescription("Starts an HTTP Server and listens on a given base path to transform incoming requests into FlowFiles. "
@@ -206,7 +205,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
         final StreamThrottler streamThrottler = (maxBytesPerSecond == null) ? null : new LeakyBucketStreamThrottler(maxBytesPerSecond.intValue());
         throttlerRef.set(streamThrottler);
 
-        final boolean needClientAuth = sslContextService == null ? false : sslContextService.getTrustStoreFile() != null;
+        final boolean needClientAuth = sslContextService != null && sslContextService.getTrustStoreFile() != null;
 
         final SslContextFactory contextFactory = new SslContextFactory();
         contextFactory.setNeedClientAuth(needClientAuth);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5d643edf/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
index 71e60a9..068be02 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.websocket.jetty;
 
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
@@ -49,13 +55,6 @@ import org.eclipse.jetty.websocket.servlet.WebSocketCreator;
 import org.eclipse.jetty.websocket.servlet.WebSocketServlet;
 import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory;
 
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
 @Tags({"WebSocket", "Jetty", "server"})
 @CapabilityDescription("Implementation of WebSocketServerService." +
         " This service uses Jetty WebSocket server module to provide" +