You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2017/11/21 16:17:34 UTC

[1/4] nifi-registry git commit: NIFIREG-45 Refactor LoginIdentityProvider interface as a more generic and flexible IdentityProvider interface

Repository: nifi-registry
Updated Branches:
  refs/heads/master 63ddf4129 -> ef8ba127c


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtService.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtService.java
index 49c17ea..4401a15 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtService.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtService.java
@@ -28,17 +28,20 @@ import io.jsonwebtoken.SignatureException;
 import io.jsonwebtoken.SigningKeyResolverAdapter;
 import io.jsonwebtoken.UnsupportedJwtException;
 import org.apache.commons.lang3.StringUtils;
-
 import org.apache.nifi.registry.exception.AdministrationException;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
 import org.apache.nifi.registry.security.key.Key;
 import org.apache.nifi.registry.security.key.KeyService;
-import org.apache.nifi.registry.web.security.authentication.token.LoginAuthenticationToken;
 import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
 
 import java.nio.charset.StandardCharsets;
+import java.text.SimpleDateFormat;
 import java.util.Calendar;
+import java.util.concurrent.TimeUnit;
 
+// TODO, look into replacing this JwtService service with Apache Licensed JJWT library
 @Service
 public class JwtService {
 
@@ -50,6 +53,7 @@ public class JwtService {
 
     private final KeyService keyService;
 
+    @Autowired
     public JwtService(final KeyService keyService) {
         this.keyService = keyService;
     }
@@ -68,7 +72,7 @@ public class JwtService {
                 throw new JwtException("No subject available in token");
             }
 
-            // TODO: Validate issuer against active registry?
+            // TODO: Validate issuer against active IdentityProvider?
             if (StringUtils.isEmpty(jws.getBody().getIssuer())) {
                 throw new JwtException("No issuer available in token");
             }
@@ -110,46 +114,48 @@ public class JwtService {
     /**
      * Generates a signed JWT token from the provided (Spring Security) login authentication token.
      *
-     * @param authenticationToken an instance of the Spring Security token after login credentials have been verified against the respective information source
+     * @param authenticationResponse an instance of the Spring Security token after login credentials have been verified against the respective information source
      * @return a signed JWT containing the user identity and the identity provider, Base64-encoded
      * @throws JwtException if there is a problem generating the signed token
      */
-    public String generateSignedToken(final LoginAuthenticationToken authenticationToken) throws JwtException {
-        if (authenticationToken == null) {
+    public String generateSignedToken(final AuthenticationResponse authenticationResponse) throws JwtException {
+        if (authenticationResponse == null) {
             throw new IllegalArgumentException("Cannot generate a JWT for a null authentication token");
         }
 
         // Set expiration from the token
-        final Calendar expiration = Calendar.getInstance();
-        expiration.setTimeInMillis(authenticationToken.getExpiration());
+        final Calendar now = Calendar.getInstance();
+        long expirationMillisRelativeToNow = validateTokenExpiration(authenticationResponse.getExpiration(), authenticationResponse.getIdentity());
+        long expirationMillis = now.getTimeInMillis() + expirationMillisRelativeToNow;
+        final Calendar expiration = new Calendar.Builder().setInstant(expirationMillis).build();
 
-        final Object principal = authenticationToken.getPrincipal();
+        final Object principal = authenticationResponse.getIdentity();
         if (principal == null || StringUtils.isEmpty(principal.toString())) {
-            final String errorMessage = "Cannot generate a JWT for a token with an empty identity issued by " + authenticationToken.getIssuer();
+            final String errorMessage = "Cannot generate a JWT for a token with an empty identity issued by " + authenticationResponse.getIssuer();
             logger.error(errorMessage);
             throw new JwtException(errorMessage);
         }
 
         // Create a JWT with the specified authentication
         final String identity = principal.toString();
-        final String username = authenticationToken.getName();
+        final String username = authenticationResponse.getUsername();
 
         try {
             // Get/create the key for this user
             final Key key = keyService.getOrCreateKey(identity);
             final byte[] keyBytes = key.getKey().getBytes(StandardCharsets.UTF_8);
 
-            logger.trace("Generating JWT for " + authenticationToken);
+            logger.trace("Generating JWT for " + describe(authenticationResponse));
 
             // TODO: Implement "jti" claim with nonce to prevent replay attacks and allow blacklisting of revoked tokens
             // Build the token
             return Jwts.builder().setSubject(identity)
-                    .setIssuer(authenticationToken.getIssuer())
-                    .setAudience(authenticationToken.getIssuer())
+                    .setIssuer(authenticationResponse.getIssuer())
+                    .setAudience(authenticationResponse.getIssuer())
                     .claim(USERNAME_CLAIM, username)
                     .claim(KEY_ID_CLAIM, key.getId())
+                    .setIssuedAt(now.getTime())
                     .setExpiration(expiration.getTime())
-                    .setIssuedAt(Calendar.getInstance().getTime())
                     .signWith(SIGNATURE_ALGORITHM, keyBytes).compact();
         } catch (NullPointerException | AdministrationException e) {
             final String errorMessage = "Could not retrieve the signing key for JWT for " + identity;
@@ -157,4 +163,44 @@ public class JwtService {
             throw new JwtException(errorMessage, e);
         }
     }
+
+    private long validateTokenExpiration(long proposedTokenExpiration, String identity) {
+        final long maxExpiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
+        final long minExpiration = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
+
+        if (proposedTokenExpiration > maxExpiration) {
+            logger.warn(String.format("Max token expiration exceeded. Setting expiration to %s from %s for %s", maxExpiration,
+                    proposedTokenExpiration, identity));
+            proposedTokenExpiration = maxExpiration;
+        } else if (proposedTokenExpiration < minExpiration) {
+            logger.warn(String.format("Min token expiration not met. Setting expiration to %s from %s for %s", minExpiration,
+                    proposedTokenExpiration, identity));
+            proposedTokenExpiration = minExpiration;
+        }
+
+        return proposedTokenExpiration;
+    }
+
+    private static String describe(AuthenticationResponse authenticationResponse) {
+        Calendar expirationTime = Calendar.getInstance();
+        expirationTime.setTimeInMillis(authenticationResponse.getExpiration());
+        long remainingTime = expirationTime.getTimeInMillis() - Calendar.getInstance().getTimeInMillis();
+
+        SimpleDateFormat dateFormat = new SimpleDateFormat("dd-MM-yyyy HH:mm:ss.SSS");
+        dateFormat.setTimeZone(expirationTime.getTimeZone());
+        String expirationTimeString = dateFormat.format(expirationTime.getTime());
+
+        return new StringBuilder("LoginAuthenticationToken for ")
+                .append(authenticationResponse.getUsername())
+                .append(" issued by ")
+                .append(authenticationResponse.getIssuer())
+                .append(" expiring at ")
+                .append(expirationTimeString)
+                .append(" [")
+                .append(authenticationResponse.getExpiration())
+                .append(" ms, ")
+                .append(remainingTime)
+                .append(" ms remaining]")
+                .toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/LoginAuthenticationToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/LoginAuthenticationToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/LoginAuthenticationToken.java
deleted file mode 100644
index 08f0637..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/LoginAuthenticationToken.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.token;
-
-import org.apache.nifi.registry.security.util.CertificateUtils;
-import org.springframework.security.authentication.AbstractAuthenticationToken;
-
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-
-/**
- * This is an Authentication Token for logging in. Once a user is authenticated, they can be issued an ID token.
- */
-public class LoginAuthenticationToken extends AbstractAuthenticationToken {
-
-    private final String identity;
-    private final String username;
-    private final long expiration;
-    private final String issuer;
-
-    /**
-     * Creates a representation of the authentication token for a user.
-     *
-     * @param identity   The unique identifier for this user
-     * @param expiration The relative time to expiration in milliseconds
-     * @param issuer     The IdentityProvider implementation that generated this token
-     */
-    public LoginAuthenticationToken(final String identity, final long expiration, final String issuer) {
-        this(identity, null, expiration, issuer);
-    }
-
-    /**
-     * Creates a representation of the authentication token for a user.
-     *
-     * @param identity   The unique identifier for this user (cannot be null or empty)
-     * @param username   The preferred username for this user
-     * @param expiration The relative time to expiration in milliseconds
-     * @param issuer     The IdentityProvider implementation that generated this token
-     */
-    public LoginAuthenticationToken(final String identity, final String username, final long expiration, final String issuer) {
-        super(null);
-        setAuthenticated(true);
-        this.identity = identity;
-        this.username = username;
-        this.issuer = issuer;
-        Calendar now = Calendar.getInstance();
-        this.expiration = now.getTimeInMillis() + expiration;
-    }
-
-    @Override
-    public Object getCredentials() {
-        return null;
-    }
-
-    @Override
-    public Object getPrincipal() {
-        return identity;
-    }
-
-    /**
-     * Returns the expiration instant in milliseconds. This value is an absolute point in time (i.e. Nov
-     * 16, 2015 11:30:00.000 GMT), not a relative time (i.e. 60 minutes). It is calculated by adding the
-     * relative expiration from the constructor to the timestamp at object creation.
-     *
-     * @return the expiration in millis
-     */
-    public long getExpiration() {
-        return expiration;
-    }
-
-    public String getIssuer() {
-        return issuer;
-    }
-
-    @Override
-    public String getName() {
-        if (username == null) {
-            // if the username is a DN this will extract the username or CN... if not will return what was passed
-            return CertificateUtils.extractUsername(identity);
-        } else {
-            return username;
-        }
-    }
-
-    @Override
-    public String toString() {
-        Calendar expirationTime = Calendar.getInstance();
-        expirationTime.setTimeInMillis(getExpiration());
-        long remainingTime = expirationTime.getTimeInMillis() - Calendar.getInstance().getTimeInMillis();
-
-        SimpleDateFormat dateFormat = new SimpleDateFormat("dd-MM-yyyy HH:mm:ss.SSS");
-        dateFormat.setTimeZone(expirationTime.getTimeZone());
-        String expirationTimeString = dateFormat.format(expirationTime.getTime());
-
-        return new StringBuilder("LoginAuthenticationToken for ")
-                .append(getName())
-                .append(" issued by ")
-                .append(getIssuer())
-                .append(" expiring at ")
-                .append(expirationTimeString)
-                .append(" [")
-                .append(getExpiration())
-                .append(" ms, ")
-                .append(remainingTime)
-                .append(" ms remaining]")
-                .toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java
deleted file mode 100644
index 19e56c5..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.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 NiFiAuthenticationToken extends AbstractAuthenticationToken {
-
-    final UserDetails nifiUserDetails;
-
-    public NiFiAuthenticationToken(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.");
-    }
-
-    @Override
-    public String toString() {
-        return nifiUserDetails.getUsername();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationFilter.java
deleted file mode 100644
index fa0fce2..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationFilter.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.x509;
-
-import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationFilter;
-import org.apache.nifi.registry.web.security.authentication.ProxiedEntitiesUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
-
-import javax.servlet.http.HttpServletRequest;
-import java.security.cert.X509Certificate;
-
-/**
- * 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 NiFiAuthenticationFilter {
-
-    private static final Logger logger = LoggerFactory.getLogger(X509AuthenticationFilter.class);
-
-    private X509CertificateExtractor certificateExtractor;
-    private X509PrincipalExtractor principalExtractor;
-
-    @Override
-    public Authentication attemptAuthentication(final HttpServletRequest request) {
-        // only suppport x509 login when running securely
-        if (!request.isSecure()) {
-            return null;
-        }
-
-        // look for a client certificate
-        final X509Certificate[] certificates = certificateExtractor.extractClientCertificate(request);
-        if (certificates == null) {
-            return null;
-        }
-
-        return new X509AuthenticationRequestToken(request.getHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN), principalExtractor, certificates, request.getRemoteAddr());
-    }
-
-    /* setters */
-    public void setCertificateExtractor(X509CertificateExtractor certificateExtractor) {
-        this.certificateExtractor = certificateExtractor;
-    }
-
-    public void setPrincipalExtractor(X509PrincipalExtractor principalExtractor) {
-        this.principalExtractor = principalExtractor;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationProvider.java
deleted file mode 100644
index 3e935a2..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationProvider.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.x509;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
-import org.apache.nifi.registry.security.authorization.Authorizer;
-import org.apache.nifi.registry.security.authorization.RequestAction;
-import org.apache.nifi.registry.security.authorization.Resource;
-import org.apache.nifi.registry.security.authorization.UserContextKeys;
-import org.apache.nifi.registry.security.authorization.resource.Authorizable;
-import org.apache.nifi.registry.security.authorization.resource.ResourceFactory;
-import org.apache.nifi.registry.security.authorization.user.NiFiUser;
-import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
-import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.web.response.AuthenticationResponse;
-import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
-import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationProvider;
-import org.apache.nifi.registry.web.security.authentication.ProxiedEntitiesUtils;
-import org.apache.nifi.registry.web.security.authentication.exception.UntrustedProxyException;
-import org.apache.nifi.registry.web.security.authentication.token.NiFiAuthenticationToken;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.core.AuthenticationException;
-import org.springframework.stereotype.Component;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Set;
-
-@Component
-public class X509AuthenticationProvider extends NiFiAuthenticationProvider {
-
-    private static final Authorizable PROXY_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getProxyResource();
-        }
-    };
-
-    private X509IdentityProvider certificateIdentityProvider;
-    private Authorizer authorizer;
-
-    @Autowired
-    public X509AuthenticationProvider(
-            final X509IdentityProvider certificateIdentityProvider,
-            final Authorizer authorizer,
-            final NiFiRegistryProperties properties) {
-        super(properties, authorizer);
-        this.certificateIdentityProvider = certificateIdentityProvider;
-        this.authorizer = authorizer;
-    }
-
-    @Override
-    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
-        final X509AuthenticationRequestToken request = (X509AuthenticationRequestToken) authentication;
-
-        // attempt to authenticate if certificates were found
-        final AuthenticationResponse authenticationResponse;
-        try {
-            authenticationResponse = certificateIdentityProvider.authenticate(request.getCertificates());
-        } catch (final IllegalArgumentException iae) {
-            throw new InvalidAuthenticationException(iae.getMessage(), iae);
-        }
-
-        if (StringUtils.isBlank(request.getProxiedEntitiesChain())) {
-            final String mappedIdentity = mapIdentity(authenticationResponse.getIdentity());
-            return new NiFiAuthenticationToken(new NiFiUserDetails(
-                    new StandardNiFiUser.Builder()
-                            .identity(mappedIdentity)
-                            .groups(getUserGroups(mappedIdentity))
-                            .clientAddress(request.getClientAddress())
-                            .build()));
-        } else {
-            // build the entire proxy chain if applicable - <end-user><proxy1><proxy2>
-            final List<String> proxyChain = new ArrayList<>(ProxiedEntitiesUtils.tokenizeProxiedEntitiesChain(request.getProxiedEntitiesChain()));
-            proxyChain.add(authenticationResponse.getIdentity());
-
-            // add the chain as appropriate to each proxy
-            NiFiUser proxy = null;
-            for (final ListIterator<String> chainIter = proxyChain.listIterator(proxyChain.size()); chainIter.hasPrevious(); ) {
-                String identity = chainIter.previous();
-
-                // determine if the user is anonymous
-                final boolean isAnonymous = StringUtils.isBlank(identity);
-                if (isAnonymous) {
-                    identity = StandardNiFiUser.ANONYMOUS_IDENTITY;
-                } else {
-                    identity = mapIdentity(identity);
-                }
-
-                final Set<String> groups = getUserGroups(identity);
-
-                // Only set the client address for client making the request because we don't know the clientAddress of the proxied entities
-                String clientAddress = (proxy == null) ? request.getClientAddress() : null;
-                proxy = createUser(identity, groups, proxy, clientAddress, isAnonymous);
-
-                if (chainIter.hasPrevious()) {
-                    try {
-                        PROXY_AUTHORIZABLE.authorize(authorizer, RequestAction.WRITE, proxy);
-                    } catch (final AccessDeniedException e) {
-                        throw new UntrustedProxyException(String.format("Untrusted proxy %s", identity));
-                    }
-                }
-            }
-
-            return new NiFiAuthenticationToken(new NiFiUserDetails(proxy));
-        }
-    }
-
-    /**
-     * Returns a regular user populated with the provided values, or if the user should be anonymous, a well-formed instance of the anonymous user with the provided values.
-     *
-     * @param identity      the user's identity
-     * @param chain         the proxied entities
-     * @param clientAddress the requesting IP address
-     * @param isAnonymous   if true, an anonymous user will be returned (identity will be ignored)
-     * @return the populated user
-     */
-    protected static NiFiUser createUser(String identity, Set<String> groups, NiFiUser chain, String clientAddress, boolean isAnonymous) {
-        if (isAnonymous) {
-            return StandardNiFiUser.populateAnonymousUser(chain, clientAddress);
-        } else {
-            return new StandardNiFiUser.Builder().identity(identity).groups(groups).chain(chain).clientAddress(clientAddress).build();
-        }
-    }
-
-    private Map<String, String> getUserContext(final X509AuthenticationRequestToken request) {
-        final Map<String, String> userContext;
-        if (!StringUtils.isBlank(request.getClientAddress())) {
-            userContext = new HashMap<>();
-            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), request.getClientAddress());
-        } else {
-            userContext = null;
-        }
-        return userContext;
-    }
-
-    @Override
-    public boolean supports(Class<?> authentication) {
-        return X509AuthenticationRequestToken.class.isAssignableFrom(authentication);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java
deleted file mode 100644
index d5aca23..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.x509;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationRequestToken;
-import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
-
-import java.security.cert.X509Certificate;
-
-/**
- * This is an authentication request with a given JWT token.
- */
-public class X509AuthenticationRequestToken extends NiFiAuthenticationRequestToken {
-
-    private final String proxiedEntitiesChain;
-    private final X509PrincipalExtractor principalExtractor;
-    private final X509Certificate[] certificates;
-
-    /**
-     * Creates a representation of the jwt authentication request for a user.
-     *
-     * @param proxiedEntitiesChain   The http servlet request
-     * @param certificates  The certificate chain
-     */
-    public X509AuthenticationRequestToken(final String proxiedEntitiesChain, final X509PrincipalExtractor principalExtractor, final X509Certificate[] certificates, final String clientAddress) {
-        super(clientAddress);
-        setAuthenticated(false);
-        this.proxiedEntitiesChain = proxiedEntitiesChain;
-        this.principalExtractor = principalExtractor;
-        this.certificates = certificates;
-    }
-
-    @Override
-    public Object getCredentials() {
-        return null;
-    }
-
-    @Override
-    public Object getPrincipal() {
-        if (StringUtils.isBlank(proxiedEntitiesChain)) {
-            return principalExtractor.extractPrincipal(certificates[0]);
-        } else {
-            return String.format("%s<%s>", proxiedEntitiesChain, principalExtractor.extractPrincipal(certificates[0]));
-        }
-    }
-
-    public String getProxiedEntitiesChain() {
-        return proxiedEntitiesChain;
-    }
-
-    public X509Certificate[] getCertificates() {
-        return certificates;
-    }
-
-    @Override
-    public String toString() {
-        return getName();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509CertificateValidator.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509CertificateValidator.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509CertificateValidator.java
deleted file mode 100644
index d748b93..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509CertificateValidator.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.x509;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.stereotype.Component;
-
-import java.security.cert.CertificateExpiredException;
-import java.security.cert.CertificateNotYetValidException;
-import java.security.cert.X509Certificate;
-
-/**
- * Extracts client certificates from Http requests.
- */
-@Component
-public class X509CertificateValidator {
-
-    private final Logger logger = LoggerFactory.getLogger(getClass());
-
-    /**
-     * Extract the client certificate from the specified HttpServletRequest or null if none is specified.
-     *
-     * @param certificates the client certificates
-     * @throws CertificateExpiredException cert is expired
-     * @throws CertificateNotYetValidException cert is not yet valid
-     */
-    public void validateClientCertificate(final X509Certificate[] certificates)
-            throws CertificateExpiredException, CertificateNotYetValidException {
-
-        // ensure the cert is valid
-        certificates[0].checkValidity();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityAuthenticationProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityAuthenticationProvider.java
new file mode 100644
index 0000000..d4be5e9
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityAuthenticationProvider.java
@@ -0,0 +1,131 @@
+/*
+ * 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.registry.web.security.authentication.x509;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.apache.nifi.registry.web.security.authentication.AuthenticationRequestToken;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.web.security.authentication.IdentityAuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.AuthenticationSuccessToken;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.Resource;
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.resource.ResourceFactory;
+import org.apache.nifi.registry.security.authorization.user.NiFiUser;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
+import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
+import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
+import org.apache.nifi.registry.web.security.authentication.exception.UntrustedProxyException;
+
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Set;
+
+public class X509IdentityAuthenticationProvider extends IdentityAuthenticationProvider {
+
+    private static final Authorizable PROXY_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getProxyResource();
+        }
+    };
+
+    public X509IdentityAuthenticationProvider(NiFiRegistryProperties properties, Authorizer authorizer, IdentityProvider identityProvider) {
+        super(properties, authorizer, identityProvider);
+    }
+
+    @Override
+    protected AuthenticationSuccessToken buildAuthenticatedToken(
+            AuthenticationRequestToken requestToken,
+            AuthenticationResponse response) {
+
+        AuthenticationRequest authenticationRequest = requestToken.getAuthenticationRequest();
+
+        String proxiedEntitiesChain = authenticationRequest.getDetails() != null
+                ? (String)authenticationRequest.getDetails()
+                : null;
+
+        if (StringUtils.isBlank(proxiedEntitiesChain)) {
+            return super.buildAuthenticatedToken(requestToken, response);
+        }
+
+        // build the entire proxy chain if applicable - <end-user><proxy1><proxy2>
+        final List<String> proxyChain = ProxiedEntitiesUtils.tokenizeProxiedEntitiesChain(proxiedEntitiesChain);
+        proxyChain.add(response.getIdentity());
+
+        // add the chain as appropriate to each proxy
+        NiFiUser proxy = null;
+        for (final ListIterator<String> chainIter = proxyChain.listIterator(proxyChain.size()); chainIter.hasPrevious(); ) {
+            String identity = chainIter.previous();
+
+            // determine if the user is anonymous
+            final boolean isAnonymous = StringUtils.isBlank(identity);
+            if (isAnonymous) {
+                identity = StandardNiFiUser.ANONYMOUS_IDENTITY;
+            } else {
+                identity = mapIdentity(identity);
+            }
+
+            final Set<String> groups = getUserGroups(identity);
+
+            // Only set the client address for client making the request because we don't know the clientAddress of the proxied entities
+            String clientAddress = (proxy == null) ? requestToken.getClientAddress() : null;
+            proxy = createUser(identity, groups, proxy, clientAddress, isAnonymous);
+
+            if (chainIter.hasPrevious()) {
+                try {
+                    PROXY_AUTHORIZABLE.authorize(authorizer, RequestAction.WRITE, proxy);
+                } catch (final AccessDeniedException e) {
+                    throw new UntrustedProxyException(String.format("Untrusted proxy %s", identity));
+                }
+            }
+        }
+
+        return new AuthenticationSuccessToken(new NiFiUserDetails(proxy));
+
+    }
+
+    /**
+     * Returns a regular user populated with the provided values, or if the user should be anonymous, a well-formed instance of the anonymous user with the provided values.
+     *
+     * @param identity      the user's identity
+     * @param chain         the proxied entities
+     * @param clientAddress the requesting IP address
+     * @param isAnonymous   if true, an anonymous user will be returned (identity will be ignored)
+     * @return the populated user
+     */
+    private static NiFiUser createUser(String identity, Set<String> groups, NiFiUser chain, String clientAddress, boolean isAnonymous) {
+        if (isAnonymous) {
+            return StandardNiFiUser.populateAnonymousUser(chain, clientAddress);
+        } else {
+            return new StandardNiFiUser.Builder().identity(identity).groups(groups).chain(chain).clientAddress(clientAddress).build();
+        }
+    }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityProvider.java
index 692b318..9631efc 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityProvider.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityProvider.java
@@ -16,13 +16,22 @@
  */
 package org.apache.nifi.registry.web.security.authentication.x509;
 
-import org.apache.nifi.registry.web.response.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProviderConfigurationContext;
+import org.apache.nifi.registry.security.authentication.IdentityProviderUsage;
+import org.apache.nifi.registry.security.authentication.exception.InvalidCredentialsException;
+import org.apache.nifi.registry.security.exception.SecurityProviderCreationException;
+import org.apache.nifi.registry.security.exception.SecurityProviderDestructionException;
+import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
 import org.springframework.stereotype.Component;
 
+import javax.servlet.http.HttpServletRequest;
 import java.security.cert.CertificateExpiredException;
 import java.security.cert.CertificateNotYetValidException;
 import java.security.cert.X509Certificate;
@@ -32,73 +41,126 @@ import java.util.concurrent.TimeUnit;
  * Identity provider for extract the authenticating a ServletRequest with a X509Certificate.
  */
 @Component
-public class X509IdentityProvider {
+public class X509IdentityProvider implements IdentityProvider {
 
     private static final Logger logger = LoggerFactory.getLogger(X509IdentityProvider.class);
 
-    private final String issuer = getClass().getSimpleName();
+    private static final String issuer = X509IdentityProvider.class.getSimpleName();
+
+    private static final long expiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
+
+    private static final IdentityProviderUsage usage = new IdentityProviderUsage() {
+        @Override
+        public String getText() {
+            return "The client must connect over HTTPS and must provide a client certificate during the TLS handshake. " +
+                    "Additionally, the client may declare itself a proxy for another user identity by populating the " +
+                    ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN + " HTTP header field with a value of the format " +
+                    "'<end-user-identity><proxy1-identity><proxy2-identity>...<proxyN-identity>'" +
+                    "for all identities in the chain prior to this client. If the " + ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN +
+                    " header is present in the request, this client's identity will be extracted from the client certificate " +
+                    "used for TLS and added to the end of the chain, and then the entire chain will be authorized. Each proxy " +
+                    "will be authorized to have 'write' access to '/proxy', and the originating user identity will be " +
+                    "authorized for access to the resource being accessed in the request.";
+        }
+    };
 
-    private X509CertificateValidator certificateValidator;
     private X509PrincipalExtractor principalExtractor;
+    private X509CertificateExtractor certificateExtractor;
 
     @Autowired
-    public X509IdentityProvider(X509CertificateValidator certificateValidator, X509PrincipalExtractor principalExtractor) {
-        this.certificateValidator = certificateValidator;
+    public X509IdentityProvider(X509PrincipalExtractor principalExtractor, X509CertificateExtractor certificateExtractor) {
         this.principalExtractor = principalExtractor;
+        this.certificateExtractor = certificateExtractor;
+    }
+
+    @Override
+    public IdentityProviderUsage getUsageInstructions() {
+        return usage;
     }
 
     /**
-     * Authenticates the specified request by checking certificate validity.
+     * Extracts certificate-based credentials from an {@link HttpServletRequest}.
+     *
+     * The resulting {@link AuthenticationRequest} will be populated as:
+     *  - username: principal DN from first client cert
+     *  - credentials: first client certificate (X509Certificate)
+     *  - details: proxied-entities chain (String)
      *
-     * @param certificates the client certificates
-     * @return an authentication response
-     * @throws IllegalArgumentException the request did not contain a valid certificate (or no certificate)
+     * @param servletRequest the {@link HttpServletRequest} request that may contain credentials understood by this IdentityProvider
+     * @return a populated AuthenticationRequest or null if the credentials could not be found.
      */
-    public AuthenticationResponse authenticate(final X509Certificate[] certificates) throws IllegalArgumentException {
-        // ensure the cert was found
+    @Override
+    public AuthenticationRequest extractCredentials(HttpServletRequest servletRequest) {
+
+        // only support x509 login when running securely
+        if (!servletRequest.isSecure()) {
+            return null;
+        }
+
+        // look for a client certificate
+        final X509Certificate[] certificates = certificateExtractor.extractClientCertificate(servletRequest);
         if (certificates == null || certificates.length == 0) {
-            throw new IllegalArgumentException("The specified request does not contain a client certificate.");
+            return null;
         }
 
         // extract the principal
         final Object certificatePrincipal = principalExtractor.extractPrincipal(certificates[0]);
         final String principal = certificatePrincipal.toString();
 
+        // extract the proxiedEntitiesChain header value from the servletRequest
+        String proxiedEntitiesChainHeader = servletRequest.getHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN);
+
+        return new AuthenticationRequest(principal, certificates[0], proxiedEntitiesChainHeader);
+
+    }
+
+    /**
+     * For a given {@link AuthenticationRequest}, this validates the client certificate and creates a populated {@link AuthenticationResponse}.
+     *
+     * The {@link AuthenticationRequest} authenticationRequest paramenter is expected to be populated as:
+     *  - username: principal DN from first client cert
+     *  - credentials: first client certificate (X509Certificate)
+     *  - details: proxied-entities chain (String)
+     *
+     * @param authenticationRequest the request, containing identity claim credentials for the IdentityProvider to authenticate and determine an identity
+     */
+    @Override
+    public AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest) throws InvalidCredentialsException {
+
+        if (authenticationRequest == null || authenticationRequest.getUsername() == null) {
+            return null;
+        }
+
+        String principal = authenticationRequest.getUsername();
+
         try {
-            certificateValidator.validateClientCertificate(certificates);
+            X509Certificate clientCertificate = (X509Certificate)authenticationRequest.getCredentials();
+            validateClientCertificate(clientCertificate);
         } 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);
-            }
-            throw new IllegalArgumentException(message, cee);
+            logger.warn(message, cee);
+            throw new InvalidCredentialsException(message, cee);
         } 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);
-            }
-            throw new IllegalArgumentException(message, cnyve);
+            logger.warn(message, cnyve);
+            throw new InvalidCredentialsException(message, cnyve);
         } catch (final Exception e) {
-            logger.info(e.getMessage());
-            if (logger.isDebugEnabled()) {
-                logger.debug("", e);
-            }
-            throw new IllegalArgumentException(e.getMessage(), e);
+            logger.warn(e.getMessage(), e);
         }
 
         // build the authentication response
-        return new AuthenticationResponse(principal, principal, TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS), issuer);
+        return new AuthenticationResponse(principal, principal, expiration, issuer);
     }
 
-    /* setters */
-    public void setCertificateValidator(X509CertificateValidator certificateValidator) {
-        this.certificateValidator = certificateValidator;
-    }
+    @Override
+    public void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {}
 
-    public void setPrincipalExtractor(X509PrincipalExtractor principalExtractor) {
-        this.principalExtractor = principalExtractor;
+    @Override
+    public void preDestruction() throws SecurityProviderDestructionException {}
+
+
+    private void validateClientCertificate(X509Certificate certificate) throws CertificateExpiredException, CertificateNotYetValidException {
+        certificate.checkValidity();
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/xsd/identity-providers.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/xsd/identity-providers.xsd b/nifi-registry-web-api/src/main/xsd/identity-providers.xsd
deleted file mode 100644
index bcca014..0000000
--- a/nifi-registry-web-api/src/main/xsd/identity-providers.xsd
+++ /dev/null
@@ -1,50 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
-    <!-- role -->
-    <xs:complexType name="Provider">
-        <xs:sequence>
-            <xs:element name="identifier" type="NonEmptyStringType"/>
-            <xs:element name="class" type="NonEmptyStringType"/>
-            <xs:element name="property" type="Property" minOccurs="0" maxOccurs="unbounded" />
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Name/Value properties-->
-    <xs:complexType name="Property">
-        <xs:simpleContent>
-            <xs:extension base="xs:string">
-                <xs:attribute name="name" type="NonEmptyStringType"/>
-                <xs:attribute name="encryption" type="xs:string"/>
-            </xs:extension>
-        </xs:simpleContent>
-    </xs:complexType>
-
-    <xs:simpleType name="NonEmptyStringType">
-        <xs:restriction base="xs:string">
-            <xs:minLength value="1"/>
-        </xs:restriction>
-    </xs:simpleType>
-
-    <!-- login identity provider -->
-    <xs:element name="identityProviders">
-        <xs:complexType>
-            <xs:sequence>
-                <xs:element name="provider" type="Provider" minOccurs="0" maxOccurs="unbounded"/>
-            </xs:sequence>
-        </xs:complexType>
-    </xs:element>
-</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
index d6b94c2..bdd8e11 100644
--- a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
@@ -45,6 +45,7 @@ import javax.ws.rs.client.Entity;
 import javax.ws.rs.core.Form;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
@@ -71,20 +72,23 @@ import static org.junit.Assert.assertTrue;
 @Sql(executionPhase = Sql.ExecutionPhase.BEFORE_TEST_METHOD, scripts = "classpath:db/clearDB.sql")
 public class SecureLdapIT extends IntegrationTestBase {
 
+    private static final String tokenLoginPath = "access/token/login";
+    private static final String tokenIdentityProviderPath = "access/token/identity-provider";
+
     @TestConfiguration
     @Profile("ITSecureLdap")
     public static class LdapTestConfiguration {
 
-        static AuthorizerFactory af;
+        static AuthorizerFactory authorizerFactory;
 
         @Primary
         @Bean
         @DependsOn({"directoryServer"}) // Can't load LdapUserGroupProvider until the embedded LDAP server, which creates the "directoryServer" bean, is running
         public static Authorizer getAuthorizer(@Autowired NiFiRegistryProperties properties, ExtensionManager extensionManager) {
-            if (af == null) {
-                af = new AuthorizerFactory(properties, extensionManager);
+            if (authorizerFactory == null) {
+                authorizerFactory = new AuthorizerFactory(properties, extensionManager);
             }
-            return af.getAuthorizer();
+            return authorizerFactory.getAuthorizer();
         }
 
     }
@@ -93,11 +97,9 @@ public class SecureLdapIT extends IntegrationTestBase {
 
     @Before
     public void generateAuthToken() {
-        final Form form = new Form()
-                .param("username", "nifiadmin")
-                .param("password", "password");
+        final Form form = encodeCredentialsForURLFormParams("nifiadmin", "password");
         final String token = client
-                .target(createURL("access/token"))
+                .target(createURL(tokenLoginPath))
                 .request()
                 .post(Entity.form(form), String.class);
         adminAuthToken = token;
@@ -121,12 +123,10 @@ public class SecureLdapIT extends IntegrationTestBase {
                 "\"status\":\"ACTIVE\"" +
                 "}";
 
-        // When: the /access/token endpoint is queried
-        final Form form = new Form()
-                .param("username", "nobel")
-                .param("password", "password");
+        // When: the /access/token/login endpoint is queried
+        final Form form = encodeCredentialsForURLFormParams("nobel", "password");
         final Response tokenResponse = client
-                .target(createURL("access/token"))
+                .target(createURL(tokenLoginPath))
                 .request()
                 .post(Entity.form(form), Response.class);
 
@@ -154,6 +154,52 @@ public class SecureLdapIT extends IntegrationTestBase {
     }
 
     @Test
+    public void testTokenGenerationWithIdentityProvider() throws Exception {
+
+        // Given: the client and server have been configured correctly for LDAP authentication
+        String expectedJwtPayloadJson = "{" +
+                "\"sub\":\"nobel\"," +
+                "\"preferred_username\":\"nobel\"," +
+                "\"iss\":\"LdapIdentityProvider\"," +
+                "\"aud\":\"LdapIdentityProvider\"" +
+                "}";
+        String expectedAccessStatusJson = "{" +
+                "\"identity\":\"nobel\"," +
+                "\"status\":\"ACTIVE\"" +
+                "}";
+
+        // When: the /access/token/identity-provider endpoint is queried
+        final String basicAuthCredentials = encodeCredentialsForBasicAuth("nobel", "password");
+        final Response tokenResponse = client
+                .target(createURL(tokenIdentityProviderPath))
+                .request()
+                .header("Authorization", "Basic " + basicAuthCredentials)
+                .post(null, Response.class);
+
+        // Then: the server returns 200 OK with an access token
+        assertEquals(201, tokenResponse.getStatus());
+        String token = tokenResponse.readEntity(String.class);
+        assertTrue(StringUtils.isNotEmpty(token));
+        String[] jwtParts = token.split("\\.");
+        assertEquals(3, jwtParts.length);
+        String jwtPayload = new String(Base64.decodeBase64(jwtParts[1]), "UTF-8");
+        JSONAssert.assertEquals(expectedJwtPayloadJson, jwtPayload, false);
+
+        // When: the token is returned in the Authorization header
+        final Response accessResponse = client
+                .target(createURL("access"))
+                .request()
+                .header("Authorization", "Bearer " + token)
+                .get(Response.class);
+
+        // Then: the server acknowledges the client has access
+        assertEquals(200, accessResponse.getStatus());
+        String accessStatus = accessResponse.readEntity(String.class);
+        JSONAssert.assertEquals(expectedAccessStatusJson, accessStatus, false);
+
+    }
+
+    @Test
     public void testUsers() throws Exception {
 
         // Given: the client and server have been configured correctly for LDAP authentication
@@ -240,15 +286,13 @@ public class SecureLdapIT extends IntegrationTestBase {
         // Given: the server has been configured with an initial admin "nifiadmin" and a user with no accessPolicies "nobel"
         String nobelId = getTenantIdentifierByIdentity("nobel");
         String chemistsId = getTenantIdentifierByIdentity("chemists"); // a group containing user "nobel"
-        final Form form = new Form()
-                .param("username", "nobel")
-                .param("password", "password");
+
+        final Form form = encodeCredentialsForURLFormParams("nobel", "password");
         final String nobelAuthToken = client
-                .target(createURL("access/token"))
+                .target(createURL(tokenLoginPath))
                 .request()
                 .post(Entity.form(form), String.class);
 
-
         // When: nifiadmin creates a bucket
         final Bucket bucket = new Bucket();
         bucket.setName("Integration Test Bucket");
@@ -382,4 +426,15 @@ public class SecureLdapIT extends IntegrationTestBase {
         return matchedTenant != null ? matchedTenant.getIdentifier() : null;
     }
 
+    private static Form encodeCredentialsForURLFormParams(String username, String password) {
+        return new Form()
+                .param("username", username)
+                .param("password", password);
+    }
+
+    private static String encodeCredentialsForBasicAuth(String username, String password) {
+        final String credentials = username + ":" + password;
+        final String base64credentials =  new String(java.util.Base64.getEncoder().encode(credentials.getBytes(Charset.forName("UTF-8"))));
+        return base64credentials;
+    }
 }


[3/4] nifi-registry git commit: NIFIREG-45 Refactor LoginIdentityProvider interface as a more generic and flexible IdentityProvider interface

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProvider.java
deleted file mode 100644
index b74069a..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProvider.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication;
-
-import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
-import org.apache.nifi.registry.security.authentication.exception.InvalidLoginCredentialsException;
-import org.apache.nifi.registry.security.authentication.exception.ProviderCreationException;
-import org.apache.nifi.registry.security.authentication.exception.ProviderDestructionException;
-
-/**
- * Identity provider that is able to authentication a user with username/password credentials.
- */
-public interface LoginIdentityProvider {
-
-    /**
-     * Authenticates the specified login credentials.
-     *
-     * @param credentials the credentials
-     * @return The authentication response
-     * @throws InvalidLoginCredentialsException The login credentials were invalid
-     * @throws IdentityAccessException Unable to register the user due to an issue accessing the underlying storage
-     */
-    AuthenticationResponse authenticate(LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException;
-
-    /**
-     * Called immediately after instance creation for implementers to perform additional setup
-     *
-     * @param initializationContext in which to initialize
-     * @throws ProviderCreationException Unable to initialize
-     */
-    void initialize(LoginIdentityProviderInitializationContext initializationContext) throws ProviderCreationException;
-
-    /**
-     * Called to configure the AuthorityProvider.
-     *
-     * @param configurationContext at the time of configuration
-     * @throws ProviderCreationException for any issues configuring the provider
-     */
-    void onConfigured(LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException;
-
-    /**
-     * Called immediately before instance destruction for implementers to release resources.
-     *
-     * @throws ProviderDestructionException If pre-destruction fails.
-     */
-    void preDestruction() throws ProviderDestructionException;
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderConfigurationContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderConfigurationContext.java
deleted file mode 100644
index a7f21be..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderConfigurationContext.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication;
-
-import java.util.Map;
-
-/**
- *
- */
-public interface LoginIdentityProviderConfigurationContext {
-
-    /**
-     * @return identifier for the authority provider
-     */
-    String getIdentifier();
-
-    /**
-     * Retrieves all properties the component currently understands regardless
-     * of whether a value has been set for them or not. If no value is present
-     * then its value is null and thus any registered default for the property
-     * descriptor applies.
-     *
-     * @return Map of all properties
-     */
-    Map<String, String> getProperties();
-
-    /**
-     * @param property to lookup the descriptor and value of
-     * @return the value the component currently understands for the given
-     * PropertyDescriptor. This method does not substitute default
-     * PropertyDescriptor values, so the value returned will be null if not set
-     */
-    String getProperty(String property);
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderInitializationContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderInitializationContext.java
deleted file mode 100644
index 755c2e8..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderInitializationContext.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication;
-
-/**
- *
- */
-public interface LoginIdentityProviderInitializationContext {
-
-    public String getIdentifier();
-
-    public LoginIdentityProviderLookup getAuthorityProviderLookup();
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderLookup.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderLookup.java
deleted file mode 100644
index 8720bba..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginIdentityProviderLookup.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication;
-
-public interface LoginIdentityProviderLookup {
-
-    LoginIdentityProvider getLoginIdentityProvider(String identifier);
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/UsernamePasswordAuthenticationRequest.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/UsernamePasswordAuthenticationRequest.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/UsernamePasswordAuthenticationRequest.java
new file mode 100644
index 0000000..3abcf94
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/UsernamePasswordAuthenticationRequest.java
@@ -0,0 +1,25 @@
+/*
+ * 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.registry.security.authentication;
+
+public class UsernamePasswordAuthenticationRequest extends AuthenticationRequest {
+
+    public UsernamePasswordAuthenticationRequest(String username, String password) {
+        super(username, password, null);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/IdentityProviderContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/IdentityProviderContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/IdentityProviderContext.java
new file mode 100644
index 0000000..8d0ddf0
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/IdentityProviderContext.java
@@ -0,0 +1,31 @@
+/*
+ * 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.registry.security.authentication.annotation;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Documented
+@Target({ElementType.FIELD, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@Inherited
+public @interface IdentityProviderContext {
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/LoginIdentityProviderContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/LoginIdentityProviderContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/LoginIdentityProviderContext.java
deleted file mode 100644
index 1c4d17f..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/annotation/LoginIdentityProviderContext.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication.annotation;
-
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Inherited;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- *
- *
- */
-@Documented
-@Target({ElementType.FIELD, ElementType.METHOD})
-@Retention(RetentionPolicy.RUNTIME)
-@Inherited
-public @interface LoginIdentityProviderContext {
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidCredentialsException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidCredentialsException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidCredentialsException.java
new file mode 100644
index 0000000..e7c7339
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidCredentialsException.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.authentication.exception;
+
+/**
+ * Represents the case when the identity could not be confirmed because the
+ * identity claim credentials were invalid.
+ */
+public class InvalidCredentialsException extends RuntimeException {
+
+    public InvalidCredentialsException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public InvalidCredentialsException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidLoginCredentialsException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidLoginCredentialsException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidLoginCredentialsException.java
deleted file mode 100644
index c432857..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/InvalidLoginCredentialsException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication.exception;
-
-/**
- * Represents the case when the identity could not be confirmed because the
- * login credentials were invalid.
- */
-public class InvalidLoginCredentialsException extends RuntimeException {
-
-    public InvalidLoginCredentialsException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public InvalidLoginCredentialsException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderCreationException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderCreationException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderCreationException.java
deleted file mode 100644
index 12844ce..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderCreationException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication.exception;
-
-/**
- * Represents the exceptional case when an AuthorityProvider fails instantiated.
- *
- */
-public class ProviderCreationException extends RuntimeException {
-
-    public ProviderCreationException() {
-    }
-
-    public ProviderCreationException(String msg) {
-        super(msg);
-    }
-
-    public ProviderCreationException(Throwable cause) {
-        super(cause);
-    }
-
-    public ProviderCreationException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderDestructionException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderDestructionException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderDestructionException.java
deleted file mode 100644
index 8a0157b..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/exception/ProviderDestructionException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication.exception;
-
-/**
- * Represents the exceptional case when an AuthorityProvider fails destruction.
- *
- */
-public class ProviderDestructionException extends RuntimeException {
-
-    public ProviderDestructionException() {
-    }
-
-    public ProviderDestructionException(String msg) {
-        super(msg);
-    }
-
-    public ProviderDestructionException(Throwable cause) {
-        super(cause);
-    }
-
-    public ProviderDestructionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerCreationException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerCreationException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerCreationException.java
index 4d34cf5..dbe90e7 100644
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerCreationException.java
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerCreationException.java
@@ -18,8 +18,9 @@ package org.apache.nifi.registry.security.authorization.exception;
 
 /**
  * Represents the exceptional case when an Authorizer fails instantiation.
- *
+ * @deprecated Migrate to use SecurityProviderCreationException
  */
+@Deprecated // TODO migrate all usages to SecurityProviderCreationException
 public class AuthorizerCreationException extends RuntimeException {
 
     public AuthorizerCreationException() {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerDestructionException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerDestructionException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerDestructionException.java
index fbec6d3..6892445 100644
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerDestructionException.java
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AuthorizerDestructionException.java
@@ -18,8 +18,9 @@ package org.apache.nifi.registry.security.authorization.exception;
 
 /**
  * Represents the exceptional case when an Authorizer fails destruction.
- *
+ * @deprecated Migrate to use SecurityProviderDestructionException
  */
+@Deprecated // TODO migrate all usages to SecurityProviderDestructionException
 public class AuthorizerDestructionException extends RuntimeException {
 
     public AuthorizerDestructionException() {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderCreationException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderCreationException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderCreationException.java
new file mode 100644
index 0000000..01531d6
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderCreationException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.registry.security.exception;
+
+/**
+ * Represents the exceptional case when a security api provider fails instantiation.
+ */
+public class SecurityProviderCreationException extends RuntimeException {
+
+    public SecurityProviderCreationException() {
+    }
+
+    public SecurityProviderCreationException(String msg) {
+        super(msg);
+    }
+
+    public SecurityProviderCreationException(Throwable cause) {
+        super(cause);
+    }
+
+    public SecurityProviderCreationException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderDestructionException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderDestructionException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderDestructionException.java
new file mode 100644
index 0000000..3370623
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/exception/SecurityProviderDestructionException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.registry.security.exception;
+
+/**
+ * Represents the exceptional case when a security api provider fails destruction.
+ */
+public class SecurityProviderDestructionException extends RuntimeException {
+
+    public SecurityProviderDestructionException() {
+    }
+
+    public SecurityProviderDestructionException(String msg) {
+        super(msg);
+    }
+
+    public SecurityProviderDestructionException(Throwable cause) {
+        super(cause);
+    }
+
+    public SecurityProviderDestructionException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/ProxiedEntitiesUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/ProxiedEntitiesUtils.java b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/ProxiedEntitiesUtils.java
new file mode 100644
index 0000000..f850341
--- /dev/null
+++ b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/ProxiedEntitiesUtils.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.registry.security.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class ProxiedEntitiesUtils {
+    private static final Logger logger = LoggerFactory.getLogger(ProxiedEntitiesUtils.class);
+
+    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 static final String GT = ">";
+    private static final String ESCAPED_GT = "\\\\>";
+    private static final String LT = "<";
+    private static final String ESCAPED_LT = "\\\\<";
+
+    private static final String ANONYMOUS_CHAIN = "<>";
+
+    /**
+     * Formats the specified DN to be set as a HTTP header using well known conventions.
+     *
+     * @param dn raw dn
+     * @return the dn formatted as an HTTP header
+     */
+    public static String formatProxyDn(String dn) {
+        return LT + sanitizeDn(dn) + GT;
+    }
+
+    /**
+     * If a user provides a DN with the sequence '><', they could escape the tokenization process and impersonate another user.
+     * <p>
+     * Example:
+     * <p>
+     * Provided DN: {@code jdoe><alopresto} -> {@code <jdoe><alopresto><proxy...>} would allow the user to impersonate jdoe
+     *
+     * @param rawDn the unsanitized DN
+     * @return the sanitized DN
+     */
+    private static String sanitizeDn(String rawDn) {
+        if (StringUtils.isEmpty(rawDn)) {
+            return rawDn;
+        } else {
+            String sanitizedDn = rawDn.replaceAll(GT, ESCAPED_GT).replaceAll(LT, ESCAPED_LT);
+            if (!sanitizedDn.equals(rawDn)) {
+                logger.warn("The provided DN [" + rawDn + "] contained dangerous characters that were escaped to [" + sanitizedDn + "]");
+            }
+            return sanitizedDn;
+        }
+    }
+
+    /**
+     * Reconstitutes the original DN from the sanitized version passed in the proxy chain.
+     * <p>
+     * Example:
+     * <p>
+     * {@code alopresto\>\<proxy1} -> {@code alopresto><proxy1}
+     *
+     * @param sanitizedDn the sanitized DN
+     * @return the original DN
+     */
+    private static String unsanitizeDn(String sanitizedDn) {
+        if (StringUtils.isEmpty(sanitizedDn)) {
+            return sanitizedDn;
+        } else {
+            String unsanitizedDn = sanitizedDn.replaceAll(ESCAPED_GT, GT).replaceAll(ESCAPED_LT, LT);
+            if (!unsanitizedDn.equals(sanitizedDn)) {
+                logger.warn("The provided DN [" + sanitizedDn + "] had been escaped, and was reconstituted to the dangerous DN [" + unsanitizedDn + "]");
+            }
+            return unsanitizedDn;
+        }
+    }
+
+    /**
+     * Tokenizes the specified proxy chain.
+     *
+     * @param rawProxyChain raw chain
+     * @return tokenized proxy chain
+     */
+    public static List<String> tokenizeProxiedEntitiesChain(String rawProxyChain) {
+        final List<String> proxyChain = new ArrayList<>();
+        if (!StringUtils.isEmpty(rawProxyChain)) {
+            // Split the String on the >< token
+            List<String> elements = Arrays.asList(StringUtils.splitByWholeSeparatorPreserveAllTokens(rawProxyChain, "><"));
+
+            // Unsanitize each DN and collect back
+            elements = elements.stream().map(ProxiedEntitiesUtils::unsanitizeDn).collect(Collectors.toList());
+
+            // Remove the leading < from the first element
+            elements.set(0, elements.get(0).replaceFirst(LT, ""));
+
+            // Remove the trailing > from the last element
+            int last = elements.size() - 1;
+            String lastElement = elements.get(last);
+            if (lastElement.endsWith(GT)) {
+                elements.set(last, lastElement.substring(0, lastElement.length() - 1));
+            }
+
+            proxyChain.addAll(elements);
+        }
+
+        return proxyChain;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/pom.xml b/nifi-registry-web-api/pom.xml
index ab7dd84..203b58d 100644
--- a/nifi-registry-web-api/pom.xml
+++ b/nifi-registry-web-api/pom.xml
@@ -31,9 +31,6 @@
             <resource>
                 <directory>src/main/resources</directory>
             </resource>
-            <resource>
-                <directory>src/main/xsd</directory>
-            </resource>
         </resources>
         <plugins>
             <plugin>
@@ -48,28 +45,6 @@
                 </configuration>
             </plugin>
             <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>jaxb2-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>identity-providers</id>
-                        <goals>
-                            <goal>xjc</goal>
-                        </goals>
-                        <configuration>
-                            <packageName>org.apache.nifi.registry.security.authentication.generated</packageName>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <configuration>
-                    <excludes>**/authentication/generated/*.java,</excludes>
-                </configuration>
-            </plugin>
-            <plugin>
                 <groupId>com.github.kongchen</groupId>
                 <artifactId>swagger-maven-plugin</artifactId>
                 <version>3.1.5</version>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessResource.java
index 2732d5a..84693b1 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessResource.java
@@ -23,16 +23,16 @@ import io.swagger.annotations.ApiResponses;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.exception.AdministrationException;
 import org.apache.nifi.registry.model.authorization.AccessStatus;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
 import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
-import org.apache.nifi.registry.security.authentication.LoginCredentials;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.security.authentication.UsernamePasswordAuthenticationRequest;
 import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
-import org.apache.nifi.registry.security.authentication.exception.InvalidLoginCredentialsException;
+import org.apache.nifi.registry.security.authentication.exception.InvalidCredentialsException;
 import org.apache.nifi.registry.security.authorization.user.NiFiUser;
 import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.service.AuthorizationService;
 import org.apache.nifi.registry.web.security.authentication.jwt.JwtService;
-import org.apache.nifi.registry.web.security.authentication.token.LoginAuthenticationToken;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -49,7 +49,6 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.net.URI;
-import java.util.concurrent.TimeUnit;
 
 @Component
 @Path("/access")
@@ -61,16 +60,16 @@ public class AccessResource extends ApplicationResource {
 
     private static final Logger logger = LoggerFactory.getLogger(AccessResource.class);
 
-    private LoginIdentityProvider loginIdentityProvider;
+    private IdentityProvider identityProvider;
     private JwtService jwtService;
 
     @Autowired
     public AccessResource(
             AuthorizationService authorizationService,
             JwtService jwtService,
-            LoginIdentityProvider loginIdentityProvider) {
+            IdentityProvider identityProvider) {
         this.jwtService = jwtService;
-        this.loginIdentityProvider = loginIdentityProvider;
+        this.identityProvider = identityProvider;
     }
 
     /**
@@ -114,17 +113,15 @@ public class AccessResource extends ApplicationResource {
     }
 
     /**
-     * Creates a token for accessing the REST API via username/password.
+     * Creates a token for accessing the REST API.
      *
      * @param httpServletRequest the servlet request
-     * @param username           the username
-     * @param password           the password
      * @return A JWT (string)
      */
     @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+    @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.TEXT_PLAIN)
-    @Path("/token")
+    @Path("/token/login")
     @ApiOperation(
             value = "Creates a token for accessing the REST API via username/password",
             notes = "The token returned is formatted as a JSON Web Token (JWT). The token is base64 encoded and comprised of three parts. The header, " +
@@ -136,9 +133,9 @@ public class AccessResource extends ApplicationResource {
             @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
             @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
             @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
-            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support username/password login."),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support login with username/password."),
             @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
-    public Response createAccessToken(
+    public Response createAccessTokenUsingFormLogin(
             @Context HttpServletRequest httpServletRequest,
             @FormParam("username") String username,
             @FormParam("password") String password) {
@@ -148,56 +145,182 @@ public class AccessResource extends ApplicationResource {
             throw new IllegalStateException("Access tokens are only issued over HTTPS");
         }
 
-        // if not configuration for login, don't consider credentials
-        if (loginIdentityProvider == null) {
+        // if not configured with custom identity provider, or if provider doesn't support username/password authentication, don't consider credentials
+        if (identityProvider == null || !identityProvider.supports(UsernamePasswordAuthenticationRequest.class)) {
             throw new IllegalStateException("Username/Password login not supported by this NiFi");
         }
 
-        final LoginAuthenticationToken loginAuthenticationToken;
-
         // ensure we have login credentials
         if (StringUtils.isBlank(username) || StringUtils.isBlank(password)) {
             throw new IllegalArgumentException("The username and password must be specified");
         }
 
+        final AuthenticationResponse authenticationResponse;
+
         try {
             // attempt to authenticate
-            final AuthenticationResponse authenticationResponse = loginIdentityProvider.authenticate(new LoginCredentials(username, password));
-            long expiration = validateTokenExpiration(authenticationResponse.getExpiration(), authenticationResponse.getIdentity());
+            AuthenticationRequest authenticationRequest = new UsernamePasswordAuthenticationRequest(username, password);
+            authenticationResponse = identityProvider.authenticate(authenticationRequest);
+        } catch (final InvalidCredentialsException ice) {
+            throw new IllegalArgumentException("The supplied client credentials are not valid.", ice);
+        } catch (final IdentityAccessException iae) {
+            throw new AdministrationException(iae.getMessage(), iae);
+        }
+
+        // generate JWT for response
+        final String token = jwtService.generateSignedToken(authenticationResponse);
 
-            // 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);
+        // build the response
+        final URI uri = URI.create(generateResourceUri("access", "token"));
+        return generateCreatedResponse(uri, token).build();
+    }
+
+    /**
+     * Creates a token for accessing the REST API using a custom identity provider configured using NiFi Registry extensions.
+     *
+     * @param httpServletRequest the servlet request
+     * @return A JWT (string)
+     */
+    @POST
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token/identity-provider")
+    @ApiOperation(
+            value = "Creates a token for accessing the REST API via a custom identity provider.",
+            notes = "The user credentials must be passed in a format understood by the custom identity provider, e.g., a third-party auth token in an HTTP header. " +
+                    "The exact format of the user credentials expected by the custom identity provider can be discovered by 'GET /token/identity-provider/usage'. " +
+                    "The token returned is formatted as a JSON Web Token (JWT). The token is base64 encoded and comprised of three parts. The header, " +
+                    "the body, and the signature. The expiration of the token is a contained within the body. The token can be used in the Authorization header " +
+                    "in the format 'Authorization: Bearer <token>'.",
+            response = String.class
+    )
+    @ApiResponses({
+            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
+            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
+            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support login with customized credentials."),
+            @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
+    public Response createAccessTokenUsingIdentityProviderCredentials(@Context HttpServletRequest httpServletRequest) {
+
+        // only support access tokens when communicating over HTTPS
+        if (!httpServletRequest.isSecure()) {
+            throw new IllegalStateException("Access tokens are only issued over HTTPS");
+        }
+
+        // if not configured with custom identity provider, don't consider credentials
+        if (identityProvider == null) {
+            throw new IllegalStateException("Custom login not supported by this NiFi Registry");
+        }
+
+        final AuthenticationResponse authenticationResponse;
+
+        try {
+            // attempt to authenticate
+            AuthenticationRequest authenticationRequest = identityProvider.extractCredentials(httpServletRequest);
+            authenticationResponse = identityProvider.authenticate(authenticationRequest);
+        } catch (final InvalidCredentialsException ice) {
+            throw new IllegalArgumentException("The supplied client credentials are not valid.", ice);
         } catch (final IdentityAccessException iae) {
             throw new AdministrationException(iae.getMessage(), iae);
         }
 
         // generate JWT for response
-        final String token = jwtService.generateSignedToken(loginAuthenticationToken);
+        final String token = jwtService.generateSignedToken(authenticationResponse);
 
         // build the response
         final URI uri = URI.create(generateResourceUri("access", "token"));
         return generateCreatedResponse(uri, token).build();
     }
 
-    private long validateTokenExpiration(long proposedTokenExpiration, String identity) {
-        final long maxExpiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
-        final long minExpiration = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
-
-        if (proposedTokenExpiration > maxExpiration) {
-            logger.warn(String.format("Max token expiration exceeded. Setting expiration to %s from %s for %s", maxExpiration,
-                    proposedTokenExpiration, identity));
-            proposedTokenExpiration = maxExpiration;
-        } else if (proposedTokenExpiration < minExpiration) {
-            logger.warn(String.format("Min token expiration not met. Setting expiration to %s from %s for %s", minExpiration,
-                    proposedTokenExpiration, identity));
-            proposedTokenExpiration = minExpiration;
+    /**
+     * Creates a token for accessing the REST API using a custom identity provider configured using NiFi Registry extensions.
+     *
+     * @param httpServletRequest the servlet request
+     * @return A JWT (string)
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token/identity-provider/usage")
+    @ApiOperation(
+            value = "Provides a description of how the currently configured identity provider expects credentials to be passed to POST /token/identity-provider",
+            response = String.class
+    )
+    @ApiResponses({
+            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support login with customized credentials."),
+            @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
+    public Response getIdentityProviderUsageInstructions(@Context HttpServletRequest httpServletRequest) {
+
+        // if not configuration for login, don't consider credentials
+        if (identityProvider == null) {
+            throw new IllegalStateException("Custom login not supported by this NiFi Registry");
+        }
+
+        Class ipClazz = identityProvider.getClass();
+        String identityProviderName = StringUtils.isNotEmpty(ipClazz.getSimpleName()) ? ipClazz.getSimpleName() : ipClazz.getName();
+
+        try {
+            String usageInstructions = "Usage Instructions for '" + identityProviderName + "': ";
+            usageInstructions += identityProvider.getUsageInstructions().getText();
+            return generateOkResponse(usageInstructions).build();
+
+        } catch (Exception e) {
+            // If, for any reason, this identity provider does not support getUsageInstructions(), e.g., throws NotImplemented Exception.
+            return Response.status(Response.Status.NOT_IMPLEMENTED)
+                    .entity("The currently configured identity provider, '" + identityProvider.getClass().getName() + "' does not provide usage instructions.")
+                    .build();
         }
 
-        return proposedTokenExpiration;
     }
 
+    /**
+     * Creates a token for accessing the REST API using a custom identity provider configured using NiFi Registry extensions.
+     *
+     * @param httpServletRequest the servlet request
+     * @return A JWT (string)
+     */
+    @POST
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token/identity-provider/test")
+    @ApiOperation(
+            value = "Tests the format of the credentials against this identity provider without preforming authentication on the credentials to validate them.",
+            notes = "The user credentials should be passed in a format understood by the custom identity provider as defined by 'GET /token/identity-provider/usage'.",
+            response = String.class
+    )
+    @ApiResponses({
+            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
+            @ApiResponse(code = 401, message = "The format of the credentials were not recognized by the currently configured identity provider."),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support login with customized credentials."),
+            @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
+    public Response testIdentityProviderRecognizesCredentialsFormat(@Context HttpServletRequest httpServletRequest) {
+
+        // only support access tokens when communicating over HTTPS
+        if (!httpServletRequest.isSecure()) {
+            throw new IllegalStateException("Access tokens are only issued over HTTPS");
+        }
+
+        // if not configured with custom identity provider, don't consider credentials
+        if (identityProvider == null) {
+            throw new IllegalStateException("Custom login not supported by this NiFi Registry");
+        }
 
+        final Class ipClazz = identityProvider.getClass();
+        final String identityProviderName = StringUtils.isNotEmpty(ipClazz.getSimpleName()) ? ipClazz.getSimpleName() : ipClazz.getName();
+
+        // attempt to extract client credentials without authenticating them
+        AuthenticationRequest authenticationRequest = identityProvider.extractCredentials(httpServletRequest);
+        if (authenticationRequest != null) {
+            final String successMessage = identityProviderName + " recognized the format of the credentials in the HTTP request.";
+            return generateOkResponse(successMessage).build();
+        }
+
+        return Response.status(Response.Status.UNAUTHORIZED)
+                .entity("The format of the credentials were not recognized by the currently configured identity provider " +
+                        "'" + identityProviderName + "'. See GET /token/identity-provider/usage for more information.")
+                .build();
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/request/IntegerParameter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/request/IntegerParameter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/request/IntegerParameter.java
deleted file mode 100644
index f5048b3..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/request/IntegerParameter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.request;
-
-/**
- * Class for parsing integer parameters and providing a user friendly error message.
- */
-public class IntegerParameter {
-
-    private static final String INVALID_INTEGER_MESSAGE = "Unable to parse '%s' as an integer value.";
-
-    private Integer integerValue;
-
-    public IntegerParameter(String rawIntegerValue) {
-        try {
-            integerValue = Integer.parseInt(rawIntegerValue);
-        } catch (NumberFormatException nfe) {
-            throw new IllegalArgumentException(String.format(INVALID_INTEGER_MESSAGE, rawIntegerValue));
-        }
-    }
-
-    public Integer getInteger() {
-        return integerValue;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/AuthenticationResponse.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/AuthenticationResponse.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/AuthenticationResponse.java
deleted file mode 100644
index a0b87b5..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/AuthenticationResponse.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.response;
-
-/**
- * Authentication response for a user login attempt.
- */
-public class AuthenticationResponse {
-
-    private final String identity;
-    private final String username;
-    private final long expiration;
-    private final String issuer;
-
-    /**
-     * Creates an authentication response. The username and how long the authentication is valid in milliseconds
-     *
-     * @param identity The user identity
-     * @param username The username
-     * @param expiration The expiration in milliseconds
-     * @param issuer The issuer of the token
-     */
-    public AuthenticationResponse(final String identity, final String username, final long expiration, final String issuer) {
-        this.identity = identity;
-        this.username = username;
-        this.expiration = expiration;
-        this.issuer = issuer;
-    }
-
-    public String getIdentity() {
-        return identity;
-    }
-
-    public String getUsername() {
-        return username;
-    }
-
-    public String getIssuer() {
-        return issuer;
-    }
-
-    /**
-     * Returns the expiration of a given authentication in milliseconds.
-     *
-     * @return The expiration in milliseconds
-     */
-    public long getExpiration() {
-        return expiration;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/NiFiRegistrySecurityConfig.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/NiFiRegistrySecurityConfig.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/NiFiRegistrySecurityConfig.java
index ce81ee2..16133ca 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/NiFiRegistrySecurityConfig.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/NiFiRegistrySecurityConfig.java
@@ -17,18 +17,17 @@
 package org.apache.nifi.registry.web.security;
 
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.web.security.authentication.NiFiAnonymousUserFilter;
-import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationFilter;
-import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationProvider;
-import org.apache.nifi.registry.web.security.authentication.x509.X509AuthenticationFilter;
-import org.apache.nifi.registry.web.security.authentication.x509.X509AuthenticationProvider;
-import org.apache.nifi.registry.web.security.authentication.x509.X509CertificateExtractor;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.web.security.authentication.AnonymousIdentityFilter;
+import org.apache.nifi.registry.web.security.authentication.IdentityAuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.IdentityFilter;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtIdentityProvider;
+import org.apache.nifi.registry.web.security.authentication.x509.X509IdentityAuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.x509.X509IdentityProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
-import org.springframework.security.authentication.AuthenticationManager;
 import org.springframework.security.config.annotation.authentication.builders.AuthenticationManagerBuilder;
 import org.springframework.security.config.annotation.method.configuration.EnableGlobalMethodSecurity;
 import org.springframework.security.config.annotation.web.builders.HttpSecurity;
@@ -37,10 +36,9 @@ import org.springframework.security.config.annotation.web.configuration.EnableWe
 import org.springframework.security.config.annotation.web.configuration.WebSecurityConfigurerAdapter;
 import org.springframework.security.config.http.SessionCreationPolicy;
 import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
-import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
 
 /**
- * NiFi Web Api Spring security
+ * NiFi Registry Web Api Spring security
  */
 @Configuration
 @EnableWebSecurity
@@ -50,18 +48,17 @@ public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
 
     @Autowired private NiFiRegistryProperties properties;
 
-    @Autowired private X509CertificateExtractor certificateExtractor;
-    @Autowired private X509PrincipalExtractor principalExtractor;
-    @Autowired private X509AuthenticationProvider x509AuthenticationProvider;
-    private X509AuthenticationFilter x509AuthenticationFilter;
+    @Autowired private Authorizer authorizer;
 
-    @Autowired private JwtAuthenticationProvider jwtAuthenticationProvider;
-    private JwtAuthenticationFilter jwtAuthenticationFilter;
+    private AnonymousIdentityFilter anonymousAuthenticationFilter = new AnonymousIdentityFilter();
 
-//    @Autowired private OtpAuthenticationProvider otpAuthenticationProvider;
-//    private OtpAuthenticationFilter otpAuthenticationFilter;
+    @Autowired private X509IdentityProvider x509IdentityProvider;
+    private IdentityFilter x509AuthenticationFilter;
+    private IdentityAuthenticationProvider x509AuthenticationProvider;
 
-    private NiFiAnonymousUserFilter anonymousAuthenticationFilter;
+    @Autowired private JwtIdentityProvider jwtIdentityProvider;
+    private IdentityFilter jwtAuthenticationFilter;
+    private IdentityAuthenticationProvider jwtAuthenticationProvider;
 
     public NiFiRegistrySecurityConfig() {
         super(true); // disable defaults
@@ -69,12 +66,8 @@ public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
 
     @Override
     public void configure(WebSecurity webSecurity) throws Exception {
-        // ignore the access endpoints for obtaining the access config, access token
-        // granting, and access status for a given user (note: we are not ignoring the
-        // the /access/download-token endpoints)
-        webSecurity
-                .ignoring()
-                .antMatchers( "/access/token");
+        // allow any client to access the endpoint for logging in to generate an access token
+        webSecurity.ignoring().antMatchers( "/access/token/*");
     }
 
     @Override
@@ -91,68 +84,48 @@ public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
         http.addFilterBefore(x509AuthenticationFilter(), AnonymousAuthenticationFilter.class);
 
         // jwt
-        http.addFilterBefore(jwtFilterBean(), AnonymousAuthenticationFilter.class);
+        http.addFilterBefore(jwtAuthenticationFilter(), AnonymousAuthenticationFilter.class);
 
         // otp
-        // http.addFilterBefore(otpFilterBean(), AnonymousAuthenticationFilter.class);
+        // todo, if needed one-time password auth filter goes here
 
         // anonymous
-        http.anonymous().authenticationFilter(anonymousFilter());
-    }
-
-    @Bean
-    @Override
-    public AuthenticationManager authenticationManagerBean() throws Exception {
-        // override xxxBean method so the authentication manager is available in app context (necessary for the method level security)
-        return super.authenticationManagerBean();
+        http.anonymous().authenticationFilter(anonymousAuthenticationFilter);
     }
 
     @Override
     protected void configure(AuthenticationManagerBuilder auth) throws Exception {
         auth
-                .authenticationProvider(x509AuthenticationProvider)
-                .authenticationProvider(jwtAuthenticationProvider);
-//                .authenticationProvider(otpAuthenticationProvider); // TODO OTP support
+                .authenticationProvider(x509AuthenticationProvider())
+                .authenticationProvider(jwtAuthenticationProvider());
     }
 
-    @Bean
-    public JwtAuthenticationFilter jwtFilterBean() throws Exception {
-        if (jwtAuthenticationFilter == null) {
-            jwtAuthenticationFilter = new JwtAuthenticationFilter();
-            jwtAuthenticationFilter.setProperties(properties);
-            jwtAuthenticationFilter.setAuthenticationManager(authenticationManager());
+    private IdentityFilter x509AuthenticationFilter() throws Exception {
+        if (x509AuthenticationFilter == null) {
+            x509AuthenticationFilter = new IdentityFilter(x509IdentityProvider);
         }
-        return jwtAuthenticationFilter;
+        return x509AuthenticationFilter;
     }
 
-//    @Bean // TODO OtpAuthenticationFilter
-//    public OtpAuthenticationFilter otpFilterBean() throws Exception {
-//        if (otpAuthenticationFilter == null) {
-//            otpAuthenticationFilter = new OtpAuthenticationFilter();
-//            otpAuthenticationFilter.setProperties(properties);
-//            otpAuthenticationFilter.setAuthenticationManager(authenticationManager());
-//        }
-//        return otpAuthenticationFilter;
-//    }
-
-    @Bean
-    public X509AuthenticationFilter x509AuthenticationFilter() throws Exception {
-        if (x509AuthenticationFilter == null) {
-            x509AuthenticationFilter = new X509AuthenticationFilter();
-            x509AuthenticationFilter.setProperties(properties);
-            x509AuthenticationFilter.setCertificateExtractor(certificateExtractor);
-            x509AuthenticationFilter.setPrincipalExtractor(principalExtractor);
-            x509AuthenticationFilter.setAuthenticationManager(authenticationManager());
+    private IdentityAuthenticationProvider x509AuthenticationProvider() {
+        if (x509AuthenticationProvider == null) {
+            x509AuthenticationProvider = new X509IdentityAuthenticationProvider(properties, authorizer, x509IdentityProvider);
         }
-        return x509AuthenticationFilter;
+        return x509AuthenticationProvider;
+    }
+
+    private IdentityFilter jwtAuthenticationFilter() throws Exception {
+        if (jwtAuthenticationFilter == null) {
+            jwtAuthenticationFilter = new IdentityFilter(jwtIdentityProvider);
+        }
+        return jwtAuthenticationFilter;
     }
 
-    @Bean
-    public NiFiAnonymousUserFilter anonymousFilter() throws Exception {
-        if (anonymousAuthenticationFilter == null) {
-            anonymousAuthenticationFilter = new NiFiAnonymousUserFilter();
+    private IdentityAuthenticationProvider jwtAuthenticationProvider() {
+        if (jwtAuthenticationProvider == null) {
+            jwtAuthenticationProvider = new X509IdentityAuthenticationProvider(properties, authorizer, jwtIdentityProvider);
         }
-        return anonymousAuthenticationFilter;
+        return jwtAuthenticationProvider;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AnonymousIdentityFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AnonymousIdentityFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AnonymousIdentityFilter.java
new file mode 100644
index 0000000..f879f0d
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AnonymousIdentityFilter.java
@@ -0,0 +1,39 @@
+/*
+ * 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.registry.web.security.authentication;
+
+import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
+import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
+
+import javax.servlet.http.HttpServletRequest;
+
+public class AnonymousIdentityFilter extends AnonymousAuthenticationFilter {
+
+    private static final String ANONYMOUS_KEY = "anonymousNifiKey";
+
+    public AnonymousIdentityFilter() {
+        super(ANONYMOUS_KEY);
+    }
+
+    @Override
+    protected Authentication createAuthentication(HttpServletRequest request) {
+        return new AuthenticationSuccessToken(new NiFiUserDetails(StandardNiFiUser.ANONYMOUS));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationRequestToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationRequestToken.java
new file mode 100644
index 0000000..a5a5ec3
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationRequestToken.java
@@ -0,0 +1,107 @@
+/*
+ * 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.registry.web.security.authentication;
+
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.GrantedAuthority;
+
+import java.security.Principal;
+import java.util.Collection;
+
+/**
+ * Wraps an AuthenticationRequest in a Token that implements the Spring Security Authentication interface.
+ */
+public class AuthenticationRequestToken implements Authentication {
+
+    private final AuthenticationRequest authenticationRequest;
+    private final Class<?> authenticationRequestOrigin;
+    private final String clientAddress;
+
+    public AuthenticationRequestToken(AuthenticationRequest authenticationRequest, Class<?> authenticationRequestOrigin, String clientAddress) {
+        this.authenticationRequest = authenticationRequest;
+        this.authenticationRequestOrigin = authenticationRequestOrigin;
+        this.clientAddress = clientAddress;
+    }
+
+    @Override
+    public Collection<? extends GrantedAuthority> getAuthorities() {
+        return null;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return authenticationRequest.getCredentials();
+    }
+
+    @Override
+    public Object getDetails() {
+        return authenticationRequest.getDetails();
+    }
+
+    @Override
+    public Object getPrincipal() {
+        return new Principal() {
+            @Override
+            public String getName() {
+                return authenticationRequest.getUsername();
+            }
+        };
+    }
+
+    @Override
+    public boolean isAuthenticated() {
+        return false;
+    }
+
+    @Override
+    public void setAuthenticated(boolean b) throws IllegalArgumentException {
+        throw new IllegalArgumentException("AuthenticationRequestWrapper cannot be trusted. It is only to be used for storing an identity claim.");
+    }
+
+    @Override
+    public String getName() {
+        return authenticationRequest.getUsername();
+    }
+
+    @Override
+    public int hashCode() {
+        return authenticationRequest.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return authenticationRequest.equals(obj);
+    }
+
+    @Override
+    public String toString() {
+        return authenticationRequest.toString();
+    }
+
+    public AuthenticationRequest getAuthenticationRequest() {
+        return authenticationRequest;
+    }
+
+    public Class<?> getAuthenticationRequestOrigin() {
+        return authenticationRequestOrigin;
+    }
+
+    public String getClientAddress() {
+        return clientAddress;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationSuccessToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationSuccessToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationSuccessToken.java
new file mode 100644
index 0000000..ea6f1e9
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/AuthenticationSuccessToken.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.registry.web.security.authentication;
+
+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 AuthenticationSuccessToken extends AbstractAuthenticationToken {
+
+    private final UserDetails nifiUserDetails;
+
+    public AuthenticationSuccessToken(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.");
+    }
+
+    @Override
+    public String toString() {
+        return nifiUserDetails.getUsername();
+    }
+}


[4/4] nifi-registry git commit: NIFIREG-45 Refactor LoginIdentityProvider interface as a more generic and flexible IdentityProvider interface

Posted by bb...@apache.org.
NIFIREG-45 Refactor LoginIdentityProvider interface as a more generic and flexible IdentityProvider interface

This closes #37.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/master
Commit: ef8ba127c83300a095c398d6ca9093fb428f524f
Parents: 63ddf41
Author: Kevin Doran <kd...@gmail.com>
Authored: Mon Oct 30 14:39:56 2017 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Tue Nov 21 11:17:10 2017 -0500

----------------------------------------------------------------------
 .../client/impl/JerseyNiFiRegistryClient.java   |  40 +--
 nifi-registry-framework/pom.xml                 |  19 ++
 .../registry/extension/ExtensionManager.java    |   4 +-
 .../authentication/IdentityProviderFactory.java | 261 ++++++++++++++++++
 ...ardIdentityProviderConfigurationContext.java |  54 ++++
 .../security/ldap/LdapIdentityProvider.java     | 170 ++++++------
 .../ldap/tenants/LdapUserGroupProvider.java     |   6 +-
 ...try.security.authentication.IdentityProvider |  15 ++
 ...ecurity.authentication.LoginIdentityProvider |  15 --
 .../src/main/xsd/identity-providers.xsd         |  50 ++++
 nifi-registry-security-api/pom.xml              |   6 +
 .../authentication/AuthenticationRequest.java   |  82 ++++++
 .../authentication/AuthenticationResponse.java  |  35 ++-
 .../BasicAuthIdentityProvider.java              |  95 +++++++
 .../BearerAuthIdentityProvider.java             |  72 +++++
 .../authentication/IdentityProvider.java        | 157 +++++++++++
 .../IdentityProviderConfigurationContext.java   |  50 ++++
 .../authentication/IdentityProviderLookup.java  |  23 ++
 .../authentication/IdentityProviderUsage.java   |  32 +++
 .../authentication/LoginCredentials.java        |  39 ---
 .../authentication/LoginIdentityProvider.java   |  61 -----
 ...ginIdentityProviderConfigurationContext.java |  48 ----
 ...inIdentityProviderInitializationContext.java |  27 --
 .../LoginIdentityProviderLookup.java            |  23 --
 .../UsernamePasswordAuthenticationRequest.java  |  25 ++
 .../annotation/IdentityProviderContext.java     |  31 +++
 .../LoginIdentityProviderContext.java           |  35 ---
 .../exception/InvalidCredentialsException.java  |  33 +++
 .../InvalidLoginCredentialsException.java       |  33 ---
 .../exception/ProviderCreationException.java    |  39 ---
 .../exception/ProviderDestructionException.java |  39 ---
 .../exception/AuthorizerCreationException.java  |   3 +-
 .../AuthorizerDestructionException.java         |   3 +-
 .../SecurityProviderCreationException.java      |  38 +++
 .../SecurityProviderDestructionException.java   |  38 +++
 .../security/util/ProxiedEntitiesUtils.java     | 127 +++++++++
 nifi-registry-web-api/pom.xml                   |  25 --
 .../nifi/registry/web/api/AccessResource.java   | 201 +++++++++++---
 .../registry/web/request/IntegerParameter.java  |  39 ---
 .../web/response/AuthenticationResponse.java    |  65 -----
 .../security/NiFiRegistrySecurityConfig.java    | 111 +++-----
 .../authentication/AnonymousIdentityFilter.java |  39 +++
 .../AuthenticationRequestToken.java             | 107 ++++++++
 .../AuthenticationSuccessToken.java             |  55 ++++
 .../IdentityAuthenticationFilter.java           | 203 ++++++++++++++
 .../IdentityAuthenticationProvider.java         | 140 ++++++++++
 .../security/authentication/IdentityFilter.java |  97 +++++++
 .../LoginIdentityProviderFactory.java           | 264 -------------------
 .../authentication/NiFiAnonymousUserFilter.java |  40 ---
 .../NiFiAuthenticationFilter.java               | 156 -----------
 .../NiFiAuthenticationProvider.java             |  84 ------
 .../NiFiAuthenticationRequestToken.java         |  41 ---
 .../authentication/ProxiedEntitiesUtils.java    | 163 ------------
 ...ginIdentityProviderConfigurationContext.java |  52 ----
 ...inIdentityProviderInitializationContext.java |  45 ----
 .../exception/UntrustedProxyException.java      |   3 -
 .../jwt/JwtAuthenticationFilter.java            |  58 ----
 .../jwt/JwtAuthenticationProvider.java          |  69 -----
 .../jwt/JwtAuthenticationRequestToken.java      |  60 -----
 .../authentication/jwt/JwtIdentityProvider.java |  76 ++++++
 .../security/authentication/jwt/JwtService.java |  76 ++++--
 .../token/LoginAuthenticationToken.java         | 123 ---------
 .../token/NiFiAuthenticationToken.java          |  55 ----
 .../x509/X509AuthenticationFilter.java          |  64 -----
 .../x509/X509AuthenticationProvider.java        | 166 ------------
 .../x509/X509AuthenticationRequestToken.java    |  75 ------
 .../x509/X509CertificateValidator.java          |  49 ----
 .../X509IdentityAuthenticationProvider.java     | 131 +++++++++
 .../x509/X509IdentityProvider.java              | 134 +++++++---
 .../src/main/xsd/identity-providers.xsd         |  50 ----
 .../nifi/registry/web/api/SecureLdapIT.java     |  91 +++++--
 71 files changed, 2625 insertions(+), 2410 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
index 454d872..c2eaee5 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
@@ -30,6 +30,7 @@ import org.apache.nifi.registry.client.ItemsClient;
 import org.apache.nifi.registry.client.NiFiRegistryClient;
 import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
 import org.apache.nifi.registry.client.UserClient;
+import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
 import org.glassfish.jersey.client.ClientConfig;
 import org.glassfish.jersey.client.ClientProperties;
 import org.glassfish.jersey.jackson.internal.jackson.jaxrs.json.JacksonJaxbJsonProvider;
@@ -52,13 +53,6 @@ import java.util.stream.Collectors;
  */
 public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
 
-    static final String PROXY_ENTITIES_CHAIN = "X-ProxiedEntitiesChain";
-
-    static final String GT = ">";
-    static final String ESCAPED_GT = "\\\\>";
-    static final String LT = "<";
-    static final String ESCAPED_LT = "\\\\<";
-
     static final String NIFI_REGISTRY_CONTEXT = "nifi-registry-api";
     static final int DEFAULT_CONNECT_TIMEOUT = 10000;
     static final int DEFAULT_READ_TIMEOUT = 10000;
@@ -184,7 +178,7 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
 
         final Map<String,String> headers = new HashMap<>();
         if (proxiedEntitiesValue != null) {
-            headers.put(PROXY_ENTITIES_CHAIN, proxiedEntitiesValue);
+            headers.put(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN, proxiedEntitiesValue);
         }
         return headers;
     }
@@ -194,38 +188,10 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
             return null;
         }
 
-        final List<String> proxiedEntityChain = Arrays.asList(proxiedEntities).stream().map(dn -> formatProxyDn(dn)).collect(Collectors.toList());
+        final List<String> proxiedEntityChain = Arrays.stream(proxiedEntities).map(ProxiedEntitiesUtils::formatProxyDn).collect(Collectors.toList());
         return StringUtils.join(proxiedEntityChain, "");
     }
 
-    /**
-     * Formats the specified DN to be set as a HTTP header using well known conventions.
-     *
-     * @param dn raw dn
-     * @return the dn formatted as an HTTP header
-     */
-    private static String formatProxyDn(String dn) {
-        return LT + sanitizeDn(dn) + GT;
-    }
-
-    /**
-     * If a user provides a DN with the sequence '><', they could escape the tokenization process and impersonate another user.
-     * <p>
-     * Example:
-     * <p>
-     * Provided DN: {@code jdoe><alopresto} -> {@code <jdoe><alopresto><proxy...>} would allow the user to impersonate jdoe
-     *
-     * @param rawDn the unsanitized DN
-     * @return the sanitized DN
-     */
-    private static String sanitizeDn(String rawDn) {
-        if (StringUtils.isEmpty(rawDn)) {
-            return rawDn;
-        } else {
-            return rawDn.replaceAll(GT, ESCAPED_GT).replaceAll(LT, ESCAPED_LT);
-        }
-    }
-
     @Override
     public void close() throws IOException {
         if (this.client != null) {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/pom.xml b/nifi-registry-framework/pom.xml
index 95381f5..e4bc462 100644
--- a/nifi-registry-framework/pom.xml
+++ b/nifi-registry-framework/pom.xml
@@ -90,6 +90,19 @@
                             <clearOutputDir>false</clearOutputDir>
                         </configuration>
                     </execution>
+                    <execution>
+                        <id>identity-providers</id>
+                        <goals>
+                            <goal>xjc</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/xsd/identity-providers.xsd</source>
+                            </sources>
+                            <packageName>org.apache.nifi.registry.security.authentication.generated</packageName>
+                            <clearOutputDir>false</clearOutputDir>
+                        </configuration>
+                    </execution>
                 </executions>
             </plugin>
             <plugin>
@@ -157,6 +170,12 @@
             <version>0.0.1-SNAPSHOT</version>
         </dependency>
         <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <version>3.1.0</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.springframework.boot</groupId>
             <artifactId>spring-boot-starter-security</artifactId>
             <version>${spring.boot.version}</version>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
index 27e8b91..4c8b5ac 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
@@ -17,7 +17,7 @@
 package org.apache.nifi.registry.extension;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
 import org.apache.nifi.registry.security.authorization.AccessPolicyProvider;
 import org.apache.nifi.registry.security.authorization.Authorizer;
 import org.apache.nifi.registry.security.authorization.UserGroupProvider;
@@ -55,7 +55,7 @@ public class ExtensionManager {
         classes.add(UserGroupProvider.class);
         classes.add(AccessPolicyProvider.class);
         classes.add(Authorizer.class);
-        classes.add(LoginIdentityProvider.class);
+        classes.add(IdentityProvider.class);
         EXTENSION_CLASSES = Collections.unmodifiableList(classes);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
new file mode 100644
index 0000000..720bd9c
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
@@ -0,0 +1,261 @@
+/*
+ * 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.registry.security.authentication;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.extension.ExtensionManager;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.security.authentication.annotation.IdentityProviderContext;
+import org.apache.nifi.registry.security.authentication.generated.IdentityProviders;
+import org.apache.nifi.registry.security.authentication.generated.Property;
+import org.apache.nifi.registry.security.authentication.generated.Provider;
+import org.apache.nifi.registry.security.util.XmlUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.DisposableBean;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.context.annotation.Primary;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.stream.XMLStreamReader;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.File;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.Map;
+
+@Configuration
+public class IdentityProviderFactory implements IdentityProviderLookup, DisposableBean {
+
+    private static final Logger logger = LoggerFactory.getLogger(IdentityProviderFactory.class);
+    private static final String LOGIN_IDENTITY_PROVIDERS_XSD = "/identity-providers.xsd";
+    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.registry.security.authentication.generated";
+    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
+
+    private static JAXBContext initializeJaxbContext() {
+        try {
+            return JAXBContext.newInstance(JAXB_GENERATED_PATH, IdentityProviderFactory.class.getClassLoader());
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext.");
+        }
+    }
+
+    private NiFiRegistryProperties properties;
+    private ExtensionManager extensionManager;
+    private IdentityProvider identityProvider;
+    private final Map<String, IdentityProvider> identityProviders = new HashMap<>();
+
+    @Autowired
+    public IdentityProviderFactory(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
+        this.properties = properties;
+        this.extensionManager = extensionManager;
+
+        if (this.properties == null) {
+            throw new IllegalStateException("NiFiRegistryProperties cannot be null");
+        }
+
+        if (this.extensionManager == null) {
+            throw new IllegalStateException("ExtensionManager cannot be null");
+        }
+    }
+
+    @Override
+    public IdentityProvider getIdentityProvider(String identifier) {
+        return identityProviders.get(identifier);
+    }
+
+    @Primary
+    @Bean
+//    @Bean("LoginIdentityProvider")
+    public IdentityProvider getIdentityProvider() throws Exception {
+        if (identityProvider == null) {
+            // look up the login identity provider to use
+            final String loginIdentityProviderIdentifier = properties.getProperty(NiFiRegistryProperties.SECURITY_IDENTITY_PROVIDER);
+
+            // ensure the login identity provider class name was specified
+            if (StringUtils.isNotBlank(loginIdentityProviderIdentifier)) {
+                final IdentityProviders loginIdentityProviderConfiguration = loadLoginIdentityProvidersConfiguration();
+
+                // create each login identity provider
+                for (final Provider provider : loginIdentityProviderConfiguration.getProvider()) {
+                    identityProviders.put(provider.getIdentifier(), createLoginIdentityProvider(provider.getIdentifier(), provider.getClazz()));
+                }
+
+                // configure each login identity provider
+                for (final Provider provider : loginIdentityProviderConfiguration.getProvider()) {
+                    final IdentityProvider instance = identityProviders.get(provider.getIdentifier());
+                    instance.onConfigured(loadLoginIdentityProviderConfiguration(provider));
+                }
+
+                // get the login identity provider instance
+                identityProvider = getIdentityProvider(loginIdentityProviderIdentifier);
+
+                // ensure it was found
+                if (identityProvider == null) {
+                    throw new Exception(String.format("The specified login identity provider '%s' could not be found.", loginIdentityProviderIdentifier));
+                }
+            }
+        }
+
+        return identityProvider;
+    }
+
+    @Override
+    public void destroy() throws Exception {
+        if (identityProviders != null) {
+            identityProviders.entrySet().stream().forEach(e -> e.getValue().preDestruction());
+        }
+    }
+
+    private IdentityProviders loadLoginIdentityProvidersConfiguration() throws Exception {
+        final File loginIdentityProvidersConfigurationFile = properties.getIdentityProviderConfigurationFile();
+
+        // load the users from the specified file
+        if (loginIdentityProvidersConfigurationFile.exists()) {
+            try {
+                // find the schema
+                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+                final Schema schema = schemaFactory.newSchema(IdentityProviders.class.getResource(LOGIN_IDENTITY_PROVIDERS_XSD));
+
+                // attempt to unmarshal
+                XMLStreamReader xsr = XmlUtils.createSafeReader(new StreamSource(loginIdentityProvidersConfigurationFile));
+                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
+                unmarshaller.setSchema(schema);
+                final JAXBElement<IdentityProviders> element = unmarshaller.unmarshal(xsr, IdentityProviders.class);
+                return element.getValue();
+            } catch (SAXException | JAXBException e) {
+                throw new Exception("Unable to load the login identity provider configuration file at: " + loginIdentityProvidersConfigurationFile.getAbsolutePath());
+            }
+        } else {
+            throw new Exception("Unable to find the login identity provider configuration file at " + loginIdentityProvidersConfigurationFile.getAbsolutePath());
+        }
+    }
+
+    private IdentityProvider createLoginIdentityProvider(final String identifier, final String loginIdentityProviderClassName) throws Exception {
+        final IdentityProvider instance;
+
+        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(loginIdentityProviderClassName);
+        if (classLoader == null) {
+            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + loginIdentityProviderClassName);
+        }
+
+        // attempt to load the class
+        Class<?> rawLoginIdentityProviderClass = Class.forName(loginIdentityProviderClassName, true, classLoader);
+        Class<? extends IdentityProvider> loginIdentityProviderClass = rawLoginIdentityProviderClass.asSubclass(IdentityProvider.class);
+
+        // otherwise create a new instance
+        Constructor constructor = loginIdentityProviderClass.getConstructor();
+        instance = (IdentityProvider) constructor.newInstance();
+
+        // method injection
+        performMethodInjection(instance, loginIdentityProviderClass);
+
+        // field injection
+        performFieldInjection(instance, loginIdentityProviderClass);
+
+        return instance;
+    }
+
+    private IdentityProviderConfigurationContext loadLoginIdentityProviderConfiguration(final Provider provider) {
+        final Map<String, String> providerProperties = new HashMap<>();
+
+        for (final Property property : provider.getProperty()) {
+            providerProperties.put(property.getName(), property.getValue());
+        }
+
+        return new StandardIdentityProviderConfigurationContext(provider.getIdentifier(), this, providerProperties);
+    }
+
+    private void performMethodInjection(final IdentityProvider instance, final Class loginIdentityProviderClass)
+            throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
+
+        for (final Method method : loginIdentityProviderClass.getMethods()) {
+            if (method.isAnnotationPresent(IdentityProviderContext.class)) {
+                // make the method accessible
+                final boolean isAccessible = method.isAccessible();
+                method.setAccessible(true);
+
+                try {
+                    final Class<?>[] argumentTypes = method.getParameterTypes();
+
+                    // look for setters (single argument)
+                    if (argumentTypes.length == 1) {
+                        final Class<?> argumentType = argumentTypes[0];
+
+                        // look for well known types
+                        if (NiFiRegistryProperties.class.isAssignableFrom(argumentType)) {
+                            // nifi properties injection
+                            method.invoke(instance, properties);
+                        }
+                    }
+                } finally {
+                    method.setAccessible(isAccessible);
+                }
+            }
+        }
+
+        final Class parentClass = loginIdentityProviderClass.getSuperclass();
+        if (parentClass != null && IdentityProvider.class.isAssignableFrom(parentClass)) {
+            performMethodInjection(instance, parentClass);
+        }
+    }
+
+    private void performFieldInjection(final IdentityProvider instance, final Class loginIdentityProviderClass) throws IllegalArgumentException, IllegalAccessException {
+        for (final Field field : loginIdentityProviderClass.getDeclaredFields()) {
+            if (field.isAnnotationPresent(IdentityProviderContext.class)) {
+                // make the method accessible
+                final boolean isAccessible = field.isAccessible();
+                field.setAccessible(true);
+
+                try {
+                    // get the type
+                    final Class<?> fieldType = field.getType();
+
+                    // only consider this field if it isn't set yet
+                    if (field.get(instance) == null) {
+                        // look for well known types
+                        if (NiFiRegistryProperties.class.isAssignableFrom(fieldType)) {
+                            // nifi properties injection
+                            field.set(instance, properties);
+                        }
+                    }
+
+                } finally {
+                    field.setAccessible(isAccessible);
+                }
+            }
+        }
+
+        final Class parentClass = loginIdentityProviderClass.getSuperclass();
+        if (parentClass != null && IdentityProvider.class.isAssignableFrom(parentClass)) {
+            performFieldInjection(instance, parentClass);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/StandardIdentityProviderConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/StandardIdentityProviderConfigurationContext.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/StandardIdentityProviderConfigurationContext.java
new file mode 100644
index 0000000..3e89dcc
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/StandardIdentityProviderConfigurationContext.java
@@ -0,0 +1,54 @@
+/*
+ * 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.registry.security.authentication;
+
+import java.util.Collections;
+import java.util.Map;
+
+public class StandardIdentityProviderConfigurationContext implements IdentityProviderConfigurationContext {
+
+    private final String identifier;
+    private final IdentityProviderLookup lookup;
+    private final Map<String, String> properties;
+
+    public StandardIdentityProviderConfigurationContext(String identifier, final IdentityProviderLookup lookup, Map<String, String> properties) {
+        this.identifier = identifier;
+        this.lookup = lookup;
+        this.properties = properties;
+    }
+
+    @Override
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    @Override
+    public IdentityProviderLookup getIdentityProviderLookup() {
+        return lookup;
+    }
+
+    @Override
+    public Map<String, String> getProperties() {
+        return Collections.unmodifiableMap(properties);
+    }
+
+    @Override
+    public String getProperty(String property) {
+        return properties.get(property);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapIdentityProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapIdentityProvider.java
index 6beebc5..bebe2d5 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapIdentityProvider.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapIdentityProvider.java
@@ -17,15 +17,15 @@
 package org.apache.nifi.registry.security.ldap;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
 import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
-import org.apache.nifi.registry.security.authentication.LoginCredentials;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderConfigurationContext;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderInitializationContext;
+import org.apache.nifi.registry.security.authentication.BasicAuthIdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProviderConfigurationContext;
 import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
-import org.apache.nifi.registry.security.authentication.exception.InvalidLoginCredentialsException;
-import org.apache.nifi.registry.security.authentication.exception.ProviderCreationException;
-import org.apache.nifi.registry.security.authentication.exception.ProviderDestructionException;
+import org.apache.nifi.registry.security.authentication.exception.InvalidCredentialsException;
+import org.apache.nifi.registry.security.exception.SecurityProviderCreationException;
+import org.apache.nifi.registry.security.exception.SecurityProviderDestructionException;
 import org.apache.nifi.registry.security.util.SslContextFactory;
 import org.apache.nifi.registry.security.util.SslContextFactory.ClientAuth;
 import org.apache.nifi.registry.util.FormatUtils;
@@ -60,33 +60,29 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 /**
- * Abstract LDAP based implementation of a login identity provider.
+ * LDAP based implementation of a login identity provider.
  */
-public class LdapIdentityProvider implements LoginIdentityProvider {
+public class LdapIdentityProvider extends BasicAuthIdentityProvider implements IdentityProvider {
 
     private static final Logger logger = LoggerFactory.getLogger(LdapIdentityProvider.class);
 
-    private AbstractLdapAuthenticationProvider provider;
-    private String issuer;
+    private static final String issuer = LdapIdentityProvider.class.getSimpleName();
+
+    private AbstractLdapAuthenticationProvider ldapAuthenticationProvider;
     private long expiration;
     private IdentityStrategy identityStrategy;
 
     @Override
-    public final void initialize(final LoginIdentityProviderInitializationContext initializationContext) throws ProviderCreationException {
-        this.issuer = getClass().getSimpleName();
-    }
-
-    @Override
-    public final void onConfigured(final LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException {
+    public final void onConfigured(final IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {
         final String rawExpiration = configurationContext.getProperty("Authentication Expiration");
         if (StringUtils.isBlank(rawExpiration)) {
-            throw new ProviderCreationException("The Authentication Expiration must be specified.");
+            throw new SecurityProviderCreationException("The Authentication Expiration must be specified.");
         }
 
         try {
             expiration = FormatUtils.getTimeDuration(rawExpiration, TimeUnit.MILLISECONDS);
         } catch (final IllegalArgumentException iae) {
-            throw new ProviderCreationException(String.format("The Expiration Duration '%s' is not a valid time duration", rawExpiration));
+            throw new SecurityProviderCreationException(String.format("The Expiration Duration '%s' is not a valid time duration", rawExpiration));
         }
 
         final LdapContextSource context = new LdapContextSource();
@@ -103,7 +99,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
         try {
             authenticationStrategy = LdapAuthenticationStrategy.valueOf(rawAuthenticationStrategy);
         } catch (final IllegalArgumentException iae) {
-            throw new ProviderCreationException(String.format("Unrecognized authentication strategy '%s'. Possible values are [%s]",
+            throw new SecurityProviderCreationException(String.format("Unrecognized authentication strategy '%s'. Possible values are [%s]",
                     rawAuthenticationStrategy, StringUtils.join(LdapAuthenticationStrategy.values(), ", ")));
         }
 
@@ -166,7 +162,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
         try {
             referralStrategy = ReferralStrategy.valueOf(rawReferralStrategy);
         } catch (final IllegalArgumentException iae) {
-            throw new ProviderCreationException(String.format("Unrecognized referral strategy '%s'. Possible values are [%s]",
+            throw new SecurityProviderCreationException(String.format("Unrecognized referral strategy '%s'. Possible values are [%s]",
                     rawReferralStrategy, StringUtils.join(ReferralStrategy.values(), ", ")));
         }
 
@@ -177,7 +173,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
         final String urls = configurationContext.getProperty("Url");
 
         if (StringUtils.isBlank(urls)) {
-            throw new ProviderCreationException("LDAP identity provider 'Url' must be specified.");
+            throw new SecurityProviderCreationException("LDAP identity provider 'Url' must be specified.");
         }
 
         // connection
@@ -188,7 +184,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
         final String userSearchFilter = configurationContext.getProperty("User Search Filter");
 
         if (StringUtils.isBlank(userSearchBase) || StringUtils.isBlank(userSearchFilter)) {
-            throw new ProviderCreationException("LDAP identity provider 'User Search Base' and 'User Search Filter' must be specified.");
+            throw new SecurityProviderCreationException("LDAP identity provider 'User Search Base' and 'User Search Filter' must be specified.");
         }
 
         final LdapUserSearch userSearch = new FilterBasedLdapUserSearch(userSearchBase, userSearchFilter, context);
@@ -210,7 +206,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
                 // attempt to get the configured identity strategy
                 identityStrategy = IdentityStrategy.valueOf(rawIdentityStrategy);
             } catch (final IllegalArgumentException iae) {
-                throw new ProviderCreationException(String.format("Unrecognized identity strategy '%s'. Possible values are [%s]",
+                throw new SecurityProviderCreationException(String.format("Unrecognized identity strategy '%s'. Possible values are [%s]",
                         rawIdentityStrategy, StringUtils.join(IdentityStrategy.values(), ", ")));
             }
         }
@@ -225,17 +221,74 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
             context.afterPropertiesSet();
             authenticator.afterPropertiesSet();
         } catch (final Exception e) {
-            throw new ProviderCreationException(e.getMessage(), e);
+            throw new SecurityProviderCreationException(e.getMessage(), e);
         }
 
         // create the underlying provider
-        provider = new LdapAuthenticationProvider(authenticator);
+        ldapAuthenticationProvider = new LdapAuthenticationProvider(authenticator);
+    }
+
+    @Override
+    public AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest) throws InvalidCredentialsException, IdentityAccessException {
+
+        if (authenticationRequest == null || StringUtils.isEmpty(authenticationRequest.getUsername())) {
+            logger.debug("Call to authenticate method with null or empty authenticationRequest, returning null without attempting to authenticate");
+            return null;
+        }
+
+        if (ldapAuthenticationProvider == null) {
+            throw new IdentityAccessException("The LDAP authentication provider is not initialized.");
+        }
+
+        try {
+            final String username = authenticationRequest.getUsername();
+            final Object credentials = authenticationRequest.getCredentials();
+            final String password = credentials != null && credentials instanceof String ? (String) credentials : null;
+
+            // perform the authentication
+            final UsernamePasswordAuthenticationToken token = new UsernamePasswordAuthenticationToken(username, credentials);
+            final Authentication authentication = ldapAuthenticationProvider.authenticate(token);
+
+            // use dn if configured
+            if (IdentityStrategy.USE_DN.equals(identityStrategy)) {
+                // attempt to get the ldap user details to get the DN
+                if (authentication.getPrincipal() instanceof LdapUserDetails) {
+                    final LdapUserDetails userDetails = (LdapUserDetails) authentication.getPrincipal();
+                    return new AuthenticationResponse(userDetails.getDn(), username, expiration, issuer);
+                } else {
+                    logger.warn(String.format("Unable to determine user DN for %s, using username.", authentication.getName()));
+                    return new AuthenticationResponse(authentication.getName(), username, expiration, issuer);
+                }
+            } else {
+                return new AuthenticationResponse(authentication.getName(), username, expiration, issuer);
+            }
+        } catch (final BadCredentialsException | UsernameNotFoundException | AuthenticationException e) {
+            throw new InvalidCredentialsException(e.getMessage(), e);
+        } catch (final Exception e) {
+            // there appears to be a bug that generates a InternalAuthenticationServiceException wrapped around an AuthenticationException. this
+            // shouldn't be the case as they the service exception suggestions that something was wrong with the service. while the authentication
+            // exception suggests that username and/or credentials were incorrect. checking the cause seems to address this scenario.
+            final Throwable cause = e.getCause();
+            if (cause instanceof AuthenticationException) {
+                throw new InvalidCredentialsException(e.getMessage(), e);
+            }
+
+            logger.error(e.getMessage());
+            if (logger.isDebugEnabled()) {
+                logger.debug(StringUtils.EMPTY, e);
+            }
+            throw new IdentityAccessException("Unable to validate the supplied credentials. Please contact the system administrator.", e);
+        }
+    }
+
+    @Override
+    public final void preDestruction() throws SecurityProviderDestructionException {
     }
 
-    private void setTimeout(final LoginIdentityProviderConfigurationContext configurationContext,
-            final Map<String, Object> baseEnvironment,
-            final String configurationProperty,
-            final String environmentKey) {
+    private void setTimeout(final IdentityProviderConfigurationContext configurationContext,
+                            final Map<String, Object> baseEnvironment,
+                            final String configurationProperty,
+                            final String environmentKey) {
 
         final String rawTimeout = configurationContext.getProperty(configurationProperty);
         if (StringUtils.isNotBlank(rawTimeout)) {
@@ -243,12 +296,12 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
                 final Long timeout = FormatUtils.getTimeDuration(rawTimeout, TimeUnit.MILLISECONDS);
                 baseEnvironment.put(environmentKey, timeout.toString());
             } catch (final IllegalArgumentException iae) {
-                throw new ProviderCreationException(String.format("The %s '%s' is not a valid time duration", configurationProperty, rawTimeout));
+                throw new SecurityProviderCreationException(String.format("The %s '%s' is not a valid time duration", configurationProperty, rawTimeout));
             }
         }
     }
 
-    private SSLContext getConfiguredSslContext(final LoginIdentityProviderConfigurationContext configurationContext) {
+    private SSLContext getConfiguredSslContext(final IdentityProviderConfigurationContext configurationContext) {
         final String rawKeystore = configurationContext.getProperty("TLS - Keystore");
         final String rawKeystorePassword = configurationContext.getProperty("TLS - Keystore Password");
         final String rawKeystoreType = configurationContext.getProperty("TLS - Keystore Type");
@@ -266,7 +319,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
             } else {
                 // ensure the protocol is specified
                 if (StringUtils.isBlank(rawProtocol)) {
-                    throw new ProviderCreationException("TLS - Protocol must be specified.");
+                    throw new SecurityProviderCreationException("TLS - Protocol must be specified.");
                 }
 
                 if (StringUtils.isBlank(rawKeystore)) {
@@ -282,7 +335,7 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
                         try {
                             clientAuth = ClientAuth.valueOf(rawClientAuth);
                         } catch (final IllegalArgumentException iae) {
-                            throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
+                            throw new SecurityProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                                     rawClientAuth, StringUtils.join(ClientAuth.values(), ", ")));
                         }
                     }
@@ -292,57 +345,10 @@ public class LdapIdentityProvider implements LoginIdentityProvider {
                 }
             }
         } catch (final KeyStoreException | NoSuchAlgorithmException | CertificateException | UnrecoverableKeyException | KeyManagementException | IOException e) {
-            throw new ProviderCreationException(e.getMessage(), e);
+            throw new SecurityProviderCreationException(e.getMessage(), e);
         }
 
         return sslContext;
     }
 
-    @Override
-    public final AuthenticationResponse authenticate(final LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException {
-        if (provider == null) {
-            throw new IdentityAccessException("The LDAP authentication provider is not initialized.");
-        }
-
-        try {
-            // perform the authentication
-            final UsernamePasswordAuthenticationToken token = new UsernamePasswordAuthenticationToken(credentials.getUsername(), credentials.getPassword());
-            final Authentication authentication = provider.authenticate(token);
-
-            // use dn if configured
-            if (IdentityStrategy.USE_DN.equals(identityStrategy)) {
-                // attempt to get the ldap user details to get the DN
-                if (authentication.getPrincipal() instanceof LdapUserDetails) {
-                    final LdapUserDetails userDetails = (LdapUserDetails) authentication.getPrincipal();
-                    return new AuthenticationResponse(userDetails.getDn(), credentials.getUsername(), expiration, issuer);
-                } else {
-                    logger.warn(String.format("Unable to determine user DN for %s, using username.", authentication.getName()));
-                    return new AuthenticationResponse(authentication.getName(), credentials.getUsername(), expiration, issuer);
-                }
-            } else {
-                return new AuthenticationResponse(authentication.getName(), credentials.getUsername(), expiration, issuer);
-            }
-        } catch (final BadCredentialsException | UsernameNotFoundException | AuthenticationException e) {
-            throw new InvalidLoginCredentialsException(e.getMessage(), e);
-        } catch (final Exception e) {
-            // there appears to be a bug that generates a InternalAuthenticationServiceException wrapped around an AuthenticationException. this
-            // shouldn't be the case as they the service exception suggestions that something was wrong with the service. while the authentication
-            // exception suggests that username and/or credentials were incorrect. checking the cause seems to address this scenario.
-            final Throwable cause = e.getCause();
-            if (cause instanceof AuthenticationException) {
-                throw new InvalidLoginCredentialsException(e.getMessage(), e);
-            }
-
-            logger.error(e.getMessage());
-            if (logger.isDebugEnabled()) {
-                logger.debug(StringUtils.EMPTY, e);
-            }
-            throw new IdentityAccessException("Unable to validate the supplied credentials. Please contact the system administrator.", e);
-        }
-    }
-
-    @Override
-    public final void preDestruction() throws ProviderDestructionException {
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
index af10ece..71bee9a 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
@@ -20,7 +20,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
 import org.apache.nifi.registry.properties.util.IdentityMapping;
 import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
-import org.apache.nifi.registry.security.authentication.exception.ProviderDestructionException;
 import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
 import org.apache.nifi.registry.security.authorization.Group;
 import org.apache.nifi.registry.security.authorization.User;
@@ -30,6 +29,7 @@ import org.apache.nifi.registry.security.authorization.UserGroupProviderInitiali
 import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
 import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
 import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
 import org.apache.nifi.registry.security.ldap.LdapAuthenticationStrategy;
 import org.apache.nifi.registry.security.ldap.LdapsSocketFactory;
 import org.apache.nifi.registry.security.ldap.ReferralStrategy;
@@ -113,8 +113,6 @@ public class LdapUserGroupProvider implements UserGroupProvider {
 
     public static final String PROP_SYNC_INTERVAL = "Sync Interval";
 
-    //private AuthorizerConfigurationContext configurationContext;
-
     private List<IdentityMapping> identityMappings;
     private NiFiRegistryProperties properties;
 
@@ -669,7 +667,7 @@ public class LdapUserGroupProvider implements UserGroupProvider {
     }
 
     @Override
-    public final void preDestruction() throws ProviderDestructionException {
+    public final void preDestruction() throws AuthorizerDestructionException {
         ldapSync.shutdown();
         try {
             if (!ldapSync.awaitTermination(10000, TimeUnit.MILLISECONDS)) {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider b/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider
new file mode 100644
index 0000000..530528f
--- /dev/null
+++ b/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider
@@ -0,0 +1,15 @@
+# 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.
+org.apache.nifi.registry.security.ldap.LdapIdentityProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.LoginIdentityProvider
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.LoginIdentityProvider b/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.LoginIdentityProvider
deleted file mode 100644
index 530528f..0000000
--- a/nifi-registry-framework/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.LoginIdentityProvider
+++ /dev/null
@@ -1,15 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-org.apache.nifi.registry.security.ldap.LdapIdentityProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-framework/src/main/xsd/identity-providers.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/xsd/identity-providers.xsd b/nifi-registry-framework/src/main/xsd/identity-providers.xsd
new file mode 100644
index 0000000..bcca014
--- /dev/null
+++ b/nifi-registry-framework/src/main/xsd/identity-providers.xsd
@@ -0,0 +1,50 @@
+<?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
+    <!-- role -->
+    <xs:complexType name="Provider">
+        <xs:sequence>
+            <xs:element name="identifier" type="NonEmptyStringType"/>
+            <xs:element name="class" type="NonEmptyStringType"/>
+            <xs:element name="property" type="Property" minOccurs="0" maxOccurs="unbounded" />
+        </xs:sequence>
+    </xs:complexType>
+
+    <!-- Name/Value properties-->
+    <xs:complexType name="Property">
+        <xs:simpleContent>
+            <xs:extension base="xs:string">
+                <xs:attribute name="name" type="NonEmptyStringType"/>
+                <xs:attribute name="encryption" type="xs:string"/>
+            </xs:extension>
+        </xs:simpleContent>
+    </xs:complexType>
+
+    <xs:simpleType name="NonEmptyStringType">
+        <xs:restriction base="xs:string">
+            <xs:minLength value="1"/>
+        </xs:restriction>
+    </xs:simpleType>
+
+    <!-- login identity provider -->
+    <xs:element name="identityProviders">
+        <xs:complexType>
+            <xs:sequence>
+                <xs:element name="provider" type="Provider" minOccurs="0" maxOccurs="unbounded"/>
+            </xs:sequence>
+        </xs:complexType>
+    </xs:element>
+</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/pom.xml b/nifi-registry-security-api/pom.xml
index 564a435..b1bcb47 100644
--- a/nifi-registry-security-api/pom.xml
+++ b/nifi-registry-security-api/pom.xml
@@ -32,6 +32,12 @@
             <groupId>org.apache.nifi.registry</groupId>
             <artifactId>nifi-registry-utils</artifactId>
         </dependency>
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <version>3.1.0</version>
+            <scope>provided</scope>
+        </dependency>
     </dependencies>
 
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationRequest.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationRequest.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationRequest.java
new file mode 100644
index 0000000..72ae50e
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationRequest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.registry.security.authentication;
+
+import java.io.Serializable;
+
+public class AuthenticationRequest implements Serializable {
+
+    private String username;
+    private Object credentials;
+    private Object details;
+
+    public AuthenticationRequest(String username, Object credentials, Object details) {
+        this.username = username;
+        this.credentials = credentials;
+        this.details = details;
+    }
+
+    public AuthenticationRequest() {}
+
+    public String getUsername() {
+        return username;
+    }
+
+    public void setUsername(String username) {
+        this.username = username;
+    }
+
+    public Object getCredentials() {
+        return credentials;
+    }
+
+    public void setCredentials(Object credentials) {
+        this.credentials = credentials;
+    }
+
+    public Object getDetails() {
+        return details;
+    }
+
+    public void setDetails(Object details) {
+        this.details = details;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        AuthenticationRequest that = (AuthenticationRequest) o;
+
+        return username != null ? username.equals(that.username) : that.username == null;
+    }
+
+    @Override
+    public int hashCode() {
+        return username != null ? username.hashCode() : 0;
+    }
+
+    @Override
+    public String toString() {
+        return "AuthenticationRequest{" +
+                "username='" + username + '\'' +
+                ", credentials=[PROTECTED]" +
+                ", details=" + details +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationResponse.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationResponse.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationResponse.java
index e6bfeb2..b8eb721 100644
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationResponse.java
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/AuthenticationResponse.java
@@ -16,10 +16,12 @@
  */
 package org.apache.nifi.registry.security.authentication;
 
+import java.io.Serializable;
+
 /**
  * Authentication response for a user login attempt.
  */
-public class AuthenticationResponse {
+public class AuthenticationResponse implements Serializable {
 
     private final String identity;
     private final String username;
@@ -62,4 +64,35 @@ public class AuthenticationResponse {
         return expiration;
     }
 
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        AuthenticationResponse that = (AuthenticationResponse) o;
+
+        if (expiration != that.expiration) return false;
+        if (identity != null ? !identity.equals(that.identity) : that.identity != null) return false;
+        if (username != null ? !username.equals(that.username) : that.username != null) return false;
+        return issuer != null ? issuer.equals(that.issuer) : that.issuer == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = identity != null ? identity.hashCode() : 0;
+        result = 31 * result + (username != null ? username.hashCode() : 0);
+        result = 31 * result + (int) (expiration ^ (expiration >>> 32));
+        result = 31 * result + (issuer != null ? issuer.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "AuthenticationResponse{" +
+                "identity='" + identity + '\'' +
+                ", username='" + username + '\'' +
+                ", expiration=" + expiration +
+                ", issuer='" + issuer + '\'' +
+                '}';
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BasicAuthIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BasicAuthIdentityProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BasicAuthIdentityProvider.java
new file mode 100644
index 0000000..10a1065
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BasicAuthIdentityProvider.java
@@ -0,0 +1,95 @@
+/*
+ * 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.registry.security.authentication;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import java.nio.charset.Charset;
+import java.util.Base64;
+
+public abstract class BasicAuthIdentityProvider implements IdentityProvider {
+
+    public static final String AUTHORIZATION = "Authorization";
+    public static final String BASIC = "Basic ";
+
+    private static final Logger logger = LoggerFactory.getLogger(BasicAuthIdentityProvider.class);
+
+    private static final IdentityProviderUsage usage = new IdentityProviderUsage() {
+        @Override
+        public String getText() {
+            return "The user credentials must be passed in standard HTTP Basic Auth format. " +
+                    "That is: 'Authorization: Basic <credentials>', " +
+                    "where <credentials> is the base64 encoded value of '<username>:<password>'.";
+        }
+    };
+
+    @Override
+    public IdentityProviderUsage getUsageInstructions() {
+        return usage;
+    }
+
+    @Override
+    public AuthenticationRequest extractCredentials(HttpServletRequest servletRequest) {
+
+        if (servletRequest == null) {
+            logger.debug("Cannot extract user credentials from null servletRequest");
+            return null;
+        }
+
+        // only support this type of login when running securely
+        if (!servletRequest.isSecure()) {
+            return null;
+        }
+
+        AuthenticationRequest authenticationRequest;
+
+        try {
+
+            final String authorization = servletRequest.getHeader(AUTHORIZATION);
+            if (authorization == null || !authorization.startsWith(BASIC)) {
+                logger.debug("HTTP Basic Auth credentials not present. Not attempting to extract credentials for authentication.");
+                return null;
+            }
+
+            // Authorization: Basic {base64credentials}
+            String base64Credentials = authorization.substring(BASIC.length()).trim();
+            String credentials = new String(Base64.getDecoder().decode(base64Credentials), Charset.forName("UTF-8"));
+            // credentials = username:password
+            final String[] credentialParts = credentials.split(":", 2);
+            String username = credentialParts[0];
+            String password = credentialParts[1];
+
+            authenticationRequest = new UsernamePasswordAuthenticationRequest(username, password);
+
+        } catch (IllegalArgumentException | IndexOutOfBoundsException e) {
+            logger.info("Failed to extract user identity credentials.");
+            logger.debug("", e);
+            return null;
+        }
+
+        return authenticationRequest;
+
+    }
+
+    @Override
+    public boolean supports(Class<? extends AuthenticationRequest> authenticationRequestClazz) {
+        return UsernamePasswordAuthenticationRequest.class.isAssignableFrom(authenticationRequestClazz);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BearerAuthIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BearerAuthIdentityProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BearerAuthIdentityProvider.java
new file mode 100644
index 0000000..5d08e9a
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/BearerAuthIdentityProvider.java
@@ -0,0 +1,72 @@
+/*
+ * 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.registry.security.authentication;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+
+public abstract class BearerAuthIdentityProvider implements IdentityProvider {
+
+    public static final String AUTHORIZATION = "Authorization";
+    public static final String BEARER = "Bearer ";
+
+    private static final Logger logger = LoggerFactory.getLogger(BearerAuthIdentityProvider.class);
+
+    private static final IdentityProviderUsage usage = new IdentityProviderUsage() {
+        @Override
+        public String getText() {
+            return "The user credentials must be passed in standard HTTP Bearer Authorization format. " +
+                    "That is: 'Authorization: Bearer <token>', " +
+                    "where <token> is a value that will be validated by this identity provider.";
+        }
+    };
+
+    @Override
+    public IdentityProviderUsage getUsageInstructions() {
+        return usage;
+    }
+
+    @Override
+    public AuthenticationRequest extractCredentials(HttpServletRequest request) {
+
+        if (request == null) {
+            logger.debug("Cannot extract user credentials from null servletRequest");
+            return null;
+        }
+
+        // only support this type of login when running securely
+        if (!request.isSecure()) {
+            return null;
+        }
+
+        // get the principal out of the user token
+        final String authorization = request.getHeader(AUTHORIZATION);
+        if (authorization == null || !authorization.startsWith(BEARER)) {
+            logger.debug("HTTP Bearer Auth credentials not present. Not attempting to extract credentials for authentication.");
+            return null;
+        }
+
+        // Extract the encoded token from the Authorization header
+        final String token = authorization.substring(BEARER.length()).trim();
+
+        return new AuthenticationRequest(null, token, null);
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProvider.java
new file mode 100644
index 0000000..88488fb
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProvider.java
@@ -0,0 +1,157 @@
+/*
+ * 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.registry.security.authentication;
+
+import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
+import org.apache.nifi.registry.security.authentication.exception.InvalidCredentialsException;
+import org.apache.nifi.registry.security.exception.SecurityProviderCreationException;
+import org.apache.nifi.registry.security.exception.SecurityProviderDestructionException;
+
+import javax.servlet.http.HttpServletRequest;
+
+/**
+ * IdentityProvider is an interface for a class that is able to establish a client identity.
+ *
+ * Specifically, this provider can:
+ *  - extract credentials from an HttpServletRequest (eg, parse a header, form parameter, or client certificates)
+ *  - authenticate those credentials and map them to an authenticated identity value
+ *    (eg, determine a username given a valid auth token)
+ */
+public interface IdentityProvider {
+
+    /**
+     * @return an IdentityProviderUsage that describes the expectations of the inputs
+     *         to {@link #authenticate(AuthenticationRequest)}
+     */
+    IdentityProviderUsage getUsageInstructions();
+
+    /**
+     * Extracts credentials from an {@link HttpServletRequest}.
+     *
+     * First, a check to the HttpServletRequest should be made to determine if this IdentityProvider is
+     * well suited to authenticate the request. For example, if the IdentityProvider is designed to read
+     * a particular header field to look for a token or identity claim, the check might be that the proper
+     * header field exists and (if a shared header field, such as "Authorization") that the format of the
+     * value in the header matches the expected format for this identity provider (e.g., must start with
+     * a prefix such as "Bearer"). Note, the expectations of the HttpServletRequest can be described by
+     * the {@link #getUsageInstructions()} method.
+     *
+     * If this check fails, this method should return null. This will indicate to the framework that the
+     * IdentityProvider does not recognize an identity claim present in the HttpServletRequest and that
+     * the framework should try another IdentityProvider.
+     *
+     * If the identity claim format is recognized, it should be extracted and returned in an
+     * {@link AuthenticationRequest}. The types and values set in the {@link AuthenticationRequest} are
+     * left to the discretion of the IdentityProvider, as the intended audience of the request is the
+     * {@link #authenticate(AuthenticationRequest)} method, where the corresponding logic to interpret
+     * an {@link AuthenticationRequest} can be implemented. As a rule of thumb, any values that could be considered
+     * sensitive, such as a password or persistent token susceptible to replay attacks, should be stored
+     * in the credentials field of the {@link AuthenticationRequest} as the framework will make the most effort
+     * to protect that value, including obscuring it in toString() output.
+     *
+     * If the {@link AuthenticationRequest} is insufficient or too generic for this IdentityProvider implementation,
+     * this IdentityProvider may subclass {@link AuthenticationRequest} to create a credentials-bearing request
+     * object that is better suited for this IdentityProvider implementation. In that case, the implementation
+     * might wish to also override the {@link #supports(Class)} method to indicate what types of request
+     * objects it supports in the call to {@link #authenticate(AuthenticationRequest)}.
+     *
+     * If credential location is recognized in the {@link HttpServletRequest} but extraction fails,
+     * in most cases that exceptional case should be caught, logged, and null should be returned, as it
+     * is possible another IdentityProvider will be able to parse the credentials or find a separate
+     * set of credentials in the {@link HttpServletRequest} (e.g., a request containing an Authorization
+     * header and a client certificate.)
+     *
+     * @param servletRequest the {@link HttpServletRequest} request that may contain credentials
+     *                       understood by this IdentityProvider
+     * @return an AuthenticationRequest containing the extracted credentials in a format this
+     *         IdentityProvider understands, or null if no credentials could be found in or extracted
+     *         successfully from the servletRequest
+     */
+    AuthenticationRequest extractCredentials(HttpServletRequest servletRequest);
+
+    /**
+     * Authenticates the credentials passed in the {@link AuthenticationRequest}.
+     *
+     * In typical usage, the AuthenticationRequest argument is expected to originate from this
+     * IdentityProvider's {@link #extractCredentials} method, so the logic for interpreting the
+     * values in the {@link AuthenticationRequest} should correspond to how the {@link AuthenticationRequest}
+     * is formed there.
+     *
+     * The first step of authentication should be to check if the credentials are understandable
+     * by this IdentityProvider. If this check fails, this method should return null. This will
+     * indicate to the framework that the IdentityProvider is not able to make a judgement call
+     * on if the request can be authenticated, and the framework can check with another IdentityProvider
+     * if one is available.
+     *
+     * If this IdentityProvider is able to interpret the AuthenticationRequest, it should perform
+     * and authentication check. If the authentication check fails, an exception should be thrown.
+     * Use an {@link InvalidCredentialsException} if the authentication check completed and the
+     * credentials failed authentication. Use an {@link IdentityAccessException} if a dependency
+     * service or provider fails, such as an failure to read a persistent store of identity or
+     * credential data. Either exception type will indicate to the framework that this IdentityProvider's
+     * opinion is that the client making the request should be blocked from accessing a resource
+     * that requires authentication. (Versus a null return value, which is an indication that this
+     * IdentityProvider is not well suited to make a judgement call one way or the other.)
+     *
+     * @param authenticationRequest the request, containing identity claim credentials for the
+     *                              IdentityProvider to authenticate and determine an identity
+     * @return The authentication response containing a fully populated identity value,
+     *         or null if identity cannot be determined
+     * @throws InvalidCredentialsException The login credentials were interpretable by this
+     *                                     IdentityProvider and failed authentication
+     * @throws IdentityAccessException Unable to assign an identity due to an issue accessing
+     *                                 underlying storage or service
+     */
+    AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest)
+            throws InvalidCredentialsException, IdentityAccessException;
+
+    /**
+     * Allows this IdentityProvider to declare support for specific subclasses of {@link AuthenticationRequest}.
+     *
+     * In normal usage, only an AuthenticationRequest originating from this IdentityProvider's
+     * {@link #extractCredentials(HttpServletRequest)} method will be passed to {@link #authenticate(AuthenticationRequest)}.
+     * However, when IdentityProviders are used with another framework,
+     * another component may formulate the AuthenticationRequest to pass to the
+     * {@link #authenticate(AuthenticationRequest)} method. This allows a caller to
+     * check if the IdentityProvider can support the AuthenticationRequest class.
+     * If the caller knows the IdentityProvider can support the AuthenticationRequest
+     * (e.g., it was generated by calling {@link #extractCredentials(HttpServletRequest)},
+     * this check is optional and does not need to be performed.
+     *
+     * @param authenticationRequestClazz the class the caller wants to check
+     * @return a boolean value indicating if this IdentityProvider supports authenticationRequestClazz
+     */
+    default boolean supports(Class<? extends AuthenticationRequest> authenticationRequestClazz) {
+        return AuthenticationRequest.class.equals(authenticationRequestClazz);
+    }
+
+    /**
+     * Called to configure the AuthorityProvider after instance creation.
+     *
+     * @param configurationContext at the time of configuration
+     * @throws SecurityProviderCreationException for any issues configuring the provider
+     */
+    void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException;
+
+    /**
+     * Called immediately before instance destruction for implementers to release resources.
+     *
+     * @throws SecurityProviderDestructionException If pre-destruction fails.
+     */
+    void preDestruction() throws SecurityProviderDestructionException;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderConfigurationContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderConfigurationContext.java
new file mode 100644
index 0000000..6be0207
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderConfigurationContext.java
@@ -0,0 +1,50 @@
+/*
+ * 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.registry.security.authentication;
+
+import java.util.Map;
+
+public interface IdentityProviderConfigurationContext {
+
+    /**
+     * @return identifier for the authority provider
+     */
+    String getIdentifier();
+
+    /**
+     * @return the IdentityProviderLookup from the factory context
+     */
+    public IdentityProviderLookup getIdentityProviderLookup();
+
+    /**
+     * Retrieves all properties the component currently understands regardless
+     * of whether a value has been set for them or not. If no value is present
+     * then its value is null and thus any registered default for the property
+     * descriptor applies.
+     *
+     * @return Map of all properties
+     */
+    Map<String, String> getProperties();
+
+    /**
+     * @param property to lookup the descriptor and value of
+     * @return the value the component currently understands for the given
+     * PropertyDescriptor. This method does not substitute default
+     * PropertyDescriptor values, so the value returned will be null if not set
+     */
+    String getProperty(String property);
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderLookup.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderLookup.java
new file mode 100644
index 0000000..dbf6d58
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderLookup.java
@@ -0,0 +1,23 @@
+/*
+ * 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.registry.security.authentication;
+
+public interface IdentityProviderLookup {
+
+    IdentityProvider getIdentityProvider(String identifier);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderUsage.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderUsage.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderUsage.java
new file mode 100644
index 0000000..98eea95
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderUsage.java
@@ -0,0 +1,32 @@
+/*
+ * 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.registry.security.authentication;
+
+public interface IdentityProviderUsage {
+
+    /**
+     * Provides the usage instructions for an identity provider.
+     *
+     * The instructions should target a human consumer of the
+     * NiFi Registry REST API that needs to know how to handle
+     * Authentication when using / programming an API client.
+     *
+     * @return the usage instructions for an identity provider
+     */
+    String getText();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginCredentials.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginCredentials.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginCredentials.java
deleted file mode 100644
index 925d36d..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authentication/LoginCredentials.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.security.authentication;
-
-/**
- * Login credentials for a user.
- */
-public class LoginCredentials {
-
-    private final String username;
-    private final String password;
-
-    public LoginCredentials(String username, String password) {
-        this.username = username;
-        this.password = password;
-    }
-
-    public String getUsername() {
-        return username;
-    }
-
-    public String getPassword() {
-        return password;
-    }
-}


[2/4] nifi-registry git commit: NIFIREG-45 Refactor LoginIdentityProvider interface as a more generic and flexible IdentityProvider interface

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationFilter.java
new file mode 100644
index 0000000..7c2b27f
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationFilter.java
@@ -0,0 +1,203 @@
+/*
+ * 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.registry.web.security.authentication;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
+import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
+import org.apache.nifi.registry.web.security.authentication.exception.UntrustedProxyException;
+import org.springframework.security.authentication.AuthenticationManager;
+import org.springframework.security.authentication.AuthenticationServiceException;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.core.GrantedAuthority;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.web.authentication.AbstractAuthenticationProcessingFilter;
+import org.springframework.security.web.util.matcher.AntPathRequestMatcher;
+import org.springframework.security.web.util.matcher.RequestMatcher;
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Collection;
+
+/**
+ * Note: This class is deprecated and is being considered for complete removal in favor of using {@link IdentityFilter}.
+ *       It is remaining in place for the time being until the pattern of authentication implemented by {@link IdentityFilter}
+ *       has been more thoroughly vetted in real use.
+ */
+@Deprecated
+public class IdentityAuthenticationFilter extends AbstractAuthenticationProcessingFilter {
+
+    private static final RequestMatcher requiresAuthenticationRequestMatcher = new RequestMatcher() {
+        @Override
+        public boolean matches(HttpServletRequest httpServletRequest) {
+            return NiFiUserUtils.getNiFiUser() == null;
+        }
+    };
+
+    private final IdentityProvider identityProvider;
+
+    public IdentityAuthenticationFilter(IdentityProvider identityProvider, AuthenticationManager authenticationManager, String defaultFilterProcessesUrl) {
+        super(defaultFilterProcessesUrl);
+        super.setRequiresAuthenticationRequestMatcher(new AntPathRequestMatcher(defaultFilterProcessesUrl)); // Authentication will only be initiated for the request url matching this pattern
+        setAuthenticationManager(authenticationManager);
+        this.identityProvider = identityProvider;
+    }
+
+    public IdentityAuthenticationFilter(IdentityProvider identityProvider, AuthenticationManager authenticationManager) {
+        super(requiresAuthenticationRequestMatcher);
+        setAuthenticationManager(authenticationManager);
+        this.identityProvider = identityProvider;
+    }
+
+    @Override
+    public Authentication attemptAuthentication(HttpServletRequest httpServletRequest, HttpServletResponse httpServletResponse) throws AuthenticationException, IOException, ServletException {
+
+        // Only require authentication from an identity provider if the NiFi registry is running securely.
+        if (!httpServletRequest.isSecure()) {
+            // Otherwise, requests will be "authenticated" by the AnonymousIdentityFilter
+            //return null;
+            return new ContinueFilterChainAuthentication(); // see successfulAuthentication for why we do this
+        }
+
+        AuthenticationRequest authenticationRequest = identityProvider.extractCredentials(httpServletRequest);
+        if (authenticationRequest == null) {
+            //return null;
+            return new ContinueFilterChainAuthentication(); // see successfulAuthentication for why we do this
+        }
+        Authentication authentication = new AuthenticationRequestToken(authenticationRequest, identityProvider.getClass(), httpServletRequest.getRemoteAddr());
+        Authentication authenticationResult = getAuthenticationManager().authenticate(authentication); // See IdentityProviderAuthenticationProvider for authentication impl.
+        if (authenticationResult == null) {
+            return new ContinueFilterChainAuthentication(); // see successfulAuthentication for why we do this
+        } else {
+            return authenticationResult;
+        }
+        // Super class will invoke successfulAuthentication() or unsuccessfulAuthentication() depending on the outcome of the authentication attempt
+    }
+
+    @Override
+    protected void successfulAuthentication(HttpServletRequest request, HttpServletResponse response, FilterChain chain, Authentication authResult) throws IOException, ServletException {
+
+        if (authResult.getClass().equals(ContinueFilterChainAuthentication.class)) {
+            logger.info("Authentication unknown, continue chain");
+            // Because this NiFi Registry might be configured with multiple AbstractAuthenticationProcessingFilter's,
+            // the request should continue through the filter chain. If none of the IdentityProviderAuthenticationFilters
+            // can authenticate the request and register a user identity, then the AnonymousIdentityFilter will assign the
+            // Anonymous identity which will not be authorized for access.
+            // A refinement of this would be to extend something other than AbstractAuthenticationProcessingFilter, such as
+            // GenericFilterBean, or to register different filter chains based on context, such as only include
+            // AbstractAuthenticationProcessingFilter(s) when running securely, otherwise don't register any and only register
+            // the AnonymousIdentityFilter.
+            chain.doFilter(request, response);
+        }
+
+        logger.info("Authentication success for " + authResult);
+
+        SecurityContextHolder.getContext().setAuthentication(authResult);
+        if (StringUtils.isNotBlank(request.getHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN))) {
+            response.setHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_ACCEPTED, Boolean.TRUE.toString());
+        }
+
+        // continue the filter chain, which now holds a NiFiUser in the SecurityContext's authentication
+        chain.doFilter(request, response);
+    }
+
+    @Override
+    protected void unsuccessfulAuthentication(HttpServletRequest request, HttpServletResponse response, AuthenticationException failed) throws IOException, ServletException {
+        this.logger.debug("Authentication request failed: " + failed.toString(), failed);
+
+        SecurityContextHolder.clearContext();
+        this.logger.debug("Updated SecurityContextHolder to contain null Authentication");
+
+        // populate the response
+        if (StringUtils.isNotBlank(request.getHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN))) {
+            response.setHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_DETAILS, failed.getMessage());
+        }
+
+        // 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 (failed instanceof InvalidAuthenticationException) {
+            response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+            out.println(failed.getMessage());
+        } else if (failed instanceof UntrustedProxyException) { // thrown in X509IdentityProviderAuthenticationProvider
+            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+            out.println(failed.getMessage());
+        } else if (failed instanceof AuthenticationServiceException) {
+            logger.error(String.format("Unable to authorize: %s", failed.getMessage()), failed);
+            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+            out.println(String.format("Unable to authorize: %s", failed.getMessage()));
+        } else {
+            logger.error(String.format("Unable to authorize: %s", failed.getMessage()), failed);
+            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+            out.println("Access is denied.");
+        }
+
+        // log the failure
+        logger.warn(String.format("Rejecting access to web api: %s", failed.getMessage()));
+        logger.debug(StringUtils.EMPTY, failed);
+    }
+
+    protected class ContinueFilterChainAuthentication implements Authentication {
+        @Override
+        public Collection<? extends GrantedAuthority> getAuthorities() {
+            return null;
+        }
+
+        @Override
+        public Object getCredentials() {
+            return null;
+        }
+
+        @Override
+        public Object getDetails() {
+            return null;
+        }
+
+        @Override
+        public Object getPrincipal() {
+            return null;
+        }
+
+        @Override
+        public boolean isAuthenticated() {
+            return false;
+        }
+
+        @Override
+        public void setAuthenticated(boolean b) throws IllegalArgumentException {
+            throw new IllegalArgumentException("Cannot set authenticated on ContinueFilterChainAuthentication");
+        }
+
+        @Override
+        public String getName() {
+            return null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationProvider.java
new file mode 100644
index 0000000..67c45d8
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityAuthenticationProvider.java
@@ -0,0 +1,140 @@
+/*
+ * 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.registry.web.security.authentication;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.properties.util.IdentityMapping;
+import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.security.authentication.exception.InvalidCredentialsException;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.ManagedAuthorizer;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
+import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.authentication.AuthenticationProvider;
+import org.springframework.security.authentication.BadCredentialsException;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class IdentityAuthenticationProvider implements AuthenticationProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(IdentityAuthenticationProvider.class);
+
+    protected NiFiRegistryProperties properties;
+    protected Authorizer authorizer;
+    protected final IdentityProvider identityProvider;
+    private List<IdentityMapping> mappings;
+
+    public IdentityAuthenticationProvider(
+            NiFiRegistryProperties properties,
+            Authorizer authorizer,
+            IdentityProvider identityProvider) {
+        this.properties = properties;
+        this.authorizer = authorizer;
+        this.identityProvider = identityProvider;
+        this.mappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
+    }
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+
+        // Determine if this AuthenticationProvider's identityProvider should be able to support this AuthenticationRequest
+        boolean tokenOriginatedFromThisIdentityProvider = checkTokenOriginatedFromThisIdentityProvider(authentication);
+
+        if (!tokenOriginatedFromThisIdentityProvider) {
+            // Returning null indicates to The Spring Security AuthenticationManager that this AuthenticationProvider
+            // cannot authenticate this token and another provider should be tried.
+            return null;
+        }
+
+        AuthenticationRequestToken authenticationRequestToken = ((AuthenticationRequestToken)authentication);
+        AuthenticationRequest authenticationRequest = authenticationRequestToken.getAuthenticationRequest();
+
+        try {
+            AuthenticationResponse authenticationResponse = identityProvider.authenticate(authenticationRequest);
+            if (authenticationResponse == null) {
+                return null;
+            }
+            return buildAuthenticatedToken(authenticationRequestToken, authenticationResponse);
+        } catch (InvalidCredentialsException e) {
+            throw new BadCredentialsException("Identity Provider authentication failed.", e);
+        }
+
+    }
+
+    @Override
+    public boolean supports(Class<?> authenticationClazz) {
+        // is authenticationClazz a subclass of AuthenticationRequestWrapper?
+        return AuthenticationRequestToken.class.isAssignableFrom(authenticationClazz);
+    }
+
+    protected AuthenticationSuccessToken buildAuthenticatedToken(
+            AuthenticationRequestToken requestToken,
+            AuthenticationResponse response) {
+
+        final String mappedIdentity = mapIdentity(response.getIdentity());
+
+        return new AuthenticationSuccessToken(new NiFiUserDetails(
+                new StandardNiFiUser.Builder()
+                        .identity(mappedIdentity)
+                        .groups(getUserGroups(mappedIdentity))
+                        .clientAddress(requestToken.getClientAddress())
+                        .build()));
+    }
+
+    protected boolean checkTokenOriginatedFromThisIdentityProvider(Authentication authentication) {
+        return (authentication instanceof AuthenticationRequestToken
+                && identityProvider.getClass().equals(((AuthenticationRequestToken) authentication).getAuthenticationRequestOrigin()));
+    }
+
+    protected String mapIdentity(final String identity) {
+        return IdentityMappingUtil.mapIdentity(identity, mappings);
+    }
+
+    protected Set<String> getUserGroups(final String identity) {
+        return getUserGroups(authorizer, identity);
+    }
+
+    private static Set<String> getUserGroups(final Authorizer authorizer, final String userIdentity) {
+        if (authorizer instanceof ManagedAuthorizer) {
+            final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+            final UserGroupProvider userGroupProvider = managedAuthorizer.getAccessPolicyProvider().getUserGroupProvider();
+            final UserAndGroups userAndGroups = userGroupProvider.getUserAndGroups(userIdentity);
+            final Set<Group> userGroups = userAndGroups.getGroups();
+
+            if (userGroups == null || userGroups.isEmpty()) {
+                return Collections.EMPTY_SET;
+            } else {
+                return userAndGroups.getGroups().stream().map(Group::getName).collect(Collectors.toSet());
+            }
+        } else {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityFilter.java
new file mode 100644
index 0000000..40c2662
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/IdentityFilter.java
@@ -0,0 +1,97 @@
+/*
+ * 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.registry.web.security.authentication;
+
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.web.filter.GenericFilterBean;
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import java.io.IOException;
+
+/**
+ * A class that will extract an identity / credentials claim from an HttpServlet Request using an injected IdentityProvider.
+ *
+ * This class is designed to be used in collaboration with an {@link IdentityAuthenticationProvider}. The identity/credentials will be
+ * extracted by this filter and later validated by the {@link IdentityAuthenticationProvider} in the default SecurityInterceptorFilter.
+ */
+public class IdentityFilter extends GenericFilterBean {
+
+    private static final Logger logger = LoggerFactory.getLogger(IdentityFilter.class);
+
+    private final IdentityProvider identityProvider;
+
+    public IdentityFilter(IdentityProvider identityProvider) {
+        this.identityProvider = identityProvider;
+    }
+
+    @Override
+    public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain) throws IOException, ServletException {
+
+        // Only require authentication from an identity provider if the NiFi registry is running securely.
+        if (!servletRequest.isSecure()) {
+            // Otherwise, requests will be "authenticated" by the AnonymousIdentityFilter
+            filterChain.doFilter(servletRequest, servletResponse);
+            return;
+        }
+
+        if (identityProvider == null) {
+            logger.warn("Identity Filter configured with NULL identity provider. Credentials will not be extracted.");
+            filterChain.doFilter(servletRequest, servletResponse);
+            return;
+        }
+
+        if (credentialsAlreadyPresent()) {
+            logger.debug("Credentials already extracted for {}, skipping credentials extraction filter for {}",
+                    SecurityContextHolder.getContext().getAuthentication().getPrincipal(),
+                    identityProvider.getClass().getSimpleName());
+            filterChain.doFilter(servletRequest, servletResponse);
+            return;
+        }
+
+        logger.debug("Attempting to extract user credentials using {}", identityProvider.getClass().getSimpleName());
+
+        try {
+            AuthenticationRequest authenticationRequest = identityProvider.extractCredentials((HttpServletRequest)servletRequest);
+            if (authenticationRequest != null) {
+                Authentication authentication = new AuthenticationRequestToken(authenticationRequest, identityProvider.getClass(), servletRequest.getRemoteAddr());
+                logger.debug("Adding credentials claim to SecurityContext to be authenticated. Credentials extracted by {}: {}",
+                        identityProvider.getClass().getSimpleName(),
+                        authenticationRequest);
+                SecurityContextHolder.getContext().setAuthentication(authentication);
+                // This filter's job, which is merely to search for and extract an identity claim, is done.
+                // The actual authentication of the identity claim will be handled by a corresponding IdentityAuthenticationProvider
+            }
+        } catch (Exception e) {
+            logger.debug("Exception occurred while extracting credentials:", e);
+        }
+
+        filterChain.doFilter(servletRequest, servletResponse);
+    }
+
+    private boolean credentialsAlreadyPresent() {
+        return SecurityContextHolder.getContext().getAuthentication() != null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/LoginIdentityProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/LoginIdentityProviderFactory.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/LoginIdentityProviderFactory.java
deleted file mode 100644
index 434d881..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/LoginIdentityProviderFactory.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderConfigurationContext;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderLookup;
-import org.apache.nifi.registry.security.authentication.annotation.LoginIdentityProviderContext;
-import org.apache.nifi.registry.extension.ExtensionManager;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.security.authentication.generated.IdentityProviders;
-import org.apache.nifi.registry.security.authentication.generated.Property;
-import org.apache.nifi.registry.security.authentication.generated.Provider;
-import org.apache.nifi.registry.security.util.XmlUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.DisposableBean;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.context.annotation.Bean;
-import org.springframework.context.annotation.Configuration;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.stream.XMLStreamReader;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.File;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.Map;
-
-@Configuration
-public class LoginIdentityProviderFactory implements LoginIdentityProviderLookup, DisposableBean {
-
-    private static final Logger logger = LoggerFactory.getLogger(LoginIdentityProviderFactory.class);
-    private static final String LOGIN_IDENTITY_PROVIDERS_XSD = "/identity-providers.xsd";
-    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.registry.security.authentication.generated";
-    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_GENERATED_PATH, LoginIdentityProviderFactory.class.getClassLoader());
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-    private NiFiRegistryProperties properties;
-    private ExtensionManager extensionManager;
-    private LoginIdentityProvider loginIdentityProvider;
-    private final Map<String, LoginIdentityProvider> loginIdentityProviders = new HashMap<>();
-
-    @Autowired
-    public LoginIdentityProviderFactory(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
-        this.properties = properties;
-        this.extensionManager = extensionManager;
-
-        if (this.properties == null) {
-            throw new IllegalStateException("NiFiRegistryProperties cannot be null");
-        }
-
-        if (this.extensionManager == null) {
-            throw new IllegalStateException("ExtensionManager cannot be null");
-        }
-    }
-
-    @Override
-    public LoginIdentityProvider getLoginIdentityProvider(String identifier) {
-        return loginIdentityProviders.get(identifier);
-    }
-
-    @Bean
-    public LoginIdentityProvider getLoginIdentityProvider() throws Exception {
-        if (loginIdentityProvider == null) {
-            // look up the login identity provider to use
-            final String loginIdentityProviderIdentifier = properties.getProperty(NiFiRegistryProperties.SECURITY_IDENTITY_PROVIDER);
-
-            // ensure the login identity provider class name was specified
-            if (StringUtils.isNotBlank(loginIdentityProviderIdentifier)) {
-                final IdentityProviders loginIdentityProviderConfiguration = loadLoginIdentityProvidersConfiguration();
-
-                // create each login identity provider
-                for (final Provider provider : loginIdentityProviderConfiguration.getProvider()) {
-                    loginIdentityProviders.put(provider.getIdentifier(), createLoginIdentityProvider(provider.getIdentifier(), provider.getClazz()));
-                }
-
-                // configure each login identity provider
-                for (final Provider provider : loginIdentityProviderConfiguration.getProvider()) {
-                    final LoginIdentityProvider instance = loginIdentityProviders.get(provider.getIdentifier());
-                    instance.onConfigured(loadLoginIdentityProviderConfiguration(provider));
-                }
-
-                // get the login identity provider instance
-                loginIdentityProvider = getLoginIdentityProvider(loginIdentityProviderIdentifier);
-
-                // ensure it was found
-                if (loginIdentityProvider == null) {
-                    throw new Exception(String.format("The specified login identity provider '%s' could not be found.", loginIdentityProviderIdentifier));
-                }
-            }
-        }
-
-        return loginIdentityProvider;
-    }
-
-    @Override
-    public void destroy() throws Exception {
-        if (loginIdentityProviders != null) {
-            loginIdentityProviders.entrySet().stream().forEach(e -> e.getValue().preDestruction());
-        }
-    }
-
-    private IdentityProviders loadLoginIdentityProvidersConfiguration() throws Exception {
-        final File loginIdentityProvidersConfigurationFile = properties.getIdentityProviderConfigurationFile();
-
-        // load the users from the specified file
-        if (loginIdentityProvidersConfigurationFile.exists()) {
-            try {
-                // find the schema
-                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-                final Schema schema = schemaFactory.newSchema(IdentityProviders.class.getResource(LOGIN_IDENTITY_PROVIDERS_XSD));
-
-                // attempt to unmarshal
-                XMLStreamReader xsr = XmlUtils.createSafeReader(new StreamSource(loginIdentityProvidersConfigurationFile));
-                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-                unmarshaller.setSchema(schema);
-                final JAXBElement<IdentityProviders> element = unmarshaller.unmarshal(xsr, IdentityProviders.class);
-                return element.getValue();
-            } catch (SAXException | JAXBException e) {
-                throw new Exception("Unable to load the login identity provider configuration file at: " + loginIdentityProvidersConfigurationFile.getAbsolutePath());
-            }
-        } else {
-            throw new Exception("Unable to find the login identity provider configuration file at " + loginIdentityProvidersConfigurationFile.getAbsolutePath());
-        }
-    }
-
-    private LoginIdentityProvider createLoginIdentityProvider(final String identifier, final String loginIdentityProviderClassName) throws Exception {
-        final LoginIdentityProvider instance;
-
-        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(loginIdentityProviderClassName);
-        if (classLoader == null) {
-            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + loginIdentityProviderClassName);
-        }
-
-        // attempt to load the class
-        Class<?> rawLoginIdentityProviderClass = Class.forName(loginIdentityProviderClassName, true, classLoader);
-        Class<? extends LoginIdentityProvider> loginIdentityProviderClass = rawLoginIdentityProviderClass.asSubclass(LoginIdentityProvider.class);
-
-        // otherwise create a new instance
-        Constructor constructor = loginIdentityProviderClass.getConstructor();
-        instance = (LoginIdentityProvider) constructor.newInstance();
-
-        // method injection
-        performMethodInjection(instance, loginIdentityProviderClass);
-
-        // field injection
-        performFieldInjection(instance, loginIdentityProviderClass);
-
-        // call post construction lifecycle event
-        instance.initialize(new StandardLoginIdentityProviderInitializationContext(identifier, this));
-
-        return instance;
-    }
-
-    private LoginIdentityProviderConfigurationContext loadLoginIdentityProviderConfiguration(final Provider provider) {
-        final Map<String, String> providerProperties = new HashMap<>();
-
-        for (final Property property : provider.getProperty()) {
-            providerProperties.put(property.getName(), property.getValue());
-        }
-
-        return new StandardLoginIdentityProviderConfigurationContext(provider.getIdentifier(), providerProperties);
-    }
-
-    private void performMethodInjection(final LoginIdentityProvider instance, final Class loginIdentityProviderClass)
-            throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
-
-        for (final Method method : loginIdentityProviderClass.getMethods()) {
-            if (method.isAnnotationPresent(LoginIdentityProviderContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = method.isAccessible();
-                method.setAccessible(true);
-
-                try {
-                    final Class<?>[] argumentTypes = method.getParameterTypes();
-
-                    // look for setters (single argument)
-                    if (argumentTypes.length == 1) {
-                        final Class<?> argumentType = argumentTypes[0];
-
-                        // look for well known types
-                        if (NiFiRegistryProperties.class.isAssignableFrom(argumentType)) {
-                            // nifi properties injection
-                            method.invoke(instance, properties);
-                        }
-                    }
-                } finally {
-                    method.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = loginIdentityProviderClass.getSuperclass();
-        if (parentClass != null && LoginIdentityProvider.class.isAssignableFrom(parentClass)) {
-            performMethodInjection(instance, parentClass);
-        }
-    }
-
-    private void performFieldInjection(final LoginIdentityProvider instance, final Class loginIdentityProviderClass) throws IllegalArgumentException, IllegalAccessException {
-        for (final Field field : loginIdentityProviderClass.getDeclaredFields()) {
-            if (field.isAnnotationPresent(LoginIdentityProviderContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = field.isAccessible();
-                field.setAccessible(true);
-
-                try {
-                    // get the type
-                    final Class<?> fieldType = field.getType();
-
-                    // only consider this field if it isn't set yet
-                    if (field.get(instance) == null) {
-                        // look for well known types
-                        if (NiFiRegistryProperties.class.isAssignableFrom(fieldType)) {
-                            // nifi properties injection
-                            field.set(instance, properties);
-                        }
-                    }
-
-                } finally {
-                    field.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = loginIdentityProviderClass.getSuperclass();
-        if (parentClass != null && LoginIdentityProvider.class.isAssignableFrom(parentClass)) {
-            performFieldInjection(instance, parentClass);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAnonymousUserFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAnonymousUserFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAnonymousUserFilter.java
deleted file mode 100644
index a27fd18..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAnonymousUserFilter.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
-import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
-import org.apache.nifi.registry.web.security.authentication.token.NiFiAuthenticationToken;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
-
-import javax.servlet.http.HttpServletRequest;
-
-public class NiFiAnonymousUserFilter extends AnonymousAuthenticationFilter {
-
-    private static final String ANONYMOUS_KEY = "anonymousNifiKey";
-
-    public NiFiAnonymousUserFilter() {
-        super(ANONYMOUS_KEY);
-    }
-
-    @Override
-    protected Authentication createAuthentication(HttpServletRequest request) {
-        return new NiFiAuthenticationToken(new NiFiUserDetails(StandardNiFiUser.ANONYMOUS));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationFilter.java
deleted file mode 100644
index 24af504..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationFilter.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
-import org.apache.nifi.registry.web.security.authentication.exception.UntrustedProxyException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.authentication.AuthenticationManager;
-import org.springframework.security.authentication.AuthenticationServiceException;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.core.AuthenticationException;
-import org.springframework.security.core.context.SecurityContextHolder;
-import org.springframework.web.filter.GenericFilterBean;
-
-import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-import java.io.PrintWriter;
-
-/**
- *
- */
-public abstract class NiFiAuthenticationFilter extends GenericFilterBean {
-
-    private static final Logger log = LoggerFactory.getLogger(NiFiAuthenticationFilter.class);
-
-    private AuthenticationManager authenticationManager;
-    private NiFiRegistryProperties properties;
-
-    @Override
-    public void doFilter(final ServletRequest request, final ServletResponse response, final FilterChain chain) throws IOException, ServletException {
-        final Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
-        if (log.isDebugEnabled()) {
-            log.debug("Checking secure context token: " + authentication);
-        }
-
-        if (requiresAuthentication((HttpServletRequest) request)) {
-            authenticate((HttpServletRequest) request, (HttpServletResponse) response, chain);
-        } else {
-            chain.doFilter(request, response);
-        }
-
-    }
-
-    private boolean requiresAuthentication(final HttpServletRequest request) {
-        return NiFiUserUtils.getNiFiUser() == null;
-    }
-
-    private void authenticate(final HttpServletRequest request, final HttpServletResponse response, final FilterChain chain) throws IOException, ServletException {
-        String dnChain = null;
-        try {
-            final Authentication authenticationRequest = attemptAuthentication(request);
-            if (authenticationRequest != null) {
-                // log the request attempt - response details will be logged later
-                log.info(String.format("Attempting request for (%s) %s %s (source ip: %s)", authenticationRequest.toString(), request.getMethod(),
-                        request.getRequestURL().toString(), request.getRemoteAddr()));
-
-                // attempt to authorize the user
-                final Authentication authenticated = authenticationManager.authenticate(authenticationRequest);
-                successfulAuthorization(request, response, authenticated);
-            }
-
-            // continue
-            chain.doFilter(request, response);
-        } catch (final AuthenticationException ae) {
-            // invalid authentication - always error out
-            unsuccessfulAuthorization(request, response, ae);
-        }
-    }
-
-    /**
-     * Attempt to extract an authentication attempt from the specified request.
-     *
-     * @param request The request
-     * @return The authentication attempt or null if none is found int he request
-     */
-    public abstract Authentication attemptAuthentication(HttpServletRequest request);
-
-    protected void successfulAuthorization(HttpServletRequest request, HttpServletResponse response, Authentication authResult) {
-        log.info("Authentication success for " + authResult);
-
-        SecurityContextHolder.getContext().setAuthentication(authResult);
-        ProxiedEntitiesUtils.successfulAuthorization(request, response, authResult);
-    }
-
-    protected void unsuccessfulAuthorization(HttpServletRequest request, HttpServletResponse response, AuthenticationException ae) throws IOException {
-        // populate the response
-        ProxiedEntitiesUtils.unsuccessfulAuthorization(request, response, ae);
-
-        // 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 InvalidAuthenticationException) {
-            response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
-            out.println(ae.getMessage());
-        } else if (ae instanceof UntrustedProxyException) {
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            out.println(ae.getMessage());
-        } else if (ae instanceof AuthenticationServiceException) {
-            log.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 {
-            log.error(String.format("Unable to authorize: %s", ae.getMessage()), ae);
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            out.println("Access is denied.");
-        }
-
-        // log the failure
-        log.warn(String.format("Rejecting access to web api: %s", ae.getMessage()));
-
-        // optionally log the stack trace
-        if (log.isDebugEnabled()) {
-            log.debug(StringUtils.EMPTY, ae);
-        }
-    }
-
-    @Override
-    public void destroy() {
-    }
-
-    public void setAuthenticationManager(AuthenticationManager authenticationManager) {
-        this.authenticationManager = authenticationManager;
-    }
-
-    public void setProperties(NiFiRegistryProperties properties) {
-        this.properties = properties;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationProvider.java
deleted file mode 100644
index 5617e0e..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationProvider.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.nifi.registry.security.authorization.Authorizer;
-import org.apache.nifi.registry.security.authorization.Group;
-import org.apache.nifi.registry.security.authorization.ManagedAuthorizer;
-import org.apache.nifi.registry.security.authorization.UserAndGroups;
-import org.apache.nifi.registry.security.authorization.UserGroupProvider;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.properties.util.IdentityMapping;
-import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.authentication.AuthenticationProvider;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-/**
- * Base AuthenticationProvider that provides common functionality to mapping identities.
- */
-public abstract class NiFiAuthenticationProvider implements AuthenticationProvider {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(NiFiAuthenticationProvider.class);
-
-    private NiFiRegistryProperties properties;
-    private Authorizer authorizer;
-    private List<IdentityMapping> mappings;
-
-    /**
-     * @param properties the NiFiProperties instance
-     */
-    public NiFiAuthenticationProvider(final NiFiRegistryProperties properties, final Authorizer authorizer) {
-        this.properties = properties;
-        this.mappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
-        this.authorizer = authorizer;
-    }
-
-    public List<IdentityMapping> getMappings() {
-        return mappings;
-    }
-
-    protected String mapIdentity(final String identity) {
-        return IdentityMappingUtil.mapIdentity(identity, mappings);
-    }
-
-    protected Set<String> getUserGroups(final String identity) {
-        return getUserGroups(authorizer, identity);
-    }
-
-    protected static Set<String> getUserGroups(final Authorizer authorizer, final String userIdentity) {
-        if (authorizer instanceof ManagedAuthorizer) {
-            final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-            final UserGroupProvider userGroupProvider = managedAuthorizer.getAccessPolicyProvider().getUserGroupProvider();
-            final UserAndGroups userAndGroups = userGroupProvider.getUserAndGroups(userIdentity);
-            final Set<Group> userGroups = userAndGroups.getGroups();
-
-            if (userGroups == null || userGroups.isEmpty()) {
-                return Collections.EMPTY_SET;
-            } else {
-                return userAndGroups.getGroups().stream().map(group -> group.getName()).collect(Collectors.toSet());
-            }
-        } else {
-            return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationRequestToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationRequestToken.java
deleted file mode 100644
index 3da95c5..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/NiFiAuthenticationRequestToken.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.springframework.security.authentication.AbstractAuthenticationToken;
-
-/**
- * Base class for authentication request tokens in NiFI.
- */
-public abstract class NiFiAuthenticationRequestToken extends AbstractAuthenticationToken {
-
-    private final String clientAddress;
-
-    /**
-     * @param clientAddress   The address of the client making the request
-     */
-    public NiFiAuthenticationRequestToken(final String clientAddress) {
-        super(null);
-        setAuthenticated(false);
-        this.clientAddress = clientAddress;
-    }
-
-    public String getClientAddress() {
-        return clientAddress;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/ProxiedEntitiesUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/ProxiedEntitiesUtils.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/ProxiedEntitiesUtils.java
deleted file mode 100644
index 05687f8..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/ProxiedEntitiesUtils.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.security.authorization.user.NiFiUser;
-import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.core.AuthenticationException;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/**
- *
- */
-public class ProxiedEntitiesUtils {
-    private static final Logger logger = LoggerFactory.getLogger(ProxiedEntitiesUtils.class);
-
-    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 static final String GT = ">";
-    private static final String ESCAPED_GT = "\\\\>";
-    private static final String LT = "<";
-    private static final String ESCAPED_LT = "\\\\<";
-
-    private static final String ANONYMOUS_CHAIN = "<>";
-
-    /**
-     * Formats the specified DN to be set as a HTTP header using well known conventions.
-     *
-     * @param dn raw dn
-     * @return the dn formatted as an HTTP header
-     */
-    public static String formatProxyDn(String dn) {
-        return LT + sanitizeDn(dn) + GT;
-    }
-
-    /**
-     * If a user provides a DN with the sequence '><', they could escape the tokenization process and impersonate another user.
-     * <p>
-     * Example:
-     * <p>
-     * Provided DN: {@code jdoe><alopresto} -> {@code <jdoe><alopresto><proxy...>} would allow the user to impersonate jdoe
-     *
-     * @param rawDn the unsanitized DN
-     * @return the sanitized DN
-     */
-    private static String sanitizeDn(String rawDn) {
-        if (StringUtils.isEmpty(rawDn)) {
-            return rawDn;
-        } else {
-            String sanitizedDn = rawDn.replaceAll(GT, ESCAPED_GT).replaceAll(LT, ESCAPED_LT);
-            if (!sanitizedDn.equals(rawDn)) {
-                logger.warn("The provided DN [" + rawDn + "] contained dangerous characters that were escaped to [" + sanitizedDn + "]");
-            }
-            return sanitizedDn;
-        }
-    }
-
-    /**
-     * Reconstitutes the original DN from the sanitized version passed in the proxy chain.
-     * <p>
-     * Example:
-     * <p>
-     * {@code alopresto\>\<proxy1} -> {@code alopresto><proxy1}
-     *
-     * @param sanitizedDn the sanitized DN
-     * @return the original DN
-     */
-    private static String unsanitizeDn(String sanitizedDn) {
-        if (StringUtils.isEmpty(sanitizedDn)) {
-            return sanitizedDn;
-        } else {
-            String unsanitizedDn = sanitizedDn.replaceAll(ESCAPED_GT, GT).replaceAll(ESCAPED_LT, LT);
-            if (!unsanitizedDn.equals(sanitizedDn)) {
-                logger.warn("The provided DN [" + sanitizedDn + "] had been escaped, and was reconstituted to the dangerous DN [" + unsanitizedDn + "]");
-            }
-            return unsanitizedDn;
-        }
-    }
-
-    /**
-     * Tokenizes the specified proxy chain.
-     *
-     * @param rawProxyChain raw chain
-     * @return tokenized proxy chain
-     */
-    public static List<String> tokenizeProxiedEntitiesChain(String rawProxyChain) {
-        final List<String> proxyChain = new ArrayList<>();
-        if (!StringUtils.isEmpty(rawProxyChain)) {
-            // Split the String on the >< token
-            List<String> elements = Arrays.asList(StringUtils.splitByWholeSeparatorPreserveAllTokens(rawProxyChain, "><"));
-
-            // Unsanitize each DN and collect back
-            elements = elements.stream().map(ProxiedEntitiesUtils::unsanitizeDn).collect(Collectors.toList());
-
-            // Remove the leading < from the first element
-            elements.set(0, elements.get(0).replaceFirst(LT, ""));
-
-            // Remove the trailing > from the last element
-            int last = elements.size() - 1;
-            String lastElement = elements.get(last);
-            if (lastElement.endsWith(GT)) {
-                elements.set(last, lastElement.substring(0, lastElement.length() - 1));
-            }
-
-            proxyChain.addAll(elements);
-        }
-
-        return proxyChain;
-    }
-
-    /**
-     * Builds the proxy chain for the specified user.
-     *
-     * @param user The current user
-     * @return The proxy chain for that user in String form
-     */
-    public static String buildProxiedEntitiesChainString(final NiFiUser user) {
-        // calculate the dn chain
-        List<String> proxyChain = NiFiUserUtils.buildProxiedEntitiesChain(user);
-        if (proxyChain.isEmpty()) {
-            return ANONYMOUS_CHAIN;
-        }
-        proxyChain = proxyChain.stream().map(ProxiedEntitiesUtils::formatProxyDn).collect(Collectors.toList());
-        return StringUtils.join(proxyChain, "");
-    }
-
-    public static void successfulAuthorization(HttpServletRequest request, HttpServletResponse response, Authentication authResult) {
-        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
-            response.setHeader(PROXY_ENTITIES_ACCEPTED, Boolean.TRUE.toString());
-        }
-    }
-
-    public static void unsuccessfulAuthorization(HttpServletRequest request, HttpServletResponse response, AuthenticationException failed) {
-        if (StringUtils.isNotBlank(request.getHeader(PROXY_ENTITIES_CHAIN))) {
-            response.setHeader(PROXY_ENTITIES_DETAILS, failed.getMessage());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderConfigurationContext.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderConfigurationContext.java
deleted file mode 100644
index 3a9cdd6..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderConfigurationContext.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderConfigurationContext;
-
-import java.util.Collections;
-import java.util.Map;
-
-/**
- *
- */
-public class StandardLoginIdentityProviderConfigurationContext implements LoginIdentityProviderConfigurationContext {
-
-    private final String identifier;
-    private final Map<String, String> properties;
-
-    public StandardLoginIdentityProviderConfigurationContext(String identifier, Map<String, String> properties) {
-        this.identifier = identifier;
-        this.properties = properties;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public Map<String, String> getProperties() {
-        return Collections.unmodifiableMap(properties);
-    }
-
-    @Override
-    public String getProperty(String property) {
-        return properties.get(property);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderInitializationContext.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderInitializationContext.java
deleted file mode 100644
index e8fba2e..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/StandardLoginIdentityProviderInitializationContext.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication;
-
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderInitializationContext;
-import org.apache.nifi.registry.security.authentication.LoginIdentityProviderLookup;
-
-/**
- *
- */
-public class StandardLoginIdentityProviderInitializationContext implements LoginIdentityProviderInitializationContext {
-
-    private final String identifier;
-    private final LoginIdentityProviderLookup lookup;
-
-    public StandardLoginIdentityProviderInitializationContext(String identifier, final LoginIdentityProviderLookup lookup) {
-        this.identifier = identifier;
-        this.lookup = lookup;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public LoginIdentityProviderLookup getAuthorityProviderLookup() {
-        return lookup;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
index 6245ac2..82570a3 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
@@ -18,9 +18,6 @@ package org.apache.nifi.registry.web.security.authentication.exception;
 
 import org.springframework.security.core.AuthenticationException;
 
-/**
- *
- */
 public class UntrustedProxyException extends AuthenticationException {
 
     public UntrustedProxyException(String msg) {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationFilter.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationFilter.java
deleted file mode 100644
index 1e5c194..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationFilter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.jwt;
-
-import org.apache.commons.lang3.StringUtils;
-
-import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationFilter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.core.Authentication;
-
-import javax.servlet.http.HttpServletRequest;
-
-/**
- */
-public class JwtAuthenticationFilter extends NiFiAuthenticationFilter {
-
-    private static final Logger logger = LoggerFactory.getLogger(JwtAuthenticationFilter.class);
-
-    public static final String AUTHORIZATION = "Authorization";
-    public static final String BEARER = "Bearer ";
-
-    @Override
-    public Authentication attemptAuthentication(final HttpServletRequest request) {
-        // only support jwt login when running securely
-        if (!request.isSecure()) {
-            return null;
-        }
-
-        // TODO: Refactor request header extraction logic to shared utility as it is duplicated in AccessResource
-
-        // get the principal out of the user token
-        final String authorization = request.getHeader(AUTHORIZATION);
-
-        // if there is no authorization header, we don't know the user
-        if (authorization == null || !StringUtils.startsWith(authorization, BEARER)) {
-            return null;
-        } else {
-            // Extract the Base64 encoded token from the Authorization header
-            final String token = StringUtils.substringAfterLast(authorization, " ");
-            return new JwtAuthenticationRequestToken(token, request.getRemoteAddr());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java
deleted file mode 100644
index 2247923..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.jwt;
-
-import io.jsonwebtoken.JwtException;
-import org.apache.nifi.registry.security.authorization.Authorizer;
-import org.apache.nifi.registry.security.authorization.user.NiFiUser;
-import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
-import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
-import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationProvider;
-import org.apache.nifi.registry.web.security.authentication.token.NiFiAuthenticationToken;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.core.AuthenticationException;
-import org.springframework.stereotype.Component;
-
-/**
- *
- */
-@Component
-public class JwtAuthenticationProvider extends NiFiAuthenticationProvider {
-
-    private final JwtService jwtService;
-
-    @Autowired
-    public JwtAuthenticationProvider(JwtService jwtService, NiFiRegistryProperties nifiProperties, Authorizer authorizer) {
-        super(nifiProperties, authorizer);
-        this.jwtService = jwtService;
-    }
-
-    @Override
-    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
-        final JwtAuthenticationRequestToken request = (JwtAuthenticationRequestToken) authentication;
-
-        try {
-            final String jwtPrincipal = jwtService.getAuthenticationFromToken(request.getToken());
-            final String mappedIdentity = mapIdentity(jwtPrincipal);
-            final NiFiUser user = new StandardNiFiUser.Builder()
-                    .identity(mappedIdentity)
-                    .groups(getUserGroups(mappedIdentity))
-                    .clientAddress(request.getClientAddress())
-                    .build();
-            return new NiFiAuthenticationToken(new NiFiUserDetails(user));
-        } catch (JwtException e) {
-            throw new InvalidAuthenticationException(e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public boolean supports(Class<?> authentication) {
-        return JwtAuthenticationRequestToken.class.isAssignableFrom(authentication);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationRequestToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationRequestToken.java
deleted file mode 100644
index e8af9ff..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationRequestToken.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.authentication.jwt;
-
-
-import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationRequestToken;
-
-/**
- * This is an authentication request with a given JWT token.
- */
-public class JwtAuthenticationRequestToken extends NiFiAuthenticationRequestToken {
-
-    private final String token;
-
-    /**
-     * Creates a representation of the jwt authentication request for a user.
-     *
-     * @param token   The unique token for this user
-     * @param clientAddress the address of the client making the request
-     */
-    public JwtAuthenticationRequestToken(final String token, final String clientAddress) {
-        super(clientAddress);
-        setAuthenticated(false);
-        this.token = token;
-    }
-
-    @Override
-    public Object getCredentials() {
-        return null;
-    }
-
-    @Override
-    public Object getPrincipal() {
-        return token;
-    }
-
-    public String getToken() {
-        return token;
-    }
-
-    @Override
-    public String toString() {
-        return "<JWT token>";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/ef8ba127/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtIdentityProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtIdentityProvider.java
new file mode 100644
index 0000000..9592c2b
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtIdentityProvider.java
@@ -0,0 +1,76 @@
+/*
+ * 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.registry.web.security.authentication.jwt;
+
+import io.jsonwebtoken.JwtException;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.security.authentication.AuthenticationRequest;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.BearerAuthIdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProviderConfigurationContext;
+import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
+import org.apache.nifi.registry.security.authentication.exception.InvalidCredentialsException;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.exception.SecurityProviderCreationException;
+import org.apache.nifi.registry.security.exception.SecurityProviderDestructionException;
+import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.concurrent.TimeUnit;
+
+@Component
+public class JwtIdentityProvider extends BearerAuthIdentityProvider implements IdentityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(JwtIdentityProvider.class);
+
+    private static final String issuer = JwtIdentityProvider.class.getSimpleName();
+
+    private static final long expiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
+
+    private final JwtService jwtService;
+
+    @Autowired
+    public JwtIdentityProvider(JwtService jwtService, NiFiRegistryProperties nifiProperties, Authorizer authorizer) {
+        this.jwtService = jwtService;
+    }
+
+    @Override
+    public AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest) throws InvalidCredentialsException, IdentityAccessException {
+        try {
+            String jwtAuthToken = (String) authenticationRequest.getCredentials();
+            final String jwtPrincipal = jwtService.getAuthenticationFromToken(jwtAuthToken);
+
+            return new AuthenticationResponse(jwtPrincipal, jwtPrincipal, expiration, issuer);
+        } catch (ClassCastException e) {
+            // token String in credentials Object
+            return null;
+        } catch (JwtException e) {
+            throw new InvalidAuthenticationException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {}
+
+    @Override
+    public void preDestruction() throws SecurityProviderDestructionException {}
+
+}