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 2015/10/13 04:19:47 UTC

[2/4] nifi git commit: NIFI-655: - Refactoring web security to use Spring Security Java Configuration. - Introducing security in Web UI in order to get JWT.

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtAuthenticationFilter.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/jwt/JwtAuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtAuthenticationFilter.java
new file mode 100644
index 0000000..5b85bd2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtAuthenticationFilter.java
@@ -0,0 +1,58 @@
+/*
+ * 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.jwt;
+
+import java.util.List;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.nifi.web.security.NiFiAuthenticationFilter;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
+import org.apache.nifi.web.security.token.NewAccountAuthenticationRequestToken;
+import org.apache.nifi.web.security.token.NiFiAuthenticationRequestToken;
+import org.apache.nifi.web.security.user.NewAccountRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.core.Authentication;
+
+/**
+ */
+public class JwtAuthenticationFilter extends NiFiAuthenticationFilter {
+
+    private static final Logger logger = LoggerFactory.getLogger(JwtAuthenticationFilter.class);
+    
+    private JwtService jwtService;
+
+    @Override
+    public Authentication attemptAuthentication(HttpServletRequest request, HttpServletResponse response) {
+        final String principal = jwtService.getAuthentication(request);
+        if (principal == null) {
+            return null;
+        }
+        
+        final List<String> proxyChain = ProxiedEntitiesUtils.buildProxyChain(request, principal);
+        if (isNewAccountRequest(request)) {
+            return new NewAccountAuthenticationRequestToken(new NewAccountRequest(proxyChain, getJustification(request)));
+        } else {
+            return new NiFiAuthenticationRequestToken(proxyChain);
+        }
+    }
+
+    public void setJwtService(JwtService jwtService) {
+        this.jwtService = jwtService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtAuthenticationProvider.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/jwt/JwtAuthenticationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtAuthenticationProvider.java
new file mode 100644
index 0000000..77e9982
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtAuthenticationProvider.java
@@ -0,0 +1,47 @@
+/*
+ * 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.jwt;
+
+import org.apache.nifi.web.security.token.NewAccountAuthenticationRequestToken;
+import org.apache.nifi.web.security.token.NewAccountAuthenticationToken;
+import org.apache.nifi.web.security.token.NiFiAuthenticationRequestToken;
+import org.springframework.security.authentication.AuthenticationProvider;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+
+/**
+ *
+ */
+public class JwtAuthenticationProvider implements AuthenticationProvider {
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+        if (NewAccountAuthenticationRequestToken.class.isAssignableFrom(authentication.getClass())) {
+            return new NewAccountAuthenticationToken(((NewAccountAuthenticationRequestToken) authentication).getNewAccountRequest());
+        } else if (NiFiAuthenticationRequestToken.class.isAssignableFrom(authentication.getClass())) {
+            return authentication;
+        } else {
+            return null;
+        }
+    }
+    
+    @Override
+    public boolean supports(Class<?> authentication) {
+        return NiFiAuthenticationRequestToken.class.isAssignableFrom(authentication);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.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/jwt/JwtService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
new file mode 100644
index 0000000..803cd79
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
@@ -0,0 +1,51 @@
+/*
+ * 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.jwt;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.springframework.security.core.Authentication;
+
+/**
+ *
+ */
+public class JwtService {
+    
+    /**
+     * Gets the Authentication by extracting a JWT token from the specified request.
+     * 
+     * @param request   Request to extract the token from
+     * @return          The user identifier from the token
+     */
+    public String getAuthentication(final HttpServletRequest request) {
+        // extract/verify token from incoming request
+        // load user details with identifier from token
+        // create authentication using user details
+        return null;
+    }
+    
+    /**
+     * Adds a token for the specified authentication in the specified response.
+     * 
+     * @param response          The response to add the token to
+     * @param authentication    The authentication to generate a token for
+     */
+    public void addToken(final HttpServletResponse response, final Authentication authentication) {
+        // create a token the specified authentication
+        // add the token as a response header
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/node/NodeAuthorizedUserFilter.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/node/NodeAuthorizedUserFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/node/NodeAuthorizedUserFilter.java
new file mode 100644
index 0000000..e73fd9e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/node/NodeAuthorizedUserFilter.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.node;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.cert.X509Certificate;
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.nifi.controller.FlowController;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.security.user.NiFiUserDetails;
+import org.apache.nifi.web.security.x509.SubjectDnX509PrincipalExtractor;
+import org.apache.nifi.web.security.x509.X509CertificateExtractor;
+import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.util.WebUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.context.ApplicationContext;
+import org.springframework.security.authentication.AuthenticationDetailsSource;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.web.authentication.WebAuthenticationDetailsSource;
+import org.springframework.security.web.authentication.preauth.PreAuthenticatedAuthenticationToken;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+import org.springframework.web.context.support.WebApplicationContextUtils;
+import org.springframework.web.filter.GenericFilterBean;
+
+/**
+ * Custom filter to extract a user's authorities from the request where the user
+ * was authenticated by the cluster manager and populate the threadlocal with
+ * the authorized user. If the request contains the appropriate header with
+ * authorities and the application instance is a node connected to the cluster,
+ * then the authentication/authorization steps remaining in the filter chain are
+ * skipped.
+ *
+ * Checking if the application instance is a connected node is important because
+ * it prevents external clients from faking the request headers and bypassing
+ * the authentication processing chain.
+ */
+public class NodeAuthorizedUserFilter extends GenericFilterBean {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(NodeAuthorizedUserFilter.class);
+
+    public static final String PROXY_USER_DETAILS = "X-ProxiedEntityUserDetails";
+
+    private final NiFiProperties properties;
+    private final AuthenticationDetailsSource authenticationDetailsSource = new WebAuthenticationDetailsSource();
+    private final X509CertificateExtractor certificateExtractor = new X509CertificateExtractor();
+    private final X509PrincipalExtractor principalExtractor = new SubjectDnX509PrincipalExtractor();
+
+    public NodeAuthorizedUserFilter(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+        final HttpServletRequest httpServletRequest = (HttpServletRequest) request;
+
+        // get the proxied user's authorities
+        final String hexEncodedUserDetails = httpServletRequest.getHeader(PROXY_USER_DETAILS);
+
+        // check if the request has the necessary header information and this instance is configured as a node
+        if (StringUtils.isNotBlank(hexEncodedUserDetails) && properties.isNode()) {
+
+            // get the flow controller from the Spring context
+            final ApplicationContext ctx = WebApplicationContextUtils.getWebApplicationContext(getServletContext());
+            final FlowController flowController = ctx.getBean("flowController", FlowController.class);
+
+            // check that we are connected to the cluster
+            if (flowController.getNodeId() != null) {
+                try {
+                    // get the DN from the cert in the request
+                    final X509Certificate certificate = certificateExtractor.extractClientCertificate((HttpServletRequest) request);
+                    if (certificate != null) {
+                        // extract the principal from the certificate
+                        final Object certificatePrincipal = principalExtractor.extractPrincipal(certificate);
+                        final String dn = certificatePrincipal.toString();
+
+                        // only consider the pre-authorized user when the request came from the NCM according to the DN in the certificate
+                        final String clusterManagerDN = flowController.getClusterManagerDN();
+                        if (clusterManagerDN != null && clusterManagerDN.equals(dn)) {
+                            // deserialize hex encoded object
+                            final Serializable userDetailsObj = WebUtils.deserializeHexToObject(hexEncodedUserDetails);
+
+                            // if we have a valid object, set the authentication token and bypass the remaining authentication processing chain
+                            if (userDetailsObj instanceof NiFiUserDetails) {
+                                final NiFiUserDetails userDetails = (NiFiUserDetails) userDetailsObj;
+                                final NiFiUser user = userDetails.getNiFiUser();
+
+                                // log the request attempt - response details will be logged later
+                                logger.info(String.format("Attempting request for (%s) %s %s (source ip: %s)", user.getDn(), httpServletRequest.getMethod(),
+                                        httpServletRequest.getRequestURL().toString(), request.getRemoteAddr()));
+
+                                // we do not create the authentication token with the X509 certificate because the certificate is from the sending system, not the proxied user
+                                final PreAuthenticatedAuthenticationToken token = new PreAuthenticatedAuthenticationToken(userDetails, null, userDetails.getAuthorities());
+                                token.setDetails(authenticationDetailsSource.buildDetails(request));
+                                SecurityContextHolder.getContext().setAuthentication(token);
+                            }
+                        }
+                    }
+                } catch (final ClassNotFoundException cnfe) {
+                    LOGGER.warn("Classpath issue detected because failed to deserialize authorized user in request header due to: " + cnfe, cnfe);
+                }
+            }
+        }
+
+        chain.doFilter(request, response);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NewAccountAuthenticationRequestToken.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/token/NewAccountAuthenticationRequestToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NewAccountAuthenticationRequestToken.java
new file mode 100644
index 0000000..52e5172
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NewAccountAuthenticationRequestToken.java
@@ -0,0 +1,41 @@
+/*
+ * 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.token;
+
+import org.apache.nifi.web.security.user.NewAccountRequest;
+
+/**
+ * This is an Authentication Token for a user that is request authentication in
+ * order to submit a new account request.
+ */
+public class NewAccountAuthenticationRequestToken extends NiFiAuthenticationRequestToken {
+
+    final NewAccountRequest newAccountRequest;
+    
+    public NewAccountAuthenticationRequestToken(final NewAccountRequest newAccountRequest) {
+        super(newAccountRequest.getChain());
+        this.newAccountRequest = newAccountRequest;
+    }
+
+    public String getJustification() {
+        return newAccountRequest.getJustification();
+    }
+    
+    public NewAccountRequest getNewAccountRequest() {
+        return newAccountRequest;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NewAccountAuthenticationToken.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/token/NewAccountAuthenticationToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NewAccountAuthenticationToken.java
new file mode 100644
index 0000000..6fe34df
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NewAccountAuthenticationToken.java
@@ -0,0 +1,48 @@
+/*
+ * 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.token;
+
+import org.apache.nifi.web.security.user.NewAccountRequest;
+import org.springframework.security.authentication.AbstractAuthenticationToken;
+
+/**
+ * This is an Authentication Token for a user that has been authenticated but is
+ * not authorized to access the NiFi APIs. Typically, this authentication token is
+ * used successfully when requesting a NiFi account. Requesting any other endpoint
+ * would be rejected due to lack of roles.
+ */
+public class NewAccountAuthenticationToken extends AbstractAuthenticationToken {
+
+    final NewAccountRequest newAccountRequest;
+    
+    public NewAccountAuthenticationToken(final NewAccountRequest newAccountRequest) {
+        super(null);
+        super.setAuthenticated(true);
+        this.newAccountRequest = newAccountRequest;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return null;
+    }
+
+    @Override
+    public Object getPrincipal() {
+        return newAccountRequest;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NiFiAuthenticationRequestToken.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/token/NiFiAuthenticationRequestToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NiFiAuthenticationRequestToken.java
new file mode 100644
index 0000000..0e00e5b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NiFiAuthenticationRequestToken.java
@@ -0,0 +1,55 @@
+/*
+ * 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.token;
+
+import java.util.Collections;
+import java.util.List;
+import org.springframework.security.authentication.AbstractAuthenticationToken;
+
+/**
+ * An authentication token that is used as an authentication request. The request
+ * chain is specified during creation and is used authenticate the user(s). If the
+ * user is authenticated, the token is used to authorized the user(s).
+ */
+public class NiFiAuthenticationRequestToken extends AbstractAuthenticationToken {
+
+    private final List<String> chain;
+    
+    public NiFiAuthenticationRequestToken(final List<String> chain) {
+        super(null);
+        this.chain = chain;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return null;
+    }
+
+    @Override
+    public Object getPrincipal() {
+        return chain;
+    }
+    
+    public List<String> getChain() {
+        return Collections.unmodifiableList(chain);
+    }
+
+    @Override
+    public final void setAuthenticated(boolean authenticated) {
+        throw new IllegalArgumentException("Cannot change the authenticated state.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NiFiAuthorizationToken.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/token/NiFiAuthorizationToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NiFiAuthorizationToken.java
new file mode 100644
index 0000000..0945d08
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/NiFiAuthorizationToken.java
@@ -0,0 +1,51 @@
+/*
+ * 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.token;
+
+import org.springframework.security.authentication.AbstractAuthenticationToken;
+import org.springframework.security.core.userdetails.UserDetails;
+
+/**
+ * An authentication token that represents an Authenticated and Authorized user
+ * of the NiFi Apis. The authorities are based off the specified UserDetails.
+ */
+public class NiFiAuthorizationToken extends AbstractAuthenticationToken {
+
+    final UserDetails nifiUserDetails;
+    
+    public NiFiAuthorizationToken(final UserDetails nifiUserDetails) {
+        super(nifiUserDetails.getAuthorities());
+        super.setAuthenticated(true);
+        setDetails(nifiUserDetails);
+        this.nifiUserDetails = nifiUserDetails;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return nifiUserDetails.getPassword();
+    }
+
+    @Override
+    public Object getPrincipal() {
+        return nifiUserDetails;
+    }
+
+    @Override
+    public final void setAuthenticated(boolean authenticated) {
+        throw new IllegalArgumentException("Cannot change the authenticated state.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NewAccountRequest.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/user/NewAccountRequest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NewAccountRequest.java
new file mode 100644
index 0000000..e70c5ab
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NewAccountRequest.java
@@ -0,0 +1,31 @@
+package org.apache.nifi.web.security.user;
+
+import java.util.List;
+
+/**
+ *
+ */
+public class NewAccountRequest {
+
+    private final List<String> chain;
+    private final String justification;
+
+    public NewAccountRequest(final List<String> chain, final String justification) {
+        this.chain = chain;
+        this.justification = justification;
+    }
+
+    public List<String> getChain() {
+        return chain;
+    }
+
+    public String getJustification() {
+        return justification;
+    }
+    
+    public String getUsername() {
+        // the end user is the first item in the chain
+        return chain.get(0);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserDetails.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/user/NiFiUserDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserDetails.java
index c69b1e6..0fc2f53 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserDetails.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserDetails.java
@@ -41,7 +41,7 @@ public class NiFiUserDetails implements UserDetails {
     public NiFiUserDetails(NiFiUser user) {
         this.user = user;
     }
-
+    
     /**
      * Get the user for this UserDetails.
      *
@@ -76,7 +76,6 @@ public class NiFiUserDetails implements UserDetails {
         return user.getDn();
     }
 
-    // TODO: not sure how to handle these yet
     @Override
     public boolean isAccountNonExpired() {
         return true;

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.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/user/NiFiUserUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
index bf1fe43..e435ade 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
@@ -78,6 +78,27 @@ public final class NiFiUserUtils {
 
         return user;
     }
+    
+    /**
+     * Returns the NewAccountRequest or null if this is not a new account request.
+     * 
+     * @return new account request
+     */
+    public static NewAccountRequest getNewAccountRequest() {
+        NewAccountRequest newAccountRequest = null;
+        
+        // obtain the principal in the current authentication
+        final SecurityContext context = SecurityContextHolder.getContext();
+        final Authentication authentication = context.getAuthentication();
+        if (authentication != null) {
+            Object principal = authentication.getPrincipal();
+            if (principal instanceof NewAccountRequest) {
+                newAccountRequest = (NewAccountRequest) principal;
+            }
+        }
+        
+        return newAccountRequest;
+    }
 
     public static String getNiFiUserName() {
         // get the nifi user to extract the username

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.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/x509/X509AuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
index 72baecb..609edd8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
@@ -17,114 +17,36 @@
 package org.apache.nifi.web.security.x509;
 
 import org.apache.nifi.web.security.x509.ocsp.OcspCertificateValidator;
-import java.io.IOException;
-import java.io.PrintWriter;
 import java.security.cert.CertificateExpiredException;
 import java.security.cert.CertificateNotYetValidException;
 import java.security.cert.X509Certificate;
-import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
+import java.util.List;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.admin.service.UserService;
-import org.apache.nifi.web.security.DnUtils;
-import org.apache.nifi.web.security.UntrustedProxyException;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.commons.lang3.StringUtils;
-import org.springframework.security.authentication.AccountStatusException;
-import org.springframework.security.authentication.AuthenticationServiceException;
+import org.apache.nifi.web.security.NiFiAuthenticationFilter;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
+import org.apache.nifi.web.security.token.NewAccountAuthenticationRequestToken;
+import org.apache.nifi.web.security.token.NiFiAuthenticationRequestToken;
+import org.apache.nifi.web.security.user.NewAccountRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.springframework.security.core.Authentication;
-import org.springframework.security.core.AuthenticationException;
-import org.springframework.security.core.userdetails.UsernameNotFoundException;
-import org.springframework.security.web.authentication.preauth.AbstractPreAuthenticatedProcessingFilter;
 import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
 
 /**
  * Custom X509 filter that will inspect the HTTP headers for a proxied user
  * before extracting the user details from the client certificate.
  */
-public class X509AuthenticationFilter extends AbstractPreAuthenticatedProcessingFilter {
+public class X509AuthenticationFilter extends NiFiAuthenticationFilter {
 
-    public static final String PROXY_ENTITIES_CHAIN = "X-ProxiedEntitiesChain";
-    public static final String PROXY_ENTITIES_ACCEPTED = "X-ProxiedEntitiesAccepted";
-    public static final String PROXY_ENTITIES_DETAILS = "X-ProxiedEntitiesDetails";
-
-    private final X509CertificateExtractor certificateExtractor = new X509CertificateExtractor();
-    private final X509PrincipalExtractor principalExtractor = new SubjectDnX509PrincipalExtractor();
+    private static final Logger logger = LoggerFactory.getLogger(X509AuthenticationFilter.class);
+    
+    private X509PrincipalExtractor principalExtractor;
+    private X509CertificateExtractor certificateExtractor;
     private OcspCertificateValidator certificateValidator;
-    private NiFiProperties properties;
-    private UserService userService;
-
-    @Override
-    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
-        final HttpServletResponse httpResponse = (HttpServletResponse) response;
-
-        // determine if this request is attempting to create a new account
-        if (isNewAccountRequest((HttpServletRequest) request)) {
-            // determine if this nifi supports new account requests
-            if (properties.getSupportNewAccountRequests()) {
-                // ensure there is a certificate in the request
-                X509Certificate certificate = certificateExtractor.extractClientCertificate((HttpServletRequest) request);
-                if (certificate != null) {
-                    // extract the principal from the certificate
-                    Object certificatePrincipal = principalExtractor.extractPrincipal(certificate);
-                    String principal = certificatePrincipal.toString();
-
-                    // log the new user account request
-                    logger.info("Requesting new user account for " + principal);
-
-                    try {
-                        // get the justification
-                        String justification = request.getParameter("justification");
-                        if (justification == null) {
-                            justification = StringUtils.EMPTY;
-                        }
-
-                        // create the pending user account
-                        userService.createPendingUserAccount(principal, justification);
-
-                        // generate a response
-                        httpResponse.setStatus(HttpServletResponse.SC_CREATED);
-                        httpResponse.setContentType("text/plain");
-
-                        // write the response message
-                        PrintWriter out = response.getWriter();
-                        out.println("Not authorized. User account created. Authorization pending.");
-                    } catch (IllegalArgumentException iae) {
-                        handleUserServiceError((HttpServletRequest) request, httpResponse, HttpServletResponse.SC_BAD_REQUEST, iae.getMessage());
-                    } catch (AdministrationException ae) {
-                        handleUserServiceError((HttpServletRequest) request, httpResponse, HttpServletResponse.SC_INTERNAL_SERVER_ERROR, ae.getMessage());
-                    }
-                } else {
-                    // can this really happen?
-                    handleMissingCertificate((HttpServletRequest) request, httpResponse);
-                }
-            } else {
-                handleUserServiceError((HttpServletRequest) request, httpResponse, HttpServletResponse.SC_NOT_FOUND, "This NiFi does not support new account requests.");
-            }
-        } else {
-            try {
-                // this not a request to create a user account - try to authorize
-                super.doFilter(request, response, chain);
-            } catch (AuthenticationException ae) {
-                // continue the filter chain since anonymous access should be supported
-                if (!properties.getNeedClientAuth()) {
-                    chain.doFilter(request, response);
-                } else {
-                    // create an appropriate response for the given exception
-                    handleUnsuccessfulAuthentication((HttpServletRequest) request, httpResponse, ae);
-                }
-            }
-        }
-    }
 
     @Override
-    protected Object getPreAuthenticatedPrincipal(HttpServletRequest request) {
-        String principal;
-
+    public Authentication attemptAuthentication(HttpServletRequest request, HttpServletResponse response) {
         // extract the cert
         X509Certificate certificate = certificateExtractor.extractClientCertificate(request);
 
@@ -135,7 +57,7 @@ public class X509AuthenticationFilter extends AbstractPreAuthenticatedProcessing
 
         // extract the principal
         Object certificatePrincipal = principalExtractor.extractPrincipal(certificate);
-        principal = DnUtils.formatProxyDn(certificatePrincipal.toString());
+        final String principal = ProxiedEntitiesUtils.formatProxyDn(certificatePrincipal.toString());
 
         try {
             // ensure the cert is valid
@@ -166,152 +88,27 @@ public class X509AuthenticationFilter extends AbstractPreAuthenticatedProcessing
             }
             return null;
         }
-
-        // look for a proxied user
-        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
-            principal = request.getHeader(PROXY_ENTITIES_CHAIN) + principal;
-        }
-
-        // log the request attempt - response details will be logged later
-        logger.info(String.format("Attempting request for (%s) %s %s (source ip: %s)", principal, request.getMethod(),
-                request.getRequestURL().toString(), request.getRemoteAddr()));
-
-        return principal;
-    }
-
-    @Override
-    protected Object getPreAuthenticatedCredentials(HttpServletRequest request) {
-        return certificateExtractor.extractClientCertificate(request);
-    }
-
-    @Override
-    protected void successfulAuthentication(HttpServletRequest request, HttpServletResponse response, Authentication authResult) {
-        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
-            response.setHeader(PROXY_ENTITIES_ACCEPTED, Boolean.TRUE.toString());
-        }
-        super.successfulAuthentication(request, response, authResult);
-    }
-
-    @Override
-    protected void unsuccessfulAuthentication(HttpServletRequest request, HttpServletResponse response, AuthenticationException failed) {
-        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
-            response.setHeader(PROXY_ENTITIES_DETAILS, failed.getMessage());
-        }
-        super.unsuccessfulAuthentication(request, response, failed);
-    }
-
-    /**
-     * Determines if the specified request is attempting to register a new user
-     * account.
-     *
-     * @param request http request
-     * @return true if new user
-     */
-    private boolean isNewAccountRequest(HttpServletRequest request) {
-        if ("POST".equalsIgnoreCase(request.getMethod())) {
-            String path = request.getPathInfo();
-            if (StringUtils.isNotBlank(path)) {
-                if ("/controller/users".equals(path)) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
-    /**
-     * Handles requests that were unable to be authorized.
-     *
-     * @param request request
-     * @param response response
-     * @param ae ex
-     * @throws IOException ex
-     */
-    private void handleUnsuccessfulAuthentication(HttpServletRequest request, HttpServletResponse response, AuthenticationException ae) throws IOException {
-        // set the response status
-        response.setContentType("text/plain");
-
-        // write the response message
-        PrintWriter out = response.getWriter();
-
-        // use the type of authentication exception to determine the response code
-        if (ae instanceof UsernameNotFoundException) {
-            if (properties.getSupportNewAccountRequests()) {
-                response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
-                out.println("Not authorized.");
-            } else {
-                response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-                out.println("Access is denied.");
-            }
-        } else if (ae instanceof AccountStatusException) {
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            out.println(ae.getMessage());
-        } else if (ae instanceof UntrustedProxyException) {
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            out.println(ae.getMessage());
-        } else if (ae instanceof AuthenticationServiceException) {
-            logger.error(String.format("Unable to authorize: %s", ae.getMessage()), ae);
-            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-            out.println(String.format("Unable to authorize: %s", ae.getMessage()));
+        
+        final List<String> proxyChain = ProxiedEntitiesUtils.buildProxyChain(request, principal);
+        if (isNewAccountRequest(request)) {
+            return new NewAccountAuthenticationRequestToken(new NewAccountRequest(proxyChain, getJustification(request)));
         } else {
-            logger.error(String.format("Unable to authorize: %s", ae.getMessage()), ae);
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            out.println("Access is denied.");
-        }
-
-        // log the failure
-        logger.info(String.format("Rejecting access to web api: %s", ae.getMessage()));
-
-        // optionally log the stack trace
-        if (logger.isDebugEnabled()) {
-            logger.debug(StringUtils.EMPTY, ae);
+            return new NiFiAuthenticationRequestToken(proxyChain);
         }
     }
 
-    private void handleUserServiceError(HttpServletRequest request, HttpServletResponse response, int responseCode, String message) throws IOException {
-        // set the response status
-        response.setContentType("text/plain");
-        response.setStatus(responseCode);
-
-        // write the response message
-        PrintWriter out = response.getWriter();
-        out.println(message);
-
-        // log the failure
-        logger.info(String.format("Unable to process request because %s", message));
-    }
-
-    /**
-     * Handles requests that failed because they were bad input.
-     *
-     * @param request request
-     * @param response response
-     * @throws IOException ioe
-     */
-    private void handleMissingCertificate(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        // set the response status
-        response.setContentType("text/plain");
-        response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-
-        // write the response message
-        PrintWriter out = response.getWriter();
-        out.println("Unable to process request because the user certificate was not specified.");
-
-        // log the failure
-        logger.info("Unable to process request because the user certificate was not specified.");
-    }
-
     /* setters */
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
+
+    public void setCertificateValidator(OcspCertificateValidator certificateValidator) {
+        this.certificateValidator = certificateValidator;
     }
 
-    public void setUserService(UserService userService) {
-        this.userService = userService;
+    public void setPrincipalExtractor(X509PrincipalExtractor principalExtractor) {
+        this.principalExtractor = principalExtractor;
     }
 
-    public void setCertificateValidator(OcspCertificateValidator certificateValidator) {
-        this.certificateValidator = certificateValidator;
+    public void setCertificateExtractor(X509CertificateExtractor certificateExtractor) {
+        this.certificateExtractor = certificateExtractor;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilterOld.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/x509/X509AuthenticationFilterOld.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilterOld.java
new file mode 100644
index 0000000..711639b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilterOld.java
@@ -0,0 +1,317 @@
+/*
+ * 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.x509;
+
+import org.apache.nifi.web.security.x509.ocsp.OcspCertificateValidator;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.security.cert.CertificateExpiredException;
+import java.security.cert.CertificateNotYetValidException;
+import java.security.cert.X509Certificate;
+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.nifi.admin.service.AdministrationException;
+import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
+import org.apache.nifi.web.security.UntrustedProxyException;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.security.authentication.AccountStatusException;
+import org.springframework.security.authentication.AuthenticationServiceException;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.core.userdetails.UsernameNotFoundException;
+import org.springframework.security.web.authentication.preauth.AbstractPreAuthenticatedProcessingFilter;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+
+/**
+ * Custom X509 filter that will inspect the HTTP headers for a proxied user
+ * before extracting the user details from the client certificate.
+ */
+public class X509AuthenticationFilterOld extends AbstractPreAuthenticatedProcessingFilter {
+
+    public static final String PROXY_ENTITIES_CHAIN = "X-ProxiedEntitiesChain";
+    public static final String PROXY_ENTITIES_ACCEPTED = "X-ProxiedEntitiesAccepted";
+    public static final String PROXY_ENTITIES_DETAILS = "X-ProxiedEntitiesDetails";
+
+    private final X509CertificateExtractor certificateExtractor = new X509CertificateExtractor();
+    private final X509PrincipalExtractor principalExtractor = new SubjectDnX509PrincipalExtractor();
+    private OcspCertificateValidator certificateValidator;
+    private NiFiProperties properties;
+    private UserService userService;
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+        final HttpServletResponse httpResponse = (HttpServletResponse) response;
+
+        // determine if this request is attempting to create a new account
+        if (isNewAccountRequest((HttpServletRequest) request)) {
+            // determine if this nifi supports new account requests
+            if (properties.getSupportNewAccountRequests()) {
+                // ensure there is a certificate in the request
+                X509Certificate certificate = certificateExtractor.extractClientCertificate((HttpServletRequest) request);
+                if (certificate != null) {
+                    // extract the principal from the certificate
+                    Object certificatePrincipal = principalExtractor.extractPrincipal(certificate);
+                    String principal = certificatePrincipal.toString();
+
+                    // log the new user account request
+                    logger.info("Requesting new user account for " + principal);
+
+                    try {
+                        // get the justification
+                        String justification = request.getParameter("justification");
+                        if (justification == null) {
+                            justification = StringUtils.EMPTY;
+                        }
+
+                        // create the pending user account
+                        userService.createPendingUserAccount(principal, justification);
+
+                        // generate a response
+                        httpResponse.setStatus(HttpServletResponse.SC_CREATED);
+                        httpResponse.setContentType("text/plain");
+
+                        // write the response message
+                        PrintWriter out = response.getWriter();
+                        out.println("Not authorized. User account created. Authorization pending.");
+                    } catch (IllegalArgumentException iae) {
+                        handleUserServiceError((HttpServletRequest) request, httpResponse, HttpServletResponse.SC_BAD_REQUEST, iae.getMessage());
+                    } catch (AdministrationException ae) {
+                        handleUserServiceError((HttpServletRequest) request, httpResponse, HttpServletResponse.SC_INTERNAL_SERVER_ERROR, ae.getMessage());
+                    }
+                } else {
+                    // can this really happen?
+                    handleMissingCertificate((HttpServletRequest) request, httpResponse);
+                }
+            } else {
+                handleUserServiceError((HttpServletRequest) request, httpResponse, HttpServletResponse.SC_NOT_FOUND, "This NiFi does not support new account requests.");
+            }
+        } else {
+            try {
+                // this not a request to create a user account - try to authorize
+                super.doFilter(request, response, chain);
+            } catch (AuthenticationException ae) {
+                // continue the filter chain since anonymous access should be supported
+                if (!properties.getNeedClientAuth()) {
+                    chain.doFilter(request, response);
+                } else {
+                    // create an appropriate response for the given exception
+                    handleUnsuccessfulAuthentication((HttpServletRequest) request, httpResponse, ae);
+                }
+            }
+        }
+    }
+
+    @Override
+    protected Object getPreAuthenticatedPrincipal(HttpServletRequest request) {
+        String principal;
+
+        // extract the cert
+        X509Certificate certificate = certificateExtractor.extractClientCertificate(request);
+
+        // ensure the cert was found
+        if (certificate == null) {
+            return null;
+        }
+
+        // extract the principal
+        Object certificatePrincipal = principalExtractor.extractPrincipal(certificate);
+        principal = ProxiedEntitiesUtils.formatProxyDn(certificatePrincipal.toString());
+
+        try {
+            // ensure the cert is valid
+            certificate.checkValidity();
+        } catch (CertificateExpiredException cee) {
+            final String message = String.format("Client certificate for (%s) is expired.", principal);
+            logger.info(message, cee);
+            if (logger.isDebugEnabled()) {
+                logger.debug("", cee);
+            }
+            return null;
+        } catch (CertificateNotYetValidException cnyve) {
+            final String message = String.format("Client certificate for (%s) is not yet valid.", principal);
+            logger.info(message, cnyve);
+            if (logger.isDebugEnabled()) {
+                logger.debug("", cnyve);
+            }
+            return null;
+        }
+
+        // validate the certificate in question
+        try {
+            certificateValidator.validate(request);
+        } catch (final Exception e) {
+            logger.info(e.getMessage());
+            if (logger.isDebugEnabled()) {
+                logger.debug("", e);
+            }
+            return null;
+        }
+
+        // look for a proxied user
+        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
+            principal = request.getHeader(PROXY_ENTITIES_CHAIN) + principal;
+        }
+
+        // log the request attempt - response details will be logged later
+        logger.info(String.format("Attempting request for (%s) %s %s (source ip: %s)", principal, request.getMethod(),
+                request.getRequestURL().toString(), request.getRemoteAddr()));
+
+        return principal;
+    }
+
+    @Override
+    protected Object getPreAuthenticatedCredentials(HttpServletRequest request) {
+        return certificateExtractor.extractClientCertificate(request);
+    }
+
+    @Override
+    protected void successfulAuthentication(HttpServletRequest request, HttpServletResponse response, Authentication authResult) {
+        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
+            response.setHeader(PROXY_ENTITIES_ACCEPTED, Boolean.TRUE.toString());
+        }
+        super.successfulAuthentication(request, response, authResult);
+    }
+
+    @Override
+    protected void unsuccessfulAuthentication(HttpServletRequest request, HttpServletResponse response, AuthenticationException failed) {
+        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
+            response.setHeader(PROXY_ENTITIES_DETAILS, failed.getMessage());
+        }
+        super.unsuccessfulAuthentication(request, response, failed);
+    }
+
+    /**
+     * Determines if the specified request is attempting to register a new user
+     * account.
+     *
+     * @param request http request
+     * @return true if new user
+     */
+    private boolean isNewAccountRequest(HttpServletRequest request) {
+        if ("POST".equalsIgnoreCase(request.getMethod())) {
+            String path = request.getPathInfo();
+            if (StringUtils.isNotBlank(path)) {
+                if ("/controller/users".equals(path)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Handles requests that were unable to be authorized.
+     *
+     * @param request request
+     * @param response response
+     * @param ae ex
+     * @throws IOException ex
+     */
+    private void handleUnsuccessfulAuthentication(HttpServletRequest request, HttpServletResponse response, AuthenticationException ae) throws IOException {
+        // set the response status
+        response.setContentType("text/plain");
+
+        // write the response message
+        PrintWriter out = response.getWriter();
+
+        // use the type of authentication exception to determine the response code
+        if (ae instanceof UsernameNotFoundException) {
+            if (properties.getSupportNewAccountRequests()) {
+                response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+                out.println("Not authorized.");
+            } else {
+                response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+                out.println("Access is denied.");
+            }
+        } else if (ae instanceof AccountStatusException) {
+            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+            out.println(ae.getMessage());
+        } else if (ae instanceof UntrustedProxyException) {
+            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+            out.println(ae.getMessage());
+        } else if (ae instanceof AuthenticationServiceException) {
+            logger.error(String.format("Unable to authorize: %s", ae.getMessage()), ae);
+            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+            out.println(String.format("Unable to authorize: %s", ae.getMessage()));
+        } else {
+            logger.error(String.format("Unable to authorize: %s", ae.getMessage()), ae);
+            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+            out.println("Access is denied.");
+        }
+
+        // log the failure
+        logger.info(String.format("Rejecting access to web api: %s", ae.getMessage()));
+
+        // optionally log the stack trace
+        if (logger.isDebugEnabled()) {
+            logger.debug(StringUtils.EMPTY, ae);
+        }
+    }
+
+    private void handleUserServiceError(HttpServletRequest request, HttpServletResponse response, int responseCode, String message) throws IOException {
+        // set the response status
+        response.setContentType("text/plain");
+        response.setStatus(responseCode);
+
+        // write the response message
+        PrintWriter out = response.getWriter();
+        out.println(message);
+
+        // log the failure
+        logger.info(String.format("Unable to process request because %s", message));
+    }
+
+    /**
+     * Handles requests that failed because they were bad input.
+     *
+     * @param request request
+     * @param response response
+     * @throws IOException ioe
+     */
+    private void handleMissingCertificate(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        // set the response status
+        response.setContentType("text/plain");
+        response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+
+        // write the response message
+        PrintWriter out = response.getWriter();
+        out.println("Unable to process request because the user certificate was not specified.");
+
+        // log the failure
+        logger.info("Unable to process request because the user certificate was not specified.");
+    }
+
+    /* setters */
+    public void setProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+    public void setUserService(UserService userService) {
+        this.userService = userService;
+    }
+
+    public void setCertificateValidator(OcspCertificateValidator certificateValidator) {
+        this.certificateValidator = certificateValidator;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.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/x509/X509AuthenticationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.java
new file mode 100644
index 0000000..d24b9cb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.java
@@ -0,0 +1,31 @@
+package org.apache.nifi.web.security.x509;
+
+import org.apache.nifi.web.security.token.NewAccountAuthenticationRequestToken;
+import org.apache.nifi.web.security.token.NewAccountAuthenticationToken;
+import org.apache.nifi.web.security.token.NiFiAuthenticationRequestToken;
+import org.springframework.security.authentication.AuthenticationProvider;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+
+/**
+ *
+ */
+public class X509AuthenticationProvider implements AuthenticationProvider {
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+        if (NewAccountAuthenticationRequestToken.class.isAssignableFrom(authentication.getClass())) {
+            return new NewAccountAuthenticationToken(((NewAccountAuthenticationRequestToken) authentication).getNewAccountRequest());
+        } else if (NiFiAuthenticationRequestToken.class.isAssignableFrom(authentication.getClass())) {
+            return authentication;
+        } else {
+            return null;
+        }
+    }
+    
+    @Override
+    public boolean supports(Class<?> authentication) {
+        return NiFiAuthenticationRequestToken.class.isAssignableFrom(authentication);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/61046707/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 fa5b9e2..b201364 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
@@ -13,71 +13,75 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
-<!-- marked as lazy so that security beans are not created when applications runs in non-secure mode -->
-<beans default-lazy-init="true"
-       xmlns="http://www.springframework.org/schema/beans"
+<beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:sec="http://www.springframework.org/schema/security"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-3.1.xsd
-    http://www.springframework.org/schema/security http://www.springframework.org/schema/security/spring-security-3.2.xsd">
+       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-3.1.xsd">
 
-    <!-- http security configuration -->
-    <sec:http create-session="stateless" auto-config="false" entry-point-ref="entryPoint">
+<!--    <sec:http create-session="stateless" auto-config="false" entry-point-ref="entryPoint">
         <sec:anonymous enabled="false"/>
         <sec:custom-filter ref="nodeAuthorizedUserFilter" before="PRE_AUTH_FILTER"/>
         <sec:custom-filter ref="x509Filter" position="PRE_AUTH_FILTER"/>
         <sec:custom-filter ref="anonymousFilter" position="ANONYMOUS_FILTER"/>
-    </sec:http>
+    </sec:http>-->
 
     <!-- enable method level security -->
-    <sec:global-method-security pre-post-annotations="enabled"/>
+    <!--<sec:global-method-security pre-post-annotations="enabled"/>-->
+
+    <!--<bean class="org.apache.nifi.web.security.NiFiSecurityWebApplicationInitializer"></bean>-->
+
+    <!-- security config -->
+<!--    <bean id="securityConfig" class="org.apache.nifi.web.security.NiFiSecurityConfig">
+        <property name="properties" ref="nifiProperties"/>
+        <property name="userDetailsService" ref="userAuthorizationService"/>
+        <property name="authorizedUserFilter" ref="nodeAuthorizedUserFilter"/>
+        <property name="entryPoint" ref="entryPoint"/>
+    </bean>-->
 
     <!-- entry point reference -->
-    <bean id="entryPoint" class="org.apache.nifi.web.security.authentication.NiFiAuthenticationEntryPoint"/>
+    <!--<bean id="entryPoint" class="org.apache.nifi.web.security.authentication.NiFiAuthenticationEntryPoint"/>-->
     
     <!-- authentication manager -->
-    <sec:authentication-manager alias="authenticationManager">
+<!--    <sec:authentication-manager alias="authenticationManager">
         <sec:authentication-provider ref="preauthAuthProvider"/>
-    </sec:authentication-manager>
+    </sec:authentication-manager>-->
 
     <!-- pre-authentication provider -->
-    <bean id="preauthAuthProvider" class="org.springframework.security.web.authentication.preauth.PreAuthenticatedAuthenticationProvider">
+<!--    <bean id="preauthAuthProvider" class="org.springframework.security.web.authentication.preauth.PreAuthenticatedAuthenticationProvider">
         <property name="preAuthenticatedUserDetailsService">
             <bean class="org.springframework.security.core.userdetails.UserDetailsByNameServiceWrapper">
                 <property name="userDetailsService" ref="userAuthorizationService"/>
             </bean>
         </property>
-    </bean>
+    </bean>-->
 
     <!-- user details service -->
-    <bean id="userAuthorizationService" class="org.apache.nifi.web.security.authorization.NiFiAuthorizationService">
+    <bean id="userDetailsService" class="org.apache.nifi.web.security.authorization.NiFiAuthorizationService">
         <property name="userService" ref="userService"/>
         <property name="properties" ref="nifiProperties"/>
     </bean>
-
+    
+    <!-- jwt service -->
+    <bean id="jwtService" class="org.apache.nifi.web.security.jwt.JwtService"></bean>
+    
     <!-- performs ocsp certificate validation -->
-    <bean id="ocspCertificateValidator" class="org.apache.nifi.web.security.x509.ocsp.OcspCertificateValidator">
+<!--    <bean id="ocspCertificateValidator" class="org.apache.nifi.web.security.x509.ocsp.OcspCertificateValidator">
         <constructor-arg ref="nifiProperties"/>
-    </bean>
+    </bean>-->
 
     <!-- custom x509 filter for checking for proxied users -->
-    <bean id="x509Filter" class="org.apache.nifi.web.security.x509.X509AuthenticationFilter">
-        <property name="authenticationManager" ref="authenticationManager"/>
-        <property name="properties" ref="nifiProperties"/>
-        <property name="userService" ref="userService"/>
+<!--    <bean id="x509Filter" class="org.apache.nifi.web.security.x509.X509AuthenticationFilter">
         <property name="certificateValidator" ref="ocspCertificateValidator"/>
-        <property name="continueFilterChainOnUnsuccessfulAuthentication" value="false"/>
-    </bean>
+    </bean>-->
 
     <!-- custom filter for checking for proxied users that are already authenticated -->
-    <bean id="nodeAuthorizedUserFilter" class="org.apache.nifi.web.security.authorization.NodeAuthorizedUserFilter">
+<!--    <bean id="nodeAuthorizedUserFilter" class="org.apache.nifi.web.security.authorization.NodeAuthorizedUserFilter">
         <property name="properties" ref="nifiProperties"/>
-    </bean>
+    </bean>-->
 
     <!-- custom anonymous filter to assign default roles based on current operating mode -->
-    <bean id="anonymousFilter" class="org.apache.nifi.web.security.anonymous.NiFiAnonymousUserFilter">
+<!--    <bean id="anonymousFilter" class="org.apache.nifi.web.security.anonymous.NiFiAnonymousUserFilter">
         <property name="userService" ref="userService"/>
         <property name="properties" ref="nifiProperties"/>
-    </bean>
+    </bean>-->
     
 </beans>