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 2018/01/19 15:58:46 UTC

nifi git commit: NIFI-4761 Added HostHeaderHandler regression unit tests before adding new functionality. Added logic for parsing nifi.web.proxy.host property. Added default property in nifi.properties and pom.xml. Added logic for IPv6 and custom default

Repository: nifi
Updated Branches:
  refs/heads/master 94eb11bfa -> 8cb09c301


NIFI-4761
Added HostHeaderHandler regression unit tests before adding new functionality.
Added logic for parsing nifi.web.proxy.host property.
Added default property in nifi.properties and pom.xml.
Added logic for IPv6 and custom default hostnames.
Improved error messaging.
Added HostHeaderHandler unit tests.
Disabled HostHeaderSanitizationCustomizer in HTTP mode.
Fixed HTML escaping in error message.
Improved error message.
Added failing unit test for parsing custom hostnames.
Fixed custom hostname parsing.
Fixed unit tests.
Added TODO for IPv6 custom hostname parsing and unit test.
Added IPv6 custom hostname parsing and unit tests.
Fixed checkstyle issues.
Removed empty element in host list when no value defined.
Improved error message formatting.
Added unit tests.
Removed HostHeaderSanitizationCustomizer.
Removed InvalidPropertiesFormatException from NiFiProperties.
Removed InvalidPropertiesFormatException from HostHeaderHandler.
This closes #2415


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

Branch: refs/heads/master
Commit: 8cb09c301d6fef70cc8a02a4a7e80f3062ab58ae
Parents: 94eb11b
Author: Andy LoPresto <al...@apache.org>
Authored: Wed Jan 17 11:36:00 2018 -0800
Committer: Matt Gilman <ma...@gmail.com>
Committed: Fri Jan 19 10:53:13 2018 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/util/NiFiProperties.java    |  55 +++-
 .../StandardNiFiPropertiesGroovyTest.groovy     | 130 +++++++++
 .../nifi-framework/nifi-resources/pom.xml       |   1 +
 .../src/main/resources/conf/nifi.properties     |   1 +
 .../nifi/web/server/HostHeaderHandler.java      | 286 ++++++++++++++++--
 .../HostHeaderSanitizationCustomizer.java       |  87 ------
 .../org/apache/nifi/web/server/JettyServer.java |  43 +--
 .../web/server/HostHeaderHandlerTest.groovy     | 287 +++++++++++++++++++
 8 files changed, 744 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/8cb09c30/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 55cae14..46704ef 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
@@ -178,6 +178,7 @@ public abstract class NiFiProperties {
     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";
+    public static final String WEB_PROXY_HOST = "nifi.web.proxy.host";
 
     // ui properties
     public static final String UI_BANNER_TEXT = "nifi.ui.banner.text";
@@ -287,6 +288,7 @@ public abstract class NiFiProperties {
     // Kerberos defaults
     public static final String DEFAULT_KERBEROS_AUTHENTICATION_EXPIRATION = "12 hours";
 
+
     /**
      * Retrieves the property value for the given property key.
      *
@@ -587,6 +589,25 @@ public abstract class NiFiProperties {
         return sslPort;
     }
 
+    public boolean isHTTPSConfigured() {
+        return getSslPort() != null;
+    }
+
+    /**
+     * Determines the HTTP/HTTPS port NiFi is configured to bind to. Prefers the HTTPS port. Throws an exception if neither is configured.
+     *
+     * @return the configured port number
+     */
+    public Integer getConfiguredHttpOrHttpsPort() throws RuntimeException {
+        if (getSslPort() != null) {
+            return getSslPort();
+        } else if (getPort() != null) {
+            return getPort();
+        } else {
+            throw new RuntimeException("The HTTP or HTTPS port must be configured");
+        }
+    }
+
     public String getWebMaxHeaderSize() {
         return getProperty(WEB_MAX_HEADER_SIZE, DEFAULT_WEB_MAX_HEADER_SIZE);
     }
@@ -1287,8 +1308,40 @@ public abstract class NiFiProperties {
     }
 
     /**
-     * Returns the whitelisted proxy context paths as a comma-delimited string. The paths have been normalized to the form {@code /some/context/path}.
+     * Returns the whitelisted proxy hostnames (and IP addresses) as a comma-delimited string.
+     * The hosts have been normalized to the form {@code somehost.com}, {@code somehost.com:port}, or {@code 127.0.0.1}.
+     * <p>
+     * Note: Calling {@code NiFiProperties.getProperty(NiFiProperties.WEB_PROXY_HOST)} will not normalize the hosts.
+     *
+     * @return the hostname(s)
+     */
+    public String getWhitelistedHosts() {
+        return StringUtils.join(getWhitelistedHostsAsList(), ",");
+    }
+
+    /**
+     * Returns the whitelisted proxy hostnames (and IP addresses) as a List. The hosts have been normalized to the form {@code somehost.com}, {@code somehost.com:port}, or {@code 127.0.0.1}.
      *
+     * @return the hostname(s)
+     */
+    public List<String> getWhitelistedHostsAsList() {
+        String rawProperty = getProperty(WEB_PROXY_HOST, "");
+        List<String> hosts = Arrays.asList(rawProperty.split(","));
+        return hosts.stream()
+                .map(this::normalizeHost).filter(host -> !StringUtils.isBlank(host)).collect(Collectors.toList());
+    }
+
+    String normalizeHost(String host) {
+        if (host == null || host.equalsIgnoreCase("")) {
+            return "";
+        } else {
+            return host.trim();
+        }
+    }
+
+    /**
+     * Returns the whitelisted proxy context paths as a comma-delimited string. The paths have been normalized to the form {@code /some/context/path}.
+     * <p>
      * Note: Calling {@code NiFiProperties.getProperty(NiFiProperties.WEB_PROXY_CONTEXT_PATH)} will not normalize the paths.
      *
      * @return the path(s)

http://git-wip-us.apache.org/repos/asf/nifi/blob/8cb09c30/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 64c4f0f..478dc2a 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
@@ -433,4 +433,134 @@ class StandardNiFiPropertiesGroovyTest extends GroovyTestCase {
         // Assert
         assert normalizedContextPath == empty
     }
+
+    @Test
+    void testShouldNormalizeProxyHostProperty() {
+        // Arrange
+        String extraSpaceHostname = "somehost.com  "
+        Properties rawProps = new Properties(["nifi.web.proxy.host": extraSpaceHostname])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${extraSpaceHostname}]")
+
+        // Act
+        String normalizedHostname = props.getWhitelistedHosts()
+        logger.info("Read from NiFiProperties instance: ${normalizedHostname}")
+
+        // Assert
+        assert extraSpaceHostname.startsWith(normalizedHostname)
+        assert extraSpaceHostname.length() == normalizedHostname.length() + 2
+    }
+
+    @Test
+    void testShouldHandleNormalizedProxyHostProperty() {
+        // Arrange
+        String hostname = "somehost.com"
+        Properties rawProps = new Properties(["nifi.web.proxy.host": hostname])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${hostname}]")
+
+        // Act
+        String normalizedHostname = props.getWhitelistedHosts()
+        logger.info("Read from NiFiProperties instance: ${normalizedHostname}")
+
+        // Assert
+        assert hostname == normalizedHostname
+    }
+
+    @Test
+    void testShouldNormalizeMultipleProxyHostsInProperty() {
+        // Arrange
+        String extraSpaceHostname = "somehost.com  "
+        String normalHostname = "someotherhost.com"
+        String hostnameWithPort = "otherhost.com:1234"
+        String extraSpaceHostnameWithPort = "  anotherhost.com:9999"
+        List<String> hosts = [extraSpaceHostname, normalHostname, hostnameWithPort, extraSpaceHostnameWithPort]
+        String combinedHosts = hosts.join(",")
+        Properties rawProps = new Properties(["nifi.web.proxy.host": combinedHosts])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${combinedHosts}]")
+
+        // Act
+        String normalizedHostname = props.getWhitelistedHosts()
+        logger.info("Read from NiFiProperties instance: ${normalizedHostname}")
+
+        // Assert
+        def splitHosts = normalizedHostname.split(",")
+        def expectedValues = hosts*.trim()
+        splitHosts.every {
+            assert it.trim() == it
+            assert expectedValues.contains(it)
+        }
+    }
+
+    @Test
+    void testShouldHandleNormalizedProxyHostPropertyAsList() {
+        // Arrange
+        String normalHostname = "someotherhost.com"
+        Properties rawProps = new Properties(["nifi.web.proxy.host": normalHostname])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${normalHostname}]")
+
+        // Act
+        def listOfHosts = props.getWhitelistedHostsAsList()
+        logger.info("Read from NiFiProperties instance: ${listOfHosts}")
+
+        // Assert
+        assert listOfHosts.size() == 1
+        assert listOfHosts.contains(normalHostname)
+    }
+
+    @Test
+    void testShouldNormalizeMultipleProxyHostsInPropertyAsList() {
+        // Arrange
+        String extraSpaceHostname = "somehost.com  "
+        String normalHostname = "someotherhost.com"
+        String hostnameWithPort = "otherhost.com:1234"
+        String extraSpaceHostnameWithPort = "  anotherhost.com:9999"
+        List<String> hosts = [extraSpaceHostname, normalHostname, hostnameWithPort, extraSpaceHostnameWithPort]
+        String combinedHosts = hosts.join(",")
+        Properties rawProps = new Properties(["nifi.web.proxy.host": combinedHosts])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${combinedHosts}]")
+
+        // Act
+        def listOfHosts = props.getWhitelistedHostsAsList()
+        logger.info("Read from NiFiProperties instance: ${listOfHosts}")
+
+        // Assert
+        assert listOfHosts.size() == 4
+        assert listOfHosts.containsAll([extraSpaceHostname[0..-3], normalHostname, hostnameWithPort, extraSpaceHostnameWithPort[2..-1]])
+    }
+
+    @Test
+    void testShouldHandleNormalizingEmptyProxyHostProperty() {
+        // Arrange
+        String empty = ""
+        Properties rawProps = new Properties(["nifi.web.proxy.host": empty])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${empty}]")
+
+        // Act
+        String normalizedHost = props.getWhitelistedHosts()
+        logger.info("Read from NiFiProperties instance: ${normalizedHost}")
+
+        // Assert
+        assert normalizedHost == empty
+    }
+
+    @Test
+    void testShouldReturnEmptyProxyHostPropertyAsList() {
+        // Arrange
+        String empty = ""
+        Properties rawProps = new Properties(["nifi.web.proxy.host": empty])
+        NiFiProperties props = new StandardNiFiProperties(rawProps)
+        logger.info("Created a NiFiProperties instance with raw proxy host property [${empty}]")
+
+        // Act
+        def hosts = props.getWhitelistedHostsAsList()
+        logger.info("Read from NiFiProperties instance: ${hosts}")
+
+        // Assert
+        assert hosts.size() == 0
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/8cb09c30/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 d4855c6..59a69c5 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
@@ -133,6 +133,7 @@
         <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.web.proxy.host />
 
         <!-- nifi.properties: security properties -->
         <nifi.security.keystore />

http://git-wip-us.apache.org/repos/asf/nifi/blob/8cb09c30/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 f6a1ed5..868ec0a 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
@@ -140,6 +140,7 @@ 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}
+nifi.web.proxy.host=${nifi.web.proxy.host}
 
 # security properties #
 nifi.sensitive.props.key=

http://git-wip-us.apache.org/repos/asf/nifi/blob/8cb09c30/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
index 61e825d..9a07fcb 100644
--- 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
@@ -16,20 +16,31 @@
  */
 package org.apache.nifi.web.server;
 
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.http.conn.util.InetAddressUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.handler.ScopedHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 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;
+import java.util.stream.Collectors;
 
 public class HostHeaderHandler extends ScopedHandler {
     private static final Logger logger = LoggerFactory.getLogger(HostHeaderHandler.class);
@@ -39,38 +50,149 @@ public class HostHeaderHandler extends ScopedHandler {
     private final List<String> validHosts;
 
     /**
+     * Instantiates a handler with a given server name and port 0.
+     *
      * @param serverName the {@code serverName} to set on the request (the {@code serverPort} will not be set)
+     * @deprecated Use {@link #HostHeaderHandler(NiFiProperties)} which accepts a {@link NiFiProperties} object to allow for custom network interface binding.
      */
     public HostHeaderHandler(String serverName) {
         this(serverName, 0);
     }
 
     /**
+     * Instantiates a handler with a given server name and port.
+     *
      * @param serverName the {@code serverName} to set on the request
      * @param serverPort the {@code serverPort} to set on the request
+     * @deprecated Use {@link #HostHeaderHandler(NiFiProperties)} which accepts a {@link NiFiProperties} object to allow for custom network interface binding.
      */
     public HostHeaderHandler(String serverName, int serverPort) {
         this.serverName = Objects.requireNonNull(serverName);
         this.serverPort = serverPort;
 
-        validHosts = new ArrayList<>();
+        // Default values across generic instances
+        validHosts = generateDefaultHostnames(null);
+
+        // specified host:port
         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
+
+        // empty is ok here
         validHosts.add("");
-        try {
-            validHosts.add(InetAddress.getLocalHost().getHostName().toLowerCase());
-            validHosts.add(InetAddress.getLocalHost().getHostName().toLowerCase() + ":" + serverPort);
-        } catch (final Exception e) {
-            logger.warn("Failed to determine local hostname.", e);
-        }
 
         logger.info("Created " + this.toString());
     }
 
+    /**
+     * Instantiates a handler which accepts incoming requests with a host header that is empty or contains one of the
+     * valid hosts. See the Apache NiFi Admin Guide for instructions on how to set valid hostnames and IP addresses.
+     *
+     * @param niFiProperties the NiFiProperties
+     */
+    public HostHeaderHandler(NiFiProperties niFiProperties) {
+        this.serverName = Objects.requireNonNull(determineServerHostname(niFiProperties));
+        this.serverPort = determineServerPort(niFiProperties);
+
+        // Default values across generic instances
+        List<String> hosts = generateDefaultHostnames(niFiProperties);
+
+        // The value from nifi.web.http|https.host
+        hosts.add(serverName.toLowerCase());
+        hosts.add(serverName.toLowerCase() + ":" + serverPort);
+
+        // The value(s) from nifi.web.proxy.host
+        hosts.addAll(parseCustomHostnames(niFiProperties));
+
+        // empty is ok here
+        hosts.add("");
+
+        this.validHosts = uniqueList(hosts);
+        logger.info("Determined {} valid hostnames and IP addresses for incoming headers: {}", new Object[]{validHosts.size(), StringUtils.join(validHosts, ", ")});
+
+        logger.debug("Created " + this.toString());
+    }
+
+    /**
+     * Returns the list of parsed custom hostnames from {@code nifi.web.proxy.host} in {@link NiFiProperties}.
+     * This list is deduplicated (if a host {@code somehost.com:1234} is provided, it will show twice, as the "portless"
+     * version {@code somehost.com} is also generated). IPv6 addresses are only modified if they adhere to the strict
+     * formatting using {@code []} around the address as specified in RFC 5952 Section 6 (i.e.
+     * {@code [1234.5678.90AB.CDEF.1234.5678.90AB.CDEF]:1234} will insert
+     * {@code [1234.5678.90AB.CDEF.1234.5678.90AB.CDEF]} as well).
+     *
+     * @param niFiProperties the properties object
+     * @return the list of parsed custom hostnames
+     */
+    List<String> parseCustomHostnames(NiFiProperties niFiProperties) {
+        // Load the custom hostnames from the properties
+        List<String> customHostnames = niFiProperties.getWhitelistedHostsAsList();
+
+        /* Each IPv4 address and hostname may have the port associated, so duplicate the list and trim the port
+        * (the port may be different from the port NiFi is running on if provided by a proxy, etc.) IPv6 addresses
+        * are not modified.
+        */
+        List<String> portlessHostnames = customHostnames.stream().map(hostname -> {
+                    if (isIPv6Address(hostname)) {
+                        return hostname;
+                    } else {
+                        return StringUtils.substringBeforeLast(hostname, ":");
+                    }
+                }
+        ).collect(Collectors.toList());
+
+        customHostnames.addAll(portlessHostnames);
+        if (logger.isDebugEnabled()) {
+            logger.debug("Parsed {} custom hostnames from nifi.web.proxy.host: {}", new Object[]{customHostnames.size(), StringUtils.join(customHostnames, ", ")});
+        }
+        return uniqueList(customHostnames);
+    }
+
+    /**
+     * Returns a unique {@code List} of the elements maintaining the original order.
+     *
+     * @param duplicateList a list that may contain duplicate elements
+     * @return a list maintaining the original order which no longer contains duplicate elements
+     */
+    private static List<String> uniqueList(List<String> duplicateList) {
+        return new ArrayList<>(new LinkedHashSet<>(duplicateList));
+    }
+
+    /**
+     * Returns true if the provided address is an IPv6 address (or could be interpreted as one). This method is more
+     * lenient than {@link InetAddressUtils#isIPv6Address(String)} because of different interpretations of IPv4-mapped
+     * IPv6 addresses.
+     *
+     * See RFC 5952 Section 4 for more information on textual representation of the IPv6 addresses.
+     *
+     * @param address the address in text form
+     * @return true if the address is or could be parsed as an IPv6 address
+     */
+    static boolean isIPv6Address(String address) {
+        // Note: InetAddressUtils#isIPv4MappedIPv64Address() fails on addresses that do not compress the leading 0:0:0... to ::
+        // Expanded for debugging purposes
+        boolean isNormalIPv6 = InetAddressUtils.isIPv6Address(address);
+
+        // If the last two hextets are written in IPv4 form, treat it as an IPv6 address as well
+        String everythingAfterLastColon = StringUtils.substringAfterLast(address, ":");
+        boolean isIPv4 = InetAddressUtils.isIPv4Address(everythingAfterLastColon);
+        boolean isIPv4Mapped = InetAddressUtils.isIPv4MappedIPv64Address(everythingAfterLastColon);
+        boolean isCompressable = address.contains("0:0") && !address.contains("::");
+
+        return isNormalIPv6 || isIPv4;
+    }
+
+    private int determineServerPort(NiFiProperties props) {
+        return props.getSslPort() != null ? props.getSslPort() : props.getPort();
+    }
+
+    private String determineServerHostname(NiFiProperties props) {
+        if (props.getSslPort() != null) {
+            return props.getProperty(NiFiProperties.WEB_HTTPS_HOST, "localhost");
+        } else {
+            return props.getProperty(NiFiProperties.WEB_HTTP_HOST, "localhost");
+        }
+    }
+
     @Override
     public void doScope(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
             throws IOException, ServletException {
@@ -78,7 +200,7 @@ public class HostHeaderHandler extends ScopedHandler {
         nextScope(target, baseRequest, request, response);
     }
 
-    private boolean hostHeaderIsValid(String hostHeader) {
+    boolean hostHeaderIsValid(String hostHeader) {
         return validHosts.contains(hostHeader.toLowerCase().trim());
     }
 
@@ -91,10 +213,10 @@ public class HostHeaderHandler extends ScopedHandler {
      * 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 target      the target (not relevant here)
      * @param baseRequest the original request object
-     * @param request the request as an HttpServletRequest
-     * @param response the current response
+     * @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 {
@@ -111,11 +233,123 @@ public class HostHeaderHandler extends ScopedHandler {
             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>");
+            // TODO: Change to org.apache.commons.text.StringEscapeUtils
+            out.println("<h2>The request contained an invalid host header [<code>" + StringEscapeUtils.escapeHtml4(hostHeader) +
+                    "</code>] in the request [<code>" + StringEscapeUtils.escapeHtml4(request.getRequestURI()) +
+                    "</code>]. Check for request manipulation or third-party intercept.</h2>");
+            out.println("<h3>Valid host headers are [<code>empty</code>] or: <br/><code>");
+            out.println(printValidHosts());
+            out.println("</code></h3>");
 
             baseRequest.setHandled(true);
         }
     }
+
+    String printValidHosts() {
+        StringBuilder sb = new StringBuilder("<ul>");
+        for (String vh : validHosts) {
+            if (StringUtils.isNotBlank(vh))
+                sb.append("<li>").append(StringEscapeUtils.escapeHtml4(vh)).append("</li>\n");
+        }
+        return sb.append("</ul>\n").toString();
+    }
+
+    public static List<String> generateDefaultHostnames(NiFiProperties niFiProperties) {
+        List<String> validHosts = new ArrayList<>();
+        int serverPort = 0;
+
+        if (niFiProperties == null) {
+            logger.warn("NiFiProperties not configured; returning minimal default hostnames");
+        } else {
+            try {
+                serverPort = niFiProperties.getConfiguredHttpOrHttpsPort();
+            } catch (RuntimeException e) {
+                logger.warn("Cannot fully generate list of default hostnames because the server port is not configured in nifi.properties. Defaulting to port 0 for host header evaluation");
+            }
+
+            // Add any custom network interfaces
+            try {
+                final int lambdaPort = serverPort;
+                List<String> customIPs = extractIPsFromNetworkInterfaces(niFiProperties);
+                customIPs.stream().forEach(ip -> {
+                    validHosts.add(ip);
+                    validHosts.add(ip + ":" + lambdaPort);
+                });
+            } catch (final Exception e) {
+                logger.warn("Failed to determine custom network interfaces.", e);
+            }
+        }
+
+        // Sometimes the hostname is left empty but the port is always populated
+        validHosts.add("127.0.0.1");
+        validHosts.add("127.0.0.1:" + serverPort);
+        validHosts.add("localhost");
+        validHosts.add("localhost:" + serverPort);
+        validHosts.add("[::1]");
+        validHosts.add("[::1]:" + serverPort);
+
+        // Add the loopback and actual IP address and hostname used
+        try {
+            validHosts.add(InetAddress.getLoopbackAddress().getHostAddress().toLowerCase());
+            validHosts.add(InetAddress.getLoopbackAddress().getHostAddress().toLowerCase() + ":" + serverPort);
+
+            validHosts.add(InetAddress.getLocalHost().getHostName().toLowerCase());
+            validHosts.add(InetAddress.getLocalHost().getHostName().toLowerCase() + ":" + serverPort);
+
+            validHosts.add(InetAddress.getLocalHost().getHostAddress().toLowerCase());
+            validHosts.add(InetAddress.getLocalHost().getHostAddress().toLowerCase() + ":" + serverPort);
+        } catch (final Exception e) {
+            logger.warn("Failed to determine local hostname.", e);
+        }
+
+        // Dedupe but maintain order
+        final List<String> uniqueHosts = uniqueList(validHosts);
+        if (logger.isDebugEnabled()) {
+            logger.debug("Determined {} valid default hostnames and IP addresses for incoming headers: {}", new Object[]{uniqueHosts.size(), StringUtils.join(uniqueHosts, ", ")});
+        }
+        return uniqueHosts;
+    }
+
+    /**
+     * Extracts the list of IP addresses from custom bound network interfaces. If both HTTPS and HTTP interfaces are
+     * defined and HTTPS is enabled, only HTTPS interfaces will be returned. If none are defined, an empty list will be
+     * returned.
+     *
+     * @param niFiProperties the NiFiProperties object
+     * @return the list of IP addresses
+     */
+    static List<String> extractIPsFromNetworkInterfaces(NiFiProperties niFiProperties) {
+        Map<String, String> networkInterfaces = niFiProperties.isHTTPSConfigured() ? niFiProperties.getHttpsNetworkInterfaces() : niFiProperties.getHttpNetworkInterfaces();
+        if (isNotDefined(networkInterfaces)) {
+            // No custom interfaces defined
+            return new ArrayList<>(0);
+        } else {
+            List<String> allIPAddresses = new ArrayList<>();
+            for (Map.Entry<String, String> entry : networkInterfaces.entrySet()) {
+                final String networkInterfaceName = entry.getValue();
+                try {
+                    NetworkInterface ni = NetworkInterface.getByName(networkInterfaceName);
+                    List<String> ipAddresses = Collections.list(ni.getInetAddresses()).stream().map(inetAddress -> inetAddress.getHostAddress().toLowerCase()).collect(Collectors.toList());
+                    logger.debug("Resolved the following IP addresses for network interface {}: {}", new Object[]{networkInterfaceName, StringUtils.join(ipAddresses, ", ")});
+                    allIPAddresses.addAll(ipAddresses);
+                } catch (SocketException e) {
+                    logger.warn("Cannot resolve network interface named " + networkInterfaceName);
+                }
+            }
+
+            // Dedupe while maintaining order
+            return uniqueList(allIPAddresses);
+        }
+    }
+
+    /**
+     * Returns true if the provided map of properties and network interfaces is null, empty, or the actual definitions are empty.
+     *
+     * @param networkInterfaces the map of properties to bindings
+     *                          ({@code ["nifi.web.http.network.interface.first":"eth0"]})
+     * @return
+     */
+    static boolean isNotDefined(Map<String, String> networkInterfaces) {
+        return networkInterfaces == null || networkInterfaces.isEmpty() || networkInterfaces.values().stream().filter(value -> !Strings.isNullOrEmpty(value)).collect(Collectors.toList()).isEmpty();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/8cb09c30/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
deleted file mode 100644
index 3a7b22b..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/HostHeaderSanitizationCustomizer.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.net.InetAddress;
-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);
-        try {
-            validHosts.add(InetAddress.getLocalHost().getHostName().toLowerCase());
-            validHosts.add(InetAddress.getLocalHost().getHostName().toLowerCase() + ":" + serverPort);
-        } catch (final Exception e) {
-            logger.warn("Failed to determine local hostname.", e);
-        }
-
-        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/8cb09c30/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 744c127..de53a24 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
@@ -155,28 +155,22 @@ public class JettyServer implements NiFiServer {
         // load wars from the bundle
         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();
-    }
+        // Only restrict the host header if running in HTTPS mode
+        if (props.isHTTPSConfigured()) {
+            // Create a handler for the host header and add it to the server
+            HostHeaderHandler hostHeaderHandler = new HostHeaderHandler(props);
+            logger.info("Created HostHeaderHandler [" + hostHeaderHandler.toString() + "]");
 
-    private String determineServerHostname() {
-        if (props.getSslPort() != null) {
-            return props.getProperty(NiFiProperties.WEB_HTTPS_HOST, "localhost");
+            // Add this before the WAR handlers
+            allHandlers.addHandler(hostHeaderHandler);
         } else {
-            return props.getProperty(NiFiProperties.WEB_HTTP_HOST, "localhost");
+            logger.info("Running in HTTP mode; host headers not restricted");
         }
+
+        allHandlers.addHandler(warHandlers);
+        server.setHandler(allHandlers);
     }
 
     private Handler loadWars(final Set<Bundle> bundles) {
@@ -607,8 +601,6 @@ 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) {
@@ -710,7 +702,6 @@ 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,
@@ -718,18 +709,6 @@ 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/8cb09c30/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/HostHeaderHandlerTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/HostHeaderHandlerTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/HostHeaderHandlerTest.groovy
new file mode 100644
index 0000000..a08be9d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/HostHeaderHandlerTest.groovy
@@ -0,0 +1,287 @@
+/*
+ * 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 org.apache.commons.lang3.StringUtils
+import org.apache.nifi.properties.StandardNiFiProperties
+import org.apache.nifi.util.NiFiProperties
+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
+
+@RunWith(JUnit4.class)
+class HostHeaderHandlerTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(HostHeaderHandlerTest.class)
+
+    private static final String DEFAULT_HOSTNAME = "nifi.apache.org"
+    private static final String ACTUAL_HOSTNAME = InetAddress.getLocalHost().getHostName().toLowerCase()
+    private static final int DEFAULT_PORT = 8080
+    private static final List<String> DEFAULT_HOSTS_1_5_0 = [DEFAULT_HOSTNAME, "localhost", ACTUAL_HOSTNAME]
+    private static
+    final List<String> DEFAULT_HOSTS_AND_PORTS_1_5_0 = DEFAULT_HOSTS_1_5_0.collectMany { it -> [it, "${it}:${DEFAULT_PORT}"] }
+
+    // Post 1.5.0 list
+    private static final String ACTUAL_IP = InetAddress.getLocalHost().getHostAddress()
+    private static final String LOOPBACK_IP = InetAddress.getLoopbackAddress().getHostAddress()
+    private static
+    final List<String> DEFAULT_HOSTS = DEFAULT_HOSTS_1_5_0 - DEFAULT_HOSTNAME + ["[::1]", "127.0.0.1", ACTUAL_IP, LOOPBACK_IP]
+    private static
+    final List<String> DEFAULT_HOSTS_AND_PORTS = DEFAULT_HOSTS.collectMany { it -> [it, "${it}:${DEFAULT_PORT}"] }
+
+    @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 {
+    }
+
+    @Test
+    void testConstructorShouldAcceptSingleValues() throws Exception {
+        // Arrange
+        String hostname = DEFAULT_HOSTNAME
+        int port = DEFAULT_PORT
+        logger.info("Hostname: ${hostname} | port: ${port}")
+
+        // Act
+        HostHeaderHandler handler = new HostHeaderHandler(hostname, port)
+        logger.info("Handler: ${handler}")
+
+        // Assert
+        assert handler.hostHeaderIsValid(hostname)
+        assert handler.hostHeaderIsValid("${hostname}:${port}")
+    }
+
+    /**
+     * The feature was introduced in Apache NiFi 1.5.0 but the behavior was changed following that release to include the actual IP address of the server, IPv6 ::1, and 127.0.0.1.
+     * @throws Exception
+     */
+    @Test
+    void testShouldHandle_1_5_0_DefaultValues() throws Exception {
+        // Arrange
+        String hostname = DEFAULT_HOSTNAME
+        int port = DEFAULT_PORT
+        logger.info("Hostname: ${hostname} | port: ${port}")
+
+        // Act
+        HostHeaderHandler handler = new HostHeaderHandler(hostname, port)
+        logger.info("Handler: ${handler}")
+
+        // Assert
+        DEFAULT_HOSTS_AND_PORTS_1_5_0.each { String host ->
+            logger.debug("Validating ${host}")
+            assert handler.hostHeaderIsValid(host)
+        }
+    }
+
+    @Test
+    void testNewConstructorShouldHandleCurrentDefaultValues() throws Exception {
+        // Arrange
+        String hostname = DEFAULT_HOSTNAME
+        int port = DEFAULT_PORT
+        logger.info("Hostname: ${hostname} | port: ${port}")
+
+        Properties rawProps = new Properties()
+        rawProps.putAll([
+                (NiFiProperties.WEB_HTTPS_HOST): DEFAULT_HOSTNAME,
+                (NiFiProperties.WEB_HTTPS_PORT): "${DEFAULT_PORT}".toString(),
+        ])
+        NiFiProperties simpleProperties = new StandardNiFiProperties(rawProps)
+
+        // Act
+        HostHeaderHandler handler = new HostHeaderHandler(simpleProperties)
+        logger.info("Handler: ${handler}")
+
+        // Assert
+        DEFAULT_HOSTS_AND_PORTS.each { String host ->
+            logger.debug("Validating ${host}")
+            assert handler.hostHeaderIsValid(host)
+        }
+    }
+
+    @Test
+    void testShouldParseCustomHostnames() throws Exception {
+        // Arrange
+        String hostname = DEFAULT_HOSTNAME
+        int port = DEFAULT_PORT
+        logger.info("Hostname: ${hostname} | port: ${port}")
+
+        List<String> otherHosts = ["someotherhost.com:9999", "yetanotherbadhost.com", "10.10.10.1:1234", "100.100.100.1"]
+        String concatenatedHosts = otherHosts.join(",")
+
+        Properties rawProps = new Properties()
+        rawProps.putAll([
+                (NiFiProperties.WEB_HTTPS_HOST): DEFAULT_HOSTNAME,
+                (NiFiProperties.WEB_HTTPS_PORT): "${DEFAULT_PORT}".toString(),
+                (NiFiProperties.WEB_PROXY_HOST): concatenatedHosts
+        ])
+        NiFiProperties simpleProperties = new StandardNiFiProperties(rawProps)
+
+        HostHeaderHandler handler = new HostHeaderHandler(simpleProperties)
+        logger.info("Handler: ${handler}")
+
+        // Act
+        List<String> customHostnames = handler.parseCustomHostnames(simpleProperties)
+        logger.info("Parsed custom hostnames: ${customHostnames}")
+
+        // Assert
+        assert customHostnames.size() == otherHosts.size() + 2 // Two provided hostnames had ports
+        otherHosts.each { String host ->
+            logger.debug("Checking ${host}")
+            assert customHostnames.contains(host)
+            String portlessHost = "${host.split(":", 2)[0]}".toString()
+            logger.debug("Checking ${portlessHost}")
+            assert customHostnames.contains(portlessHost)
+        }
+    }
+
+    @Test
+    void testParseCustomHostnamesShouldHandleIPv6WithoutPorts() throws Exception {
+        // Arrange
+        String hostname = DEFAULT_HOSTNAME
+        int port = DEFAULT_PORT
+        logger.info("Hostname: ${hostname} | port: ${port}")
+
+        List<String> ipv6Hosts = ["ABCD:EF01:2345:6789:ABCD:EF01:2345:6789",
+                                  "2001:DB8:0:0:8:800:200C:417A",
+                                  "FF01:0:0:0:0:0:0:101",
+                                  "0:0:0:0:0:0:0:1",
+                                  "0:0:0:0:0:0:0:0",
+                                  "2001:DB8::8:800:200C:417A",
+                                  "FF01::101",
+                                  "::1",
+                                  "::",
+                                  "0:0:0:0:0:0:13.1.68.3",
+                                  "0:0:0:0:0:FFFF:129.144.52.38",
+                                  "::13.1.68.3",
+                                  "FFFF:129.144.52.38",
+                                  "::FFFF:129.144.52.38"]
+        String concatenatedHosts = ipv6Hosts.join(",")
+
+        Properties rawProps = new Properties()
+        rawProps.putAll([
+                (NiFiProperties.WEB_HTTPS_HOST): DEFAULT_HOSTNAME,
+                (NiFiProperties.WEB_HTTPS_PORT): "${DEFAULT_PORT}".toString(),
+                (NiFiProperties.WEB_PROXY_HOST): concatenatedHosts
+        ])
+        NiFiProperties simpleProperties = new StandardNiFiProperties(rawProps)
+
+        HostHeaderHandler handler = new HostHeaderHandler(simpleProperties)
+        logger.info("Handler: ${handler}")
+
+        // Act
+        List<String> customHostnames = handler.parseCustomHostnames(simpleProperties)
+        logger.info("Parsed custom hostnames: ${customHostnames}")
+
+        // Assert
+        assert customHostnames.size() == ipv6Hosts.size()
+        ipv6Hosts.each { String host ->
+            logger.debug("Checking ${host}")
+            assert customHostnames.contains(host)
+        }
+    }
+
+    @Test
+    void testParseCustomHostnamesShouldHandleIPv6WithPorts() throws Exception {
+        // Arrange
+        String hostname = DEFAULT_HOSTNAME
+        int port = DEFAULT_PORT
+        logger.info("Hostname: ${hostname} | port: ${port}")
+
+        List<String> ipv6Hosts = ["[ABCD:EF01:2345:6789:ABCD:EF01:2345:6789]:1234",
+                                  "[2001:DB8:0:0:8:800:200C:417A]:1234",
+                                  "[FF01:0:0:0:0:0:0:101]:1234",
+                                  "[0:0:0:0:0:0:0:1]:1234",
+                                  "[0:0:0:0:0:0:0:0]:1234",
+                                  "[2001:DB8::8:800:200C:417A]:1234",
+                                  "[FF01::101]:1234",
+                                  "[::1]:1234",
+                                  "[::]:1234",
+                                  "[0:0:0:0:0:0:13.1.68.3]:1234",
+                                  "[0:0:0:0:0:FFFF:129.144.52.38]:1234",
+                                  "[::13.1.68.3]:1234",
+                                  "[FFFF:129.144.52.38]:1234",
+                                  "[::FFFF:129.144.52.38]:1234"]
+        String concatenatedHosts = ipv6Hosts.join(",")
+
+        Properties rawProps = new Properties()
+        rawProps.putAll([
+                (NiFiProperties.WEB_HTTPS_HOST): DEFAULT_HOSTNAME,
+                (NiFiProperties.WEB_HTTPS_PORT): "${DEFAULT_PORT}".toString(),
+                (NiFiProperties.WEB_PROXY_HOST): concatenatedHosts
+        ])
+        NiFiProperties simpleProperties = new StandardNiFiProperties(rawProps)
+
+        HostHeaderHandler handler = new HostHeaderHandler(simpleProperties)
+        logger.info("Handler: ${handler}")
+
+        // Act
+        List<String> customHostnames = handler.parseCustomHostnames(simpleProperties)
+        logger.info("Parsed custom hostnames: ${customHostnames}")
+
+        // Assert
+        assert customHostnames.size() == ipv6Hosts.size() * 2
+        ipv6Hosts.each { String host ->
+            logger.debug("Checking ${host}")
+            assert customHostnames.contains(host)
+            String portlessHost = "${StringUtils.substringBeforeLast(host, ":")}".toString()
+            logger.debug("Checking ${portlessHost}")
+            assert customHostnames.contains(portlessHost)
+        }
+    }
+
+    @Test
+    void testShouldIdentifyIPv6Addresses() throws Exception {
+        // Arrange
+        List<String> ipv6Hosts = ["ABCD:EF01:2345:6789:ABCD:EF01:2345:6789",
+                                  "2001:DB8:0:0:8:800:200C:417A",
+                                  "FF01:0:0:0:0:0:0:101",
+                                  "0:0:0:0:0:0:0:1",
+                                  "0:0:0:0:0:0:0:0",
+                                  "2001:DB8::8:800:200C:417A",
+                                  "FF01::101",
+                                  "::1",
+                                  "::",
+                                  "0:0:0:0:0:0:13.1.68.3",
+                                  "0:0:0:0:0:FFFF:129.144.52.38",
+                                  "::13.1.68.3",
+                                  "FFFF:129.144.52.38",
+                                  "::FFFF:129.144.52.38"]
+
+        // Act
+        List<Boolean> hostsAreIPv6 = ipv6Hosts.collect { String host ->
+            boolean isIPv6 = HostHeaderHandler.isIPv6Address(host)
+            logger.info("Hostname is IPv6: ${host} | ${isIPv6}")
+            isIPv6
+        }
+
+        // Assert
+        assert hostsAreIPv6.every()
+    }
+}