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/11/23 21:46:57 UTC

[46/50] [abbrv] 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/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index bd7482e..e7a3328 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -170,6 +170,7 @@ import org.apache.nifi.web.api.dto.status.ClusterProcessGroupStatusDTO;
 import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusDTO;
 import org.apache.nifi.web.dao.ControllerServiceDAO;
 import org.apache.nifi.web.dao.ReportingTaskDAO;
+import org.apache.nifi.web.security.user.NewAccountRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.security.access.AccessDeniedException;
@@ -762,7 +763,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         if (user == null) {
             throw new WebApplicationException(new Throwable("Unable to access details for current user."));
         }
-        final String userDn = user.getDn();
+        final String userDn = user.getIdentity();
 
         if (Node.Status.CONNECTING.name().equalsIgnoreCase(nodeDTO.getStatus())) {
             clusterManager.requestReconnection(nodeDTO.getNodeId(), userDn);
@@ -1782,7 +1783,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
         // create a purge action to record that records are being removed
         FlowChangeAction purgeAction = new FlowChangeAction();
-        purgeAction.setUserIdentity(user.getDn());
+        purgeAction.setUserIdentity(user.getIdentity());
         purgeAction.setUserName(user.getUserName());
         purgeAction.setOperation(Operation.Purge);
         purgeAction.setTimestamp(new Date());
@@ -1820,6 +1821,23 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public UserDTO createUser() {
+        NewAccountRequest newAccountRequest = NiFiUserUtils.getNewAccountRequest();
+
+        // log the new user account request
+        logger.info("Requesting new user account for " + newAccountRequest.getUsername());
+
+        // get the justification
+        String justification = newAccountRequest.getJustification();
+        if (justification == null) {
+            justification = StringUtils.EMPTY;
+        }
+
+        // create the pending user account
+        return dtoFactory.createUserDTO(userService.createPendingUserAccount(newAccountRequest.getUsername(), justification));
+    }
+
+    @Override
     public UserDTO updateUser(UserDTO userDto) {
         NiFiUser user;
 
@@ -2256,7 +2274,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         }
 
         final Set<String> allowedUsers = port.getUserAccessControl();
-        if (allowedUsers.contains(user.getDn())) {
+        if (allowedUsers.contains(user.getIdentity())) {
             return true;
         }
 
@@ -2632,7 +2650,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                         final UserDTO groupedUser = groupedUserDTOs.get(user.getUserGroup());
                         groupedUser.setId(groupedUser.getId() + "," + String.valueOf(user.getId()));
                         groupedUser.setUserName(groupedUser.getUserName() + ", " + user.getUserName());
-                        groupedUser.setDn(groupedUser.getDn() + ", " + user.getDn());
+                        groupedUser.setDn(groupedUser.getDn() + ", " + user.getIdentity());
                         groupedUser.setCreation(getOldestDate(groupedUser.getCreation(), user.getCreation()));
                         groupedUser.setLastAccessed(getNewestDate(groupedUser.getLastAccessed(), user.getLastAccessed()));
                         groupedUser.setLastVerified(getNewestDate(groupedUser.getLastVerified(), user.getLastVerified()));
@@ -2731,7 +2749,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             throw new WebApplicationException(new Throwable("Unable to access details for current user."));
         }
 
-        final String userDn = user.getDn();
+        final String userDn = user.getIdentity();
         clusterManager.deleteNode(nodeId, userDn);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
index 8e56fc2..9502bf2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
@@ -158,19 +158,19 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
 
     @Override
     public String getCurrentUserDn() {
-        String userDn = NiFiUser.ANONYMOUS_USER_DN;
+        String userIdentity = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {
-            userDn = user.getDn();
+            userIdentity = user.getIdentity();
         }
 
-        return userDn;
+        return userIdentity;
     }
 
     @Override
     public String getCurrentUserName() {
-        String userName = NiFiUser.ANONYMOUS_USER_DN;
+        String userName = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
index 03ce630..7e9e77e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
@@ -129,19 +129,19 @@ public class StandardNiFiWebContext implements NiFiWebContext {
 
     @Override
     public String getCurrentUserDn() {
-        String userDn = NiFiUser.ANONYMOUS_USER_DN;
+        String userIdentity = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {
-            userDn = user.getDn();
+            userIdentity = user.getIdentity();
         }
 
-        return userDn;
+        return userIdentity;
     }
 
     @Override
     public String getCurrentUserName() {
-        String userName = NiFiUser.ANONYMOUS_USER_DN;
+        String userName = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/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
new file mode 100644
index 0000000..e198438
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import io.jsonwebtoken.JwtException;
+import org.apache.nifi.util.NiFiProperties;
+import com.wordnik.swagger.annotations.Api;
+import com.wordnik.swagger.annotations.ApiOperation;
+import com.wordnik.swagger.annotations.ApiParam;
+import com.wordnik.swagger.annotations.ApiResponse;
+import com.wordnik.swagger.annotations.ApiResponses;
+import java.net.URI;
+import java.security.cert.X509Certificate;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.FormParam;
+import javax.ws.rs.POST;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.authentication.AuthenticationResponse;
+import org.apache.nifi.authentication.LoginCredentials;
+import org.apache.nifi.authentication.LoginIdentityProvider;
+import org.apache.nifi.authentication.exception.IdentityAccessException;
+import org.apache.nifi.authentication.exception.InvalidLoginCredentialsException;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.apache.nifi.web.api.dto.AccessStatusDTO;
+import org.apache.nifi.web.api.dto.AccessConfigurationDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.AccessStatusEntity;
+import org.apache.nifi.web.api.entity.AccessConfigurationEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+import org.apache.nifi.web.security.InvalidAuthenticationException;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
+import org.apache.nifi.web.security.UntrustedProxyException;
+import org.apache.nifi.web.security.jwt.JwtService;
+import org.apache.nifi.web.security.token.LoginAuthenticationToken;
+import org.apache.nifi.web.security.token.NiFiAuthenticationRequestToken;
+import org.apache.nifi.web.security.x509.X509CertificateExtractor;
+import org.apache.nifi.web.security.x509.X509IdentityProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.security.authentication.AccountStatusException;
+import org.springframework.security.authentication.AuthenticationServiceException;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.core.userdetails.AuthenticationUserDetailsService;
+import org.springframework.security.core.userdetails.UsernameNotFoundException;
+
+/**
+ * RESTful endpoint for managing a cluster.
+ */
+@Path("/access")
+@Api(
+        value = "/access",
+        description = "Endpoints for obtaining an access token or checking access status"
+)
+public class AccessResource extends ApplicationResource {
+
+    private static final Logger logger = LoggerFactory.getLogger(AccessResource.class);
+
+    private static final String AUTHORIZATION = "Authorization";
+
+    private NiFiProperties properties;
+
+    private LoginIdentityProvider loginIdentityProvider;
+    private X509CertificateExtractor certificateExtractor;
+    private X509IdentityProvider certificateIdentityProvider;
+    private JwtService jwtService;
+
+    private AuthenticationUserDetailsService<NiFiAuthenticationRequestToken> userDetailsService;
+
+    /**
+     * Retrieves the access configuration for this NiFi.
+     *
+     * @param httpServletRequest the servlet request
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
+     * @return A accessConfigurationEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Path("/config")
+    @ApiOperation(
+            value = "Retrieves the access configuration for this NiFi",
+            response = AccessConfigurationEntity.class
+    )
+    public Response getLoginConfig(
+            @Context HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
+                    required = false
+            )
+            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
+
+        final AccessConfigurationDTO accessConfiguration = new AccessConfigurationDTO();
+
+        // specify whether login should be supported and only support for secure requests
+        accessConfiguration.setSupportsLogin(loginIdentityProvider != null && httpServletRequest.isSecure());
+        accessConfiguration.setSupportsAnonymous(!properties.getAnonymousAuthorities().isEmpty() || !httpServletRequest.isSecure());
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // create the response entity
+        final AccessConfigurationEntity entity = new AccessConfigurationEntity();
+        entity.setRevision(revision);
+        entity.setConfig(accessConfiguration);
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
+
+    /**
+     * Gets the status the client's access.
+     *
+     * @param httpServletRequest the servlet request
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
+     * @return A accessStatusEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces({MediaType.APPLICATION_XML, MediaType.APPLICATION_JSON})
+    @Path("")
+    @ApiOperation(
+            value = "Gets the status the client's access",
+            response = AccessStatusEntity.class
+    )
+    @ApiResponses(
+            value = {
+                @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                @ApiResponse(code = 401, message = "Unable to determine access status because the client could not be authenticated."),
+                @ApiResponse(code = 403, message = "Unable to determine access status because the client is not authorized to make this request."),
+                @ApiResponse(code = 409, message = "Unable to determine access status because NiFi is not in the appropriate state."),
+                @ApiResponse(code = 500, message = "Unable to determine access status because an unexpected error occurred.")
+            }
+    )
+    public Response getAccessStatus(
+            @Context HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
+                    required = false
+            )
+            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
+
+        // only consider user specific access over https
+        if (!httpServletRequest.isSecure()) {
+            throw new IllegalStateException("User authentication/authorization is only supported when running over HTTPS.");
+        }
+
+        final AccessStatusDTO accessStatus = new AccessStatusDTO();
+
+        try {
+            final X509Certificate[] certificates = certificateExtractor.extractClientCertificate(httpServletRequest);
+
+            // if there is not certificate, consider a token
+            if (certificates == null) {
+                // look for an authorization token
+                final String authorization = httpServletRequest.getHeader(AUTHORIZATION);
+
+                // if there is no authorization header, we don't know the user
+                if (authorization == null) {
+                    accessStatus.setStatus(AccessStatusDTO.Status.UNKNOWN.name());
+                    accessStatus.setMessage("No credentials supplied, unknown user.");
+                } else {
+                    try {
+                        // Extract the Base64 encoded token from the Authorization header
+                        final String token = StringUtils.substringAfterLast(authorization, " ");
+                        final String principal = jwtService.getAuthenticationFromToken(token);
+
+                        // set the user identity
+                        accessStatus.setIdentity(principal);
+                        accessStatus.setUsername(CertificateUtils.extractUsername(principal));
+
+                        // without a certificate, this is not a proxied request
+                        final List<String> chain = Arrays.asList(principal);
+
+                        // check authorization for this user
+                        checkAuthorization(chain);
+
+                        // no issues with authorization
+                        accessStatus.setStatus(AccessStatusDTO.Status.ACTIVE.name());
+                        accessStatus.setMessage("Account is active and authorized");
+                    } catch (JwtException e) {
+                        throw new InvalidAuthenticationException(e.getMessage(), e);
+                    }
+                }
+            } else {
+                try {
+                    final AuthenticationResponse authenticationResponse = certificateIdentityProvider.authenticate(certificates);
+
+                    // get the proxy chain and ensure its populated
+                    final List<String> proxyChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(httpServletRequest, authenticationResponse.getIdentity());
+                    if (proxyChain.isEmpty()) {
+                        logger.error(String.format("Unable to parse the proxy chain %s from the incoming request.", authenticationResponse.getIdentity()));
+                        throw new IllegalArgumentException("Unable to determine the user from the incoming request.");
+                    }
+
+                    // set the user identity
+                    accessStatus.setIdentity(proxyChain.get(0));
+                    accessStatus.setUsername(CertificateUtils.extractUsername(proxyChain.get(0)));
+
+                    // ensure the proxy chain is authorized
+                    checkAuthorization(proxyChain);
+
+                    // no issues with authorization
+                    accessStatus.setStatus(AccessStatusDTO.Status.ACTIVE.name());
+                    accessStatus.setMessage("Account is active and authorized");
+                } catch (final IllegalArgumentException iae) {
+                    throw new InvalidAuthenticationException(iae.getMessage(), iae);
+                }
+            }
+        } catch (final UsernameNotFoundException unfe) {
+            accessStatus.setStatus(AccessStatusDTO.Status.UNREGISTERED.name());
+            accessStatus.setMessage(String.format("Unregistered user %s", accessStatus.getIdentity()));
+        } catch (final AccountStatusException ase) {
+            accessStatus.setStatus(AccessStatusDTO.Status.NOT_ACTIVE.name());
+            accessStatus.setMessage(ase.getMessage());
+        } catch (final UntrustedProxyException upe) {
+            throw new AccessDeniedException(upe.getMessage(), upe);
+        } catch (final AuthenticationServiceException ase) {
+            throw new AdministrationException(ase.getMessage(), ase);
+        }
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // create the entity
+        final AccessStatusEntity entity = new AccessStatusEntity();
+        entity.setRevision(revision);
+        entity.setAccessStatus(accessStatus);
+
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Checks the status of the proxy.
+     *
+     * @param proxyChain the proxy chain
+     * @throws AuthenticationException if the proxy chain is not authorized
+     */
+    private void checkAuthorization(final List<String> proxyChain) throws AuthenticationException {
+        userDetailsService.loadUserDetails(new NiFiAuthenticationRequestToken(proxyChain));
+    }
+
+    /**
+     * Creates a token for accessing the REST API via username/password.
+     *
+     * @param httpServletRequest the servlet request
+     * @param username the username
+     * @param password the password
+     * @return A JWT (string)
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token")
+    @ApiOperation(
+            value = "Creates a token for accessing the REST API via username/password",
+            response = String.class
+    )
+    @ApiResponses(
+            value = {
+                @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                @ApiResponse(code = 409, message = "Unable to create access token because NiFi is not in the appropriate state. (i.e. may not be configured to support username/password login."),
+                @ApiResponse(code = 500, message = "Unable to create access token because an unexpected error occurred.")
+            }
+    )
+    public Response createAccessToken(
+            @Context HttpServletRequest httpServletRequest,
+            @FormParam("username") String username,
+            @FormParam("password") String password) {
+
+        // only support access tokens when communicating over HTTPS
+        if (!httpServletRequest.isSecure()) {
+            throw new IllegalStateException("Access tokens are only issued over HTTPS.");
+        }
+
+        // if not configuration for login, don't consider credentials
+        if (loginIdentityProvider == null) {
+            throw new IllegalStateException("Username/Password login not supported by this NiFi.");
+        }
+
+        final LoginAuthenticationToken loginAuthenticationToken;
+
+        final X509Certificate[] certificates = certificateExtractor.extractClientCertificate(httpServletRequest);
+
+        // if there is not certificate, consider login credentials
+        if (certificates == null) {
+            // ensure we have login credentials
+            if (StringUtils.isBlank(username) || StringUtils.isBlank(password)) {
+                throw new IllegalArgumentException("The username and password must be specified.");
+            }
+
+            try {
+                // attempt to authenticate
+                final AuthenticationResponse authenticationResponse = loginIdentityProvider.authenticate(new LoginCredentials(username, password));
+                final long maxExpiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
+                final long minExpiration = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
+
+                long expiration = authenticationResponse.getExpiration();
+                if (expiration > maxExpiration) {
+                    expiration = maxExpiration;
+
+                    logger.warn(String.format("Max token expiration exceeded. Setting expiration to %s from %s for %s", expiration,
+                            authenticationResponse.getExpiration(), authenticationResponse.getIdentity()));
+                } else if (expiration < minExpiration) {
+                    expiration = minExpiration;
+
+                    logger.warn(String.format("Min token expiration not met. Setting expiration to %s from %s for %s", expiration,
+                            authenticationResponse.getExpiration(), authenticationResponse.getIdentity()));
+                }
+
+                // create the authentication token
+                loginAuthenticationToken = new LoginAuthenticationToken(authenticationResponse.getIdentity(), expiration, authenticationResponse.getIssuer());
+            } catch (final InvalidLoginCredentialsException ilce) {
+                throw new IllegalArgumentException("The supplied username and password are not valid.", ilce);
+            } catch (final IdentityAccessException iae) {
+                throw new AdministrationException(iae.getMessage(), iae);
+            }
+        } else {
+            // consider a certificate
+            final AuthenticationResponse authenticationResponse = certificateIdentityProvider.authenticate(certificates);
+
+            // get the proxy chain and ensure its populated
+            final List<String> proxyChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(httpServletRequest, authenticationResponse.getIdentity());
+            if (proxyChain.isEmpty()) {
+                logger.error(String.format("Unable to parse the proxy chain %s from the incoming request.", authenticationResponse.getIdentity()));
+                throw new IllegalArgumentException("Unable to determine the user from the incoming request.");
+            }
+
+            // authorize the proxy if necessary
+            authorizeProxyIfNecessary(proxyChain);
+
+            // create the authentication token
+            loginAuthenticationToken = new LoginAuthenticationToken(proxyChain.get(0), authenticationResponse.getExpiration(), authenticationResponse.getIssuer());
+        }
+
+        // generate JWT for response
+        final String token = jwtService.generateSignedToken(loginAuthenticationToken);
+
+        // build the response
+        final URI uri = URI.create(generateResourceUri("access", "token"));
+        return generateCreatedResponse(uri, token).build();
+    }
+
+    /**
+     * Ensures the proxyChain is authorized before allowing the user to be authenticated.
+     *
+     * @param proxyChain the proxy chain
+     * @throws AuthenticationException if the proxy chain is not authorized
+     */
+    private void authorizeProxyIfNecessary(final List<String> proxyChain) throws AuthenticationException {
+        if (proxyChain.size() > 1) {
+            try {
+                userDetailsService.loadUserDetails(new NiFiAuthenticationRequestToken(proxyChain));
+            } catch (final UsernameNotFoundException unfe) {
+                // if a username not found exception was thrown, the proxies were authorized and now
+                // we can issue a new token to the end user which they will use to identify themselves
+                // when they enter a new account request
+            } catch (final AuthenticationServiceException ase) {
+                // throw an administration exception which will return a 500
+                throw new AdministrationException(ase.getMessage(), ase);
+            } catch (final Exception e) {
+                // any other issue we're going to treat as access denied exception which will return 403
+                throw new AccessDeniedException(e.getMessage(), e);
+            }
+        }
+    }
+
+    // setters
+    public void setProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+    public void setLoginIdentityProvider(LoginIdentityProvider loginIdentityProvider) {
+        this.loginIdentityProvider = loginIdentityProvider;
+    }
+
+    public void setJwtService(JwtService jwtService) {
+        this.jwtService = jwtService;
+    }
+
+    public void setCertificateExtractor(X509CertificateExtractor certificateExtractor) {
+        this.certificateExtractor = certificateExtractor;
+    }
+
+    public void setCertificateIdentityProvider(X509IdentityProvider certificateIdentityProvider) {
+        this.certificateIdentityProvider = certificateIdentityProvider;
+    }
+
+    public void setUserDetailsService(AuthenticationUserDetailsService<NiFiAuthenticationRequestToken> userDetailsService) {
+        this.userDetailsService = userDetailsService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index 3bad5e3..d0c36d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -23,7 +23,6 @@ import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
 import java.io.Serializable;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.security.cert.X509Certificate;
 import java.util.Collection;
 import java.util.Enumeration;
 import java.util.HashMap;
@@ -45,9 +44,8 @@ import org.apache.nifi.action.Operation;
 import org.apache.nifi.cluster.context.ClusterContext;
 import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.web.security.DnUtils;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
 import org.apache.nifi.web.security.user.NiFiUserDetails;
-import org.apache.nifi.web.security.x509.X509CertificateExtractor;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
@@ -55,6 +53,8 @@ import org.apache.nifi.web.util.WebUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.security.core.Authentication;
@@ -362,14 +362,12 @@ public abstract class ApplicationResource {
             result.put(PROXY_SCHEME_HTTP_HEADER, httpServletRequest.getScheme());
         }
 
-        // if this is a secure request, add the custom headers for proxying user requests
-        final X509Certificate cert = new X509CertificateExtractor().extractClientCertificate(httpServletRequest);
-        if (cert != null) {
+        if (httpServletRequest.isSecure()) {
 
             // add the certificate DN to the proxy chain
-            final String xProxiedEntitiesChain = DnUtils.getXProxiedEntitiesChain(httpServletRequest);
-            if (StringUtils.isNotBlank(xProxiedEntitiesChain)) {
-                result.put(PROXIED_ENTITIES_CHAIN_HTTP_HEADER, xProxiedEntitiesChain);
+            final NiFiUser user = NiFiUserUtils.getNiFiUser();
+            if (user != null) {
+                result.put(PROXIED_ENTITIES_CHAIN_HTTP_HEADER, ProxiedEntitiesUtils.buildProxiedEntitiesChainString(user));
             }
 
             // add the user's authorities (if any) to the headers

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.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/ControllerResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 4883721..6e6739d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -79,6 +79,7 @@ import org.apache.nifi.web.api.request.IntegerParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity;
+import org.apache.nifi.web.api.entity.IdentityEntity;
 import org.apache.nifi.web.api.entity.ReportingTaskTypesEntity;
 import org.springframework.security.access.prepost.PreAuthorize;
 
@@ -239,7 +240,7 @@ public class ControllerResource extends ApplicationResource {
     public ProcessGroupResource getGroupResource(
             @ApiParam(
                     value = "The id of the process group that is the parent of the requested resource(s). If the desired process group is "
-                            + "the root group an alias 'root' may be used as the process-group-id.",
+                    + "the root group an alias 'root' may be used as the process-group-id.",
                     required = true
             )
             @PathParam("process-group-id") String groupId) {
@@ -454,13 +455,13 @@ public class ControllerResource extends ApplicationResource {
     @ApiOperation(
             value = "Gets the current revision of this NiFi",
             notes = "NiFi employs an optimistic locking strategy where the client must include a revision in their request when "
-                    + "performing an update. If the specified revision does not match the current base revision a 409 status code "
-                    + "is returned. The revision is comprised of a clientId and a version number. The version is a simple integer "
-                    + "value that is incremented with each change. Including the most recent version tells NiFi that your working "
-                    + "with the most recent flow. In addition to the version the client who is performing the updates is recorded. "
-                    + "This allows the same client to submit multiple requests without having to wait for the previously ones to "
-                    + "return. Invoking this endpoint will return the current base revision. It is also available when retrieving "
-                    + "a process group and in the response of all mutable requests.",
+            + "performing an update. If the specified revision does not match the current base revision a 409 status code "
+            + "is returned. The revision is comprised of a clientId and a version number. The version is a simple integer "
+            + "value that is incremented with each change. Including the most recent version tells NiFi that your working "
+            + "with the most recent flow. In addition to the version the client who is performing the updates is recorded. "
+            + "This allows the same client to submit multiple requests without having to wait for the previously ones to "
+            + "return. Invoking this endpoint will return the current base revision. It is also available when retrieving "
+            + "a process group and in the response of all mutable requests.",
             response = Entity.class,
             authorizations = {
                 @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
@@ -845,6 +846,47 @@ public class ControllerResource extends ApplicationResource {
     }
 
     /**
+     * Retrieves the identity of the user making the request.
+     *
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
+     * @return An identityEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Path("/identity")
+    @ApiOperation(
+            value = "Retrieves the user identity of the user making the request",
+            response = IdentityEntity.class
+    )
+    public Response getIdentity(
+            @ApiParam(
+                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
+                    required = false
+            )
+            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
+
+        // note that the cluster manager will handle this request directly
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        if (user == null) {
+            throw new WebApplicationException(new Throwable("Unable to access details for current user."));
+        }
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // create the response entity
+        IdentityEntity entity = new IdentityEntity();
+        entity.setRevision(revision);
+        entity.setUserId(user.getId());
+        entity.setIdentity(user.getUserName());
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
+
+    /**
      * Retrieves the user details, including the authorities, about the user making the request.
      *
      * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
@@ -854,14 +896,17 @@ public class ControllerResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("/authorities")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN', 'ROLE_PROXY', 'ROLE_NIFI', 'ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Retrieves the user details, including the authorities, about the user making the request",
             response = AuthorityEntity.class,
             authorizations = {
                 @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
                 @Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
-                @Authorization(value = "Administrator", type = "ROLE_ADMIN")
+                @Authorization(value = "Administrator", type = "ROLE_ADMIN"),
+                @Authorization(value = "Proxy", type = "ROLE_PROXY"),
+                @Authorization(value = "NiFi", type = "ROLE_NIFI"),
+                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
             }
     )
     @ApiResponses(

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/UserResource.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/UserResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/UserResource.java
index 4a61ef4..4b9d2ae 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/UserResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/UserResource.java
@@ -16,12 +16,14 @@
  */
 package org.apache.nifi.web.api;
 
+import com.sun.jersey.api.Responses;
 import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiParam;
 import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -39,6 +41,7 @@ import javax.ws.rs.DefaultValue;
 import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
 import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
@@ -59,9 +62,11 @@ import org.apache.nifi.web.api.entity.UserSearchResultsEntity;
 import org.apache.nifi.web.api.entity.UsersEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.web.NiFiServiceFacade;
 import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID;
 import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.springframework.security.access.prepost.PreAuthorize;
 
 /**
@@ -83,12 +88,35 @@ public class UserResource extends ApplicationResource {
     private NiFiProperties properties;
     private NiFiServiceFacade serviceFacade;
 
+    @POST
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("") // necessary due to a bug in swagger
+    @ApiOperation(
+            value = "Creates a user",
+            response = String.class
+    )
+    public Response createUser() {
+        if (!properties.getSupportNewAccountRequests()) {
+            return Responses.notFound().entity("This NiFi does not support new account requests.").build();
+        }
+
+        final NiFiUser nifiUser = NiFiUserUtils.getNiFiUser();
+        if (nifiUser != null) {
+            throw new IllegalArgumentException("User account already created " + nifiUser.getIdentity());
+        }
+
+        // create an account request for the current user
+        final UserDTO user = serviceFacade.createUser();
+
+        final String uri = generateResourceUri("controller", "templates", user.getId());
+        return generateCreatedResponse(URI.create(uri), "Not authorized. User account created. Authorization pending.").build();
+    }
+
     /**
      * Gets all users that are registered within this Controller.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param grouped Whether to return the users in their groups.
      * @return A usersEntity.
      */
@@ -144,9 +172,7 @@ public class UserResource extends ApplicationResource {
     /**
      * Gets the details for the specified user.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The user id.
      * @return A userEntity.
      */
@@ -315,12 +341,9 @@ public class UserResource extends ApplicationResource {
      * Updates the specified user.
      *
      * @param httpServletRequest request
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the user to update.
-     * @param rawAuthorities Array of authorities to assign to the specified
-     * user.
+     * @param rawAuthorities Array of authorities to assign to the specified user.
      * @param status The status of the specified users account.
      * @param formParams form params
      * @return A userEntity
@@ -491,9 +514,7 @@ public class UserResource extends ApplicationResource {
      *
      * @param httpServletRequest request
      * @param id The user id
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @return A userEntity.
      */
     @DELETE

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AccessDeniedExceptionMapper.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/config/AccessDeniedExceptionMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AccessDeniedExceptionMapper.java
index bc6dded..5d50e70 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AccessDeniedExceptionMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AccessDeniedExceptionMapper.java
@@ -39,7 +39,7 @@ public class AccessDeniedExceptionMapper implements ExceptionMapper<AccessDenied
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {
-            logger.info(String.format("%s does not have permission to access the requested resource. Returning %s response.", user.getDn(), Response.Status.FORBIDDEN));
+            logger.info(String.format("%s does not have permission to access the requested resource. Returning %s response.", user.getIdentity(), Response.Status.FORBIDDEN));
         } else {
             logger.info(String.format("User does not have permission to access the requested resource. Returning %s response.", Response.Status.FORBIDDEN));
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/InvalidAuthenticationExceptionMapper.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/config/InvalidAuthenticationExceptionMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/InvalidAuthenticationExceptionMapper.java
new file mode 100644
index 0000000..14d5139
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/InvalidAuthenticationExceptionMapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.config;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.security.InvalidAuthenticationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maps access denied exceptions into a client response.
+ */
+@Provider
+public class InvalidAuthenticationExceptionMapper implements ExceptionMapper<InvalidAuthenticationException> {
+
+    private static final Logger logger = LoggerFactory.getLogger(InvalidAuthenticationExceptionMapper.class);
+
+    @Override
+    public Response toResponse(InvalidAuthenticationException exception) {
+        if (logger.isDebugEnabled()) {
+            logger.debug(StringUtils.EMPTY, exception);
+        }
+
+        return Response.status(Response.Status.UNAUTHORIZED).entity(exception.getMessage()).type("text/plain").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.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/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 87277ae..c16653e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -2382,7 +2382,7 @@ public final class DtoFactory {
         // create the user
         UserDTO userDTO = new UserDTO();
         userDTO.setId(String.valueOf(user.getId()));
-        userDTO.setDn(user.getDn());
+        userDTO.setDn(user.getIdentity());
         userDTO.setUserName(user.getUserName());
         userDTO.setUserGroup(user.getUserGroup());
         userDTO.setJustification(user.getJustification());

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 0e3bcac..47ebd30 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -113,6 +113,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
 import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
 import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
 import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
 import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -822,17 +823,7 @@ public class ControllerFacade {
             final Map<String, String> attributes = event.getAttributes();
 
             // calculate the dn chain
-            final List<String> dnChain = new ArrayList<>();
-
-            // build the dn chain
-            NiFiUser chainedUser = user;
-            do {
-                // add the entry for this user
-                dnChain.add(chainedUser.getDn());
-
-                // go to the next user in the chain
-                chainedUser = chainedUser.getChain();
-            } while (chainedUser != null);
+            final List<String> dnChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(user);
 
             // ensure the users in this chain are allowed to download this content
             final DownloadAuthorization downloadAuthorization = userService.authorizeDownload(dnChain, attributes);
@@ -850,7 +841,7 @@ public class ControllerFacade {
             final String type = event.getAttributes().get(CoreAttributes.MIME_TYPE.key());
 
             // get the content
-            final InputStream content = flowController.getContent(event, contentDirection, user.getDn(), uri);
+            final InputStream content = flowController.getContent(event, contentDirection, user.getIdentity(), uri);
             return new DownloadableContent(filename, type, content);
         } catch (final ContentNotFoundException cnfe) {
             throw new ResourceNotFoundException("Unable to find the specified content.");
@@ -880,7 +871,7 @@ public class ControllerFacade {
             }
 
             // replay the flow file
-            final ProvenanceEventRecord event = flowController.replayFlowFile(originalEvent, user.getDn());
+            final ProvenanceEventRecord event = flowController.replayFlowFile(originalEvent, user.getIdentity());
 
             // convert the event record
             return createProvenanceEventDto(event);

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
index df271a5..565e5af 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
@@ -321,7 +321,7 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
             throw new WebApplicationException(new Throwable("Unable to access details for current user."));
         }
 
-        return queue.dropFlowFiles(dropRequestId, user.getDn());
+        return queue.dropFlowFiles(dropRequestId, user.getIdentity());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/RequestLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/RequestLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/RequestLogger.java
index bfd2df6..9f63611 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/RequestLogger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/RequestLogger.java
@@ -52,13 +52,13 @@ public class RequestLogger implements Filter {
             final NiFiUser user = NiFiUserUtils.getNiFiUser();
 
             // get the user details for the log message
-            String dn = "<no user found>";
+            String identity = "<no user found>";
             if (user != null) {
-                dn = user.getDn();
+                identity = user.getIdentity();
             }
 
             // log the request attempt - response details will be logged later
-            logger.info(String.format("Attempting request for (%s) %s %s (source ip: %s)", dn, request.getMethod(),
+            logger.info(String.format("Attempting request for (%s) %s %s (source ip: %s)", identity, request.getMethod(),
                     request.getRequestURL().toString(), request.getRemoteAddr()));
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/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 e034baa..9f3d2f5 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
@@ -241,12 +241,21 @@
     <bean id="systemDiagnosticsResource" class="org.apache.nifi.web.api.SystemDiagnosticsResource" scope="singleton">
         <property name="serviceFacade" ref="serviceFacade"/>
     </bean>
+    <bean id="accessResource" class="org.apache.nifi.web.api.AccessResource" scope="singleton">
+        <property name="properties" ref="nifiProperties"/>
+        <property name="certificateExtractor" ref="certificateExtractor"/>
+        <property name="certificateIdentityProvider" ref="certificateIdentityProvider"/>
+        <property name="loginIdentityProvider" ref="loginIdentityProvider"/>
+        <property name="jwtService" ref="jwtService"/>
+        <property name="userDetailsService" ref="userDetailsService"/>
+    </bean>
 
     <!-- configuration for jaxb serialization -->
     <bean class="org.apache.nifi.web.util.ObjectMapperResolver" scope="singleton"/>
 
     <!-- exception mapping -->
     <bean class="org.apache.nifi.web.api.config.AccessDeniedExceptionMapper" scope="singleton"/>
+    <bean class="org.apache.nifi.web.api.config.InvalidAuthenticationExceptionMapper" scope="singleton"/>
     <bean class="org.apache.nifi.web.api.config.AuthenticationCredentialsNotFoundExceptionMapper" scope="singleton"/>
     <bean class="org.apache.nifi.web.api.config.AccountNotFoundExceptionMapper" scope="singleton"/>
     <bean class="org.apache.nifi.web.api.config.AdministrationExceptionMapper" scope="singleton"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml
index 4ce319e..b57998d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml
@@ -16,15 +16,12 @@
 <web-app version="3.0" xmlns="http://java.sun.com/xml/ns/javaee" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd">
     <display-name>nifi-api</display-name>
     <context-param>
+        <param-name>contextClass</param-name>
+        <param-value>org.springframework.web.context.support.AnnotationConfigWebApplicationContext</param-value>
+    </context-param>
+    <context-param>
         <param-name>contextConfigLocation</param-name>
-        <param-value>
-            classpath:nifi-context.xml
-            classpath:nifi-web-api-context.xml
-            classpath:nifi-web-security-context.xml
-            classpath:nifi-administration-context.xml
-            classpath:nifi-cluster-manager-context.xml
-            classpath:nifi-cluster-protocol-context.xml
-        </param-value>
+        <param-value>org.apache.nifi.web</param-value>
     </context-param>
     <listener>
         <listener-class>org.springframework.web.context.ContextLoaderListener</listener-class>

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessTokenEndpointTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessTokenEndpointTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessTokenEndpointTest.java
new file mode 100644
index 0000000..82fe73a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessTokenEndpointTest.java
@@ -0,0 +1,292 @@
+/*
+ * 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.integration.accesscontrol;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import javax.net.ssl.SSLContext;
+import org.apache.commons.io.FileUtils;
+import org.apache.nifi.integration.util.NiFiTestServer;
+import org.apache.nifi.integration.util.NiFiTestUser;
+import org.apache.nifi.integration.util.SourceTestProcessor;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.api.dto.AccessConfigurationDTO;
+import org.apache.nifi.web.api.dto.AccessStatusDTO;
+import org.apache.nifi.web.api.dto.ProcessorDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.AccessConfigurationEntity;
+import org.apache.nifi.web.api.entity.AccessStatusEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.util.WebUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Access token endpoint test.
+ */
+public class AccessTokenEndpointTest {
+
+    private static final String CLIENT_ID = "token-endpoint-id";
+    private static final String CONTEXT_PATH = "/nifi-api";
+    private static final String FLOW_XML_PATH = "target/test-classes/access-control/flow-admin.xml";
+
+    private static NiFiTestServer SERVER;
+    private static NiFiTestUser TOKEN_USER;
+    private static String BASE_URL;
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        // configure the location of the nifi properties
+        File nifiPropertiesFile = new File("src/test/resources/access-control/nifi.properties");
+        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, nifiPropertiesFile.getAbsolutePath());
+
+        // update the flow.xml property
+        NiFiProperties props = NiFiProperties.getInstance();
+        props.setProperty("nifi.flow.configuration.file", FLOW_XML_PATH);
+
+        // delete the database directory to avoid issues with re-registration in testRequestAccessUsingToken
+        FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile());
+
+        // load extensions
+        NarClassLoaders.load(props);
+        ExtensionManager.discoverExtensions();
+
+        // start the server
+        SERVER = new NiFiTestServer("src/main/webapp", CONTEXT_PATH);
+        SERVER.startServer();
+        SERVER.loadFlow();
+
+        // get the base url
+        BASE_URL = SERVER.getBaseUrl() + CONTEXT_PATH;
+
+        // create the user
+        final Client client = WebUtils.createClient(null, createTrustContext(props));
+        TOKEN_USER = new NiFiTestUser(client, null);
+    }
+
+    private static SSLContext createTrustContext(final NiFiProperties props) throws Exception {
+        return SslContextFactory.createTrustSslContext(props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE),
+                props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD).toCharArray(),
+                props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE), "TLS");
+    }
+
+    // -----------
+    // LOGIN CONIG
+    // -----------
+    /**
+     * Test getting access configuration.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testGetAccessConfig() throws Exception {
+        String url = BASE_URL + "/access/config";
+
+        ClientResponse response = TOKEN_USER.testGet(url);
+
+        // ensure the request is successful
+        Assert.assertEquals(200, response.getStatus());
+
+        // extract the process group
+        AccessConfigurationEntity accessConfigEntity = response.getEntity(AccessConfigurationEntity.class);
+
+        // ensure there is content
+        Assert.assertNotNull(accessConfigEntity);
+
+        // extract the process group dto
+        AccessConfigurationDTO accessConfig = accessConfigEntity.getConfig();
+
+        // verify config
+        Assert.assertTrue(accessConfig.getSupportsLogin());
+        Assert.assertFalse(accessConfig.getSupportsAnonymous());
+    }
+
+    /**
+     * Obtains a token and creates a processor using it.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testCreateProcessorUsingToken() throws Exception {
+        String url = BASE_URL + "/access/token";
+
+        ClientResponse response = TOKEN_USER.testCreateToken(url, "user@nifi", "whateve");
+
+        // ensure the request is successful
+        Assert.assertEquals(201, response.getStatus());
+
+        // get the token
+        String token = response.getEntity(String.class);
+
+        // attempt to create a processor with it
+        createProcessor(token);
+    }
+
+    private ProcessorDTO createProcessor(final String token) throws Exception {
+        String url = BASE_URL + "/controller/process-groups/root/processors";
+
+        // authorization header
+        Map<String, String> headers = new HashMap<>();
+        headers.put("Authorization", "Bearer " + token);
+
+        // create the processor
+        ProcessorDTO processor = new ProcessorDTO();
+        processor.setName("Copy");
+        processor.setType(SourceTestProcessor.class.getName());
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(CLIENT_ID);
+        revision.setVersion(NiFiTestUser.REVISION);
+
+        // create the entity body
+        ProcessorEntity entity = new ProcessorEntity();
+        entity.setRevision(revision);
+        entity.setProcessor(processor);
+
+        // perform the request
+        ClientResponse response = TOKEN_USER.testPostWithHeaders(url, entity, headers);
+
+        // ensure the request is successful
+        Assert.assertEquals(201, response.getStatus());
+
+        // get the entity body
+        entity = response.getEntity(ProcessorEntity.class);
+
+        // verify creation
+        processor = entity.getProcessor();
+        Assert.assertEquals("Copy", processor.getName());
+        Assert.assertEquals("org.apache.nifi.integration.util.SourceTestProcessor", processor.getType());
+
+        return processor;
+    }
+
+    /**
+     * Verifies the response when bad credentials are specified.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testInvalidCredentials() throws Exception {
+        String url = BASE_URL + "/access/token";
+
+        ClientResponse response = TOKEN_USER.testCreateToken(url, "user@nifi", "not a real password");
+
+        // ensure the request is successful
+        Assert.assertEquals(400, response.getStatus());
+    }
+
+    /**
+     * Verifies the response when the user is known.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testUnkownUser() throws Exception {
+        String url = BASE_URL + "/access/token";
+
+        ClientResponse response = TOKEN_USER.testCreateToken(url, "not a real user", "not a real password");
+
+        // ensure the request is successful
+        Assert.assertEquals(400, response.getStatus());
+    }
+
+    /**
+     * Request access using access token.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testRequestAccessUsingToken() throws Exception {
+        String accessStatusUrl = BASE_URL + "/access";
+        String accessTokenUrl = BASE_URL + "/access/token";
+        String registrationUrl = BASE_URL + "/controller/users";
+
+        ClientResponse response = TOKEN_USER.testGet(accessStatusUrl);
+
+        // ensure the request is successful
+        Assert.assertEquals(200, response.getStatus());
+
+        AccessStatusEntity accessStatusEntity = response.getEntity(AccessStatusEntity.class);
+        AccessStatusDTO accessStatus = accessStatusEntity.getAccessStatus();
+
+        // verify unknown
+        Assert.assertEquals("UNKNOWN", accessStatus.getStatus());
+
+        response = TOKEN_USER.testCreateToken(accessTokenUrl, "unregistered-user@nifi", "password");
+
+        // ensure the request is successful
+        Assert.assertEquals(201, response.getStatus());
+
+        // get the token
+        String token = response.getEntity(String.class);
+
+        // authorization header
+        Map<String, String> headers = new HashMap<>();
+        headers.put("Authorization", "Bearer " + token);
+
+        // check the status with the token
+        response = TOKEN_USER.testGetWithHeaders(accessStatusUrl, null, headers);
+
+        // ensure the request is successful
+        Assert.assertEquals(200, response.getStatus());
+
+        accessStatusEntity = response.getEntity(AccessStatusEntity.class);
+        accessStatus = accessStatusEntity.getAccessStatus();
+
+        // verify unregistered
+        Assert.assertEquals("UNREGISTERED", accessStatus.getStatus());
+
+        response = TOKEN_USER.testRegisterUser(registrationUrl, "Gimme access", headers);
+
+        // ensure the request is successful
+        Assert.assertEquals(201, response.getStatus());
+
+        // check the status with the token
+        response = TOKEN_USER.testGetWithHeaders(accessStatusUrl, null, headers);
+
+        // ensure the request is successful
+        Assert.assertEquals(200, response.getStatus());
+
+        accessStatusEntity = response.getEntity(AccessStatusEntity.class);
+        accessStatus = accessStatusEntity.getAccessStatus();
+
+        // verify unregistered
+        Assert.assertEquals("NOT_ACTIVE", accessStatus.getStatus());
+    }
+
+    @AfterClass
+    public static void cleanup() throws Exception {
+        // shutdown the server
+        SERVER.shutdownServer();
+        SERVER = null;
+
+        // look for the flow.xml
+        File flow = new File(FLOW_XML_PATH);
+        if (flow.exists()) {
+            flow.delete();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
index d51b7df..d29be92 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
@@ -37,7 +37,7 @@ import org.apache.nifi.authorization.DownloadAuthorization;
  */
 public class NiFiTestAuthorizationProvider implements AuthorityProvider {
 
-    private Map<String, Set<Authority>> users;
+    private final Map<String, Set<Authority>> users;
 
     /**
      * Creates a new FileAuthorizationProvider.
@@ -48,6 +48,7 @@ public class NiFiTestAuthorizationProvider implements AuthorityProvider {
         users.put("CN=Lastname Firstname Middlename monitor, OU=Unknown, OU=Unknown, OU=Unknown, O=Unknown, C=Unknown", EnumSet.of(Authority.ROLE_MONITOR));
         users.put("CN=Lastname Firstname Middlename dfm, OU=Unknown, OU=Unknown, OU=Unknown, O=Unknown, C=Unknown", EnumSet.of(Authority.ROLE_DFM));
         users.put("CN=Lastname Firstname Middlename admin, OU=Unknown, OU=Unknown, OU=Unknown, O=Unknown, C=Unknown", EnumSet.of(Authority.ROLE_ADMIN));
+        users.put("user@nifi", EnumSet.of(Authority.ROLE_DFM));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestLoginIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestLoginIdentityProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestLoginIdentityProvider.java
new file mode 100644
index 0000000..c023ce1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestLoginIdentityProvider.java
@@ -0,0 +1,75 @@
+/*
+ * 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.integration.util;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.authentication.AuthenticationResponse;
+import org.apache.nifi.authentication.LoginCredentials;
+import org.apache.nifi.authentication.LoginIdentityProvider;
+import org.apache.nifi.authentication.LoginIdentityProviderConfigurationContext;
+import org.apache.nifi.authentication.LoginIdentityProviderInitializationContext;
+import org.apache.nifi.authentication.exception.IdentityAccessException;
+import org.apache.nifi.authentication.exception.InvalidLoginCredentialsException;
+
+/**
+ *
+ */
+public class NiFiTestLoginIdentityProvider implements LoginIdentityProvider {
+
+    private final Map<String, String> users;
+
+    /**
+     * Creates a new FileAuthorizationProvider.
+     */
+    public NiFiTestLoginIdentityProvider() {
+        users = new HashMap<>();
+        users.put("user@nifi", "whateve");
+        users.put("unregistered-user@nifi", "password");
+    }
+
+    private void checkUser(final String user, final String password) {
+        if (!users.containsKey(user)) {
+            throw new InvalidLoginCredentialsException("Unknown user");
+        }
+
+        if (!users.get(user).equals(password)) {
+            throw new InvalidLoginCredentialsException("Invalid password");
+        }
+    }
+
+    @Override
+    public AuthenticationResponse authenticate(LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException {
+        checkUser(credentials.getUsername(), credentials.getPassword());
+        return new AuthenticationResponse(credentials.getUsername(), credentials.getUsername(), TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS), getClass().getSimpleName());
+    }
+
+    @Override
+    public void initialize(LoginIdentityProviderInitializationContext initializationContext) throws ProviderCreationException {
+    }
+
+    @Override
+    public void onConfigured(LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException {
+    }
+
+    @Override
+    public void preDestruction() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
index 42b0aab..38c2d41 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
@@ -78,8 +78,12 @@ public class NiFiTestServer {
     private void createSecureConnector() {
         org.eclipse.jetty.util.ssl.SslContextFactory contextFactory = new org.eclipse.jetty.util.ssl.SslContextFactory();
 
-        // need client auth
-        contextFactory.setNeedClientAuth(properties.getNeedClientAuth());
+        // require client auth when not supporting login or anonymous access
+        if (StringUtils.isBlank(properties.getProperty(NiFiProperties.SECURITY_USER_LOGIN_IDENTITY_PROVIDER)) && properties.getAnonymousAuthorities().isEmpty()) {
+            contextFactory.setNeedClientAuth(true);
+        } else {
+            contextFactory.setWantClientAuth(true);
+        }
 
         /* below code sets JSSE system properties when values are provided */
         // keystore properties
@@ -163,7 +167,6 @@ public class NiFiTestServer {
     }
 
     public Client getClient() {
-        // create the client
         return WebUtils.createClient(null, SslContextFactory.createSslContext(properties));
     }