You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by js...@apache.org on 2017/09/27 20:23:49 UTC

nifi git commit: NIFI-4382: - Adding support for KnoxSSO. - Updated the docs for nifi.security.user.knox.audiences. - The KnoxSSO cookie is removed prior to request replication.

Repository: nifi
Updated Branches:
  refs/heads/master d47bbd12c -> 6c798d18e


NIFI-4382:
- Adding support for KnoxSSO.
- Updated the docs for nifi.security.user.knox.audiences.
- The KnoxSSO cookie is removed prior to request replication.

This closes #2177


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

Branch: refs/heads/master
Commit: 6c798d18ef2e20ff6d575cdcd63618fe05e71012
Parents: d47bbd1
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Sep 14 12:45:23 2017 -0400
Committer: Jeff Storck <jt...@gmail.com>
Committed: Wed Sep 27 16:22:18 2017 -0400

----------------------------------------------------------------------
 .../org/apache/nifi/util/NiFiProperties.java    |  65 ++++-
 .../src/main/asciidoc/administration-guide.adoc |  30 ++-
 .../ThreadPoolRequestReplicator.java            |  94 ++++---
 .../nifi-framework/nifi-resources/pom.xml       |   6 +
 .../src/main/resources/conf/nifi.properties     |   8 +-
 .../org/apache/nifi/web/server/JettyServer.java |   1 +
 .../web/NiFiWebApiSecurityConfiguration.java    |  28 ++-
 .../org/apache/nifi/web/api/AccessResource.java |  63 +++++
 .../src/main/resources/nifi-web-api-context.xml |   1 +
 .../web/security/NiFiAuthenticationFilter.java  |  20 +-
 .../security/knox/KnoxAuthenticationFilter.java |  71 ++++++
 .../knox/KnoxAuthenticationProvider.java        |  68 ++++++
 .../knox/KnoxAuthenticationRequestToken.java    |  59 +++++
 .../web/security/knox/KnoxConfiguration.java    |  33 +++
 .../nifi/web/security/knox/KnoxService.java     | 244 +++++++++++++++++++
 .../security/knox/KnoxServiceFactoryBean.java   |  56 +++++
 .../knox/StandardKnoxConfiguration.java         |  73 ++++++
 .../oidc/StandardOidcIdentityProvider.java      |   4 +-
 .../resources/nifi-web-security-context.xml     |  12 +
 .../knox/KnoxAuthenticationFilterTest.java      | 104 ++++++++
 .../nifi/web/security/knox/KnoxServiceTest.java | 217 +++++++++++++++++
 .../org/apache/nifi/web/filter/LoginFilter.java |   4 +
 22 files changed, 1203 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/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 fd26a97..d51dea4 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
@@ -33,6 +33,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * The NiFiProperties class holds all properties which are needed for various
@@ -156,6 +157,12 @@ public abstract class NiFiProperties {
     public static final String SECURITY_USER_OIDC_CLIENT_SECRET = "nifi.security.user.oidc.client.secret";
     public static final String SECURITY_USER_OIDC_PREFERRED_JWSALGORITHM = "nifi.security.user.oidc.preferred.jwsalgorithm";
 
+    // apache knox
+    public static final String SECURITY_USER_KNOX_URL = "nifi.security.user.knox.url";
+    public static final String SECURITY_USER_KNOX_PUBLIC_KEY = "nifi.security.user.knox.publicKey";
+    public static final String SECURITY_USER_KNOX_COOKIE_NAME = "nifi.security.user.knox.cookieName";
+    public static final String SECURITY_USER_KNOX_AUDIENCES = "nifi.security.user.knox.audiences";
+
     // web properties
     public static final String WEB_WAR_DIR = "nifi.web.war.directory";
     public static final String WEB_HTTP_PORT = "nifi.web.http.port";
@@ -886,18 +893,70 @@ public abstract class NiFiProperties {
     }
 
     /**
+     * Returns whether Knox SSO is enabled.
+     *
+     * @return whether Knox SSO is enabled
+     */
+    public boolean isKnoxSsoEnabled() {
+        return !StringUtils.isBlank(getKnoxUrl());
+    }
+
+    /**
+     * Returns the Knox URL.
+     *
+     * @return Knox URL
+     */
+    public String getKnoxUrl() {
+        return getProperty(SECURITY_USER_KNOX_URL);
+    }
+
+    /**
+     * Gets the configured Knox Audiences.
+     *
+     * @return Knox audiences
+     */
+    public Set<String> getKnoxAudiences() {
+        final String rawAudiences = getProperty(SECURITY_USER_KNOX_AUDIENCES);
+        if (StringUtils.isBlank(rawAudiences)) {
+            return null;
+        } else {
+            final String[] audienceTokens = rawAudiences.split(",");
+            return Stream.of(audienceTokens).map(String::trim).filter(aud -> !StringUtils.isEmpty(aud)).collect(Collectors.toSet());
+        }
+    }
+
+    /**
+     * Returns the path to the Knox public key.
+     *
+     * @return path to the Knox public key
+     */
+    public Path getKnoxPublicKeyPath() {
+        return Paths.get(getProperty(SECURITY_USER_KNOX_PUBLIC_KEY));
+    }
+
+    /**
+     * Returns the name of the Knox cookie.
+     *
+     * @return name of the Knox cookie
+     */
+    public String getKnoxCookieName() {
+        return getProperty(SECURITY_USER_KNOX_COOKIE_NAME);
+    }
+
+    /**
      * Returns true if client certificates are required for REST API. Determined
      * if the following conditions are all true:
      * <p>
      * - login identity provider is not populated
      * - Kerberos service support is not enabled
      * - openid connect is not enabled
+     * - knox sso is not enabled
+     * </p>
      *
-     * @return true if client certificates are required for access to the REST
-     * API
+     * @return true if client certificates are required for access to the REST API
      */
     public boolean isClientAuthRequiredForRestApi() {
-        return !isLoginIdentityProviderEnabled() && !isKerberosSpnegoSupportEnabled() && !isOidcEnabled();
+        return !isLoginIdentityProviderEnabled() && !isKerberosSpnegoSupportEnabled() && !isOidcEnabled() && !isKnoxSsoEnabled();
     }
 
     public InetSocketAddress getNodeApiAddress() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 4ad79ee..540e817 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -282,20 +282,24 @@ For a client certificate that can be easily imported into the browser, specify:
 User Authentication
 -------------------
 
-NiFi supports user authentication via client certificates, via username/password, or using OpenId Connect (http://openid.net/connect).
+NiFi supports user authentication via client certificates, via username/password, via Apache Knox, or via OpenId Connect (http://openid.net/connect).
 
 Username/password authentication is performed by a 'Login Identity Provider'. The Login Identity Provider is a pluggable mechanism for
-authenticating users via their username/password. Which Login Identity Provider to use is configured in two properties in the _nifi.properties_ file.
+authenticating users via their username/password. Which Login Identity Provider to use is configured in the _nifi.properties_ file.
+Currently NiFi offers username/password with Login Identity Providers options for LDAP and Kerberos.
 
 The `nifi.login.identity.provider.configuration.file` property specifies the configuration file for Login Identity Providers.
 The `nifi.security.user.login.identity.provider` property indicates which of the configured Login Identity Provider should be
-used. If this property is not configured, NiFi will not support username/password authentication and will require client
-certificates for authenticating users over HTTPS. By default, this property is not configured meaning that username/password must be explicitly enabled.
+used. By default, this property is not configured meaning that username/password must be explicitly enabled.
 
 During OpenId Connect authentication, NiFi will redirect users to login with the Provider before returning to NiFi. NiFi will then
 call the Provider to obtain the user identity.
 
-NOTE: NiFi cannot be configured for both username/password and OpenId Connect authentication at the same time.
+During Apache Knox authentication, NiFi will redirect users to login with Apache Knox before returning to NiFi. NiFi will verify the Apache Knox
+token during authentication.
+
+NOTE: NiFi can only be configured for username/password, OpenId Connect, or Apache Knox at a given time. It does not support running each of
+these concurrently. NiFi will require client certificates for authenticating users over HTTPS if none of these are configured.
 
 A secured instance of NiFi cannot be accessed anonymously unless configured to use an LDAP or Kerberos Login Identity Provider, which in turn must be configured to explicitly allow anonymous access. Anonymous access is not currently possible by the default FileAuthorizer (see <<authorizer-configuration>>), but is a future effort (https://issues.apache.org/jira/browse/NIFI-2730[NIFI-2730]).
 
@@ -423,6 +427,22 @@ If this value is 'none', NiFi will attempt to validate unsecured/plain tokens. O
 JSON Web Key (JWK) provided through the jwks_uri in the metadata found at the discovery URL.
 |==================================================================================================================================================
 
+[[apache_knox]]
+Apache Knox
+~~~~~~~~~~~
+
+To enable authentication via Apache Knox the following properties must be configured in nifi.properties.
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`nifi.security.user.knox.url` | The URL for the Apache Knox log in page.
+|`nifi.security.user.knox.publicKey` | The path to the Apache Knox public key that will be used to verify the signatures of the authentication tokens in the HTTP Cookie.
+|`nifi.security.user.knox.cookieName` | The name of the HTTP Cookie that Apache Knox will generate after successful log in.
+|`nifi.security.user.knox.audiences` | Optional. A comma separate listed of allowed audiences. If set, the audience in the token must be present in
+this listing. The audience that is populated in the token can be configured in Knox.
+|==================================================================================================================================================
+
 [[multi-tenant-authorization]]
 Multi-Tenant Authorization
 --------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
index 7bdf6fa..6f57ad0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
@@ -17,37 +17,13 @@
 
 package org.apache.nifi.cluster.coordination.http.replication;
 
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.LongSummaryStatistics;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.Response.Status;
-
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -74,12 +50,36 @@ import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response.Status;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.LongSummaryStatistics;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 public class ThreadPoolRequestReplicator implements RequestReplicator {
 
@@ -93,6 +93,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
     private final EventReporter eventReporter;
     private final RequestCompletionCallback callback;
     private final ClusterCoordinator clusterCoordinator;
+    private final NiFiProperties nifiProperties;
 
     private ThreadPoolExecutor executorService;
     private ScheduledExecutorService maintenanceExecutor;
@@ -154,6 +155,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
         this.responseMapper = new StandardHttpResponseMapper(nifiProperties);
         this.eventReporter = eventReporter;
         this.callback = callback;
+        this.nifiProperties = nifiProperties;
 
         client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs);
         client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs);
@@ -248,6 +250,24 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
         // will happen when the request is replicated using the proxy chain above
         headers.remove(JwtAuthenticationFilter.AUTHORIZATION);
 
+        // if knox sso cookie name is set, remove any authentication cookie since this user is already authenticated
+        // and will be included in the proxied entities chain above... authorization will happen when the
+        // request is replicated
+        final String knoxCookieName = nifiProperties.getKnoxCookieName();
+        if (headers.containsKey("Cookie") && StringUtils.isNotBlank(knoxCookieName)) {
+            final String rawCookies = headers.get("Cookie");
+            final String[] rawCookieParts = rawCookies.split(";");
+            final Set<String> filteredCookieParts = Stream.of(rawCookieParts).map(String::trim).filter(cookie -> !cookie.startsWith(knoxCookieName + "=")).collect(Collectors.toSet());
+
+            // if that was the only cookie, remove it
+            if (filteredCookieParts.isEmpty()) {
+                headers.remove("Cookie");
+            } else {
+                // otherwise rebuild the cookies without the knox token
+                headers.put("Cookie", StringUtils.join(filteredCookieParts, "; "));
+            }
+        }
+
         // remove the host header
         headers.remove("Host");
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/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 b61da28..f4fe36d 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
@@ -155,6 +155,12 @@
         <nifi.security.user.oidc.client.secret />
         <nifi.security.user.oidc.preferred.jwsalgorithm />
 
+        <!-- nifi.properties: apache knox -->
+        <nifi.security.user.knox.url />
+        <nifi.security.user.knox.publicKey />
+        <nifi.security.user.knox.cookieName>hadoop-jwt</nifi.security.user.knox.cookieName>
+        <nifi.security.user.knox.audiences />
+
         <!-- nifi.properties: cluster common properties (cluster manager and nodes must have same values) -->
         <nifi.cluster.protocol.heartbeat.interval>5 sec</nifi.cluster.protocol.heartbeat.interval>
         <nifi.cluster.protocol.is.secure>false</nifi.cluster.protocol.is.secure>

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/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 54c6e5d..f867ed3 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
@@ -156,7 +156,7 @@ nifi.security.user.login.identity.provider=${nifi.security.user.login.identity.p
 nifi.security.ocsp.responder.url=${nifi.security.ocsp.responder.url}
 nifi.security.ocsp.responder.certificate=${nifi.security.ocsp.responder.certificate}
 
-# OpenId Connect Properties #
+# OpenId Connect SSO Properties #
 nifi.security.user.oidc.discovery.url=${nifi.security.user.oidc.discovery.url}
 nifi.security.user.oidc.connect.timeout=${nifi.security.user.oidc.connect.timeout}
 nifi.security.user.oidc.read.timeout=${nifi.security.user.oidc.read.timeout}
@@ -164,6 +164,12 @@ nifi.security.user.oidc.client.id=${nifi.security.user.oidc.client.id}
 nifi.security.user.oidc.client.secret=${nifi.security.user.oidc.client.secret}
 nifi.security.user.oidc.preferred.jwsalgorithm=${nifi.security.user.oidc.preferred.jwsalgorithm}
 
+# Apache Knox SSO Properties #
+nifi.security.user.knox.url=${nifi.security.user.knox.url}
+nifi.security.user.knox.publicKey=${nifi.security.user.knox.publicKey}
+nifi.security.user.knox.cookieName=${nifi.security.user.knox.cookieName}
+nifi.security.user.knox.audiences=${nifi.security.user.knox.audiences}
+
 # Identity Mapping Properties #
 # These properties allow normalizing user identities such that identities coming from different identity providers
 # (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/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 cab3922..7332f92 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
@@ -291,6 +291,7 @@ public class JettyServer implements NiFiServer {
         // load the web ui app
         final WebAppContext webUiContext = loadWar(webUiWar, "/nifi", frameworkClassLoader);
         webUiContext.getInitParams().put("oidc-supported", String.valueOf(props.isOidcEnabled()));
+        webUiContext.getInitParams().put("knox-supported", String.valueOf(props.isKnoxSsoEnabled()));
         handlers.addHandler(webUiContext);
 
         // load the web api app

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
index 26f2443..8d76bf3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
@@ -20,6 +20,8 @@ import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.security.anonymous.NiFiAnonymousUserFilter;
 import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
 import org.apache.nifi.web.security.jwt.JwtAuthenticationProvider;
+import org.apache.nifi.web.security.knox.KnoxAuthenticationFilter;
+import org.apache.nifi.web.security.knox.KnoxAuthenticationProvider;
 import org.apache.nifi.web.security.otp.OtpAuthenticationFilter;
 import org.apache.nifi.web.security.otp.OtpAuthenticationProvider;
 import org.apache.nifi.web.security.x509.X509AuthenticationFilter;
@@ -65,6 +67,9 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
     private OtpAuthenticationFilter otpAuthenticationFilter;
     private OtpAuthenticationProvider otpAuthenticationProvider;
 
+    private KnoxAuthenticationFilter knoxAuthenticationFilter;
+    private KnoxAuthenticationProvider knoxAuthenticationProvider;
+
     private NiFiAnonymousUserFilter anonymousAuthenticationFilter;
 
     public NiFiWebApiSecurityConfiguration() {
@@ -78,7 +83,7 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
         // the /access/download-token and /access/ui-extension-token endpoints
         webSecurity
                 .ignoring()
-                    .antMatchers("/access", "/access/config", "/access/token", "/access/kerberos", "/access/oidc/**");
+                    .antMatchers("/access", "/access/config", "/access/token", "/access/kerberos", "/access/oidc/**", "/access/knox/**");
     }
 
     @Override
@@ -100,6 +105,9 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
         // otp
         http.addFilterBefore(otpFilterBean(), AnonymousAuthenticationFilter.class);
 
+        // knox
+        http.addFilterBefore(knoxFilterBean(), AnonymousAuthenticationFilter.class);
+
         // anonymous
         http.anonymous().authenticationFilter(anonymousFilterBean());
     }
@@ -116,7 +124,8 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
         auth
                 .authenticationProvider(x509AuthenticationProvider)
                 .authenticationProvider(jwtAuthenticationProvider)
-                .authenticationProvider(otpAuthenticationProvider);
+                .authenticationProvider(otpAuthenticationProvider)
+                .authenticationProvider(knoxAuthenticationProvider);
     }
 
     @Bean
@@ -140,6 +149,16 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
     }
 
     @Bean
+    public KnoxAuthenticationFilter knoxFilterBean() throws Exception {
+        if (knoxAuthenticationFilter == null) {
+            knoxAuthenticationFilter = new KnoxAuthenticationFilter();
+            knoxAuthenticationFilter.setProperties(properties);
+            knoxAuthenticationFilter.setAuthenticationManager(authenticationManager());
+        }
+        return knoxAuthenticationFilter;
+    }
+
+    @Bean
     public X509AuthenticationFilter x509FilterBean() throws Exception {
         if (x509AuthenticationFilter == null) {
             x509AuthenticationFilter = new X509AuthenticationFilter();
@@ -175,6 +194,11 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
     }
 
     @Autowired
+    public void setKnoxAuthenticationProvider(KnoxAuthenticationProvider knoxAuthenticationProvider) {
+        this.knoxAuthenticationProvider = knoxAuthenticationProvider;
+    }
+
+    @Autowired
     public void setX509AuthenticationProvider(X509AuthenticationProvider x509AuthenticationProvider) {
         this.x509AuthenticationProvider = x509AuthenticationProvider;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.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/AccessResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
index 5c0360e..3031e2e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
@@ -53,6 +53,7 @@ import org.apache.nifi.web.security.jwt.JwtAuthenticationProvider;
 import org.apache.nifi.web.security.jwt.JwtAuthenticationRequestToken;
 import org.apache.nifi.web.security.jwt.JwtService;
 import org.apache.nifi.web.security.kerberos.KerberosService;
+import org.apache.nifi.web.security.knox.KnoxService;
 import org.apache.nifi.web.security.oidc.OidcService;
 import org.apache.nifi.web.security.otp.OtpService;
 import org.apache.nifi.web.security.token.LoginAuthenticationToken;
@@ -111,6 +112,7 @@ public class AccessResource extends ApplicationResource {
     private JwtService jwtService;
     private OtpService otpService;
     private OidcService oidcService;
+    private KnoxService knoxService;
 
     private KerberosService kerberosService;
 
@@ -313,6 +315,63 @@ public class AccessResource extends ApplicationResource {
         return generateOkResponse(jwt).build();
     }
 
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.WILDCARD)
+    @Path("knox/request")
+    @ApiOperation(
+            value = "Initiates a request to authenticate through Apache Knox.",
+            notes = NON_GUARANTEED_ENDPOINT
+    )
+    public void knoxRequest(@Context HttpServletRequest httpServletRequest, @Context HttpServletResponse httpServletResponse) throws Exception {
+        // only consider user specific access over https
+        if (!httpServletRequest.isSecure()) {
+            forwardToMessagePage(httpServletRequest, httpServletResponse, "User authentication/authorization is only supported when running over HTTPS.");
+            return;
+        }
+
+        // ensure knox is enabled
+        if (!knoxService.isKnoxEnabled()) {
+            forwardToMessagePage(httpServletRequest, httpServletResponse, "Apache Knox SSO support is not configured.");
+            return;
+        }
+
+        // build the originalUri, and direct back to the ui
+        final String originalUri = generateResourceUri("access", "knox", "callback");
+
+        // build the authorization uri
+        final URI authorizationUri = UriBuilder.fromUri(knoxService.getKnoxUrl())
+                .queryParam("originalUrl", originalUri.toString())
+                .build();
+
+        // generate the response
+        httpServletResponse.sendRedirect(authorizationUri.toString());
+    }
+
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.WILDCARD)
+    @Path("knox/callback")
+    @ApiOperation(
+            value = "Redirect/callback URI for processing the result of the Apache Knox login sequence.",
+            notes = NON_GUARANTEED_ENDPOINT
+    )
+    public void knoxCallback(@Context HttpServletRequest httpServletRequest, @Context HttpServletResponse httpServletResponse) throws Exception {
+        // only consider user specific access over https
+        if (!httpServletRequest.isSecure()) {
+            forwardToMessagePage(httpServletRequest, httpServletResponse, "User authentication/authorization is only supported when running over HTTPS.");
+            return;
+        }
+
+        // ensure knox is enabled
+        if (!knoxService.isKnoxEnabled()) {
+            forwardToMessagePage(httpServletRequest, httpServletResponse, "Apache Knox SSO support is not configured.");
+            return;
+        }
+
+        httpServletResponse.sendRedirect("../../../nifi");
+    }
+
     /**
      * Gets the status the client's access.
      *
@@ -735,4 +794,8 @@ public class AccessResource extends ApplicationResource {
     public void setOidcService(OidcService oidcService) {
         this.oidcService = oidcService;
     }
+
+    public void setKnoxService(KnoxService knoxService) {
+        this.knoxService = knoxService;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 8ee39f0..f79b7c8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -384,6 +384,7 @@
     <bean id="accessResource" class="org.apache.nifi.web.api.AccessResource" scope="singleton">
         <property name="loginIdentityProvider" ref="loginIdentityProvider"/>
         <property name="oidcService" ref="oidcService"/>
+        <property name="knoxService" ref="knoxService"/>
         <property name="x509AuthenticationProvider" ref="x509AuthenticationProvider"/>
         <property name="certificateExtractor" ref="certificateExtractor"/>
         <property name="principalExtractor" ref="principalExtractor"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/NiFiAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/NiFiAuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/NiFiAuthenticationFilter.java
index ec41ebf..75f1c56 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/NiFiAuthenticationFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/NiFiAuthenticationFilter.java
@@ -16,14 +16,6 @@
  */
 package org.apache.nifi.web.security;
 
-import java.io.IOException;
-import java.io.PrintWriter;
-import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.util.NiFiProperties;
@@ -36,6 +28,15 @@ import org.springframework.security.core.AuthenticationException;
 import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.web.filter.GenericFilterBean;
 
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+
 /**
  *
  */
@@ -150,4 +151,7 @@ public abstract class NiFiAuthenticationFilter extends GenericFilterBean {
         this.properties = properties;
     }
 
+    public NiFiProperties getProperties() {
+        return properties;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilter.java
new file mode 100644
index 0000000..ed73366
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilter.java
@@ -0,0 +1,71 @@
+/*
+ * 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.security.knox;
+
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.NiFiAuthenticationFilter;
+import org.springframework.security.core.Authentication;
+
+import javax.servlet.http.Cookie;
+import javax.servlet.http.HttpServletRequest;
+
+/**
+ */
+public class KnoxAuthenticationFilter extends NiFiAuthenticationFilter {
+
+    @Override
+    public Authentication attemptAuthentication(final HttpServletRequest request) {
+        // only support knox login when running securely
+        if (!request.isSecure()) {
+            return null;
+        }
+
+        // ensure knox sso support is enabled
+        final NiFiProperties properties = getProperties();
+        if (!properties.isKnoxSsoEnabled()) {
+            return null;
+        }
+
+        // get the principal out of the user token
+        final String knoxJwt = getJwtFromCookie(request, properties.getKnoxCookieName());
+
+        // if there is no cookie, return null to attempt another authentication
+        if (knoxJwt == null) {
+            return null;
+        } else {
+            // otherwise create the authentication request token
+            return new KnoxAuthenticationRequestToken(knoxJwt, request.getRemoteAddr());
+        }
+    }
+
+    public String getJwtFromCookie(final HttpServletRequest request, final String cookieName) {
+        String jwt = null;
+
+        final Cookie[] cookies = request.getCookies();
+        if (cookies != null) {
+            for (Cookie cookie : cookies) {
+                if (cookieName.equals(cookie.getName())) {
+                    jwt = cookie.getValue();
+                    break;
+                }
+            }
+        }
+
+        return jwt;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationProvider.java
new file mode 100644
index 0000000..9727d38
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationProvider.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.knox;
+
+import com.nimbusds.jose.JOSEException;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUserDetails;
+import org.apache.nifi.authorization.user.StandardNiFiUser.Builder;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.InvalidAuthenticationException;
+import org.apache.nifi.web.security.NiFiAuthenticationProvider;
+import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+
+import java.text.ParseException;
+
+/**
+ *
+ */
+public class KnoxAuthenticationProvider extends NiFiAuthenticationProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(KnoxAuthenticationProvider.class);
+
+    private final KnoxService knoxService;
+
+    public KnoxAuthenticationProvider(KnoxService knoxService, NiFiProperties nifiProperties, Authorizer authorizer) {
+        super(nifiProperties, authorizer);
+        this.knoxService = knoxService;
+    }
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+        final KnoxAuthenticationRequestToken request = (KnoxAuthenticationRequestToken) authentication;
+
+        try {
+            final String jwtPrincipal = knoxService.getAuthenticationFromToken(request.getToken());
+            final String mappedIdentity = mapIdentity(jwtPrincipal);
+            final NiFiUser user = new Builder().identity(mappedIdentity).groups(getUserGroups(mappedIdentity)).clientAddress(request.getClientAddress()).build();
+            return new NiFiAuthenticationToken(new NiFiUserDetails(user));
+        } catch (ParseException | JOSEException e) {
+            logger.info("Unable to validate the access token: " + e.getMessage(), e);
+            throw new InvalidAuthenticationException("Unable to validate the access token.", e);
+        }
+    }
+
+    @Override
+    public boolean supports(Class<?> authentication) {
+        return KnoxAuthenticationRequestToken.class.isAssignableFrom(authentication);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationRequestToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationRequestToken.java
new file mode 100644
index 0000000..e22c844
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxAuthenticationRequestToken.java
@@ -0,0 +1,59 @@
+/*
+ * 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.security.knox;
+
+import org.apache.nifi.web.security.NiFiAuthenticationRequestToken;
+
+/**
+ * This is an authentication request with a given JWT token.
+ */
+public class KnoxAuthenticationRequestToken extends NiFiAuthenticationRequestToken {
+
+    private final String token;
+
+    /**
+     * Creates a representation of the jwt authentication request for a user.
+     *
+     * @param token   The unique token for this user
+     * @param clientAddress the address of the client making the request
+     */
+    public KnoxAuthenticationRequestToken(final String token, final String clientAddress) {
+        super(clientAddress);
+        setAuthenticated(false);
+        this.token = token;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return null;
+    }
+
+    @Override
+    public Object getPrincipal() {
+        return token;
+    }
+
+    public String getToken() {
+        return token;
+    }
+
+    @Override
+    public String toString() {
+        return "<Knox JWT token>";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxConfiguration.java
new file mode 100644
index 0000000..2c162d1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxConfiguration.java
@@ -0,0 +1,33 @@
+/*
+ * 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.security.knox;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Set;
+
+public interface KnoxConfiguration {
+
+    boolean isKnoxEnabled();
+
+    String getKnoxUrl();
+
+    Set<String> getAudiences();
+
+    String getKnoxCookieName();
+
+    RSAPublicKey getKnoxPublicKey();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxService.java
new file mode 100644
index 0000000..c72de12
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxService.java
@@ -0,0 +1,244 @@
+/*
+ * 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.security.knox;
+
+import com.nimbusds.jose.JOSEException;
+import com.nimbusds.jose.JWSObject;
+import com.nimbusds.jose.JWSVerifier;
+import com.nimbusds.jose.crypto.RSASSAVerifier;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.security.InvalidAuthenticationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.text.ParseException;
+import java.util.Date;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * KnoxService is a service for managing the Apache Knox SSO.
+ */
+public class KnoxService {
+
+    private static final Logger logger = LoggerFactory.getLogger(KnoxService.class);
+
+    private KnoxConfiguration configuration;
+    private JWSVerifier verifier;
+    private String knoxUrl;
+    private Set<String> audiences;
+
+    /**
+     * Creates a new KnoxService.
+     *
+     * @param configuration          knox configuration
+     */
+    public KnoxService(final KnoxConfiguration configuration) {
+        this.configuration = configuration;
+
+        // if knox sso support is enabled, validate the configuration
+        if (configuration.isKnoxEnabled()) {
+            // ensure the url is provided
+            knoxUrl = configuration.getKnoxUrl();
+            if (StringUtils.isBlank(knoxUrl)) {
+                throw new RuntimeException("Knox URL is required when Apache Knox SSO support is enabled.");
+            }
+
+            // ensure the cookie name is set
+            if (StringUtils.isBlank(configuration.getKnoxCookieName())) {
+                throw new RuntimeException("Knox Cookie Name is required when Apache Knox SSO support is enabled.");
+            }
+
+            // create the verifier
+            verifier = new RSASSAVerifier(configuration.getKnoxPublicKey());
+
+            // get the audience
+            audiences = configuration.getAudiences();
+        }
+    }
+
+    /**
+     * Returns whether Knox support is enabled.
+     *
+     * @return whether Knox support is enabled
+     */
+    public boolean isKnoxEnabled() {
+        return configuration.isKnoxEnabled();
+    }
+
+    /**
+     * Returns the Knox Url.
+     *
+     * @return knox url
+     */
+    public String getKnoxUrl() {
+        if (!configuration.isKnoxEnabled()) {
+            throw new IllegalStateException("Apache Knox SSO is not enabled.");
+        }
+
+        return knoxUrl;
+    }
+
+    /**
+     * Extracts the authentication from the token and verify it.
+     *
+     * @param jwt signed jwt string
+     * @return the user authentication
+     * @throws ParseException if the payload of the jwt doesn't represent a valid json object and a jwt claims set
+     * @throws JOSEException if the JWS object couldn't be verified
+     */
+    public String getAuthenticationFromToken(final String jwt) throws ParseException, JOSEException {
+        if (!configuration.isKnoxEnabled()) {
+            throw new IllegalStateException("Apache Knox SSO is not enabled.");
+        }
+
+        // attempt to parse the signed jwt
+        final SignedJWT signedJwt = SignedJWT.parse(jwt);
+
+        // validate the token
+        if (validateToken(signedJwt)) {
+            final JWTClaimsSet claimsSet = signedJwt.getJWTClaimsSet();
+            if (claimsSet == null) {
+                logger.info("Claims set is missing from Knox JWT.");
+                throw new InvalidAuthenticationException("The Knox JWT token is not valid.");
+            }
+
+            // extract the user identity from the token
+            return claimsSet.getSubject();
+        } else {
+            throw new InvalidAuthenticationException("The Knox JWT token is not valid.");
+        }
+    }
+
+    /**
+     * Validate the specified jwt.
+     *
+     * @param jwtToken knox jwt
+     * @return whether this jwt is valid
+     * @throws JOSEException if the jws object couldn't be verified
+     * @throws ParseException if the payload of the jwt doesn't represent a valid json object and a jwt claims set
+     */
+    private boolean validateToken(final SignedJWT jwtToken) throws JOSEException, ParseException {
+        final boolean validSignature = validateSignature(jwtToken);
+        final boolean validAudience = validateAudience(jwtToken);
+        final boolean notExpired = validateExpiration(jwtToken);
+
+        return validSignature && validAudience && notExpired;
+    }
+
+    /**
+     * Validate the jwt signature.
+     *
+     * @param jwtToken knox jwt
+     * @return whether this jwt signature is valid
+     * @throws JOSEException if the jws object couldn't be verified
+     */
+    private boolean validateSignature(final SignedJWT jwtToken) throws JOSEException {
+        boolean valid = false;
+
+        // ensure the token is signed
+        if (JWSObject.State.SIGNED.equals(jwtToken.getState())) {
+
+            // ensure the signature is present
+            if (jwtToken.getSignature() != null) {
+
+                // verify the token
+                valid = jwtToken.verify(verifier);
+            }
+        }
+
+        if (!valid) {
+            logger.error("The Knox JWT has an invalid signature.");
+        }
+
+        return valid;
+    }
+
+    /**
+     * Validate the jwt audience.
+     *
+     * @param jwtToken knox jwt
+     * @return whether this jwt audience is valid
+     * @throws ParseException if the payload of the jwt doesn't represent a valid json object and a jwt claims set
+     */
+    private boolean validateAudience(final SignedJWT jwtToken) throws ParseException {
+        if (audiences == null) {
+            return true;
+        }
+
+        final JWTClaimsSet claimsSet = jwtToken.getJWTClaimsSet();
+        if (claimsSet == null) {
+            logger.error("Claims set is missing from Knox JWT.");
+            return false;
+        }
+
+        final List<String> tokenAudiences = claimsSet.getAudience();
+        if (tokenAudiences == null) {
+            logger.error("Audience is missing from the Knox JWT.");
+            return false;
+        }
+
+        boolean valid = false;
+        for (final String tokenAudience : tokenAudiences) {
+            // ensure one of the audiences is matched
+            if (audiences.contains(tokenAudience)) {
+                valid = true;
+                break;
+            }
+        }
+
+        if (!valid) {
+            logger.error(String.format("The Knox JWT does not have the required audience(s). Required one of [%s]. Present in JWT [%s].",
+                    StringUtils.join(audiences, ", "), StringUtils.join(tokenAudiences, ", ")));
+        }
+
+        return valid;
+    }
+
+    /**
+     * Validate the jwt expiration.
+     *
+     * @param jwtToken knox jwt
+     * @return whether this jwt is not expired
+     * @throws ParseException if the payload of the jwt doesn't represent a valid json object and a jwt claims set
+     */
+    private boolean validateExpiration(final SignedJWT jwtToken) throws ParseException {
+        boolean valid = false;
+
+        final JWTClaimsSet claimsSet = jwtToken.getJWTClaimsSet();
+        if (claimsSet == null) {
+            logger.error("Claims set is missing from Knox JWT.");
+            return false;
+        }
+
+        final Date now = new Date();
+        final Date expiration = claimsSet.getExpirationTime();
+
+        // the token is not expired if the expiration isn't present or the expiration is after now
+        if (expiration == null || now.before(expiration)) {
+            valid = true;
+        }
+
+        if (!valid) {
+            logger.error("The Knox JWT is expired.");
+        }
+
+        return valid;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxServiceFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxServiceFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxServiceFactoryBean.java
new file mode 100644
index 0000000..2a83105
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/KnoxServiceFactoryBean.java
@@ -0,0 +1,56 @@
+/*
+ * 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.security.knox;
+
+import org.apache.nifi.util.NiFiProperties;
+import org.springframework.beans.factory.FactoryBean;
+
+public class KnoxServiceFactoryBean implements FactoryBean<KnoxService> {
+
+    private KnoxService knoxService = null;
+    private NiFiProperties properties = null;
+
+    @Override
+    public KnoxService getObject() throws Exception {
+        if (knoxService == null) {
+            // ensure we only allow knox if login and oidc are disabled
+            if (properties.isKnoxSsoEnabled() && (properties.isLoginIdentityProviderEnabled() || properties.isOidcEnabled())) {
+                throw new RuntimeException("Apache Knox SSO support cannot be enabled if the Login Identity Provider or OpenId Connect is configured.");
+            }
+
+            final KnoxConfiguration configuration = new StandardKnoxConfiguration(properties);
+            knoxService = new KnoxService(configuration);
+        }
+
+        return knoxService;
+    }
+
+    @Override
+    public Class<?> getObjectType() {
+        return KnoxService.class;
+    }
+
+    @Override
+    public boolean isSingleton() {
+        return true;
+    }
+
+    public void setProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/StandardKnoxConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/StandardKnoxConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/StandardKnoxConfiguration.java
new file mode 100644
index 0000000..5132843
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/knox/StandardKnoxConfiguration.java
@@ -0,0 +1,73 @@
+/*
+ * 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.security.knox;
+
+import org.apache.nifi.util.NiFiProperties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.security.cert.CertificateException;
+import java.security.cert.CertificateFactory;
+import java.security.cert.X509Certificate;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Set;
+
+public class StandardKnoxConfiguration implements KnoxConfiguration {
+
+    private final NiFiProperties properties;
+
+    public StandardKnoxConfiguration(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+    public boolean isKnoxEnabled() {
+        return properties.isKnoxSsoEnabled();
+    }
+
+    public String getKnoxUrl() {
+        return properties.getKnoxUrl();
+    }
+
+    @Override
+    public Set<String> getAudiences() {
+        return properties.getKnoxAudiences();
+    }
+
+    public String getKnoxCookieName() {
+        return properties.getKnoxCookieName();
+    }
+
+    public RSAPublicKey getKnoxPublicKey() {
+        // get the path to the public key
+        final Path knoxPublicKeyPath = properties.getKnoxPublicKeyPath();
+
+        // ensure the file exists
+        if (Files.isRegularFile(knoxPublicKeyPath) && Files.exists(knoxPublicKeyPath)) {
+            try (final InputStream publicKeyStream = Files.newInputStream(knoxPublicKeyPath)) {
+                final CertificateFactory certificateFactory = CertificateFactory.getInstance("X.509");
+                final X509Certificate certificate = (X509Certificate) certificateFactory.generateCertificate(publicKeyStream);
+                return (RSAPublicKey) certificate.getPublicKey();
+            } catch (final IOException | CertificateException e) {
+                throw new RuntimeException(e.getMessage(), e);
+            }
+        } else {
+            throw new RuntimeException(String.format("The specified Knox public key path does not exist '%s'", knoxPublicKeyPath.toString()));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/oidc/StandardOidcIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/oidc/StandardOidcIdentityProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/oidc/StandardOidcIdentityProvider.java
index 6f4b6b9..62e0c0c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/oidc/StandardOidcIdentityProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/oidc/StandardOidcIdentityProvider.java
@@ -96,8 +96,8 @@ public class StandardOidcIdentityProvider implements OidcIdentityProvider {
 
         // attempt to process the oidc configuration if configured
         if (properties.isOidcEnabled()) {
-            if (properties.isLoginIdentityProviderEnabled()) {
-                throw new RuntimeException("OpenId Connect support cannot be enabled if the Login Identity Provider is configured.");
+            if (properties.isLoginIdentityProviderEnabled() || properties.isKnoxSsoEnabled()) {
+                throw new RuntimeException("OpenId Connect support cannot be enabled if the Login Identity Provider or Apache Knox SSO is configured.");
             }
 
             // oidc connect timeout

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
index 6b0c0c1..0c1009a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
@@ -68,6 +68,18 @@
         <constructor-arg ref="authorizer" index="2"/>
     </bean>
 
+    <!-- knox service -->
+    <bean id="knoxService" class="org.apache.nifi.web.security.knox.KnoxServiceFactoryBean">
+        <property name="properties" ref="nifiProperties"/>
+    </bean>
+
+    <!-- knox authentication provider -->
+    <bean id="knoxAuthenticationProvider" class="org.apache.nifi.web.security.knox.KnoxAuthenticationProvider">
+        <constructor-arg ref="knoxService" index="0"/>
+        <constructor-arg ref="nifiProperties" index="1"/>
+        <constructor-arg ref="authorizer" index="2"/>
+    </bean>
+
     <!-- Kerberos service -->
     <bean id="kerberosService" class="org.apache.nifi.web.security.spring.KerberosServiceFactoryBean">
         <property name="properties" ref="nifiProperties"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilterTest.java
new file mode 100644
index 0000000..38a9efe
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxAuthenticationFilterTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.security.knox;
+
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import javax.servlet.http.Cookie;
+import javax.servlet.http.HttpServletRequest;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class KnoxAuthenticationFilterTest {
+
+    private static final String COOKIE_NAME = "hadoop-jwt";
+
+    private KnoxAuthenticationFilter knoxAuthenticationFilter;
+
+    @Before
+    public void setUp() throws Exception {
+        final NiFiProperties nifiProperties = Mockito.mock(NiFiProperties.class);
+        when(nifiProperties.isKnoxSsoEnabled()).thenReturn(true);
+        when(nifiProperties.getKnoxCookieName()).thenReturn(COOKIE_NAME);
+
+        knoxAuthenticationFilter = new KnoxAuthenticationFilter();
+        knoxAuthenticationFilter.setProperties(nifiProperties);
+    }
+
+    @Test
+    public void testInsecureHttp() throws Exception {
+        final HttpServletRequest request = mock(HttpServletRequest.class);
+        when(request.isSecure()).thenReturn(false);
+        assertNull(knoxAuthenticationFilter.attemptAuthentication(request));
+    }
+
+    @Test
+    public void testNullCookies() throws Exception {
+        final HttpServletRequest request = mock(HttpServletRequest.class);
+        when(request.isSecure()).thenReturn(true);
+        when(request.getCookies()).thenReturn(null);
+        assertNull(knoxAuthenticationFilter.attemptAuthentication(request));
+    }
+
+    @Test
+    public void testNoCookies() throws Exception {
+        final HttpServletRequest request = mock(HttpServletRequest.class);
+        when(request.isSecure()).thenReturn(true);
+        when(request.getCookies()).thenReturn(new Cookie[] {});
+        assertNull(knoxAuthenticationFilter.attemptAuthentication(request));
+    }
+
+    @Test
+    public void testWrongCookieName() throws Exception {
+        final String jwt = "my-jwt";
+
+        final Cookie knoxCookie = mock(Cookie.class);
+        when(knoxCookie.getName()).thenReturn("not-hadoop-jwt");
+        when(knoxCookie.getValue()).thenReturn(jwt);
+
+        final HttpServletRequest request = mock(HttpServletRequest.class);
+        when(request.isSecure()).thenReturn(true);
+        when(request.getCookies()).thenReturn(new Cookie[] {knoxCookie});
+
+        final KnoxAuthenticationRequestToken authRequest = (KnoxAuthenticationRequestToken) knoxAuthenticationFilter.attemptAuthentication(request);
+        assertNull(authRequest);
+    }
+
+    @Test
+    public void testKnoxCookie() throws Exception {
+        final String jwt = "my-jwt";
+
+        final Cookie knoxCookie = mock(Cookie.class);
+        when(knoxCookie.getName()).thenReturn(COOKIE_NAME);
+        when(knoxCookie.getValue()).thenReturn(jwt);
+
+        final HttpServletRequest request = mock(HttpServletRequest.class);
+        when(request.isSecure()).thenReturn(true);
+        when(request.getCookies()).thenReturn(new Cookie[] {knoxCookie});
+
+        final KnoxAuthenticationRequestToken authRequest = (KnoxAuthenticationRequestToken) knoxAuthenticationFilter.attemptAuthentication(request);
+        assertNotNull(authRequest);
+        assertEquals(jwt, authRequest.getToken());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java
new file mode 100644
index 0000000..709a303
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java
@@ -0,0 +1,217 @@
+/*
+ * 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.security.knox;
+
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.PlainJWT;
+import com.nimbusds.oauth2.sdk.auth.JWTAuthenticationClaimsSet;
+import com.nimbusds.oauth2.sdk.auth.PrivateKeyJWT;
+import com.nimbusds.oauth2.sdk.id.Audience;
+import com.nimbusds.oauth2.sdk.id.ClientID;
+import com.nimbusds.oauth2.sdk.id.JWTID;
+import org.apache.nifi.web.security.InvalidAuthenticationException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.interfaces.RSAPrivateKey;
+import java.security.interfaces.RSAPublicKey;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class KnoxServiceTest {
+
+    private static final String AUDIENCE = "https://apache-knox/token";
+    private static final String AUDIENCE_2 = "https://apache-knox-2/token";
+
+    @Test(expected = IllegalStateException.class)
+    public void testKnoxSsoNotEnabledGetKnoxUrl() throws Exception {
+        final KnoxConfiguration configuration = mock(KnoxConfiguration.class);
+        when(configuration.isKnoxEnabled()).thenReturn(false);
+
+        final KnoxService service = new KnoxService(configuration);
+        assertFalse(service.isKnoxEnabled());
+
+        service.getKnoxUrl();
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testKnoxSsoNotEnabledGetAuthenticatedFromToken() throws Exception {
+        final KnoxConfiguration configuration = mock(KnoxConfiguration.class);
+        when(configuration.isKnoxEnabled()).thenReturn(false);
+
+        final KnoxService service = new KnoxService(configuration);
+        assertFalse(service.isKnoxEnabled());
+
+        service.getAuthenticationFromToken("jwt-token-value");
+    }
+
+    private JWTAuthenticationClaimsSet getAuthenticationClaimsSet(final String subject, final String audience, final Date expiration) {
+        return new JWTAuthenticationClaimsSet(
+                new ClientID(subject),
+                new Audience(audience).toSingleAudienceList(),
+                expiration,
+                null,
+                null,
+                new JWTID());
+    }
+
+    @Test
+    public void testSignedJwt() throws Exception {
+        final String subject = "user-1";
+        final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS));
+
+        final KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+        final KeyPair pair = keyGen.generateKeyPair();
+        final RSAPrivateKey privateKey = (RSAPrivateKey) pair.getPrivate();
+        final RSAPublicKey publicKey = (RSAPublicKey) pair.getPublic();
+
+        final JWTAuthenticationClaimsSet claimsSet = getAuthenticationClaimsSet(subject, AUDIENCE, expiration);
+        final PrivateKeyJWT privateKeyJWT = new PrivateKeyJWT(claimsSet, JWSAlgorithm.RS256, privateKey, null, null);
+
+        final KnoxConfiguration configuration = getConfiguration(publicKey);
+        final KnoxService service = new KnoxService(configuration);
+
+        Assert.assertEquals(subject, service.getAuthenticationFromToken(privateKeyJWT.getClientAssertion().serialize()));
+    }
+
+    @Test(expected = InvalidAuthenticationException.class)
+    public void testBadSignedJwt() throws Exception {
+        final String subject = "user-1";
+        final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS));
+
+        final KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+
+        final KeyPair pair1 = keyGen.generateKeyPair();
+        final RSAPrivateKey privateKey1 = (RSAPrivateKey) pair1.getPrivate();
+
+        final KeyPair pair2 = keyGen.generateKeyPair();
+        final RSAPublicKey publicKey2 = (RSAPublicKey) pair2.getPublic();
+
+        // sign the jwt with pair 1
+        final JWTAuthenticationClaimsSet claimsSet = getAuthenticationClaimsSet(subject, AUDIENCE, expiration);
+        final PrivateKeyJWT privateKeyJWT = new PrivateKeyJWT(claimsSet, JWSAlgorithm.RS256, privateKey1, null, null);
+
+        // attempt to verify it with pair 2
+        final KnoxConfiguration configuration = getConfiguration(publicKey2);
+        final KnoxService service = new KnoxService(configuration);
+
+        service.getAuthenticationFromToken(privateKeyJWT.getClientAssertion().serialize());
+    }
+
+    @Test(expected = ParseException.class)
+    public void testPlainJwt() throws Exception {
+        final KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+        final KeyPair pair = keyGen.generateKeyPair();
+        final RSAPublicKey publicKey = (RSAPublicKey) pair.getPublic();
+
+        final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS));
+        final JWTClaimsSet claimsSet = new JWTClaimsSet.Builder()
+                .subject("user-1")
+                .expirationTime(expiration)
+                .build();
+
+        final PlainJWT plainJWT = new PlainJWT(claimsSet);
+
+        final KnoxConfiguration configuration = getConfiguration(publicKey);
+        final KnoxService service = new KnoxService(configuration);
+
+        service.getAuthenticationFromToken(plainJWT.serialize());
+    }
+
+    @Test(expected = InvalidAuthenticationException.class)
+    public void testExpiredJwt() throws Exception {
+        final String subject = "user-1";
+
+        // token expires in 1 sec
+        final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(1, TimeUnit.SECONDS));
+
+        final KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+        final KeyPair pair = keyGen.generateKeyPair();
+        final RSAPrivateKey privateKey = (RSAPrivateKey) pair.getPrivate();
+        final RSAPublicKey publicKey = (RSAPublicKey) pair.getPublic();
+
+        // wait 2 sec
+        Thread.sleep(TimeUnit.MILLISECONDS.convert(2, TimeUnit.SECONDS));
+
+        final JWTAuthenticationClaimsSet claimsSet = getAuthenticationClaimsSet(subject, AUDIENCE, expiration);
+        final PrivateKeyJWT privateKeyJWT = new PrivateKeyJWT(claimsSet, JWSAlgorithm.RS256, privateKey, null, null);
+
+        final KnoxConfiguration configuration = getConfiguration(publicKey);
+        final KnoxService service = new KnoxService(configuration);
+
+        service.getAuthenticationFromToken(privateKeyJWT.getClientAssertion().serialize());
+    }
+
+    @Test
+    public void testRequiredAudience() throws Exception {
+        final String subject = "user-1";
+        final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS));
+
+        final KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+        final KeyPair pair = keyGen.generateKeyPair();
+        final RSAPrivateKey privateKey = (RSAPrivateKey) pair.getPrivate();
+        final RSAPublicKey publicKey = (RSAPublicKey) pair.getPublic();
+
+        final JWTAuthenticationClaimsSet claimsSet = getAuthenticationClaimsSet(subject, AUDIENCE, expiration);
+        final PrivateKeyJWT privateKeyJWT = new PrivateKeyJWT(claimsSet, JWSAlgorithm.RS256, privateKey, null, null);
+
+        final KnoxConfiguration configuration = getConfiguration(publicKey);
+        when(configuration.getAudiences()).thenReturn(null);
+        final KnoxService service = new KnoxService(configuration);
+
+        Assert.assertEquals(subject, service.getAuthenticationFromToken(privateKeyJWT.getClientAssertion().serialize()));
+    }
+
+    @Test(expected = InvalidAuthenticationException.class)
+    public void testInvalidAudience() throws Exception {
+        final String subject = "user-1";
+        final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS));
+
+        final KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+        final KeyPair pair = keyGen.generateKeyPair();
+        final RSAPrivateKey privateKey = (RSAPrivateKey) pair.getPrivate();
+        final RSAPublicKey publicKey = (RSAPublicKey) pair.getPublic();
+
+        final JWTAuthenticationClaimsSet claimsSet = getAuthenticationClaimsSet(subject, "incorrect-audience", expiration);
+        final PrivateKeyJWT privateKeyJWT = new PrivateKeyJWT(claimsSet, JWSAlgorithm.RS256, privateKey, null, null);
+
+        final KnoxConfiguration configuration = getConfiguration(publicKey);
+        final KnoxService service = new KnoxService(configuration);
+
+        Assert.assertEquals(subject, service.getAuthenticationFromToken(privateKeyJWT.getClientAssertion().serialize()));
+    }
+
+    private KnoxConfiguration getConfiguration(final RSAPublicKey publicKey) throws Exception {
+        final KnoxConfiguration configuration = mock(KnoxConfiguration.class);
+        when(configuration.isKnoxEnabled()).thenReturn(true);
+        when(configuration.getKnoxUrl()).thenReturn("knox-sso-url");
+        when(configuration.getKnoxCookieName()).thenReturn("knox-cookie-name");
+        when(configuration.getAudiences()).thenReturn(Stream.of(AUDIENCE, AUDIENCE_2).collect(Collectors.toSet()));
+        when(configuration.getKnoxPublicKey()).thenReturn(publicKey);
+        return configuration;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c798d18/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/filter/LoginFilter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/filter/LoginFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/filter/LoginFilter.java
index 4f70ed8..c8025bd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/filter/LoginFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/filter/LoginFilter.java
@@ -40,10 +40,14 @@ public class LoginFilter implements Filter {
     @Override
     public void doFilter(ServletRequest request, ServletResponse response, FilterChain filterChain) throws IOException, ServletException {
         final boolean supportsOidc = Boolean.parseBoolean(servletContext.getInitParameter("oidc-supported"));
+        final boolean supportsKnoxSso = Boolean.parseBoolean(servletContext.getInitParameter("knox-supported"));
 
         if (supportsOidc) {
             final ServletContext apiContext = servletContext.getContext("/nifi-api");
             apiContext.getRequestDispatcher("/access/oidc/request").forward(request, response);
+        } else if (supportsKnoxSso) {
+            final ServletContext apiContext = servletContext.getContext("/nifi-api");
+            apiContext.getRequestDispatcher("/access/knox/request").forward(request, response);
         } else {
             filterChain.doFilter(request, response);
         }